diff --git a/CLAUDE.md b/CLAUDE.md index 7a91641978c..e75586924ae 100644 --- a/CLAUDE.md +++ b/CLAUDE.md @@ -1,12 +1,20 @@ -When navigating the codebase, look at relevant README.md for more project context. +- To gather more context beyond README.md: + - Look at the outstanding changes in the tree + - If on a branch check the last 2-3 to commits + - Look at relevant README.md in sub-folders -- If on a branch maybe check the last 2-3 to add commits for more context -- Look at the outstanding changes in the tree -- Write production quality code -- Make sure the code compiles +- Write production quality code. + - Adhere to rules in "Code Complete" by Steve McConnell. + - Adhere to rules in "The Art of Readable Code" by Dustin Boswell & Trevor Foucher. +- Make sure the code compiles. - When adding code always ensure that tests cover the newly added code: - Unit tests that validate for regular and exceptional inputs - Use property based testing/model based testing/fuzzing when appropriate - Integration tests for big platform-level features (in @python/tests) -At the start of every conversation, offer the user to run `scripts/claude.sh` to pull in shared LLM context files as unstaged changes. These should not be committed outside the `claude-context` branch. \ No newline at end of file +- Add/update documenation and comments. + - Adhere to rules in "Bugs in Writing: A Guide to Debugging Your Prose" by Lyn Dupre. + - Adhere to rules in "The Elements of Style, Fourth Edition" by William Strunk Jr. & E. B. White + +At the start of every conversation, offer the user to run `scripts/claude.sh` to pull in shared LLM context files as unstaged changes. +These should not be committed outside the `claude-context` branch. diff --git a/Cargo.lock b/Cargo.lock index 250ed62ef8b..6ae3bbd6580 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3761,7 +3761,7 @@ dependencies = [ [[package]] name = "dbsp" -version = "0.276.0" +version = "0.277.0" dependencies = [ "anyhow", "arc-swap", @@ -3849,7 +3849,7 @@ dependencies = [ [[package]] name = "dbsp_adapters" -version = "0.276.0" +version = "0.277.0" dependencies = [ "actix", "actix-codec", @@ -3986,7 +3986,7 @@ dependencies = [ [[package]] name = "dbsp_nexmark" -version = "0.276.0" +version = "0.277.0" dependencies = [ "anyhow", "ascii_table", @@ -4861,7 +4861,7 @@ dependencies = [ [[package]] name = "fda" -version = "0.276.0" +version = "0.277.0" dependencies = [ "anyhow", "arrow", @@ -4913,7 +4913,7 @@ dependencies = [ [[package]] name = "feldera-adapterlib" -version = "0.276.0" +version = "0.277.0" dependencies = [ "actix-web", "anyhow", @@ -4944,10 +4944,11 @@ dependencies = [ [[package]] name = "feldera-buffer-cache" -version = "0.276.0" +version = "0.277.0" dependencies = [ "crossbeam-utils", "enum-map", + "feldera-types", "proptest", "quick_cache", "rand 0.8.5", @@ -4971,7 +4972,7 @@ dependencies = [ [[package]] name = "feldera-datagen" -version = "0.276.0" +version = "0.277.0" dependencies = [ "anyhow", "async-channel 2.5.0", @@ -4997,7 +4998,7 @@ dependencies = [ [[package]] name = "feldera-fxp" -version = "0.276.0" +version = "0.277.0" dependencies = [ "bytecheck", "dbsp", @@ -5017,7 +5018,7 @@ dependencies = [ [[package]] name = "feldera-iceberg" -version = "0.276.0" +version = "0.277.0" dependencies = [ "anyhow", "chrono", @@ -5037,7 +5038,7 @@ dependencies = [ [[package]] name = "feldera-ir" -version = "0.276.0" +version = "0.277.0" dependencies = [ "proptest", "proptest-derive", @@ -5049,7 +5050,7 @@ dependencies = [ [[package]] name = "feldera-macros" -version = "0.276.0" +version = "0.277.0" dependencies = [ "prettyplease", "proc-macro2", @@ -5059,7 +5060,7 @@ dependencies = [ [[package]] name = "feldera-observability" -version = "0.276.0" +version = "0.277.0" dependencies = [ "actix-http", "awc", @@ -5074,7 +5075,7 @@ dependencies = [ [[package]] name = "feldera-rest-api" -version = "0.276.0" +version = "0.277.0" dependencies = [ "chrono", "feldera-observability", @@ -5108,7 +5109,7 @@ dependencies = [ [[package]] name = "feldera-sqllib" -version = "0.276.0" +version = "0.277.0" dependencies = [ "arcstr", "base58", @@ -5149,7 +5150,7 @@ dependencies = [ [[package]] name = "feldera-storage" -version = "0.276.0" +version = "0.277.0" dependencies = [ "anyhow", "crossbeam", @@ -5172,7 +5173,7 @@ dependencies = [ [[package]] name = "feldera-types" -version = "0.276.0" +version = "0.277.0" dependencies = [ "actix-web", "anyhow", @@ -8093,7 +8094,7 @@ checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" [[package]] name = "pipeline-manager" -version = "0.276.0" +version = "0.277.0" dependencies = [ "actix-cors", "actix-files", @@ -9187,7 +9188,7 @@ dependencies = [ [[package]] name = "readers" -version = "0.276.0" +version = "0.277.0" dependencies = [ "async-std", "csv", @@ -10763,7 +10764,7 @@ dependencies = [ [[package]] name = "sltsqlvalue" -version = "0.276.0" +version = "0.277.0" dependencies = [ "dbsp", "feldera-sqllib", @@ -11066,7 +11067,7 @@ checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" [[package]] name = "storage-test-compat" -version = "0.276.0" +version = "0.277.0" dependencies = [ "dbsp", "derive_more 1.0.0", diff --git a/Cargo.toml b/Cargo.toml index e3a574babd2..1c455e33f64 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,6 +1,6 @@ [workspace.package] authors = ["Feldera Team "] -version = "0.276.0" +version = "0.277.0" license = "MIT OR Apache-2.0" homepage = "https://github.com/feldera/feldera" repository = "https://github.com/feldera/feldera" @@ -101,7 +101,7 @@ csv = "1.2.2" csv-core = "0.1.10" dashmap = "6.1.0" datafusion = "51.0" -dbsp = { path = "crates/dbsp", version = "0.276.0" } +dbsp = { path = "crates/dbsp", version = "0.277.0" } dbsp_nexmark = { path = "crates/nexmark" } deadpool-postgres = "0.14.1" #deltalake = "0.30.2" @@ -121,19 +121,19 @@ erased-serde = "0.3.31" fake = "2.10" fastbloom = "0.14.0" fdlimit = "0.3.0" -feldera-buffer-cache = { version = "0.276.0", path = "crates/buffer-cache" } +feldera-buffer-cache = { version = "0.277.0", path = "crates/buffer-cache" } feldera-cloud1-client = "0.1.2" feldera-datagen = { path = "crates/datagen" } -feldera-fxp = { version = "0.276.0", path = "crates/fxp", features = ["dbsp"] } +feldera-fxp = { version = "0.277.0", path = "crates/fxp", features = ["dbsp"] } feldera-iceberg = { path = "crates/iceberg" } -feldera-observability = { version = "0.276.0", path = "crates/feldera-observability" } -feldera-macros = { version = "0.276.0", path = "crates/feldera-macros" } -feldera-sqllib = { version = "0.276.0", path = "crates/sqllib" } -feldera-storage = { version = "0.276.0", path = "crates/storage" } -feldera-types = { version = "0.276.0", path = "crates/feldera-types" } -feldera-rest-api = { version = "0.276.0", path = "crates/rest-api" } -feldera-ir = { version = "0.276.0", path = "crates/ir" } -feldera-adapterlib = { version = "0.276.0", path = "crates/adapterlib" } +feldera-observability = { version = "0.277.0", path = "crates/feldera-observability" } +feldera-macros = { version = "0.277.0", path = "crates/feldera-macros" } +feldera-sqllib = { version = "0.277.0", path = "crates/sqllib" } +feldera-storage = { version = "0.277.0", path = "crates/storage" } +feldera-types = { version = "0.277.0", path = "crates/feldera-types" } +feldera-rest-api = { version = "0.277.0", path = "crates/rest-api" } +feldera-ir = { version = "0.277.0", path = "crates/ir" } +feldera-adapterlib = { version = "0.277.0", path = "crates/adapterlib" } flate2 = "1.1.0" form_urlencoded = "1.2.0" futures = "0.3.30" diff --git a/bunfig.toml b/bunfig.toml index 5b58efbd575..804af07d4d2 100644 --- a/bunfig.toml +++ b/bunfig.toml @@ -3,6 +3,4 @@ [install] # Only install package versions published at least 7 days ago -# minimumReleaseAge = 604800 -# Temporarily removed minimumReleaseAge restriction for a security patch; revert on or after 22.01.2026 -minimumReleaseAge = 0 \ No newline at end of file +minimumReleaseAge = 604800 \ No newline at end of file diff --git a/crates/adapters/src/controller.rs b/crates/adapters/src/controller.rs index 5817b0f7178..94a16110029 100644 --- a/crates/adapters/src/controller.rs +++ b/crates/adapters/src/controller.rs @@ -58,7 +58,7 @@ use dbsp::circuit::metrics::{ DBSP_STEP_LATENCY_MICROSECONDS, FILES_CREATED, FILES_DELETED, TOTAL_LATE_RECORDS, }; use dbsp::circuit::tokio::TOKIO; -use dbsp::circuit::{CheckpointCommitter, CircuitStorageConfig, DevTweaks, Mode}; +use dbsp::circuit::{CheckpointCommitter, CircuitStorageConfig, Mode}; use dbsp::samply::{MARKER_BYTES, Markers, SamplySpan}; use dbsp::storage::backend::{StorageBackend, StoragePath}; use dbsp::utils::process_rss_bytes; @@ -159,8 +159,8 @@ pub use feldera_types::config::{ RuntimeConfig, TransportConfig, }; use feldera_types::config::{ - DEFAULT_MAX_WORKER_BATCH_SIZE, FileBackendConfig, FtConfig, FtModel, OutputBufferConfig, - StorageBackendConfig, SyncConfig, + DEFAULT_MAX_WORKER_BATCH_SIZE, DevTweaks, FileBackendConfig, FtConfig, FtModel, + OutputBufferConfig, StorageBackendConfig, SyncConfig, }; use feldera_types::constants::{STATE_FILE, STEPS_FILE}; use feldera_types::format::json::{JsonFlavor, JsonParserConfig, JsonUpdateFormat}; @@ -4400,7 +4400,10 @@ impl ControllerInit { pipeline_config: &PipelineConfig, storage: Option, ) -> Result { - let dev_tweaks = DevTweaks::from_config(&pipeline_config.global.dev_tweaks); + let dev_tweaks = pipeline_config.global.dev_tweaks.clone(); + if dev_tweaks != DevTweaks::default() { + info!("using non-default `dev_tweaks`: {dev_tweaks:#?}") + } let mut max_rss_mb = pipeline_config.global.max_rss_mb; diff --git a/crates/adapters/src/server.rs b/crates/adapters/src/server.rs index bdac39de478..58d886153c8 100644 --- a/crates/adapters/src/server.rs +++ b/crates/adapters/src/server.rs @@ -625,13 +625,7 @@ pub fn run_server( // Install stack overflow handler early, before creating the controller and parsing DevTweaks. #[cfg(target_family = "unix")] - if config - .global - .dev_tweaks - .get("stack_overflow_backtrace") - .cloned() - == Some(serde_json::Value::Bool(true)) - { + if config.global.dev_tweaks.stack_overflow_backtrace() { unsafe { use crate::server::stack_overflow_backtrace::enable_stack_overflow_backtrace_with_limit; diff --git a/crates/adapters/src/util.rs b/crates/adapters/src/util.rs index 29566f0d120..f5989e46f13 100644 --- a/crates/adapters/src/util.rs +++ b/crates/adapters/src/util.rs @@ -718,7 +718,7 @@ pub(crate) fn run_in_posix_runtime( F: FnOnce() + Send + 'static, { use dbsp::Runtime; - use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, DevTweaks, Layout, Mode}; + use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, Layout, Mode}; use feldera_types::config::{StorageCacheConfig, StorageConfig, StorageOptions}; use std::sync::{Arc, Mutex}; @@ -742,7 +742,7 @@ pub(crate) fn run_in_posix_runtime( ) .expect("failed to configure storage"), ), - dev_tweaks: DevTweaks::default(), + dev_tweaks: Default::default(), }; let test_fn: Arc>> = Arc::new(Mutex::new(Some(test_fn))); diff --git a/crates/buffer-cache/Cargo.toml b/crates/buffer-cache/Cargo.toml index f6adcc56b81..82970a0dba9 100644 --- a/crates/buffer-cache/Cargo.toml +++ b/crates/buffer-cache/Cargo.toml @@ -13,6 +13,7 @@ description = "Weighted in-memory buffer caches with LRU and S3-FIFO eviction" [dependencies] crossbeam-utils = { workspace = true } enum-map = { workspace = true } +feldera-types = { workspace = true } quick_cache = { workspace = true } serde = { workspace = true, features = ["derive"] } tracing = { workspace = true } diff --git a/crates/buffer-cache/src/builder.rs b/crates/buffer-cache/src/builder.rs index b02be83f5ed..363e31b7b33 100644 --- a/crates/buffer-cache/src/builder.rs +++ b/crates/buffer-cache/src/builder.rs @@ -1,39 +1,12 @@ use crate::ThreadType; use crate::{CacheEntry, LruCache, S3FifoCache, SharedBufferCache}; use enum_map::{Enum, EnumMap}; -use serde::{Deserialize, Serialize}; +use feldera_types::config::dev_tweaks::{BufferCacheAllocationStrategy, BufferCacheStrategy}; use std::fmt::Debug; use std::hash::{BuildHasher, Hash, RandomState}; use std::marker::PhantomData; use tracing::warn; -/// Selects which eviction strategy backs a cache instance. -#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize)] -#[serde(rename_all = "snake_case")] -pub enum BufferCacheStrategy { - /// Use the sharded S3-FIFO cache backed by `quick_cache`. - #[default] - S3Fifo, - - /// Use the mutex-protected weighted LRU cache. - Lru, -} - -/// Controls how caches are shared across a foreground/background worker pair. -#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize)] -#[serde(rename_all = "snake_case")] -pub enum BufferCacheAllocationStrategy { - /// Share one cache across a foreground/background worker pair. - #[default] - SharedPerWorkerPair, - - /// Create a separate cache for each foreground/background thread. - PerThread, - - /// Share one cache across all foreground/background threads. - Global, -} - /// Builds the cache layout used by DBSP runtime worker pairs. pub struct BufferCacheBuilder { /// Eviction strategy used for newly constructed caches. diff --git a/crates/buffer-cache/src/lib.rs b/crates/buffer-cache/src/lib.rs index ce300f325af..238f99bafa3 100644 --- a/crates/buffer-cache/src/lib.rs +++ b/crates/buffer-cache/src/lib.rs @@ -8,7 +8,8 @@ mod thread_type; use std::any::Any; use std::sync::Arc; -pub use builder::{BufferCacheAllocationStrategy, BufferCacheBuilder, BufferCacheStrategy}; +pub use builder::BufferCacheBuilder; +use feldera_types::config::dev_tweaks::BufferCacheStrategy; pub use lru::LruCache; pub use s3_fifo::S3FifoCache; pub use thread_type::ThreadType; diff --git a/crates/buffer-cache/src/lru.rs b/crates/buffer-cache/src/lru.rs index 2708b8a1840..e1758879c8d 100644 --- a/crates/buffer-cache/src/lru.rs +++ b/crates/buffer-cache/src/lru.rs @@ -1,4 +1,6 @@ -use crate::{BufferCache, BufferCacheStrategy, CacheEntry}; +use feldera_types::config::dev_tweaks::BufferCacheStrategy; + +use crate::{BufferCache, CacheEntry}; use std::any::Any; use std::collections::BTreeMap; use std::fmt::Debug; diff --git a/crates/buffer-cache/src/s3_fifo.rs b/crates/buffer-cache/src/s3_fifo.rs index 240f4e0f939..2e7c7748b6f 100644 --- a/crates/buffer-cache/src/s3_fifo.rs +++ b/crates/buffer-cache/src/s3_fifo.rs @@ -1,4 +1,5 @@ -use crate::{BufferCache, BufferCacheStrategy, CacheEntry}; +use crate::{BufferCache, CacheEntry}; +use feldera_types::config::dev_tweaks::BufferCacheStrategy; use quick_cache::{OptionsBuilder, Weighter, sync::Cache as QuickCache}; use std::any::Any; use std::hash::{BuildHasher, Hash, RandomState}; diff --git a/crates/buffer-cache/src/tests/builder.rs b/crates/buffer-cache/src/tests/builder.rs index f470afc6558..dfd662964f9 100644 --- a/crates/buffer-cache/src/tests/builder.rs +++ b/crates/buffer-cache/src/tests/builder.rs @@ -1,6 +1,6 @@ -use crate::{ - BufferCacheAllocationStrategy, BufferCacheBuilder, BufferCacheStrategy, CacheEntry, ThreadType, -}; +use feldera_types::config::dev_tweaks::{BufferCacheAllocationStrategy, BufferCacheStrategy}; + +use crate::{BufferCacheBuilder, CacheEntry, ThreadType}; use std::sync::Arc; #[derive(Clone)] diff --git a/crates/dbsp/Cargo.toml b/crates/dbsp/Cargo.toml index 3cb3f7f5e01..c6da2493f09 100644 --- a/crates/dbsp/Cargo.toml +++ b/crates/dbsp/Cargo.toml @@ -56,7 +56,7 @@ futures = { workspace = true } tokio = { workspace = true, features = ["macros", "rt", "rt-multi-thread"] } paste = { workspace = true } seq-macro = { workspace = true } -derive_more = { workspace = true, features = ["add", "not", "from", "debug"] } +derive_more = { workspace = true, features = ["add", "add_assign", "sum", "not", "from", "debug"] } dyn-clone = { workspace = true } rand_chacha = { workspace = true } tempfile = { workspace = true } diff --git a/crates/dbsp/benches/cursor_list.rs b/crates/dbsp/benches/cursor_list.rs index c414f8c250b..c9f361c641f 100644 --- a/crates/dbsp/benches/cursor_list.rs +++ b/crates/dbsp/benches/cursor_list.rs @@ -7,7 +7,7 @@ //! //! Run with: cargo bench -p dbsp --bench cursor_list -use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, DevTweaks, Layout, Mode}; +use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, Layout, Mode}; use dbsp::{ OrdIndexedZSet, Runtime, ZWeight, trace::cursor::CursorList, @@ -102,7 +102,7 @@ fn bench(storage: bool) { ) .expect("failed to configure POSIX storage"), ), - dev_tweaks: DevTweaks::default(), + dev_tweaks: Default::default(), }; let results: Arc>> = Arc::new(Mutex::new(Vec::new())); diff --git a/crates/dbsp/benches/input_map_ingest.rs b/crates/dbsp/benches/input_map_ingest.rs index 684ad6dde58..6578a03bcb6 100644 --- a/crates/dbsp/benches/input_map_ingest.rs +++ b/crates/dbsp/benches/input_map_ingest.rs @@ -1,6 +1,6 @@ use anyhow::{Context, Result, anyhow}; use crossbeam::channel::{Sender, bounded}; -use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, DevTweaks, Layout, Mode}; +use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, Layout, Mode}; use dbsp::{ Runtime, mimalloc::MiMalloc, @@ -54,7 +54,7 @@ fn main() -> Result<()> { ) .context("failed to configure POSIX storage backend")?, ), - dev_tweaks: DevTweaks::default(), + dev_tweaks: Default::default(), }; let total_batches = (TOTAL_RECORDS / BATCH_SIZE as u64) as usize; diff --git a/crates/dbsp/benches/list_merger.rs b/crates/dbsp/benches/list_merger.rs index 50afb78a780..7ba953eb998 100644 --- a/crates/dbsp/benches/list_merger.rs +++ b/crates/dbsp/benches/list_merger.rs @@ -7,7 +7,7 @@ //! //! Run with: `cargo bench -p dbsp --bench list_merger` -use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, DevTweaks, Layout, Mode}; +use dbsp::circuit::{CircuitConfig, CircuitStorageConfig, Layout, Mode}; use dbsp::{ OrdIndexedZSet, Runtime, ZWeight, trace::{Batch as DynBatch, BatchLocation, BatchReader as DynBatchReader, Builder, ListMerger}, @@ -125,7 +125,7 @@ fn bench(generate_on_storage: bool) { ) .expect("failed to configure POSIX storage"), ), - dev_tweaks: DevTweaks::default(), + dev_tweaks: Default::default(), }; let results: Arc>> = Arc::new(Mutex::new(Vec::new())); diff --git a/crates/dbsp/benches/window_min.rs b/crates/dbsp/benches/window_min.rs index 0c6da07868d..15053a57a84 100644 --- a/crates/dbsp/benches/window_min.rs +++ b/crates/dbsp/benches/window_min.rs @@ -7,7 +7,8 @@ use anyhow::{Context, Result, anyhow}; use dbsp::circuit::CircuitConfig; use dbsp::operator::Min; use dbsp::utils::Tup2; -use dbsp::{Runtime, TypedBox, ZWeight, circuit::DevTweaks}; +use dbsp::{Runtime, TypedBox, ZWeight}; +use feldera_types::config::DevTweaks; use rand::{Rng, SeedableRng}; use rand_chacha::ChaCha8Rng; use std::time::{Duration, Instant}; @@ -43,7 +44,7 @@ fn main() -> Result<()> { fn run_workload(negative_weight_multiplier: u16) -> Result { let mut config = CircuitConfig::from(WORKERS); config.dev_tweaks = DevTweaks { - negative_weight_multiplier, + negative_weight_multiplier: Some(negative_weight_multiplier), ..DevTweaks::default() }; diff --git a/crates/dbsp/src/circuit.rs b/crates/dbsp/src/circuit.rs index 766d6a42ea9..eed18377e3f 100644 --- a/crates/dbsp/src/circuit.rs +++ b/crates/dbsp/src/circuit.rs @@ -38,7 +38,7 @@ pub use circuit_builder::{ NestedCircuit, NodeId, OwnershipPreference, RootCircuit, Scope, Stream, WithClock, }; pub use dbsp_handle::{ - CheckpointCommitter, CircuitConfig, CircuitStorageConfig, DBSPHandle, DevTweaks, Host, Layout, + CheckpointCommitter, CircuitConfig, CircuitStorageConfig, DBSPHandle, Host, Layout, LayoutError, Mode, StorageCacheConfig, StorageConfig, StorageOptions, adaptive_joins_enabled, balancer_balance_tax, balancer_key_distribution_refresh_threshold, balancer_min_absolute_improvement_threshold, balancer_min_relative_improvement_threshold, diff --git a/crates/dbsp/src/circuit/dbsp_handle.rs b/crates/dbsp/src/circuit/dbsp_handle.rs index 00dae9ede9d..697da655911 100644 --- a/crates/dbsp/src/circuit/dbsp_handle.rs +++ b/crates/dbsp/src/circuit/dbsp_handle.rs @@ -3,13 +3,8 @@ use crate::circuit::checkpointer::Checkpointer; use crate::circuit::metrics::{DBSP_STEP, DBSP_STEP_LATENCY_MICROSECONDS}; use crate::circuit::schedule::CommitProgress; use crate::monitor::visual_graph::Graph; -use crate::operator::dynamic::balance::{ - BALANCE_TAX, BalancerHint, KEY_DISTRIBUTION_REFRESH_THRESHOLD, - MIN_ABSOLUTE_IMPROVEMENT_THRESHOLD, MIN_RELATIVE_IMPROVEMENT_THRESHOLD, PartitioningPolicy, -}; +use crate::operator::dynamic::balance::{BalancerHint, PartitioningPolicy}; use crate::storage::backend::StorageError; -use crate::storage::file::BLOOM_FILTER_FALSE_POSITIVE_RATE; -use crate::trace::MergerType; use crate::trace::spine_async::MAX_LEVEL0_BATCH_SIZE_RECORDS; use crate::{ Error as DbspError, RootCircuit, Runtime, RuntimeError, circuit::runtime::RuntimeHandle, @@ -17,16 +12,15 @@ use crate::{ }; use anyhow::Error as AnyError; use crossbeam::channel::{Receiver, Select, Sender, TryRecvError, bounded}; -use feldera_buffer_cache::{BufferCacheAllocationStrategy, BufferCacheStrategy, ThreadType}; +use feldera_buffer_cache::ThreadType; use feldera_ir::LirCircuit; -use feldera_storage::fbuf::slab::FBufSlabs; use feldera_storage::{FileCommitter, StorageBackend, StoragePath}; use feldera_types::checkpoint::CheckpointMetadata; +use feldera_types::config::DevTweaks; +use feldera_types::config::dev_tweaks::{BufferCacheAllocationStrategy, BufferCacheStrategy}; pub use feldera_types::config::{StorageCacheConfig, StorageConfig, StorageOptions}; use feldera_types::transaction::CommitProgressSummary; use itertools::Either; -use serde::Deserialize; -use serde_json::Value; use std::collections::BTreeMap; use std::sync::atomic::Ordering; use std::sync::{Arc, Mutex}; @@ -301,246 +295,43 @@ pub struct CircuitConfig { pub dev_tweaks: DevTweaks, } -#[derive(Debug, Clone, PartialEq, Deserialize)] -#[serde(default)] -pub struct DevTweaks { - /// Buffer-cache implementation to use for storage reads. - /// - /// The default is `s3_fifo`. - pub buffer_cache_strategy: BufferCacheStrategy, - - /// Override the number of buckets/shards used by sharded buffer caches. - /// - /// This only applies when `buffer_cache_strategy = "s3_fifo"`. Values are - /// rounded up to the next power of two because the current implementation - /// shards by `hash(key) & (n - 1)`. - pub buffer_max_buckets: Option, - - /// How S3-FIFO caches are assigned to foreground/background workers. - /// - /// This only applies when `buffer_cache_strategy = "s3_fifo"`. The - /// default is `shared_per_worker_pair`; LRU always uses `per_thread`. - pub buffer_cache_allocation_strategy: BufferCacheAllocationStrategy, - - /// Target number of cached bytes retained in each `FBuf` slab size class. - /// - /// The default value is [`FBufSlabs::DEFAULT_BYTES_PER_CLASS`]. - pub fbuf_slab_bytes_per_class: usize, - - /// Whether to asynchronously fetch keys needed for the join operator from - /// storage. Asynchronous fetching should be faster for high-latency - /// storage, such as object storage, but it could use excessive amounts of - /// memory if the number of keys fetched is very large. - pub fetch_join: bool, - - /// Whether to asynchronously fetch keys needed for the distinct operator - /// from storage. Asynchronous fetching should be faster for high-latency - /// storage, such as object storage, but it could use excessive amounts of - /// memory if the number of keys fetched is very large. - pub fetch_distinct: bool, - - /// Which merger to use. - pub merger: MergerType, - - /// If set, the maximum amount of storage, in MiB, for the POSIX backend to - /// allow to be in use before failing all writes with [StorageFull]. This - /// is useful for testing on top of storage that does not implement its own - /// quota mechanism. - /// - /// [StorageFull]: std::io::ErrorKind::StorageFull - pub storage_mb_max: Option, - - /// Attempt to print a stack trace on stack overflow. - /// - /// To be used for debugging only; do not enable in production. - // NOTE: this flag is handled manually in `adapters/src/server.rs` before - // parsing DevTweaks. If the name or type of this field changes, make sure to - // adjust `server.rs` accordingly. - pub stack_overflow_backtrace: bool, - - /// Controls the maximal number of records output by splitter operators - /// (joins, distinct, aggregation, rolling window and group operators) at - /// each step. - /// - /// The default value is 10,000 records. - // TODO: splitter_chunk_size_bytes, per-operator chunk size. - pub splitter_chunk_size_records: u64, - - /// Enable adaptive joins. - /// - /// Adaptive joins dynamically change their partitioning policy to avoid skew. - /// - /// Adaptive joins are disabled by default. - pub adaptive_joins: bool, - - /// The minimum relative improvement threshold for the join balancer. - /// - /// This parameter prevents the join balancer from making changes to the - /// partitioning policy if the improvement is not significant, since the overhead - /// of such rebalancing, especially when performed frequently, can exceed the benefits. - /// - /// A rebalancing is considered significant if the relative estimated improvement for the cluster - /// of joins where the rebalancing is applied is at least this threshold. - /// - /// A rebalancing is applied if both this threshold and `balancer_min_absolute_improvement_threshold` are met. - /// - /// The default value is 1.2. - pub balancer_min_relative_improvement_threshold: f64, - - /// The minimum absolute improvement threshold for the balancer. - /// - /// This parameter prevents the join balancer from making changes to the - /// partitioning policy if the improvement is not significant, since the overhead - /// of such rebalancing, especially when performed frequently, can exceed the benefits. - /// - /// A rebalancing is considered significant if the absolute estimated improvement for the cluster - /// of joins where the rebalancing is applied is at least this threshold. The cost model used by - /// the balancer is based on the number of records in the largest partition of a collection. - /// - /// A rebalancing is applied if both this threshold and `balancer_min_relative_improvement_threshold` are met. - /// - /// The default value is 10,000. - pub balancer_min_absolute_improvement_threshold: u64, - - /// Factor that discourages the use of the Balance policy in a perfectly balanced collection. - /// - /// Assuming a perfectly balanced key distribution, the Balance policy is slightly less efficient than Shard, - /// since it requires computing the hash of the entire key/value pair. This factor discourages the use of this policy - /// if the skew is `, - - /// Additional bias the merger assigns to records with negative weights - /// (retractions) in order to promote them to higher levels of the LSM tree sooner. - /// - /// Reasonable values for this parameter are in the range [0, 10]. - /// - /// The default value is 0, which means that retractions are not given - /// any additional bias. - pub negative_weight_multiplier: u16, -} - -impl Default for DevTweaks { - fn default() -> Self { - Self { - buffer_cache_strategy: BufferCacheStrategy::default(), - buffer_max_buckets: None, - buffer_cache_allocation_strategy: BufferCacheAllocationStrategy::default(), - fbuf_slab_bytes_per_class: FBufSlabs::DEFAULT_BYTES_PER_CLASS, - fetch_join: false, - fetch_distinct: false, - merger: MergerType::default(), - storage_mb_max: None, - stack_overflow_backtrace: false, - splitter_chunk_size_records: 10_000, - bloom_false_positive_rate: BLOOM_FILTER_FALSE_POSITIVE_RATE, - balancer_min_relative_improvement_threshold: MIN_RELATIVE_IMPROVEMENT_THRESHOLD, - balancer_min_absolute_improvement_threshold: MIN_ABSOLUTE_IMPROVEMENT_THRESHOLD, - balancer_balance_tax: BALANCE_TAX, - balancer_key_distribution_refresh_threshold: KEY_DISTRIBUTION_REFRESH_THRESHOLD, - adaptive_joins: false, - max_level0_batch_size_records: MAX_LEVEL0_BATCH_SIZE_RECORDS, - merger_threads: None, - negative_weight_multiplier: 0, - } - } -} - -impl DevTweaks { - pub fn from_config(config: &BTreeMap) -> Self { - let tweaks: Self = serde_json::to_value(config) - .and_then(serde_json::from_value) - .inspect_err(|error| { - tracing::error!("falling back to default `dev_tweaks` due to error ({error}) with configuration: {config:#?}") - }) - .unwrap_or_default(); - if tweaks != DevTweaks::default() { - info!("using non-default `dev_tweaks`: {tweaks:#?}") - } - tweaks - } - - pub(crate) fn effective_buffer_cache_allocation_strategy( - &self, - ) -> BufferCacheAllocationStrategy { - match self.buffer_cache_strategy { - BufferCacheStrategy::S3Fifo => self.buffer_cache_allocation_strategy, - BufferCacheStrategy::Lru => BufferCacheAllocationStrategy::PerThread, - } - } -} - /// Returns the chunk size for splitter operators, in records. /// /// Operators that split their output into multiple chunks, such as joins, distinct, and aggregation, /// should attempt to limit their output to this chunk size. pub fn splitter_output_chunk_size() -> usize { - Runtime::with_dev_tweaks(|d| d.splitter_chunk_size_records as usize) + Runtime::with_dev_tweaks(|d| d.splitter_chunk_size_records() as usize) } pub fn balancer_min_absolute_improvement_threshold() -> u64 { - Runtime::with_dev_tweaks(|d| d.balancer_min_absolute_improvement_threshold) + Runtime::with_dev_tweaks(|d| d.balancer_min_absolute_improvement_threshold()) } pub fn balancer_min_relative_improvement_threshold() -> f64 { - Runtime::with_dev_tweaks(|d| d.balancer_min_relative_improvement_threshold) + Runtime::with_dev_tweaks(|d| d.balancer_min_relative_improvement_threshold()) } pub fn balancer_balance_tax() -> f64 { - Runtime::with_dev_tweaks(|d| d.balancer_balance_tax) + Runtime::with_dev_tweaks(|d| d.balancer_balance_tax()) } pub fn balancer_key_distribution_refresh_threshold() -> f64 { - Runtime::with_dev_tweaks(|d| d.balancer_key_distribution_refresh_threshold) + Runtime::with_dev_tweaks(|d| d.balancer_key_distribution_refresh_threshold()) } pub fn adaptive_joins_enabled() -> bool { - Runtime::with_dev_tweaks(|d| d.adaptive_joins) + Runtime::with_dev_tweaks(|d| d.adaptive_joins()) } pub fn max_level0_batch_size_records() -> u16 { - Runtime::with_dev_tweaks(|d| d.max_level0_batch_size_records) + Runtime::with_dev_tweaks(|d| { + d.max_level0_batch_size_records + .unwrap_or(MAX_LEVEL0_BATCH_SIZE_RECORDS) + }) } pub fn negative_weight_multiplier() -> u16 { - Runtime::with_dev_tweaks(|d| d.negative_weight_multiplier) + Runtime::with_dev_tweaks(|d| d.negative_weight_multiplier()) } /// Configuration for storage in a [Runtime]-hosted circuit. @@ -621,12 +412,12 @@ impl CircuitConfig { } pub fn with_splitter_chunk_size_records(mut self, records: u64) -> Self { - self.dev_tweaks.splitter_chunk_size_records = records; + self.dev_tweaks.splitter_chunk_size_records = Some(records); self } pub fn with_buffer_cache_strategy(mut self, strategy: BufferCacheStrategy) -> Self { - self.dev_tweaks.buffer_cache_strategy = strategy; + self.dev_tweaks.buffer_cache_strategy = Some(strategy); self } @@ -639,28 +430,28 @@ impl CircuitConfig { mut self, strategy: BufferCacheAllocationStrategy, ) -> Self { - self.dev_tweaks.buffer_cache_allocation_strategy = strategy; + self.dev_tweaks.buffer_cache_allocation_strategy = Some(strategy); self } #[cfg(test)] pub fn with_fbuf_slab_bytes_per_class(mut self, bytes_per_class: usize) -> Self { - self.dev_tweaks.fbuf_slab_bytes_per_class = bytes_per_class; + self.dev_tweaks.fbuf_slab_bytes_per_class = Some(bytes_per_class); self } pub fn with_balancer_min_relative_improvement_threshold(mut self, threshold: f64) -> Self { - self.dev_tweaks.balancer_min_relative_improvement_threshold = threshold; + self.dev_tweaks.balancer_min_relative_improvement_threshold = Some(threshold); self } pub fn with_balancer_min_absolute_improvement_threshold(mut self, threshold: u64) -> Self { - self.dev_tweaks.balancer_min_absolute_improvement_threshold = threshold; + self.dev_tweaks.balancer_min_absolute_improvement_threshold = Some(threshold); self } pub fn with_balancer_balance_tax(mut self, tax: f64) -> Self { - self.dev_tweaks.balancer_balance_tax = tax; + self.dev_tweaks.balancer_balance_tax = Some(tax); self } diff --git a/crates/dbsp/src/circuit/metadata.rs b/crates/dbsp/src/circuit/metadata.rs index aaa8a3c8861..5522bd42e96 100644 --- a/crates/dbsp/src/circuit/metadata.rs +++ b/crates/dbsp/src/circuit/metadata.rs @@ -136,6 +136,12 @@ pub const BLOOM_FILTER_MISSES_COUNT: MetricId = pub const BLOOM_FILTER_HIT_RATE_PERCENT: MetricId = MetricId(Cow::Borrowed("bloom_filter_hit_rate_percent")); pub const BLOOM_FILTER_SIZE_BYTES: MetricId = MetricId(Cow::Borrowed("bloom_filter_size_bytes")); +pub const RANGE_FILTER_HITS_COUNT: MetricId = MetricId(Cow::Borrowed("range_filter_hits_count")); +pub const RANGE_FILTER_MISSES_COUNT: MetricId = + MetricId(Cow::Borrowed("range_filter_misses_count")); +pub const RANGE_FILTER_HIT_RATE_PERCENT: MetricId = + MetricId(Cow::Borrowed("range_filter_hit_rate_percent")); +pub const RANGE_FILTER_SIZE_BYTES: MetricId = MetricId(Cow::Borrowed("range_filter_size_bytes")); pub const SPINE_BATCHES_COUNT: MetricId = MetricId(Cow::Borrowed("spine_batches_count")); pub const SPINE_STORAGE_SIZE_BYTES: MetricId = MetricId(Cow::Borrowed("spine_storage_size_bytes")); pub const MERGING_SIZE_BYTES: MetricId = MetricId(Cow::Borrowed("merging_size_bytes")); @@ -161,7 +167,7 @@ pub const PREFIX_BATCHES_STATS: MetricId = MetricId(Cow::Borrowed("prefix_batche pub const INPUT_INTEGRAL_RECORDS_COUNT: MetricId = MetricId(Cow::Borrowed("input_integral_records_count")); -pub const CIRCUIT_METRICS: [CircuitMetric; 66] = [ +pub const CIRCUIT_METRICS: [CircuitMetric; 70] = [ // State CircuitMetric { name: USED_MEMORY_BYTES, @@ -289,6 +295,30 @@ pub const CIRCUIT_METRICS: [CircuitMetric; 66] = [ advanced: false, description: "Hit rate of the Bloom filter.", }, + CircuitMetric { + name: RANGE_FILTER_SIZE_BYTES, + category: CircuitMetricCategory::State, + advanced: true, + description: "Size of the cached range filter in bytes.", + }, + CircuitMetric { + name: RANGE_FILTER_HITS_COUNT, + category: CircuitMetricCategory::State, + advanced: false, + description: "The number of hits across all range filters. The hits are summed across the range filters for all batches in the spine.", + }, + CircuitMetric { + name: RANGE_FILTER_MISSES_COUNT, + category: CircuitMetricCategory::State, + advanced: false, + description: "The number of misses across all range filters. The misses are summed across the range filters for all batches in the spine.", + }, + CircuitMetric { + name: RANGE_FILTER_HIT_RATE_PERCENT, + category: CircuitMetricCategory::State, + advanced: false, + description: "Hit rate of the range filter.", + }, CircuitMetric { name: RETAINMENT_BOUNDS, category: CircuitMetricCategory::State, diff --git a/crates/dbsp/src/circuit/runtime.rs b/crates/dbsp/src/circuit/runtime.rs index 6950c1e19e8..cc9f8dd7ff1 100644 --- a/crates/dbsp/src/circuit/runtime.rs +++ b/crates/dbsp/src/circuit/runtime.rs @@ -4,14 +4,12 @@ use super::CircuitConfig; use super::dbsp_handle::{Layout, Mode}; use crate::SchedulerError; -use crate::circuit::DevTweaks; use crate::circuit::checkpointer::Checkpointer; use crate::error::Error as DbspError; use crate::operator::communication::Exchange; use crate::storage::backend::StorageBackend; use crate::storage::file::format::Compression; use crate::storage::file::writer::Parameters; -use crate::trace::aligned_deserialize; use crate::utils::process_rss_bytes; use crate::{ DetailedError, @@ -27,7 +25,7 @@ use enum_map::{Enum, EnumMap, enum_map}; use feldera_buffer_cache::ThreadType; use feldera_storage::fbuf::FBuf; use feldera_storage::fbuf::slab::{FBufSlabs, FBufSlabsStats, set_thread_slab_pool}; -use feldera_types::config::{StorageCompression, StorageConfig, StorageOptions}; +use feldera_types::config::{DevTweaks, StorageCompression, StorageConfig, StorageOptions}; use feldera_types::memory_pressure::{ CRITICAL_MEMORY_PRESSURE_THRESHOLD, HIGH_MEMORY_PRESSURE_THRESHOLD, MODERATE_MEMORY_PRESSURE_THRESHOLD, MemoryPressure, @@ -35,6 +33,7 @@ use feldera_types::memory_pressure::{ use indexmap::IndexSet; use once_cell::sync::Lazy; use serde::Serialize; +use std::convert::identity; use std::iter::repeat; use std::ops::Range; use std::path::Path; @@ -389,12 +388,15 @@ fn map_pin_cpus(config: &CircuitConfig) -> (Vec, Vec) { impl RuntimeInner { fn new(config: CircuitConfig) -> Result { let nworkers = config.layout.local_workers().len(); - let buffer_cache_strategy = config.dev_tweaks.buffer_cache_strategy; + let buffer_cache_strategy = config.dev_tweaks.buffer_cache_strategy(); let buffer_max_buckets = config.dev_tweaks.buffer_max_buckets; let buffer_cache_allocation_strategy = config .dev_tweaks .effective_buffer_cache_allocation_strategy(); - let fbuf_slab_bytes_per_class = config.dev_tweaks.fbuf_slab_bytes_per_class; + let fbuf_slab_bytes_per_class = config + .dev_tweaks + .fbuf_slab_bytes_per_class + .unwrap_or(FBufSlabs::DEFAULT_BYTES_PER_CLASS); let storage = if let Some(storage) = config.storage.clone() { let locked_directory = LockedDirectory::new_blocking(storage.config.path(), Duration::from_secs(60))?; @@ -1283,48 +1285,11 @@ impl Runtime { /// A synchronization primitive that allows multiple threads within a runtime to agree /// when a condition is satisfied. -pub(crate) enum Consensus { - SingleThreaded, - MultiThreaded { - notify_sender: Arc, - notify_receiver: Arc, - exchange: Arc>, - }, -} +pub(crate) struct Consensus(Broadcast); impl Consensus { pub fn new() -> Self { - match Runtime::runtime() { - Some(runtime) if Runtime::num_workers() > 1 => { - let worker_index = Runtime::worker_index(); - let exchange_id = runtime.sequence_next().try_into().unwrap(); - let exchange = Exchange::with_runtime( - &runtime, - exchange_id, - Box::new(|data| aligned_deserialize(&data[..])), - ); - - let notify_sender = Arc::new(Notify::new()); - let notify_sender_clone = notify_sender.clone(); - let notify_receiver = Arc::new(Notify::new()); - let notify_receiver_clone = notify_receiver.clone(); - - exchange.register_sender_callback(worker_index, move || { - notify_sender_clone.notify_one() - }); - - exchange.register_receiver_callback(worker_index, move || { - notify_receiver_clone.notify_one() - }); - - Self::MultiThreaded { - notify_sender, - notify_receiver, - exchange, - } - } - _ => Self::SingleThreaded, - } + Self(Broadcast::new()) } /// Returns `true` if all workers vote `true`. @@ -1333,37 +1298,7 @@ impl Consensus { /// /// * `local` - Local vote by the current worker. pub async fn check(&self, local: bool) -> Result { - match self { - Self::SingleThreaded => Ok(local), - Self::MultiThreaded { - notify_sender, - notify_receiver, - exchange, - } => { - while !exchange.try_send_all_with_serializer( - Runtime::worker_index(), - repeat(local), - |local| FBuf::from_slice(&[local as u8]), - ) { - if Runtime::kill_in_progress() { - return Err(SchedulerError::Killed); - } - notify_sender.notified().await; - } - // Receive the status of each peer, compute global result - // as a logical and of all peer statuses. - let mut global = true; - while !exchange.try_receive_all(Runtime::worker_index(), |status| global &= status) - { - if Runtime::kill_in_progress() { - return Err(SchedulerError::Killed); - } - // Sleep if other threads are still working. - notify_receiver.notified().await; - } - Ok(global) - } - } + Ok(self.0.collect(local).await?.into_iter().all(identity)) } } @@ -1444,8 +1379,7 @@ where } notify_sender.notified().await; } - // Receive the status of each peer, compute global result - // as a logical and of all peer statuses. + // Receive and collect the status of each peer. let mut result = Vec::with_capacity(Runtime::num_workers()); while !exchange .try_receive_all(Runtime::worker_index(), |status| result.push(status)) @@ -1692,7 +1626,7 @@ mod tests { Circuit, RootCircuit, circuit::{ CircuitConfig, Layout, - dbsp_handle::{CircuitStorageConfig, DevTweaks, Mode}, + dbsp_handle::{CircuitStorageConfig, Mode}, metadata::{LOOSE_MEMORY_RECORDS_COUNT, MERGING_MEMORY_RECORDS_COUNT}, schedule::{DynamicScheduler, Scheduler}, }, @@ -1701,11 +1635,12 @@ mod tests { storage::backend::FileId, }; use enum_map::Enum; - use feldera_buffer_cache::{ - BufferCacheAllocationStrategy, BufferCacheStrategy, CacheEntry, ThreadType, - }; + use feldera_buffer_cache::{CacheEntry, ThreadType}; use feldera_storage::fbuf::{FBuf, slab::set_thread_slab_pool}; - use feldera_types::config::{StorageCacheConfig, StorageConfig, StorageOptions}; + use feldera_types::config::{ + DevTweaks, StorageCacheConfig, StorageConfig, StorageOptions, + dev_tweaks::{BufferCacheAllocationStrategy, BufferCacheStrategy}, + }; use feldera_types::memory_pressure::MemoryPressure; use std::{ cell::RefCell, diff --git a/crates/dbsp/src/operator/dynamic/distinct.rs b/crates/dbsp/src/operator/dynamic/distinct.rs index 9a8b9521b7f..473f0ab8794 100644 --- a/crates/dbsp/src/operator/dynamic/distinct.rs +++ b/crates/dbsp/src/operator/dynamic/distinct.rs @@ -517,7 +517,7 @@ where let mut builder = Z::Builder::with_capacity(&self.input_factories, self.chunk_size, self.chunk_size); let mut delta_cursor = delta.cursor(); - let fetched = if Runtime::with_dev_tweaks(|d| d.fetch_distinct) { + let fetched = if Runtime::with_dev_tweaks(|d| d.fetch_distinct == Some(true)) { delayed_integral.as_ref().unwrap().fetch(&delta).await } else { None diff --git a/crates/dbsp/src/operator/dynamic/join.rs b/crates/dbsp/src/operator/dynamic/join.rs index bbf89a17658..c38148a0336 100644 --- a/crates/dbsp/src/operator/dynamic/join.rs +++ b/crates/dbsp/src/operator/dynamic/join.rs @@ -1569,7 +1569,7 @@ where self.stats.borrow_mut().lhs_tuples += delta.len(); self.stats.borrow_mut().rhs_tuples = trace.len(); - let fetched = if Runtime::with_dev_tweaks(|dev_tweaks| dev_tweaks.fetch_join) { + let fetched = if Runtime::with_dev_tweaks(|dev_tweaks| dev_tweaks.fetch_join == Some(true)) { trace.fetch(&delta).await } else { None diff --git a/crates/dbsp/src/storage.rs b/crates/dbsp/src/storage.rs index cb2fa45d11a..25c5557567c 100644 --- a/crates/dbsp/src/storage.rs +++ b/crates/dbsp/src/storage.rs @@ -7,6 +7,7 @@ pub mod backend; pub mod buffer_cache; pub mod dirlock; pub mod file; +pub mod filter_stats; pub mod tracking_bloom_filter; use fdlimit::{Outcome::LimitRaised, raise_fd_limit}; diff --git a/crates/dbsp/src/storage/buffer_cache/cache.rs b/crates/dbsp/src/storage/buffer_cache/cache.rs index 60e4eb6e59c..0f131c15482 100644 --- a/crates/dbsp/src/storage/buffer_cache/cache.rs +++ b/crates/dbsp/src/storage/buffer_cache/cache.rs @@ -7,9 +7,9 @@ use std::time::Duration; use enum_map::{Enum, EnumMap}; use feldera_buffer_cache::{ - BufferCacheAllocationStrategy, BufferCacheBuilder, BufferCacheStrategy, CacheEntry, LruCache, - SharedBufferCache, ThreadType, + BufferCacheBuilder, CacheEntry, LruCache, SharedBufferCache, ThreadType, }; +use feldera_types::config::dev_tweaks::{BufferCacheAllocationStrategy, BufferCacheStrategy}; use serde::{Deserialize, Serialize}; use size_of::SizeOf; diff --git a/crates/dbsp/src/storage/file/reader.rs b/crates/dbsp/src/storage/file/reader.rs index b3d04ffe866..8dd404b6e10 100644 --- a/crates/dbsp/src/storage/file/reader.rs +++ b/crates/dbsp/src/storage/file/reader.rs @@ -7,7 +7,7 @@ use super::{AnyFactories, Deserializer, Factories}; use crate::dynamic::{DynVec, WeightTrait}; use crate::storage::buffer_cache::CacheAccess; use crate::storage::file::format::{BatchMetadata, FilterBlock}; -use crate::storage::tracking_bloom_filter::{BloomFilterStats, TrackingBloomFilter}; +use crate::storage::tracking_bloom_filter::TrackingBloomFilter; use crate::storage::{ backend::StorageError, buffer_cache::{BufferCache, FBuf}, @@ -655,6 +655,12 @@ where DeserializeDyn::deserialize_with(item.fst(), key, &mut deserializer) } } + unsafe fn key_range(&self, factories: &Factories, min: &mut K, max: &mut K) { + unsafe { + self.key(factories, 0, min); + self.key(factories, self.n_values() - 1, max); + } + } unsafe fn aux(&self, factories: &Factories, index: usize, aux: &mut A) { unsafe { let item = self.archived_item(factories, index); @@ -807,6 +813,33 @@ impl TreeNode { NodeType::Index => Ok(TreeBlock::Index(IndexBlock::new(file, self)?)), } } + + fn key_range( + &self, + file: &ImmutableFileRef, + factories: &Factories, + ) -> Result<(Box, Box), Error> + where + K: DataTrait + ?Sized, + A: DataTrait + ?Sized, + { + let key_factory = factories.key_factory; + let mut min = key_factory.default_box(); + let mut max = key_factory.default_box(); + + match self.read::(file)? { + // SAFETY: Unsafe because of serialization + TreeBlock::Data(data_block) => unsafe { + data_block.key_range(factories, min.as_mut(), max.as_mut()); + }, + // SAFETY: Unsafe because of serialization + TreeBlock::Index(index_block) => unsafe { + index_block.key_range(min.as_mut(), max.as_mut()); + }, + } + + Ok((min, max)) + } } enum TreeBlock { @@ -1112,6 +1145,17 @@ where } } + unsafe fn key_range(&self, min: &mut K, max: &mut K) { + unsafe { + self.get_bound(0, min); + self.max_bound(max); + } + } + + unsafe fn max_bound(&self, bound: &mut K) { + unsafe { self.get_bound(self.last_bound_index(), bound) } + } + /// Returns the index of the child of this index block that contains a row /// in the range `target_rows` and which may contain a key for which /// `compare` returns `bias` or [Equal], or `None` if there is no such @@ -1205,6 +1249,10 @@ where self.child_offsets.count } + fn last_bound_index(&self) -> usize { + self.n_children() * 2 - 1 + } + /// Returns the comparison of the largest bound key using `compare`. unsafe fn compare_max(&self, key_factory: &dyn Factory, compare: &C) -> Ordering where @@ -1213,7 +1261,7 @@ where unsafe { let mut ordering = Equal; key_factory.with(&mut |key| { - self.get_bound(self.n_children() * 2 - 1, key); + self.max_bound(key); ordering = compare(key); }); ordering @@ -1498,7 +1546,6 @@ where #[derive(Debug)] pub struct Reader { file: ImmutableFileRef, - bloom_filter: Option, columns: Vec, /// Additional metadata added to the file by the writer. @@ -1515,7 +1562,6 @@ where { fn size_of_children(&self, context: &mut size_of::Context) { self.file.size_of_with_context(context); - context.add(self.filter_stats().size_byte); self.columns.size_of_with_context(context); } } @@ -1529,8 +1575,17 @@ where factories: &[&AnyFactories], cache: fn() -> Arc, file: Arc, - bloom_filter: Option, ) -> Result { + let (reader, _membership_filter) = Self::new_with_filter(factories, cache, file, None)?; + Ok(reader) + } + + pub(crate) fn new_with_filter( + factories: &[&AnyFactories], + cache: fn() -> Arc, + file: Arc, + membership_filter: Option, + ) -> Result<(Self, Option), Error> { let file_size = file.get_size()?; if file_size < 512 || (file_size % 512) != 0 { return Err(CorruptionError::InvalidFileSize(file_size).into()); @@ -1611,34 +1666,39 @@ where )? .into()) } - let bloom_filter = match bloom_filter { - Some(bloom_filter) => Some(bloom_filter), - None if file_trailer.has_filter64() => Some(read_filter_block( + let membership_filter = if let Some(membership_filter) = membership_filter { + Some(membership_filter) + } else if file_trailer.has_filter64() { + Some(read_filter_block( &*file, file_trailer.filter_offset64, file_trailer.filter_size64 as usize, - )?), - None if file_trailer.filter_offset != 0 => Some(read_filter_block( + )?) + } else if file_trailer.filter_offset != 0 { + Some(read_filter_block( &*file, file_trailer.filter_offset, file_trailer.filter_size as usize, - )?), - None => None, + )?) + } else { + None }; - Ok(Self { - file: ImmutableFileRef::new( - cache, - file, - file_trailer.compression, - stats, - file_trailer.version, - ), - columns, - bloom_filter, - metadata: file_trailer.metadata.clone(), - _phantom: PhantomData, - }) + Ok(( + Self { + file: ImmutableFileRef::new( + cache, + file, + file_trailer.compression, + stats, + file_trailer.version, + ), + columns, + metadata: file_trailer.metadata.clone(), + _phantom: PhantomData, + }, + membership_filter, + )) } /// Marks the file of the reader as being part of a checkpoint. @@ -1653,7 +1713,16 @@ where storage_backend: &dyn StorageBackend, path: &StoragePath, ) -> Result { - Self::new(factories, cache, storage_backend.open(path)?, None) + Self::new(factories, cache, storage_backend.open(path)?) + } + + pub(crate) fn open_with_filter( + factories: &[&AnyFactories], + cache: fn() -> Arc, + storage_backend: &dyn StorageBackend, + path: &StoragePath, + ) -> Result<(Self, Option), Error> { + Self::new_with_filter(factories, cache, storage_backend.open(path)?, None) } /// The number of columns in the layer file. @@ -1683,17 +1752,6 @@ where Ok(self.file.file_handle.get_size()?) } - /// Returns statistics of the Bloom filter, including its size in bytes. - /// - /// If the file doesn't have a Bloom filter, returns a default of zeros. - pub fn filter_stats(&self) -> BloomFilterStats { - if let Some(bloom_filter) = &self.bloom_filter { - bloom_filter.stats() - } else { - BloomFilterStats::default() - } - } - /// Evict this file from the cache. #[cfg(test)] pub fn evict(&self) { @@ -1723,11 +1781,17 @@ where A: DataTrait + ?Sized, (&'static K, &'static A, N): ColumnSpec, { - /// Asks the bloom filter of the reader if we have the key. - pub fn maybe_contains_key(&self, hash: u64) -> bool { - self.bloom_filter - .as_ref() - .is_none_or(|b| b.contains_hash(hash)) + /// Returns the min and max keys stored in column 0. + /// + /// The bounds are loaded from the root node when first requested and can + /// then be cached by higher-level batch types. + pub fn key_range(&self) -> Result, Box)>, Error> { + let Some(root) = self.columns[0].root.as_ref() else { + return Ok(None); + }; + + let factories = self.columns[0].factories.factories::(); + Ok(Some(root.key_range(&self.file, &factories)?)) } /// Returns a [`RowGroup`] for all of the rows in column 0. @@ -1749,9 +1813,9 @@ where /// Returns a [`FetchZSet`], which will subset this reader to just the rows /// in colunn 0 whose keys are in `keys` (which must be sorted) and return /// it as a Z-set, treating auxiliary values as weights. - pub fn fetch_zset<'a, 'b>( + pub(crate) fn fetch_zset<'a, 'b>( &'a self, - keys: &'b DynVec, + keys: FilteredKeys<'b, K>, ) -> Result, Error> { FetchZSet::new(self, keys) } @@ -1767,14 +1831,70 @@ where /// Returns a [`FetchIndexedZSet`], which will build an indexed Z-set from /// this reader containing just the rows whose keys are in `keys` (which /// must be sorted). - pub fn fetch_indexed_zset<'a, 'b>( + pub(crate) fn fetch_indexed_zset<'a, 'b>( &'a self, - keys: &'b DynVec, + keys: FilteredKeys<'b, K0>, ) -> Result, Error> { FetchIndexedZSet::new(self, keys) } } +/// A `DynVec`, possibly filtered by a higher-level exact-seek filter chain. +pub(crate) struct FilteredKeys<'a, K> +where + K: DataTrait + ?Sized, +{ + queried_keys: &'a DynVec, + filter_pass_keys: Option>, +} + +impl<'a, K> FilteredKeys<'a, K> +where + K: DataTrait + ?Sized, +{ + pub(crate) fn all(queried_keys: &'a DynVec) -> Self { + Self { + queried_keys, + filter_pass_keys: None, + } + } + + pub(crate) fn with_filter_pass_keys( + queried_keys: &'a DynVec, + filter_pass_keys: Option>, + ) -> Self { + Self { + queried_keys, + filter_pass_keys, + } + } + + pub(crate) fn len(&self) -> usize { + match &self.filter_pass_keys { + Some(filter_pass_keys) => filter_pass_keys.len(), + None => self.queried_keys.len(), + } + } + + pub(crate) fn is_empty(&self) -> bool { + self.len() == 0 + } +} + +impl Index for FilteredKeys<'_, K> +where + K: DataTrait + ?Sized, +{ + type Output = K; + + fn index(&self, index: usize) -> &Self::Output { + match &self.filter_pass_keys { + Some(filter_pass_keys) => &self.queried_keys[filter_pass_keys[index]], + None => &self.queried_keys[index], + } + } +} + /// A sorted, indexed group of unique rows in a [`Reader`]. /// /// Column 0 in a layer file has a single [`RowGroup`] that includes all of the @@ -3051,80 +3171,3 @@ where } } } - -/// A `DynVec`, possibly filtered by a Bloom filter. -struct FilteredKeys<'b, K> -where - K: ?Sized, -{ - /// Sorted array to keys to retrieve. - queried_keys: &'b DynVec, - - /// Indexes into `queried_keys` of the keys that pass the Bloom filter. If - /// this is `None`, then enough of the keys passed the Bloom filter that we - /// just take all of them. - bloom_keys: Option>, -} - -impl<'b, K> FilteredKeys<'b, K> -where - K: DataTrait + ?Sized, -{ - /// Returns `keys`, filtered using `reader.maybe_contains_key()`. - fn new<'a, A, N>(reader: &'a Reader<(&'static K, &'static A, N)>, keys: &'b DynVec) -> Self - where - A: DataTrait + ?Sized, - N: ColumnSpec, - { - debug_assert!(keys.is_sorted_by(&|a, b| a.cmp(b))); - - // Pass keys into the Bloom filter until 1/300th of them pass the Bloom - // filter. Empirically, this seems to good enough for the common case - // where the data passed into a "distinct" operator is actually distinct - // but we get some false positives from the Bloom filter. Because the - // keys that go into a "distinct" operator are often large, we don't - // want to pass all of them into the Bloom filter if we're going to have - // to deserialize them anyhow later. - let mut bloom_keys = SmallVec::<[_; 50]>::new(); - for (index, key) in keys.dyn_iter().enumerate() { - if reader.maybe_contains_key(key.default_hash()) { - bloom_keys.push(index); - if bloom_keys.len() >= keys.len() / 300 { - return Self { - queried_keys: keys, - bloom_keys: None, - }; - } - } - } - Self { - queried_keys: keys, - bloom_keys: Some(bloom_keys.into_vec()), - } - } - - fn len(&self) -> usize { - match &self.bloom_keys { - Some(bloom_keys) => bloom_keys.len(), - None => self.queried_keys.len(), - } - } - - fn is_empty(&self) -> bool { - self.len() == 0 - } -} - -impl<'b, K> Index for FilteredKeys<'b, K> -where - K: DataTrait + ?Sized, -{ - type Output = K; - - fn index(&self, index: usize) -> &Self::Output { - match &self.bloom_keys { - Some(bloom_keys) => &self.queried_keys[bloom_keys[index]], - None => &self.queried_keys[index], - } - } -} diff --git a/crates/dbsp/src/storage/file/reader/fetch_indexed_zset.rs b/crates/dbsp/src/storage/file/reader/fetch_indexed_zset.rs index 62e10ce0e1e..3785eafbe27 100644 --- a/crates/dbsp/src/storage/file/reader/fetch_indexed_zset.rs +++ b/crates/dbsp/src/storage/file/reader/fetch_indexed_zset.rs @@ -50,7 +50,7 @@ where { pub(super) fn new( reader: &'a Reader<(&'static K0, &'static A0, (&'static K1, &'static A1, ()))>, - keys: &'b DynVec, + keys: FilteredKeys<'b, K0>, ) -> Result { Ok(Self(FetchIndexedZSetInner::Column0(Some(Fetch0::new( reader, keys, @@ -172,7 +172,7 @@ where { fn new( reader: &'a Reader<(&'static K, &'static A, N)>, - keys: &'b DynVec, + keys: FilteredKeys<'b, K>, ) -> Result { let (sender, receiver) = tokio::sync::mpsc::unbounded_channel(); let factories = reader.columns[0].factories.factories(); @@ -185,7 +185,7 @@ where key_stack.reserve_exact(10); let mut this = Self { - keys: FilteredKeys::new(reader, keys), + keys, reader, cache: (reader.file.cache)(), factories, diff --git a/crates/dbsp/src/storage/file/reader/fetch_zset.rs b/crates/dbsp/src/storage/file/reader/fetch_zset.rs index a8853bdafe8..cf6acb83d58 100644 --- a/crates/dbsp/src/storage/file/reader/fetch_zset.rs +++ b/crates/dbsp/src/storage/file/reader/fetch_zset.rs @@ -1,5 +1,5 @@ use super::super::Factories; -use crate::dynamic::{DataTrait, DynVec, WeightTrait}; +use crate::dynamic::{DataTrait, WeightTrait}; use crate::storage::file::reader::{ DataBlock, Error, FilteredKeys, Reader, TreeBlock, TreeNode, decompress, }; @@ -60,13 +60,13 @@ where { pub(super) fn new( reader: &'a Reader<(&'static K, &'static A, ())>, - keys: &'b DynVec, + keys: FilteredKeys<'b, K>, ) -> Result { let (sender, receiver) = tokio::sync::mpsc::unbounded_channel(); let factories = reader.columns[0].factories.factories(); let tmp_key = factories.key_factory.default_box(); let mut this = Self { - keys: FilteredKeys::new(reader, keys), + keys, reader, cache: (reader.file.cache)(), factories, diff --git a/crates/dbsp/src/storage/file/test.rs b/crates/dbsp/src/storage/file/test.rs index ef9132d3a43..8c8336088c9 100644 --- a/crates/dbsp/src/storage/file/test.rs +++ b/crates/dbsp/src/storage/file/test.rs @@ -2,18 +2,21 @@ use std::{marker::PhantomData, sync::Arc}; use crate::{ DBWeight, - dynamic::{Data, DynWeight, Factory, LeanVec, Vector, WithFactory}, + dynamic::{DataTrait, DowncastTrait, DynWeight, Factory, LeanVec, Vector, WithFactory}, storage::{ backend::StorageBackend, buffer_cache::BufferCache, file::{ format::{BatchMetadata, Compression}, - reader::{BulkRows, Reader}, + reader::{BulkRows, FilteredKeys, Reader}, }, }, trace::{ BatchReaderFactories, Builder, VecIndexedWSetFactories, VecWSetFactories, - ord::vec::{indexed_wset_batch::VecIndexedWSetBuilder, wset_batch::VecWSetBuilder}, + ord::{ + batch_filter::BatchFilters, + vec::{indexed_wset_batch::VecIndexedWSetBuilder, wset_batch::VecWSetBuilder}, + }, }, utils::test::init_test_logger, }; @@ -603,7 +606,7 @@ fn test_multifetch_zset( } let expected = expected.done(); - let mut multifetch = reader.fetch_zset(&*keys).unwrap(); + let mut multifetch = reader.fetch_zset(FilteredKeys::all(&*keys)).unwrap(); while !multifetch.is_done() { multifetch.wait().unwrap(); } @@ -641,7 +644,9 @@ fn test_multifetch_two_columns( } let expected = expected.done(); - let mut multifetch = reader.fetch_indexed_zset(&*keys).unwrap(); + let mut multifetch = reader + .fetch_indexed_zset(FilteredKeys::all(&*keys)) + .unwrap(); while !multifetch.is_done() { multifetch.wait().unwrap(); } @@ -649,23 +654,22 @@ fn test_multifetch_two_columns( assert_eq!(&output, &expected); } -fn test_bloom( - reader: &Reader<(&'static DynData, &'static DynData, N)>, +fn test_bloom( + filters: &BatchFilters, n: usize, expected: impl Fn(usize) -> (K, K, K, A), ) where - K: DBData, + K: DBData + Erase, A: DBData, - N: ColumnSpec, { let mut false_positives = 0; for row in 0..n { let (before, key, after, _aux) = expected(row); - assert!(reader.maybe_contains_key(key.default_hash())); - if reader.maybe_contains_key(before.default_hash()) { + assert!(filters.maybe_contains_key(key.erase(), None)); + if filters.maybe_contains_key(before.erase(), None) { false_positives += 1; } - if reader.maybe_contains_key(after.default_hash()) { + if filters.maybe_contains_key(after.erase(), None) { false_positives += 1; } } @@ -683,6 +687,31 @@ fn test_bloom( } } +fn test_key_range( + reader: &Reader<(&'static DynData, &'static Aux, N)>, + n: usize, + expected: impl Fn(usize) -> (K, K, K, A), +) where + K: DBData, + A: DBData, + Aux: DataTrait + ?Sized, + N: ColumnSpec, +{ + let key_range = reader.key_range().unwrap(); + if n == 0 { + assert!(key_range.is_none()); + return; + } + + let Some((min, max)) = key_range else { + panic!("expected non-empty key range"); + }; + let (_, expected_min, _, _) = expected(0); + let (_, expected_max, _, _) = expected(n - 1); + assert_eq!(min.downcast_checked::(), &expected_min); + assert_eq!(max.downcast_checked::(), &expected_max); +} + fn test_two_columns(parameters: Parameters) where T: TwoColumns, @@ -718,7 +747,7 @@ where layer_file.write0((&T::key0(row0), &T::aux0(row0))).unwrap(); } - let reader = layer_file.into_reader(BatchMetadata::default()).unwrap(); + let (reader, filters) = layer_file.into_reader(BatchMetadata::default()).unwrap(); reader.evict(); let rows0 = reader.rows(); let expected0 = |row0| { @@ -728,7 +757,8 @@ where (before0, key0, after0, aux0) }; test_cursor(&rows0, n0, expected0); - test_bloom(&reader, n0, expected0); + test_bloom(&filters, n0, expected0); + test_key_range(&reader, n0, expected0); let expected1 = |row0, row1| { let key1 = T::key1(row0, row1); @@ -783,7 +813,7 @@ where layer_file.write0((&T::key0(row0), &T::aux0(row0))).unwrap(); } - let reader = layer_file.into_reader(BatchMetadata::default()).unwrap(); + let (reader, _filters) = layer_file.into_reader(BatchMetadata::default()).unwrap(); reader.evict(); test_multifetch_two_columns::(&reader); } @@ -923,25 +953,31 @@ where writer.write0((&key, &aux)).unwrap(); } - let reader = if reopen { + let (reader, filters) = if reopen { println!("closing writer and reopening as reader"); let path = writer.path().clone(); - let (_file_handle, _bloom_filter) = writer.close(BatchMetadata::default()).unwrap(); - Reader::open( + let (_file_handle, _bloom_filter, _key_bounds) = + writer.close(BatchMetadata::default()).unwrap(); + let (reader, membership_filter) = Reader::open_with_filter( &[&factories.any_factories()], test_buffer_cache, &*storage_backend, &path, ) - .unwrap() + .unwrap(); + let key_range = reader.key_range().unwrap().map(Into::into); + let filters = BatchFilters::from_file(key_range, membership_filter); + (reader, filters) } else { println!("transforming writer into reader"); - writer.into_reader(BatchMetadata::default()).unwrap() + let (reader, filters) = writer.into_reader(BatchMetadata::default()).unwrap(); + (reader, filters) }; reader.evict(); assert_eq!(reader.rows().len(), n as u64); test_cursor(&reader.rows(), n, &expected); - test_bloom(&reader, n, &expected); + test_bloom(&filters, n, &expected); + test_key_range(&reader, n, &expected); test_bulk_rows(reader.bulk_rows().unwrap(), OneColumn::new(&expected, n)); } } @@ -981,7 +1017,8 @@ fn test_one_column_zset( let reader = if reopen { println!("closing writer and reopening as reader"); let path = writer.path().clone(); - let (_file_handle, _bloom_filter) = writer.close(BatchMetadata::default()).unwrap(); + let (_file_handle, _bloom_filter, _key_bounds) = + writer.close(BatchMetadata::default()).unwrap(); Reader::open( &[&factories.any_factories()], test_buffer_cache, @@ -991,14 +1028,77 @@ fn test_one_column_zset( .unwrap() } else { println!("transforming writer into reader"); - writer.into_reader(BatchMetadata::default()).unwrap() + let (reader, _filters) = writer.into_reader(BatchMetadata::default()).unwrap(); + reader }; reader.evict(); assert_eq!(reader.rows().len(), n as u64); + test_key_range(&reader, n, &expected); test_multifetch_zset(&reader, n, &expected); } } +#[test] +fn one_column_key_range() { + init_test_logger(); + + for reopen in [false, true] { + for (label, keys) in [ + ("negative", [-30_i64, -20, -10]), + ("positive", [10_i64, 20, 30]), + ("mixed", [-30_i64, 0, 20]), + ] { + let factories = Factories::::new::(); + let tempdir = tempdir().unwrap(); + let storage_backend = ::new( + &StorageConfig { + path: tempdir.path().to_string_lossy().to_string(), + cache: Default::default(), + }, + &StorageOptions::default(), + ) + .unwrap(); + let mut writer = Writer1::new( + &factories, + test_buffer_cache, + &*storage_backend, + Parameters::default(), + keys.len(), + ) + .unwrap(); + for key in keys { + writer.write0((&key, &())).unwrap(); + } + + let reader = if reopen { + let path = writer.path().clone(); + let (_file_handle, _bloom_filter, _key_bounds) = + writer.close(BatchMetadata::default()).unwrap(); + Reader::open( + &[&factories.any_factories()], + test_buffer_cache, + &*storage_backend, + &path, + ) + .unwrap() + } else { + let (reader, _filters) = writer.into_reader(BatchMetadata::default()).unwrap(); + reader + }; + + let Some((min, max)) = reader.key_range().unwrap() else { + panic!("expected non-empty key range for {label}"); + }; + assert_eq!(*min.downcast_checked::(), keys[0], "{label}"); + assert_eq!( + *max.downcast_checked::(), + keys[keys.len() - 1], + "{label}" + ); + } + } +} + fn test_i64_helper(parameters: Parameters) { init_test_logger(); test_one_column( diff --git a/crates/dbsp/src/storage/file/writer.rs b/crates/dbsp/src/storage/file/writer.rs index 012c5c69ccc..213ea629d68 100644 --- a/crates/dbsp/src/storage/file/writer.rs +++ b/crates/dbsp/src/storage/file/writer.rs @@ -47,6 +47,7 @@ use crate::{ Runtime, dynamic::{DataTrait, DeserializeDyn, SerializeDyn}, storage::file::ItemFactory, + trace::ord::{BatchFilters, key_range::KeyRange}, }; struct VarintWriter { @@ -238,7 +239,7 @@ impl ColumnWriter { &mut self, block_writer: &mut BlockWriter, serializer: &mut SerializerInner, - ) -> Result + ) -> Result<(FileTrailerColumn, Option<(Box, Box)>), StorageError> where K: DataTrait + ?Sized, A: DataTrait + ?Sized, @@ -255,29 +256,50 @@ impl ColumnWriter { if level == self.index_blocks.len() - 1 && self.index_blocks[level].entries.len() == 1 { let builder = &self.index_blocks[level]; let entry = &builder.entries[0]; - return Ok(FileTrailerColumn { - node_type: builder.child_type, - node_offset: entry.child.offset, - node_size: entry.child.size.try_into().unwrap_or_else(|_| { - unreachable!( - "Individual blocks should be much less than 4 GiB, tried to write {:?}", - &entry.child - ) - }), - n_rows: entry.row_total, - }); + return Ok(( + FileTrailerColumn { + node_type: builder.child_type, + node_offset: entry.child.offset, + node_size: entry.child.size.try_into().unwrap_or_else(|_| { + unreachable!( + "Individual blocks should be much less than 4 GiB, tried to write {:?}", + &entry.child + ) + }), + n_rows: entry.row_total, + }, + Some(self.key_bounds::(&builder.raw, entry)), + )); } else if !self.index_blocks[level].is_empty() { let index_block = self.index_blocks[level].build(); self.write_index_block::(block_writer, index_block, level, serializer)?; } level += 1; } - Ok(FileTrailerColumn { - node_type: NodeType::Data, - node_offset: 0, - node_size: 0, - n_rows: 0, - }) + Ok(( + FileTrailerColumn { + node_type: NodeType::Data, + node_offset: 0, + node_size: 0, + n_rows: 0, + }, + None, + )) + } + + fn key_bounds(&self, raw: &FBuf, entry: &IndexEntry) -> (Box, Box) + where + K: DataTrait + ?Sized, + { + let key_factory = self.factories.key_factory::(); + + let mut min = key_factory.default_box(); + rkyv_deserialize(raw, entry.min_offset, min.as_mut()); + + let mut max = key_factory.default_box(); + rkyv_deserialize(raw, entry.max_offset, max.as_mut()); + + (min, max) } fn get_index_block(&mut self, level: usize) -> &mut IndexBlockBuilder { @@ -1126,7 +1148,7 @@ struct Writer { impl Writer { fn bloom_false_positive_rate() -> Option { - let rate = Runtime::with_dev_tweaks(|dev_tweaks| dev_tweaks.bloom_false_positive_rate); + let rate = Runtime::with_dev_tweaks(|dev_tweaks| dev_tweaks.bloom_false_positive_rate()); let rate = (rate > 0.0 && rate < 1.0).then_some(rate); static ONCE: Once = Once::new(); @@ -1205,7 +1227,10 @@ impl Writer { self.cws[column].add_item(&mut self.writer, item, &row_group, &mut self.serializer) } - pub fn finish_column(&mut self, column: usize) -> Result<(), StorageError> + pub fn finish_column( + &mut self, + column: usize, + ) -> Result, Box)>, StorageError> where K: DataTrait + ?Sized, A: DataTrait + ?Sized, @@ -1215,9 +1240,10 @@ impl Writer { assert!(cw.rows.is_empty()); } - self.finished_columns - .push(self.cws[column].finish::(&mut self.writer, &mut self.serializer)?); - Ok(()) + let (trailer, key_bounds) = + self.cws[column].finish::(&mut self.writer, &mut self.serializer)?; + self.finished_columns.push(trailer); + Ok(key_bounds) } pub fn close( @@ -1393,8 +1419,8 @@ where self.inner.n_rows() } - /// Finishes writing the layer file and returns the writer passed to - /// [`new`](Self::new). + /// Finishes writing the layer file and returns the file handle, optional + /// bloom filter, and column-0 key bounds. /// /// # Arguments /// @@ -1402,9 +1428,17 @@ where pub fn close( mut self, metadata: BatchMetadata, - ) -> Result<(Arc, Option), StorageError> { - self.inner.finish_column::(0)?; - self.inner.close(metadata) + ) -> Result< + ( + Arc, + Option, + Option<(Box, Box)>, + ), + StorageError, + > { + let key_bounds = self.inner.finish_column::(0)?; + let (file_handle, bloom_filter) = self.inner.close(metadata)?; + Ok((file_handle, bloom_filter, key_bounds)) } /// Returns the path for the file being written. @@ -1417,21 +1451,32 @@ where self.inner.storage() } - /// Finishes writing the layer file and returns a reader for it. - /// - /// # Arguments - /// - /// * `metadata` - Batch metadata to include in the trailer. - pub fn into_reader( + fn into_reader_impl( self, metadata: BatchMetadata, - ) -> Result, super::reader::Error> { + ) -> Result<(Reader<(&'static K0, &'static A0, ())>, BatchFilters), super::reader::Error> + { let any_factories = self.factories.any_factories(); let cache = self.inner.cache; - let (file_handle, bloom_filter) = self.close(metadata)?; - - Reader::new(&[&any_factories], cache, file_handle, bloom_filter) + let (file_handle, bloom_filter, key_bounds) = self.close(metadata)?; + let key_range = key_bounds + .as_ref() + .map(|(min, max)| KeyRange::from_refs(min.as_ref(), max.as_ref())); + let (reader, membership_filter) = + Reader::new_with_filter(&[&any_factories], cache, file_handle, bloom_filter)?; + let filters = BatchFilters::from_file(key_range, membership_filter); + Ok((reader, filters)) + } + + /// Finishes writing the layer file and returns a reader for it together + /// with exact-seek filters. + pub fn into_reader( + self, + metadata: BatchMetadata, + ) -> Result<(Reader<(&'static K0, &'static A0, ())>, BatchFilters), super::reader::Error> + { + self.into_reader_impl(metadata) } } @@ -1577,8 +1622,8 @@ where self.inner.n_rows() } - /// Finishes writing the layer file and returns the writer passed to - /// [`new`](Self::new). + /// Finishes writing the layer file and returns the file handle, optional + /// bloom filter, and column-0 key bounds. /// /// This function will panic if [`write1`](Self::write1) has been called /// without a subsequent call to [`write0`](Self::write0). @@ -1589,10 +1634,18 @@ where pub fn close( mut self, metadata: BatchMetadata, - ) -> Result<(Arc, Option), StorageError> { - self.inner.finish_column::(0)?; - self.inner.finish_column::(1)?; - self.inner.close(metadata) + ) -> Result< + ( + Arc, + Option, + Option<(Box, Box)>, + ), + StorageError, + > { + let key_bounds = self.inner.finish_column::(0)?; + let _ = self.inner.finish_column::(1)?; + let (file_handle, bloom_filter) = self.inner.close(metadata)?; + Ok((file_handle, bloom_filter, key_bounds)) } /// Returns the storage used for this writer. @@ -1605,28 +1658,46 @@ where self.inner.path() } - /// Finishes writing the layer file and returns a reader for it. - /// - /// # Arguments - /// - /// * `metadata` - Batch metadata to include in the trailer. - #[allow(clippy::type_complexity)] - pub fn into_reader( + fn into_reader_impl( self, metadata: BatchMetadata, ) -> Result< - Reader<(&'static K0, &'static A0, (&'static K1, &'static A1, ()))>, + ( + Reader<(&'static K0, &'static A0, (&'static K1, &'static A1, ()))>, + BatchFilters, + ), super::reader::Error, > { let any_factories0 = self.factories0.any_factories(); let any_factories1 = self.factories1.any_factories(); let cache = self.inner.cache; - let (file_handle, bloom_filter) = self.close(metadata)?; - Reader::new( + let (file_handle, bloom_filter, key_bounds) = self.close(metadata)?; + let key_range = key_bounds + .as_ref() + .map(|(min, max)| KeyRange::from_refs(min.as_ref(), max.as_ref())); + let (reader, membership_filter) = Reader::new_with_filter( &[&any_factories0, &any_factories1], cache, file_handle, bloom_filter, - ) + )?; + let filters = BatchFilters::from_file(key_range, membership_filter); + Ok((reader, filters)) + } + + /// Finishes writing the layer file and returns a reader for it together + /// with exact-seek filters. + #[allow(clippy::type_complexity)] + pub fn into_reader( + self, + metadata: BatchMetadata, + ) -> Result< + ( + Reader<(&'static K0, &'static A0, (&'static K1, &'static A1, ()))>, + BatchFilters, + ), + super::reader::Error, + > { + self.into_reader_impl(metadata) } } diff --git a/crates/dbsp/src/storage/filter_stats.rs b/crates/dbsp/src/storage/filter_stats.rs new file mode 100644 index 00000000000..54167363333 --- /dev/null +++ b/crates/dbsp/src/storage/filter_stats.rs @@ -0,0 +1,70 @@ +use crossbeam::utils::CachePadded; +use std::sync::atomic::{AtomicUsize, Ordering}; + +/// Statistics about an in-memory key filter. +/// +/// The statistics implement addition such that they can be summed across +/// batches. Their addition loses information about individual sizes, hits, +/// misses and by extension, hit rates. +#[derive( + Clone, + Copy, + Debug, + Default, + PartialEq, + derive_more::Add, + derive_more::AddAssign, + derive_more::Sum, +)] +pub struct FilterStats { + /// Filter size in bytes. + pub size_byte: usize, + /// Number of hits. + pub hits: usize, + /// Number of misses. + pub misses: usize, +} + +/// Shared hit/miss accounting for key filters. +#[derive(Debug)] +pub struct TrackingFilterStats { + size_byte: usize, + counts: CachePadded, +} + +#[derive(Debug)] +struct FilterCounts { + hits: AtomicUsize, + misses: AtomicUsize, +} + +impl TrackingFilterStats { + /// Creates tracking state for a filter of the given size. + pub fn new(size_byte: usize) -> Self { + Self { + size_byte, + counts: CachePadded::new(FilterCounts { + hits: AtomicUsize::new(0), + misses: AtomicUsize::new(0), + }), + } + } + + /// Retrieves statistics. + pub fn stats(&self) -> FilterStats { + FilterStats { + size_byte: self.size_byte, + hits: self.counts.hits.load(Ordering::Relaxed), + misses: self.counts.misses.load(Ordering::Relaxed), + } + } + + /// Records the result of one filter probe. + pub fn record(&self, is_hit: bool) { + if is_hit { + self.counts.hits.fetch_add(1, Ordering::Relaxed); + } else { + self.counts.misses.fetch_add(1, Ordering::Relaxed); + } + } +} diff --git a/crates/dbsp/src/storage/tracking_bloom_filter.rs b/crates/dbsp/src/storage/tracking_bloom_filter.rs index f8ad767f608..7f226789112 100644 --- a/crates/dbsp/src/storage/tracking_bloom_filter.rs +++ b/crates/dbsp/src/storage/tracking_bloom_filter.rs @@ -1,7 +1,5 @@ +use crate::storage::filter_stats::{FilterStats, TrackingFilterStats}; use fastbloom::BloomFilter; -use std::iter::Sum; -use std::ops::{Add, AddAssign}; -use std::sync::atomic::{AtomicUsize, Ordering}; /// Bloom filter which tracks the number of hits and misses when lookups are performed. /// It implements the subset of [`BloomFilter`] functions that are used by Feldera storage. @@ -9,68 +7,23 @@ use std::sync::atomic::{AtomicUsize, Ordering}; pub struct TrackingBloomFilter { /// Underlying Bloom filter. bloom_filter: BloomFilter, - /// Number of hits. - hits: AtomicUsize, - /// Number of misses. - misses: AtomicUsize, -} - -/// Statistics about the Bloom filter. -/// -/// The statistics implement addition such that they can be summed (e.g., when you have a spine -/// consisting out of multiple batches, each with their own Bloom filter). However, their addition -/// will lose information about individual sizes, hits, misses and by extension, hit rates. -#[derive(Clone, Copy, Debug, Default, PartialEq)] -pub struct BloomFilterStats { - /// Bloom filter size in bytes. - pub size_byte: usize, - /// Number of hits. - pub hits: usize, - /// Number of misses. - pub misses: usize, -} - -impl Add for BloomFilterStats { - type Output = Self; - - fn add(mut self, rhs: Self) -> Self::Output { - self.add_assign(rhs); - self - } -} - -impl AddAssign for BloomFilterStats { - fn add_assign(&mut self, rhs: Self) { - self.size_byte += rhs.size_byte; - self.hits += rhs.hits; - self.misses += rhs.misses; - } -} - -impl Sum for BloomFilterStats { - fn sum>(iter: I) -> Self { - iter.fold(Self::default(), Add::add) - } + tracking: TrackingFilterStats, } impl TrackingBloomFilter { /// Constructs a tracking Bloom filter which wraps a regular Bloom filter instance. /// It is assumed the underlying Bloom filter has not yet been used. pub fn new(bloom_filter: BloomFilter) -> Self { + let size_byte = size_of_val(&bloom_filter) + bloom_filter.num_bits() / 8; Self { + tracking: TrackingFilterStats::new(size_byte), bloom_filter, - hits: AtomicUsize::new(0), - misses: AtomicUsize::new(0), } } /// Retrieves statistics. - pub fn stats(&self) -> BloomFilterStats { - BloomFilterStats { - size_byte: size_of_val(&self.bloom_filter) + self.bloom_filter.num_bits() / 8, - hits: self.hits.load(Ordering::Acquire), - misses: self.misses.load(Ordering::Acquire), - } + pub fn stats(&self) -> FilterStats { + self.tracking.stats() } /// See [`BloomFilter::num_hashes`]. @@ -92,18 +45,15 @@ impl TrackingBloomFilter { /// It additionally counts the hits or misses, before returning. pub fn contains_hash(&self, hash: u64) -> bool { let is_hit = self.bloom_filter.contains_hash(hash); - if is_hit { - self.hits.fetch_add(1, Ordering::Release); - } else { - self.misses.fetch_add(1, Ordering::Release); - } + self.tracking.record(is_hit); is_hit } } #[cfg(test)] mod tests { - use super::{BloomFilterStats, TrackingBloomFilter}; + use super::TrackingBloomFilter; + use crate::storage::filter_stats::FilterStats; use fastbloom::BloomFilter; #[test] @@ -114,7 +64,7 @@ mod tests { assert!(filter.num_hashes() >= 1); assert_eq!( filter.stats(), - BloomFilterStats { + FilterStats { size_byte: 96 + 8192 / 8, hits: 0, misses: 0, @@ -126,7 +76,7 @@ mod tests { assert!(!filter.contains_hash(789)); assert_eq!( filter.stats(), - BloomFilterStats { + FilterStats { size_byte: 96 + 8192 / 8, hits: 1, misses: 2, @@ -137,8 +87,8 @@ mod tests { #[test] fn tracking_bloom_filter_stats_default() { assert_eq!( - BloomFilterStats::default(), - BloomFilterStats { + FilterStats::default(), + FilterStats { size_byte: 0, hits: 0, misses: 0, @@ -148,24 +98,24 @@ mod tests { #[test] fn tracking_bloom_filter_stats_addition() { - let stats1 = BloomFilterStats { + let stats1 = FilterStats { size_byte: 123, hits: 456, misses: 789, }; - let stats2 = BloomFilterStats { + let stats2 = FilterStats { size_byte: 100, hits: 200, misses: 300, }; - let stats3 = BloomFilterStats { + let stats3 = FilterStats { size_byte: 223, hits: 656, misses: 1089, }; assert_eq!(stats1 + stats2, stats3); assert_eq!( - vec![stats1, stats2].into_iter().sum::(), + vec![stats1, stats2].into_iter().sum::(), stats3 ); } diff --git a/crates/dbsp/src/trace.rs b/crates/dbsp/src/trace.rs index 461803a7165..573331b9aec 100644 --- a/crates/dbsp/src/trace.rs +++ b/crates/dbsp/src/trace.rs @@ -53,9 +53,7 @@ pub mod filter; pub mod layers; pub mod ord; pub mod spine_async; -pub use spine_async::{ - BatchReaderWithSnapshot, ListMerger, MergerType, Spine, SpineSnapshot, WithSnapshot, -}; +pub use spine_async::{BatchReaderWithSnapshot, ListMerger, Spine, SpineSnapshot, WithSnapshot}; #[cfg(test)] pub mod test; @@ -74,12 +72,12 @@ pub use ord::{ use rkyv::{Deserialize, archived_root}; -use crate::storage::tracking_bloom_filter::BloomFilterStats; use crate::{ Error, NumEntries, Timestamp, algebra::MonoidValue, dynamic::{DataTrait, DynPair, DynVec, DynWeightedPairs, Erase, Factory, WeightTrait}, storage::file::reader::Error as ReaderError, + storage::filter_stats::FilterStats, }; pub use cursor::{Cursor, MergeCursor}; pub use filter::{Filter, GroupFilter}; @@ -471,14 +469,20 @@ where /// the implementation need not attempt to cache the return value. fn approximate_byte_size(&self) -> usize; - /// Statistics of the Bloom filter used by [Cursor::seek_key_exact]. - /// The Bloom filter (kept in memory) is used there to quickly check - /// whether a key might be present in the batch, before doing a - /// binary tree lookup within the batch to be exactly sure. - /// The statistics include for example the size in bytes and the hit rate. - /// Only some kinds of batches use a filter; others should return - /// `BloomFilterStats::default()`. - fn filter_stats(&self) -> BloomFilterStats; + /// Statistics of the secondary membership filter used by + /// [Cursor::seek_key_exact] after the range filter. + /// + /// Today this is usually a Bloom filter. Batches without such a filter + /// should return `FilterStats::default()`. + fn membership_filter_stats(&self) -> FilterStats; + + /// Statistics of the in-memory range filter used by + /// [Cursor::seek_key_exact]. + /// + /// Batches without a range filter should return `FilterStats::default()`. + fn range_filter_stats(&self) -> FilterStats { + FilterStats::default() + } /// Where the batch's data is stored. fn location(&self) -> BatchLocation { @@ -497,12 +501,6 @@ where self.len() == 0 } - /// A method that returns either true (possibly in the batch) or false - /// (definitely not in the batch). - fn maybe_contains_key(&self, _hash: u64) -> bool { - true - } - /// Returns a uniform random sample of distincts keys from the batch. /// /// Does not take into account the number values associated with each @@ -659,8 +657,11 @@ where fn approximate_byte_size(&self) -> usize { (**self).approximate_byte_size() } - fn filter_stats(&self) -> BloomFilterStats { - (**self).filter_stats() + fn membership_filter_stats(&self) -> FilterStats { + (**self).membership_filter_stats() + } + fn range_filter_stats(&self) -> FilterStats { + (**self).range_filter_stats() } fn location(&self) -> BatchLocation { (**self).location() @@ -671,9 +672,6 @@ where fn is_empty(&self) -> bool { (**self).is_empty() } - fn maybe_contains_key(&self, hash: u64) -> bool { - (**self).maybe_contains_key(hash) - } fn sample_keys(&self, rng: &mut RG, sample_size: usize, sample: &mut DynVec) where Self::Time: PartialEq<()>, diff --git a/crates/dbsp/src/trace/ord.rs b/crates/dbsp/src/trace/ord.rs index fc5726d67df..e0bf2b0c689 100644 --- a/crates/dbsp/src/trace/ord.rs +++ b/crates/dbsp/src/trace/ord.rs @@ -1,8 +1,11 @@ +pub(crate) mod batch_filter; pub mod fallback; pub mod file; +pub(crate) mod key_range; pub mod merge_batcher; pub mod vec; +pub use batch_filter::{BatchFilterStats, BatchFilters}; pub use fallback::{ indexed_wset::{ FallbackIndexedWSet, FallbackIndexedWSet as OrdIndexedWSet, FallbackIndexedWSetBuilder, diff --git a/crates/dbsp/src/trace/ord/batch_filter.rs b/crates/dbsp/src/trace/ord/batch_filter.rs new file mode 100644 index 00000000000..6a47f708398 --- /dev/null +++ b/crates/dbsp/src/trace/ord/batch_filter.rs @@ -0,0 +1,259 @@ +//! Key filters for exact seeks. +//! +//! These filters only answer "definitely not present" versus "might still be +//! present", so callers can reject cheap cases before doing the exact lookup. + +use crate::{ + dynamic::{DataTrait, DynVec}, + storage::{ + file::reader::FilteredKeys, + filter_stats::{FilterStats, TrackingFilterStats}, + tracking_bloom_filter::TrackingBloomFilter, + }, + trace::ord::key_range::KeyRange, +}; +use size_of::SizeOf; +use smallvec::SmallVec; +use std::sync::Arc; + +/// A cheap, in-memory precheck used by `seek_key_exact`. +/// +/// Each filter may only reject a key early. Returning `true` means "the key +/// might still be present", so the caller must continue with the next filter or +/// the exact lookup. +pub(crate) trait BatchFilter: Send + Sync +where + K: DataTrait + ?Sized, +{ + /// Returns `false` only when the key is definitely absent. + /// + /// Filters that need a hash compute it through `hash`, so a chain of + /// filters pays that cost at most once. + fn maybe_contains_key(&self, key: &K, hash: &mut Option) -> bool; + + /// Statistics for this filter. + fn stats(&self) -> FilterStats; +} + +/// Ordered key filters for exact-key probes against file-backed batches. +/// +/// The key range is kept separately from the trait objects because batches +/// thread typed bounds through file-writer finalization and rebuild the range +/// when they create a new reader. All other filters are stored behind a trait +/// object so we can swap bloom for bitmap or another single post-range filter +/// without changing the batch structs again. +pub struct BatchFilters +where + K: DataTrait + ?Sized, +{ + range_filter: Arc>, + membership_filter: Option>>, +} + +/// Runtime statistics for the range and membership filters inside +/// [`BatchFilters`]. +#[derive(Clone, Copy, Debug, Default, PartialEq)] +pub struct BatchFilterStats { + pub range_filter: FilterStats, + pub membership_filter: FilterStats, +} + +/// A range filter with statistics. +#[derive(SizeOf)] +struct TrackedRangeFilter +where + K: DataTrait + ?Sized, +{ + range: Option>, + #[size_of(skip)] + tracking: TrackingFilterStats, +} + +impl TrackedRangeFilter +where + K: DataTrait + ?Sized, +{ + fn new(range: Option>) -> Self { + let size_byte = range + .as_ref() + .map(|range| range.size_of().total_bytes()) + .unwrap_or_default(); + Self { + range, + tracking: TrackingFilterStats::new(size_byte), + } + } + + fn stats(&self) -> FilterStats { + self.tracking.stats() + } +} + +impl BatchFilters +where + K: DataTrait + ?Sized, +{ + pub(crate) fn new( + range_filter: Option>, + membership_filter: Option>>, + ) -> Self { + Self { + range_filter: Arc::new(TrackedRangeFilter::new(range_filter)), + membership_filter, + } + } + + /// Builds the current file-backed filter chain. + /// + /// The range check runs before the bloom filter so out-of-range keys never + /// pay the hash or bloom lookup cost. + pub(crate) fn from_file( + key_range: Option>, + membership_filter: Option, + ) -> Self { + Self::new( + key_range, + membership_filter + .map(Arc::new) + .map(|filter| filter as Arc>), + ) + } + + /// Returns cumulative statistics for the range and membership filters. + pub fn stats(&self) -> BatchFilterStats { + BatchFilterStats { + range_filter: self.range_filter.stats(), + membership_filter: self + .membership_filter + .as_ref() + .map(|filter| filter.stats()) + .unwrap_or_default(), + } + } + + /// Returns the cached key bounds, when available. + pub fn key_bounds(&self) -> Option<(&K, &K)> { + self.range_filter.range.as_ref().map(|range| range.bounds()) + } + + /// Returns `keys`, optionally narrowed to the indexes that pass the filter + /// chain when that is cheap enough to be worthwhile. + pub(crate) fn filtered_keys<'a>(&self, keys: &'a DynVec) -> FilteredKeys<'a, K> { + debug_assert!(keys.is_sorted_by(&|a, b| a.cmp(b))); + + // Preserve the old `FilteredKeys` heuristic: if too many keys pass, + // avoid allocating the index vector and just keep the original slice. + let mut filter_pass_keys = SmallVec::<[_; 50]>::new(); + for (index, key) in keys.dyn_iter().enumerate() { + if self.maybe_contains_key(key, None) { + filter_pass_keys.push(index); + if filter_pass_keys.len() >= keys.len() / 300 { + return FilteredKeys::all(keys); + } + } + } + + FilteredKeys::with_filter_pass_keys(keys, Some(filter_pass_keys.into_vec())) + } + + /// Returns `false` only when `key` is definitely not present. + /// + /// Passing a cached `hash` avoids recomputing it when the caller already + /// has one available. + pub fn maybe_contains_key(&self, key: &K, mut hash: Option) -> bool { + if !self.range_filter.maybe_contains_key(key, &mut hash) { + return false; + } + + self.membership_filter + .as_ref() + .is_none_or(|filter| filter.maybe_contains_key(key, &mut hash)) + } +} + +impl Clone for BatchFilters +where + K: DataTrait + ?Sized, +{ + fn clone(&self) -> Self { + Self { + range_filter: self.range_filter.clone(), + membership_filter: self.membership_filter.clone(), + } + } +} + +impl SizeOf for BatchFilters +where + K: DataTrait + ?Sized, +{ + fn size_of_children(&self, context: &mut size_of::Context) { + self.range_filter.size_of_with_context(context); + context.add( + self.membership_filter + .as_ref() + .map(|filter| filter.stats().size_byte) + .unwrap_or_default(), + ); + } +} + +impl BatchFilter for Arc> +where + K: DataTrait + ?Sized, +{ + fn maybe_contains_key(&self, key: &K, _hash: &mut Option) -> bool { + let is_hit = self.range.as_ref().is_some_and(|range| range.contains(key)); + self.tracking.record(is_hit); + is_hit + } + + fn stats(&self) -> FilterStats { + self.as_ref().stats() + } +} + +impl BatchFilter for TrackingBloomFilter +where + K: DataTrait + ?Sized, +{ + fn maybe_contains_key(&self, key: &K, hash: &mut Option) -> bool { + let hash = hash.get_or_insert_with(|| key.default_hash()); + self.contains_hash(*hash) + } + + fn stats(&self) -> FilterStats { + TrackingBloomFilter::stats(self) + } +} + +#[cfg(test)] +mod tests { + use super::{BatchFilter, TrackedRangeFilter}; + use crate::{ + dynamic::DynData, storage::filter_stats::FilterStats, trace::ord::key_range::KeyRange, + }; + use std::sync::Arc; + + #[test] + fn tracked_range_filter_stats() { + let filter = Arc::new(TrackedRangeFilter::new(Some(KeyRange::from_refs( + (&1i32) as &DynData, + (&10i32) as &DynData, + )))); + + assert!(filter.maybe_contains_key((&5i32) as &DynData, &mut None)); + assert!(!filter.maybe_contains_key((&11i32) as &DynData, &mut None)); + + let stats = filter.stats(); + assert!(stats.size_byte > 0); + assert_eq!( + stats, + FilterStats { + size_byte: stats.size_byte, + hits: 1, + misses: 1, + } + ); + } +} diff --git a/crates/dbsp/src/trace/ord/fallback/indexed_wset.rs b/crates/dbsp/src/trace/ord/fallback/indexed_wset.rs index 2cd69b29539..54dbbc0b4ac 100644 --- a/crates/dbsp/src/trace/ord/fallback/indexed_wset.rs +++ b/crates/dbsp/src/trace/ord/fallback/indexed_wset.rs @@ -1,6 +1,6 @@ use super::utils::{copy_to_builder, pick_merge_destination}; use crate::storage::file::SerializerInner; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::{ DBWeight, Error, NumEntries, algebra::{AddAssignByRef, AddByRef, NegByRef, ZRingValue}, @@ -276,10 +276,17 @@ where } #[inline] - fn filter_stats(&self) -> BloomFilterStats { + fn membership_filter_stats(&self) -> FilterStats { match &self.inner { - Inner::File(file) => file.filter_stats(), - Inner::Vec(vec) => vec.filter_stats(), + Inner::File(file) => file.membership_filter_stats(), + Inner::Vec(vec) => vec.membership_filter_stats(), + } + } + + fn range_filter_stats(&self) -> FilterStats { + match &self.inner { + Inner::File(file) => file.range_filter_stats(), + Inner::Vec(vec) => vec.range_filter_stats(), } } @@ -308,13 +315,6 @@ where } } - fn maybe_contains_key(&self, hash: u64) -> bool { - match &self.inner { - Inner::Vec(vec) => vec.maybe_contains_key(hash), - Inner::File(file) => file.maybe_contains_key(hash), - } - } - async fn fetch( &self, keys: &B, diff --git a/crates/dbsp/src/trace/ord/fallback/key_batch.rs b/crates/dbsp/src/trace/ord/fallback/key_batch.rs index e306165c225..e6725e7dcd2 100644 --- a/crates/dbsp/src/trace/ord/fallback/key_batch.rs +++ b/crates/dbsp/src/trace/ord/fallback/key_batch.rs @@ -1,5 +1,5 @@ use super::utils::{copy_to_builder, pick_merge_destination}; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::{ DBData, DBWeight, NumEntries, Timestamp, dynamic::{ @@ -267,10 +267,18 @@ where } #[inline] - fn filter_stats(&self) -> BloomFilterStats { + fn membership_filter_stats(&self) -> FilterStats { match &self.inner { - Inner::File(file) => file.filter_stats(), - Inner::Vec(vec) => vec.filter_stats(), + Inner::File(file) => file.membership_filter_stats(), + Inner::Vec(vec) => vec.membership_filter_stats(), + } + } + + #[inline] + fn range_filter_stats(&self) -> FilterStats { + match &self.inner { + Inner::File(file) => file.range_filter_stats(), + Inner::Vec(vec) => vec.range_filter_stats(), } } @@ -299,13 +307,6 @@ where Inner::File(file) => file.sample_keys(rng, sample_size, output), } } - - fn maybe_contains_key(&self, hash: u64) -> bool { - match &self.inner { - Inner::Vec(vec) => vec.maybe_contains_key(hash), - Inner::File(file) => file.maybe_contains_key(hash), - } - } } impl Batch for FallbackKeyBatch diff --git a/crates/dbsp/src/trace/ord/fallback/val_batch.rs b/crates/dbsp/src/trace/ord/fallback/val_batch.rs index da52a9339dd..f376a626681 100644 --- a/crates/dbsp/src/trace/ord/fallback/val_batch.rs +++ b/crates/dbsp/src/trace/ord/fallback/val_batch.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use super::utils::{copy_to_builder, pick_merge_destination}; use crate::storage::buffer_cache::CacheStats; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::trace::cursor::{DelegatingCursor, PushCursor}; use crate::trace::ord::file::val_batch::FileValBuilder; use crate::trace::ord::vec::val_batch::VecValBuilder; @@ -274,10 +274,18 @@ where } #[inline] - fn filter_stats(&self) -> BloomFilterStats { + fn membership_filter_stats(&self) -> FilterStats { match &self.inner { - Inner::File(file) => file.filter_stats(), - Inner::Vec(vec) => vec.filter_stats(), + Inner::File(file) => file.membership_filter_stats(), + Inner::Vec(vec) => vec.membership_filter_stats(), + } + } + + #[inline] + fn range_filter_stats(&self) -> FilterStats { + match &self.inner { + Inner::File(file) => file.range_filter_stats(), + Inner::Vec(vec) => vec.range_filter_stats(), } } @@ -306,13 +314,6 @@ where Inner::File(file) => file.sample_keys(rng, sample_size, output), } } - - fn maybe_contains_key(&self, hash: u64) -> bool { - match &self.inner { - Inner::Vec(vec) => vec.maybe_contains_key(hash), - Inner::File(file) => file.maybe_contains_key(hash), - } - } } impl Batch for FallbackValBatch diff --git a/crates/dbsp/src/trace/ord/fallback/wset.rs b/crates/dbsp/src/trace/ord/fallback/wset.rs index 74131771217..c1de66b50e4 100644 --- a/crates/dbsp/src/trace/ord/fallback/wset.rs +++ b/crates/dbsp/src/trace/ord/fallback/wset.rs @@ -1,5 +1,5 @@ use super::utils::{copy_to_builder, pick_merge_destination}; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::{ DBWeight, NumEntries, algebra::{AddAssignByRef, AddByRef, NegByRef, ZRingValue}, @@ -274,10 +274,17 @@ where } #[inline] - fn filter_stats(&self) -> BloomFilterStats { + fn membership_filter_stats(&self) -> FilterStats { match &self.inner { - Inner::File(file) => file.filter_stats(), - Inner::Vec(vec) => vec.filter_stats(), + Inner::File(file) => file.membership_filter_stats(), + Inner::Vec(vec) => vec.membership_filter_stats(), + } + } + + fn range_filter_stats(&self) -> FilterStats { + match &self.inner { + Inner::File(file) => file.range_filter_stats(), + Inner::Vec(vec) => vec.range_filter_stats(), } } @@ -306,13 +313,6 @@ where } } - fn maybe_contains_key(&self, hash: u64) -> bool { - match &self.inner { - Inner::Vec(vec) => vec.maybe_contains_key(hash), - Inner::File(file) => file.maybe_contains_key(hash), - } - } - async fn fetch( &self, keys: &B, diff --git a/crates/dbsp/src/trace/ord/file/indexed_wset_batch.rs b/crates/dbsp/src/trace/ord/file/indexed_wset_batch.rs index d1e4cbeb9f7..4148e0e134c 100644 --- a/crates/dbsp/src/trace/ord/file/indexed_wset_batch.rs +++ b/crates/dbsp/src/trace/ord/file/indexed_wset_batch.rs @@ -1,5 +1,5 @@ use crate::storage::file::format::BatchMetadata; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::{ DBData, DBWeight, NumEntries, Runtime, algebra::{AddAssignByRef, AddByRef, NegByRef}, @@ -20,7 +20,7 @@ use crate::{ FileValBatch, VecIndexedWSetFactories, WeightedItem, cursor::{CursorFactory, CursorFactoryWrapper, Pending, Position, PushCursor}, merge_batches_by_reference, - ord::{file::UnwrapStorage, merge_batcher::MergeBatcher}, + ord::{batch_filter::BatchFilters, file::UnwrapStorage, merge_batcher::MergeBatcher}, }, }; use crate::{DynZWeight, ZWeight}; @@ -144,6 +144,7 @@ where factories: FileIndexedWSetFactories, #[allow(clippy::type_complexity)] file: Arc>, + filters: BatchFilters, } impl FileIndexedWSet @@ -202,6 +203,26 @@ where Self { factories: self.factories.clone(), file: self.file.clone(), + filters: self.filters.clone(), + } + } +} + +impl FileIndexedWSet +where + K: DataTrait + ?Sized, + V: DataTrait + ?Sized, + R: WeightTrait + ?Sized, +{ + fn from_parts( + factories: FileIndexedWSetFactories, + file: Arc>, + filters: BatchFilters, + ) -> Self { + Self { + factories, + file, + filters, } } } @@ -282,10 +303,8 @@ where negative_weight_count: (self.len() as u64) .saturating_sub(self.metadata().negative_weight_count), }; - Self { - factories: self.factories.clone(), - file: Arc::new(writer.into_reader(stats.clone()).unwrap_storage()), - } + let (file, filters) = writer.into_reader(stats).unwrap_storage(); + Self::from_parts(self.factories.clone(), Arc::new(file), filters) } } @@ -375,8 +394,12 @@ where self.file.byte_size().unwrap_storage() as usize } - fn filter_stats(&self) -> BloomFilterStats { - self.file.filter_stats() + fn membership_filter_stats(&self) -> FilterStats { + self.filters.stats().membership_filter + } + + fn range_filter_stats(&self) -> FilterStats { + self.filters.stats().range_filter } #[inline] @@ -388,10 +411,6 @@ where self.file.cache_stats() } - fn maybe_contains_key(&self, hash: u64) -> bool { - self.file.maybe_contains_key(hash) - } - fn sample_keys(&self, rng: &mut RG, sample_size: usize, output: &mut DynVec) where RG: Rng, @@ -438,9 +457,10 @@ where &*keys_vec }; + let filtered_keys = self.filters.filtered_keys(keys); let results = self .file - .fetch_indexed_zset(keys) + .fetch_indexed_zset(filtered_keys) .unwrap_storage() .async_results(self.factories.vec_indexed_wset_factory.clone()) .await @@ -468,16 +488,16 @@ where fn from_path(factories: &Self::Factories, path: &StoragePath) -> Result { let any_factory0 = factories.factories0.any_factories(); let any_factory1 = factories.factories1.any_factories(); - let file = Arc::new(Reader::open( + let (file, membership_filter) = Reader::open_with_filter( &[&any_factory0, &any_factory1], Runtime::buffer_cache, &*Runtime::storage_backend().unwrap_storage(), path, - )?); - Ok(Self { - factories: factories.clone(), - file, - }) + )?; + let file = Arc::new(file); + let key_range = file.key_range()?.map(Into::into); + let filters = BatchFilters::from_file(key_range, membership_filter); + Ok(Self::from_parts(factories.clone(), file, filters)) } fn negative_weight_count(&self) -> Option { @@ -772,8 +792,7 @@ where } fn seek_key_exact(&mut self, key: &K, hash: Option) -> bool { - let hash = hash.unwrap_or_else(|| key.default_hash()); - if !self.wset.maybe_contains_key(hash) { + if !self.wset.filters.maybe_contains_key(key, hash) { return false; } self.seek_key(key); @@ -904,10 +923,9 @@ where } fn done(self) -> FileIndexedWSet { - FileIndexedWSet { - factories: self.factories, - file: Arc::new(self.writer.into_reader(self.stats.clone()).unwrap_storage()), - } + let (file, filters) = self.writer.into_reader(self.stats).unwrap_storage(); + let file = Arc::new(file); + FileIndexedWSet::from_parts(self.factories, file, filters) } fn push_key(&mut self, key: &K) { diff --git a/crates/dbsp/src/trace/ord/file/key_batch.rs b/crates/dbsp/src/trace/ord/file/key_batch.rs index 26f5b4a2318..4c02b61b065 100644 --- a/crates/dbsp/src/trace/ord/file/key_batch.rs +++ b/crates/dbsp/src/trace/ord/file/key_batch.rs @@ -1,5 +1,5 @@ use crate::storage::file::format::BatchMetadata; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::trace::cursor::Position; use crate::{ DBData, DBWeight, NumEntries, Runtime, Timestamp, @@ -18,7 +18,7 @@ use crate::{ trace::{ Batch, BatchFactories, BatchLocation, BatchReader, BatchReaderFactories, Builder, Cursor, WeightedItem, - ord::{file::UnwrapStorage, merge_batcher::MergeBatcher}, + ord::{batch_filter::BatchFilters, file::UnwrapStorage, merge_batcher::MergeBatcher}, }, utils::Tup2, }; @@ -173,6 +173,7 @@ where (&'static DynDataTyped, &'static R, ()), )>, >, + filters: BatchFilters, } impl Debug for FileKeyBatch @@ -216,6 +217,32 @@ where Self { factories: self.factories.clone(), file: self.file.clone(), + filters: self.filters.clone(), + } + } +} + +impl FileKeyBatch +where + K: DataTrait + ?Sized, + T: Timestamp, + R: WeightTrait + ?Sized, +{ + fn from_parts( + factories: FileKeyBatchFactories, + file: Arc< + Reader<( + &'static K, + &'static DynUnit, + (&'static DynDataTyped, &'static R, ()), + )>, + >, + filters: BatchFilters, + ) -> Self { + Self { + factories, + file, + filters, } } } @@ -272,8 +299,12 @@ where self.file.byte_size().unwrap_storage() as usize } - fn filter_stats(&self) -> BloomFilterStats { - self.file.filter_stats() + fn membership_filter_stats(&self) -> FilterStats { + self.filters.stats().membership_filter + } + + fn range_filter_stats(&self) -> FilterStats { + self.filters.stats().range_filter } #[inline] @@ -309,10 +340,6 @@ where } } } - - fn maybe_contains_key(&self, hash: u64) -> bool { - self.file.maybe_contains_key(hash) - } } impl Batch for FileKeyBatch @@ -333,17 +360,17 @@ where fn from_path(factories: &Self::Factories, path: &StoragePath) -> Result { let any_factory0 = factories.factories0.any_factories(); let any_factory1 = factories.factories1.any_factories(); - let file = Arc::new(Reader::open( + let (file, membership_filter) = Reader::open_with_filter( &[&any_factory0, &any_factory1], Runtime::buffer_cache, &*Runtime::storage_backend().unwrap_storage(), path, - )?); + )?; + let file = Arc::new(file); + let key_range = file.key_range()?.map(Into::into); + let filters = BatchFilters::from_file(key_range, membership_filter); - Ok(Self { - factories: factories.clone(), - file, - }) + Ok(Self::from_parts(factories.clone(), file, filters)) } fn negative_weight_count(&self) -> Option { @@ -540,8 +567,7 @@ where } fn seek_key_exact(&mut self, key: &K, hash: Option) -> bool { - let hash = hash.unwrap_or_else(|| key.default_hash()); - if !self.batch.maybe_contains_key(hash) { + if !self.batch.filters.maybe_contains_key(key, hash) { return false; } self.seek_key(key); @@ -669,10 +695,9 @@ where } fn done(self) -> FileKeyBatch { - FileKeyBatch { - factories: self.factories, - file: Arc::new(self.writer.into_reader(self.stats).unwrap_storage()), - } + let (file, filters) = self.writer.into_reader(self.stats).unwrap_storage(); + let file = Arc::new(file); + FileKeyBatch::from_parts(self.factories, file, filters) } fn num_keys(&self) -> usize { diff --git a/crates/dbsp/src/trace/ord/file/val_batch.rs b/crates/dbsp/src/trace/ord/file/val_batch.rs index fdfe6ded9bb..6c3824a302f 100644 --- a/crates/dbsp/src/trace/ord/file/val_batch.rs +++ b/crates/dbsp/src/trace/ord/file/val_batch.rs @@ -1,5 +1,5 @@ use crate::storage::buffer_cache::CacheStats; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::trace::BatchLocation; use crate::trace::cursor::Position; use crate::trace::ord::file::UnwrapStorage; @@ -17,7 +17,7 @@ use crate::{ }, trace::{ Batch, BatchFactories, BatchReader, BatchReaderFactories, Builder, Cursor, WeightedItem, - ord::merge_batcher::MergeBatcher, + ord::{batch_filter::BatchFilters, merge_batcher::MergeBatcher}, }, utils::Tup2, }; @@ -212,7 +212,10 @@ where #[size_of(skip)] #[debug(skip)] factories: FileValBatchFactories, + #[debug(skip)] pub file: RawValBatch, + #[debug(skip)] + filters: BatchFilters, } impl Clone for FileValBatch @@ -226,6 +229,27 @@ where Self { factories: self.factories.clone(), file: self.file.clone(), + filters: self.filters.clone(), + } + } +} + +impl FileValBatch +where + K: DataTrait + ?Sized, + V: DataTrait + ?Sized, + T: Timestamp, + R: WeightTrait + ?Sized, +{ + fn from_parts( + factories: FileValBatchFactories, + file: RawValBatch, + filters: BatchFilters, + ) -> Self { + Self { + factories, + file, + filters, } } } @@ -297,8 +321,12 @@ where self.file.byte_size().unwrap_storage() as usize } - fn filter_stats(&self) -> BloomFilterStats { - self.file.filter_stats() + fn membership_filter_stats(&self) -> FilterStats { + self.filters.stats().membership_filter + } + + fn range_filter_stats(&self) -> FilterStats { + self.filters.stats().range_filter } #[inline] @@ -333,10 +361,6 @@ where } } } - - fn maybe_contains_key(&self, hash: u64) -> bool { - self.file.maybe_contains_key(hash) - } } impl Batch for FileValBatch @@ -358,16 +382,16 @@ where fn from_path(factories: &Self::Factories, path: &StoragePath) -> Result { let any_factory0 = factories.factories0.any_factories(); let any_factory1 = factories.factories1.any_factories(); - let file = Arc::new(Reader::open( + let (file, membership_filter) = Reader::open_with_filter( &[&any_factory0, &any_factory1], Runtime::buffer_cache, &*Runtime::storage_backend().unwrap_storage(), path, - )?); - Ok(Self { - factories: factories.clone(), - file, - }) + )?; + let file = Arc::new(file); + let key_range = file.key_range()?.map(Into::into); + let filters = BatchFilters::from_file(key_range, membership_filter); + Ok(Self::from_parts(factories.clone(), file, filters)) } fn negative_weight_count(&self) -> Option { @@ -577,8 +601,7 @@ where } fn seek_key_exact(&mut self, key: &K, hash: Option) -> bool { - let hash = hash.unwrap_or_else(|| key.default_hash()); - if !self.batch.maybe_contains_key(hash) { + if !self.batch.filters.maybe_contains_key(key, hash) { return false; } self.seek_key(key); @@ -703,10 +726,9 @@ where } fn done(self) -> FileValBatch { - FileValBatch { - factories: self.factories, - file: Arc::new(self.writer.into_reader(self.stats).unwrap_storage()), - } + let (file, filters) = self.writer.into_reader(self.stats).unwrap_storage(); + let file = Arc::new(file); + FileValBatch::from_parts(self.factories, file, filters) } fn push_key(&mut self, key: &K) { diff --git a/crates/dbsp/src/trace/ord/file/wset_batch.rs b/crates/dbsp/src/trace/ord/file/wset_batch.rs index 9175b165761..33a005f98d6 100644 --- a/crates/dbsp/src/trace/ord/file/wset_batch.rs +++ b/crates/dbsp/src/trace/ord/file/wset_batch.rs @@ -1,5 +1,5 @@ use crate::storage::file::format::BatchMetadata; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::{ DBData, DBWeight, NumEntries, Runtime, algebra::{AddAssignByRef, AddByRef, NegByRef}, @@ -20,7 +20,7 @@ use crate::{ DbspSerializer, Deserializer, FileKeyBatch, VecWSetFactories, WeightedItem, cursor::{CursorFactoryWrapper, Pending, Position, PushCursor}, merge_batches_by_reference, - ord::{file::UnwrapStorage, merge_batcher::MergeBatcher}, + ord::{batch_filter::BatchFilters, file::UnwrapStorage, merge_batcher::MergeBatcher}, }, }; use crate::{DynZWeight, ZWeight}; @@ -137,6 +137,7 @@ where #[size_of(skip)] factories: FileWSetFactories, file: Arc>, + filters: BatchFilters, } impl Debug for FileWSet @@ -171,6 +172,7 @@ where Self { factories: self.factories.clone(), file: self.file.clone(), + filters: self.filters.clone(), } } } @@ -180,6 +182,18 @@ where K: DataTrait + ?Sized, R: WeightTrait + ?Sized, { + fn from_parts( + factories: FileWSetFactories, + file: Arc>, + filters: BatchFilters, + ) -> Self { + Self { + factories, + file, + filters, + } + } + #[inline] pub fn len(&self) -> usize { self.file.n_rows(0) as usize @@ -250,7 +264,7 @@ where let mut cursor = self.cursor(); while cursor.key_valid() { - let diff = cursor.diff.neg_by_ref(); + let diff = cursor.weight().neg_by_ref(); writer.write0((cursor.key(), diff.erase())).unwrap_storage(); cursor.step_key(); } @@ -258,10 +272,8 @@ where negative_weight_count: (self.len() as u64) .saturating_sub(self.stats().negative_weight_count), }; - Self { - factories: self.factories.clone(), - file: Arc::new(writer.into_reader(stats).unwrap_storage()), - } + let (file, filters) = writer.into_reader(stats).unwrap_storage(); + Self::from_parts(self.factories.clone(), Arc::new(file), filters) } } @@ -371,8 +383,12 @@ where self.file.byte_size().unwrap_storage() as usize } - fn filter_stats(&self) -> BloomFilterStats { - self.file.filter_stats() + fn membership_filter_stats(&self) -> FilterStats { + self.filters.stats().membership_filter + } + + fn range_filter_stats(&self) -> FilterStats { + self.filters.stats().range_filter } #[inline] @@ -384,10 +400,6 @@ where self.file.cache_stats() } - fn maybe_contains_key(&self, hash: u64) -> bool { - self.file.maybe_contains_key(hash) - } - fn sample_keys(&self, rng: &mut RG, sample_size: usize, output: &mut DynVec) where RG: Rng, @@ -436,9 +448,10 @@ where &*keys_vec }; + let filtered_keys = self.filters.filtered_keys(keys); let results = self .file - .fetch_zset(keys) + .fetch_zset(filtered_keys) .unwrap_storage() .async_results(self.factories.vec_wset_factory.clone()) .await @@ -464,17 +477,17 @@ where fn from_path(factories: &Self::Factories, path: &StoragePath) -> Result { let any_factory0 = factories.file_factories.any_factories(); - let file = Arc::new(Reader::open( + let (file, membership_filter) = Reader::open_with_filter( &[&any_factory0], Runtime::buffer_cache, &*Runtime::storage_backend().unwrap(), path, - )?); + )?; + let file = Arc::new(file); + let key_range = file.key_range()?.map(Into::into); + let filters = BatchFilters::from_file(key_range, membership_filter); - Ok(Self { - factories: factories.clone(), - file, - }) + Ok(Self::from_parts(factories.clone(), file, filters)) } fn negative_weight_count(&self) -> Option { @@ -686,8 +699,7 @@ where } fn seek_key_exact(&mut self, key: &K, hash: Option) -> bool { - let hash = hash.unwrap_or_else(|| key.default_hash()); - if !self.wset.maybe_contains_key(hash) { + if !self.wset.filters.maybe_contains_key(key, hash) { return false; } self.seek_key(key); @@ -824,10 +836,9 @@ where } fn done(self) -> FileWSet { - FileWSet { - factories: self.factories, - file: Arc::new(self.writer.into_reader(self.stats).unwrap_storage()), - } + let (file, filters) = self.writer.into_reader(self.stats).unwrap_storage(); + let file = Arc::new(file); + FileWSet::from_parts(self.factories, file, filters) } fn push_key(&mut self, key: &K) { diff --git a/crates/dbsp/src/trace/ord/key_range.rs b/crates/dbsp/src/trace/ord/key_range.rs new file mode 100644 index 00000000000..7aff3bb2fa8 --- /dev/null +++ b/crates/dbsp/src/trace/ord/key_range.rs @@ -0,0 +1,62 @@ +//! In-memory key bounds cached for ordered batches. + +use crate::dynamic::DataTrait; +use dyn_clone::clone_box; +use size_of::SizeOf; + +/// Closed key interval for a batch. +/// +/// We materialize the endpoints once and keep them in memory so exact-key seeks +/// can reject out-of-range keys before touching slower filters. +#[derive(Debug, SizeOf)] +pub(crate) struct KeyRange +where + K: DataTrait + ?Sized, +{ + min: Box, + max: Box, +} + +impl KeyRange +where + K: DataTrait + ?Sized, +{ + /// Creates a range from owned endpoints. + pub(crate) fn new(min: Box, max: Box) -> Self { + assert!(min.as_ref() <= max.as_ref()); + Self { min, max } + } + + /// Clones a range from borrowed endpoints. + pub(crate) fn from_refs(min: &K, max: &K) -> Self { + Self::new(clone_box(min), clone_box(max)) + } + + /// Returns `true` when `key` is inside the closed interval. + pub(crate) fn contains(&self, key: &K) -> bool { + self.min.as_ref() <= key && key <= self.max.as_ref() + } + + /// Returns the bounds of the interval. + pub(crate) fn bounds(&self) -> (&K, &K) { + (self.min.as_ref(), self.max.as_ref()) + } +} + +impl From<(Box, Box)> for KeyRange +where + K: DataTrait + ?Sized, +{ + fn from((min, max): (Box, Box)) -> Self { + Self::new(min, max) + } +} + +impl Clone for KeyRange +where + K: DataTrait + ?Sized, +{ + fn clone(&self) -> Self { + Self::from_refs(self.min.as_ref(), self.max.as_ref()) + } +} diff --git a/crates/dbsp/src/trace/ord/vec/indexed_wset_batch.rs b/crates/dbsp/src/trace/ord/vec/indexed_wset_batch.rs index 2c4be1eb7cf..092e87d6717 100644 --- a/crates/dbsp/src/trace/ord/vec/indexed_wset_batch.rs +++ b/crates/dbsp/src/trace/ord/vec/indexed_wset_batch.rs @@ -1,5 +1,5 @@ use crate::storage::file::SerializerInner; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::trace::ord::merge_batcher::MergeBatcher; use crate::{ DBData, DBWeight, Error, NumEntries, @@ -457,8 +457,8 @@ where self.layer.approximate_byte_size() } - fn filter_stats(&self) -> BloomFilterStats { - BloomFilterStats::default() + fn membership_filter_stats(&self) -> FilterStats { + FilterStats::default() } fn sample_keys(&self, rng: &mut RG, sample_size: usize, sample: &mut DynVec) diff --git a/crates/dbsp/src/trace/ord/vec/key_batch.rs b/crates/dbsp/src/trace/ord/vec/key_batch.rs index 860737f3866..a1598db2d41 100644 --- a/crates/dbsp/src/trace/ord/vec/key_batch.rs +++ b/crates/dbsp/src/trace/ord/vec/key_batch.rs @@ -1,4 +1,4 @@ -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::trace::ord::merge_batcher::MergeBatcher; use crate::{ DBData, DBWeight, NumEntries, Timestamp, @@ -316,8 +316,8 @@ where self.layer.approximate_byte_size() } - fn filter_stats(&self) -> BloomFilterStats { - BloomFilterStats::default() + fn membership_filter_stats(&self) -> FilterStats { + FilterStats::default() } fn sample_keys(&self, rng: &mut RG, sample_size: usize, sample: &mut DynVec) diff --git a/crates/dbsp/src/trace/ord/vec/val_batch.rs b/crates/dbsp/src/trace/ord/vec/val_batch.rs index 6b25a842973..e19217ba530 100644 --- a/crates/dbsp/src/trace/ord/vec/val_batch.rs +++ b/crates/dbsp/src/trace/ord/vec/val_batch.rs @@ -1,5 +1,5 @@ use crate::ZWeight; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::trace::cursor::Position; use crate::trace::ord::merge_batcher::MergeBatcher; use crate::{ @@ -377,8 +377,8 @@ where self.layer.approximate_byte_size() } - fn filter_stats(&self) -> BloomFilterStats { - BloomFilterStats::default() + fn membership_filter_stats(&self) -> FilterStats { + FilterStats::default() } fn sample_keys(&self, rng: &mut RG, sample_size: usize, sample: &mut DynVec) diff --git a/crates/dbsp/src/trace/ord/vec/wset_batch.rs b/crates/dbsp/src/trace/ord/vec/wset_batch.rs index 86505464e97..2e262d21ede 100644 --- a/crates/dbsp/src/trace/ord/vec/wset_batch.rs +++ b/crates/dbsp/src/trace/ord/vec/wset_batch.rs @@ -1,4 +1,4 @@ -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::{ DBData, DBWeight, NumEntries, algebra::{NegByRef, ZRingValue}, @@ -359,8 +359,8 @@ impl BatchReader for VecWSet BloomFilterStats { - BloomFilterStats::default() + fn membership_filter_stats(&self) -> FilterStats { + FilterStats::default() } fn sample_keys(&self, rng: &mut RG, sample_size: usize, sample: &mut DynVec) diff --git a/crates/dbsp/src/trace/spine_async.rs b/crates/dbsp/src/trace/spine_async.rs index 88fd455ce42..11defddadf1 100644 --- a/crates/dbsp/src/trace/spine_async.rs +++ b/crates/dbsp/src/trace/spine_async.rs @@ -17,7 +17,8 @@ use crate::{ MERGE_BACKPRESSURE_WAIT_TIME_SECONDS, MERGE_REDUCTION_PERCENT, MERGING_BATCHES_COUNT, MERGING_MEMORY_RECORDS_COUNT, MERGING_SIZE_BYTES, MERGING_STORAGE_RECORDS_COUNT, MetaItem, MetricId, MetricReading, NEGATIVE_WEIGHT_COUNT, OperatorMeta, - SPINE_BATCHES_COUNT, SPINE_STORAGE_SIZE_BYTES, + RANGE_FILTER_HIT_RATE_PERCENT, RANGE_FILTER_HITS_COUNT, RANGE_FILTER_MISSES_COUNT, + RANGE_FILTER_SIZE_BYTES, SPINE_BATCHES_COUNT, SPINE_STORAGE_SIZE_BYTES, }, metrics::COMPACTION_STALL_TIME_NANOSECONDS, negative_weight_multiplier, @@ -25,7 +26,10 @@ use crate::{ }, dynamic::{DynVec, Factory, Weight}, samply::SamplySpan, - storage::buffer_cache::{BufferCache, CacheStats}, + storage::{ + buffer_cache::{BufferCache, CacheStats}, + filter_stats::FilterStats, + }, time::Timestamp, trace::{ Batch, BatchReader, BatchReaderFactories, Builder, Cursor, Filter, GroupFilter, Trace, @@ -40,15 +44,14 @@ use crate::{ use crate::storage::file::{Deserializer, to_bytes}; use crate::trace::CommittedSpine; -use dbsp::storage::tracking_bloom_filter::BloomFilterStats; use enum_map::EnumMap; use feldera_buffer_cache::ThreadType; use feldera_storage::{ FileCommitter, StoragePath, fbuf::slab::{FBufSlabs, TOKIO_FBUF_SLABS}, }; -use feldera_types::checkpoint::PSpineBatches; use feldera_types::memory_pressure::MemoryPressure; +use feldera_types::{checkpoint::PSpineBatches, config::dev_tweaks::MergerType}; use ouroboros::self_referencing; use rand::Rng; use rkyv::{Archive, Archived, Deserialize, Fallible, Serialize, ser::Serializer}; @@ -551,7 +554,7 @@ where let idle = Arc::clone(&idle); let no_backpressure = Arc::clone(&no_backpressure); let mut merger = None; - let merger_type = Runtime::with_dev_tweaks(|tweaks| tweaks.merger); + let merger_type = Runtime::with_dev_tweaks(|tweaks| tweaks.merger()); let notify = state.lock().unwrap().slots[level].notify.clone(); loop { @@ -777,11 +780,13 @@ where let mut cache_stats = spine_stats.cache_stats; let mut storage_size = 0; let mut merging_size = 0; - let mut filter_stats = BloomFilterStats::default(); + let mut membership_filter_stats = FilterStats::default(); + let mut range_filter_stats = FilterStats::default(); let mut storage_records = 0; for (batch, merging) in batches { cache_stats += batch.cache_stats(); - filter_stats += batch.filter_stats(); + membership_filter_stats += batch.membership_filter_stats(); + range_filter_stats += batch.range_filter_stats(); let on_storage = batch.location() == BatchLocation::Storage; if on_storage || merging { let size = batch.approximate_byte_size(); @@ -796,7 +801,8 @@ where } if storage_records > 0 { - let bits_per_key = filter_stats.size_byte as f64 * 8.0 / storage_records as f64; + let bits_per_key = + membership_filter_stats.size_byte as f64 * 8.0 / storage_records as f64; let bits_per_key = bits_per_key as usize; meta.extend(metadata! { BLOOM_FILTER_BITS_PER_KEY => MetaItem::Int(bits_per_key) @@ -833,24 +839,48 @@ where MetricReading::new( BLOOM_FILTER_SIZE_BYTES, Vec::new(), - MetaItem::bytes(filter_stats.size_byte), + MetaItem::bytes(membership_filter_stats.size_byte), ), MetricReading::new( BLOOM_FILTER_HITS_COUNT, Vec::new(), - MetaItem::Count(filter_stats.hits), + MetaItem::Count(membership_filter_stats.hits), ), MetricReading::new( BLOOM_FILTER_MISSES_COUNT, Vec::new(), - MetaItem::Count(filter_stats.misses), + MetaItem::Count(membership_filter_stats.misses), ), MetricReading::new( BLOOM_FILTER_HIT_RATE_PERCENT, Vec::new(), MetaItem::Percent { - numerator: filter_stats.hits as u64, - denominator: filter_stats.hits as u64 + filter_stats.misses as u64, + numerator: membership_filter_stats.hits as u64, + denominator: membership_filter_stats.hits as u64 + + membership_filter_stats.misses as u64, + }, + ), + MetricReading::new( + RANGE_FILTER_SIZE_BYTES, + Vec::new(), + MetaItem::bytes(range_filter_stats.size_byte), + ), + MetricReading::new( + RANGE_FILTER_HITS_COUNT, + Vec::new(), + MetaItem::Count(range_filter_stats.hits), + ), + MetricReading::new( + RANGE_FILTER_MISSES_COUNT, + Vec::new(), + MetaItem::Count(range_filter_stats.misses), + ), + MetricReading::new( + RANGE_FILTER_HIT_RATE_PERCENT, + Vec::new(), + MetaItem::Percent { + numerator: range_filter_stats.hits as u64, + denominator: range_filter_stats.hits as u64 + range_filter_stats.misses as u64, }, ), ]); @@ -1000,19 +1030,6 @@ impl WorkerState { } } -/// Which merger to use. -#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, serde::Deserialize)] -#[serde(rename_all = "snake_case")] -pub enum MergerType { - /// Newer merger, which should be faster for high-latency storage, such as - /// object storage, but it likely needs tuning. - PushMerger, - - /// The old standby, with known performance. - #[default] - ListMerger, -} - /// A single merge in progress in an [AsyncMerger]. struct Merge where @@ -1365,11 +1382,19 @@ where .sum() } - fn filter_stats(&self) -> BloomFilterStats { + fn membership_filter_stats(&self) -> FilterStats { + self.merger + .get_batches() + .iter() + .map(|batch| batch.membership_filter_stats()) + .sum() + } + + fn range_filter_stats(&self) -> FilterStats { self.merger .get_batches() .iter() - .map(|batch| batch.filter_stats()) + .map(|batch| batch.range_filter_stats()) .sum() } diff --git a/crates/dbsp/src/trace/spine_async/snapshot.rs b/crates/dbsp/src/trace/spine_async/snapshot.rs index e88f1b6365c..8f7e080779d 100644 --- a/crates/dbsp/src/trace/spine_async/snapshot.rs +++ b/crates/dbsp/src/trace/spine_async/snapshot.rs @@ -13,7 +13,7 @@ use size_of::SizeOf; use super::SpineCursor; use crate::NumEntries; use crate::dynamic::{DynVec, Factory}; -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::trace::cursor::{CursorFactory, CursorList}; use crate::trace::spine_async::sample_keys_from_batches; use crate::trace::{Batch, BatchReader, BatchReaderFactories, Cursor, Spine, merge_batches}; @@ -228,8 +228,15 @@ where .fold(0, |acc, batch| acc + batch.approximate_byte_size()) } - fn filter_stats(&self) -> BloomFilterStats { - self.batches.iter().map(|b| b.filter_stats()).sum() + fn membership_filter_stats(&self) -> FilterStats { + self.batches + .iter() + .map(|b| b.membership_filter_stats()) + .sum() + } + + fn range_filter_stats(&self) -> FilterStats { + self.batches.iter().map(|b| b.range_filter_stats()).sum() } fn sample_keys(&self, rng: &mut RG, sample_size: usize, sample: &mut DynVec) diff --git a/crates/dbsp/src/trace/test.rs b/crates/dbsp/src/trace/test.rs index 7abe4c1c623..b1849209bd9 100644 --- a/crates/dbsp/src/trace/test.rs +++ b/crates/dbsp/src/trace/test.rs @@ -12,15 +12,15 @@ use size_of::SizeOf; use crate::{ DynZWeight, Runtime, ZWeight, algebra::{ - IndexedZSet, OrdIndexedZSet, OrdIndexedZSetFactories, OrdZSet, OrdZSetFactories, ZBatch, - ZSet, + IndexedZSet, NegByRef, OrdIndexedZSet, OrdIndexedZSetFactories, OrdZSet, OrdZSetFactories, + ZBatch, ZSet, }, circuit::{CircuitConfig, mkconfig}, dynamic::{DowncastTrait, DynData, DynUnit, DynWeightedPairs, Erase, LeanVec, pair::DynPair}, trace::{ Batch, BatchReader, BatchReaderFactories, Builder, FileIndexedWSetFactories, FileWSetFactories, GroupFilter, Spine, Trace, - cursor::CursorPair, + cursor::{Cursor, CursorPair}, ord::{ FileIndexedWSet, FileKeyBatch, FileKeyBatchFactories, FileValBatch, FileValBatchFactories, FileWSet, OrdKeyBatch, OrdKeyBatchFactories, OrdValBatch, @@ -469,6 +469,87 @@ fn test_key_batch_spine>( } } +fn assert_out_of_range_seek_uses_range_filter(batch: &B, low: i32, high: i32) +where + B: BatchReader, +{ + let range_before = batch.range_filter_stats(); + let membership_before = batch.membership_filter_stats(); + assert!(range_before.size_byte > 0); + + let low: Box = Box::new(low).erase_box(); + let high: Box = Box::new(high).erase_box(); + let mut cursor = batch.cursor(); + assert!(!cursor.seek_key_exact(low.as_ref(), None)); + assert!(!cursor.seek_key_exact(high.as_ref(), None)); + + let range_after = batch.range_filter_stats(); + let membership_after = batch.membership_filter_stats(); + + assert_eq!(range_after.size_byte, range_before.size_byte); + assert_eq!(range_after.hits, range_before.hits); + assert_eq!(range_after.misses, range_before.misses + 2); + assert_eq!(membership_after, membership_before); +} + +#[test] +fn test_file_wset_neg_by_ref_preserves_key_bounds() { + run_in_circuit_with_storage(|| { + let factories = >::new::(); + let tuples = vec![Tup2(-10, 1), Tup2(0, -2), Tup2(25, 3)]; + + let mut erased_tuples = zset_tuples(tuples.clone()); + let batch = + FileWSet::::dyn_from_tuples(&factories, (), &mut erased_tuples); + let negated = batch.neg_by_ref(); + + let mut expected_tuples = zset_tuples( + tuples + .into_iter() + .map(|Tup2(key, weight)| Tup2(key, -weight)) + .collect(), + ); + let expected = + TestBatch::dyn_from_tuples(&TestBatchFactories::new(), (), &mut expected_tuples); + + assert_batch_eq(&negated, &expected); + assert_out_of_range_seek_uses_range_filter(&negated, -20, 40); + }); +} + +#[test] +fn test_file_indexed_wset_neg_by_ref_preserves_key_bounds() { + run_in_circuit_with_storage(|| { + let factories = + >::new::(); + let tuples = vec![ + Tup2(Tup2(-10, 1), 1), + Tup2(Tup2(0, 5), -2), + Tup2(Tup2(25, 7), 3), + ]; + + let mut erased_tuples = indexed_zset_tuples(tuples.clone()); + let batch = FileIndexedWSet::::dyn_from_tuples( + &factories, + (), + &mut erased_tuples, + ); + let negated = batch.neg_by_ref(); + + let mut expected_tuples = indexed_zset_tuples( + tuples + .into_iter() + .map(|Tup2(Tup2(key, val), weight)| Tup2(Tup2(key, val), -weight)) + .collect(), + ); + let expected = + TestBatch::dyn_from_tuples(&TestBatchFactories::new(), (), &mut expected_tuples); + + assert_batch_eq(&negated, &expected); + assert_out_of_range_seek_uses_range_filter(&negated, -20, 40); + }); +} + proptest! { #[test] fn test_truncate_key_bounded_memory(batches in kvr_batches_monotone_keys(100, 20, 50, 20, 500)) { diff --git a/crates/dbsp/src/trace/test/test_batch.rs b/crates/dbsp/src/trace/test/test_batch.rs index e7cb85b9274..99d64417552 100644 --- a/crates/dbsp/src/trace/test/test_batch.rs +++ b/crates/dbsp/src/trace/test/test_batch.rs @@ -3,7 +3,7 @@ //! So far, only methods/traits used in tests have been implemented. #![allow(clippy::type_complexity)] -use crate::storage::tracking_bloom_filter::BloomFilterStats; +use crate::storage::filter_stats::FilterStats; use crate::{ DBData, DBWeight, NumEntries, Timestamp, dynamic::{ @@ -1264,8 +1264,8 @@ where self.size_of().total_bytes() } - fn filter_stats(&self) -> BloomFilterStats { - BloomFilterStats::default() + fn membership_filter_stats(&self) -> FilterStats { + FilterStats::default() } fn sample_keys(&self, _rng: &mut RG, _sample_size: usize, _sample: &mut DynVec) diff --git a/crates/fda/src/main.rs b/crates/fda/src/main.rs index a0dfc85b9a7..d306993f129 100644 --- a/crates/fda/src/main.rs +++ b/crates/fda/src/main.rs @@ -14,7 +14,6 @@ use log::{debug, error, info, trace, warn}; use reqwest::StatusCode; use reqwest::header::{HeaderMap, HeaderValue, InvalidHeaderValue}; use serde_json::json; -use std::collections::BTreeMap; use std::convert::Infallible; use std::fs::File; use std::io::{ErrorKind, Read, Write, stdout}; @@ -467,8 +466,7 @@ fn patch_runtime_config( rc.io_workers = Some(value.parse().map_err(|_| ())?); } RuntimeConfigKey::DevTweaks => { - rc.dev_tweaks = serde_json::from_str::>(value) - .map_err(|_| ())?; + rc.dev_tweaks = serde_json::from_str(value).map_err(|_| ())?; } }; diff --git a/crates/feldera-types/src/config.rs b/crates/feldera-types/src/config.rs index d86041be915..33254934128 100644 --- a/crates/feldera-types/src/config.rs +++ b/crates/feldera-types/src/config.rs @@ -37,6 +37,9 @@ use std::{borrow::Cow, cmp::max, collections::BTreeMap}; use utoipa::ToSchema; use utoipa::openapi::{ObjectBuilder, OneOfBuilder, Ref, RefOr, Schema, SchemaType}; +pub mod dev_tweaks; +pub use dev_tweaks::DevTweaks; + const DEFAULT_MAX_PARALLEL_CONNECTOR_INIT: u64 = 10; /// Default value of `ConnectorConfig::max_queued_records`. @@ -926,11 +929,7 @@ pub struct RuntimeConfig { pub env: BTreeMap, /// Optional settings for tweaking Feldera internals. - /// - /// The available key-value pairs change from one version of Feldera to - /// another, so users should not depend on particular settings being - /// available, or on their behavior. - pub dev_tweaks: BTreeMap, + pub dev_tweaks: DevTweaks, /// Log filtering directives. /// @@ -1086,7 +1085,7 @@ impl Default for RuntimeConfig { io_workers: None, http_workers: None, env: BTreeMap::default(), - dev_tweaks: BTreeMap::default(), + dev_tweaks: DevTweaks::default(), logging: None, pipeline_template_configmap: None, } diff --git a/crates/feldera-types/src/config/dev_tweaks.rs b/crates/feldera-types/src/config/dev_tweaks.rs new file mode 100644 index 00000000000..057daa4c3cb --- /dev/null +++ b/crates/feldera-types/src/config/dev_tweaks.rs @@ -0,0 +1,286 @@ +use serde::{Deserialize, Serialize}; +use utoipa::ToSchema; + +/// Optional settings for tweaking Feldera internals. +/// +/// These settings reflect experiments that may come and go and change from +/// version to version. Users should not consider them to be stable. +#[derive(Debug, Clone, Default, PartialEq, Serialize, Deserialize, ToSchema)] +#[serde(default)] +pub struct DevTweaks { + /// Buffer-cache implementation to use for storage reads. + /// + /// The default is `s3_fifo`. + #[serde(skip_serializing_if = "Option::is_none")] + pub buffer_cache_strategy: Option, + + /// Override the number of buckets/shards used by sharded buffer caches. + /// + /// This only applies when `buffer_cache_strategy = "s3_fifo"`. Values are + /// rounded up to the next power of two because the current implementation + /// shards by `hash(key) & (n - 1)`. + #[serde(skip_serializing_if = "Option::is_none")] + pub buffer_max_buckets: Option, + + /// How S3-FIFO caches are assigned to foreground/background workers. + /// + /// This only applies when `buffer_cache_strategy = "s3_fifo"`. The + /// default is `shared_per_worker_pair`; LRU always uses `per_thread`. + #[serde(skip_serializing_if = "Option::is_none")] + pub buffer_cache_allocation_strategy: Option, + + /// Target number of cached bytes retained in each `FBuf` slab size class. + /// + /// The default is 16 MiB. + #[serde(skip_serializing_if = "Option::is_none")] + pub fbuf_slab_bytes_per_class: Option, + + /// Whether to asynchronously fetch keys needed for the join operator from + /// storage. Asynchronous fetching should be faster for high-latency + /// storage, such as object storage, but it could use excessive amounts of + /// memory if the number of keys fetched is very large. + #[serde(skip_serializing_if = "Option::is_none")] + pub fetch_join: Option, + + /// Whether to asynchronously fetch keys needed for the distinct operator + /// from storage. Asynchronous fetching should be faster for high-latency + /// storage, such as object storage, but it could use excessive amounts of + /// memory if the number of keys fetched is very large. + #[serde(skip_serializing_if = "Option::is_none")] + pub fetch_distinct: Option, + + /// Which merger to use. + #[serde(skip_serializing_if = "Option::is_none")] + pub merger: Option, + + /// If set, the maximum amount of storage, in MiB, for the POSIX backend to + /// allow to be in use before failing all writes with [StorageFull]. This + /// is useful for testing on top of storage that does not implement its own + /// quota mechanism. + /// + /// [StorageFull]: std::io::ErrorKind::StorageFull + #[serde(skip_serializing_if = "Option::is_none")] + pub storage_mb_max: Option, + + /// Attempt to print a stack trace on stack overflow. + /// + /// To be used for debugging only; do not enable in production. + // NOTE: this flag is handled manually in `adapters/src/server.rs` before + // parsing DevTweaks. If the name or type of this field changes, make sure to + // adjust `server.rs` accordingly. + #[serde(skip_serializing_if = "Option::is_none")] + pub stack_overflow_backtrace: Option, + + /// Controls the maximal number of records output by splitter operators + /// (joins, distinct, aggregation, rolling window and group operators) at + /// each step. + /// + /// The default value is 10,000 records. + // TODO: It would be better if the value were denominated in bytes rather + // than records, and if it were configurable per-operator. + #[serde(skip_serializing_if = "Option::is_none")] + pub splitter_chunk_size_records: Option, + + /// Enable adaptive joins. + /// + /// Adaptive joins dynamically change their partitioning policy to avoid skew. + /// + /// Adaptive joins are disabled by default. + #[serde(skip_serializing_if = "Option::is_none")] + pub adaptive_joins: Option, + + /// The minimum relative improvement threshold for the join balancer. + /// + /// The join balancer is a component that dynamically chooses an optimal + /// partitioning policy for adaptive join operators. This parameter + /// prevents the join balancer from making changes to the partitioning + /// policy if the improvement is not significant, since the overhead of such + /// rebalancing, especially when performed frequently, can exceed the + /// benefits. + /// + /// A rebalancing is considered significant if the relative estimated + /// improvement for the cluster of joins where the rebalancing is applied is + /// at least this threshold. + /// + /// A rebalancing is applied if both this threshold and + /// `balancer_min_absolute_improvement_threshold` are met. + /// + /// The default value is 1.2. + #[serde(skip_serializing_if = "Option::is_none")] + pub balancer_min_relative_improvement_threshold: Option, + + /// The minimum absolute improvement threshold for the balancer. + /// + /// The join balancer is a component that dynamically chooses an optimal + /// partitioning policy for adaptive join operators. This parameter + /// prevents the join balancer from making changes to the partitioning + /// policy if the improvement is not significant, since the overhead of such + /// rebalancing, especially when performed frequently, can exceed the + /// benefits. + /// + /// A rebalancing is considered significant if the absolute estimated + /// improvement for the cluster of joins where the rebalancing is applied is + /// at least this threshold. The cost model used by the balancer is based on + /// the number of records in the largest partition of a collection. + /// + /// A rebalancing is applied if both this threshold and + /// `balancer_min_relative_improvement_threshold` are met. + /// + /// The default value is 10,000. + #[serde(skip_serializing_if = "Option::is_none")] + pub balancer_min_absolute_improvement_threshold: Option, + + /// Factor that discourages the use of the Balance policy in a perfectly balanced collection. + /// + /// Assuming a perfectly balanced key distribution, the Balance policy is slightly less efficient than Shard, + /// since it requires computing the hash of the entire key/value pair. This factor discourages the use of this policy + /// if the skew is `, + + /// The balancer threshold for checking for an improved partitioning policy for a stream. + /// + /// Finding a good partitioning policy for a circuit involves solving an optimization problem, + /// which can be relatively expensive. Instead of doing this on every step, the balancer only + /// checks for an improved partitioning policy if the key distribution of a stream has changed + /// significantly since the current solution was computed. Specifically, it only kicks in when + /// the size of at least one shard of at least one stream in the cluster has changed by more than + /// this threshold. + /// + /// The default value is 0.1. + #[serde(skip_serializing_if = "Option::is_none")] + pub balancer_key_distribution_refresh_threshold: Option, + + /// False-positive rate for Bloom filters on batches on storage, as a + /// fraction f, where 0 < f < 1. + /// + /// The false-positive rate trades off between the amount of memory used by + /// Bloom filters and how frequently storage needs to be searched for keys + /// that are not actually present. Typical false-positive rates and their + /// corresponding memory costs are: + /// + /// - 0.1: 4.8 bits per key + /// - 0.01: 9.6 bits per key + /// - 0.001: 14.4 bits per key + /// - 0.0001: 19.2 bits per key (default) + /// + /// Values outside the valid range, such as 0.0, disable Bloom filters. + #[serde(skip_serializing_if = "Option::is_none")] + pub bloom_false_positive_rate: Option, + + /// Maximum batch size in records for level 0 merges. + #[serde(skip_serializing_if = "Option::is_none")] + pub max_level0_batch_size_records: Option, + + /// The number of merger threads. + /// + /// The default value is equal to the number of worker threads. + #[serde(skip_serializing_if = "Option::is_none")] + pub merger_threads: Option, + + /// Additional bias the merger assigns to records with negative weights + /// (retractions) to promote them to higher levels of the LSM tree sooner. + /// + /// Reasonable values for this parameter are in the range [0, 10]. + /// + /// The default value is 0, which means that retractions are not given + /// any additional bias. + #[serde(skip_serializing_if = "Option::is_none")] + pub negative_weight_multiplier: Option, +} + +impl DevTweaks { + pub fn buffer_cache_strategy(&self) -> BufferCacheStrategy { + self.buffer_cache_strategy.unwrap_or_default() + } + pub fn buffer_cache_allocation_strategy(&self) -> BufferCacheAllocationStrategy { + self.buffer_cache_allocation_strategy.unwrap_or_default() + } + pub fn effective_buffer_cache_allocation_strategy(&self) -> BufferCacheAllocationStrategy { + match self.buffer_cache_strategy() { + BufferCacheStrategy::S3Fifo => self.buffer_cache_allocation_strategy(), + BufferCacheStrategy::Lru => BufferCacheAllocationStrategy::PerThread, + } + } + pub fn fetch_join(&self) -> bool { + self.fetch_join.unwrap_or(false) + } + pub fn fetch_distinct(&self) -> bool { + self.fetch_distinct.unwrap_or(false) + } + pub fn merger(&self) -> MergerType { + self.merger.unwrap_or_default() + } + pub fn stack_overflow_backtrace(&self) -> bool { + self.stack_overflow_backtrace.unwrap_or(false) + } + pub fn splitter_chunk_size_records(&self) -> u64 { + self.splitter_chunk_size_records.unwrap_or(10_000) + } + pub fn adaptive_joins(&self) -> bool { + self.adaptive_joins.unwrap_or(false) + } + pub fn balancer_min_relative_improvement_threshold(&self) -> f64 { + self.balancer_min_relative_improvement_threshold + .unwrap_or(1.2) + } + pub fn balancer_min_absolute_improvement_threshold(&self) -> u64 { + self.balancer_min_absolute_improvement_threshold + .unwrap_or(10_000) + } + pub fn balancer_balance_tax(&self) -> f64 { + self.balancer_balance_tax.unwrap_or(1.1) + } + pub fn balancer_key_distribution_refresh_threshold(&self) -> f64 { + self.balancer_key_distribution_refresh_threshold + .unwrap_or(0.1) + } + pub fn bloom_false_positive_rate(&self) -> f64 { + self.bloom_false_positive_rate.unwrap_or(0.0001) + } + pub fn negative_weight_multiplier(&self) -> u16 { + self.negative_weight_multiplier.unwrap_or(0) + } +} + +/// Selects which eviction strategy backs a cache instance. +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize, ToSchema)] +#[serde(rename_all = "snake_case")] +pub enum BufferCacheStrategy { + /// Use the sharded S3-FIFO cache backed by `quick_cache`. + #[default] + S3Fifo, + + /// Use the mutex-protected weighted LRU cache. + Lru, +} + +/// Controls how caches are shared across a foreground/background worker pair. +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize, ToSchema)] +#[serde(rename_all = "snake_case")] +pub enum BufferCacheAllocationStrategy { + /// Share one cache across a foreground/background worker pair. + #[default] + SharedPerWorkerPair, + + /// Create a separate cache for each foreground/background thread. + PerThread, + + /// Share one cache across all foreground/background threads. + Global, +} + +/// Which merger to use. +#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize, ToSchema)] +#[serde(rename_all = "snake_case")] +pub enum MergerType { + /// Newer merger, which should be faster for high-latency storage, such as + /// object storage, but it likely needs tuning. + PushMerger, + + /// The old standby, with known performance. + #[default] + ListMerger, +} diff --git a/crates/pipeline-manager/src/api/examples.rs b/crates/pipeline-manager/src/api/examples.rs index de86ee60834..7606165694c 100644 --- a/crates/pipeline-manager/src/api/examples.rs +++ b/crates/pipeline-manager/src/api/examples.rs @@ -19,7 +19,7 @@ use crate::runner::error::RunnerError; use crate::runner::interaction::{ format_disconnected_error_message, format_timeout_error_message, RunnerInteraction, }; -use feldera_types::config::{FtConfig, ResourceConfig, StorageOptions}; +use feldera_types::config::{DevTweaks, FtConfig, ResourceConfig, StorageOptions}; use feldera_types::{config::RuntimeConfig, error::ErrorResponse}; use uuid::uuid; @@ -123,7 +123,7 @@ fn extended_pipeline_2() -> ExtendedPipelineDescr { io_workers: None, http_workers: None, env: BTreeMap::new(), - dev_tweaks: BTreeMap::new(), + dev_tweaks: DevTweaks::default(), logging: None, pipeline_template_configmap: None, }) diff --git a/crates/pipeline-manager/src/api/main.rs b/crates/pipeline-manager/src/api/main.rs index 43e421260fc..a70b6e0151a 100644 --- a/crates/pipeline-manager/src/api/main.rs +++ b/crates/pipeline-manager/src/api/main.rs @@ -356,6 +356,10 @@ It contains the following fields: feldera_types::config::ObjectStorageConfig, feldera_types::config::FtModel, feldera_types::config::PipelineTemplateConfig, + feldera_types::config::DevTweaks, + feldera_types::config::dev_tweaks::BufferCacheStrategy, + feldera_types::config::dev_tweaks::BufferCacheAllocationStrategy, + feldera_types::config::dev_tweaks::MergerType, feldera_types::transport::adhoc::AdHocInputConfig, feldera_types::transport::clock::ClockConfig, feldera_types::transport::file::FileInputConfig, diff --git a/crates/pipeline-manager/src/db/test.rs b/crates/pipeline-manager/src/db/test.rs index 3b762f9a9a4..f2609a25936 100644 --- a/crates/pipeline-manager/src/db/test.rs +++ b/crates/pipeline-manager/src/db/test.rs @@ -29,7 +29,9 @@ use crate::db::types::utils::{ use crate::db::types::version::Version; use async_trait::async_trait; use chrono::{TimeZone, Utc}; -use feldera_types::config::{FtConfig, PipelineConfig, ProgramIr, ResourceConfig, RuntimeConfig}; +use feldera_types::config::{ + DevTweaks, FtConfig, PipelineConfig, ProgramIr, ResourceConfig, RuntimeConfig, +}; use feldera_types::error::ErrorResponse; use feldera_types::program_schema::ProgramSchema; use feldera_types::runtime_status::{ @@ -322,7 +324,7 @@ fn map_val_to_limited_runtime_config(val: RuntimeConfigPropVal) -> serde_json::V http_workers: val.val18, io_workers: val.val19, env: BTreeMap::new(), - dev_tweaks: BTreeMap::new(), + dev_tweaks: DevTweaks::default(), logging: None, pipeline_template_configmap: None, }) @@ -1207,7 +1209,7 @@ async fn pipeline_versioning() { http_workers: None, io_workers: None, env: BTreeMap::new(), - dev_tweaks: BTreeMap::new(), + dev_tweaks: DevTweaks::default(), logging: None, pipeline_template_configmap: None, }) @@ -2254,7 +2256,7 @@ async fn pipeline_provision_version_guard() { max_parallel_connector_init: None, init_containers: None, checkpoint_during_suspend: false, - dev_tweaks: BTreeMap::new(), + dev_tweaks: DevTweaks::default(), http_workers: None, io_workers: None, env: BTreeMap::new(), diff --git a/crates/pipeline-manager/src/db/types/program.rs b/crates/pipeline-manager/src/db/types/program.rs index db465e483ea..4d3e381b79d 100644 --- a/crates/pipeline-manager/src/db/types/program.rs +++ b/crates/pipeline-manager/src/db/types/program.rs @@ -845,9 +845,7 @@ pub fn generate_pipeline_config( global: runtime_config.clone(), storage_config: None, // Set by the runner based on global field secrets_dir: None, - multihost: if runtime_config.hosts > 1 - || runtime_config.dev_tweaks.contains_key("multihost") - { + multihost: if runtime_config.hosts > 1 { Some(MultihostConfig { hosts: runtime_config.hosts, }) diff --git a/crates/sqllib/tests/tuple_proptest.rs b/crates/sqllib/tests/tuple_proptest.rs index 683a1508e41..e604a03ceaa 100644 --- a/crates/sqllib/tests/tuple_proptest.rs +++ b/crates/sqllib/tests/tuple_proptest.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use dbsp::DBData; use dbsp::algebra::{F32, F64}; -use dbsp::dynamic::{DynData, Erase}; +use dbsp::dynamic::{DowncastTrait, DynData, Erase}; use dbsp::storage::backend::memory_impl::MemoryBackend; use dbsp::storage::buffer_cache::BufferCache; use dbsp::storage::file::Factories; @@ -315,9 +315,21 @@ where .map_err(|err| TestCaseError::fail(format!("write failed: {err:?}")))?; } - let reader = writer + let (reader, filters) = writer .into_reader(BatchMetadata::default()) .map_err(|err| TestCaseError::fail(format!("reader init failed: {err:?}")))?; + match (values.first(), values.last(), filters.key_bounds()) { + (None, None, None) => {} + (Some(expected_min), Some(expected_max), Some((min, max))) => { + prop_assert_eq!(min.downcast_checked::(), expected_min); + prop_assert_eq!(max.downcast_checked::(), expected_max); + } + _ => { + return Err(TestCaseError::fail( + "writer returned unexpected filter bounds for sorted input".to_string(), + )); + } + } prop_assert_eq!(reader.n_rows(0) as usize, values.len()); let mut bulk = reader diff --git a/crates/storage-test-compat/src/bin/golden-writer.rs b/crates/storage-test-compat/src/bin/golden-writer.rs index 9d09a58db2b..ba8f2a2a8d5 100644 --- a/crates/storage-test-compat/src/bin/golden-writer.rs +++ b/crates/storage-test-compat/src/bin/golden-writer.rs @@ -112,7 +112,7 @@ where } let tmp_path = writer.path().clone(); - let (_file_handle, _bloom_filter) = writer.close(BatchMetadata::default())?; + let (_file_handle, _bloom_filter, _key_bounds) = writer.close(BatchMetadata::default())?; let content = storage_backend.read(&tmp_path)?; storage_backend.write(&output_storage_path, (*content).clone())?; storage_backend.delete(&tmp_path)?; diff --git a/crates/storage-test-compat/src/lib.rs b/crates/storage-test-compat/src/lib.rs index 75fdbe5774c..588b54cc72c 100644 --- a/crates/storage-test-compat/src/lib.rs +++ b/crates/storage-test-compat/src/lib.rs @@ -402,7 +402,7 @@ mod tests { buffer_cache, golden_aux, golden_file_directory, golden_row, golden_row_small, GoldenRow, GoldenRowSmall, }; - use dbsp::dynamic::{DynData, Erase}; + use dbsp::dynamic::{DowncastTrait, DynData, Erase}; use dbsp::storage::backend::{StorageBackend, StoragePath}; use dbsp::storage::file::format::BatchMetadata; use dbsp::storage::file::reader::Reader; @@ -410,6 +410,29 @@ mod tests { use dbsp::DBData; use feldera_types::config::{StorageConfig, StorageOptions}; + fn validate_key_range( + reader: &Reader<(&'static DynData, &'static DynData, ())>, + row_builder: fn(usize) -> T, + ) where + T: DBData + Default + Erase, + { + let n_rows = reader.n_rows(0) as usize; + let key_range = reader.key_range().unwrap(); + + if n_rows == 0 { + assert!(key_range.is_none()); + return; + } + + // The golden writer emits keys in deterministic sorted order, so the + // persisted range should match the first and last logical rows. + let expected_min = row_builder(0); + let expected_max = row_builder(n_rows - 1); + let (min, max) = key_range.expect("expected non-empty key range"); + assert_eq!(min.downcast_checked::(), &expected_min); + assert_eq!(max.downcast_checked::(), &expected_max); + } + fn validate_rows( storage_backend: &dyn StorageBackend, storage_path: StoragePath, @@ -429,6 +452,7 @@ mod tests { assert_eq!(reader.metadata(), &BatchMetadata::default()); let n_rows = reader.n_rows(0) as usize; + validate_key_range(&reader, row_builder); let mut bulk = reader.bulk_rows().unwrap(); let mut tmp_key = T::default(); let mut tmp_aux = 0i64; diff --git a/docs.feldera.com/docs/operations/metrics.md b/docs.feldera.com/docs/operations/metrics.md index a8cebfc3152..b1369868a5c 100644 --- a/docs.feldera.com/docs/operations/metrics.md +++ b/docs.feldera.com/docs/operations/metrics.md @@ -102,6 +102,14 @@ to work with data larger than memory. | `storage_byte_seconds_total` |counter | Storage usage integrated over time during this run of the pipeline, in bytes × seconds. | | `storage_cache_usage_bytes` |gauge | The number of bytes of memory currently in use for caching data on storage. | | `storage_cache_usage_limit_bytes_total` |counter | The limit for the number of bytes of memory for caching data on storage. | +| `storage_fbuf_slab_alloc_served_by_slab_percent` |gauge | The percentage of `FBuf` allocation requests served by slab pools. | +| `storage_fbuf_slab_alloc_total` |counter | The total number of `FBuf` allocation requests across all slab size classes and fallbacks. | +| `storage_fbuf_slab_free_served_by_slab_percent` |gauge | The percentage of `FBuf` deallocation requests served by slab pools. | +| `storage_fbuf_slab_free_total` |counter | The total number of `FBuf` deallocation requests across all slab size classes and fallbacks. | +| `storage_fbuf_slab_size_class_alloc_served_by_slab_percent` |gauge | The percentage of `FBuf` allocation requests served by each slab size class. | +| `storage_fbuf_slab_size_class_alloc_total` |counter | The total number of `FBuf` allocation requests for each slab size class. | +| `storage_fbuf_slab_size_class_free_served_by_slab_percent` |gauge | The percentage of `FBuf` deallocation requests served by each slab size class. | +| `storage_fbuf_slab_size_class_free_total` |counter | The total number of `FBuf` deallocation requests for each slab size class. | | `storage_read_block_bytes` |histogram | Sizes in bytes of blocks read from storage. | | `storage_read_latency_seconds` |histogram | Read latency for storage blocks in seconds | | `storage_sync_latency_seconds` |histogram | Sync latency in seconds | diff --git a/docs.feldera.com/package.json b/docs.feldera.com/package.json index 7a98592e51a..4a1f820fe23 100644 --- a/docs.feldera.com/package.json +++ b/docs.feldera.com/package.json @@ -55,9 +55,12 @@ "resolutions": { "openapi-to-postmanv2": "^5.4.0", "lodash": "^4.17.23", - "serialize-javascript": "^7.0.3", + "serialize-javascript": "^7.0.5", "gray-matter/js-yaml": "3.14.2", "json-refs/js-yaml": "3.14.2", - "*/js-yaml": "4.1.1" + "*/js-yaml": "4.1.1", + "yaml": "^1.10.3", + "path-to-regexp": "0.1.13", + "brace-expansion": "1.1.13" } } diff --git a/docs.feldera.com/yarn.lock b/docs.feldera.com/yarn.lock index 20a7cb5cf56..58a05a15eed 100644 --- a/docs.feldera.com/yarn.lock +++ b/docs.feldera.com/yarn.lock @@ -3584,21 +3584,14 @@ boxen@^7.0.0: widest-line "^4.0.1" wrap-ansi "^8.1.0" -brace-expansion@^1.1.7: - version "1.1.12" - resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-1.1.12.tgz#ab9b454466e5a8cc3a187beaad580412a9c5b843" - integrity sha512-9T9UjW3r0UW5c1Q7GTwllptXwhvYmEzFhzMfZ9H7FQWt+uZePjZPjBP/W1ZEyZ1twGWom5/56TF4lPcqjnDHcg== +brace-expansion@1.1.13, brace-expansion@^1.1.7, brace-expansion@^2.0.1: + version "1.1.13" + resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-1.1.13.tgz#d37875c01dc9eff988dd49d112a57cb67b54efe6" + integrity sha512-9ZLprWS6EENmhEOpjCYW2c8VkmOvckIJZfkr7rBW6dObmfgJ/L1GpSYW5Hpo9lDz4D1+n0Ckz8rU7FwHDQiG/w== dependencies: balanced-match "^1.0.0" concat-map "0.0.1" -brace-expansion@^2.0.1: - version "2.0.2" - resolved "https://registry.yarnpkg.com/brace-expansion/-/brace-expansion-2.0.2.tgz#54fc53237a613d854c7bd37463aad17df87214e7" - integrity sha512-Jt0vHyM+jmUBqojB7E1NIYadt0vI0Qxjxd2TErW94wDz+E2LAm5vKMXXwg6ZZBTHPuUlDgQHKXvjGBdfcF1ZDQ== - dependencies: - balanced-match "^1.0.0" - braces@^3.0.3, braces@~3.0.2: version "3.0.3" resolved "https://registry.yarnpkg.com/braces/-/braces-3.0.3.tgz#490332f40919452272d55a8480adc0c441358789" @@ -6132,11 +6125,6 @@ is-yarn-global@^0.4.0: resolved "https://registry.yarnpkg.com/is-yarn-global/-/is-yarn-global-0.4.1.tgz#b312d902b313f81e4eaf98b6361ba2b45cd694bb" integrity sha512-/kppl+R+LO5VmhYSEWARUFjodS25D68gvj8W7z0I7OWhUla5xWu8KL6CtB2V0R6yqhnRgbcaREMr4EEM6htLPQ== -isarray@0.0.1: - version "0.0.1" - resolved "https://registry.yarnpkg.com/isarray/-/isarray-0.0.1.tgz#8a18acfca9a8f4177e09abfc6038939b05d1eedf" - integrity sha512-D2S+3GLxWH+uhrNEcoh/fnmYeP8E8/zHl644d/jdA0g2uyXvy3sb0qxotE+ne0LtccHknQzWwZEzhak7oJ0COQ== - isarray@~1.0.0: version "1.0.0" resolved "https://registry.yarnpkg.com/isarray/-/isarray-1.0.0.tgz#bb935d48582cba168c06834957a54a3e07124f11" @@ -7802,22 +7790,10 @@ path-parse@^1.0.7: resolved "https://registry.yarnpkg.com/path-parse/-/path-parse-1.0.7.tgz#fbc114b60ca42b30d9daf5858e4bd68bbedb6735" integrity sha512-LDJzPVEEEPR+y48z93A0Ed0yXb8pAByGWo/k5YYdYgpY2/2EsOsksJrq7lOHxryrVOn1ejG6oAp8ahvOIQD8sw== -path-to-regexp@3.3.0: - version "3.3.0" - resolved "https://registry.yarnpkg.com/path-to-regexp/-/path-to-regexp-3.3.0.tgz#f7f31d32e8518c2660862b644414b6d5c63a611b" - integrity sha512-qyCH421YQPS2WFDxDjftfc1ZR5WKQzVzqsp4n9M2kQhVOo/ByahFoUNJfl58kOcEGfQ//7weFTDhm+ss8Ecxgw== - -path-to-regexp@^1.7.0: - version "1.9.0" - resolved "https://registry.yarnpkg.com/path-to-regexp/-/path-to-regexp-1.9.0.tgz#5dc0753acbf8521ca2e0f137b4578b917b10cf24" - integrity sha512-xIp7/apCFJuUHdDLWe8O1HIkb0kQrOMb/0u6FXQjemHn/ii5LrIzU6bdECnsiTF/GjZkMEKg1xdiZwNqDYlZ6g== - dependencies: - isarray "0.0.1" - -path-to-regexp@~0.1.12: - version "0.1.12" - resolved "https://registry.yarnpkg.com/path-to-regexp/-/path-to-regexp-0.1.12.tgz#d5e1a12e478a976d432ef3c58d534b9923164bb7" - integrity sha512-RA1GjUVMnvYFxuqovrEqZoxxW5NUZqbwKtYz/Tt7nXerk0LbLblQmrsgdeOxV5SFHf0UDggjS/bSeOZwt1pmEQ== +path-to-regexp@0.1.13, path-to-regexp@3.3.0, path-to-regexp@^1.7.0, path-to-regexp@~0.1.12: + version "0.1.13" + resolved "https://registry.yarnpkg.com/path-to-regexp/-/path-to-regexp-0.1.13.tgz#9b22ec16bc3ab88d05a0c7e369869421401ab17d" + integrity sha512-A/AGNMFN3c8bOlvV9RreMdrv7jsmF9XIfDeCd87+I8RNg6s78BhJxMu69NEMHBSJFxKidViTEdruRwEk/WIKqA== path-type@^4.0.0: version "4.0.0" @@ -9329,10 +9305,10 @@ send@~0.19.0, send@~0.19.1: range-parser "~1.2.1" statuses "~2.0.2" -serialize-javascript@^6.0.0, serialize-javascript@^6.0.1, serialize-javascript@^7.0.3: - version "7.0.4" - resolved "https://registry.yarnpkg.com/serialize-javascript/-/serialize-javascript-7.0.4.tgz#c517735bd5b7631dd1fc191ee19cbb713ff8e05c" - integrity sha512-DuGdB+Po43Q5Jxwpzt1lhyFSYKryqoNjQSA9M92tyw0lyHIOur+XCalOUe0KTJpyqzT8+fQ5A0Jf7vCx/NKmIg== +serialize-javascript@^6.0.0, serialize-javascript@^6.0.1, serialize-javascript@^7.0.5: + version "7.0.5" + resolved "https://registry.yarnpkg.com/serialize-javascript/-/serialize-javascript-7.0.5.tgz#c798cc0552ffbb08981914a42a8756e339d0d5b1" + integrity sha512-F4LcB0UqUl1zErq+1nYEEzSHJnIwb3AF2XWB94b+afhrekOUijwooAYqFyRbjYkm2PAKBabx6oYv/xDxNi8IBw== serve-handler@^6.1.6: version "6.1.7" @@ -10559,12 +10535,7 @@ yaml-ast-parser@0.0.43: resolved "https://registry.yarnpkg.com/yaml-ast-parser/-/yaml-ast-parser-0.0.43.tgz#e8a23e6fb4c38076ab92995c5dca33f3d3d7c9bb" integrity sha512-2PTINUwsRqSd+s8XxKaJWQlUuEMHJQyEuh2edBbW8KNJz0SJPwUSD2zRWqezFEdN7IzAgeuYHFUCF7o8zRdZ0A== -yaml@1.10.2: - version "1.10.2" - resolved "https://registry.yarnpkg.com/yaml/-/yaml-1.10.2.tgz#2301c5ffbf12b467de8da2333a459e29e7920e4b" - integrity sha512-r3vXyErRCYJ7wg28yvBY5VSoAF8ZvlcW9/BwUzEtUsjvX/DKs24dIkuwjtuprwJJHsbyUbLApepYTR1BN4uHrg== - -yaml@^1.10.0: +yaml@1.10.2, yaml@^1.10.0, yaml@^1.10.3: version "1.10.3" resolved "https://registry.yarnpkg.com/yaml/-/yaml-1.10.3.tgz#76e407ed95c42684fb8e14641e5de62fe65bbcb3" integrity sha512-vIYeF1u3CjlhAFekPPAk2h/Kv4T3mAkMox5OymRiJQB0spDP10LHvt+K7G9Ny6NuuMAb25/6n1qyUjAcGNf/AA== diff --git a/js-packages/web-console/src/lib/services/manager/index.ts b/js-packages/web-console/src/lib/services/manager/index.ts index b24294d9e3f..0ceaae35fa7 100644 --- a/js-packages/web-console/src/lib/services/manager/index.ts +++ b/js-packages/web-console/src/lib/services/manager/index.ts @@ -69,6 +69,8 @@ export type { Auth, AuthProvider, BootstrapPolicy, + BufferCacheAllocationStrategy, + BufferCacheStrategy, BuildInformation, CalciteId, CalcitePlan, @@ -81,6 +83,8 @@ export type { CheckpointPipelineResponses, CheckpointResponse, CheckpointStatus, + CheckpointSyncFailure, + CheckpointSyncStatus, Chunk, ClientOptions, ClockConfig, @@ -142,6 +146,7 @@ export type { DeltaTableWriteMode, DeltaTableWriterConfig, Demo, + DevTweaks, DisplaySchedule, ErrorResponse, ExtendedClusterMonitorEvent, @@ -325,6 +330,8 @@ export type { ListPipelinesErrors, ListPipelinesResponse, ListPipelinesResponses, + MemoryPressure, + MergerType, MetricsFormat, MetricsParameters, MirInput, diff --git a/js-packages/web-console/src/lib/services/manager/types.gen.ts b/js-packages/web-console/src/lib/services/manager/types.gen.ts index 5b63c800846..f07e7a5d5a9 100644 --- a/js-packages/web-console/src/lib/services/manager/types.gen.ts +++ b/js-packages/web-console/src/lib/services/manager/types.gen.ts @@ -80,6 +80,16 @@ export type AuthProvider = export type BootstrapPolicy = 'allow' | 'reject' | 'await_approval' +/** + * Controls how caches are shared across a foreground/background worker pair. + */ +export type BufferCacheAllocationStrategy = 'shared_per_worker_pair' | 'per_thread' | 'global' + +/** + * Selects which eviction strategy backs a cache instance. + */ +export type BufferCacheStrategy = 's3_fifo' | 'lru' + /** * Information about the build of the platform. */ @@ -212,6 +222,35 @@ export type CheckpointStatus = { success?: number | null } +/** + * Information about a failed checkpoint sync. + */ +export type CheckpointSyncFailure = { + /** + * Error message associated with the failure. + */ + error: string + /** + * UUID of the failed checkpoint. + */ + uuid: string +} + +/** + * Checkpoint status returned by the `/checkpoint/sync_status` endpoint. + */ +export type CheckpointSyncStatus = { + failure?: CheckpointSyncFailure | null + /** + * Most recently successful automated periodic checkpoint sync. + */ + periodic?: string | null + /** + * Most recently successful checkpoint sync. + */ + success?: string | null +} + /** * A set of updates to a SQL table or view. * @@ -1195,6 +1234,155 @@ export type Demo = { udf_toml: string } +/** + * Optional settings for tweaking Feldera internals. + */ +export type DevTweaks = { + /** + * Enable adaptive joins. + * + * Adaptive joins dynamically change their partitioning policy to avoid skew. + * + * Adaptive joins are disabled by default. + */ + adaptive_joins?: boolean | null + /** + * Factor that discourages the use of the Balance policy in a perfectly balanced collection. + * + * Assuming a perfectly balanced key distribution, the Balance policy is slightly less efficient than Shard, + * since it requires computing the hash of the entire key/value pair. This factor discourages the use of this policy + * if the skew is `