From 4b3563d47e827e8781a00257d29e0484ce2a0819 Mon Sep 17 00:00:00 2001 From: zz_y Date: Sat, 21 Feb 2026 15:23:09 -0700 Subject: [PATCH 01/38] Add standalone precompute engine to replace Arroyo streaming pipeline Implements a single-node multi-threaded precompute engine as a new module and binary target within QueryEngineRust. The engine ingests Prometheus remote write samples, buffers them per-series with out-of-order handling, detects closed tumbling/sliding windows via event-time watermarks, feeds samples into accumulator wrappers for all existing sketch types, and emits PrecomputedOutput directly to the store. New modules: config, series_buffer, window_manager, accumulator_factory, series_router, worker, output_sink, and the PrecomputeEngine orchestrator. The binary supports embedded store + query HTTP server for single-process deployment. Co-Authored-By: Claude Opus 4.6 --- asap-query-engine/Cargo.toml | 4 + .../src/bin/precompute_engine.rs | 138 ++++ asap-query-engine/src/lib.rs | 1 + .../precompute_engine/accumulator_factory.rs | 661 ++++++++++++++++++ .../src/precompute_engine/config.rs | 48 ++ .../src/precompute_engine/mod.rs | 177 +++++ .../src/precompute_engine/output_sink.rs | 83 +++ .../src/precompute_engine/series_buffer.rs | 161 +++++ .../src/precompute_engine/series_router.rs | 103 +++ .../src/precompute_engine/window_manager.rs | 154 ++++ .../src/precompute_engine/worker.rs | 472 +++++++++++++ 11 files changed, 2002 insertions(+) create mode 100644 asap-query-engine/src/bin/precompute_engine.rs create mode 100644 asap-query-engine/src/precompute_engine/accumulator_factory.rs create mode 100644 asap-query-engine/src/precompute_engine/config.rs create mode 100644 asap-query-engine/src/precompute_engine/mod.rs create mode 100644 asap-query-engine/src/precompute_engine/output_sink.rs create mode 100644 asap-query-engine/src/precompute_engine/series_buffer.rs create mode 100644 asap-query-engine/src/precompute_engine/series_router.rs create mode 100644 asap-query-engine/src/precompute_engine/window_manager.rs create mode 100644 asap-query-engine/src/precompute_engine/worker.rs diff --git a/asap-query-engine/Cargo.toml b/asap-query-engine/Cargo.toml index 11484c6..5686815 100644 --- a/asap-query-engine/Cargo.toml +++ b/asap-query-engine/Cargo.toml @@ -55,6 +55,10 @@ zstd = "0.13" reqwest = { version = "0.11", features = ["json"] } tracing-appender = "0.2" +[[bin]] +name = "precompute_engine" +path = "src/bin/precompute_engine.rs" + [dev-dependencies] tempfile = "3.20.0" diff --git a/asap-query-engine/src/bin/precompute_engine.rs b/asap-query-engine/src/bin/precompute_engine.rs new file mode 100644 index 0000000..70b90f3 --- /dev/null +++ b/asap-query-engine/src/bin/precompute_engine.rs @@ -0,0 +1,138 @@ +use clap::Parser; +use query_engine_rust::data_model::{ + CleanupPolicy, InferenceConfig, LockStrategy, StreamingConfig, +}; +use query_engine_rust::drivers::query::adapters::AdapterConfig; +use query_engine_rust::engines::SimpleEngine; +use query_engine_rust::precompute_engine::config::PrecomputeEngineConfig; +use query_engine_rust::precompute_engine::output_sink::StoreOutputSink; +use query_engine_rust::precompute_engine::PrecomputeEngine; +use query_engine_rust::stores::SimpleMapStore; +use query_engine_rust::{HttpServer, HttpServerConfig}; +use sketch_db_common::enums::QueryLanguage; +use std::sync::Arc; +use tracing::info; + +#[derive(Parser, Debug)] +#[command(name = "precompute_engine")] +#[command(about = "Standalone precompute engine for SketchDB")] +struct Args { + /// Path to streaming config YAML file + #[arg(long)] + streaming_config: String, + + /// Port for Prometheus remote write ingest + #[arg(long, default_value_t = 9090)] + ingest_port: u16, + + /// Number of worker threads + #[arg(long, default_value_t = 4)] + num_workers: usize, + + /// Maximum allowed lateness for out-of-order samples (ms) + #[arg(long, default_value_t = 5000)] + allowed_lateness_ms: i64, + + /// Maximum buffered samples per series + #[arg(long, default_value_t = 10000)] + max_buffer_per_series: usize, + + /// Flush interval for idle window detection (ms) + #[arg(long, default_value_t = 1000)] + flush_interval_ms: u64, + + /// MPSC channel buffer size per worker + #[arg(long, default_value_t = 10000)] + channel_buffer_size: usize, + + /// Port for the query HTTP server (0 to disable) + #[arg(long, default_value_t = 8080)] + query_port: u16, + + /// Lock strategy for the store + #[arg(long, value_enum, default_value_t = LockStrategy::PerKey)] + lock_strategy: LockStrategy, +} + +#[tokio::main] +async fn main() -> Result<(), Box> { + // Initialize tracing + tracing_subscriber::fmt() + .with_env_filter( + tracing_subscriber::EnvFilter::try_from_default_env() + .unwrap_or_else(|_| tracing_subscriber::EnvFilter::new("info")), + ) + .init(); + + let args = Args::parse(); + + info!("Loading streaming config from: {}", args.streaming_config); + let streaming_config = Arc::new(StreamingConfig::from_yaml_file(&args.streaming_config)?); + + info!( + "Loaded {} aggregation configs", + streaming_config.get_all_aggregation_configs().len() + ); + + // Create the store + let store: Arc = Arc::new( + SimpleMapStore::new_with_strategy( + streaming_config.clone(), + CleanupPolicy::CircularBuffer, + args.lock_strategy, + ), + ); + + // Optionally start the query HTTP server + if args.query_port > 0 { + let inference_config = InferenceConfig::new( + QueryLanguage::promql, + CleanupPolicy::CircularBuffer, + ); + let query_engine = Arc::new(SimpleEngine::new( + store.clone(), + inference_config, + streaming_config.clone(), + 15, // default prometheus scrape interval + QueryLanguage::promql, + )); + let http_config = HttpServerConfig { + port: args.query_port, + handle_http_requests: true, + adapter_config: AdapterConfig { + protocol: query_engine_rust::data_model::QueryProtocol::PrometheusHttp, + language: QueryLanguage::promql, + fallback: None, + }, + }; + let http_server = HttpServer::new(http_config, query_engine, store.clone()); + tokio::spawn(async move { + if let Err(e) = http_server.run().await { + tracing::error!("Query server error: {}", e); + } + }); + info!("Query server started on port {}", args.query_port); + } + + // Build the precompute engine config + let engine_config = PrecomputeEngineConfig { + num_workers: args.num_workers, + ingest_port: args.ingest_port, + allowed_lateness_ms: args.allowed_lateness_ms, + max_buffer_per_series: args.max_buffer_per_series, + flush_interval_ms: args.flush_interval_ms, + channel_buffer_size: args.channel_buffer_size, + }; + + // Create the output sink (writes directly to the store) + let output_sink: Arc = + Arc::new(StoreOutputSink::new(store)); + + // Build and run the engine + let engine = PrecomputeEngine::new(engine_config, streaming_config, output_sink); + + info!("Starting precompute engine..."); + engine.run().await?; + + Ok(()) +} diff --git a/asap-query-engine/src/lib.rs b/asap-query-engine/src/lib.rs index 47893c1..ada350d 100644 --- a/asap-query-engine/src/lib.rs +++ b/asap-query-engine/src/lib.rs @@ -1,6 +1,7 @@ pub mod data_model; pub mod drivers; pub mod engines; +pub mod precompute_engine; pub mod precompute_operators; pub mod stores; diff --git a/asap-query-engine/src/precompute_engine/accumulator_factory.rs b/asap-query-engine/src/precompute_engine/accumulator_factory.rs new file mode 100644 index 0000000..cd44e37 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/accumulator_factory.rs @@ -0,0 +1,661 @@ +use crate::data_model::{AggregateCore, KeyByLabelValues, Measurement}; +use crate::precompute_operators::{ + CountMinSketchAccumulator, DatasketchesKLLAccumulator, HydraKllSketchAccumulator, + IncreaseAccumulator, MinMaxAccumulator, MultipleIncreaseAccumulator, + MultipleMinMaxAccumulator, MultipleSumAccumulator, SumAccumulator, +}; +use sketch_db_common::aggregation_config::AggregationConfig; + +/// Trait for feeding samples into accumulators in the precompute engine. +/// +/// This provides a uniform interface over all accumulator types so that the +/// worker loop doesn't need to know which concrete type it's dealing with. +pub trait AccumulatorUpdater: Send { + /// Feed a single (value, timestamp_ms) pair — for SingleSubpopulation types. + fn update_single(&mut self, value: f64, timestamp_ms: i64); + + /// Feed a keyed (key, value, timestamp_ms) triple — for MultipleSubpopulation types. + fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, timestamp_ms: i64); + + /// Extract the final accumulator as a boxed `AggregateCore`. + fn take_accumulator(&mut self) -> Box; + + /// Reset internal state for reuse (avoids re-allocation). + fn reset(&mut self); + + /// Whether this updater is keyed (MultipleSubpopulation). + fn is_keyed(&self) -> bool; + + /// Estimated memory usage in bytes. + fn memory_usage_bytes(&self) -> usize; +} + +// --------------------------------------------------------------------------- +// SumAccumulatorUpdater +// --------------------------------------------------------------------------- + +pub struct SumAccumulatorUpdater { + acc: SumAccumulator, +} + +impl SumAccumulatorUpdater { + pub fn new() -> Self { + Self { + acc: SumAccumulator::new(), + } + } +} + +impl AccumulatorUpdater for SumAccumulatorUpdater { + fn update_single(&mut self, value: f64, _timestamp_ms: i64) { + self.acc.update(value); + } + + fn update_keyed(&mut self, _key: &KeyByLabelValues, value: f64, timestamp_ms: i64) { + self.update_single(value, timestamp_ms); + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = SumAccumulator::new(); + } + + fn is_keyed(&self) -> bool { + false + } + + fn memory_usage_bytes(&self) -> usize { + std::mem::size_of::() + } +} + +// --------------------------------------------------------------------------- +// MinMaxAccumulatorUpdater +// --------------------------------------------------------------------------- + +pub struct MinMaxAccumulatorUpdater { + acc: MinMaxAccumulator, + sub_type: String, +} + +impl MinMaxAccumulatorUpdater { + pub fn new(sub_type: String) -> Self { + Self { + acc: MinMaxAccumulator::new(sub_type.clone()), + sub_type, + } + } +} + +impl AccumulatorUpdater for MinMaxAccumulatorUpdater { + fn update_single(&mut self, value: f64, _timestamp_ms: i64) { + self.acc.update(value); + } + + fn update_keyed(&mut self, _key: &KeyByLabelValues, value: f64, timestamp_ms: i64) { + self.update_single(value, timestamp_ms); + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = MinMaxAccumulator::new(self.sub_type.clone()); + } + + fn is_keyed(&self) -> bool { + false + } + + fn memory_usage_bytes(&self) -> usize { + std::mem::size_of::() + } +} + +// --------------------------------------------------------------------------- +// IncreaseAccumulatorUpdater +// --------------------------------------------------------------------------- + +pub struct IncreaseAccumulatorUpdater { + acc: Option, +} + +impl IncreaseAccumulatorUpdater { + pub fn new() -> Self { + Self { acc: None } + } +} + +impl AccumulatorUpdater for IncreaseAccumulatorUpdater { + fn update_single(&mut self, value: f64, timestamp_ms: i64) { + let measurement = Measurement::new(value); + match &mut self.acc { + Some(acc) => acc.update(measurement, timestamp_ms), + None => { + self.acc = Some(IncreaseAccumulator::new( + measurement.clone(), + timestamp_ms, + measurement, + timestamp_ms, + )); + } + } + } + + fn update_keyed(&mut self, _key: &KeyByLabelValues, value: f64, timestamp_ms: i64) { + self.update_single(value, timestamp_ms); + } + + fn take_accumulator(&mut self) -> Box { + let acc = self.acc.take().unwrap_or_else(|| { + IncreaseAccumulator::new( + Measurement::new(0.0), + 0, + Measurement::new(0.0), + 0, + ) + }); + let result = Box::new(acc); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = None; + } + + fn is_keyed(&self) -> bool { + false + } + + fn memory_usage_bytes(&self) -> usize { + std::mem::size_of::>() + } +} + +// --------------------------------------------------------------------------- +// KllAccumulatorUpdater +// --------------------------------------------------------------------------- + +pub struct KllAccumulatorUpdater { + acc: DatasketchesKLLAccumulator, +} + +impl KllAccumulatorUpdater { + pub fn new(k: u16) -> Self { + Self { + acc: DatasketchesKLLAccumulator::new(k), + } + } +} + +impl AccumulatorUpdater for KllAccumulatorUpdater { + fn update_single(&mut self, value: f64, _timestamp_ms: i64) { + self.acc._update(value); + } + + fn update_keyed(&mut self, _key: &KeyByLabelValues, value: f64, timestamp_ms: i64) { + self.update_single(value, timestamp_ms); + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + // Re-create with same k + self.acc = DatasketchesKLLAccumulator::new(200); // TODO: preserve k from config + } + + fn is_keyed(&self) -> bool { + false + } + + fn memory_usage_bytes(&self) -> usize { + // KLL sketch size is hard to estimate precisely; use a rough estimate + std::mem::size_of::() + 4096 + } +} + +// --------------------------------------------------------------------------- +// MultipleSumUpdater +// --------------------------------------------------------------------------- + +pub struct MultipleSumUpdater { + acc: MultipleSumAccumulator, +} + +impl MultipleSumUpdater { + pub fn new() -> Self { + Self { + acc: MultipleSumAccumulator::new(), + } + } +} + +impl AccumulatorUpdater for MultipleSumUpdater { + fn update_single(&mut self, _value: f64, _timestamp_ms: i64) { + // Multiple-subpopulation — use update_keyed instead + } + + fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, _timestamp_ms: i64) { + self.acc.update(key.clone(), value); + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = MultipleSumAccumulator::new(); + } + + fn is_keyed(&self) -> bool { + true + } + + fn memory_usage_bytes(&self) -> usize { + std::mem::size_of::() + + self.acc.sums.len() * (std::mem::size_of::() + 8) + } +} + +// --------------------------------------------------------------------------- +// MultipleMinMaxUpdater +// --------------------------------------------------------------------------- + +pub struct MultipleMinMaxUpdater { + acc: MultipleMinMaxAccumulator, + sub_type: String, +} + +impl MultipleMinMaxUpdater { + pub fn new(sub_type: String) -> Self { + Self { + acc: MultipleMinMaxAccumulator::new(sub_type.clone()), + sub_type, + } + } +} + +impl AccumulatorUpdater for MultipleMinMaxUpdater { + fn update_single(&mut self, _value: f64, _timestamp_ms: i64) { + // Multiple-subpopulation — use update_keyed instead + } + + fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, _timestamp_ms: i64) { + self.acc.update(key.clone(), value); + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = MultipleMinMaxAccumulator::new(self.sub_type.clone()); + } + + fn is_keyed(&self) -> bool { + true + } + + fn memory_usage_bytes(&self) -> usize { + std::mem::size_of::() + + self.acc.values.len() * (std::mem::size_of::() + 8) + } +} + +// --------------------------------------------------------------------------- +// MultipleIncreaseUpdater +// --------------------------------------------------------------------------- + +pub struct MultipleIncreaseUpdater { + acc: MultipleIncreaseAccumulator, +} + +impl MultipleIncreaseUpdater { + pub fn new() -> Self { + Self { + acc: MultipleIncreaseAccumulator::new(), + } + } +} + +impl AccumulatorUpdater for MultipleIncreaseUpdater { + fn update_single(&mut self, _value: f64, _timestamp_ms: i64) { + // Multiple-subpopulation — use update_keyed instead + } + + fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, timestamp_ms: i64) { + let measurement = Measurement::new(value); + // If key already exists, update it; otherwise create new + if self.acc.increases.contains_key(key) { + if let Some(existing) = self.acc.increases.get_mut(key) { + existing.update(measurement, timestamp_ms); + } + } else { + let new_acc = IncreaseAccumulator::new( + measurement.clone(), + timestamp_ms, + measurement, + timestamp_ms, + ); + self.acc.update(key.clone(), new_acc); + } + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = MultipleIncreaseAccumulator::new(); + } + + fn is_keyed(&self) -> bool { + true + } + + fn memory_usage_bytes(&self) -> usize { + std::mem::size_of::() + + self.acc.increases.len() + * (std::mem::size_of::() + + std::mem::size_of::()) + } +} + +// --------------------------------------------------------------------------- +// CmsAccumulatorUpdater (CountMinSketch) +// --------------------------------------------------------------------------- + +pub struct CmsAccumulatorUpdater { + acc: CountMinSketchAccumulator, + row_num: usize, + col_num: usize, +} + +impl CmsAccumulatorUpdater { + pub fn new(row_num: usize, col_num: usize) -> Self { + Self { + acc: CountMinSketchAccumulator::new(row_num, col_num), + row_num, + col_num, + } + } +} + +impl AccumulatorUpdater for CmsAccumulatorUpdater { + fn update_single(&mut self, _value: f64, _timestamp_ms: i64) { + // CMS is keyed — use update_keyed + } + + fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, _timestamp_ms: i64) { + // CMS._update is private, so we access the sketch directly + // We replicate the hash logic from CountMinSketchAccumulator._update + let key_json = key.serialize_to_json(); + let key_values: Vec = if let Some(obj) = key_json.as_object() { + obj.values() + .map(|v| v.as_str().unwrap_or("").to_string()) + .collect() + } else { + vec!["".to_string()] + }; + let key_str = key_values.join(";"); + let key_bytes = key_str.as_bytes(); + + for i in 0..self.acc.row_num { + let hash_value = xxhash_rust::xxh32::xxh32(key_bytes, i as u32); + let col_index = (hash_value as usize) % self.acc.col_num; + self.acc.sketch[i][col_index] += value; + } + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = CountMinSketchAccumulator::new(self.row_num, self.col_num); + } + + fn is_keyed(&self) -> bool { + true + } + + fn memory_usage_bytes(&self) -> usize { + std::mem::size_of::() + + self.row_num * self.col_num * std::mem::size_of::() + } +} + +// --------------------------------------------------------------------------- +// HydraKllAccumulatorUpdater +// --------------------------------------------------------------------------- + +pub struct HydraKllAccumulatorUpdater { + acc: HydraKllSketchAccumulator, + row_num: usize, + col_num: usize, + k: u16, +} + +impl HydraKllAccumulatorUpdater { + pub fn new(row_num: usize, col_num: usize, k: u16) -> Self { + Self { + acc: HydraKllSketchAccumulator::new(row_num, col_num, k), + row_num, + col_num, + k, + } + } +} + +impl AccumulatorUpdater for HydraKllAccumulatorUpdater { + fn update_single(&mut self, _value: f64, _timestamp_ms: i64) { + // HydraKLL is keyed — use update_keyed + } + + fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, _timestamp_ms: i64) { + self.acc.update(key, value); + } + + fn take_accumulator(&mut self) -> Box { + let result = Box::new(self.acc.clone()); + self.reset(); + result + } + + fn reset(&mut self) { + self.acc = HydraKllSketchAccumulator::new(self.row_num, self.col_num, self.k); + } + + fn is_keyed(&self) -> bool { + true + } + + fn memory_usage_bytes(&self) -> usize { + // Rough estimate: each cell is a KLL sketch + std::mem::size_of::() + + self.row_num * self.col_num * 4096 + } +} + +// --------------------------------------------------------------------------- +// Factory function +// --------------------------------------------------------------------------- + +/// Create an appropriate `AccumulatorUpdater` from an `AggregationConfig`. +pub fn create_accumulator_updater( + config: &AggregationConfig, +) -> Box { + let agg_type = config.aggregation_type.as_str(); + let sub_type = config.aggregation_sub_type.as_str(); + + match agg_type { + "SingleSubpopulation" => match sub_type { + "Sum" | "sum" => Box::new(SumAccumulatorUpdater::new()), + "Min" | "min" => Box::new(MinMaxAccumulatorUpdater::new("min".to_string())), + "Max" | "max" => Box::new(MinMaxAccumulatorUpdater::new("max".to_string())), + "Increase" | "increase" => Box::new(IncreaseAccumulatorUpdater::new()), + "DatasketchesKLL" | "datasketches_kll" | "KLL" | "kll" => { + let k = config + .parameters + .get("k") + .and_then(|v| v.as_u64()) + .unwrap_or(200) as u16; + Box::new(KllAccumulatorUpdater::new(k)) + } + other => { + tracing::warn!( + "Unknown SingleSubpopulation sub_type '{}', defaulting to Sum", + other + ); + Box::new(SumAccumulatorUpdater::new()) + } + }, + "MultipleSubpopulation" => match sub_type { + "Sum" | "sum" => Box::new(MultipleSumUpdater::new()), + "Min" | "min" => Box::new(MultipleMinMaxUpdater::new("min".to_string())), + "Max" | "max" => Box::new(MultipleMinMaxUpdater::new("max".to_string())), + "Increase" | "increase" => Box::new(MultipleIncreaseUpdater::new()), + "CountMinSketch" | "count_min_sketch" | "CMS" | "cms" => { + let row_num = config + .parameters + .get("row_num") + .and_then(|v| v.as_u64()) + .unwrap_or(4) as usize; + let col_num = config + .parameters + .get("col_num") + .and_then(|v| v.as_u64()) + .unwrap_or(1000) as usize; + Box::new(CmsAccumulatorUpdater::new(row_num, col_num)) + } + "HydraKLL" | "hydra_kll" => { + let row_num = config + .parameters + .get("row_num") + .and_then(|v| v.as_u64()) + .unwrap_or(4) as usize; + let col_num = config + .parameters + .get("col_num") + .and_then(|v| v.as_u64()) + .unwrap_or(1000) as usize; + let k = config + .parameters + .get("k") + .and_then(|v| v.as_u64()) + .unwrap_or(200) as u16; + Box::new(HydraKllAccumulatorUpdater::new(row_num, col_num, k)) + } + other => { + tracing::warn!( + "Unknown MultipleSubpopulation sub_type '{}', defaulting to Sum", + other + ); + Box::new(MultipleSumUpdater::new()) + } + }, + other => { + tracing::warn!( + "Unknown aggregation_type '{}', defaulting to SingleSubpopulation Sum", + other + ); + Box::new(SumAccumulatorUpdater::new()) + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_sum_updater() { + let mut updater = SumAccumulatorUpdater::new(); + assert!(!updater.is_keyed()); + + updater.update_single(1.0, 1000); + updater.update_single(2.0, 2000); + updater.update_single(3.0, 3000); + + let acc = updater.take_accumulator(); + assert_eq!(acc.type_name(), "SumAccumulator"); + } + + #[test] + fn test_minmax_updater() { + let mut updater = MinMaxAccumulatorUpdater::new("max".to_string()); + updater.update_single(5.0, 1000); + updater.update_single(3.0, 2000); + updater.update_single(7.0, 3000); + + let acc = updater.take_accumulator(); + assert_eq!(acc.type_name(), "MinMaxAccumulator"); + } + + #[test] + fn test_increase_updater() { + let mut updater = IncreaseAccumulatorUpdater::new(); + updater.update_single(10.0, 1000); + updater.update_single(15.0, 2000); + + let acc = updater.take_accumulator(); + assert_eq!(acc.type_name(), "IncreaseAccumulator"); + } + + #[test] + fn test_kll_updater() { + let mut updater = KllAccumulatorUpdater::new(200); + for i in 1..=10 { + updater.update_single(i as f64, i * 1000); + } + + let acc = updater.take_accumulator(); + assert_eq!(acc.type_name(), "DatasketchesKLLAccumulator"); + } + + #[test] + fn test_multiple_sum_updater() { + let mut updater = MultipleSumUpdater::new(); + assert!(updater.is_keyed()); + + let key_a = KeyByLabelValues::new_with_labels(vec!["a".to_string()]); + let key_b = KeyByLabelValues::new_with_labels(vec!["b".to_string()]); + + updater.update_keyed(&key_a, 1.0, 1000); + updater.update_keyed(&key_b, 2.0, 2000); + + let acc = updater.take_accumulator(); + assert_eq!(acc.type_name(), "MultipleSumAccumulator"); + } + + #[test] + fn test_reset_clears_state() { + let mut updater = SumAccumulatorUpdater::new(); + updater.update_single(100.0, 1000); + updater.reset(); + // After reset, should produce a fresh accumulator + let acc = updater.take_accumulator(); + assert_eq!(acc.type_name(), "SumAccumulator"); + } +} diff --git a/asap-query-engine/src/precompute_engine/config.rs b/asap-query-engine/src/precompute_engine/config.rs new file mode 100644 index 0000000..ba02a71 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/config.rs @@ -0,0 +1,48 @@ +use serde::{Deserialize, Serialize}; + +/// Configuration for the precompute engine. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct PrecomputeEngineConfig { + /// Number of worker threads for parallel processing. + pub num_workers: usize, + /// Port for the Prometheus remote write ingest endpoint. + pub ingest_port: u16, + /// Maximum allowed lateness for out-of-order samples (milliseconds). + /// Samples arriving later than this behind the watermark are dropped. + pub allowed_lateness_ms: i64, + /// Maximum number of buffered samples per series before oldest are evicted. + pub max_buffer_per_series: usize, + /// Interval at which the flush timer fires to close idle windows (milliseconds). + pub flush_interval_ms: u64, + /// Capacity of the MPSC channel between router and each worker. + pub channel_buffer_size: usize, +} + +impl Default for PrecomputeEngineConfig { + fn default() -> Self { + Self { + num_workers: 4, + ingest_port: 9090, + allowed_lateness_ms: 5_000, + max_buffer_per_series: 10_000, + flush_interval_ms: 1_000, + channel_buffer_size: 10_000, + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_default_config() { + let config = PrecomputeEngineConfig::default(); + assert_eq!(config.num_workers, 4); + assert_eq!(config.ingest_port, 9090); + assert_eq!(config.allowed_lateness_ms, 5_000); + assert_eq!(config.max_buffer_per_series, 10_000); + assert_eq!(config.flush_interval_ms, 1_000); + assert_eq!(config.channel_buffer_size, 10_000); + } +} diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs new file mode 100644 index 0000000..71536e2 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -0,0 +1,177 @@ +pub mod accumulator_factory; +pub mod config; +pub mod output_sink; +pub mod series_buffer; +pub mod series_router; +pub mod window_manager; +pub mod worker; + +use crate::data_model::StreamingConfig; +use crate::drivers::ingest::prometheus_remote_write::decode_prometheus_remote_write; +use crate::precompute_engine::config::PrecomputeEngineConfig; +use crate::precompute_engine::output_sink::OutputSink; +use crate::precompute_engine::series_router::{SeriesRouter, WorkerMessage}; +use crate::precompute_engine::worker::Worker; +use axum::{body::Bytes, extract::State, http::StatusCode, routing::post, Router}; +use std::collections::HashMap; +use std::sync::Arc; +use tokio::net::TcpListener; +use tokio::sync::mpsc; +use tracing::{info, warn}; + +/// Shared state for the ingest HTTP handler. +struct IngestState { + router: SeriesRouter, + samples_ingested: std::sync::atomic::AtomicU64, +} + +/// The top-level precompute engine orchestrator. +/// +/// Creates worker threads, the series router, and the Axum ingest server. +pub struct PrecomputeEngine { + config: PrecomputeEngineConfig, + streaming_config: Arc, + output_sink: Arc, +} + +impl PrecomputeEngine { + pub fn new( + config: PrecomputeEngineConfig, + streaming_config: Arc, + output_sink: Arc, + ) -> Self { + Self { + config, + streaming_config, + output_sink, + } + } + + /// Start the precompute engine. This spawns worker tasks and the HTTP + /// ingest server, then blocks until shutdown. + pub async fn run(self) -> Result<(), Box> { + let num_workers = self.config.num_workers; + let channel_size = self.config.channel_buffer_size; + + // Build MPSC channels for each worker + let mut senders = Vec::with_capacity(num_workers); + let mut receivers = Vec::with_capacity(num_workers); + for _ in 0..num_workers { + let (tx, rx) = mpsc::channel::(channel_size); + senders.push(tx); + receivers.push(rx); + } + + // Build the router + let router = SeriesRouter::new(senders); + + // Build aggregation config map from streaming config + let agg_configs: HashMap = self + .streaming_config + .get_all_aggregation_configs() + .clone(); + + // Spawn workers + let mut worker_handles = Vec::with_capacity(num_workers); + for (id, rx) in receivers.into_iter().enumerate() { + let worker = Worker::new( + id, + rx, + self.output_sink.clone(), + agg_configs.clone(), + self.config.max_buffer_per_series, + self.config.allowed_lateness_ms, + ); + let handle = tokio::spawn(async move { + worker.run().await; + }); + worker_handles.push(handle); + } + + info!( + "PrecomputeEngine started with {} workers on port {}", + num_workers, self.config.ingest_port + ); + + // Build the ingest state + let ingest_state = Arc::new(IngestState { + router, + samples_ingested: std::sync::atomic::AtomicU64::new(0), + }); + + // Start flush timer + let flush_state = ingest_state.clone(); + let flush_interval_ms = self.config.flush_interval_ms; + tokio::spawn(async move { + let mut interval = + tokio::time::interval(tokio::time::Duration::from_millis(flush_interval_ms)); + loop { + interval.tick().await; + if let Err(e) = flush_state.router.broadcast_flush().await { + warn!("Flush broadcast error: {}", e); + break; + } + } + }); + + // Start the Axum HTTP server for Prometheus remote write ingest + let app = Router::new() + .route("/api/v1/write", post(handle_ingest)) + .with_state(ingest_state); + + let addr = format!("0.0.0.0:{}", self.config.ingest_port); + info!("Ingest server listening on {}", addr); + + let listener = TcpListener::bind(&addr).await?; + axum::serve(listener, app).await?; + + // Wait for workers to finish (this only happens on shutdown) + for handle in worker_handles { + let _ = handle.await; + } + + Ok(()) + } +} + +/// Axum handler for Prometheus remote write. +async fn handle_ingest( + State(state): State>, + body: Bytes, +) -> StatusCode { + let samples = match decode_prometheus_remote_write(&body) { + Ok(s) => s, + Err(e) => { + warn!("Failed to decode remote write: {}", e); + return StatusCode::BAD_REQUEST; + } + }; + + if samples.is_empty() { + return StatusCode::NO_CONTENT; + } + + let count = samples.len() as u64; + state + .samples_ingested + .fetch_add(count, std::sync::atomic::Ordering::Relaxed); + + // Group samples by series key for batch routing + let mut by_series: HashMap<&str, Vec<(i64, f64)>> = HashMap::new(); + for s in &samples { + by_series + .entry(&s.labels) + .or_default() + .push((s.timestamp_ms, s.value)); + } + + // Route each series batch to the correct worker + for (series_key, batch) in by_series { + if let Err(e) = state.router.route(series_key, batch).await { + warn!("Routing error for {}: {}", series_key, e); + return StatusCode::INTERNAL_SERVER_ERROR; + } + } + + StatusCode::NO_CONTENT +} diff --git a/asap-query-engine/src/precompute_engine/output_sink.rs b/asap-query-engine/src/precompute_engine/output_sink.rs new file mode 100644 index 0000000..445cb49 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/output_sink.rs @@ -0,0 +1,83 @@ +use crate::data_model::{AggregateCore, PrecomputedOutput}; +use crate::stores::Store; +use std::sync::Arc; + +/// Trait for emitting completed window outputs. +pub trait OutputSink: Send + Sync { + fn emit_batch( + &self, + outputs: Vec<(PrecomputedOutput, Box)>, + ) -> Result<(), Box>; +} + +/// Output sink that writes directly to a `Store`. +pub struct StoreOutputSink { + store: Arc, +} + +impl StoreOutputSink { + pub fn new(store: Arc) -> Self { + Self { store } + } +} + +impl OutputSink for StoreOutputSink { + fn emit_batch( + &self, + outputs: Vec<(PrecomputedOutput, Box)>, + ) -> Result<(), Box> { + if outputs.is_empty() { + return Ok(()); + } + self.store.insert_precomputed_output_batch(outputs) + } +} + +/// Output sink for raw passthrough mode — forwards raw samples to the store +/// without sketch computation. In this mode the samples are stored as +/// SumAccumulators (one per sample). +pub struct RawPassthroughSink { + store: Arc, +} + +impl RawPassthroughSink { + pub fn new(store: Arc) -> Self { + Self { store } + } +} + +impl OutputSink for RawPassthroughSink { + fn emit_batch( + &self, + outputs: Vec<(PrecomputedOutput, Box)>, + ) -> Result<(), Box> { + if outputs.is_empty() { + return Ok(()); + } + self.store.insert_precomputed_output_batch(outputs) + } +} + +/// A no-op sink for testing that just counts emitted batches. +pub struct NoopOutputSink { + pub emit_count: std::sync::atomic::AtomicU64, +} + +impl NoopOutputSink { + pub fn new() -> Self { + Self { + emit_count: std::sync::atomic::AtomicU64::new(0), + } + } +} + +impl OutputSink for NoopOutputSink { + fn emit_batch( + &self, + outputs: Vec<(PrecomputedOutput, Box)>, + ) -> Result<(), Box> { + self.emit_count + .fetch_add(outputs.len() as u64, std::sync::atomic::Ordering::Relaxed); + Ok(()) + } +} diff --git a/asap-query-engine/src/precompute_engine/series_buffer.rs b/asap-query-engine/src/precompute_engine/series_buffer.rs new file mode 100644 index 0000000..a663142 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/series_buffer.rs @@ -0,0 +1,161 @@ +use std::collections::BTreeMap; + +/// Per-series sample buffer backed by a `BTreeMap` for automatic +/// ordering by timestamp. Tracks a per-series watermark. +pub struct SeriesBuffer { + /// Samples keyed by timestamp_ms. BTreeMap keeps them sorted. + samples: BTreeMap, + /// High-watermark: the maximum timestamp seen so far for this series. + watermark_ms: i64, + /// Maximum number of samples to retain. When exceeded, oldest are evicted. + max_buffer_size: usize, +} + +impl SeriesBuffer { + pub fn new(max_buffer_size: usize) -> Self { + Self { + samples: BTreeMap::new(), + watermark_ms: i64::MIN, + max_buffer_size, + } + } + + /// Insert a sample. Updates the watermark if `timestamp_ms` is the new max. + /// Returns `true` if the sample was actually inserted (not a duplicate timestamp + /// with the same value). + pub fn insert(&mut self, timestamp_ms: i64, value: f64) -> bool { + if timestamp_ms > self.watermark_ms { + self.watermark_ms = timestamp_ms; + } + self.samples.insert(timestamp_ms, value); + + // Enforce max buffer size by evicting oldest entries + while self.samples.len() > self.max_buffer_size { + self.samples.pop_first(); + } + + true + } + + /// Return current watermark. + pub fn watermark_ms(&self) -> i64 { + self.watermark_ms + } + + /// Read all samples in `[start_ms, end_ms)` — inclusive start, exclusive end. + /// Returns them in timestamp order. + pub fn read_range(&self, start_ms: i64, end_ms: i64) -> Vec<(i64, f64)> { + self.samples + .range(start_ms..end_ms) + .map(|(&ts, &val)| (ts, val)) + .collect() + } + + /// Drain (remove and return) all samples with `timestamp_ms < up_to_ms`. + pub fn drain_up_to(&mut self, up_to_ms: i64) -> Vec<(i64, f64)> { + let mut drained = Vec::new(); + // split_off returns everything >= up_to_ms; we keep that part + let remaining = self.samples.split_off(&up_to_ms); + // self.samples now contains everything < up_to_ms + drained.extend(self.samples.iter().map(|(&ts, &val)| (ts, val))); + self.samples = remaining; + drained + } + + /// Number of buffered samples. + pub fn len(&self) -> usize { + self.samples.len() + } + + /// Whether the buffer is empty. + pub fn is_empty(&self) -> bool { + self.samples.is_empty() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_insert_and_watermark() { + let mut buf = SeriesBuffer::new(100); + assert_eq!(buf.watermark_ms(), i64::MIN); + + buf.insert(1000, 1.0); + assert_eq!(buf.watermark_ms(), 1000); + + buf.insert(500, 0.5); // out-of-order + assert_eq!(buf.watermark_ms(), 1000); // watermark should not go back + + buf.insert(2000, 2.0); + assert_eq!(buf.watermark_ms(), 2000); + } + + #[test] + fn test_sorted_order() { + let mut buf = SeriesBuffer::new(100); + buf.insert(3000, 3.0); + buf.insert(1000, 1.0); + buf.insert(2000, 2.0); + + let all = buf.read_range(0, 4000); + assert_eq!(all, vec![(1000, 1.0), (2000, 2.0), (3000, 3.0)]); + } + + #[test] + fn test_read_range() { + let mut buf = SeriesBuffer::new(100); + for t in [1000, 2000, 3000, 4000, 5000] { + buf.insert(t, t as f64); + } + + // [2000, 4000) should return 2000, 3000 + let range = buf.read_range(2000, 4000); + assert_eq!(range, vec![(2000, 2000.0), (3000, 3000.0)]); + } + + #[test] + fn test_drain_up_to() { + let mut buf = SeriesBuffer::new(100); + for t in [1000, 2000, 3000, 4000, 5000] { + buf.insert(t, t as f64); + } + + let drained = buf.drain_up_to(3000); + assert_eq!(drained, vec![(1000, 1000.0), (2000, 2000.0)]); + assert_eq!(buf.len(), 3); // 3000, 4000, 5000 remain + } + + #[test] + fn test_max_buffer_enforcement() { + let mut buf = SeriesBuffer::new(3); + buf.insert(1000, 1.0); + buf.insert(2000, 2.0); + buf.insert(3000, 3.0); + buf.insert(4000, 4.0); // should evict 1000 + assert_eq!(buf.len(), 3); + + let all = buf.read_range(0, 5000); + assert_eq!(all, vec![(2000, 2.0), (3000, 3.0), (4000, 4.0)]); + } + + #[test] + fn test_dedup_by_timestamp() { + let mut buf = SeriesBuffer::new(100); + buf.insert(1000, 1.0); + buf.insert(1000, 2.0); // same timestamp, overwrites + assert_eq!(buf.len(), 1); + + let all = buf.read_range(0, 2000); + assert_eq!(all, vec![(1000, 2.0)]); + } + + #[test] + fn test_empty_operations() { + let buf = SeriesBuffer::new(100); + assert!(buf.is_empty()); + assert_eq!(buf.len(), 0); + assert_eq!(buf.read_range(0, 1000), vec![]); + } +} diff --git a/asap-query-engine/src/precompute_engine/series_router.rs b/asap-query-engine/src/precompute_engine/series_router.rs new file mode 100644 index 0000000..fd05817 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/series_router.rs @@ -0,0 +1,103 @@ +use tokio::sync::mpsc; +use xxhash_rust::xxh64::xxh64; + +/// A message sent from the router to a worker. +#[derive(Debug)] +pub enum WorkerMessage { + /// A batch of samples for the same series. + Samples { + series_key: String, + samples: Vec<(i64, f64)>, // (timestamp_ms, value) + }, + /// Signal the worker to flush/check idle windows. + Flush, + /// Graceful shutdown. + Shutdown, +} + +/// Routes incoming samples to one of N workers based on a consistent hash +/// of the series label string. +pub struct SeriesRouter { + senders: Vec>, + num_workers: usize, +} + +impl SeriesRouter { + pub fn new(senders: Vec>) -> Self { + let num_workers = senders.len(); + Self { + senders, + num_workers, + } + } + + /// Route a batch of samples for one series to the appropriate worker. + pub async fn route( + &self, + series_key: &str, + samples: Vec<(i64, f64)>, + ) -> Result<(), Box> { + let worker_idx = self.worker_for(series_key); + self.senders[worker_idx] + .send(WorkerMessage::Samples { + series_key: series_key.to_string(), + samples, + }) + .await + .map_err(|e| format!("Failed to send to worker {}: {}", worker_idx, e))?; + Ok(()) + } + + /// Broadcast a flush signal to all workers. + pub async fn broadcast_flush(&self) -> Result<(), Box> { + for (i, sender) in self.senders.iter().enumerate() { + sender + .send(WorkerMessage::Flush) + .await + .map_err(|e| format!("Failed to send flush to worker {}: {}", i, e))?; + } + Ok(()) + } + + /// Broadcast shutdown to all workers. + pub async fn broadcast_shutdown(&self) -> Result<(), Box> { + for (i, sender) in self.senders.iter().enumerate() { + sender + .send(WorkerMessage::Shutdown) + .await + .map_err(|e| format!("Failed to send shutdown to worker {}: {}", i, e))?; + } + Ok(()) + } + + /// Determine which worker handles a given series key. + fn worker_for(&self, series_key: &str) -> usize { + let hash = xxh64(series_key.as_bytes(), 0); + (hash as usize) % self.num_workers + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_consistent_routing() { + // Build a router with dummy senders (we only test the hash logic) + let (senders, _receivers): (Vec<_>, Vec<_>) = (0..4) + .map(|_| mpsc::channel::(10)) + .unzip(); + + let router = SeriesRouter::new(senders); + + // Same key should always go to the same worker + let w1 = router.worker_for("cpu{host=\"a\"}"); + let w2 = router.worker_for("cpu{host=\"a\"}"); + assert_eq!(w1, w2); + + // Different keys may go to different workers (probabilistic, but verifiable) + let _ = router.worker_for("cpu{host=\"b\"}"); + // Just ensure no panic and result is in range + assert!(router.worker_for("mem{host=\"a\"}") < 4); + } +} diff --git a/asap-query-engine/src/precompute_engine/window_manager.rs b/asap-query-engine/src/precompute_engine/window_manager.rs new file mode 100644 index 0000000..929f928 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/window_manager.rs @@ -0,0 +1,154 @@ +/// Manages tumbling window boundaries and detects which windows have closed +/// based on watermark advancement. +pub struct WindowManager { + /// Window size in milliseconds. + window_size_ms: i64, + /// Slide interval in milliseconds (== window_size_ms for tumbling windows). + slide_interval_ms: i64, +} + +impl WindowManager { + /// Create a new WindowManager. + /// + /// `window_size_secs` and `slide_interval_secs` come from `AggregationConfig` + /// (which stores them in seconds). They are converted to milliseconds internally. + pub fn new(window_size_secs: u64, slide_interval_secs: u64) -> Self { + let window_size_ms = (window_size_secs * 1000) as i64; + let slide_interval_ms = if slide_interval_secs == 0 { + window_size_ms // tumbling window + } else { + (slide_interval_secs * 1000) as i64 + }; + Self { + window_size_ms, + slide_interval_ms, + } + } + + pub fn window_size_ms(&self) -> i64 { + self.window_size_ms + } + + /// Compute the window start for a given timestamp. + /// Windows are aligned to epoch (multiples of slide_interval_ms). + pub fn window_start_for(&self, timestamp_ms: i64) -> i64 { + // Floor-divide to the nearest slide interval boundary + let n = timestamp_ms.div_euclid(self.slide_interval_ms); + n * self.slide_interval_ms + } + + /// Return window starts whose windows are now closed, given that the + /// watermark advanced from `previous_wm` to `current_wm`. + /// + /// A window `[start, start + window_size_ms)` is closed when + /// `current_wm >= start + window_size_ms`. + /// + /// Returns window starts in ascending order. + pub fn closed_windows(&self, previous_wm: i64, current_wm: i64) -> Vec { + if current_wm <= previous_wm || previous_wm == i64::MIN { + // No watermark advancement, or first sample ever (nothing to close yet + // — the window that contains the first sample is still open). + return Vec::new(); + } + + let mut closed = Vec::new(); + + // The earliest window start that *could* have been open at previous_wm. + // A window is open if its end (start + window_size_ms) > previous_wm. + // So the oldest open window start was: previous_wm - window_size_ms + 1, + // aligned down to slide_interval. + let earliest_open_start = self.window_start_for( + (previous_wm - self.window_size_ms + 1).max(0), + ); + + let mut start = earliest_open_start; + while start + self.window_size_ms <= current_wm { + // This window was NOT closed at previous_wm but IS closed at current_wm + if start + self.window_size_ms > previous_wm { + closed.push(start); + } + start += self.slide_interval_ms; + } + + closed + } + + /// Return the window `[start, end)` boundaries for a given window start. + pub fn window_bounds(&self, window_start: i64) -> (i64, i64) { + (window_start, window_start + self.window_size_ms) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_tumbling_window_start() { + // 60-second (60000ms) tumbling windows + let wm = WindowManager::new(60, 0); + + assert_eq!(wm.window_start_for(0), 0); + assert_eq!(wm.window_start_for(59_999), 0); + assert_eq!(wm.window_start_for(60_000), 60_000); + assert_eq!(wm.window_start_for(119_999), 60_000); + assert_eq!(wm.window_start_for(120_000), 120_000); + } + + #[test] + fn test_no_closed_windows_on_first_sample() { + let wm = WindowManager::new(60, 0); + let closed = wm.closed_windows(i64::MIN, 30_000); + assert!(closed.is_empty()); + } + + #[test] + fn test_tumbling_window_close() { + // 60s tumbling windows + let wm = WindowManager::new(60, 0); + + // Watermark advances from 30_000 to 70_000 + // Window [0, 60_000) closes when wm >= 60_000 + let closed = wm.closed_windows(30_000, 70_000); + assert_eq!(closed, vec![0]); + } + + #[test] + fn test_multiple_window_closes() { + // 10s (10000ms) tumbling windows + let wm = WindowManager::new(10, 0); + + // Watermark jumps from 5_000 to 35_000 — closes windows 0, 10_000, 20_000 + let closed = wm.closed_windows(5_000, 35_000); + assert_eq!(closed, vec![0, 10_000, 20_000]); + } + + #[test] + fn test_no_close_when_watermark_stagnant() { + let wm = WindowManager::new(60, 0); + let closed = wm.closed_windows(30_000, 30_000); + assert!(closed.is_empty()); + } + + #[test] + fn test_window_bounds() { + let wm = WindowManager::new(60, 0); + assert_eq!(wm.window_bounds(0), (0, 60_000)); + assert_eq!(wm.window_bounds(60_000), (60_000, 120_000)); + } + + #[test] + fn test_sliding_window() { + // 30s window, 10s slide + let wm = WindowManager::new(30, 10); + + assert_eq!(wm.window_start_for(0), 0); + assert_eq!(wm.window_start_for(9_999), 0); + assert_eq!(wm.window_start_for(10_000), 10_000); + + // Watermark advances from 15_000 to 35_000 + // Window [0, 30_000) closes at wm=30_000 (was open at 15_000) + let closed = wm.closed_windows(15_000, 35_000); + assert_eq!(closed, vec![0]); + } +} diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs new file mode 100644 index 0000000..08951f7 --- /dev/null +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -0,0 +1,472 @@ +use crate::data_model::{AggregateCore, KeyByLabelValues, PrecomputedOutput}; +use crate::precompute_engine::accumulator_factory::{create_accumulator_updater, AccumulatorUpdater}; +use crate::precompute_engine::output_sink::OutputSink; +use crate::precompute_engine::series_buffer::SeriesBuffer; +use crate::precompute_engine::series_router::WorkerMessage; +use crate::precompute_engine::window_manager::WindowManager; +use sketch_db_common::aggregation_config::AggregationConfig; +use std::collections::HashMap; +use std::sync::Arc; +use tokio::sync::mpsc; +use tracing::{debug, info, warn}; + +/// Per-aggregation state within a series: the window manager and active +/// window accumulators. +struct AggregationState { + config: AggregationConfig, + window_manager: WindowManager, + /// Active windows keyed by window_start_ms. + active_windows: HashMap>, +} + +/// Per-series state owned by the worker. +struct SeriesState { + buffer: SeriesBuffer, + previous_watermark_ms: i64, + /// One AggregationState per matching aggregation config. + aggregations: Vec, +} + +/// Worker that processes samples for a shard of the series space. +pub struct Worker { + id: usize, + receiver: mpsc::Receiver, + output_sink: Arc, + /// Map from series key to per-series state. + series_map: HashMap, + /// Aggregation configs, keyed by aggregation_id. + agg_configs: HashMap, + /// Max buffer size per series. + max_buffer_per_series: usize, + /// Allowed lateness in ms. + allowed_lateness_ms: i64, +} + +impl Worker { + pub fn new( + id: usize, + receiver: mpsc::Receiver, + output_sink: Arc, + agg_configs: HashMap, + max_buffer_per_series: usize, + allowed_lateness_ms: i64, + ) -> Self { + Self { + id, + receiver, + output_sink, + series_map: HashMap::new(), + agg_configs, + max_buffer_per_series, + allowed_lateness_ms, + } + } + + /// Run the worker loop. Blocks until shutdown. + pub async fn run(mut self) { + info!("Worker {} started", self.id); + + while let Some(msg) = self.receiver.recv().await { + match msg { + WorkerMessage::Samples { + series_key, + samples, + } => { + if let Err(e) = self.process_samples(&series_key, samples) { + warn!("Worker {} error processing {}: {}", self.id, series_key, e); + } + } + WorkerMessage::Flush => { + if let Err(e) = self.flush_all() { + warn!("Worker {} flush error: {}", self.id, e); + } + } + WorkerMessage::Shutdown => { + info!("Worker {} shutting down", self.id); + // Final flush before shutdown + if let Err(e) = self.flush_all() { + warn!("Worker {} final flush error: {}", self.id, e); + } + break; + } + } + } + + info!( + "Worker {} stopped, {} active series", + self.id, + self.series_map.len() + ); + } + + /// Find all aggregation configs whose metric/spatial_filter matches this series. + fn matching_agg_configs(&self, series_key: &str) -> Vec<(u64, &AggregationConfig)> { + let metric_name = extract_metric_name(series_key); + + self.agg_configs + .iter() + .filter(|(_, config)| { + // Match on metric name + config.metric == metric_name + || config.spatial_filter_normalized == metric_name + || config.spatial_filter == metric_name + }) + .map(|(&id, config)| (id, config)) + .collect() + } + + /// Get or create the SeriesState for a series key. + fn get_or_create_series_state(&mut self, series_key: &str) -> &mut SeriesState { + if !self.series_map.contains_key(series_key) { + let matching = self.matching_agg_configs(series_key); + let aggregations = matching + .into_iter() + .map(|(_, config)| AggregationState { + window_manager: WindowManager::new( + config.window_size, + config.slide_interval, + ), + config: config.clone(), + active_windows: HashMap::new(), + }) + .collect(); + + self.series_map.insert( + series_key.to_string(), + SeriesState { + buffer: SeriesBuffer::new(self.max_buffer_per_series), + previous_watermark_ms: i64::MIN, + aggregations, + }, + ); + } + + self.series_map.get_mut(series_key).unwrap() + } + + fn process_samples( + &mut self, + series_key: &str, + samples: Vec<(i64, f64)>, + ) -> Result<(), Box> { + // Copy scalars out of self before taking &mut self.series_map + let worker_id = self.id; + let allowed_lateness_ms = self.allowed_lateness_ms; + + // Ensure state exists + self.get_or_create_series_state(series_key); + + let state = self.series_map.get_mut(series_key).unwrap(); + + if state.aggregations.is_empty() { + return Ok(()); + } + + // Insert samples into buffer, dropping late arrivals + for &(ts, val) in &samples { + if state.buffer.watermark_ms() != i64::MIN + && ts < state.buffer.watermark_ms() - allowed_lateness_ms + { + debug!( + "Worker {} dropping late sample for {}: ts={} watermark={}", + worker_id, + series_key, + ts, + state.buffer.watermark_ms() + ); + continue; + } + state.buffer.insert(ts, val); + } + + let current_wm = state.buffer.watermark_ms(); + let previous_wm = state.previous_watermark_ms; + + let mut emit_batch: Vec<(PrecomputedOutput, Box)> = Vec::new(); + + for agg_state in &mut state.aggregations { + let closed = agg_state + .window_manager + .closed_windows(previous_wm, current_wm); + + // Feed each incoming sample to the correct active window accumulator + for &(ts, val) in &samples { + if current_wm != i64::MIN && ts < current_wm - allowed_lateness_ms { + continue; // already dropped + } + + let window_start = agg_state.window_manager.window_start_for(ts); + + let updater = agg_state + .active_windows + .entry(window_start) + .or_insert_with(|| create_accumulator_updater(&agg_state.config)); + + if updater.is_keyed() { + let key = extract_key_from_series(series_key, &agg_state.config); + updater.update_keyed(&key, val, ts); + } else { + updater.update_single(val, ts); + } + } + + // Emit closed windows + for window_start in &closed { + if let Some(mut updater) = agg_state.active_windows.remove(window_start) { + let (_, window_end) = + agg_state.window_manager.window_bounds(*window_start); + + let key = if updater.is_keyed() { + Some(extract_key_from_series(series_key, &agg_state.config)) + } else { + None + }; + + let output = PrecomputedOutput::new( + *window_start as u64, + window_end as u64, + key, + agg_state.config.aggregation_id, + ); + + let accumulator = updater.take_accumulator(); + emit_batch.push((output, accumulator)); + } + } + } + + state.previous_watermark_ms = current_wm; + + // Emit to output sink + if !emit_batch.is_empty() { + debug!( + "Worker {} emitting {} outputs for {}", + worker_id, + emit_batch.len(), + series_key + ); + self.output_sink.emit_batch(emit_batch)?; + } + + Ok(()) + } + + /// Flush all series — force-close windows that are past due. + fn flush_all(&mut self) -> Result<(), Box> { + let mut emit_batch: Vec<(PrecomputedOutput, Box)> = Vec::new(); + + for (series_key, state) in &mut self.series_map { + let current_wm = state.buffer.watermark_ms(); + let previous_wm = state.previous_watermark_ms; + + for agg_state in &mut state.aggregations { + let closed = agg_state + .window_manager + .closed_windows(previous_wm, current_wm); + + for window_start in &closed { + if let Some(mut updater) = agg_state.active_windows.remove(window_start) { + let (_, window_end) = + agg_state.window_manager.window_bounds(*window_start); + + let key = if updater.is_keyed() { + Some(extract_key_from_series( + series_key, + &agg_state.config, + )) + } else { + None + }; + + let output = PrecomputedOutput::new( + *window_start as u64, + window_end as u64, + key, + agg_state.config.aggregation_id, + ); + + let accumulator = updater.take_accumulator(); + emit_batch.push((output, accumulator)); + } + } + } + + state.previous_watermark_ms = current_wm; + } + + if !emit_batch.is_empty() { + debug!( + "Worker {} flush emitting {} outputs", + self.id, + emit_batch.len() + ); + self.output_sink.emit_batch(emit_batch)?; + } + + Ok(()) + } +} + +/// Extract the metric name from a series key like `"metric_name{key1=\"val1\"}"`. +pub fn extract_metric_name(series_key: &str) -> &str { + match series_key.find('{') { + Some(pos) => &series_key[..pos], + None => series_key, + } +} + +/// Extract grouping label values from a series key string based on the +/// aggregation config's `grouping_labels`. +/// +/// The series key format is: `metric_name{label1="val1",label2="val2",...}` +pub fn extract_key_from_series( + series_key: &str, + config: &AggregationConfig, +) -> KeyByLabelValues { + let labels = parse_labels_from_series_key(series_key); + let mut values = Vec::new(); + + for label_name in &config.grouping_labels.labels { + if let Some(val) = labels.get(label_name.as_str()) { + values.push(val.to_string()); + } else { + values.push(String::new()); + } + } + + KeyByLabelValues::new_with_labels(values) +} + +/// Parse label key-value pairs from a series key string. +/// `"metric{a=\"b\",c=\"d\"}"` → `{("a", "b"), ("c", "d")}` +fn parse_labels_from_series_key(series_key: &str) -> HashMap<&str, &str> { + let mut labels = HashMap::new(); + + let start = match series_key.find('{') { + Some(pos) => pos + 1, + None => return labels, + }; + let end = match series_key.rfind('}') { + Some(pos) => pos, + None => return labels, + }; + + if start >= end { + return labels; + } + + let label_str = &series_key[start..end]; + + // Parse comma-separated key="value" pairs + // Simple parser that handles the expected format + let mut remaining = label_str; + while !remaining.is_empty() { + // Find the '=' separator + let eq_pos = match remaining.find('=') { + Some(pos) => pos, + None => break, + }; + let key = remaining[..eq_pos].trim(); + + // Expect "value" after = + let after_eq = &remaining[eq_pos + 1..]; + if !after_eq.starts_with('"') { + break; + } + + // Find closing quote + let value_start = 1; // skip opening quote + let value_end = match after_eq[value_start..].find('"') { + Some(pos) => value_start + pos, + None => break, + }; + + let value = &after_eq[value_start..value_end]; + labels.insert(key, value); + + // Move past the closing quote and optional comma + let consumed = value_end + 1; // past closing quote + remaining = &after_eq[consumed..]; + if remaining.starts_with(',') { + remaining = &remaining[1..]; + } + } + + labels +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_extract_metric_name() { + assert_eq!( + extract_metric_name("http_requests_total{method=\"GET\"}"), + "http_requests_total" + ); + assert_eq!(extract_metric_name("up"), "up"); + assert_eq!( + extract_metric_name("cpu_usage{host=\"a\",zone=\"us\"}"), + "cpu_usage" + ); + } + + #[test] + fn test_parse_labels() { + let labels = + parse_labels_from_series_key("metric{method=\"GET\",status=\"200\"}"); + assert_eq!(labels.get("method"), Some(&"GET")); + assert_eq!(labels.get("status"), Some(&"200")); + } + + #[test] + fn test_parse_labels_no_labels() { + let labels = parse_labels_from_series_key("metric"); + assert!(labels.is_empty()); + } + + #[test] + fn test_parse_labels_empty_braces() { + let labels = parse_labels_from_series_key("metric{}"); + assert!(labels.is_empty()); + } + + #[test] + fn test_extract_key_from_series() { + use serde_json::json; + + let config = AggregationConfig::new( + 1, + "SingleSubpopulation".to_string(), + "Sum".to_string(), + HashMap::new(), + promql_utilities::data_model::key_by_label_names::KeyByLabelNames::new(vec![ + "method".to_string(), + "status".to_string(), + ]), + promql_utilities::data_model::key_by_label_names::KeyByLabelNames::new(vec![]), + promql_utilities::data_model::key_by_label_names::KeyByLabelNames::new(vec![]), + String::new(), + 60, + "http_requests_total".to_string(), + "http_requests_total".to_string(), + None, + None, + Some(60), + Some(0), + None, + None, + None, + ); + + let key = extract_key_from_series( + "http_requests_total{method=\"GET\",status=\"200\"}", + &config, + ); + assert_eq!( + key.labels, + vec!["GET".to_string(), "200".to_string()] + ); + } +} From 898f1dd031d5232395501228936d1f433a2c55e5 Mon Sep 17 00:00:00 2001 From: zz_y Date: Mon, 23 Feb 2026 11:23:02 -0700 Subject: [PATCH 02/38] Restore standalone precompute engine binary with fixed imports Replace sketch_db_common::enums::QueryLanguage import with query_engine_rust::data_model::QueryLanguage in the standalone binary, and integrate PrecomputeEngine into main.rs as an alternative to the removed PrometheusRemoteWriteServer. Co-Authored-By: Claude Opus 4.6 --- .../src/bin/precompute_engine.rs | 2 +- asap-query-engine/src/lib.rs | 4 + asap-query-engine/src/main.rs | 85 ++++++++++++------- 3 files changed, 60 insertions(+), 31 deletions(-) diff --git a/asap-query-engine/src/bin/precompute_engine.rs b/asap-query-engine/src/bin/precompute_engine.rs index 70b90f3..c8e8fd8 100644 --- a/asap-query-engine/src/bin/precompute_engine.rs +++ b/asap-query-engine/src/bin/precompute_engine.rs @@ -9,7 +9,7 @@ use query_engine_rust::precompute_engine::output_sink::StoreOutputSink; use query_engine_rust::precompute_engine::PrecomputeEngine; use query_engine_rust::stores::SimpleMapStore; use query_engine_rust::{HttpServer, HttpServerConfig}; -use sketch_db_common::enums::QueryLanguage; +use query_engine_rust::data_model::QueryLanguage; use std::sync::Arc; use tracing::info; diff --git a/asap-query-engine/src/lib.rs b/asap-query-engine/src/lib.rs index ada350d..81f9824 100644 --- a/asap-query-engine/src/lib.rs +++ b/asap-query-engine/src/lib.rs @@ -27,6 +27,10 @@ pub use engines::{InstantVector, QueryResult, SimpleEngine}; pub use drivers::{HttpServer, HttpServerConfig, KafkaConsumer, KafkaConsumerConfig}; +pub use precompute_engine::config::PrecomputeEngineConfig; +pub use precompute_engine::output_sink::StoreOutputSink; +pub use precompute_engine::PrecomputeEngine; + pub use utils::{normalize_spatial_filter, read_inference_config, read_streaming_config}; pub type Result = std::result::Result>; diff --git a/asap-query-engine/src/main.rs b/asap-query-engine/src/main.rs index 0cc5d95..9f67964 100644 --- a/asap-query-engine/src/main.rs +++ b/asap-query-engine/src/main.rs @@ -9,8 +9,8 @@ use query_engine_rust::data_model::enums::{InputFormat, LockStrategy, StreamingE use query_engine_rust::drivers::AdapterConfig; use query_engine_rust::utils::file_io::{read_inference_config, read_streaming_config}; use query_engine_rust::{ - HttpServer, HttpServerConfig, KafkaConsumer, KafkaConsumerConfig, Result, SimpleEngine, - SimpleMapStore, + HttpServer, HttpServerConfig, KafkaConsumer, KafkaConsumerConfig, PrecomputeEngine, + PrecomputeEngineConfig, Result, SimpleEngine, SimpleMapStore, StoreOutputSink, }; #[derive(Parser, Debug)] @@ -100,13 +100,29 @@ struct Args { #[arg(long, default_value = "9090")] prometheus_remote_write_port: u16, - /// Automatically initialize all sketch types for newly seen series during ingestion - #[arg(long, default_value = "true")] - auto_init_sketches: bool, - /// Path to promsketch configuration YAML file (optional; uses defaults if omitted) #[arg(long)] promsketch_config: Option, + + /// Number of precompute engine worker threads + #[arg(long, default_value = "4")] + precompute_num_workers: usize, + + /// Maximum allowed lateness for out-of-order samples (milliseconds) + #[arg(long, default_value = "5000")] + precompute_allowed_lateness_ms: i64, + + /// Maximum buffered samples per series before eviction + #[arg(long, default_value = "10000")] + precompute_max_buffer_per_series: usize, + + /// Interval at which the flush timer fires (milliseconds) + #[arg(long, default_value = "1000")] + precompute_flush_interval_ms: u64, + + /// Capacity of the channel between router and each worker + #[arg(long, default_value = "10000")] + precompute_channel_buffer_size: usize, } #[tokio::main] @@ -219,25 +235,34 @@ async fn main() -> Result<()> { } }; - // Setup Prometheus remote write server - // let prometheus_remote_write_handle = if args.enable_prometheus_remote_write { - // let prw_config = PrometheusRemoteWriteConfig { - // port: args.prometheus_remote_write_port, - // auto_init_sketches: args.auto_init_sketches, - // }; - // let server = PrometheusRemoteWriteServer::new(prw_config, promsketch_store.clone()); - // info!( - // "Starting Prometheus remote write server on port {}", - // args.prometheus_remote_write_port - // ); - // Some(tokio::spawn(async move { - // if let Err(e) = server.run().await { - // error!("Prometheus remote write server error: {}", e); - // } - // })) - // } else { - // None - // }; + // Setup precompute engine (replaces standalone Prometheus remote write server) + let precompute_handle = if args.enable_prometheus_remote_write { + let precompute_config = PrecomputeEngineConfig { + num_workers: args.precompute_num_workers, + ingest_port: args.prometheus_remote_write_port, + allowed_lateness_ms: args.precompute_allowed_lateness_ms, + max_buffer_per_series: args.precompute_max_buffer_per_series, + flush_interval_ms: args.precompute_flush_interval_ms, + channel_buffer_size: args.precompute_channel_buffer_size, + }; + let output_sink = Arc::new(StoreOutputSink::new(store.clone())); + let engine = PrecomputeEngine::new( + precompute_config, + streaming_config.clone(), + output_sink, + ); + info!( + "Starting precompute engine on port {}", + args.prometheus_remote_write_port + ); + Some(tokio::spawn(async move { + if let Err(e) = engine.run().await { + error!("Precompute engine error: {}", e); + } + })) + } else { + None + }; //info!("=== TEMPORARY: Using ClickHouse HTTP adapter ==="); //info!("ClickHouse endpoint will be available at: /clickhouse/query"); @@ -283,11 +308,11 @@ async fn main() -> Result<()> { let _ = handle.await; } - // if let Some(handle) = prometheus_remote_write_handle { - // info!("Shutting down Prometheus remote write server..."); - // handle.abort(); - // let _ = handle.await; - // } + if let Some(handle) = precompute_handle { + info!("Shutting down precompute engine..."); + handle.abort(); + let _ = handle.await; + } info!("Shutdown complete"); Ok(()) From c7cd8af333c3b45188888977c02011e62161245b Mon Sep 17 00:00:00 2001 From: zz_y Date: Mon, 23 Feb 2026 12:14:53 -0700 Subject: [PATCH 03/38] Wire up DatasketchesKLL in accumulator factory and add E2E test Handle top-level aggregation types (DatasketchesKLL, Sum, Min, Max, etc.) directly in the factory match, fixing the fallback to Sum that broke quantile queries. Also preserve the K parameter in KllAccumulatorUpdater::reset() instead of hardcoding 200. Add test_e2e_precompute binary that validates the full ingest -> precompute -> store -> query pipeline end-to-end. Co-Authored-By: Claude Opus 4.6 --- asap-query-engine/Cargo.toml | 4 + .../src/bin/test_e2e_precompute.rs | 262 ++++++++++++++++++ .../precompute_engine/accumulator_factory.rs | 51 +++- 3 files changed, 315 insertions(+), 2 deletions(-) create mode 100644 asap-query-engine/src/bin/test_e2e_precompute.rs diff --git a/asap-query-engine/Cargo.toml b/asap-query-engine/Cargo.toml index 5686815..37c4de2 100644 --- a/asap-query-engine/Cargo.toml +++ b/asap-query-engine/Cargo.toml @@ -59,6 +59,10 @@ tracing-appender = "0.2" name = "precompute_engine" path = "src/bin/precompute_engine.rs" +[[bin]] +name = "test_e2e_precompute" +path = "src/bin/test_e2e_precompute.rs" + [dev-dependencies] tempfile = "3.20.0" diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs new file mode 100644 index 0000000..c4370cb --- /dev/null +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -0,0 +1,262 @@ +//! End-to-end test for the standalone precompute_engine binary. +//! +//! This binary: +//! 1. Starts a PrecomputeEngine in-process (same as the precompute_engine binary) +//! 2. Sends Prometheus remote write samples via HTTP +//! 3. Queries the PromQL endpoint and prints results +//! +//! Usage: +//! cargo run --bin test_e2e_precompute + +use prost::Message; +use query_engine_rust::data_model::{LockStrategy, QueryLanguage}; +use query_engine_rust::drivers::ingest::prometheus_remote_write::{ + Label, Sample, TimeSeries, WriteRequest, +}; +use query_engine_rust::drivers::query::adapters::AdapterConfig; +use query_engine_rust::engines::SimpleEngine; +use query_engine_rust::precompute_engine::config::PrecomputeEngineConfig; +use query_engine_rust::precompute_engine::output_sink::StoreOutputSink; +use query_engine_rust::precompute_engine::PrecomputeEngine; +use query_engine_rust::stores::SimpleMapStore; +use query_engine_rust::utils::file_io::{read_inference_config, read_streaming_config}; +use query_engine_rust::{HttpServer, HttpServerConfig}; +use std::sync::Arc; + +const INGEST_PORT: u16 = 19090; +const QUERY_PORT: u16 = 18080; +const SCRAPE_INTERVAL: u64 = 1; // 1 second to match tumblingWindowSize + +fn build_remote_write_body(timeseries: Vec) -> Vec { + let write_req = WriteRequest { timeseries }; + let proto_bytes = write_req.encode_to_vec(); + snap::raw::Encoder::new() + .compress_vec(&proto_bytes) + .expect("snappy compress failed") +} + +fn make_sample(metric: &str, label_0: &str, timestamp_ms: i64, value: f64) -> TimeSeries { + TimeSeries { + labels: vec![ + Label { + name: "__name__".into(), + value: metric.into(), + }, + Label { + name: "instance".into(), + value: "i1".into(), + }, + Label { + name: "job".into(), + value: "test".into(), + }, + Label { + name: "label_0".into(), + value: label_0.into(), + }, + Label { + name: "label_1".into(), + value: "v1".into(), + }, + ], + samples: vec![Sample { + value, + timestamp: timestamp_ms, + }], + } +} + +#[tokio::main] +async fn main() -> Result<(), Box> { + tracing_subscriber::fmt() + .with_env_filter( + tracing_subscriber::EnvFilter::try_from_default_env() + .unwrap_or_else(|_| tracing_subscriber::EnvFilter::new("info")), + ) + .init(); + + // Load configs the same way main.rs does + let inference_config = + read_inference_config("examples/promql/inference_config.yaml", QueryLanguage::promql)?; + println!( + "Loaded inference config with {} query configs", + inference_config.query_configs.len() + ); + for qc in &inference_config.query_configs { + println!(" Query: '{}' -> {:?}", qc.query, qc.aggregations); + } + + let cleanup_policy = inference_config.cleanup_policy; + let streaming_config = Arc::new(read_streaming_config( + "examples/promql/streaming_config.yaml", + &inference_config, + )?); + println!( + "Loaded streaming config with {} aggregation configs", + streaming_config.get_all_aggregation_configs().len() + ); + + println!("\n=== Starting precompute engine (ingest={INGEST_PORT}, query={QUERY_PORT}) ==="); + + // Create store + let store: Arc = Arc::new( + SimpleMapStore::new_with_strategy( + streaming_config.clone(), + cleanup_policy, + LockStrategy::PerKey, + ), + ); + + // Start query server + let query_engine = Arc::new(SimpleEngine::new( + store.clone(), + inference_config, + streaming_config.clone(), + SCRAPE_INTERVAL, + QueryLanguage::promql, + )); + let http_config = HttpServerConfig { + port: QUERY_PORT, + handle_http_requests: true, + adapter_config: AdapterConfig { + protocol: query_engine_rust::data_model::QueryProtocol::PrometheusHttp, + language: QueryLanguage::promql, + fallback: None, + }, + }; + let http_server = HttpServer::new(http_config, query_engine, store.clone()); + tokio::spawn(async move { + if let Err(e) = http_server.run().await { + eprintln!("Query server error: {e}"); + } + }); + + // Start precompute engine + let engine_config = PrecomputeEngineConfig { + num_workers: 2, + ingest_port: INGEST_PORT, + allowed_lateness_ms: 5000, + max_buffer_per_series: 10000, + flush_interval_ms: 200, + channel_buffer_size: 10000, + }; + let output_sink = Arc::new(StoreOutputSink::new(store.clone())); + let engine = PrecomputeEngine::new(engine_config, streaming_config, output_sink); + tokio::spawn(async move { + if let Err(e) = engine.run().await { + eprintln!("Precompute engine error: {e}"); + } + }); + + // Wait for servers to bind + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + + let client = reqwest::Client::new(); + + // ----------------------------------------------------------------------- + // Send samples across multiple 1-second tumbling windows. + // tumblingWindowSize=1 means windows are [0,1000), [1000,2000), etc. + // We need enough windows of data so the query engine can find results. + // ----------------------------------------------------------------------- + println!("\n=== Sending remote write samples ==="); + + // Send 20 windows worth of data (timestamps 0ms..20000ms = 0s..20s) + // Each window gets one sample. + for window in 0..20 { + let ts = window * 1000 + 500; // mid-window + let val = 10.0 + window as f64; + let body = build_remote_write_body(vec![make_sample("fake_metric", "groupA", ts, val)]); + + let resp = client + .post(format!("http://localhost:{INGEST_PORT}/api/v1/write")) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(body) + .send() + .await?; + + println!(" Sent t={ts}ms v={val} -> HTTP {}", resp.status().as_u16()); + } + + // Advance watermark well past to close all windows + println!("\n=== Advancing watermark to close all windows ==="); + let body = build_remote_write_body(vec![make_sample("fake_metric", "groupA", 25000, 0.0)]); + let resp = client + .post(format!("http://localhost:{INGEST_PORT}/api/v1/write")) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(body) + .send() + .await?; + println!(" Sent t=25000ms v=0 -> HTTP {}", resp.status().as_u16()); + + // Wait for flush + processing + println!("\n Waiting for flush..."); + tokio::time::sleep(tokio::time::Duration::from_secs(2)).await; + + // ----------------------------------------------------------------------- + // Query the PromQL endpoint + // The inference_config has: "quantile by (label_0) (0.99, fake_metric)" + // which maps to aggregation_id 1. + // ----------------------------------------------------------------------- + println!("\n=== Querying PromQL endpoint ==="); + + // Use the exact query pattern from inference_config + let queries_instant = vec![ + ("quantile by (label_0) (0.99, fake_metric)", "10", "Configured query at t=10"), + ("quantile by (label_0) (0.99, fake_metric)", "15", "Configured query at t=15"), + ("sum_over_time(fake_metric[1s])", "10", "Temporal: sum_over_time at t=10"), + ("sum(fake_metric)", "10", "Spatial: sum at t=10"), + ]; + + for (query, time, label) in &queries_instant { + println!("\n--- Instant query: {label} ---"); + let resp = client + .get(format!("http://localhost:{QUERY_PORT}/api/v1/query")) + .query(&[("query", *query), ("time", *time)]) + .send() + .await? + .text() + .await?; + print_json(&resp); + } + + // Range query + println!("\n--- Range query: quantile by (label_0) (0.99, fake_metric) t=5..20 step=1 ---"); + let resp = client + .get(format!("http://localhost:{QUERY_PORT}/api/v1/query_range")) + .query(&[ + ("query", "quantile by (label_0) (0.99, fake_metric)"), + ("start", "5"), + ("end", "20"), + ("step", "1"), + ]) + .send() + .await? + .text() + .await?; + print_json(&resp); + + // Runtime info + println!("\n--- Runtime info ---"); + let resp = client + .get(format!( + "http://localhost:{QUERY_PORT}/api/v1/status/runtimeinfo" + )) + .send() + .await? + .text() + .await?; + print_json(&resp); + + println!("\n=== E2E test complete ==="); + + Ok(()) +} + +fn print_json(s: &str) { + match serde_json::from_str::(s) { + Ok(v) => println!("{}", serde_json::to_string_pretty(&v).unwrap()), + Err(_) => println!("{s}"), + } +} diff --git a/asap-query-engine/src/precompute_engine/accumulator_factory.rs b/asap-query-engine/src/precompute_engine/accumulator_factory.rs index cd44e37..47e66a4 100644 --- a/asap-query-engine/src/precompute_engine/accumulator_factory.rs +++ b/asap-query-engine/src/precompute_engine/accumulator_factory.rs @@ -187,12 +187,14 @@ impl AccumulatorUpdater for IncreaseAccumulatorUpdater { pub struct KllAccumulatorUpdater { acc: DatasketchesKLLAccumulator, + k: u16, } impl KllAccumulatorUpdater { pub fn new(k: u16) -> Self { Self { acc: DatasketchesKLLAccumulator::new(k), + k, } } } @@ -213,8 +215,7 @@ impl AccumulatorUpdater for KllAccumulatorUpdater { } fn reset(&mut self) { - // Re-create with same k - self.acc = DatasketchesKLLAccumulator::new(200); // TODO: preserve k from config + self.acc = DatasketchesKLLAccumulator::new(self.k); } fn is_keyed(&self) -> bool { @@ -575,6 +576,52 @@ pub fn create_accumulator_updater( Box::new(MultipleSumUpdater::new()) } }, + // Top-level aggregation types (e.g. "DatasketchesKLL" directly in aggregationType) + "DatasketchesKLL" | "datasketches_kll" | "KLL" | "kll" => { + let k = config + .parameters + .get("K") + .or_else(|| config.parameters.get("k")) + .and_then(|v| v.as_u64()) + .unwrap_or(200) as u16; + Box::new(KllAccumulatorUpdater::new(k)) + } + "Sum" | "sum" => Box::new(SumAccumulatorUpdater::new()), + "Min" | "min" => Box::new(MinMaxAccumulatorUpdater::new("min".to_string())), + "Max" | "max" => Box::new(MinMaxAccumulatorUpdater::new("max".to_string())), + "Increase" | "increase" => Box::new(IncreaseAccumulatorUpdater::new()), + "CountMinSketch" | "count_min_sketch" | "CMS" | "cms" => { + let row_num = config + .parameters + .get("row_num") + .and_then(|v| v.as_u64()) + .unwrap_or(4) as usize; + let col_num = config + .parameters + .get("col_num") + .and_then(|v| v.as_u64()) + .unwrap_or(1000) as usize; + Box::new(CmsAccumulatorUpdater::new(row_num, col_num)) + } + "HydraKLL" | "hydra_kll" => { + let row_num = config + .parameters + .get("row_num") + .and_then(|v| v.as_u64()) + .unwrap_or(4) as usize; + let col_num = config + .parameters + .get("col_num") + .and_then(|v| v.as_u64()) + .unwrap_or(1000) as usize; + let k = config + .parameters + .get("K") + .or_else(|| config.parameters.get("k")) + .and_then(|v| v.as_u64()) + .unwrap_or(200) as u16; + Box::new(HydraKllAccumulatorUpdater::new(row_num, col_num, k)) + } other => { tracing::warn!( "Unknown aggregation_type '{}', defaulting to SingleSubpopulation Sum", From 8804d0300909329e2a6f34f4052cada9086808f3 Mon Sep 17 00:00:00 2001 From: zz_y Date: Mon, 23 Feb 2026 13:59:41 -0700 Subject: [PATCH 04/38] Add raw-only mode to PrecomputeEngine that skips aggregation When pass_raw_samples is enabled, each incoming sample is emitted directly to the store as a SumAccumulator without windowing or watermark advancement. This supports use cases where raw passthrough is needed instead of sketch-based aggregation. Co-Authored-By: Claude Opus 4.6 --- .../src/bin/precompute_engine.rs | 18 ++++- .../src/bin/test_e2e_precompute.rs | 81 ++++++++++++++++++- asap-query-engine/src/main.rs | 2 + .../src/precompute_engine/config.rs | 9 +++ .../src/precompute_engine/mod.rs | 2 + .../src/precompute_engine/worker.rs | 50 ++++++++++++ 6 files changed, 158 insertions(+), 4 deletions(-) diff --git a/asap-query-engine/src/bin/precompute_engine.rs b/asap-query-engine/src/bin/precompute_engine.rs index c8e8fd8..ad91c3f 100644 --- a/asap-query-engine/src/bin/precompute_engine.rs +++ b/asap-query-engine/src/bin/precompute_engine.rs @@ -5,7 +5,7 @@ use query_engine_rust::data_model::{ use query_engine_rust::drivers::query::adapters::AdapterConfig; use query_engine_rust::engines::SimpleEngine; use query_engine_rust::precompute_engine::config::PrecomputeEngineConfig; -use query_engine_rust::precompute_engine::output_sink::StoreOutputSink; +use query_engine_rust::precompute_engine::output_sink::{RawPassthroughSink, StoreOutputSink}; use query_engine_rust::precompute_engine::PrecomputeEngine; use query_engine_rust::stores::SimpleMapStore; use query_engine_rust::{HttpServer, HttpServerConfig}; @@ -52,6 +52,14 @@ struct Args { /// Lock strategy for the store #[arg(long, value_enum, default_value_t = LockStrategy::PerKey)] lock_strategy: LockStrategy, + + /// Skip aggregation and pass each raw sample directly to the store + #[arg(long, default_value_t = false)] + pass_raw_samples: bool, + + /// Aggregation ID to stamp on each raw-mode output + #[arg(long, default_value_t = 0)] + raw_mode_aggregation_id: u64, } #[tokio::main] @@ -122,11 +130,17 @@ async fn main() -> Result<(), Box> { max_buffer_per_series: args.max_buffer_per_series, flush_interval_ms: args.flush_interval_ms, channel_buffer_size: args.channel_buffer_size, + pass_raw_samples: args.pass_raw_samples, + raw_mode_aggregation_id: args.raw_mode_aggregation_id, }; // Create the output sink (writes directly to the store) let output_sink: Arc = - Arc::new(StoreOutputSink::new(store)); + if args.pass_raw_samples { + Arc::new(RawPassthroughSink::new(store)) + } else { + Arc::new(StoreOutputSink::new(store)) + }; // Build and run the engine let engine = PrecomputeEngine::new(engine_config, streaming_config, output_sink); diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index c4370cb..dadabe4 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -16,7 +16,7 @@ use query_engine_rust::drivers::ingest::prometheus_remote_write::{ use query_engine_rust::drivers::query::adapters::AdapterConfig; use query_engine_rust::engines::SimpleEngine; use query_engine_rust::precompute_engine::config::PrecomputeEngineConfig; -use query_engine_rust::precompute_engine::output_sink::StoreOutputSink; +use query_engine_rust::precompute_engine::output_sink::{RawPassthroughSink, StoreOutputSink}; use query_engine_rust::precompute_engine::PrecomputeEngine; use query_engine_rust::stores::SimpleMapStore; use query_engine_rust::utils::file_io::{read_inference_config, read_streaming_config}; @@ -25,6 +25,7 @@ use std::sync::Arc; const INGEST_PORT: u16 = 19090; const QUERY_PORT: u16 = 18080; +const RAW_INGEST_PORT: u16 = 19091; const SCRAPE_INTERVAL: u64 = 1; // 1 second to match tumblingWindowSize fn build_remote_write_body(timeseries: Vec) -> Vec { @@ -139,9 +140,11 @@ async fn main() -> Result<(), Box> { max_buffer_per_series: 10000, flush_interval_ms: 200, channel_buffer_size: 10000, + pass_raw_samples: false, + raw_mode_aggregation_id: 0, }; let output_sink = Arc::new(StoreOutputSink::new(store.clone())); - let engine = PrecomputeEngine::new(engine_config, streaming_config, output_sink); + let engine = PrecomputeEngine::new(engine_config, streaming_config.clone(), output_sink); tokio::spawn(async move { if let Err(e) = engine.run().await { eprintln!("Precompute engine error: {e}"); @@ -249,6 +252,80 @@ async fn main() -> Result<(), Box> { .await?; print_json(&resp); + // ----------------------------------------------------------------------- + // RAW MODE TEST + // ----------------------------------------------------------------------- + println!("\n=== Starting raw-mode precompute engine (ingest={RAW_INGEST_PORT}) ==="); + + // The raw engine reuses the same store so we can query results directly. + // Pick aggregation_id = 1 to match the existing streaming config. + let raw_agg_id: u64 = 1; + let raw_engine_config = PrecomputeEngineConfig { + num_workers: 1, + ingest_port: RAW_INGEST_PORT, + allowed_lateness_ms: 5000, + max_buffer_per_series: 10000, + flush_interval_ms: 200, + channel_buffer_size: 10000, + pass_raw_samples: true, + raw_mode_aggregation_id: raw_agg_id, + }; + let raw_sink = Arc::new(RawPassthroughSink::new(store.clone())); + let raw_engine = PrecomputeEngine::new( + raw_engine_config, + streaming_config.clone(), + raw_sink, + ); + tokio::spawn(async move { + if let Err(e) = raw_engine.run().await { + eprintln!("Raw precompute engine error: {e}"); + } + }); + + // Wait for server to bind + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + + // Send a few raw samples — no need to advance watermark. + println!("\n=== Sending raw-mode samples ==="); + let raw_timestamps = vec![100_000i64, 101_000, 102_000]; + let raw_values = vec![42.0f64, 43.0, 44.0]; + for (&ts, &val) in raw_timestamps.iter().zip(raw_values.iter()) { + let body = build_remote_write_body(vec![make_sample("fake_metric", "groupA", ts, val)]); + let resp = client + .post(format!("http://localhost:{RAW_INGEST_PORT}/api/v1/write")) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(body) + .send() + .await?; + println!(" Sent raw t={ts}ms v={val} -> HTTP {}", resp.status().as_u16()); + } + + // Short wait for processing (no watermark advancement needed) + tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; + + // Verify raw samples appeared in the store + println!("\n=== Verifying raw samples in store ==="); + let results = store.query_precomputed_output( + "fake_metric", + raw_agg_id, + 100_000, + 103_000, + )?; + let total_buckets: usize = results.values().map(|v| v.len()).sum(); + println!(" Found {total_buckets} buckets for aggregation_id={raw_agg_id} in [100000, 103000)"); + assert!( + total_buckets >= 3, + "Expected at least 3 raw samples in store, got {total_buckets}" + ); + + for (key, buckets) in &results { + for ((start, end), _acc) in buckets { + println!(" key={key:?} start={start} end={end}"); + } + } + println!(" Raw mode test PASSED"); + println!("\n=== E2E test complete ==="); Ok(()) diff --git a/asap-query-engine/src/main.rs b/asap-query-engine/src/main.rs index 9f67964..f32cecb 100644 --- a/asap-query-engine/src/main.rs +++ b/asap-query-engine/src/main.rs @@ -244,6 +244,8 @@ async fn main() -> Result<()> { max_buffer_per_series: args.precompute_max_buffer_per_series, flush_interval_ms: args.precompute_flush_interval_ms, channel_buffer_size: args.precompute_channel_buffer_size, + pass_raw_samples: false, + raw_mode_aggregation_id: 0, }; let output_sink = Arc::new(StoreOutputSink::new(store.clone())); let engine = PrecomputeEngine::new( diff --git a/asap-query-engine/src/precompute_engine/config.rs b/asap-query-engine/src/precompute_engine/config.rs index ba02a71..7bc8df9 100644 --- a/asap-query-engine/src/precompute_engine/config.rs +++ b/asap-query-engine/src/precompute_engine/config.rs @@ -16,6 +16,11 @@ pub struct PrecomputeEngineConfig { pub flush_interval_ms: u64, /// Capacity of the MPSC channel between router and each worker. pub channel_buffer_size: usize, + /// When true, skip all aggregation and pass each raw sample directly to the + /// output sink as a `SumAccumulator::with_sum(value)`. + pub pass_raw_samples: bool, + /// Aggregation ID to stamp on each raw-mode output. + pub raw_mode_aggregation_id: u64, } impl Default for PrecomputeEngineConfig { @@ -27,6 +32,8 @@ impl Default for PrecomputeEngineConfig { max_buffer_per_series: 10_000, flush_interval_ms: 1_000, channel_buffer_size: 10_000, + pass_raw_samples: false, + raw_mode_aggregation_id: 0, } } } @@ -44,5 +51,7 @@ mod tests { assert_eq!(config.max_buffer_per_series, 10_000); assert_eq!(config.flush_interval_ms, 1_000); assert_eq!(config.channel_buffer_size, 10_000); + assert!(!config.pass_raw_samples); + assert_eq!(config.raw_mode_aggregation_id, 0); } } diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs index 71536e2..ad1a2a6 100644 --- a/asap-query-engine/src/precompute_engine/mod.rs +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -81,6 +81,8 @@ impl PrecomputeEngine { agg_configs.clone(), self.config.max_buffer_per_series, self.config.allowed_lateness_ms, + self.config.pass_raw_samples, + self.config.raw_mode_aggregation_id, ); let handle = tokio::spawn(async move { worker.run().await; diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs index 08951f7..347a679 100644 --- a/asap-query-engine/src/precompute_engine/worker.rs +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -4,6 +4,7 @@ use crate::precompute_engine::output_sink::OutputSink; use crate::precompute_engine::series_buffer::SeriesBuffer; use crate::precompute_engine::series_router::WorkerMessage; use crate::precompute_engine::window_manager::WindowManager; +use crate::precompute_operators::sum_accumulator::SumAccumulator; use sketch_db_common::aggregation_config::AggregationConfig; use std::collections::HashMap; use std::sync::Arc; @@ -40,6 +41,10 @@ pub struct Worker { max_buffer_per_series: usize, /// Allowed lateness in ms. allowed_lateness_ms: i64, + /// When true, skip aggregation and pass raw samples through. + pass_raw_samples: bool, + /// Aggregation ID stamped on each raw-mode output. + raw_mode_aggregation_id: u64, } impl Worker { @@ -50,6 +55,8 @@ impl Worker { agg_configs: HashMap, max_buffer_per_series: usize, allowed_lateness_ms: i64, + pass_raw_samples: bool, + raw_mode_aggregation_id: u64, ) -> Self { Self { id, @@ -59,6 +66,8 @@ impl Worker { agg_configs, max_buffer_per_series, allowed_lateness_ms, + pass_raw_samples, + raw_mode_aggregation_id, } } @@ -149,6 +158,10 @@ impl Worker { series_key: &str, samples: Vec<(i64, f64)>, ) -> Result<(), Box> { + if self.pass_raw_samples { + return self.process_samples_raw(series_key, samples); + } + // Copy scalars out of self before taking &mut self.series_map let worker_id = self.id; let allowed_lateness_ms = self.allowed_lateness_ms; @@ -251,8 +264,45 @@ impl Worker { Ok(()) } + /// Raw fast-path: emit each sample as a standalone `SumAccumulator`. + fn process_samples_raw( + &self, + series_key: &str, + samples: Vec<(i64, f64)>, + ) -> Result<(), Box> { + let mut emit_batch: Vec<(PrecomputedOutput, Box)> = + Vec::with_capacity(samples.len()); + + for (ts, val) in samples { + let output = PrecomputedOutput::new( + ts as u64, + ts as u64, + None, + self.raw_mode_aggregation_id, + ); + let accumulator = SumAccumulator::with_sum(val); + emit_batch.push((output, Box::new(accumulator))); + } + + if !emit_batch.is_empty() { + debug!( + "Worker {} raw-emitting {} samples for {}", + self.id, + emit_batch.len(), + series_key + ); + self.output_sink.emit_batch(emit_batch)?; + } + + Ok(()) + } + /// Flush all series — force-close windows that are past due. fn flush_all(&mut self) -> Result<(), Box> { + if self.pass_raw_samples { + return Ok(()); + } + let mut emit_batch: Vec<(PrecomputedOutput, Box)> = Vec::new(); for (series_key, state) in &mut self.series_map { From 3ee048ffc0cb824d136dfdf330e3aa9f0182b8db Mon Sep 17 00:00:00 2001 From: zz_y Date: Mon, 23 Feb 2026 14:16:58 -0700 Subject: [PATCH 05/38] Add debug-level tracing spans for e2e latency measurement in precompute engine MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Instrument the ingest→worker→store pipeline with debug_span! and an Instant-based e2e latency log. All spans are at debug level so there is zero overhead at the default info level. Enable FmtSpan::CLOSE in both binaries to emit span durations when RUST_LOG=debug. Co-Authored-By: Claude Opus 4.6 --- .../src/bin/precompute_engine.rs | 2 + .../src/bin/test_e2e_precompute.rs | 1 + .../src/precompute_engine/mod.rs | 70 +++++++++++-------- .../src/precompute_engine/output_sink.rs | 3 + .../src/precompute_engine/series_router.rs | 4 ++ .../src/precompute_engine/worker.rs | 15 +++- 6 files changed, 63 insertions(+), 32 deletions(-) diff --git a/asap-query-engine/src/bin/precompute_engine.rs b/asap-query-engine/src/bin/precompute_engine.rs index ad91c3f..bb71efd 100644 --- a/asap-query-engine/src/bin/precompute_engine.rs +++ b/asap-query-engine/src/bin/precompute_engine.rs @@ -12,6 +12,7 @@ use query_engine_rust::{HttpServer, HttpServerConfig}; use query_engine_rust::data_model::QueryLanguage; use std::sync::Arc; use tracing::info; +use tracing_subscriber::fmt::format::FmtSpan; #[derive(Parser, Debug)] #[command(name = "precompute_engine")] @@ -70,6 +71,7 @@ async fn main() -> Result<(), Box> { tracing_subscriber::EnvFilter::try_from_default_env() .unwrap_or_else(|_| tracing_subscriber::EnvFilter::new("info")), ) + .with_span_events(FmtSpan::CLOSE) .init(); let args = Args::parse(); diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index dadabe4..22d4be8 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -74,6 +74,7 @@ async fn main() -> Result<(), Box> { tracing_subscriber::EnvFilter::try_from_default_env() .unwrap_or_else(|_| tracing_subscriber::EnvFilter::new("info")), ) + .with_span_events(tracing_subscriber::fmt::format::FmtSpan::CLOSE) .init(); // Load configs the same way main.rs does diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs index ad1a2a6..1fe7110 100644 --- a/asap-query-engine/src/precompute_engine/mod.rs +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -17,7 +17,8 @@ use std::collections::HashMap; use std::sync::Arc; use tokio::net::TcpListener; use tokio::sync::mpsc; -use tracing::{info, warn}; +use std::time::Instant; +use tracing::{debug_span, info, warn, Instrument}; /// Shared state for the ingest HTTP handler. struct IngestState { @@ -141,39 +142,46 @@ async fn handle_ingest( State(state): State>, body: Bytes, ) -> StatusCode { - let samples = match decode_prometheus_remote_write(&body) { - Ok(s) => s, - Err(e) => { - warn!("Failed to decode remote write: {}", e); - return StatusCode::BAD_REQUEST; - } - }; + let ingest_span = debug_span!("ingest", body_len = body.len()); + let ingest_received_at = Instant::now(); + + async { + let samples = match decode_prometheus_remote_write(&body) { + Ok(s) => s, + Err(e) => { + warn!("Failed to decode remote write: {}", e); + return StatusCode::BAD_REQUEST; + } + }; - if samples.is_empty() { - return StatusCode::NO_CONTENT; - } + if samples.is_empty() { + return StatusCode::NO_CONTENT; + } - let count = samples.len() as u64; - state - .samples_ingested - .fetch_add(count, std::sync::atomic::Ordering::Relaxed); - - // Group samples by series key for batch routing - let mut by_series: HashMap<&str, Vec<(i64, f64)>> = HashMap::new(); - for s in &samples { - by_series - .entry(&s.labels) - .or_default() - .push((s.timestamp_ms, s.value)); - } + let count = samples.len() as u64; + state + .samples_ingested + .fetch_add(count, std::sync::atomic::Ordering::Relaxed); + + // Group samples by series key for batch routing + let mut by_series: HashMap<&str, Vec<(i64, f64)>> = HashMap::new(); + for s in &samples { + by_series + .entry(&s.labels) + .or_default() + .push((s.timestamp_ms, s.value)); + } - // Route each series batch to the correct worker - for (series_key, batch) in by_series { - if let Err(e) = state.router.route(series_key, batch).await { - warn!("Routing error for {}: {}", series_key, e); - return StatusCode::INTERNAL_SERVER_ERROR; + // Route each series batch to the correct worker + for (series_key, batch) in by_series { + if let Err(e) = state.router.route(series_key, batch, ingest_received_at).await { + warn!("Routing error for {}: {}", series_key, e); + return StatusCode::INTERNAL_SERVER_ERROR; + } } - } - StatusCode::NO_CONTENT + StatusCode::NO_CONTENT + } + .instrument(ingest_span) + .await } diff --git a/asap-query-engine/src/precompute_engine/output_sink.rs b/asap-query-engine/src/precompute_engine/output_sink.rs index 445cb49..6654902 100644 --- a/asap-query-engine/src/precompute_engine/output_sink.rs +++ b/asap-query-engine/src/precompute_engine/output_sink.rs @@ -1,6 +1,7 @@ use crate::data_model::{AggregateCore, PrecomputedOutput}; use crate::stores::Store; use std::sync::Arc; +use tracing::debug_span; /// Trait for emitting completed window outputs. pub trait OutputSink: Send + Sync { @@ -29,6 +30,7 @@ impl OutputSink for StoreOutputSink { if outputs.is_empty() { return Ok(()); } + let _span = debug_span!("store_insert", batch_size = outputs.len()).entered(); self.store.insert_precomputed_output_batch(outputs) } } @@ -54,6 +56,7 @@ impl OutputSink for RawPassthroughSink { if outputs.is_empty() { return Ok(()); } + let _span = debug_span!("store_insert_raw", batch_size = outputs.len()).entered(); self.store.insert_precomputed_output_batch(outputs) } } diff --git a/asap-query-engine/src/precompute_engine/series_router.rs b/asap-query-engine/src/precompute_engine/series_router.rs index fd05817..4642f7f 100644 --- a/asap-query-engine/src/precompute_engine/series_router.rs +++ b/asap-query-engine/src/precompute_engine/series_router.rs @@ -1,3 +1,4 @@ +use std::time::Instant; use tokio::sync::mpsc; use xxhash_rust::xxh64::xxh64; @@ -8,6 +9,7 @@ pub enum WorkerMessage { Samples { series_key: String, samples: Vec<(i64, f64)>, // (timestamp_ms, value) + ingest_received_at: Instant, }, /// Signal the worker to flush/check idle windows. Flush, @@ -36,12 +38,14 @@ impl SeriesRouter { &self, series_key: &str, samples: Vec<(i64, f64)>, + ingest_received_at: Instant, ) -> Result<(), Box> { let worker_idx = self.worker_for(series_key); self.senders[worker_idx] .send(WorkerMessage::Samples { series_key: series_key.to_string(), samples, + ingest_received_at, }) .await .map_err(|e| format!("Failed to send to worker {}: {}", worker_idx, e))?; diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs index 347a679..4719f1c 100644 --- a/asap-query-engine/src/precompute_engine/worker.rs +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -9,7 +9,7 @@ use sketch_db_common::aggregation_config::AggregationConfig; use std::collections::HashMap; use std::sync::Arc; use tokio::sync::mpsc; -use tracing::{debug, info, warn}; +use tracing::{debug, debug_span, info, warn}; /// Per-aggregation state within a series: the window manager and active /// window accumulators. @@ -80,10 +80,23 @@ impl Worker { WorkerMessage::Samples { series_key, samples, + ingest_received_at, } => { + let sample_count = samples.len(); + let _span = debug_span!( + "worker_process", + worker_id = self.id, + series = %series_key, + sample_count, + ) + .entered(); if let Err(e) = self.process_samples(&series_key, samples) { warn!("Worker {} error processing {}: {}", self.id, series_key, e); } + debug!( + e2e_latency_us = ingest_received_at.elapsed().as_micros() as u64, + "e2e: ingest->worker complete" + ); } WorkerMessage::Flush => { if let Err(e) = self.flush_all() { From 9e19d84a51164cdc8d6c8eaba5b9f43d699a53b2 Mon Sep 17 00:00:00 2001 From: zz_y Date: Mon, 23 Feb 2026 14:26:27 -0700 Subject: [PATCH 06/38] Add 1000-sample batch latency test to E2E precompute test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Sends 10 series × 100 samples in a single HTTP request to the raw-mode engine and verifies all 1000 samples land in the store. Prints client RTT and per-series e2e_latency_us at debug level. Co-Authored-By: Claude Opus 4.6 --- .../src/bin/test_e2e_precompute.rs | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index 22d4be8..f5993f3 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -327,6 +327,60 @@ async fn main() -> Result<(), Box> { } println!(" Raw mode test PASSED"); + // ----------------------------------------------------------------------- + // BATCH LATENCY TEST + // Send 1000 samples in a single HTTP request to measure realistic e2e + // latency. Uses the raw-mode engine to avoid window-close dependencies. + // ----------------------------------------------------------------------- + println!("\n=== Batch latency test: 1000 samples in one request ==="); + + // Build a single WriteRequest with 1000 TimeSeries entries spread across + // 10 series × 100 timestamps each, so routing fans out to workers. + let mut batch_timeseries = Vec::with_capacity(1000); + for series_idx in 0..10 { + let label_val = format!("batch_{series_idx}"); + for t in 0..100 { + let ts = 200_000 + (series_idx as i64) * 1000 + t; // unique ts per sample + let val = (series_idx * 100 + t) as f64; + batch_timeseries.push(make_sample("fake_metric", &label_val, ts, val)); + } + } + let batch_body = build_remote_write_body(batch_timeseries); + println!(" Payload size: {} bytes (snappy-compressed)", batch_body.len()); + + let t0 = std::time::Instant::now(); + let resp = client + .post(format!("http://localhost:{RAW_INGEST_PORT}/api/v1/write")) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(batch_body) + .send() + .await?; + let client_rtt = t0.elapsed(); + println!( + " HTTP response: {} in {:.3}ms", + resp.status().as_u16(), + client_rtt.as_secs_f64() * 1000.0, + ); + + // Wait for all workers to finish processing + tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; + + // Verify samples landed in the store + let batch_results = store.query_precomputed_output( + "fake_metric", + raw_agg_id, + 200_000, + 210_000, + )?; + let batch_buckets: usize = batch_results.values().map(|v| v.len()).sum(); + println!(" Stored {batch_buckets} buckets from batch (expected 1000)"); + assert!( + batch_buckets >= 1000, + "Expected at least 1000 raw samples in store, got {batch_buckets}" + ); + println!(" Batch latency test PASSED"); + println!("\n=== E2E test complete ==="); Ok(()) From bdfc2ea76c771a67363a4fd79c38126adb73b11b Mon Sep 17 00:00:00 2001 From: zz_y Date: Mon, 23 Feb 2026 14:35:18 -0700 Subject: [PATCH 07/38] Add 10M-sample throughput test to E2E precompute test MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Sends 1000 requests × 10000 samples (50 distinct series) to the raw-mode engine and polls until all samples are stored. Reports both send throughput and e2e throughput including drain time. Co-Authored-By: Claude Opus 4.6 --- .../src/bin/test_e2e_precompute.rs | 83 +++++++++++++++++++ 1 file changed, 83 insertions(+) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index f5993f3..3535dd3 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -381,6 +381,89 @@ async fn main() -> Result<(), Box> { ); println!(" Batch latency test PASSED"); + // ----------------------------------------------------------------------- + // THROUGHPUT TEST + // Send many requests back-to-back and measure sustained throughput + // (samples/sec). Uses the raw-mode engine for a clean measurement. + // ----------------------------------------------------------------------- + println!("\n=== Throughput test: 1000 requests × 10000 samples ==="); + + let num_requests = 1000u64; + let samples_per_request = 10_000u64; + let total_samples = num_requests * samples_per_request; + + // Pre-build all request bodies so serialization doesn't count against throughput + let mut bodies = Vec::with_capacity(num_requests as usize); + for req_idx in 0..num_requests { + let mut timeseries = Vec::with_capacity(samples_per_request as usize); + for s in 0..samples_per_request { + let series_label = format!("tp_{}", s % 50); // 50 distinct series + let ts = 300_000 + req_idx as i64 * 10_000 + s as i64; + timeseries.push(make_sample("fake_metric", &series_label, ts, s as f64)); + } + bodies.push(build_remote_write_body(timeseries)); + } + + let throughput_start = std::time::Instant::now(); + + for (i, body) in bodies.into_iter().enumerate() { + let resp = client + .post(format!("http://localhost:{RAW_INGEST_PORT}/api/v1/write")) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(body) + .send() + .await?; + if resp.status() != reqwest::StatusCode::NO_CONTENT { + eprintln!(" Request {i} failed: {}", resp.status()); + } + } + + let send_elapsed = throughput_start.elapsed(); + println!( + " All {} requests sent in {:.1}ms", + num_requests, + send_elapsed.as_secs_f64() * 1000.0, + ); + + // Poll until workers drain or timeout after 60s + let max_ts = 300_000u64 + num_requests * 10_000 + samples_per_request; + let drain_deadline = std::time::Instant::now() + std::time::Duration::from_secs(60); + let mut tp_buckets: usize; + loop { + let tp_results = store.query_precomputed_output( + "fake_metric", + raw_agg_id, + 300_000, + max_ts, + )?; + tp_buckets = tp_results.values().map(|v| v.len()).sum(); + if tp_buckets as u64 >= total_samples || std::time::Instant::now() > drain_deadline { + break; + } + tokio::time::sleep(tokio::time::Duration::from_millis(200)).await; + } + let total_elapsed = throughput_start.elapsed(); + + let send_throughput = total_samples as f64 / send_elapsed.as_secs_f64(); + let e2e_throughput = tp_buckets as f64 / total_elapsed.as_secs_f64(); + println!(" Stored {tp_buckets}/{total_samples} samples"); + println!( + " Send throughput: {:.0} samples/sec ({:.1}ms for {total_samples} samples)", + send_throughput, + send_elapsed.as_secs_f64() * 1000.0, + ); + println!( + " E2E throughput: {:.0} samples/sec ({:.1}ms until all stored)", + e2e_throughput, + total_elapsed.as_secs_f64() * 1000.0, + ); + assert!( + tp_buckets as u64 >= total_samples, + "Expected at least {total_samples} samples in store, got {tp_buckets}" + ); + println!(" Throughput test PASSED"); + println!("\n=== E2E test complete ==="); Ok(()) From d8f4a6c55c9490d30c8f025da9b5db11a0e04d24 Mon Sep 17 00:00:00 2001 From: zz_y Date: Mon, 23 Feb 2026 17:42:19 -0700 Subject: [PATCH 08/38] update --- .../src/bin/test_e2e_precompute.rs | 2 +- .../src/precompute_engine/mod.rs | 16 ++++--- .../src/precompute_engine/series_router.rs | 43 +++++++++++++++++++ 3 files changed, 54 insertions(+), 7 deletions(-) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index 3535dd3..2c7bb84 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -262,7 +262,7 @@ async fn main() -> Result<(), Box> { // Pick aggregation_id = 1 to match the existing streaming config. let raw_agg_id: u64 = 1; let raw_engine_config = PrecomputeEngineConfig { - num_workers: 1, + num_workers: 4, ingest_port: RAW_INGEST_PORT, allowed_lateness_ms: 5000, max_buffer_per_series: 10000, diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs index 1fe7110..5471618 100644 --- a/asap-query-engine/src/precompute_engine/mod.rs +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -172,12 +172,16 @@ async fn handle_ingest( .push((s.timestamp_ms, s.value)); } - // Route each series batch to the correct worker - for (series_key, batch) in by_series { - if let Err(e) = state.router.route(series_key, batch, ingest_received_at).await { - warn!("Routing error for {}: {}", series_key, e); - return StatusCode::INTERNAL_SERVER_ERROR; - } + // Convert to owned keys for batch routing + let by_series_owned: HashMap> = by_series + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect(); + + // Route all series to workers concurrently + if let Err(e) = state.router.route_batch(by_series_owned, ingest_received_at).await { + warn!("Batch routing error: {}", e); + return StatusCode::INTERNAL_SERVER_ERROR; } StatusCode::NO_CONTENT diff --git a/asap-query-engine/src/precompute_engine/series_router.rs b/asap-query-engine/src/precompute_engine/series_router.rs index 4642f7f..a159b02 100644 --- a/asap-query-engine/src/precompute_engine/series_router.rs +++ b/asap-query-engine/src/precompute_engine/series_router.rs @@ -1,4 +1,6 @@ +use std::collections::HashMap; use std::time::Instant; +use futures::future::try_join_all; use tokio::sync::mpsc; use xxhash_rust::xxh64::xxh64; @@ -52,6 +54,47 @@ impl SeriesRouter { Ok(()) } + /// Route a pre-grouped batch of series to workers concurrently. + /// + /// Groups messages by target worker, then sends to each worker in parallel + /// (messages within a single worker are sent sequentially to preserve ordering). + pub async fn route_batch( + &self, + by_series: HashMap>, + ingest_received_at: Instant, + ) -> Result<(), Box> { + // Group messages by target worker index + let mut per_worker: HashMap> = HashMap::new(); + for (series_key, samples) in by_series { + let worker_idx = self.worker_for(&series_key); + per_worker + .entry(worker_idx) + .or_default() + .push(WorkerMessage::Samples { + series_key, + samples, + ingest_received_at, + }); + } + + // Send to each worker concurrently + try_join_all(per_worker.into_iter().map(|(worker_idx, messages)| { + let sender = &self.senders[worker_idx]; + async move { + for msg in messages { + sender.send(msg).await.map_err(|e| { + format!("Failed to send to worker {}: {}", worker_idx, e) + })?; + } + Ok::<(), String>(()) + } + })) + .await + .map_err(|e| -> Box { Box::new(std::io::Error::new(std::io::ErrorKind::Other, e)) })?; + + Ok(()) + } + /// Broadcast a flush signal to all workers. pub async fn broadcast_flush(&self) -> Result<(), Box> { for (i, sender) in self.senders.iter().enumerate() { From b5c2eb8853ee735a3a871ef1628be78368a386c5 Mon Sep 17 00:00:00 2001 From: zz_y Date: Tue, 24 Feb 2026 12:25:22 -0700 Subject: [PATCH 09/38] Fix sliding window aggregation: feed samples into all overlapping windows Previously each sample was assigned to only one window via window_start_for(), which is incorrect for sliding windows where window_size > slide_interval. Added window_starts_containing() that returns all window starts whose range covers the timestamp, and use it in the worker aggregation loop. Co-Authored-By: Claude Opus 4.6 --- .../src/precompute_engine/window_manager.rs | 44 +++++++++++++++++++ .../src/precompute_engine/worker.rs | 26 ++++++----- 2 files changed, 58 insertions(+), 12 deletions(-) diff --git a/asap-query-engine/src/precompute_engine/window_manager.rs b/asap-query-engine/src/precompute_engine/window_manager.rs index 929f928..e397fb7 100644 --- a/asap-query-engine/src/precompute_engine/window_manager.rs +++ b/asap-query-engine/src/precompute_engine/window_manager.rs @@ -73,6 +73,20 @@ impl WindowManager { closed } + /// Return all window starts whose window `[start, start + window_size_ms)` + /// contains the given timestamp. For tumbling windows this returns exactly + /// one start; for sliding windows it returns `ceil(window_size / slide)` + /// starts. + pub fn window_starts_containing(&self, timestamp_ms: i64) -> Vec { + let mut starts = Vec::new(); + let mut start = self.window_start_for(timestamp_ms); + while start + self.window_size_ms > timestamp_ms { + starts.push(start); + start -= self.slide_interval_ms; + } + starts + } + /// Return the window `[start, end)` boundaries for a given window start. pub fn window_bounds(&self, window_start: i64) -> (i64, i64) { (window_start, window_start + self.window_size_ms) @@ -151,4 +165,34 @@ mod tests { let closed = wm.closed_windows(15_000, 35_000); assert_eq!(closed, vec![0]); } + + #[test] + fn test_window_starts_containing_tumbling() { + // 60s tumbling windows — each sample belongs to exactly one window + let wm = WindowManager::new(60, 0); + let mut starts = wm.window_starts_containing(15_000); + starts.sort(); + assert_eq!(starts, vec![0]); + + let mut starts = wm.window_starts_containing(60_000); + starts.sort(); + assert_eq!(starts, vec![60_000]); + } + + #[test] + fn test_window_starts_containing_sliding() { + // 30s window, 10s slide — each sample belongs to 3 windows + let wm = WindowManager::new(30, 10); + + // t=15_000 belongs to [0, 30_000), [10_000, 40_000) + // and [-10_000, 20_000) which starts negative — still returned + let mut starts = wm.window_starts_containing(15_000); + starts.sort(); + assert_eq!(starts, vec![-10_000, 0, 10_000]); + + // t=30_000 belongs to [10_000, 40_000), [20_000, 50_000), [30_000, 60_000) + let mut starts = wm.window_starts_containing(30_000); + starts.sort(); + assert_eq!(starts, vec![10_000, 20_000, 30_000]); + } } diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs index 4719f1c..8d0fb28 100644 --- a/asap-query-engine/src/precompute_engine/worker.rs +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -221,18 +221,20 @@ impl Worker { continue; // already dropped } - let window_start = agg_state.window_manager.window_start_for(ts); - - let updater = agg_state - .active_windows - .entry(window_start) - .or_insert_with(|| create_accumulator_updater(&agg_state.config)); - - if updater.is_keyed() { - let key = extract_key_from_series(series_key, &agg_state.config); - updater.update_keyed(&key, val, ts); - } else { - updater.update_single(val, ts); + let window_starts = agg_state.window_manager.window_starts_containing(ts); + + for window_start in window_starts { + let updater = agg_state + .active_windows + .entry(window_start) + .or_insert_with(|| create_accumulator_updater(&agg_state.config)); + + if updater.is_keyed() { + let key = extract_key_from_series(series_key, &agg_state.config); + updater.update_keyed(&key, val, ts); + } else { + updater.update_single(val, ts); + } } } From 0045f98f3a9d472f17755834c271264d8ab1888e Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 15:20:40 -0700 Subject: [PATCH 10/38] Add late data handling for closed windows and precompute engine design doc Fix ghost accumulator bug where late samples passing the allowed_lateness check could create orphaned accumulators for already-closed windows. Add configurable LateDataPolicy (Drop/ForwardToStore) to control behavior. Drop prevents ghost windows; ForwardToStore emits mini-accumulators for query-time merge. Also add precompute engine design document. Co-Authored-By: Claude Opus 4.6 --- .../src/bin/precompute_engine.rs | 7 +- .../src/bin/test_e2e_precompute.rs | 4 +- asap-query-engine/src/lib.rs | 2 +- .../src/precompute_engine/config.rs | 13 + .../src/precompute_engine/mod.rs | 1 + .../precompute_engine_design_doc.md | 431 ++++++++++++++++++ .../src/precompute_engine/worker.rs | 51 +++ 7 files changed, 506 insertions(+), 3 deletions(-) create mode 100644 asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md diff --git a/asap-query-engine/src/bin/precompute_engine.rs b/asap-query-engine/src/bin/precompute_engine.rs index bb71efd..b634c85 100644 --- a/asap-query-engine/src/bin/precompute_engine.rs +++ b/asap-query-engine/src/bin/precompute_engine.rs @@ -4,7 +4,7 @@ use query_engine_rust::data_model::{ }; use query_engine_rust::drivers::query::adapters::AdapterConfig; use query_engine_rust::engines::SimpleEngine; -use query_engine_rust::precompute_engine::config::PrecomputeEngineConfig; +use query_engine_rust::precompute_engine::config::{LateDataPolicy, PrecomputeEngineConfig}; use query_engine_rust::precompute_engine::output_sink::{RawPassthroughSink, StoreOutputSink}; use query_engine_rust::precompute_engine::PrecomputeEngine; use query_engine_rust::stores::SimpleMapStore; @@ -61,6 +61,10 @@ struct Args { /// Aggregation ID to stamp on each raw-mode output #[arg(long, default_value_t = 0)] raw_mode_aggregation_id: u64, + + /// Policy for handling late samples that arrive after their window has closed + #[arg(long, value_enum, default_value_t = LateDataPolicy::Drop)] + late_data_policy: LateDataPolicy, } #[tokio::main] @@ -134,6 +138,7 @@ async fn main() -> Result<(), Box> { channel_buffer_size: args.channel_buffer_size, pass_raw_samples: args.pass_raw_samples, raw_mode_aggregation_id: args.raw_mode_aggregation_id, + late_data_policy: args.late_data_policy, }; // Create the output sink (writes directly to the store) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index 2c7bb84..668d819 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -15,7 +15,7 @@ use query_engine_rust::drivers::ingest::prometheus_remote_write::{ }; use query_engine_rust::drivers::query::adapters::AdapterConfig; use query_engine_rust::engines::SimpleEngine; -use query_engine_rust::precompute_engine::config::PrecomputeEngineConfig; +use query_engine_rust::precompute_engine::config::{LateDataPolicy, PrecomputeEngineConfig}; use query_engine_rust::precompute_engine::output_sink::{RawPassthroughSink, StoreOutputSink}; use query_engine_rust::precompute_engine::PrecomputeEngine; use query_engine_rust::stores::SimpleMapStore; @@ -143,6 +143,7 @@ async fn main() -> Result<(), Box> { channel_buffer_size: 10000, pass_raw_samples: false, raw_mode_aggregation_id: 0, + late_data_policy: LateDataPolicy::Drop, }; let output_sink = Arc::new(StoreOutputSink::new(store.clone())); let engine = PrecomputeEngine::new(engine_config, streaming_config.clone(), output_sink); @@ -270,6 +271,7 @@ async fn main() -> Result<(), Box> { channel_buffer_size: 10000, pass_raw_samples: true, raw_mode_aggregation_id: raw_agg_id, + late_data_policy: LateDataPolicy::Drop, }; let raw_sink = Arc::new(RawPassthroughSink::new(store.clone())); let raw_engine = PrecomputeEngine::new( diff --git a/asap-query-engine/src/lib.rs b/asap-query-engine/src/lib.rs index 81f9824..0216825 100644 --- a/asap-query-engine/src/lib.rs +++ b/asap-query-engine/src/lib.rs @@ -27,7 +27,7 @@ pub use engines::{InstantVector, QueryResult, SimpleEngine}; pub use drivers::{HttpServer, HttpServerConfig, KafkaConsumer, KafkaConsumerConfig}; -pub use precompute_engine::config::PrecomputeEngineConfig; +pub use precompute_engine::config::{LateDataPolicy, PrecomputeEngineConfig}; pub use precompute_engine::output_sink::StoreOutputSink; pub use precompute_engine::PrecomputeEngine; diff --git a/asap-query-engine/src/precompute_engine/config.rs b/asap-query-engine/src/precompute_engine/config.rs index 7bc8df9..c509ae5 100644 --- a/asap-query-engine/src/precompute_engine/config.rs +++ b/asap-query-engine/src/precompute_engine/config.rs @@ -1,5 +1,14 @@ use serde::{Deserialize, Serialize}; +/// Policy for handling late samples that arrive after their window has closed. +#[derive(Debug, Clone, Copy, PartialEq, Serialize, Deserialize, clap::ValueEnum)] +pub enum LateDataPolicy { + /// Drop late samples that arrive after their window has closed. + Drop, + /// Forward late samples to the store to be merged with existing window data. + ForwardToStore, +} + /// Configuration for the precompute engine. #[derive(Debug, Clone, Serialize, Deserialize)] pub struct PrecomputeEngineConfig { @@ -21,6 +30,8 @@ pub struct PrecomputeEngineConfig { pub pass_raw_samples: bool, /// Aggregation ID to stamp on each raw-mode output. pub raw_mode_aggregation_id: u64, + /// Policy for handling late samples that arrive after their window has closed. + pub late_data_policy: LateDataPolicy, } impl Default for PrecomputeEngineConfig { @@ -34,6 +45,7 @@ impl Default for PrecomputeEngineConfig { channel_buffer_size: 10_000, pass_raw_samples: false, raw_mode_aggregation_id: 0, + late_data_policy: LateDataPolicy::Drop, } } } @@ -53,5 +65,6 @@ mod tests { assert_eq!(config.channel_buffer_size, 10_000); assert!(!config.pass_raw_samples); assert_eq!(config.raw_mode_aggregation_id, 0); + assert_eq!(config.late_data_policy, LateDataPolicy::Drop); } } diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs index 5471618..819502a 100644 --- a/asap-query-engine/src/precompute_engine/mod.rs +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -84,6 +84,7 @@ impl PrecomputeEngine { self.config.allowed_lateness_ms, self.config.pass_raw_samples, self.config.raw_mode_aggregation_id, + self.config.late_data_policy, ); let handle = tokio::spawn(async move { worker.run().await; diff --git a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md new file mode 100644 index 0000000..f59a42f --- /dev/null +++ b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md @@ -0,0 +1,431 @@ +# Precompute Engine Design Document + +## 1. Overview + +The Precompute Engine is a real-time streaming aggregation system that sits between +Prometheus-compatible metric producers and ASAP storage and query engine. It accepts raw +time-series samples via the Prometheus remote-write protocol, buffers them, computes +windowed aggregations (sketches), and writes the results to a store for fast +query-time retrieval. + +**Key properties:** +- Watermark-based windowed aggregation (tumbling and sliding windows) +- Multi-threaded, shard-nothing worker architecture +- Pluggable accumulator types (Sum, Min/Max, Increase, KLL, CMS, HydraKLL) +- Configurable late-data handling (Drop or ForwardToStore) +- Optional raw passthrough mode for bypassing aggregation + +## 2. Architecture + +``` + Prometheus Remote Write + | + Axum HTTP Server (:9090) + POST /api/v1/write + | + decode + group by series + | + SeriesRouter (hash) + / | \ + Worker 0 Worker 1 Worker 2 ... Worker N-1 + (shard 0) (shard 1) (shard 2) (shard N-1) + | | | | + +---------- + --------- + ----------- + + | + OutputSink.emit_batch() + | + Store + (SimpleMapStore / PerKey) + | + Query Engine + (PromQL / SQL / etc.) +``` + +A periodic **flush timer** broadcasts `Flush` messages to all workers so that +windows that would otherwise remain open (no new samples arriving) are closed +and emitted. + +## 3. Components + +### 3.1 PrecomputeEngine (`mod.rs`) + +Top-level orchestrator. On `run()`: + +1. Creates one `mpsc::channel` per worker. +2. Constructs a `SeriesRouter` with the sender halves. +3. Spawns `Worker` tasks, each owning its receiver. +4. Spawns a flush timer that calls `router.broadcast_flush()` every + `flush_interval_ms`. +5. Starts the Axum HTTP server and blocks until shutdown. + +```rust +pub struct PrecomputeEngine { + config: PrecomputeEngineConfig, + streaming_config: Arc, + output_sink: Arc, +} +``` + +### 3.2 Configuration (`config.rs`) + +```rust +pub struct PrecomputeEngineConfig { + pub num_workers: usize, // default: 4 + pub ingest_port: u16, // default: 9090 + pub allowed_lateness_ms: i64, // default: 5,000 + pub max_buffer_per_series: usize, // default: 10,000 + pub flush_interval_ms: u64, // default: 1,000 + pub channel_buffer_size: usize, // default: 10,000 + pub pass_raw_samples: bool, // default: false + pub raw_mode_aggregation_id: u64, // default: 0 + pub late_data_policy: LateDataPolicy, // default: Drop +} + +pub enum LateDataPolicy { + Drop, // Silently discard late samples for closed windows + ForwardToStore, // Emit a mini-accumulator for query-time merge +} +``` + +### 3.3 SeriesRouter (`series_router.rs`) + +Deterministic hash-based routing using XXHash64: + +``` +worker_idx = xxhash64(series_key) % num_workers +``` + +All samples for a given series always land on the same worker, so per-series +state (buffer, watermark, active windows) needs no synchronization. + +**Message types:** +```rust +enum WorkerMessage { + Samples { series_key: String, samples: Vec<(i64, f64)>, ingest_received_at: Instant }, + Flush, + Shutdown, +} +``` + +`route_batch()` groups messages by target worker and sends them in parallel for +throughput while preserving per-worker ordering. + +### 3.4 Worker (`worker.rs`) + +Each worker owns an isolated shard of the series space. + +```rust +struct Worker { + id: usize, + receiver: mpsc::Receiver, + output_sink: Arc, + series_map: HashMap, + agg_configs: HashMap, + max_buffer_per_series: usize, + allowed_lateness_ms: i64, + pass_raw_samples: bool, + raw_mode_aggregation_id: u64, + late_data_policy: LateDataPolicy, +} +``` + +**Per-series state:** +```rust +struct SeriesState { + buffer: SeriesBuffer, // sorted sample buffer + previous_watermark_ms: i64, // last-seen watermark + aggregations: Vec, // one per matching config +} + +struct AggregationState { + config: AggregationConfig, + window_manager: WindowManager, + active_windows: HashMap>, +} +``` + +#### Processing pipeline (`process_samples`) + +``` +1. Match series to AggregationConfigs (by metric name / spatial_filter) +2. Insert samples into SeriesBuffer, update watermark +3. Drop samples beyond allowed_lateness_ms behind watermark +4. For each sample × each aggregation: + a. Compute window_starts_containing(ts) + b. For each window_start: + - If window not in active_windows AND already closed (watermark >= window_end): + → late_data_policy == Drop: skip + → late_data_policy == ForwardToStore: create mini-accumulator, emit + - Else: insert into active_windows, feed value to AccumulatorUpdater +5. Detect newly closed windows via closed_windows(prev_wm, current_wm) +6. Extract closed window accumulators → PrecomputedOutput + AggregateCore +7. Emit batch to OutputSink +8. Update previous_watermark_ms +``` + +#### Raw mode + +When `pass_raw_samples = true`, the entire aggregation pipeline is bypassed. +Each sample is emitted as a `SumAccumulator::with_sum(value)` with point-window +bounds `[ts, ts]` and the configured `raw_mode_aggregation_id`. + +### 3.5 SeriesBuffer (`series_buffer.rs`) + +Per-series in-memory buffer backed by `BTreeMap`. + +```rust +struct SeriesBuffer { + samples: BTreeMap, // timestamp_ms → value + watermark_ms: i64, // max timestamp ever seen (monotonic) + max_buffer_size: usize, +} +``` + +- Samples are automatically sorted by timestamp. +- Watermark only advances forward (monotonic). +- When the buffer exceeds `max_buffer_size`, the oldest samples are evicted. +- Supports range reads (`read_range`) and destructive drains (`drain_up_to`). + +### 3.6 WindowManager (`window_manager.rs`) + +Handles both tumbling and sliding window semantics. + +```rust +struct WindowManager { + window_size_ms: i64, // e.g. 60_000 + slide_interval_ms: i64, // == window_size for tumbling; < window_size for sliding +} +``` + +**Key methods:** + +| Method | Description | +|--------|-------------| +| `window_start_for(ts)` | Align timestamp down to nearest slide boundary | +| `window_starts_containing(ts)` | All windows whose `[start, start+size)` includes `ts`. Tumbling → 1 window; sliding → `ceil(size/slide)` windows | +| `closed_windows(prev_wm, curr_wm)` | Windows that transitioned open→closed as the watermark advanced | +| `window_bounds(start)` | Returns `(start, start + window_size_ms)` | + +**Window closure rule:** a window `[S, S + size)` closes when `watermark >= S + size`. +Once closed, a window never reopens. + +### 3.7 AccumulatorUpdater (`accumulator_factory.rs`) + +Trait-based interface for feeding samples into sketch accumulators: + +```rust +trait AccumulatorUpdater: Send { + fn update_single(&mut self, value: f64, timestamp_ms: i64); + fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, timestamp_ms: i64); + fn take_accumulator(&mut self) -> Box; + fn reset(&mut self); + fn is_keyed(&self) -> bool; + fn memory_usage_bytes(&self) -> usize; +} +``` + +The factory function `create_accumulator_updater(config)` dispatches on +`(aggregation_type, aggregation_sub_type)`: + +| Type | Sub-type | Updater | +|------|----------|---------| +| SingleSubpopulation | Sum | SumAccumulatorUpdater | +| SingleSubpopulation | Min/Max | MinMaxAccumulatorUpdater | +| SingleSubpopulation | Increase | IncreaseAccumulatorUpdater | +| SingleSubpopulation | KLL | KllAccumulatorUpdater | +| MultipleSubpopulation | Sum | MultipleSumUpdater | +| MultipleSubpopulation | Min/Max | MultipleMinMaxUpdater | +| MultipleSubpopulation | Increase | MultipleIncreaseUpdater | +| MultipleSubpopulation | CMS | CmsAccumulatorUpdater | +| MultipleSubpopulation | HydraKLL | HydraKllAccumulatorUpdater | + +### 3.8 OutputSink (`output_sink.rs`) + +```rust +trait OutputSink: Send + Sync { + fn emit_batch( + &self, + outputs: Vec<(PrecomputedOutput, Box)>, + ) -> Result<(), Box>; +} +``` + +**Implementations:** +- `StoreOutputSink` — calls `store.insert_precomputed_output_batch()` +- `RawPassthroughSink` — same interface, used for raw mode +- `NoopOutputSink` — testing helper that counts emitted items + +## 4. Data Model + +### PrecomputedOutput + +```rust +pub struct PrecomputedOutput { + pub start_timestamp: u64, // window start (ms) + pub end_timestamp: u64, // window end (ms) + pub key: Option, // grouping key (e.g. method="GET") + pub aggregation_id: u64, +} +``` + +### KeyByLabelValues + +Ordered vector of label values matching the `grouping_labels` in the aggregation +config. Serialized as semicolon-delimited strings for hashing/storage. + +### AggregationConfig + +Loaded from `streaming_config.yaml`: + +```rust +pub struct AggregationConfig { + pub aggregation_id: u64, + pub aggregation_type: String, // "SingleSubpopulation" | "MultipleSubpopulation" + pub aggregation_sub_type: String, // "Sum" | "Min" | "Max" | "Increase" | "KLL" | ... + pub parameters: HashMap, + pub grouping_labels: KeyByLabelNames, + pub window_size: u64, // seconds + pub slide_interval: u64, // seconds (0 = tumbling) + pub metric: String, + pub spatial_filter: String, + pub num_aggregates_to_retain: Option, + // ... +} +``` + +## 5. Store Integration + +### Write path + +`OutputSink.emit_batch()` → `Store.insert_precomputed_output_batch()` + +The `SimpleMapStore` (PerKey variant) uses: +``` +DashMap>> +``` +where: +```rust +struct StoreKeyData { + time_map: HashMap<(u64, u64), Vec<(Option, Box)>>, + read_counts: HashMap<(u64, u64), u64>, +} +``` + +Multiple entries per `(start_ts, end_ts)` are allowed — they are appended, not +overwritten. This is what makes `ForwardToStore` late-data policy work: the late +mini-accumulator is stored alongside the original window accumulator. + +### Read path / query-time merge + +At query time, `PrecomputedSummaryReadExec` reads sparse buckets from the store. +`SummaryMergeMultipleExec` groups by label key and merges via +`accumulator.merge_with()`. + +The `NaiveMerger` re-merges all accumulators in the window on each slide. +The store's existing multi-entry-per-window design means late data is +automatically combined with original window data at query time. + +### Cleanup policies + +| Policy | Behavior | +|--------|----------| +| CircularBuffer | Keep N most recent windows (4x `num_aggregates_to_retain`) | +| ReadBased | Remove after `read_count >= threshold` | +| NoCleanup | Retain forever | + +## 6. Late Data Handling + +Two checks determine whether a sample is "late": + +1. **Watermark check** (sample-level): `ts < watermark - allowed_lateness_ms` → + sample is dropped entirely before reaching any aggregation logic. + +2. **Window closure check** (window-level): the sample passes the watermark check + but targets a window that is already closed + (`window not in active_windows && watermark >= window_end`). + +For case 2, the `LateDataPolicy` controls behavior: + +- **Drop**: log at debug level and skip. No ghost accumulator is created + (fixing the original bug where `or_insert_with` would create orphaned entries). + +- **ForwardToStore**: create a fresh `AccumulatorUpdater`, feed the single + late sample, wrap as `PrecomputedOutput`, and push into the same `emit_batch` + as normal closed-window outputs. The store appends it alongside the original + window data, and query-time merge combines them. + +## 7. Concurrency Model + +- **Ingest HTTP handler**: Axum async handler, stateless decoding. +- **SeriesRouter**: Lock-free hash routing. No shared mutable state. +- **Workers**: Each worker is single-threaded (owns its `series_map`). + No locks needed within a worker. +- **OutputSink / Store**: Thread-safe (`Arc`, DashMap-backed store). + Workers emit concurrently; the PerKey store uses per-aggregation_id RwLocks + to minimize contention. +- **Flush timer**: Separate tokio task, communicates via the same MPSC channels. + +## 8. Performance Characteristics + +**Ingest path (per batch):** +- Sample insert: O(log B) per sample (BTreeMap, B = buffer size) +- Window assignment: O(A × W) per sample (A = matching aggregations, + W = windows per sample — 1 for tumbling, ~3 for typical sliding) +- Accumulator update: O(1) for Sum/MinMax, O(log k) for KLL + +**Memory:** +- O(S × N) buffered samples (S = max per series, N = active series) +- O(A × W_open) active window accumulators + +**Throughput:** +- Workers process in parallel with no cross-shard coordination. +- E2E test demonstrates ~10M samples/sec sustained throughput with raw mode. + +## 9. CLI Usage + +### Standalone binary + +```bash +cargo run --bin precompute_engine -- \ + --streaming-config streaming_config.yaml \ + --ingest-port 9090 \ + --num-workers 4 \ + --allowed-lateness-ms 5000 \ + --max-buffer-per-series 10000 \ + --flush-interval-ms 1000 \ + --channel-buffer-size 10000 \ + --query-port 8080 \ + --lock-strategy per-key \ + --late-data-policy drop +``` + +### Embedded in main binary + +The precompute engine is also embedded in the main `query_engine_rust` binary, +enabled via `--enable-prometheus-remote-write`. In this mode it shares the +store with the Kafka consumer path. + +## 10. Testing + +- **Unit tests**: `worker.rs` (metric/label extraction), `window_manager.rs` + (tumbling/sliding arithmetic), `series_buffer.rs` (ordering, watermark), + `accumulator_factory.rs` (updater creation), `series_router.rs` (hashing), + `config.rs` (defaults). +- **E2E test** (`bin/test_e2e_precompute.rs`): Starts engine + store + query + server in-process, sends remote-write samples, queries via PromQL HTTP, + validates results. Includes batch latency and throughput benchmarks. + +## 11. File Map + +| File | Purpose | +|------|---------| +| `precompute_engine/mod.rs` | Orchestrator, HTTP ingest handler | +| `precompute_engine/config.rs` | `PrecomputeEngineConfig`, `LateDataPolicy` | +| `precompute_engine/worker.rs` | Per-shard processing, aggregation, window management | +| `precompute_engine/series_router.rs` | Hash-based series → worker routing | +| `precompute_engine/series_buffer.rs` | Per-series BTreeMap sample buffer | +| `precompute_engine/window_manager.rs` | Tumbling/sliding window logic | +| `precompute_engine/accumulator_factory.rs` | `AccumulatorUpdater` trait + factory | +| `precompute_engine/output_sink.rs` | `OutputSink` trait + Store/Noop impls | +| `bin/precompute_engine.rs` | Standalone CLI binary | +| `bin/test_e2e_precompute.rs` | End-to-end integration test | diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs index 8d0fb28..241362f 100644 --- a/asap-query-engine/src/precompute_engine/worker.rs +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -1,5 +1,6 @@ use crate::data_model::{AggregateCore, KeyByLabelValues, PrecomputedOutput}; use crate::precompute_engine::accumulator_factory::{create_accumulator_updater, AccumulatorUpdater}; +use crate::precompute_engine::config::LateDataPolicy; use crate::precompute_engine::output_sink::OutputSink; use crate::precompute_engine::series_buffer::SeriesBuffer; use crate::precompute_engine::series_router::WorkerMessage; @@ -45,6 +46,8 @@ pub struct Worker { pass_raw_samples: bool, /// Aggregation ID stamped on each raw-mode output. raw_mode_aggregation_id: u64, + /// Policy for handling late samples that arrive after their window has closed. + late_data_policy: LateDataPolicy, } impl Worker { @@ -57,6 +60,7 @@ impl Worker { allowed_lateness_ms: i64, pass_raw_samples: bool, raw_mode_aggregation_id: u64, + late_data_policy: LateDataPolicy, ) -> Self { Self { id, @@ -68,6 +72,7 @@ impl Worker { allowed_lateness_ms, pass_raw_samples, raw_mode_aggregation_id, + late_data_policy, } } @@ -178,6 +183,7 @@ impl Worker { // Copy scalars out of self before taking &mut self.series_map let worker_id = self.id; let allowed_lateness_ms = self.allowed_lateness_ms; + let late_data_policy = self.late_data_policy; // Ensure state exists self.get_or_create_series_state(series_key); @@ -224,6 +230,51 @@ impl Worker { let window_starts = agg_state.window_manager.window_starts_containing(ts); for window_start in window_starts { + let window_end = window_start + agg_state.window_manager.window_size_ms(); + + // Check if this window was already closed in a previous batch + if !agg_state.active_windows.contains_key(&window_start) + && current_wm >= window_end + { + // Window already closed — handle according to policy + match late_data_policy { + LateDataPolicy::Drop => { + debug!( + "Dropping late sample for closed window [{}, {})", + window_start, window_end + ); + continue; + } + LateDataPolicy::ForwardToStore => { + let mut updater = create_accumulator_updater(&agg_state.config); + if updater.is_keyed() { + let key = extract_key_from_series(series_key, &agg_state.config); + updater.update_keyed(&key, val, ts); + } else { + updater.update_single(val, ts); + } + let key = if updater.is_keyed() { + Some(extract_key_from_series(series_key, &agg_state.config)) + } else { + None + }; + let output = PrecomputedOutput::new( + window_start as u64, + window_end as u64, + key, + agg_state.config.aggregation_id, + ); + emit_batch.push((output, updater.take_accumulator())); + debug!( + "Forwarding late sample to store for closed window [{}, {})", + window_start, window_end + ); + continue; + } + } + } + + // Normal path: window is still open (or newly opened) let updater = agg_state .active_windows .entry(window_start) From 3feb7a0bddbd2fb1c537d0c2e23c1feb58f49577 Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 15:23:55 -0700 Subject: [PATCH 11/38] Minor wording fix in precompute engine design doc Co-Authored-By: Claude Opus 4.6 --- .../src/precompute_engine/precompute_engine_design_doc.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md index f59a42f..4cfe0ab 100644 --- a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md +++ b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md @@ -4,7 +4,7 @@ The Precompute Engine is a real-time streaming aggregation system that sits between Prometheus-compatible metric producers and ASAP storage and query engine. It accepts raw -time-series samples via the Prometheus remote-write protocol, buffers them, computes +time-series samples via the Prometheus remote-write protocol ingestion connector, buffers them, computes windowed aggregations (sketches), and writes the results to a store for fast query-time retrieval. From 2218997f562d3ff027d0cd6fbfc65d9f4a602317 Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 15:27:05 -0700 Subject: [PATCH 12/38] Clarify single-machine multi-threaded architecture in design doc Co-Authored-By: Claude Opus 4.6 --- .../precompute_engine_design_doc.md | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md index 4cfe0ab..64d6955 100644 --- a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md +++ b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md @@ -9,8 +9,9 @@ windowed aggregations (sketches), and writes the results to a store for fast query-time retrieval. **Key properties:** +- Single-machine, multi-threaded architecture (all workers run as async tasks within one process) - Watermark-based windowed aggregation (tumbling and sliding windows) -- Multi-threaded, shard-nothing worker architecture +- Shared-nothing worker design: series are hash-partitioned across threads with no cross-worker coordination - Pluggable accumulator types (Sum, Min/Max, Increase, KLL, CMS, HydraKLL) - Configurable late-data handling (Drop or ForwardToStore) - Optional raw passthrough mode for bypassing aggregation @@ -356,14 +357,26 @@ For case 2, the `LateDataPolicy` controls behavior: ## 7. Concurrency Model +The current implementation is **single-machine, multi-threaded**. All components +(HTTP server, workers, store) run within a single OS process as Tokio async +tasks on a shared thread pool. There is no distributed coordination, no +cross-machine communication, and no external dependency beyond the store. + - **Ingest HTTP handler**: Axum async handler, stateless decoding. - **SeriesRouter**: Lock-free hash routing. No shared mutable state. -- **Workers**: Each worker is single-threaded (owns its `series_map`). - No locks needed within a worker. +- **Workers**: Each worker is a single Tokio task that owns its `series_map` + exclusively. No locks needed within a worker — thread safety comes from + the hash-partitioning guarantee that each series is assigned to exactly one + worker. - **OutputSink / Store**: Thread-safe (`Arc`, DashMap-backed store). Workers emit concurrently; the PerKey store uses per-aggregation_id RwLocks to minimize contention. -- **Flush timer**: Separate tokio task, communicates via the same MPSC channels. +- **Flush timer**: Separate Tokio task, communicates via the same MPSC channels. + +Scaling beyond a single machine would require partitioning the series space +across multiple engine instances (e.g. via consistent hashing at the load +balancer level), each running this same single-process architecture +independently. ## 8. Performance Characteristics From 2ff938fc7325a8b35e9548233e91a0cd44426907 Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 15:38:24 -0700 Subject: [PATCH 13/38] Add multi-connector ingest support (Prometheus + VictoriaMetrics) Extract format-agnostic routing logic into route_decoded_samples() helper and register a VictoriaMetrics remote write endpoint at /api/v1/import alongside the existing Prometheus endpoint at /api/v1/write. Co-Authored-By: Claude Opus 4.6 --- .../src/precompute_engine/mod.rs | 112 +++++++++++------- .../precompute_engine_design_doc.md | 25 ++-- 2 files changed, 81 insertions(+), 56 deletions(-) diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs index 819502a..f114ef8 100644 --- a/asap-query-engine/src/precompute_engine/mod.rs +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -8,6 +8,7 @@ pub mod worker; use crate::data_model::StreamingConfig; use crate::drivers::ingest::prometheus_remote_write::decode_prometheus_remote_write; +use crate::drivers::ingest::victoriametrics_remote_write::decode_victoriametrics_remote_write; use crate::precompute_engine::config::PrecomputeEngineConfig; use crate::precompute_engine::output_sink::OutputSink; use crate::precompute_engine::series_router::{SeriesRouter, WorkerMessage}; @@ -18,7 +19,7 @@ use std::sync::Arc; use tokio::net::TcpListener; use tokio::sync::mpsc; use std::time::Instant; -use tracing::{debug_span, info, warn, Instrument}; +use tracing::{info, warn}; /// Shared state for the ingest HTTP handler. struct IngestState { @@ -118,9 +119,10 @@ impl PrecomputeEngine { } }); - // Start the Axum HTTP server for Prometheus remote write ingest + // Start the Axum HTTP server for ingest (Prometheus + VictoriaMetrics) let app = Router::new() - .route("/api/v1/write", post(handle_ingest)) + .route("/api/v1/write", post(handle_prometheus_ingest)) + .route("/api/v1/import", post(handle_victoriametrics_ingest)) .with_state(ingest_state); let addr = format!("0.0.0.0:{}", self.config.ingest_port); @@ -138,55 +140,73 @@ impl PrecomputeEngine { } } -/// Axum handler for Prometheus remote write. -async fn handle_ingest( - State(state): State>, - body: Bytes, +/// Shared logic: group decoded samples by series key and route to workers. +async fn route_decoded_samples( + state: &IngestState, + samples: Vec, + ingest_received_at: Instant, ) -> StatusCode { - let ingest_span = debug_span!("ingest", body_len = body.len()); - let ingest_received_at = Instant::now(); + if samples.is_empty() { + return StatusCode::NO_CONTENT; + } - async { - let samples = match decode_prometheus_remote_write(&body) { - Ok(s) => s, - Err(e) => { - warn!("Failed to decode remote write: {}", e); - return StatusCode::BAD_REQUEST; - } - }; + let count = samples.len() as u64; + state + .samples_ingested + .fetch_add(count, std::sync::atomic::Ordering::Relaxed); + + // Group samples by series key for batch routing + let mut by_series: HashMap<&str, Vec<(i64, f64)>> = HashMap::new(); + for s in &samples { + by_series + .entry(&s.labels) + .or_default() + .push((s.timestamp_ms, s.value)); + } - if samples.is_empty() { - return StatusCode::NO_CONTENT; - } + // Convert to owned keys for batch routing + let by_series_owned: HashMap> = by_series + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect(); - let count = samples.len() as u64; - state - .samples_ingested - .fetch_add(count, std::sync::atomic::Ordering::Relaxed); - - // Group samples by series key for batch routing - let mut by_series: HashMap<&str, Vec<(i64, f64)>> = HashMap::new(); - for s in &samples { - by_series - .entry(&s.labels) - .or_default() - .push((s.timestamp_ms, s.value)); - } + // Route all series to workers concurrently + if let Err(e) = state.router.route_batch(by_series_owned, ingest_received_at).await { + warn!("Batch routing error: {}", e); + return StatusCode::INTERNAL_SERVER_ERROR; + } - // Convert to owned keys for batch routing - let by_series_owned: HashMap> = by_series - .into_iter() - .map(|(k, v)| (k.to_string(), v)) - .collect(); + StatusCode::NO_CONTENT +} - // Route all series to workers concurrently - if let Err(e) = state.router.route_batch(by_series_owned, ingest_received_at).await { - warn!("Batch routing error: {}", e); - return StatusCode::INTERNAL_SERVER_ERROR; +/// Axum handler for Prometheus remote write (Snappy + Protobuf). +async fn handle_prometheus_ingest( + State(state): State>, + body: Bytes, +) -> StatusCode { + let ingest_received_at = Instant::now(); + let samples = match decode_prometheus_remote_write(&body) { + Ok(s) => s, + Err(e) => { + warn!("Failed to decode Prometheus remote write: {}", e); + return StatusCode::BAD_REQUEST; } + }; + route_decoded_samples(&state, samples, ingest_received_at).await +} - StatusCode::NO_CONTENT - } - .instrument(ingest_span) - .await +/// Axum handler for VictoriaMetrics remote write (Zstd + Protobuf). +async fn handle_victoriametrics_ingest( + State(state): State>, + body: Bytes, +) -> StatusCode { + let ingest_received_at = Instant::now(); + let samples = match decode_victoriametrics_remote_write(&body) { + Ok(s) => s, + Err(e) => { + warn!("Failed to decode VictoriaMetrics remote write: {}", e); + return StatusCode::BAD_REQUEST; + } + }; + route_decoded_samples(&state, samples, ingest_received_at).await } diff --git a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md index 64d6955..d487767 100644 --- a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md +++ b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md @@ -3,10 +3,10 @@ ## 1. Overview The Precompute Engine is a real-time streaming aggregation system that sits between -Prometheus-compatible metric producers and ASAP storage and query engine. It accepts raw -time-series samples via the Prometheus remote-write protocol ingestion connector, buffers them, computes -windowed aggregations (sketches), and writes the results to a store for fast -query-time retrieval. +metric producers and ASAP storage and query engine. It accepts raw +time-series samples via multiple ingestion connectors (Prometheus remote write +and VictoriaMetrics remote write), buffers them, computes windowed aggregations +(sketches), and writes the results to a store for fast query-time retrieval. **Key properties:** - Single-machine, multi-threaded architecture (all workers run as async tasks within one process) @@ -19,12 +19,17 @@ query-time retrieval. ## 2. Architecture ``` - Prometheus Remote Write - | + Prometheus Remote Write VictoriaMetrics Remote Write + (Snappy + Protobuf) (Zstd + Protobuf) + | | + v v + POST /api/v1/write POST /api/v1/import + \ / + \ / Axum HTTP Server (:9090) - POST /api/v1/write | - decode + group by series + route_decoded_samples() + (group by series key) | SeriesRouter (hash) / | \ @@ -57,7 +62,7 @@ Top-level orchestrator. On `run()`: 3. Spawns `Worker` tasks, each owning its receiver. 4. Spawns a flush timer that calls `router.broadcast_flush()` every `flush_interval_ms`. -5. Starts the Axum HTTP server and blocks until shutdown. +5. Starts the Axum HTTP server with routes for each ingest connector and blocks until shutdown. ```rust pub struct PrecomputeEngine { @@ -362,7 +367,7 @@ The current implementation is **single-machine, multi-threaded**. All components tasks on a shared thread pool. There is no distributed coordination, no cross-machine communication, and no external dependency beyond the store. -- **Ingest HTTP handler**: Axum async handler, stateless decoding. +- **Ingest HTTP handlers**: Per-connector Axum async handlers (Prometheus, VictoriaMetrics) with shared format-agnostic routing logic. - **SeriesRouter**: Lock-free hash routing. No shared mutable state. - **Workers**: Each worker is a single Tokio task that owns its `series_map` exclusively. No locks needed within a worker — thread safety comes from From 222582b0e37fd6d5565a2270279c5435c51cf183 Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 15:43:30 -0700 Subject: [PATCH 14/38] Run cargo fmt Co-Authored-By: Claude Opus 4.6 --- .../src/bin/precompute_engine.rs | 15 ++-- .../src/bin/test_e2e_precompute.rs | 70 ++++++++++--------- asap-query-engine/src/main.rs | 7 +- .../precompute_engine/accumulator_factory.rs | 18 ++--- .../src/precompute_engine/mod.rs | 14 ++-- .../src/precompute_engine/series_router.rs | 18 ++--- .../src/precompute_engine/window_manager.rs | 5 +- .../src/precompute_engine/worker.rs | 44 ++++-------- 8 files changed, 83 insertions(+), 108 deletions(-) diff --git a/asap-query-engine/src/bin/precompute_engine.rs b/asap-query-engine/src/bin/precompute_engine.rs index b634c85..d5944a0 100644 --- a/asap-query-engine/src/bin/precompute_engine.rs +++ b/asap-query-engine/src/bin/precompute_engine.rs @@ -1,4 +1,5 @@ use clap::Parser; +use query_engine_rust::data_model::QueryLanguage; use query_engine_rust::data_model::{ CleanupPolicy, InferenceConfig, LockStrategy, StreamingConfig, }; @@ -9,7 +10,6 @@ use query_engine_rust::precompute_engine::output_sink::{RawPassthroughSink, Stor use query_engine_rust::precompute_engine::PrecomputeEngine; use query_engine_rust::stores::SimpleMapStore; use query_engine_rust::{HttpServer, HttpServerConfig}; -use query_engine_rust::data_model::QueryLanguage; use std::sync::Arc; use tracing::info; use tracing_subscriber::fmt::format::FmtSpan; @@ -89,20 +89,17 @@ async fn main() -> Result<(), Box> { ); // Create the store - let store: Arc = Arc::new( - SimpleMapStore::new_with_strategy( + let store: Arc = + Arc::new(SimpleMapStore::new_with_strategy( streaming_config.clone(), CleanupPolicy::CircularBuffer, args.lock_strategy, - ), - ); + )); // Optionally start the query HTTP server if args.query_port > 0 { - let inference_config = InferenceConfig::new( - QueryLanguage::promql, - CleanupPolicy::CircularBuffer, - ); + let inference_config = + InferenceConfig::new(QueryLanguage::promql, CleanupPolicy::CircularBuffer); let query_engine = Arc::new(SimpleEngine::new( store.clone(), inference_config, diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index 668d819..77e41a5 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -78,8 +78,10 @@ async fn main() -> Result<(), Box> { .init(); // Load configs the same way main.rs does - let inference_config = - read_inference_config("examples/promql/inference_config.yaml", QueryLanguage::promql)?; + let inference_config = read_inference_config( + "examples/promql/inference_config.yaml", + QueryLanguage::promql, + )?; println!( "Loaded inference config with {} query configs", inference_config.query_configs.len() @@ -101,13 +103,12 @@ async fn main() -> Result<(), Box> { println!("\n=== Starting precompute engine (ingest={INGEST_PORT}, query={QUERY_PORT}) ==="); // Create store - let store: Arc = Arc::new( - SimpleMapStore::new_with_strategy( + let store: Arc = + Arc::new(SimpleMapStore::new_with_strategy( streaming_config.clone(), cleanup_policy, LockStrategy::PerKey, - ), - ); + )); // Start query server let query_engine = Arc::new(SimpleEngine::new( @@ -208,9 +209,21 @@ async fn main() -> Result<(), Box> { // Use the exact query pattern from inference_config let queries_instant = vec![ - ("quantile by (label_0) (0.99, fake_metric)", "10", "Configured query at t=10"), - ("quantile by (label_0) (0.99, fake_metric)", "15", "Configured query at t=15"), - ("sum_over_time(fake_metric[1s])", "10", "Temporal: sum_over_time at t=10"), + ( + "quantile by (label_0) (0.99, fake_metric)", + "10", + "Configured query at t=10", + ), + ( + "quantile by (label_0) (0.99, fake_metric)", + "15", + "Configured query at t=15", + ), + ( + "sum_over_time(fake_metric[1s])", + "10", + "Temporal: sum_over_time at t=10", + ), ("sum(fake_metric)", "10", "Spatial: sum at t=10"), ]; @@ -274,11 +287,7 @@ async fn main() -> Result<(), Box> { late_data_policy: LateDataPolicy::Drop, }; let raw_sink = Arc::new(RawPassthroughSink::new(store.clone())); - let raw_engine = PrecomputeEngine::new( - raw_engine_config, - streaming_config.clone(), - raw_sink, - ); + let raw_engine = PrecomputeEngine::new(raw_engine_config, streaming_config.clone(), raw_sink); tokio::spawn(async move { if let Err(e) = raw_engine.run().await { eprintln!("Raw precompute engine error: {e}"); @@ -301,7 +310,10 @@ async fn main() -> Result<(), Box> { .body(body) .send() .await?; - println!(" Sent raw t={ts}ms v={val} -> HTTP {}", resp.status().as_u16()); + println!( + " Sent raw t={ts}ms v={val} -> HTTP {}", + resp.status().as_u16() + ); } // Short wait for processing (no watermark advancement needed) @@ -309,12 +321,7 @@ async fn main() -> Result<(), Box> { // Verify raw samples appeared in the store println!("\n=== Verifying raw samples in store ==="); - let results = store.query_precomputed_output( - "fake_metric", - raw_agg_id, - 100_000, - 103_000, - )?; + let results = store.query_precomputed_output("fake_metric", raw_agg_id, 100_000, 103_000)?; let total_buckets: usize = results.values().map(|v| v.len()).sum(); println!(" Found {total_buckets} buckets for aggregation_id={raw_agg_id} in [100000, 103000)"); assert!( @@ -348,7 +355,10 @@ async fn main() -> Result<(), Box> { } } let batch_body = build_remote_write_body(batch_timeseries); - println!(" Payload size: {} bytes (snappy-compressed)", batch_body.len()); + println!( + " Payload size: {} bytes (snappy-compressed)", + batch_body.len() + ); let t0 = std::time::Instant::now(); let resp = client @@ -369,12 +379,8 @@ async fn main() -> Result<(), Box> { tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; // Verify samples landed in the store - let batch_results = store.query_precomputed_output( - "fake_metric", - raw_agg_id, - 200_000, - 210_000, - )?; + let batch_results = + store.query_precomputed_output("fake_metric", raw_agg_id, 200_000, 210_000)?; let batch_buckets: usize = batch_results.values().map(|v| v.len()).sum(); println!(" Stored {batch_buckets} buckets from batch (expected 1000)"); assert!( @@ -433,12 +439,8 @@ async fn main() -> Result<(), Box> { let drain_deadline = std::time::Instant::now() + std::time::Duration::from_secs(60); let mut tp_buckets: usize; loop { - let tp_results = store.query_precomputed_output( - "fake_metric", - raw_agg_id, - 300_000, - max_ts, - )?; + let tp_results = + store.query_precomputed_output("fake_metric", raw_agg_id, 300_000, max_ts)?; tp_buckets = tp_results.values().map(|v| v.len()).sum(); if tp_buckets as u64 >= total_samples || std::time::Instant::now() > drain_deadline { break; diff --git a/asap-query-engine/src/main.rs b/asap-query-engine/src/main.rs index f32cecb..57ebbcf 100644 --- a/asap-query-engine/src/main.rs +++ b/asap-query-engine/src/main.rs @@ -248,11 +248,8 @@ async fn main() -> Result<()> { raw_mode_aggregation_id: 0, }; let output_sink = Arc::new(StoreOutputSink::new(store.clone())); - let engine = PrecomputeEngine::new( - precompute_config, - streaming_config.clone(), - output_sink, - ); + let engine = + PrecomputeEngine::new(precompute_config, streaming_config.clone(), output_sink); info!( "Starting precompute engine on port {}", args.prometheus_remote_write_port diff --git a/asap-query-engine/src/precompute_engine/accumulator_factory.rs b/asap-query-engine/src/precompute_engine/accumulator_factory.rs index 47e66a4..ab2b58e 100644 --- a/asap-query-engine/src/precompute_engine/accumulator_factory.rs +++ b/asap-query-engine/src/precompute_engine/accumulator_factory.rs @@ -1,8 +1,8 @@ use crate::data_model::{AggregateCore, KeyByLabelValues, Measurement}; use crate::precompute_operators::{ CountMinSketchAccumulator, DatasketchesKLLAccumulator, HydraKllSketchAccumulator, - IncreaseAccumulator, MinMaxAccumulator, MultipleIncreaseAccumulator, - MultipleMinMaxAccumulator, MultipleSumAccumulator, SumAccumulator, + IncreaseAccumulator, MinMaxAccumulator, MultipleIncreaseAccumulator, MultipleMinMaxAccumulator, + MultipleSumAccumulator, SumAccumulator, }; use sketch_db_common::aggregation_config::AggregationConfig; @@ -156,12 +156,7 @@ impl AccumulatorUpdater for IncreaseAccumulatorUpdater { fn take_accumulator(&mut self) -> Box { let acc = self.acc.take().unwrap_or_else(|| { - IncreaseAccumulator::new( - Measurement::new(0.0), - 0, - Measurement::new(0.0), - 0, - ) + IncreaseAccumulator::new(Measurement::new(0.0), 0, Measurement::new(0.0), 0) }); let result = Box::new(acc); self.reset(); @@ -494,8 +489,7 @@ impl AccumulatorUpdater for HydraKllAccumulatorUpdater { fn memory_usage_bytes(&self) -> usize { // Rough estimate: each cell is a KLL sketch - std::mem::size_of::() - + self.row_num * self.col_num * 4096 + std::mem::size_of::() + self.row_num * self.col_num * 4096 } } @@ -504,9 +498,7 @@ impl AccumulatorUpdater for HydraKllAccumulatorUpdater { // --------------------------------------------------------------------------- /// Create an appropriate `AccumulatorUpdater` from an `AggregationConfig`. -pub fn create_accumulator_updater( - config: &AggregationConfig, -) -> Box { +pub fn create_accumulator_updater(config: &AggregationConfig) -> Box { let agg_type = config.aggregation_type.as_str(); let sub_type = config.aggregation_sub_type.as_str(); diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs index f114ef8..5801724 100644 --- a/asap-query-engine/src/precompute_engine/mod.rs +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -16,9 +16,9 @@ use crate::precompute_engine::worker::Worker; use axum::{body::Bytes, extract::State, http::StatusCode, routing::post, Router}; use std::collections::HashMap; use std::sync::Arc; +use std::time::Instant; use tokio::net::TcpListener; use tokio::sync::mpsc; -use std::time::Instant; use tracing::{info, warn}; /// Shared state for the ingest HTTP handler. @@ -68,10 +68,8 @@ impl PrecomputeEngine { let router = SeriesRouter::new(senders); // Build aggregation config map from streaming config - let agg_configs: HashMap = self - .streaming_config - .get_all_aggregation_configs() - .clone(); + let agg_configs: HashMap = + self.streaming_config.get_all_aggregation_configs().clone(); // Spawn workers let mut worker_handles = Vec::with_capacity(num_workers); @@ -171,7 +169,11 @@ async fn route_decoded_samples( .collect(); // Route all series to workers concurrently - if let Err(e) = state.router.route_batch(by_series_owned, ingest_received_at).await { + if let Err(e) = state + .router + .route_batch(by_series_owned, ingest_received_at) + .await + { warn!("Batch routing error: {}", e); return StatusCode::INTERNAL_SERVER_ERROR; } diff --git a/asap-query-engine/src/precompute_engine/series_router.rs b/asap-query-engine/src/precompute_engine/series_router.rs index a159b02..1cb4fca 100644 --- a/asap-query-engine/src/precompute_engine/series_router.rs +++ b/asap-query-engine/src/precompute_engine/series_router.rs @@ -1,6 +1,6 @@ +use futures::future::try_join_all; use std::collections::HashMap; use std::time::Instant; -use futures::future::try_join_all; use tokio::sync::mpsc; use xxhash_rust::xxh64::xxh64; @@ -82,15 +82,18 @@ impl SeriesRouter { let sender = &self.senders[worker_idx]; async move { for msg in messages { - sender.send(msg).await.map_err(|e| { - format!("Failed to send to worker {}: {}", worker_idx, e) - })?; + sender + .send(msg) + .await + .map_err(|e| format!("Failed to send to worker {}: {}", worker_idx, e))?; } Ok::<(), String>(()) } })) .await - .map_err(|e| -> Box { Box::new(std::io::Error::new(std::io::ErrorKind::Other, e)) })?; + .map_err(|e| -> Box { + Box::new(std::io::Error::new(std::io::ErrorKind::Other, e)) + })?; Ok(()) } @@ -131,9 +134,8 @@ mod tests { #[test] fn test_consistent_routing() { // Build a router with dummy senders (we only test the hash logic) - let (senders, _receivers): (Vec<_>, Vec<_>) = (0..4) - .map(|_| mpsc::channel::(10)) - .unzip(); + let (senders, _receivers): (Vec<_>, Vec<_>) = + (0..4).map(|_| mpsc::channel::(10)).unzip(); let router = SeriesRouter::new(senders); diff --git a/asap-query-engine/src/precompute_engine/window_manager.rs b/asap-query-engine/src/precompute_engine/window_manager.rs index e397fb7..d8bf23b 100644 --- a/asap-query-engine/src/precompute_engine/window_manager.rs +++ b/asap-query-engine/src/precompute_engine/window_manager.rs @@ -57,9 +57,8 @@ impl WindowManager { // A window is open if its end (start + window_size_ms) > previous_wm. // So the oldest open window start was: previous_wm - window_size_ms + 1, // aligned down to slide_interval. - let earliest_open_start = self.window_start_for( - (previous_wm - self.window_size_ms + 1).max(0), - ); + let earliest_open_start = + self.window_start_for((previous_wm - self.window_size_ms + 1).max(0)); let mut start = earliest_open_start; while start + self.window_size_ms <= current_wm { diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs index 241362f..f59f691 100644 --- a/asap-query-engine/src/precompute_engine/worker.rs +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -1,5 +1,7 @@ use crate::data_model::{AggregateCore, KeyByLabelValues, PrecomputedOutput}; -use crate::precompute_engine::accumulator_factory::{create_accumulator_updater, AccumulatorUpdater}; +use crate::precompute_engine::accumulator_factory::{ + create_accumulator_updater, AccumulatorUpdater, +}; use crate::precompute_engine::config::LateDataPolicy; use crate::precompute_engine::output_sink::OutputSink; use crate::precompute_engine::series_buffer::SeriesBuffer; @@ -149,10 +151,7 @@ impl Worker { let aggregations = matching .into_iter() .map(|(_, config)| AggregationState { - window_manager: WindowManager::new( - config.window_size, - config.slide_interval, - ), + window_manager: WindowManager::new(config.window_size, config.slide_interval), config: config.clone(), active_windows: HashMap::new(), }) @@ -248,7 +247,8 @@ impl Worker { LateDataPolicy::ForwardToStore => { let mut updater = create_accumulator_updater(&agg_state.config); if updater.is_keyed() { - let key = extract_key_from_series(series_key, &agg_state.config); + let key = + extract_key_from_series(series_key, &agg_state.config); updater.update_keyed(&key, val, ts); } else { updater.update_single(val, ts); @@ -292,8 +292,7 @@ impl Worker { // Emit closed windows for window_start in &closed { if let Some(mut updater) = agg_state.active_windows.remove(window_start) { - let (_, window_end) = - agg_state.window_manager.window_bounds(*window_start); + let (_, window_end) = agg_state.window_manager.window_bounds(*window_start); let key = if updater.is_keyed() { Some(extract_key_from_series(series_key, &agg_state.config)) @@ -340,12 +339,8 @@ impl Worker { Vec::with_capacity(samples.len()); for (ts, val) in samples { - let output = PrecomputedOutput::new( - ts as u64, - ts as u64, - None, - self.raw_mode_aggregation_id, - ); + let output = + PrecomputedOutput::new(ts as u64, ts as u64, None, self.raw_mode_aggregation_id); let accumulator = SumAccumulator::with_sum(val); emit_batch.push((output, Box::new(accumulator))); } @@ -382,14 +377,10 @@ impl Worker { for window_start in &closed { if let Some(mut updater) = agg_state.active_windows.remove(window_start) { - let (_, window_end) = - agg_state.window_manager.window_bounds(*window_start); + let (_, window_end) = agg_state.window_manager.window_bounds(*window_start); let key = if updater.is_keyed() { - Some(extract_key_from_series( - series_key, - &agg_state.config, - )) + Some(extract_key_from_series(series_key, &agg_state.config)) } else { None }; @@ -435,10 +426,7 @@ pub fn extract_metric_name(series_key: &str) -> &str { /// aggregation config's `grouping_labels`. /// /// The series key format is: `metric_name{label1="val1",label2="val2",...}` -pub fn extract_key_from_series( - series_key: &str, - config: &AggregationConfig, -) -> KeyByLabelValues { +pub fn extract_key_from_series(series_key: &str, config: &AggregationConfig) -> KeyByLabelValues { let labels = parse_labels_from_series_key(series_key); let mut values = Vec::new(); @@ -530,8 +518,7 @@ mod tests { #[test] fn test_parse_labels() { - let labels = - parse_labels_from_series_key("metric{method=\"GET\",status=\"200\"}"); + let labels = parse_labels_from_series_key("metric{method=\"GET\",status=\"200\"}"); assert_eq!(labels.get("method"), Some(&"GET")); assert_eq!(labels.get("status"), Some(&"200")); } @@ -580,9 +567,6 @@ mod tests { "http_requests_total{method=\"GET\",status=\"200\"}", &config, ); - assert_eq!( - key.labels, - vec!["GET".to_string(), "200".to_string()] - ); + assert_eq!(key.labels, vec!["GET".to_string(), "200".to_string()]); } } From 2b120e5e0c3065a44aaf35a7fa61022a0951ed74 Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 16:16:27 -0700 Subject: [PATCH 15/38] Add pane-based incremental sliding window computation Route each sample to exactly 1 pane (sub-window of size slide_interval) instead of W overlapping window accumulators. When a window closes, merge its constituent panes via AggregateCore::merge_with(). This reduces per-sample accumulator updates from W to 1 for sliding windows. - Add snapshot_accumulator() to AccumulatorUpdater trait (9 implementations) - Add pane_start_for(), panes_for_window(), slide_interval_ms() to WindowManager - Replace active_windows HashMap with active_panes BTreeMap in worker - Rewrite sample routing and window close logic with pane merging - Extract PrecomputeEngine to engine.rs, ingest handlers to ingest_handler.rs - Update design doc with pane-based architecture Co-Authored-By: Claude Opus 4.6 --- .../precompute_engine/accumulator_factory.rs | 48 ++++ .../src/precompute_engine/engine.rs | 125 ++++++++++ .../src/precompute_engine/ingest_handler.rs | 89 +++++++ .../src/precompute_engine/mod.rs | 209 +---------------- .../precompute_engine_design_doc.md | 142 ++++++++++- .../src/precompute_engine/window_manager.rs | 96 ++++++++ .../src/precompute_engine/worker.rs | 220 ++++++++++++------ 7 files changed, 636 insertions(+), 293 deletions(-) create mode 100644 asap-query-engine/src/precompute_engine/engine.rs create mode 100644 asap-query-engine/src/precompute_engine/ingest_handler.rs diff --git a/asap-query-engine/src/precompute_engine/accumulator_factory.rs b/asap-query-engine/src/precompute_engine/accumulator_factory.rs index ab2b58e..66e6bb9 100644 --- a/asap-query-engine/src/precompute_engine/accumulator_factory.rs +++ b/asap-query-engine/src/precompute_engine/accumulator_factory.rs @@ -20,6 +20,10 @@ pub trait AccumulatorUpdater: Send { /// Extract the final accumulator as a boxed `AggregateCore`. fn take_accumulator(&mut self) -> Box; + /// Non-destructive read of the current accumulator state (clone without reset). + /// Used by pane-based sliding windows to read shared panes. + fn snapshot_accumulator(&self) -> Box; + /// Reset internal state for reuse (avoids re-allocation). fn reset(&mut self); @@ -61,6 +65,10 @@ impl AccumulatorUpdater for SumAccumulatorUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = SumAccumulator::new(); } @@ -107,6 +115,10 @@ impl AccumulatorUpdater for MinMaxAccumulatorUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = MinMaxAccumulator::new(self.sub_type.clone()); } @@ -163,6 +175,18 @@ impl AccumulatorUpdater for IncreaseAccumulatorUpdater { result } + fn snapshot_accumulator(&self) -> Box { + match &self.acc { + Some(acc) => Box::new(acc.clone()), + None => Box::new(IncreaseAccumulator::new( + Measurement::new(0.0), + 0, + Measurement::new(0.0), + 0, + )), + } + } + fn reset(&mut self) { self.acc = None; } @@ -209,6 +233,10 @@ impl AccumulatorUpdater for KllAccumulatorUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = DatasketchesKLLAccumulator::new(self.k); } @@ -254,6 +282,10 @@ impl AccumulatorUpdater for MultipleSumUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = MultipleSumAccumulator::new(); } @@ -301,6 +333,10 @@ impl AccumulatorUpdater for MultipleMinMaxUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = MultipleMinMaxAccumulator::new(self.sub_type.clone()); } @@ -360,6 +396,10 @@ impl AccumulatorUpdater for MultipleIncreaseUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = MultipleIncreaseAccumulator::new(); } @@ -428,6 +468,10 @@ impl AccumulatorUpdater for CmsAccumulatorUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = CountMinSketchAccumulator::new(self.row_num, self.col_num); } @@ -479,6 +523,10 @@ impl AccumulatorUpdater for HydraKllAccumulatorUpdater { result } + fn snapshot_accumulator(&self) -> Box { + Box::new(self.acc.clone()) + } + fn reset(&mut self) { self.acc = HydraKllSketchAccumulator::new(self.row_num, self.col_num, self.k); } diff --git a/asap-query-engine/src/precompute_engine/engine.rs b/asap-query-engine/src/precompute_engine/engine.rs new file mode 100644 index 0000000..5df148c --- /dev/null +++ b/asap-query-engine/src/precompute_engine/engine.rs @@ -0,0 +1,125 @@ +use crate::data_model::StreamingConfig; +use crate::precompute_engine::config::PrecomputeEngineConfig; +use crate::precompute_engine::ingest_handler::{ + handle_prometheus_ingest, handle_victoriametrics_ingest, IngestState, +}; +use crate::precompute_engine::output_sink::OutputSink; +use crate::precompute_engine::series_router::{SeriesRouter, WorkerMessage}; +use crate::precompute_engine::worker::Worker; +use axum::{routing::post, Router}; +use std::collections::HashMap; +use std::sync::Arc; +use tokio::net::TcpListener; +use tokio::sync::mpsc; +use tracing::{info, warn}; + +/// The top-level precompute engine orchestrator. +/// +/// Creates worker threads, the series router, and the Axum ingest server. +pub struct PrecomputeEngine { + config: PrecomputeEngineConfig, + streaming_config: Arc, + output_sink: Arc, +} + +impl PrecomputeEngine { + pub fn new( + config: PrecomputeEngineConfig, + streaming_config: Arc, + output_sink: Arc, + ) -> Self { + Self { + config, + streaming_config, + output_sink, + } + } + + /// Start the precompute engine. This spawns worker tasks and the HTTP + /// ingest server, then blocks until shutdown. + pub async fn run(self) -> Result<(), Box> { + let num_workers = self.config.num_workers; + let channel_size = self.config.channel_buffer_size; + + // Build MPSC channels for each worker + let mut senders = Vec::with_capacity(num_workers); + let mut receivers = Vec::with_capacity(num_workers); + for _ in 0..num_workers { + let (tx, rx) = mpsc::channel::(channel_size); + senders.push(tx); + receivers.push(rx); + } + + // Build the router + let router = SeriesRouter::new(senders); + + // Build aggregation config map from streaming config + let agg_configs: HashMap = + self.streaming_config.get_all_aggregation_configs().clone(); + + // Spawn workers + let mut worker_handles = Vec::with_capacity(num_workers); + for (id, rx) in receivers.into_iter().enumerate() { + let worker = Worker::new( + id, + rx, + self.output_sink.clone(), + agg_configs.clone(), + self.config.max_buffer_per_series, + self.config.allowed_lateness_ms, + self.config.pass_raw_samples, + self.config.raw_mode_aggregation_id, + self.config.late_data_policy, + ); + let handle = tokio::spawn(async move { + worker.run().await; + }); + worker_handles.push(handle); + } + + info!( + "PrecomputeEngine started with {} workers on port {}", + num_workers, self.config.ingest_port + ); + + // Build the ingest state + let ingest_state = Arc::new(IngestState { + router, + samples_ingested: std::sync::atomic::AtomicU64::new(0), + }); + + // Start flush timer + let flush_state = ingest_state.clone(); + let flush_interval_ms = self.config.flush_interval_ms; + tokio::spawn(async move { + let mut interval = + tokio::time::interval(tokio::time::Duration::from_millis(flush_interval_ms)); + loop { + interval.tick().await; + if let Err(e) = flush_state.router.broadcast_flush().await { + warn!("Flush broadcast error: {}", e); + break; + } + } + }); + + // Start the Axum HTTP server for ingest (Prometheus + VictoriaMetrics) + let app = Router::new() + .route("/api/v1/write", post(handle_prometheus_ingest)) + .route("/api/v1/import", post(handle_victoriametrics_ingest)) + .with_state(ingest_state); + + let addr = format!("0.0.0.0:{}", self.config.ingest_port); + info!("Ingest server listening on {}", addr); + + let listener = TcpListener::bind(&addr).await?; + axum::serve(listener, app).await?; + + // Wait for workers to finish (this only happens on shutdown) + for handle in worker_handles { + let _ = handle.await; + } + + Ok(()) + } +} diff --git a/asap-query-engine/src/precompute_engine/ingest_handler.rs b/asap-query-engine/src/precompute_engine/ingest_handler.rs new file mode 100644 index 0000000..57537dd --- /dev/null +++ b/asap-query-engine/src/precompute_engine/ingest_handler.rs @@ -0,0 +1,89 @@ +use crate::drivers::ingest::prometheus_remote_write::decode_prometheus_remote_write; +use crate::drivers::ingest::victoriametrics_remote_write::decode_victoriametrics_remote_write; +use crate::precompute_engine::series_router::SeriesRouter; +use axum::{body::Bytes, extract::State, http::StatusCode}; +use std::collections::HashMap; +use std::sync::Arc; +use std::time::Instant; +use tracing::warn; + +/// Shared state for the ingest HTTP handler. +pub(crate) struct IngestState { + pub(crate) router: SeriesRouter, + pub(crate) samples_ingested: std::sync::atomic::AtomicU64, +} + +/// Shared logic: group decoded samples by series key and route to workers. +async fn route_decoded_samples( + state: &IngestState, + samples: Vec, + ingest_received_at: Instant, +) -> StatusCode { + if samples.is_empty() { + return StatusCode::NO_CONTENT; + } + + let count = samples.len() as u64; + state + .samples_ingested + .fetch_add(count, std::sync::atomic::Ordering::Relaxed); + + // Group samples by series key for batch routing + let mut by_series: HashMap<&str, Vec<(i64, f64)>> = HashMap::new(); + for s in &samples { + by_series + .entry(&s.labels) + .or_default() + .push((s.timestamp_ms, s.value)); + } + + // Convert to owned keys for batch routing + let by_series_owned: HashMap> = by_series + .into_iter() + .map(|(k, v)| (k.to_string(), v)) + .collect(); + + // Route all series to workers concurrently + if let Err(e) = state + .router + .route_batch(by_series_owned, ingest_received_at) + .await + { + warn!("Batch routing error: {}", e); + return StatusCode::INTERNAL_SERVER_ERROR; + } + + StatusCode::NO_CONTENT +} + +/// Axum handler for Prometheus remote write (Snappy + Protobuf). +pub(crate) async fn handle_prometheus_ingest( + State(state): State>, + body: Bytes, +) -> StatusCode { + let ingest_received_at = Instant::now(); + let samples = match decode_prometheus_remote_write(&body) { + Ok(s) => s, + Err(e) => { + warn!("Failed to decode Prometheus remote write: {}", e); + return StatusCode::BAD_REQUEST; + } + }; + route_decoded_samples(&state, samples, ingest_received_at).await +} + +/// Axum handler for VictoriaMetrics remote write (Zstd + Protobuf). +pub(crate) async fn handle_victoriametrics_ingest( + State(state): State>, + body: Bytes, +) -> StatusCode { + let ingest_received_at = Instant::now(); + let samples = match decode_victoriametrics_remote_write(&body) { + Ok(s) => s, + Err(e) => { + warn!("Failed to decode VictoriaMetrics remote write: {}", e); + return StatusCode::BAD_REQUEST; + } + }; + route_decoded_samples(&state, samples, ingest_received_at).await +} diff --git a/asap-query-engine/src/precompute_engine/mod.rs b/asap-query-engine/src/precompute_engine/mod.rs index 5801724..7b960ca 100644 --- a/asap-query-engine/src/precompute_engine/mod.rs +++ b/asap-query-engine/src/precompute_engine/mod.rs @@ -1,214 +1,11 @@ pub mod accumulator_factory; pub mod config; +mod engine; +mod ingest_handler; pub mod output_sink; pub mod series_buffer; pub mod series_router; pub mod window_manager; pub mod worker; -use crate::data_model::StreamingConfig; -use crate::drivers::ingest::prometheus_remote_write::decode_prometheus_remote_write; -use crate::drivers::ingest::victoriametrics_remote_write::decode_victoriametrics_remote_write; -use crate::precompute_engine::config::PrecomputeEngineConfig; -use crate::precompute_engine::output_sink::OutputSink; -use crate::precompute_engine::series_router::{SeriesRouter, WorkerMessage}; -use crate::precompute_engine::worker::Worker; -use axum::{body::Bytes, extract::State, http::StatusCode, routing::post, Router}; -use std::collections::HashMap; -use std::sync::Arc; -use std::time::Instant; -use tokio::net::TcpListener; -use tokio::sync::mpsc; -use tracing::{info, warn}; - -/// Shared state for the ingest HTTP handler. -struct IngestState { - router: SeriesRouter, - samples_ingested: std::sync::atomic::AtomicU64, -} - -/// The top-level precompute engine orchestrator. -/// -/// Creates worker threads, the series router, and the Axum ingest server. -pub struct PrecomputeEngine { - config: PrecomputeEngineConfig, - streaming_config: Arc, - output_sink: Arc, -} - -impl PrecomputeEngine { - pub fn new( - config: PrecomputeEngineConfig, - streaming_config: Arc, - output_sink: Arc, - ) -> Self { - Self { - config, - streaming_config, - output_sink, - } - } - - /// Start the precompute engine. This spawns worker tasks and the HTTP - /// ingest server, then blocks until shutdown. - pub async fn run(self) -> Result<(), Box> { - let num_workers = self.config.num_workers; - let channel_size = self.config.channel_buffer_size; - - // Build MPSC channels for each worker - let mut senders = Vec::with_capacity(num_workers); - let mut receivers = Vec::with_capacity(num_workers); - for _ in 0..num_workers { - let (tx, rx) = mpsc::channel::(channel_size); - senders.push(tx); - receivers.push(rx); - } - - // Build the router - let router = SeriesRouter::new(senders); - - // Build aggregation config map from streaming config - let agg_configs: HashMap = - self.streaming_config.get_all_aggregation_configs().clone(); - - // Spawn workers - let mut worker_handles = Vec::with_capacity(num_workers); - for (id, rx) in receivers.into_iter().enumerate() { - let worker = Worker::new( - id, - rx, - self.output_sink.clone(), - agg_configs.clone(), - self.config.max_buffer_per_series, - self.config.allowed_lateness_ms, - self.config.pass_raw_samples, - self.config.raw_mode_aggregation_id, - self.config.late_data_policy, - ); - let handle = tokio::spawn(async move { - worker.run().await; - }); - worker_handles.push(handle); - } - - info!( - "PrecomputeEngine started with {} workers on port {}", - num_workers, self.config.ingest_port - ); - - // Build the ingest state - let ingest_state = Arc::new(IngestState { - router, - samples_ingested: std::sync::atomic::AtomicU64::new(0), - }); - - // Start flush timer - let flush_state = ingest_state.clone(); - let flush_interval_ms = self.config.flush_interval_ms; - tokio::spawn(async move { - let mut interval = - tokio::time::interval(tokio::time::Duration::from_millis(flush_interval_ms)); - loop { - interval.tick().await; - if let Err(e) = flush_state.router.broadcast_flush().await { - warn!("Flush broadcast error: {}", e); - break; - } - } - }); - - // Start the Axum HTTP server for ingest (Prometheus + VictoriaMetrics) - let app = Router::new() - .route("/api/v1/write", post(handle_prometheus_ingest)) - .route("/api/v1/import", post(handle_victoriametrics_ingest)) - .with_state(ingest_state); - - let addr = format!("0.0.0.0:{}", self.config.ingest_port); - info!("Ingest server listening on {}", addr); - - let listener = TcpListener::bind(&addr).await?; - axum::serve(listener, app).await?; - - // Wait for workers to finish (this only happens on shutdown) - for handle in worker_handles { - let _ = handle.await; - } - - Ok(()) - } -} - -/// Shared logic: group decoded samples by series key and route to workers. -async fn route_decoded_samples( - state: &IngestState, - samples: Vec, - ingest_received_at: Instant, -) -> StatusCode { - if samples.is_empty() { - return StatusCode::NO_CONTENT; - } - - let count = samples.len() as u64; - state - .samples_ingested - .fetch_add(count, std::sync::atomic::Ordering::Relaxed); - - // Group samples by series key for batch routing - let mut by_series: HashMap<&str, Vec<(i64, f64)>> = HashMap::new(); - for s in &samples { - by_series - .entry(&s.labels) - .or_default() - .push((s.timestamp_ms, s.value)); - } - - // Convert to owned keys for batch routing - let by_series_owned: HashMap> = by_series - .into_iter() - .map(|(k, v)| (k.to_string(), v)) - .collect(); - - // Route all series to workers concurrently - if let Err(e) = state - .router - .route_batch(by_series_owned, ingest_received_at) - .await - { - warn!("Batch routing error: {}", e); - return StatusCode::INTERNAL_SERVER_ERROR; - } - - StatusCode::NO_CONTENT -} - -/// Axum handler for Prometheus remote write (Snappy + Protobuf). -async fn handle_prometheus_ingest( - State(state): State>, - body: Bytes, -) -> StatusCode { - let ingest_received_at = Instant::now(); - let samples = match decode_prometheus_remote_write(&body) { - Ok(s) => s, - Err(e) => { - warn!("Failed to decode Prometheus remote write: {}", e); - return StatusCode::BAD_REQUEST; - } - }; - route_decoded_samples(&state, samples, ingest_received_at).await -} - -/// Axum handler for VictoriaMetrics remote write (Zstd + Protobuf). -async fn handle_victoriametrics_ingest( - State(state): State>, - body: Bytes, -) -> StatusCode { - let ingest_received_at = Instant::now(); - let samples = match decode_victoriametrics_remote_write(&body) { - Ok(s) => s, - Err(e) => { - warn!("Failed to decode VictoriaMetrics remote write: {}", e); - return StatusCode::BAD_REQUEST; - } - }; - route_decoded_samples(&state, samples, ingest_received_at).await -} +pub use engine::PrecomputeEngine; diff --git a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md index d487767..fd181f6 100644 --- a/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md +++ b/asap-query-engine/src/precompute_engine/precompute_engine_design_doc.md @@ -146,10 +146,54 @@ struct SeriesState { struct AggregationState { config: AggregationConfig, window_manager: WindowManager, - active_windows: HashMap>, + active_panes: BTreeMap>, } ``` +#### Pane-Based Sliding Window Optimization + +The worker uses **pane-based incremental computation** to reduce per-sample +work for sliding windows. The timeline is divided into non-overlapping **panes** +of size `slide_interval`. Each window is composed of `W = window_size / +slide_interval` consecutive panes. Consecutive windows share W-1 panes. + +``` +Panes: [0,10) [10,20) [20,30) [30,40) [40,50) +Window A: [───────── 0,30 ─────────) +Window B: [───────── 10,40 ─────────) +Window C: [───────── 20,50 ─────────) +``` + +**Why panes instead of subtraction?** Only Sum is invertible. MinMax, Increase, +KLL, CMS, HydraKLL are all non-invertible. Pane+merge works universally because +all accumulator types implement `AggregateCore::merge_with()`. + +**Performance comparison** (N samples per window, W = window_size / slide_interval): + +| | Per-window approach | Pane-based | +|--|---------|------------| +| Per-sample accumulator updates | N × W | N × 1 | +| Per-window-close merges | 0 | W - 1 | +| Per-window-close clones | 0 | W - 2 (shared panes) | + +Net win when N >> W (typical: thousands of samples per window, W = 3-5). +For tumbling windows (W=1), panes degenerate to 1 pane = 1 window with +zero merges — identical behavior to the non-pane approach. + +**Key methods:** + +| Method | Description | +|--------|-------------| +| `pane_start_for(ts)` | Align timestamp to slide grid (same as `window_start_for`) | +| `panes_for_window(ws)` | All pane starts composing window `[ws, ws+size)` | +| `snapshot_accumulator()` | Non-destructive read of a pane's accumulator (for shared panes) | + +**Pane eviction:** When window `[S, S+W)` closes, pane `[S, S+slide)` is the +oldest pane and is not needed by any later window (next window starts at +`S+slide`). It is destructively consumed via `take_accumulator()` and removed +from `active_panes`. Remaining panes are read non-destructively via +`snapshot_accumulator()`. + #### Processing pipeline (`process_samples`) ``` @@ -157,14 +201,18 @@ struct AggregationState { 2. Insert samples into SeriesBuffer, update watermark 3. Drop samples beyond allowed_lateness_ms behind watermark 4. For each sample × each aggregation: - a. Compute window_starts_containing(ts) - b. For each window_start: - - If window not in active_windows AND already closed (watermark >= window_end): - → late_data_policy == Drop: skip - → late_data_policy == ForwardToStore: create mini-accumulator, emit - - Else: insert into active_windows, feed value to AccumulatorUpdater + a. Compute pane_start = pane_start_for(ts) + b. If pane was evicted (late data for closed window): + → late_data_policy == Drop: skip + → late_data_policy == ForwardToStore: create mini-accumulator, emit + c. Else: get-or-create pane in active_panes, feed value (1 update per sample) 5. Detect newly closed windows via closed_windows(prev_wm, current_wm) -6. Extract closed window accumulators → PrecomputedOutput + AggregateCore +6. For each closed window: + a. Get pane starts via panes_for_window(window_start) + b. Oldest pane: take_accumulator() + remove from active_panes (destructive) + c. Remaining panes: snapshot_accumulator() (non-destructive, shared) + d. Merge all pane accumulators via AggregateCore::merge_with() + e. Emit merged result as PrecomputedOutput + AggregateCore 7. Emit batch to OutputSink 8. Update previous_watermark_ms ``` @@ -211,10 +259,53 @@ struct WindowManager { | `window_starts_containing(ts)` | All windows whose `[start, start+size)` includes `ts`. Tumbling → 1 window; sliding → `ceil(size/slide)` windows | | `closed_windows(prev_wm, curr_wm)` | Windows that transitioned open→closed as the watermark advanced | | `window_bounds(start)` | Returns `(start, start + window_size_ms)` | +| `pane_start_for(ts)` | Pane start for a timestamp (same slide-aligned grid as `window_start_for`) | +| `panes_for_window(ws)` | All pane starts composing window `[ws, ws+size)`, in ascending order | +| `slide_interval_ms()` | Slide interval accessor | **Window closure rule:** a window `[S, S + size)` closes when `watermark >= S + size`. Once closed, a window never reopens. +#### Sliding window mechanics + +Tumbling windows are a special case of sliding windows where +`slide_interval == window_size`. The same code handles both — no separate paths. + +**`window_start_for(ts)`** aligns a timestamp to the slide grid: +```rust +let n = timestamp_ms.div_euclid(slide_interval_ms); +n * slide_interval_ms +``` + +**`window_starts_containing(ts)`** returns all windows whose `[start, start+size)` +contains the timestamp, by walking backwards from the aligned start: +```rust +let mut start = window_start_for(timestamp_ms); +while start + window_size_ms > timestamp_ms { + starts.push(start); + start -= slide_interval_ms; +} +``` + +For tumbling windows this always yields exactly 1 result. For sliding windows, +each sample belongs to `ceil(window_size / slide_interval)` overlapping windows. + +**Example** (30s window, 10s slide): +``` +t=15s → belongs to windows [0, 30s), [10s, 40s), [-10s, 20s) (3 windows) +t=35s → belongs to windows [30s, 60s), [20s, 50s), [10s, 40s) (3 windows) +``` + +**`closed_windows(prev_wm, curr_wm)`** finds windows that transitioned open→closed +as the watermark advanced. It scans forward from the earliest possibly-open window +start, collecting those where `start + size <= curr_wm` (now closed) AND +`start + size > prev_wm` (was still open before). + +The worker calls `window_starts_containing(ts)` for each incoming sample and feeds +the value into the accumulator for every matching window. When +`closed_windows()` fires, each closed window's accumulator is extracted and +emitted independently. + ### 3.7 AccumulatorUpdater (`accumulator_factory.rs`) Trait-based interface for feeding samples into sketch accumulators: @@ -224,12 +315,17 @@ trait AccumulatorUpdater: Send { fn update_single(&mut self, value: f64, timestamp_ms: i64); fn update_keyed(&mut self, key: &KeyByLabelValues, value: f64, timestamp_ms: i64); fn take_accumulator(&mut self) -> Box; + fn snapshot_accumulator(&self) -> Box; // non-destructive clone fn reset(&mut self); fn is_keyed(&self) -> bool; fn memory_usage_bytes(&self) -> usize; } ``` +`snapshot_accumulator()` returns a clone of the current state without resetting. +Used by pane-based sliding windows to read shared panes that are still needed +by future windows. + The factory function `create_accumulator_updater(config)` dispatches on `(aggregation_type, aggregation_sub_type)`: @@ -305,6 +401,28 @@ pub struct AggregationConfig { `OutputSink.emit_batch()` → `Store.insert_precomputed_output_batch()` +Because the precompute engine runs in the same process as the store, the write +path involves **zero serialization and zero network hops**. Closed window +accumulators flow from worker to store entirely as in-memory trait objects: + +``` +Worker: updater.take_accumulator() → Box (in-memory) + ↓ (direct function call, no IPC) +OutputSink: store.insert_precomputed_output_batch(outputs) (pass-through) + ↓ (direct function call, same process) +SimpleMapStore: HashMap entry insert → Box (stored as-is) +``` + +No serialization, deserialization, compression, or network transfer occurs +between the worker extracting an accumulator and the store persisting it. +The only network hops in the system are at the edges: HTTP ingest (in) and +HTTP query (out). Serialization of accumulators only happens on the read path +when query results are returned to clients. + +This is in contrast to the external Kafka ingest path, where precomputes from +Arroyo/Flink arrive hex-encoded + gzip-compressed + MessagePack-serialized and +require multiple deserialization steps. + The `SimpleMapStore` (PerKey variant) uses: ``` DashMap>> @@ -387,13 +505,15 @@ independently. **Ingest path (per batch):** - Sample insert: O(log B) per sample (BTreeMap, B = buffer size) -- Window assignment: O(A × W) per sample (A = matching aggregations, - W = windows per sample — 1 for tumbling, ~3 for typical sliding) +- Pane routing: O(A) per sample (A = matching aggregations; each sample + touches exactly 1 pane per aggregation, regardless of window overlap) - Accumulator update: O(1) for Sum/MinMax, O(log k) for KLL +- Window close: O(W-1) merges per closed window (W = window_size / slide_interval) **Memory:** - O(S × N) buffered samples (S = max per series, N = active series) -- O(A × W_open) active window accumulators +- O(A × W_open) active pane accumulators (fewer than window accumulators + since panes are shared across overlapping windows) **Throughput:** - Workers process in parallel with no cross-shard coordination. diff --git a/asap-query-engine/src/precompute_engine/window_manager.rs b/asap-query-engine/src/precompute_engine/window_manager.rs index d8bf23b..14b4d1e 100644 --- a/asap-query-engine/src/precompute_engine/window_manager.rs +++ b/asap-query-engine/src/precompute_engine/window_manager.rs @@ -90,6 +90,27 @@ impl WindowManager { pub fn window_bounds(&self, window_start: i64) -> (i64, i64) { (window_start, window_start + self.window_size_ms) } + + /// Slide interval accessor. + pub fn slide_interval_ms(&self) -> i64 { + self.slide_interval_ms + } + + /// Pane start for a timestamp. Panes are aligned to the slide_interval grid, + /// which is the same grid as `window_start_for`. + pub fn pane_start_for(&self, timestamp_ms: i64) -> i64 { + self.window_start_for(timestamp_ms) + } + + /// All pane starts composing a window, in ascending order. + /// A window `[ws, ws + window_size)` is composed of + /// `window_size / slide_interval` consecutive panes. + pub fn panes_for_window(&self, window_start: i64) -> Vec { + let num_panes = self.window_size_ms / self.slide_interval_ms; + (0..num_panes) + .map(|i| window_start + i * self.slide_interval_ms) + .collect() + } } #[cfg(test)] @@ -194,4 +215,79 @@ mod tests { starts.sort(); assert_eq!(starts, vec![10_000, 20_000, 30_000]); } + + // --- Pane method tests --- + + #[test] + fn test_pane_start_for_equals_window_start_for() { + // Pane start and window start use the same slide-aligned grid + let wm = WindowManager::new(30, 10); + for ts in [0, 5_000, 9_999, 10_000, 15_000, 25_000, 30_000] { + assert_eq!(wm.pane_start_for(ts), wm.window_start_for(ts)); + } + } + + #[test] + fn test_panes_for_window_sliding() { + // 30s window, 10s slide → 3 panes per window + let wm = WindowManager::new(30, 10); + + assert_eq!(wm.panes_for_window(0), vec![0, 10_000, 20_000]); + assert_eq!( + wm.panes_for_window(10_000), + vec![10_000, 20_000, 30_000] + ); + assert_eq!( + wm.panes_for_window(20_000), + vec![20_000, 30_000, 40_000] + ); + } + + #[test] + fn test_panes_for_window_tumbling_degeneration() { + // 60s tumbling window → 1 pane per window (no merges needed) + let wm = WindowManager::new(60, 0); + + assert_eq!(wm.panes_for_window(0), vec![0]); + assert_eq!(wm.panes_for_window(60_000), vec![60_000]); + } + + #[test] + fn test_slide_interval_ms_accessor() { + let wm_tumbling = WindowManager::new(60, 0); + assert_eq!(wm_tumbling.slide_interval_ms(), 60_000); + + let wm_sliding = WindowManager::new(30, 10); + assert_eq!(wm_sliding.slide_interval_ms(), 10_000); + } + + #[test] + fn test_panes_for_window_count() { + // W = window_size / slide_interval + let wm = WindowManager::new(30, 10); + assert_eq!(wm.panes_for_window(0).len(), 3); // 30/10 = 3 + + let wm2 = WindowManager::new(50, 10); + assert_eq!(wm2.panes_for_window(0).len(), 5); // 50/10 = 5 + + let wm3 = WindowManager::new(60, 0); + assert_eq!(wm3.panes_for_window(0).len(), 1); // tumbling = 1 + } + + #[test] + fn test_consecutive_windows_share_panes() { + // 30s window, 10s slide — consecutive windows share W-1 = 2 panes + let wm = WindowManager::new(30, 10); + + let panes_a = wm.panes_for_window(0); // [0, 10_000, 20_000] + let panes_b = wm.panes_for_window(10_000); // [10_000, 20_000, 30_000] + + // Shared panes: 10_000 and 20_000 + let shared: Vec = panes_a + .iter() + .filter(|p| panes_b.contains(p)) + .copied() + .collect(); + assert_eq!(shared, vec![10_000, 20_000]); + } } diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs index f59f691..43ac528 100644 --- a/asap-query-engine/src/precompute_engine/worker.rs +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -9,18 +9,24 @@ use crate::precompute_engine::series_router::WorkerMessage; use crate::precompute_engine::window_manager::WindowManager; use crate::precompute_operators::sum_accumulator::SumAccumulator; use sketch_db_common::aggregation_config::AggregationConfig; -use std::collections::HashMap; +use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use tokio::sync::mpsc; use tracing::{debug, debug_span, info, warn}; /// Per-aggregation state within a series: the window manager and active -/// window accumulators. +/// pane accumulators. +/// +/// Uses pane-based sliding window computation: each sample is routed to +/// exactly 1 pane (sub-window of size `slide_interval`). When a window +/// closes, its constituent panes are merged. This reduces per-sample +/// accumulator updates from W to 1 (where W = window_size / slide_interval). struct AggregationState { config: AggregationConfig, window_manager: WindowManager, - /// Active windows keyed by window_start_ms. - active_windows: HashMap>, + /// Active panes keyed by pane_start_ms. + /// BTreeMap for ordered iteration (needed for pane eviction). + active_panes: BTreeMap>, } /// Per-series state owned by the worker. @@ -153,7 +159,7 @@ impl Worker { .map(|(_, config)| AggregationState { window_manager: WindowManager::new(config.window_size, config.slide_interval), config: config.clone(), - active_windows: HashMap::new(), + active_panes: BTreeMap::new(), }) .collect(); @@ -220,84 +226,119 @@ impl Worker { .window_manager .closed_windows(previous_wm, current_wm); - // Feed each incoming sample to the correct active window accumulator + // Pane-based sample routing: each sample goes to exactly 1 pane for &(ts, val) in &samples { if current_wm != i64::MIN && ts < current_wm - allowed_lateness_ms { continue; // already dropped } - let window_starts = agg_state.window_manager.window_starts_containing(ts); - - for window_start in window_starts { - let window_end = window_start + agg_state.window_manager.window_size_ms(); - - // Check if this window was already closed in a previous batch - if !agg_state.active_windows.contains_key(&window_start) - && current_wm >= window_end - { - // Window already closed — handle according to policy - match late_data_policy { - LateDataPolicy::Drop => { - debug!( - "Dropping late sample for closed window [{}, {})", - window_start, window_end - ); - continue; - } - LateDataPolicy::ForwardToStore => { - let mut updater = create_accumulator_updater(&agg_state.config); - if updater.is_keyed() { - let key = - extract_key_from_series(series_key, &agg_state.config); - updater.update_keyed(&key, val, ts); - } else { - updater.update_single(val, ts); - } - let key = if updater.is_keyed() { - Some(extract_key_from_series(series_key, &agg_state.config)) - } else { - None - }; - let output = PrecomputedOutput::new( - window_start as u64, - window_end as u64, - key, - agg_state.config.aggregation_id, - ); - emit_batch.push((output, updater.take_accumulator())); - debug!( - "Forwarding late sample to store for closed window [{}, {})", - window_start, window_end - ); - continue; + let pane_start = agg_state.window_manager.pane_start_for(ts); + let pane_end = pane_start + agg_state.window_manager.slide_interval_ms(); + + // Check if pane was already evicted (late data for a closed window). + // A pane is evicted when its oldest window closes, i.e. the window + // starting at pane_start. If that window is closed, the pane is gone. + if !agg_state.active_panes.contains_key(&pane_start) + && current_wm >= pane_start + agg_state.window_manager.window_size_ms() + { + // The window starting at this pane_start is already closed, + // so this pane was evicted — handle as late data. + let window_start = pane_start; + let window_end = pane_start + agg_state.window_manager.window_size_ms(); + match late_data_policy { + LateDataPolicy::Drop => { + debug!( + "Dropping late sample for evicted pane [{}, {})", + pane_start, pane_end + ); + continue; + } + LateDataPolicy::ForwardToStore => { + let mut updater = create_accumulator_updater(&agg_state.config); + if updater.is_keyed() { + let key = + extract_key_from_series(series_key, &agg_state.config); + updater.update_keyed(&key, val, ts); + } else { + updater.update_single(val, ts); } + let key = if updater.is_keyed() { + Some(extract_key_from_series(series_key, &agg_state.config)) + } else { + None + }; + let output = PrecomputedOutput::new( + window_start as u64, + window_end as u64, + key, + agg_state.config.aggregation_id, + ); + emit_batch.push((output, updater.take_accumulator())); + debug!( + "Forwarding late sample to store for evicted pane [{}, {})", + pane_start, pane_end + ); + continue; } } + } - // Normal path: window is still open (or newly opened) - let updater = agg_state - .active_windows - .entry(window_start) - .or_insert_with(|| create_accumulator_updater(&agg_state.config)); - - if updater.is_keyed() { - let key = extract_key_from_series(series_key, &agg_state.config); - updater.update_keyed(&key, val, ts); - } else { - updater.update_single(val, ts); - } + // Normal path: route sample to its single pane + let updater = agg_state + .active_panes + .entry(pane_start) + .or_insert_with(|| create_accumulator_updater(&agg_state.config)); + + if updater.is_keyed() { + let key = extract_key_from_series(series_key, &agg_state.config); + updater.update_keyed(&key, val, ts); + } else { + updater.update_single(val, ts); } } - // Emit closed windows + // Emit closed windows by merging their constituent panes for window_start in &closed { - if let Some(mut updater) = agg_state.active_windows.remove(window_start) { - let (_, window_end) = agg_state.window_manager.window_bounds(*window_start); - - let key = if updater.is_keyed() { - Some(extract_key_from_series(series_key, &agg_state.config)) + let (_, window_end) = agg_state.window_manager.window_bounds(*window_start); + let pane_starts = agg_state.window_manager.panes_for_window(*window_start); + + // Merge pane accumulators for this window. + // - Oldest pane (index 0): take_accumulator + remove (no future window needs it) + // - Remaining panes: snapshot_accumulator (shared with newer windows) + let mut merged: Option> = None; + + for (i, &ps) in pane_starts.iter().enumerate() { + let pane_acc = if i == 0 { + // Oldest pane: destructive take + evict from active_panes + agg_state + .active_panes + .remove(&ps) + .map(|mut updater| updater.take_accumulator()) } else { - None + // Shared pane: non-destructive snapshot + agg_state + .active_panes + .get(&ps) + .map(|updater| updater.snapshot_accumulator()) + }; + + if let Some(acc) = pane_acc { + merged = Some(match merged { + None => acc, + Some(existing) => existing.merge_with(acc.as_ref()).unwrap_or(existing), + }); + } + } + + if let Some(accumulator) = merged { + let key = { + // Check keyed-ness from accumulator type name or config + let test_updater = create_accumulator_updater(&agg_state.config); + if test_updater.is_keyed() { + Some(extract_key_from_series(series_key, &agg_state.config)) + } else { + None + } }; let output = PrecomputedOutput::new( @@ -307,7 +348,6 @@ impl Worker { agg_state.config.aggregation_id, ); - let accumulator = updater.take_accumulator(); emit_batch.push((output, accumulator)); } } @@ -376,13 +416,42 @@ impl Worker { .closed_windows(previous_wm, current_wm); for window_start in &closed { - if let Some(mut updater) = agg_state.active_windows.remove(window_start) { - let (_, window_end) = agg_state.window_manager.window_bounds(*window_start); + let (_, window_end) = agg_state.window_manager.window_bounds(*window_start); + let pane_starts = agg_state.window_manager.panes_for_window(*window_start); - let key = if updater.is_keyed() { - Some(extract_key_from_series(series_key, &agg_state.config)) + let mut merged: Option> = None; + + for (i, &ps) in pane_starts.iter().enumerate() { + let pane_acc = if i == 0 { + agg_state + .active_panes + .remove(&ps) + .map(|mut updater| updater.take_accumulator()) } else { - None + agg_state + .active_panes + .get(&ps) + .map(|updater| updater.snapshot_accumulator()) + }; + + if let Some(acc) = pane_acc { + merged = Some(match merged { + None => acc, + Some(existing) => { + existing.merge_with(acc.as_ref()).unwrap_or(existing) + } + }); + } + } + + if let Some(accumulator) = merged { + let key = { + let test_updater = create_accumulator_updater(&agg_state.config); + if test_updater.is_keyed() { + Some(extract_key_from_series(series_key, &agg_state.config)) + } else { + None + } }; let output = PrecomputedOutput::new( @@ -392,7 +461,6 @@ impl Worker { agg_state.config.aggregation_id, ); - let accumulator = updater.take_accumulator(); emit_batch.push((output, accumulator)); } } From 930101780999ce64334beb4da381874e2e1c7182 Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 16:45:18 -0700 Subject: [PATCH 16/38] Add windowed aggregation benchmarks and fix build issues - Add tumbling and sliding window throughput benchmarks to E2E test (Tumbling 10s, Sliding 30s/10s W=3, Sliding 60s/10s W=6) - Each benchmark uses NoopOutputSink to isolate worker throughput - Fix cargo fmt formatting in window_manager.rs and worker.rs - Pin sketchlib-rust to rev 663a1df for Chapter enum compatibility - Fix CMS updater field access (self.acc.inner.{row_num,col_num,sketch}) - Fix SimpleEngine::new missing promsketch_store arg in E2E test Co-Authored-By: Claude Opus 4.6 --- Cargo.lock | 233 +++++++++++------- asap-query-engine/Cargo.toml | 2 +- .../src/bin/test_e2e_precompute.rs | 227 ++++++++++++++++- .../precompute_engine/accumulator_factory.rs | 6 +- .../src/precompute_engine/window_manager.rs | 12 +- .../src/precompute_engine/worker.rs | 3 +- 6 files changed, 380 insertions(+), 103 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f8793a9..e53f07d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1410,9 +1410,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.8" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7cd812cc2bc1d69d4764bd80df88b4317eaef9e773c75226407d9bc0876b211c" +checksum = "cc3dc5ad92c2e2d1c193bbbbdf2ea477cb81331de4f3103f267ca18368b988c4" dependencies = [ "powerfmt", ] @@ -1736,19 +1736,19 @@ checksum = "899def5c37c4fd7b2664648c28120ecec138e4d395b459e5ca34f9cce2dd77fd" dependencies = [ "cfg-if", "libc", - "r-efi 5.3.0", + "r-efi", "wasip2", ] [[package]] name = "getrandom" -version = "0.4.2" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0de51e6874e94e7bf76d726fc5d13ba782deca734ff60d5bb2fb2607c7406555" +checksum = "139ef39800118c7683f2fd3c98c1b23c09ae076556b435f8e9064ae108aaeeec" dependencies = [ "cfg-if", "libc", - "r-efi 6.0.0", + "r-efi", "wasip2", "wasip3", ] @@ -2179,9 +2179,9 @@ checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" [[package]] name = "ipnet" -version = "2.12.0" +version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d98f6fed1fde3f8c21bc40a1abb88dd75e67924f9cffc3ef95607bad8017f8e2" +checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" [[package]] name = "is_terminal_polyfill" @@ -2215,9 +2215,9 @@ checksum = "92ecc6618181def0457392ccd0ee51198e065e016d1d527a7ac1b6dc7c1f09d2" [[package]] name = "jiff" -version = "0.2.23" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a3546dc96b6d42c5f24902af9e2538e82e39ad350b0c766eb3fbf2d8f3d8359" +checksum = "c867c356cc096b33f4981825ab281ecba3db0acefe60329f044c1789d94c6543" dependencies = [ "jiff-static", "jiff-tzdb-platform", @@ -2230,9 +2230,9 @@ dependencies = [ [[package]] name = "jiff-static" -version = "0.2.23" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a8c8b344124222efd714b73bb41f8b5120b27a7cc1c75593a6ff768d9d05aa4" +checksum = "f7946b4325269738f270bb55b3c19ab5c5040525f83fd625259422a9d25d9be5" dependencies = [ "proc-macro2", "quote", @@ -2241,9 +2241,9 @@ dependencies = [ [[package]] name = "jiff-tzdb" -version = "0.1.6" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c900ef84826f1338a557697dc8fc601df9ca9af4ac137c7fb61d4c6f2dfd3076" +checksum = "68971ebff725b9e2ca27a601c5eb38a4c5d64422c4cbab0c535f248087eda5c2" [[package]] name = "jiff-tzdb-platform" @@ -2266,9 +2266,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.91" +version = "0.3.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b49715b7073f385ba4bc528e5747d02e66cb39c6146efb66b781f131f0fb399c" +checksum = "8c942ebf8e95485ca0d52d97da7c5a2c387d0e7f0ba4c35e93bfcaee045955b3" dependencies = [ "once_cell", "wasm-bindgen", @@ -2367,14 +2367,13 @@ checksum = "b6d2cec3eae94f9f509c767b45932f1ada8350c4bdb85af2fcab4a3c14807981" [[package]] name = "libredox" -version = "0.1.14" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1744e39d1d6a9948f4f388969627434e31128196de472883b39f148769bfe30a" +checksum = "3d0b95e02c851351f877147b7deea7b1afb1df71b63aa5f8270716e0c5720616" dependencies = [ "bitflags 2.11.0", "libc", - "plain", - "redox_syscall 0.7.3", + "redox_syscall 0.7.1", ] [[package]] @@ -2390,9 +2389,9 @@ dependencies = [ [[package]] name = "libz-sys" -version = "1.1.24" +version = "1.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4735e9cbde5aac84a5ce588f6b23a90b9b0b528f6c5a8db8a4aff300463a0839" +checksum = "15d118bbf3771060e7311cc7bb0545b01d08a8b4a7de949198dec1fa0ca1c0f7" dependencies = [ "cc", "libc", @@ -2411,9 +2410,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.12.1" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a66949e030da00e8c7d4434b251670a91556f4144941d37452769c25d58a53" +checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" @@ -2947,9 +2946,9 @@ dependencies = [ [[package]] name = "pin-project-lite" -version = "0.2.17" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a89322df9ebe1c1578d689c92318e070967d1042b512afbe49518723f4e6d5cd" +checksum = "3b3cff922bd51709b605d9ead9aa71031d81447142d828eb4a6eba76fe619f9b" [[package]] name = "pin-utils" @@ -2963,12 +2962,6 @@ version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" -[[package]] -name = "plain" -version = "0.2.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4596b6d070b27117e987119b4dac604f3c58cfb0b191112e24771b2faeac1a6" - [[package]] name = "portable-atomic" version = "1.13.1" @@ -3020,9 +3013,9 @@ dependencies = [ [[package]] name = "proc-macro-crate" -version = "3.5.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e67ba7e9b2b56446f1d419b1d807906278ffa1a658a8a5d8a39dcb1f5a78614f" +checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" dependencies = [ "toml_edit", ] @@ -3195,9 +3188,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.45" +version = "1.0.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41f2619966050689382d2b44f664f4bc593e129785a36d6ee376ddf37259b924" +checksum = "21b2ebcf727b7760c461f091f9f0f539b77b8e87f2fd88131e7f1b433b3cece4" dependencies = [ "proc-macro2", ] @@ -3208,12 +3201,6 @@ version = "5.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" -[[package]] -name = "r-efi" -version = "6.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8dcc9c7d52a811697d2151c701e0d08956f92b0e24136cf4cf27b57a6a0d9bf" - [[package]] name = "rand" version = "0.8.5" @@ -3334,9 +3321,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.7.3" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ce70a74e890531977d37e532c34d45e9055d2409ed08ddba14529471ed0be16" +checksum = "35985aa610addc02e24fc232012c86fd11f14111180f902b67e2d5331f8ebf2b" dependencies = [ "bitflags 2.11.0", ] @@ -3372,9 +3359,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.8.10" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc897dd8d9e8bd1ed8cdad82b5966c3e0ecae09fb1907d58efaa013543185d0a" +checksum = "a96887878f22d7bad8a3b6dc5b7440e0ada9a245242924394987b21cf2210a4c" [[package]] name = "reqwest" @@ -3460,9 +3447,9 @@ dependencies = [ [[package]] name = "rustix" -version = "1.1.4" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6fe4565b9518b83ef4f91bb47ce29620ca828bd32cb7e408f0062e9930ba190" +checksum = "146c9e247ccc180c1f61615433868c99f3de3ae256a30a43b49f67c2d9171f34" dependencies = [ "bitflags 2.11.0", "errno 0.3.14", @@ -3719,7 +3706,7 @@ dependencies = [ [[package]] name = "sketchlib-rust" version = "0.1.0" -source = "git+https://github.com/ProjectASAP/sketchlib-rust#348db8415f97246c42de68b407b47fa038cf8b1f" +source = "git+https://github.com/ProjectASAP/sketchlib-rust?rev=663a1df#663a1df32ffaf8243af3b57bebfe33376ddeeb09" dependencies = [ "ahash", "clap 4.5.60", @@ -3783,12 +3770,12 @@ dependencies = [ [[package]] name = "socket2" -version = "0.6.3" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a766e1110788c36f4fa1c2b71b387a7815aa65f88ce0229841826633d93723e" +checksum = "86f4aa3ad99f2088c990dfa82d367e19cb29268ed67c574d10d0a4bfe71f07e0" dependencies = [ "libc", - "windows-sys 0.61.2", + "windows-sys 0.60.2", ] [[package]] @@ -4001,12 +3988,12 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.26.0" +version = "3.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "82a72c767771b47409d2345987fda8628641887d5466101319899796367354a0" +checksum = "0136791f7c95b1f6dd99f9cc786b91bb81c3800b639b3478e561ddb7be95e5f1" dependencies = [ "fastrand", - "getrandom 0.4.2", + "getrandom 0.4.1", "once_cell", "rustix", "windows-sys 0.61.2", @@ -4150,9 +4137,9 @@ dependencies = [ [[package]] name = "tokio" -version = "1.50.0" +version = "1.49.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27ad5e34374e03cfffefc301becb44e9dc3c17584f414349ebe29ed26661822d" +checksum = "72a2903cd7736441aac9df9d7688bd0ce48edccaadf181c3b90be801e81d3d86" dependencies = [ "bytes", "libc", @@ -4160,16 +4147,16 @@ dependencies = [ "parking_lot", "pin-project-lite", "signal-hook-registry", - "socket2 0.6.3", + "socket2 0.6.2", "tokio-macros", "windows-sys 0.61.2", ] [[package]] name = "tokio-macros" -version = "2.6.1" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c55a2eff8b69ce66c84f85e1da1c233edc36ceb85a2058d11b0d6a3c7e7569c" +checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", @@ -4223,18 +4210,18 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "1.0.0+spec-1.1.0" +version = "0.7.5+spec-1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32c2555c699578a4f59f0cc68e5116c8d7cabbd45e1409b989d4be085b53f13e" +checksum = "92e1cfed4a3038bc5a127e35a2d360f145e1f4b971b551a2ba5fd7aedf7e1347" dependencies = [ "serde_core", ] [[package]] name = "toml_edit" -version = "0.25.4+spec-1.1.0" +version = "0.23.10+spec-1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7193cbd0ce53dc966037f54351dbbcf0d5a642c7f0038c382ef9e677ce8c13f2" +checksum = "84c8b9f757e028cee9fa244aea147aab2a9ec09d5325a9b01e0a49730c2b5269" dependencies = [ "indexmap", "toml_datetime", @@ -4452,11 +4439,11 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.22.0" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a68d3c8f01c0cfa54a75291d83601161799e4a89a39e0929f4b0354d88757a37" +checksum = "b672338555252d43fd2240c714dc444b8c6fb0a5c5335e65a07bba7742735ddb" dependencies = [ - "getrandom 0.4.2", + "getrandom 0.4.1", "js-sys", "wasm-bindgen", ] @@ -4551,9 +4538,9 @@ dependencies = [ [[package]] name = "wasm-bindgen" -version = "0.2.114" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6532f9a5c1ece3798cb1c2cfdba640b9b3ba884f5db45973a6f442510a87d38e" +checksum = "64024a30ec1e37399cf85a7ffefebdb72205ca1c972291c51512360d90bd8566" dependencies = [ "cfg-if", "once_cell", @@ -4564,9 +4551,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.64" +version = "0.4.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9c5522b3a28661442748e09d40924dfb9ca614b21c00d3fd135720e48b67db8" +checksum = "70a6e77fd0ae8029c9ea0063f87c46fde723e7d887703d74ad2616d792e51e6f" dependencies = [ "cfg-if", "futures-util", @@ -4578,9 +4565,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.114" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18a2d50fcf105fb33bb15f00e7a77b772945a2ee45dcf454961fd843e74c18e6" +checksum = "008b239d9c740232e71bd39e8ef6429d27097518b6b30bdf9086833bd5b6d608" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -4588,9 +4575,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.114" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03ce4caeaac547cdf713d280eda22a730824dd11e6b8c3ca9e42247b25c631e3" +checksum = "5256bae2d58f54820e6490f9839c49780dff84c65aeab9e772f15d5f0e913a55" dependencies = [ "bumpalo", "proc-macro2", @@ -4601,9 +4588,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-shared" -version = "0.2.114" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75a326b8c223ee17883a4251907455a2431acc2791c98c26279376490c378c16" +checksum = "1f01b580c9ac74c8d8f0c0e4afb04eeef2acf145458e52c03845ee9cd23e3d12" dependencies = [ "unicode-ident", ] @@ -4644,9 +4631,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.91" +version = "0.3.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "854ba17bb104abfb26ba36da9729addc7ce7f06f5c0f90f3c391f8461cca21f9" +checksum = "312e32e551d92129218ea9a2452120f4aabc03529ef03e4d0d82fb2780608598" dependencies = [ "js-sys", "wasm-bindgen", @@ -4782,6 +4769,15 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "windows-sys" +version = "0.60.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" +dependencies = [ + "windows-targets 0.53.5", +] + [[package]] name = "windows-sys" version = "0.61.2" @@ -4815,13 +4811,30 @@ dependencies = [ "windows_aarch64_gnullvm 0.52.6", "windows_aarch64_msvc 0.52.6", "windows_i686_gnu 0.52.6", - "windows_i686_gnullvm", + "windows_i686_gnullvm 0.52.6", "windows_i686_msvc 0.52.6", "windows_x86_64_gnu 0.52.6", "windows_x86_64_gnullvm 0.52.6", "windows_x86_64_msvc 0.52.6", ] +[[package]] +name = "windows-targets" +version = "0.53.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4945f9f551b88e0d65f3db0bc25c33b8acea4d9e41163edf90dcd0b19f9069f3" +dependencies = [ + "windows-link", + "windows_aarch64_gnullvm 0.53.1", + "windows_aarch64_msvc 0.53.1", + "windows_i686_gnu 0.53.1", + "windows_i686_gnullvm 0.53.1", + "windows_i686_msvc 0.53.1", + "windows_x86_64_gnu 0.53.1", + "windows_x86_64_gnullvm 0.53.1", + "windows_x86_64_msvc 0.53.1", +] + [[package]] name = "windows_aarch64_gnullvm" version = "0.48.5" @@ -4834,6 +4847,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a9d8416fa8b42f5c947f8482c43e7d89e73a173cead56d044f6a56104a6d1b53" + [[package]] name = "windows_aarch64_msvc" version = "0.36.1" @@ -4852,6 +4871,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" +[[package]] +name = "windows_aarch64_msvc" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b9d782e804c2f632e395708e99a94275910eb9100b2114651e04744e9b125006" + [[package]] name = "windows_i686_gnu" version = "0.36.1" @@ -4870,12 +4895,24 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" +[[package]] +name = "windows_i686_gnu" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "960e6da069d81e09becb0ca57a65220ddff016ff2d6af6a223cf372a506593a3" + [[package]] name = "windows_i686_gnullvm" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" +[[package]] +name = "windows_i686_gnullvm" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa7359d10048f68ab8b09fa71c3daccfb0e9b559aed648a8f95469c27057180c" + [[package]] name = "windows_i686_msvc" version = "0.36.1" @@ -4894,6 +4931,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" +[[package]] +name = "windows_i686_msvc" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e7ac75179f18232fe9c285163565a57ef8d3c89254a30685b57d83a38d326c2" + [[package]] name = "windows_x86_64_gnu" version = "0.36.1" @@ -4912,6 +4955,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" +[[package]] +name = "windows_x86_64_gnu" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c3842cdd74a865a8066ab39c8a7a473c0778a3f29370b5fd6b4b9aa7df4a499" + [[package]] name = "windows_x86_64_gnullvm" version = "0.48.5" @@ -4924,6 +4973,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ffa179e2d07eee8ad8f57493436566c7cc30ac536a3379fdf008f47f6bb7ae1" + [[package]] name = "windows_x86_64_msvc" version = "0.36.1" @@ -4942,11 +4997,17 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" +[[package]] +name = "windows_x86_64_msvc" +version = "0.53.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" + [[package]] name = "winnow" -version = "0.7.15" +version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df79d97927682d2fd8adb29682d1140b343be4ac0f08fd68b7765d9c059d3945" +checksum = "5a5364e9d77fcdeeaa6062ced926ee3381faa2ee02d3eb83a5c27a8825540829" dependencies = [ "memchr", ] @@ -5095,18 +5156,18 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.8.40" +version = "0.8.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a789c6e490b576db9f7e6b6d661bcc9799f7c0ac8352f56ea20193b2681532e5" +checksum = "db6d35d663eadb6c932438e763b262fe1a70987f9ae936e60158176d710cae4a" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.8.40" +version = "0.8.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f65c489a7071a749c849713807783f70672b28094011623e200cb86dcb835953" +checksum = "4122cd3169e94605190e77839c9a40d40ed048d305bfdc146e7df40ab0f3e517" dependencies = [ "proc-macro2", "quote", diff --git a/asap-query-engine/Cargo.toml b/asap-query-engine/Cargo.toml index 37c4de2..310724e 100644 --- a/asap-query-engine/Cargo.toml +++ b/asap-query-engine/Cargo.toml @@ -39,7 +39,7 @@ flate2 = "1.0" async-trait = "0.1" xxhash-rust = { version = "0.8", features = ["xxh32", "xxh64"] } dsrs = { git = "https://github.com/ProjectASAP/datasketches-rs" } -sketchlib-rust = { git = "https://github.com/ProjectASAP/sketchlib-rust" } +sketchlib-rust = { git = "https://github.com/ProjectASAP/sketchlib-rust", rev = "663a1df" } base64 = "0.21" hex = "0.4" sqlparser = "0.59.0" diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index 77e41a5..b00fa3e 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -9,18 +9,22 @@ //! cargo run --bin test_e2e_precompute use prost::Message; -use query_engine_rust::data_model::{LockStrategy, QueryLanguage}; +use query_engine_rust::data_model::{LockStrategy, QueryLanguage, StreamingConfig}; use query_engine_rust::drivers::ingest::prometheus_remote_write::{ Label, Sample, TimeSeries, WriteRequest, }; use query_engine_rust::drivers::query::adapters::AdapterConfig; use query_engine_rust::engines::SimpleEngine; use query_engine_rust::precompute_engine::config::{LateDataPolicy, PrecomputeEngineConfig}; -use query_engine_rust::precompute_engine::output_sink::{RawPassthroughSink, StoreOutputSink}; +use query_engine_rust::precompute_engine::output_sink::{ + NoopOutputSink, RawPassthroughSink, StoreOutputSink, +}; use query_engine_rust::precompute_engine::PrecomputeEngine; use query_engine_rust::stores::SimpleMapStore; use query_engine_rust::utils::file_io::{read_inference_config, read_streaming_config}; use query_engine_rust::{HttpServer, HttpServerConfig}; +use sketch_db_common::aggregation_config::AggregationConfig; +use std::collections::HashMap; use std::sync::Arc; const INGEST_PORT: u16 = 19090; @@ -113,6 +117,7 @@ async fn main() -> Result<(), Box> { // Start query server let query_engine = Arc::new(SimpleEngine::new( store.clone(), + None, // no PromSketchStore for precompute E2E test inference_config, streaming_config.clone(), SCRAPE_INTERVAL, @@ -468,11 +473,229 @@ async fn main() -> Result<(), Box> { ); println!(" Throughput test PASSED"); + // ----------------------------------------------------------------------- + // WINDOWED AGGREGATION THROUGHPUT BENCHMARKS + // Compare tumbling vs sliding window performance with the pane-based + // engine. Each benchmark spins up its own PrecomputeEngine with a + // NoopOutputSink (to isolate worker throughput from store I/O). + // ----------------------------------------------------------------------- + let bench_results = run_windowed_benchmarks(&client).await?; + println!("\n=== Windowed aggregation benchmark summary ==="); + println!( + " {:<30} {:>12} {:>12} {:>14}", + "Config", "Send (s/s)", "E2E (s/s)", "Latency (ms)" + ); + for r in &bench_results { + println!( + " {:<30} {:>12.0} {:>12.0} {:>14.1}", + r.label, r.send_throughput, r.e2e_throughput, r.batch_latency_ms + ); + } + println!("\n=== E2E test complete ==="); Ok(()) } +// --------------------------------------------------------------------------- +// Windowed aggregation benchmarks +// --------------------------------------------------------------------------- + +struct BenchResult { + label: String, + send_throughput: f64, + e2e_throughput: f64, + batch_latency_ms: f64, +} + +/// Build an AggregationConfig for Sum with specified window parameters. +fn make_sum_agg_config( + agg_id: u64, + window_size_secs: u64, + slide_interval_secs: u64, +) -> AggregationConfig { + let window_type = if slide_interval_secs == 0 || slide_interval_secs == window_size_secs { + "tumbling" + } else { + "sliding" + }; + AggregationConfig::new( + agg_id, + "SingleSubpopulation".to_string(), + "Sum".to_string(), + HashMap::new(), + promql_utilities::data_model::key_by_label_names::KeyByLabelNames::new(vec![]), + promql_utilities::data_model::key_by_label_names::KeyByLabelNames::new(vec![]), + promql_utilities::data_model::key_by_label_names::KeyByLabelNames::new(vec![]), + String::new(), + window_size_secs, + "bench_metric".to_string(), + "bench_metric".to_string(), + None, + None, + Some(window_size_secs), + Some(slide_interval_secs), + Some(window_type.to_string()), + None, + None, + ) +} + +/// Run a single windowed benchmark and return the results. +async fn run_single_bench( + client: &reqwest::Client, + label: &str, + port: u16, + streaming_config: Arc, + num_requests: u64, + samples_per_request: u64, + num_series: u64, +) -> Result> { + let total_samples = num_requests * samples_per_request; + + let noop_sink = Arc::new(NoopOutputSink::new()); + let engine_config = PrecomputeEngineConfig { + num_workers: 4, + ingest_port: port, + allowed_lateness_ms: 5000, + max_buffer_per_series: 100_000, + flush_interval_ms: 100, + channel_buffer_size: 50_000, + pass_raw_samples: false, + raw_mode_aggregation_id: 0, + late_data_policy: LateDataPolicy::Drop, + }; + let engine = PrecomputeEngine::new(engine_config, streaming_config, noop_sink.clone()); + tokio::spawn(async move { + if let Err(e) = engine.run().await { + eprintln!("Bench engine error: {e}"); + } + }); + tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; + + // Pre-build request bodies. Timestamps are monotonically increasing + // across requests so windows close naturally as the watermark advances. + let mut bodies = Vec::with_capacity(num_requests as usize); + for req_idx in 0..num_requests { + let mut timeseries = Vec::with_capacity(samples_per_request as usize); + for s in 0..samples_per_request { + let series_label = format!("s_{}", s % num_series); + // Each request advances time by 1000ms (1 second) + let ts = (req_idx as i64) * 1000 + (s as i64 % 1000); + timeseries.push(make_sample("bench_metric", &series_label, ts, s as f64)); + } + bodies.push(build_remote_write_body(timeseries)); + } + + // --- Batch latency: single request --- + let latency_body = bodies[0].clone(); + let t0 = std::time::Instant::now(); + client + .post(format!("http://localhost:{port}/api/v1/write")) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(latency_body) + .send() + .await?; + let batch_latency_ms = t0.elapsed().as_secs_f64() * 1000.0; + + // --- Throughput: all requests --- + let throughput_start = std::time::Instant::now(); + for body in bodies { + let resp = client + .post(format!("http://localhost:{port}/api/v1/write")) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(body) + .send() + .await?; + if resp.status() != reqwest::StatusCode::NO_CONTENT { + eprintln!(" {label}: request failed: {}", resp.status()); + } + } + let send_elapsed = throughput_start.elapsed(); + + // Wait for workers to drain (poll emit_count on noop sink) + let drain_deadline = std::time::Instant::now() + std::time::Duration::from_secs(30); + loop { + let emitted = noop_sink + .emit_count + .load(std::sync::atomic::Ordering::Relaxed); + if emitted > 0 || std::time::Instant::now() > drain_deadline { + break; + } + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + } + // Give workers a bit more time to finish in-flight work + tokio::time::sleep(tokio::time::Duration::from_millis(500)).await; + let total_elapsed = throughput_start.elapsed(); + + let emitted = noop_sink + .emit_count + .load(std::sync::atomic::Ordering::Relaxed); + let send_throughput = total_samples as f64 / send_elapsed.as_secs_f64(); + let e2e_throughput = total_samples as f64 / total_elapsed.as_secs_f64(); + + println!(" {label}:"); + println!( + " Sent {total_samples} samples in {:.1}ms ({:.0} samples/sec)", + send_elapsed.as_secs_f64() * 1000.0, + send_throughput + ); + println!( + " E2E: {:.1}ms ({:.0} samples/sec), emitted {emitted} windows", + total_elapsed.as_secs_f64() * 1000.0, + e2e_throughput + ); + println!(" Batch latency: {batch_latency_ms:.1}ms"); + + Ok(BenchResult { + label: label.to_string(), + send_throughput, + e2e_throughput, + batch_latency_ms, + }) +} + +async fn run_windowed_benchmarks( + client: &reqwest::Client, +) -> Result, Box> { + let num_requests = 200u64; + let samples_per_request = 5_000u64; + let num_series = 50u64; + + let configs: Vec<(&str, u16, u64, u64)> = vec![ + // (label, port, window_size_secs, slide_interval_secs) + ("Tumbling 10s Sum", 19100, 10, 0), + ("Sliding 30s/10s Sum", 19101, 30, 10), + ("Sliding 60s/10s Sum (W=6)", 19102, 60, 10), + ]; + + println!("\n=== Windowed aggregation benchmarks ({num_requests} req × {samples_per_request} samples, {num_series} series) ==="); + + let mut results = Vec::new(); + for (label, port, window_size, slide_interval) in configs { + let agg_config = make_sum_agg_config(100, window_size, slide_interval); + let mut agg_map = HashMap::new(); + agg_map.insert(100u64, agg_config); + let sc = Arc::new(StreamingConfig::new(agg_map)); + + let r = run_single_bench( + client, + label, + port, + sc, + num_requests, + samples_per_request, + num_series, + ) + .await?; + results.push(r); + } + + Ok(results) +} + fn print_json(s: &str) { match serde_json::from_str::(s) { Ok(v) => println!("{}", serde_json::to_string_pretty(&v).unwrap()), diff --git a/asap-query-engine/src/precompute_engine/accumulator_factory.rs b/asap-query-engine/src/precompute_engine/accumulator_factory.rs index 66e6bb9..be50097 100644 --- a/asap-query-engine/src/precompute_engine/accumulator_factory.rs +++ b/asap-query-engine/src/precompute_engine/accumulator_factory.rs @@ -455,10 +455,10 @@ impl AccumulatorUpdater for CmsAccumulatorUpdater { let key_str = key_values.join(";"); let key_bytes = key_str.as_bytes(); - for i in 0..self.acc.row_num { + for i in 0..self.acc.inner.row_num { let hash_value = xxhash_rust::xxh32::xxh32(key_bytes, i as u32); - let col_index = (hash_value as usize) % self.acc.col_num; - self.acc.sketch[i][col_index] += value; + let col_index = (hash_value as usize) % self.acc.inner.col_num; + self.acc.inner.sketch[i][col_index] += value; } } diff --git a/asap-query-engine/src/precompute_engine/window_manager.rs b/asap-query-engine/src/precompute_engine/window_manager.rs index 14b4d1e..841c182 100644 --- a/asap-query-engine/src/precompute_engine/window_manager.rs +++ b/asap-query-engine/src/precompute_engine/window_manager.rs @@ -233,14 +233,8 @@ mod tests { let wm = WindowManager::new(30, 10); assert_eq!(wm.panes_for_window(0), vec![0, 10_000, 20_000]); - assert_eq!( - wm.panes_for_window(10_000), - vec![10_000, 20_000, 30_000] - ); - assert_eq!( - wm.panes_for_window(20_000), - vec![20_000, 30_000, 40_000] - ); + assert_eq!(wm.panes_for_window(10_000), vec![10_000, 20_000, 30_000]); + assert_eq!(wm.panes_for_window(20_000), vec![20_000, 30_000, 40_000]); } #[test] @@ -279,7 +273,7 @@ mod tests { // 30s window, 10s slide — consecutive windows share W-1 = 2 panes let wm = WindowManager::new(30, 10); - let panes_a = wm.panes_for_window(0); // [0, 10_000, 20_000] + let panes_a = wm.panes_for_window(0); // [0, 10_000, 20_000] let panes_b = wm.panes_for_window(10_000); // [10_000, 20_000, 30_000] // Shared panes: 10_000 and 20_000 diff --git a/asap-query-engine/src/precompute_engine/worker.rs b/asap-query-engine/src/precompute_engine/worker.rs index 43ac528..1f9f15a 100644 --- a/asap-query-engine/src/precompute_engine/worker.rs +++ b/asap-query-engine/src/precompute_engine/worker.rs @@ -256,8 +256,7 @@ impl Worker { LateDataPolicy::ForwardToStore => { let mut updater = create_accumulator_updater(&agg_state.config); if updater.is_keyed() { - let key = - extract_key_from_series(series_key, &agg_state.config); + let key = extract_key_from_series(series_key, &agg_state.config); updater.update_keyed(&key, val, ts); } else { updater.update_single(val, ts); From 583257290d2b50b086f9025a5822b3bef71f2093 Mon Sep 17 00:00:00 2001 From: zz_y Date: Thu, 26 Feb 2026 16:53:16 -0700 Subject: [PATCH 17/38] Add scalability benchmark varying worker count (1-16) Parameterize run_single_bench with num_workers and add run_scalability_benchmark() that measures throughput across 1, 2, 4, 8, 16 workers with sliding 30s/10s Sum (W=3). Co-Authored-By: Claude Opus 4.6 --- .../src/bin/test_e2e_precompute.rs | 71 ++++++++++++++++++- 1 file changed, 70 insertions(+), 1 deletion(-) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index b00fa3e..b397da4 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -492,6 +492,32 @@ async fn main() -> Result<(), Box> { ); } + // ----------------------------------------------------------------------- + // SCALABILITY TEST + // Measure throughput as a function of worker count (1, 2, 4, 8, 16) + // to verify linear scaling with cores. Uses sliding 30s/10s Sum (W=3) + // with NoopOutputSink. + // ----------------------------------------------------------------------- + let scale_results = run_scalability_benchmark(&client).await?; + println!("\n=== Scalability benchmark summary (Sliding 30s/10s Sum, W=3) ==="); + println!( + " {:<10} {:>12} {:>12} {:>10}", + "Workers", "Send (s/s)", "E2E (s/s)", "Speedup" + ); + let baseline_e2e = scale_results + .first() + .map(|r| r.e2e_throughput) + .unwrap_or(1.0); + for r in &scale_results { + println!( + " {:<10} {:>12.0} {:>12.0} {:>10.2}x", + r.label, + r.send_throughput, + r.e2e_throughput, + r.e2e_throughput / baseline_e2e + ); + } + println!("\n=== E2E test complete ==="); Ok(()) @@ -547,6 +573,7 @@ async fn run_single_bench( label: &str, port: u16, streaming_config: Arc, + num_workers: usize, num_requests: u64, samples_per_request: u64, num_series: u64, @@ -555,7 +582,7 @@ async fn run_single_bench( let noop_sink = Arc::new(NoopOutputSink::new()); let engine_config = PrecomputeEngineConfig { - num_workers: 4, + num_workers, ingest_port: port, allowed_lateness_ms: 5000, max_buffer_per_series: 100_000, @@ -685,6 +712,48 @@ async fn run_windowed_benchmarks( label, port, sc, + 4, + num_requests, + samples_per_request, + num_series, + ) + .await?; + results.push(r); + } + + Ok(results) +} + +async fn run_scalability_benchmark( + client: &reqwest::Client, +) -> Result, Box> { + let num_requests = 200u64; + let samples_per_request = 5_000u64; + let num_series = 100u64; // more series to give workers enough parallel work + let worker_counts: Vec = vec![1, 2, 4, 8, 16]; + let base_port: u16 = 19200; + + println!( + "\n=== Scalability benchmark ({num_requests} req × {samples_per_request} samples, \ + {num_series} series, Sliding 30s/10s Sum) ===" + ); + + let mut results = Vec::new(); + for (i, &num_workers) in worker_counts.iter().enumerate() { + let port = base_port + i as u16; + let label = format!("{num_workers}"); + + let agg_config = make_sum_agg_config(200 + i as u64, 30, 10); + let mut agg_map = HashMap::new(); + agg_map.insert(200 + i as u64, agg_config); + let sc = Arc::new(StreamingConfig::new(agg_map)); + + let r = run_single_bench( + client, + &label, + port, + sc, + num_workers, num_requests, samples_per_request, num_series, From a43f9252ba2f2d7a46ec9e5ba06601d0e34d7179 Mon Sep 17 00:00:00 2001 From: zz_y Date: Fri, 27 Feb 2026 09:41:09 -0700 Subject: [PATCH 18/38] Make throughput test multi-threaded for data generation and sending Use tokio::task::spawn_blocking to build request payloads in parallel across 8 threads, and send HTTP requests concurrently via 8 tokio tasks (matching the pattern used in run_single_bench). Co-Authored-By: Claude Opus 4.6 --- .../src/bin/test_e2e_precompute.rs | 118 +++++++++++++----- 1 file changed, 86 insertions(+), 32 deletions(-) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index b397da4..1aa9363 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -399,37 +399,70 @@ async fn main() -> Result<(), Box> { // Send many requests back-to-back and measure sustained throughput // (samples/sec). Uses the raw-mode engine for a clean measurement. // ----------------------------------------------------------------------- - println!("\n=== Throughput test: 1000 requests × 10000 samples ==="); + let num_concurrent_senders = 8usize; + println!("\n=== Throughput test: 1000 requests × 10000 samples ({num_concurrent_senders} concurrent senders) ==="); let num_requests = 1000u64; let samples_per_request = 10_000u64; let total_samples = num_requests * samples_per_request; - // Pre-build all request bodies so serialization doesn't count against throughput + // Pre-build all request bodies in parallel using rayon-style chunking via tokio tasks. + // Each task builds its share of requests, then we flatten the results. + let num_build_tasks = num_concurrent_senders; + let requests_per_task = (num_requests as usize + num_build_tasks - 1) / num_build_tasks; + let mut build_handles = Vec::with_capacity(num_build_tasks); + for task_idx in 0..num_build_tasks { + let start = task_idx * requests_per_task; + let end = ((task_idx + 1) * requests_per_task).min(num_requests as usize); + build_handles.push(tokio::task::spawn_blocking(move || { + let mut chunk = Vec::with_capacity(end - start); + for req_idx in start..end { + let mut timeseries = Vec::with_capacity(samples_per_request as usize); + for s in 0..samples_per_request { + let series_label = format!("tp_{}", s % 50); // 50 distinct series + let ts = 300_000 + req_idx as i64 * 10_000 + s as i64; + timeseries.push(make_sample("fake_metric", &series_label, ts, s as f64)); + } + chunk.push(build_remote_write_body(timeseries)); + } + chunk + })); + } let mut bodies = Vec::with_capacity(num_requests as usize); - for req_idx in 0..num_requests { - let mut timeseries = Vec::with_capacity(samples_per_request as usize); - for s in 0..samples_per_request { - let series_label = format!("tp_{}", s % 50); // 50 distinct series - let ts = 300_000 + req_idx as i64 * 10_000 + s as i64; - timeseries.push(make_sample("fake_metric", &series_label, ts, s as f64)); - } - bodies.push(build_remote_write_body(timeseries)); + for handle in build_handles { + bodies.extend(handle.await?); } let throughput_start = std::time::Instant::now(); + // Send requests using multiple concurrent sender tasks + let mut body_chunks: Vec>> = vec![Vec::new(); num_concurrent_senders]; for (i, body) in bodies.into_iter().enumerate() { - let resp = client - .post(format!("http://localhost:{RAW_INGEST_PORT}/api/v1/write")) - .header("Content-Type", "application/x-protobuf") - .header("Content-Encoding", "snappy") - .body(body) - .send() - .await?; - if resp.status() != reqwest::StatusCode::NO_CONTENT { - eprintln!(" Request {i} failed: {}", resp.status()); - } + body_chunks[i % num_concurrent_senders].push(body); + } + let mut send_handles = Vec::new(); + for chunk in body_chunks { + let client = client.clone(); + let url = format!("http://localhost:{RAW_INGEST_PORT}/api/v1/write"); + send_handles.push(tokio::spawn(async move { + for body in chunk { + let resp = client + .post(&url) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(body) + .send() + .await; + if let Ok(r) = resp { + if r.status() != reqwest::StatusCode::NO_CONTENT { + eprintln!(" request failed: {}", r.status()); + } + } + } + })); + } + for handle in send_handles { + handle.await?; } let send_elapsed = throughput_start.elapsed(); @@ -574,6 +607,7 @@ async fn run_single_bench( port: u16, streaming_config: Arc, num_workers: usize, + num_concurrent_senders: usize, num_requests: u64, samples_per_request: u64, num_series: u64, @@ -626,19 +660,37 @@ async fn run_single_bench( .await?; let batch_latency_ms = t0.elapsed().as_secs_f64() * 1000.0; - // --- Throughput: all requests --- + // --- Throughput: all requests with concurrent senders --- let throughput_start = std::time::Instant::now(); - for body in bodies { - let resp = client - .post(format!("http://localhost:{port}/api/v1/write")) - .header("Content-Type", "application/x-protobuf") - .header("Content-Encoding", "snappy") - .body(body) - .send() - .await?; - if resp.status() != reqwest::StatusCode::NO_CONTENT { - eprintln!(" {label}: request failed: {}", resp.status()); - } + + // Round-robin distribute request bodies across concurrent sender tasks + let mut body_chunks: Vec>> = vec![Vec::new(); num_concurrent_senders]; + for (i, body) in bodies.into_iter().enumerate() { + body_chunks[i % num_concurrent_senders].push(body); + } + let mut send_handles = Vec::new(); + for chunk in body_chunks { + let client = client.clone(); + let url = format!("http://localhost:{port}/api/v1/write"); + send_handles.push(tokio::spawn(async move { + for body in chunk { + let resp = client + .post(&url) + .header("Content-Type", "application/x-protobuf") + .header("Content-Encoding", "snappy") + .body(body) + .send() + .await; + if let Ok(r) = resp { + if r.status() != reqwest::StatusCode::NO_CONTENT { + eprintln!(" request failed: {}", r.status()); + } + } + } + })); + } + for handle in send_handles { + handle.await?; } let send_elapsed = throughput_start.elapsed(); @@ -713,6 +765,7 @@ async fn run_windowed_benchmarks( port, sc, 4, + 4, // concurrent senders to saturate workers num_requests, samples_per_request, num_series, @@ -754,6 +807,7 @@ async fn run_scalability_benchmark( port, sc, num_workers, + num_workers, // concurrent senders match worker count num_requests, samples_per_request, num_series, From 2ee928419180e19fe63f72b9361e9617a16707eb Mon Sep 17 00:00:00 2001 From: zz_y Date: Fri, 27 Feb 2026 10:39:05 -0700 Subject: [PATCH 19/38] Increase throughput test drain timeout from 60s to 120s The 10M-sample throughput test needs ~94s to fully drain through the store, exceeding the previous 60s deadline. Co-Authored-By: Claude Opus 4.6 --- asap-query-engine/src/bin/test_e2e_precompute.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index 1aa9363..3a0535e 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -474,7 +474,7 @@ async fn main() -> Result<(), Box> { // Poll until workers drain or timeout after 60s let max_ts = 300_000u64 + num_requests * 10_000 + samples_per_request; - let drain_deadline = std::time::Instant::now() + std::time::Duration::from_secs(60); + let drain_deadline = std::time::Instant::now() + std::time::Duration::from_secs(120); let mut tp_buckets: usize; loop { let tp_results = From 8a9f83d30cd180aa9a883b1d3ce13addd850ad6c Mon Sep 17 00:00:00 2001 From: zz_y Date: Fri, 6 Mar 2026 14:58:48 -0600 Subject: [PATCH 20/38] Fix PR405 reorg integration for ingest decode and config APIs --- .../src/bin/test_e2e_precompute.rs | 1 - .../drivers/ingest/prometheus_remote_write.rs | 111 ++++++++++++++++++ .../ingest/victoriametrics_remote_write.rs | 36 ++++++ asap-query-engine/src/main.rs | 2 + 4 files changed, 149 insertions(+), 1 deletion(-) diff --git a/asap-query-engine/src/bin/test_e2e_precompute.rs b/asap-query-engine/src/bin/test_e2e_precompute.rs index 3a0535e..f31da5c 100644 --- a/asap-query-engine/src/bin/test_e2e_precompute.rs +++ b/asap-query-engine/src/bin/test_e2e_precompute.rs @@ -117,7 +117,6 @@ async fn main() -> Result<(), Box> { // Start query server let query_engine = Arc::new(SimpleEngine::new( store.clone(), - None, // no PromSketchStore for precompute E2E test inference_config, streaming_config.clone(), SCRAPE_INTERVAL, diff --git a/asap-query-engine/src/drivers/ingest/prometheus_remote_write.rs b/asap-query-engine/src/drivers/ingest/prometheus_remote_write.rs index 428c9de..694064c 100644 --- a/asap-query-engine/src/drivers/ingest/prometheus_remote_write.rs +++ b/asap-query-engine/src/drivers/ingest/prometheus_remote_write.rs @@ -20,6 +20,117 @@ // pub timeseries: Vec, // } +use prost::Message; + +/// Protobuf payload root for Prometheus remote-write. +#[derive(Clone, PartialEq, Message)] +pub struct WriteRequest { + #[prost(message, repeated, tag = "1")] + pub timeseries: Vec, +} + +#[derive(Clone, PartialEq, Message)] +pub struct TimeSeries { + #[prost(message, repeated, tag = "1")] + pub labels: Vec