From 72f289df503d9c4496d383362f35de152775e489 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Mon, 23 Mar 2026 15:12:39 +0800 Subject: [PATCH 01/13] chore: remove GrpcQueryHandler::put_record_batch (#7844) chore: remove GrpcQueryHandler::put_record_batch, we should use GrpcQueryHandler::handle_put_record_batch_stream instead Signed-off-by: Lei, HUANG --- src/frontend/src/instance/grpc.rs | 57 --------------------------- src/servers/src/query_handler/grpc.rs | 11 +----- src/servers/tests/mod.rs | 10 ----- 3 files changed, 1 insertion(+), 77 deletions(-) diff --git a/src/frontend/src/instance/grpc.rs b/src/frontend/src/instance/grpc.rs index c4191145f8..70ff50fadc 100644 --- a/src/frontend/src/instance/grpc.rs +++ b/src/frontend/src/instance/grpc.rs @@ -27,7 +27,6 @@ use api::v1::{ use async_stream::try_stream; use async_trait::async_trait; use auth::{PermissionChecker, PermissionCheckerRef, PermissionReq}; -use common_base::AffectedRows; use common_error::ext::BoxedError; use common_grpc::flight::do_put::DoPutResponse; use common_query::Output; @@ -260,62 +259,6 @@ impl GrpcQueryHandler for Instance { .context(server_error::ExecuteGrpcQuerySnafu) } - async fn put_record_batch( - &self, - request: servers::grpc::flight::PutRecordBatchRequest, - table_ref: &mut Option, - ctx: QueryContextRef, - ) -> server_error::Result { - let result: Result = async { - let table = if let Some(table) = table_ref { - table.clone() - } else { - let table = self - .catalog_manager() - .table( - &request.table_name.catalog_name, - &request.table_name.schema_name, - &request.table_name.table_name, - None, - ) - .await - .context(CatalogSnafu)? - .with_context(|| TableNotFoundSnafu { - table_name: request.table_name.to_string(), - })?; - *table_ref = Some(table.clone()); - table - }; - - let interceptor_ref = self.plugins.get::>(); - let interceptor = interceptor_ref.as_ref(); - interceptor.pre_bulk_insert(table.clone(), ctx.clone())?; - - self.plugins - .get::() - .as_ref() - .check_permission(ctx.current_user(), PermissionReq::BulkInsert) - .context(PermissionSnafu)?; - - // do we check limit for bulk insert? - - self.inserter - .handle_bulk_insert( - table, - request.flight_data, - request.record_batch, - request.schema_bytes, - ) - .await - .context(TableOperationSnafu) - } - .await; - - result - .map_err(BoxedError::new) - .context(server_error::ExecuteGrpcRequestSnafu) - } - fn handle_put_record_batch_stream( &self, stream: servers::grpc::flight::PutRecordBatchRequestStream, diff --git a/src/servers/src/query_handler/grpc.rs b/src/servers/src/query_handler/grpc.rs index 67d8b3890e..d66a76464e 100644 --- a/src/servers/src/query_handler/grpc.rs +++ b/src/servers/src/query_handler/grpc.rs @@ -17,15 +17,13 @@ use std::sync::Arc; use api::v1::greptime_request::Request; use async_trait::async_trait; -use common_base::AffectedRows; use common_grpc::flight::do_put::DoPutResponse; use common_query::Output; use futures::Stream; use session::context::QueryContextRef; -use table::TableRef; use crate::error::Result; -use crate::grpc::flight::{PutRecordBatchRequest, PutRecordBatchRequestStream}; +use crate::grpc::flight::PutRecordBatchRequestStream; pub type ServerGrpcQueryHandlerRef = Arc; @@ -35,13 +33,6 @@ pub type RawRecordBatch = bytes::Bytes; pub trait GrpcQueryHandler { async fn do_query(&self, query: Request, ctx: QueryContextRef) -> Result; - async fn put_record_batch( - &self, - request: PutRecordBatchRequest, - table_ref: &mut Option, - ctx: QueryContextRef, - ) -> Result; - fn handle_put_record_batch_stream( &self, stream: PutRecordBatchRequestStream, diff --git a/src/servers/tests/mod.rs b/src/servers/tests/mod.rs index e3f8f8fc79..c4f83c5e6c 100644 --- a/src/servers/tests/mod.rs +++ b/src/servers/tests/mod.rs @@ -18,7 +18,6 @@ use api::v1::greptime_request::Request; use api::v1::query_request::Query; use async_trait::async_trait; use catalog::memory::MemoryCatalogManager; -use common_base::AffectedRows; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_grpc::flight::do_put::DoPutResponse; use common_query::Output; @@ -149,15 +148,6 @@ impl GrpcQueryHandler for DummyInstance { Ok(output) } - async fn put_record_batch( - &self, - _request: servers::grpc::flight::PutRecordBatchRequest, - _table_ref: &mut Option, - _ctx: QueryContextRef, - ) -> Result { - unimplemented!() - } - fn handle_put_record_batch_stream( &self, _stream: servers::grpc::flight::PutRecordBatchRequestStream, From 78742820891c245e277260fd3f62bd478d6fdc34 Mon Sep 17 00:00:00 2001 From: "Lei, HUANG" <6406592+v0y4g3r@users.noreply.github.com> Date: Tue, 24 Mar 2026 03:39:57 +0800 Subject: [PATCH 02/13] feat(mito): flat scan for time series memtable (#7814) * feat/flat-for-time-series: ### Commit Message Enhance `TimeSeriesMemtable` with Record Batch Support - **`time_series.rs`**: - Introduced `BatchToRecordBatchContext` to facilitate conversion of batch iterators to record batch iterators. - Added `build_record_batch` method in `TimeSeriesIterBuilder` to support record batch creation. - Implemented multiple test cases to validate the functionality of record batch creation, including tests for projections, deduplication, sequence filtering, and data correctness. Signed-off-by: Lei, HUANG * feat/flat-for-time-series: Refactor `TimeSeriesMemtable` and `TimeSeriesIterBuilder` - Renamed `adapter_context` to `batch_to_record_batch` in `TimeSeriesMemtable` for clarity. - Simplified `MemtableRangeContext` initialization by removing the `batch_to_record_batch` parameter. - Added `is_record_batch` method to `TimeSeriesIterBuilder` to indicate record batch status. Signed-off-by: Lei, HUANG * feat/flat-for-time-series: ### Add Time Range Filtering and Predicate Group Enhancements - **`memtable.rs`**: Updated `IterBuilder` to include `time_range` parameter in `build_record_batch` method, enhancing record batch iteration with time range filtering. - **`time_series.rs`**: Modified `TimeSeriesIterBuilder` to use `PredicateGroup` instead of `Predicate`, and integrated `PruneTimeIterator` for time-based filtering. - **`memtable_util.rs`**: Removed unused `Predicate` import, reflecting changes in predicate handling. Signed-off-by: Lei, HUANG --------- Signed-off-by: Lei, HUANG --- src/mito2/src/memtable.rs | 6 +- src/mito2/src/memtable/bulk.rs | 4 + src/mito2/src/memtable/time_series.rs | 323 +++++++++++++++++++++-- src/mito2/src/test_util/memtable_util.rs | 1 - 4 files changed, 310 insertions(+), 24 deletions(-) diff --git a/src/mito2/src/memtable.rs b/src/mito2/src/memtable.rs index 7494ec68ed..3ebfdd3628 100644 --- a/src/mito2/src/memtable.rs +++ b/src/mito2/src/memtable.rs @@ -537,11 +537,15 @@ pub trait IterBuilder: Send + Sync { } /// Returns the record batch iterator to read the range. + /// ## Note + /// Implementations should ensure the iterator yields data within given time range. fn build_record_batch( &self, + time_range: Option<(Timestamp, Timestamp)>, metrics: Option, ) -> Result { let _metrics = metrics; + let _ = time_range; UnsupportedOperationSnafu { err_msg: "Record batch iterator is not supported by this memtable", } @@ -700,7 +704,7 @@ impl MemtableRange { metrics: Option, ) -> Result { if self.context.builder.is_record_batch() { - return self.context.builder.build_record_batch(metrics); + return self.context.builder.build_record_batch(time_range, metrics); } if let Some(context) = self.context.batch_to_record_batch.as_ref() { diff --git a/src/mito2/src/memtable/bulk.rs b/src/mito2/src/memtable/bulk.rs index 4dad4fb885..e649681b76 100644 --- a/src/mito2/src/memtable/bulk.rs +++ b/src/mito2/src/memtable/bulk.rs @@ -34,6 +34,7 @@ fn env_usize(name: &str, default: usize) -> usize { .unwrap_or(default) } +use common_time::Timestamp; use datatypes::arrow::datatypes::SchemaRef; use mito_codec::key_values::KeyValue; use rayon::prelude::*; @@ -792,6 +793,7 @@ impl IterBuilder for BulkRangeIterBuilder { fn build_record_batch( &self, + _time_range: Option<(Timestamp, Timestamp)>, metrics: Option, ) -> Result { let series_count = self.part.estimated_series_count(); @@ -825,6 +827,7 @@ impl IterBuilder for MultiBulkRangeIterBuilder { fn build_record_batch( &self, + _time_range: Option<(Timestamp, Timestamp)>, metrics: Option, ) -> Result { self.part @@ -864,6 +867,7 @@ impl IterBuilder for EncodedBulkRangeIterBuilder { fn build_record_batch( &self, + _time_range: Option<(Timestamp, Timestamp)>, metrics: Option, ) -> Result { if let Some(iter) = self diff --git a/src/mito2/src/memtable/time_series.rs b/src/mito2/src/memtable/time_series.rs index 97f5f3c9ce..d3d00d0703 100644 --- a/src/mito2/src/memtable/time_series.rs +++ b/src/mito2/src/memtable/time_series.rs @@ -51,15 +51,18 @@ use crate::memtable::bulk::part::BulkPart; use crate::memtable::simple_bulk_memtable::SimpleBulkMemtable; use crate::memtable::stats::WriteMetrics; use crate::memtable::{ - AllocTracker, BatchToRecordBatchContext, BoxedBatchIterator, IterBuilder, KeyValues, - MemScanMetrics, Memtable, MemtableBuilder, MemtableId, MemtableRange, MemtableRangeContext, - MemtableRanges, MemtableRef, MemtableStats, RangesOptions, read_column_ids_from_projection, + AllocTracker, BatchToRecordBatchContext, BoxedBatchIterator, BoxedRecordBatchIterator, + IterBuilder, KeyValues, MemScanMetrics, Memtable, MemtableBuilder, MemtableId, MemtableRange, + MemtableRangeContext, MemtableRanges, MemtableRef, MemtableStats, RangesOptions, + read_column_ids_from_projection, }; use crate::metrics::{ MEMTABLE_ACTIVE_FIELD_BUILDER_COUNT, MEMTABLE_ACTIVE_SERIES_COUNT, READ_ROWS_TOTAL, READ_STAGE_ELAPSED, }; use crate::read::dedup::LastNonNullIter; +use crate::read::prune::PruneTimeIterator; +use crate::read::scan_region::PredicateGroup; use crate::read::{Batch, BatchBuilder, BatchColumn}; use crate::region::options::MergeMode; @@ -283,25 +286,20 @@ impl Memtable for TimeSeriesMemtable { .map(|c| c.column_id) .collect() }; - let builder = Box::new(TimeSeriesIterBuilder { - series_set: self.series_set.clone(), - projection, - predicate: predicate.predicate().cloned(), - dedup: self.dedup, - merge_mode: self.merge_mode, - sequence, - }); - let adapter_context = Arc::new(BatchToRecordBatchContext::new( + let batch_to_record_batch = Arc::new(BatchToRecordBatchContext::new( self.region_metadata.clone(), read_column_ids, )); - let context = Arc::new(MemtableRangeContext::new_with_batch_to_record_batch( - self.id, - builder, - predicate, - Some(adapter_context), - )); - + let builder = Box::new(TimeSeriesIterBuilder { + series_set: self.series_set.clone(), + projection, + predicate: predicate.clone(), + dedup: self.dedup, + merge_mode: self.merge_mode, + sequence, + batch_to_record_batch, + }); + let context = Arc::new(MemtableRangeContext::new(self.id, builder, predicate)); let range_stats = self.stats(); let range = MemtableRange::new(context, range_stats); Ok(MemtableRanges { @@ -443,7 +441,7 @@ impl SeriesSet { fn iter_series( &self, projection: HashSet, - predicate: Option, + predicate: PredicateGroup, dedup: bool, merge_mode: MergeMode, sequence: Option, @@ -460,7 +458,7 @@ impl SeriesSet { self.region_metadata.clone(), self.series.clone(), projection, - predicate, + predicate.predicate().cloned(), primary_key_schema, primary_key_datatypes, self.codec.clone(), @@ -1245,10 +1243,11 @@ impl From for Values { struct TimeSeriesIterBuilder { series_set: SeriesSet, projection: HashSet, - predicate: Option, + predicate: PredicateGroup, dedup: bool, sequence: Option, merge_mode: MergeMode, + batch_to_record_batch: Arc, } impl IterBuilder for TimeSeriesIterBuilder { @@ -1268,6 +1267,25 @@ impl IterBuilder for TimeSeriesIterBuilder { Ok(Box::new(iter)) } } + + fn is_record_batch(&self) -> bool { + true + } + + fn build_record_batch( + &self, + time_range: Option<(Timestamp, Timestamp)>, + metrics: Option, + ) -> Result { + let iter = self.build(metrics)?; + let iter: BoxedBatchIterator = if let Some(time_range) = time_range { + let time_filters = self.predicate.time_filters(); + Box::new(PruneTimeIterator::new(iter, time_range, time_filters)) + } else { + iter + }; + Ok(self.batch_to_record_batch.adapt_iter(iter)) + } } #[cfg(test)] @@ -2014,4 +2032,265 @@ mod tests { all_timestamps.sort(); assert_eq!(vec![3, 4, 5, 6, 7], all_timestamps); } + + /// Helper to create a TimeSeriesIterBuilder from a memtable and schema. + fn build_iter_builder( + schema: &RegionMetadataRef, + memtable: &TimeSeriesMemtable, + projection: Option<&[ColumnId]>, + dedup: bool, + merge_mode: MergeMode, + sequence: Option, + ) -> TimeSeriesIterBuilder { + let read_column_ids = read_column_ids_from_projection(schema, projection); + let field_projection = if let Some(projection) = projection { + projection.iter().copied().collect() + } else { + schema.field_columns().map(|c| c.column_id).collect() + }; + let adapter_context = Arc::new(BatchToRecordBatchContext::new( + schema.clone(), + read_column_ids, + )); + TimeSeriesIterBuilder { + series_set: memtable.series_set.clone(), + projection: field_projection, + predicate: PredicateGroup::default(), + dedup, + merge_mode, + sequence, + batch_to_record_batch: adapter_context, + } + } + + #[test] + fn test_iter_builder_build_record_batch_basic() { + let schema = schema_for_test(); + let memtable = TimeSeriesMemtable::new(schema.clone(), 1, None, true, MergeMode::LastRow); + + let kvs = build_key_values(&schema, "hello".to_string(), 42, 10); + memtable.write(&kvs).unwrap(); + + let builder = build_iter_builder(&schema, &memtable, None, true, MergeMode::LastRow, None); + + let mut iter = builder.build_record_batch(None, None).unwrap(); + let rb = iter.next().transpose().unwrap().unwrap(); + assert_eq!(10, rb.num_rows()); + + let rb_schema = rb.schema(); + let col_names: Vec<_> = rb_schema + .fields() + .iter() + .map(|f| f.name().as_str()) + .collect(); + assert_eq!( + col_names, + vec![ + "k0", + "k1", + "v0", + "v1", + "ts", + "__primary_key", + "__sequence", + "__op_type", + ] + ); + + assert!(iter.next().is_none()); + } + + #[test] + fn test_iter_builder_build_record_batch_with_projection() { + let schema = schema_for_test(); + let memtable = TimeSeriesMemtable::new(schema.clone(), 1, None, true, MergeMode::LastRow); + + let kvs = build_key_values(&schema, "test".to_string(), 1, 5); + memtable.write(&kvs).unwrap(); + + // Project only field v0 (column_id=3) and ts (column_id=2). + let projection = vec![2, 3]; + let builder = build_iter_builder( + &schema, + &memtable, + Some(&projection), + true, + MergeMode::LastRow, + None, + ); + + let mut iter = builder.build_record_batch(None, None).unwrap(); + let rb = iter.next().transpose().unwrap().unwrap(); + assert_eq!(5, rb.num_rows()); + + let rb_schema = rb.schema(); + let col_names: Vec<_> = rb_schema + .fields() + .iter() + .map(|f| f.name().as_str()) + .collect(); + // Only projected columns + internal columns. + assert_eq!( + col_names, + vec!["v0", "ts", "__primary_key", "__sequence", "__op_type",] + ); + + assert!(iter.next().is_none()); + } + + #[test] + fn test_iter_builder_build_record_batch_multiple_series() { + let schema = schema_for_test(); + let memtable = TimeSeriesMemtable::new(schema.clone(), 1, None, true, MergeMode::LastRow); + + let kvs_a = build_key_values(&schema, "aaa".to_string(), 1, 3); + let kvs_b = build_key_values(&schema, "bbb".to_string(), 2, 4); + memtable.write(&kvs_a).unwrap(); + memtable.write(&kvs_b).unwrap(); + + let builder = build_iter_builder(&schema, &memtable, None, true, MergeMode::LastRow, None); + + let iter = builder.build_record_batch(None, None).unwrap(); + let mut total_rows = 0; + for rb in iter { + let rb = rb.unwrap(); + total_rows += rb.num_rows(); + assert_eq!(8, rb.num_columns()); + } + assert_eq!(7, total_rows); + } + + #[test] + fn test_iter_builder_build_record_batch_dedup() { + let schema = schema_for_test(); + let memtable = TimeSeriesMemtable::new(schema.clone(), 1, None, true, MergeMode::LastRow); + + // Write same data twice — dedup should keep only one copy per timestamp. + let kvs = build_key_values(&schema, "dup".to_string(), 10, 5); + memtable.write(&kvs).unwrap(); + memtable.write(&kvs).unwrap(); + + let builder = build_iter_builder(&schema, &memtable, None, true, MergeMode::LastRow, None); + + let iter = builder.build_record_batch(None, None).unwrap(); + let total_rows: usize = iter.map(|rb| rb.unwrap().num_rows()).sum(); + assert_eq!(5, total_rows); + } + + #[test] + fn test_iter_builder_build_record_batch_no_dedup() { + let schema = schema_for_test(); + let memtable = TimeSeriesMemtable::new(schema.clone(), 1, None, false, MergeMode::LastRow); + + let kvs = build_key_values(&schema, "dup".to_string(), 10, 5); + memtable.write(&kvs).unwrap(); + memtable.write(&kvs).unwrap(); + + let builder = build_iter_builder(&schema, &memtable, None, false, MergeMode::LastRow, None); + + let iter = builder.build_record_batch(None, None).unwrap(); + let total_rows: usize = iter.map(|rb| rb.unwrap().num_rows()).sum(); + assert_eq!(10, total_rows); + } + + #[test] + fn test_iter_builder_build_record_batch_with_sequence_filter() { + let schema = schema_for_test(); + let memtable = TimeSeriesMemtable::new(schema.clone(), 1, None, true, MergeMode::LastRow); + + // build_key_values creates a mutation with base sequence=0. + // Each row gets sequence = base + row_index, so 5 rows get sequences 0,1,2,3,4. + let kvs = build_key_values(&schema, "seq".to_string(), 1, 5); + memtable.write(&kvs).unwrap(); + + // Filter to sequence > 4 — should yield no rows. + let builder = build_iter_builder( + &schema, + &memtable, + None, + true, + MergeMode::LastRow, + Some(SequenceRange::Gt { min: 4 }), + ); + + let iter = builder.build_record_batch(None, None).unwrap(); + let total_rows: usize = iter.map(|rb| rb.unwrap().num_rows()).sum(); + assert_eq!(0, total_rows); + + // Filter to sequence <= 2 — should yield 3 rows (sequences 0, 1, 2). + let builder = build_iter_builder( + &schema, + &memtable, + None, + true, + MergeMode::LastRow, + Some(SequenceRange::LtEq { max: 2 }), + ); + + let iter = builder.build_record_batch(None, None).unwrap(); + let total_rows: usize = iter.map(|rb| rb.unwrap().num_rows()).sum(); + assert_eq!(3, total_rows); + } + + #[test] + fn test_iter_builder_build_record_batch_data_correctness() { + use datatypes::arrow::array::{ + Float64Array, Int64Array, TimestampMillisecondArray, UInt8Array, + }; + + let schema = schema_for_test(); + let memtable = TimeSeriesMemtable::new(schema.clone(), 1, None, true, MergeMode::LastRow); + + let kvs = build_key_values(&schema, "check".to_string(), 7, 3); + memtable.write(&kvs).unwrap(); + + let builder = build_iter_builder(&schema, &memtable, None, true, MergeMode::LastRow, None); + + let mut iter = builder.build_record_batch(None, None).unwrap(); + let rb = iter.next().transpose().unwrap().unwrap(); + assert_eq!(3, rb.num_rows()); + + // Verify timestamp values. + let ts_col = rb + .column_by_name("ts") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + let timestamps: Vec<_> = (0..ts_col.len()).map(|i| ts_col.value(i)).collect(); + assert_eq!(vec![0, 1, 2], timestamps); + + // Verify field v0 values. + let v0_col = rb + .column_by_name("v0") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + let v0_values: Vec<_> = (0..v0_col.len()).map(|i| v0_col.value(i)).collect(); + assert_eq!(vec![0, 1, 2], v0_values); + + // Verify field v1 values. + let v1_col = rb + .column_by_name("v1") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + let v1_values: Vec<_> = (0..v1_col.len()).map(|i| v1_col.value(i)).collect(); + assert_eq!(vec![0.0, 1.0, 2.0], v1_values); + + // Verify op_type is all Put (1). + let op_col = rb + .column_by_name("__op_type") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + for i in 0..op_col.len() { + assert_eq!(OpType::Put as u8, op_col.value(i)); + } + + assert!(iter.next().is_none()); + } } diff --git a/src/mito2/src/test_util/memtable_util.rs b/src/mito2/src/test_util/memtable_util.rs index 58ea49fa41..8917875250 100644 --- a/src/mito2/src/test_util/memtable_util.rs +++ b/src/mito2/src/test_util/memtable_util.rs @@ -31,7 +31,6 @@ use store_api::metadata::{ ColumnMetadata, RegionMetadata, RegionMetadataBuilder, RegionMetadataRef, }; use store_api::storage::{ColumnId, RegionId, SequenceNumber, SequenceRange}; -use table::predicate::Predicate; use crate::error::Result; use crate::memtable::bulk::part::BulkPart; From f999d5e70e3076d7c45223613a7d6465bfa07c3e Mon Sep 17 00:00:00 2001 From: Ruihang Xia Date: Tue, 24 Mar 2026 08:11:37 +0800 Subject: [PATCH 03/13] feat: avoid some vector-array conversions on flat projection (#7804) * perf(mito2): optimize flat projection conversion * shrink the diff size Signed-off-by: Ruihang Xia * apply gemini's sugg Signed-off-by: Ruihang Xia * nit Signed-off-by: Ruihang Xia --------- Signed-off-by: Ruihang Xia --- src/mito2/src/read/flat_projection.rs | 78 +++++++++++++++++++++++++-- src/mito2/src/read/projection.rs | 23 ++++---- src/mito2/src/read/stream.rs | 5 +- 3 files changed, 89 insertions(+), 17 deletions(-) diff --git a/src/mito2/src/read/flat_projection.rs b/src/mito2/src/read/flat_projection.rs index 3e0f1169df..02b4c6b3c1 100644 --- a/src/mito2/src/read/flat_projection.rs +++ b/src/mito2/src/read/flat_projection.rs @@ -18,18 +18,21 @@ use std::sync::Arc; use api::v1::SemanticType; use common_error::ext::BoxedError; -use common_recordbatch::error::{ArrowComputeSnafu, ExternalSnafu}; +use common_recordbatch::error::{ArrowComputeSnafu, ExternalSnafu, NewDfRecordBatchSnafu}; use common_recordbatch::{DfRecordBatch, RecordBatch}; -use datatypes::arrow::datatypes::Field; +use datatypes::arrow::array::Array; +use datatypes::arrow::datatypes::{DataType as ArrowDataType, Field}; use datatypes::prelude::{ConcreteDataType, DataType}; use datatypes::schema::{Schema, SchemaRef}; +use datatypes::value::Value; use datatypes::vectors::Helper; use snafu::{OptionExt, ResultExt}; use store_api::metadata::{RegionMetadata, RegionMetadataRef}; use store_api::storage::ColumnId; +use crate::cache::CacheStrategy; use crate::error::{InvalidRequestSnafu, RecordBatchSnafu, Result}; -use crate::read::projection::read_column_ids_from_projection; +use crate::read::projection::{read_column_ids_from_projection, repeated_vector_with_cache}; use crate::sst::parquet::flat_format::sst_column_id_indices; use crate::sst::parquet::format::FormatProjection; use crate::sst::{ @@ -248,12 +251,55 @@ impl FlatProjectionMapper { pub(crate) fn convert( &self, batch: &datatypes::arrow::record_batch::RecordBatch, + cache_strategy: &CacheStrategy, ) -> common_recordbatch::error::Result { if self.is_empty_projection { return RecordBatch::new_with_count(self.output_schema.clone(), batch.num_rows()); } - let columns = self.project_vectors(batch)?; - RecordBatch::new(self.output_schema.clone(), columns) + // Construct output record batch directly from Arrow arrays to avoid + // Arrow -> Vector -> Arrow roundtrips in the hot path. + let mut arrays = Vec::with_capacity(self.output_schema.num_columns()); + for (output_idx, index) in self.batch_indices.iter().enumerate() { + let mut array = batch.column(*index).clone(); + // Cast dictionary values to the target type. + if let ArrowDataType::Dictionary(_key_type, value_type) = array.data_type() { + // When a string dictionary column contains only a single value, reuse a cached + // repeated vector to avoid repeatedly expanding the dictionary. + if let Some(dict_array) = single_value_string_dictionary( + &array, + &self.output_schema.column_schemas()[output_idx].data_type, + value_type.as_ref(), + ) { + let dict_values = dict_array.values(); + let value = if dict_values.is_null(0) { + Value::Null + } else { + Value::from(datatypes::arrow_array::string_array_value(dict_values, 0)) + }; + + let repeated = repeated_vector_with_cache( + &self.output_schema.column_schemas()[output_idx].data_type, + &value, + batch.num_rows(), + cache_strategy, + )?; + array = repeated.to_arrow_array(); + } else { + let casted = datatypes::arrow::compute::cast(&array, value_type) + .context(ArrowComputeSnafu)?; + array = casted; + } + } + arrays.push(array); + } + + let df_record_batch = + DfRecordBatch::try_new(self.output_schema.arrow_schema().clone(), arrays) + .context(NewDfRecordBatchSnafu)?; + Ok(RecordBatch::from_df_record_batch( + self.output_schema.clone(), + df_record_batch, + )) } /// Projects columns from the input batch and converts them into vectors. @@ -281,6 +327,28 @@ impl FlatProjectionMapper { } } +fn single_value_string_dictionary<'a>( + array: &'a Arc, + output_type: &ConcreteDataType, + value_type: &ArrowDataType, +) -> Option<&'a datatypes::arrow::array::DictionaryArray> { + if !matches!( + value_type, + ArrowDataType::Utf8 | ArrowDataType::LargeUtf8 | ArrowDataType::Utf8View + ) || !output_type.is_string() + { + return None; + } + + let dict_array = array + .as_any() + .downcast_ref::>()?; + + (dict_array.values().len() == 1 && dict_array.null_count() == 0).then_some(dict_array) +} + /// Returns ids and datatypes of columns of the output batch after applying the `projection`. /// /// It adds the time index column if it doesn't present in the projection. diff --git a/src/mito2/src/read/projection.rs b/src/mito2/src/read/projection.rs index 2c000e7bdc..b5b6904521 100644 --- a/src/mito2/src/read/projection.rs +++ b/src/mito2/src/read/projection.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use api::v1::SemanticType; use common_error::ext::BoxedError; use common_recordbatch::RecordBatch; -use common_recordbatch::error::ExternalSnafu; +use common_recordbatch::error::{DataTypesSnafu, ExternalSnafu}; use datatypes::prelude::{ConcreteDataType, DataType}; use datatypes::schema::{Schema, SchemaRef}; use datatypes::value::Value; @@ -37,7 +37,7 @@ use crate::read::Batch; use crate::read::flat_projection::FlatProjectionMapper; /// Only cache vector when its length `<=` this value. -const MAX_VECTOR_LENGTH_TO_CACHE: usize = 16384; +pub(crate) const MAX_VECTOR_LENGTH_TO_CACHE: usize = 16384; /// Wrapper enum for different projection mapper implementations. pub enum ProjectionMapper { @@ -423,7 +423,7 @@ enum BatchIndex { } /// Gets a vector with repeated values from specific cache or creates a new one. -fn repeated_vector_with_cache( +pub(crate) fn repeated_vector_with_cache( data_type: &ConcreteDataType, value: &Value, num_rows: usize, @@ -450,7 +450,7 @@ fn repeated_vector_with_cache( } /// Returns a vector with repeated values. -fn new_repeated_vector( +pub(crate) fn new_repeated_vector( data_type: &ConcreteDataType, value: &Value, num_rows: usize, @@ -458,8 +458,7 @@ fn new_repeated_vector( let mut mutable_vector = data_type.create_mutable_vector(1); mutable_vector .try_push_value_ref(&value.as_value_ref()) - .map_err(BoxedError::new) - .context(ExternalSnafu)?; + .context(DataTypesSnafu)?; // This requires an additional allocation. let base_vector = mutable_vector.to_vector(); Ok(base_vector.replicate(&[num_rows])) @@ -809,6 +808,7 @@ mod tests { .num_fields(2) .build(), ); + let cache = CacheStrategy::Disabled; let mapper = ProjectionMapper::all(&metadata, true).unwrap(); assert_eq!([0, 1, 2, 3, 4], mapper.column_ids()); assert_eq!( @@ -823,7 +823,7 @@ mod tests { ); let batch = new_flat_batch(Some(0), &[(1, 1), (2, 2)], &[(3, 3), (4, 4)], 3); - let record_batch = mapper.as_flat().unwrap().convert(&batch).unwrap(); + let record_batch = mapper.as_flat().unwrap().convert(&batch, &cache).unwrap(); let expect = "\ +---------------------+----+----+----+----+ | ts | k0 | k1 | v0 | v1 | @@ -843,6 +843,7 @@ mod tests { .num_fields(2) .build(), ); + let cache = CacheStrategy::Disabled; // Columns v1, k0 let mapper = ProjectionMapper::new(&metadata, [4, 1].into_iter(), true).unwrap(); assert_eq!([4, 1], mapper.column_ids()); @@ -856,7 +857,7 @@ mod tests { ); let batch = new_flat_batch(None, &[(1, 1)], &[(4, 4)], 3); - let record_batch = mapper.as_flat().unwrap().convert(&batch).unwrap(); + let record_batch = mapper.as_flat().unwrap().convert(&batch, &cache).unwrap(); let expect = "\ +----+----+ | v1 | k0 | @@ -876,6 +877,7 @@ mod tests { .num_fields(2) .build(), ); + let cache = CacheStrategy::Disabled; // Output columns v1, k0. Read also includes v0. let mapper = ProjectionMapper::new_with_read_columns( &metadata, @@ -887,7 +889,7 @@ mod tests { assert_eq!([4, 1, 3], mapper.column_ids()); let batch = new_flat_batch(None, &[(1, 1)], &[(3, 3), (4, 4)], 3); - let record_batch = mapper.as_flat().unwrap().convert(&batch).unwrap(); + let record_batch = mapper.as_flat().unwrap().convert(&batch, &cache).unwrap(); let expect = "\ +----+----+ | v1 | k0 | @@ -907,6 +909,7 @@ mod tests { .num_fields(2) .build(), ); + let cache = CacheStrategy::Disabled; // Empty projection let mapper = ProjectionMapper::new(&metadata, [].into_iter(), true).unwrap(); assert_eq!([0], mapper.column_ids()); // Should still read the time index column @@ -918,7 +921,7 @@ mod tests { ); let batch = new_flat_batch(Some(0), &[], &[], 3); - let record_batch = flat_mapper.convert(&batch).unwrap(); + let record_batch = flat_mapper.convert(&batch, &cache).unwrap(); assert_eq!(3, record_batch.num_rows()); assert_eq!(0, record_batch.num_columns()); assert!(record_batch.schema.is_empty()); diff --git a/src/mito2/src/read/stream.rs b/src/mito2/src/read/stream.rs index dd85616241..80002147ea 100644 --- a/src/mito2/src/read/stream.rs +++ b/src/mito2/src/read/stream.rs @@ -99,7 +99,8 @@ impl ConvertBatchStream { let mapper = self.projection_mapper.as_flat().unwrap(); for batch in flat_batch.batches { - self.pending.push_back(mapper.convert(&batch)?); + self.pending + .push_back(mapper.convert(&batch, &self.cache_strategy)?); } } } @@ -114,7 +115,7 @@ impl ConvertBatchStream { // Safety: Only flat format returns this batch. let mapper = self.projection_mapper.as_flat().unwrap(); - mapper.convert(&df_record_batch) + mapper.convert(&df_record_batch, &self.cache_strategy) } } } From 223f6cfdf727f9b7622126d24a98ac19bec61353 Mon Sep 17 00:00:00 2001 From: dennis zhuang Date: Tue, 24 Mar 2026 10:05:16 +0800 Subject: [PATCH 04/13] feat: supports sst_format for x-greptime-hints and database options (#7843) Signed-off-by: Dennis Zhuang --- src/table/src/requests.rs | 9 ++- tests-integration/tests/http.rs | 38 +++++++++ .../common/alter/alter_database.result | 79 +++++++++++++++++++ .../common/alter/alter_database.sql | 22 +++++- 4 files changed, 144 insertions(+), 4 deletions(-) diff --git a/src/table/src/requests.rs b/src/table/src/requests.rs index 43fc36644b..15b4278f51 100644 --- a/src/table/src/requests.rs +++ b/src/table/src/requests.rs @@ -36,8 +36,9 @@ use store_api::metric_engine_consts::{ LOGICAL_TABLE_METADATA_KEY, PHYSICAL_TABLE_METADATA_KEY, is_metric_engine_option_key, }; use store_api::mito_engine_options::{ - APPEND_MODE_KEY, COMPACTION_TYPE, MEMTABLE_TYPE, MERGE_MODE_KEY, TWCS_FALLBACK_TO_LOCAL, - TWCS_MAX_OUTPUT_FILE_SIZE, TWCS_TIME_WINDOW, TWCS_TRIGGER_FILE_NUM, is_mito_engine_option_key, + APPEND_MODE_KEY, COMPACTION_TYPE, MEMTABLE_TYPE, MERGE_MODE_KEY, SST_FORMAT_KEY, + TWCS_FALLBACK_TO_LOCAL, TWCS_MAX_OUTPUT_FILE_SIZE, TWCS_TIME_WINDOW, TWCS_TRIGGER_FILE_NUM, + is_mito_engine_option_key, }; use store_api::region_request::{SetRegionOption, UnsetRegionOption}; @@ -56,13 +57,14 @@ pub const TABLE_DATA_MODEL_TRACE_V1: &str = "greptime_trace_v1"; pub const OTLP_METRIC_COMPAT_KEY: &str = "otlp_metric_compat"; pub const OTLP_METRIC_COMPAT_PROM: &str = "prom"; -pub const VALID_TABLE_OPTION_KEYS: [&str; 12] = [ +pub const VALID_TABLE_OPTION_KEYS: [&str; 13] = [ // common keys: WRITE_BUFFER_SIZE_KEY, TTL_KEY, STORAGE_KEY, COMMENT_KEY, SKIP_WAL_KEY, + SST_FORMAT_KEY, // file engine keys: FILE_TABLE_LOCATION_KEY, FILE_TABLE_FORMAT_KEY, @@ -94,6 +96,7 @@ static VALID_DB_OPT_KEYS: Lazy> = Lazy::new(|| { set.insert(TWCS_TIME_WINDOW); set.insert(TWCS_TRIGGER_FILE_NUM); set.insert(TWCS_MAX_OUTPUT_FILE_SIZE); + set.insert(SST_FORMAT_KEY); set }); diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index c259d3ff24..65e56fa15e 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -148,6 +148,7 @@ macro_rules! http_tests { test_jaeger_query_api_for_trace_v1, test_influxdb_write, + test_influxdb_write_with_hints, test_http_memory_limit, ); )* @@ -3638,6 +3639,43 @@ transform: guard.remove_all().await; } +pub async fn test_influxdb_write_with_hints(storage_type: StorageType) { + common_telemetry::init_default_ut_logging(); + let (app, mut guard) = + setup_test_http_app_with_frontend(storage_type, "test_influxdb_write_with_hints").await; + + let client = TestClient::new(app).await; + + let result = client + .post("/v1/influxdb/write?db=public") + .header("x-greptime-hints", "sst_format=flat,ttl=30d,skip_wal=true") + .body("sst_fmt_table,host=host1 cpu=1.2 1664370459457010101") + .send() + .await; + assert_eq!(result.status(), 204); + + let res = client + .get("/v1/sql?sql=show create table sst_fmt_table") + .send() + .await; + assert_eq!(res.status(), StatusCode::OK); + let resp = res.text().await; + assert!( + resp.contains("sst_format = 'flat'"), + "expected sst_format = 'flat' in SHOW CREATE TABLE output, got: {resp}" + ); + assert!( + resp.contains("ttl = '30days'"), + "expected ttl = '30days' in SHOW CREATE TABLE output, got: {resp}" + ); + assert!( + resp.contains("skip_wal = 'true'"), + "expected skip_wal = 'true' in SHOW CREATE TABLE output, got: {resp}" + ); + + guard.remove_all().await; +} + /// Test one-to-many VRL pipeline expansion. /// This test verifies that a VRL processor can return an array, which results in /// multiple output rows from a single input row. diff --git a/tests/cases/standalone/common/alter/alter_database.result b/tests/cases/standalone/common/alter/alter_database.result index 911ef5ddfc..2fccce10de 100644 --- a/tests/cases/standalone/common/alter/alter_database.result +++ b/tests/cases/standalone/common/alter/alter_database.result @@ -314,6 +314,85 @@ SHOW CREATE DATABASE alter_database; | | ) | +----------------+----------------------------------------------+ +-- Test sst_format option +ALTER DATABASE alter_database SET 'sst_format'='flat'; + +Affected Rows: 0 + +SHOW CREATE DATABASE alter_database; + ++----------------+----------------------------------------------+ +| Database | Create Database | ++----------------+----------------------------------------------+ +| alter_database | CREATE DATABASE IF NOT EXISTS alter_database | +| | WITH( | +| | 'compaction.twcs.time_window' = '30m', | +| | 'compaction.type' = 'twcs', | +| | sst_format = 'flat' | +| | ) | ++----------------+----------------------------------------------+ + +USE alter_database; + +Affected Rows: 0 + +CREATE TABLE monitor(ts TIMESTAMP TIME INDEX); + +Affected Rows: 0 + +SHOW CREATE TABLE monitor; + ++---------+----------------------------------------+ +| Table | Create Table | ++---------+----------------------------------------+ +| monitor | CREATE TABLE IF NOT EXISTS "monitor" ( | +| | "ts" TIMESTAMP(3) NOT NULL, | +| | TIME INDEX ("ts") | +| | ) | +| | | +| | ENGINE=mito | +| | WITH( | +| | sst_format = 'flat' | +| | ) | ++---------+----------------------------------------+ + +USE public; + +Affected Rows: 0 + +ALTER DATABASE alter_database SET 'sst_format'='primary_key'; + +Affected Rows: 0 + +SHOW CREATE DATABASE alter_database; + ++----------------+----------------------------------------------+ +| Database | Create Database | ++----------------+----------------------------------------------+ +| alter_database | CREATE DATABASE IF NOT EXISTS alter_database | +| | WITH( | +| | 'compaction.twcs.time_window' = '30m', | +| | 'compaction.type' = 'twcs', | +| | sst_format = 'primary_key' | +| | ) | ++----------------+----------------------------------------------+ + +ALTER DATABASE alter_database UNSET 'sst_format'; + +Affected Rows: 0 + +SHOW CREATE DATABASE alter_database; + ++----------------+----------------------------------------------+ +| Database | Create Database | ++----------------+----------------------------------------------+ +| alter_database | CREATE DATABASE IF NOT EXISTS alter_database | +| | WITH( | +| | 'compaction.twcs.time_window' = '30m', | +| | 'compaction.type' = 'twcs' | +| | ) | ++----------------+----------------------------------------------+ + DROP DATABASE alter_database; Affected Rows: 0 diff --git a/tests/cases/standalone/common/alter/alter_database.sql b/tests/cases/standalone/common/alter/alter_database.sql index 1b2f75637a..33b309153e 100644 --- a/tests/cases/standalone/common/alter/alter_database.sql +++ b/tests/cases/standalone/common/alter/alter_database.sql @@ -90,5 +90,25 @@ ALTER DATABASE alter_database UNSET 'ttl'; SHOW CREATE DATABASE alter_database; -DROP DATABASE alter_database; +-- Test sst_format option +ALTER DATABASE alter_database SET 'sst_format'='flat'; +SHOW CREATE DATABASE alter_database; + +USE alter_database; + +CREATE TABLE monitor(ts TIMESTAMP TIME INDEX); + +SHOW CREATE TABLE monitor; + +USE public; + +ALTER DATABASE alter_database SET 'sst_format'='primary_key'; + +SHOW CREATE DATABASE alter_database; + +ALTER DATABASE alter_database UNSET 'sst_format'; + +SHOW CREATE DATABASE alter_database; + +DROP DATABASE alter_database; From 7afe16ddf75d8857ad75d98108e4740c87eac966 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Mar 2026 10:15:06 +0800 Subject: [PATCH 05/13] chore(deps): bump rustls-webpki from 0.103.3 to 0.103.10 (#7847) Bumps [rustls-webpki](https://github.com/rustls/webpki) from 0.103.3 to 0.103.10. - [Release notes](https://github.com/rustls/webpki/releases) - [Commits](https://github.com/rustls/webpki/compare/v/0.103.3...v/0.103.10) --- updated-dependencies: - dependency-name: rustls-webpki dependency-version: 0.103.10 dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 073ae03525..2e419019c7 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7301,7 +7301,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" dependencies = [ "cfg-if", - "windows-targets 0.52.6", + "windows-targets 0.48.5", ] [[package]] @@ -11635,9 +11635,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.3" +version = "0.103.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4a72fe2bcf7a6ac6fd7d0b9e5cb68aeb7d4c0a0271730218b3e92d43b4eb435" +checksum = "df33b2b81ac578cabaf06b89b0631153a3f416b0a886e8a7a1707fb51abbd1ef" dependencies = [ "ring", "rustls-pki-types", From 6bebf93caf18022e985da867be3d703e67bb002c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 24 Mar 2026 10:15:27 +0800 Subject: [PATCH 06/13] chore(deps): bump tar from 0.4.44 to 0.4.45 (#7846) Bumps [tar](https://github.com/alexcrichton/tar-rs) from 0.4.44 to 0.4.45. - [Commits](https://github.com/alexcrichton/tar-rs/compare/0.4.44...0.4.45) --- updated-dependencies: - dependency-name: tar dependency-version: 0.4.45 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2e419019c7..32f9aa27d4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -13404,9 +13404,9 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "tar" -version = "0.4.44" +version = "0.4.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d863878d212c87a19c1a610eb53bb01fe12951c0501cf5a0d65f724914a667a" +checksum = "22692a6476a21fa75fdfc11d452fda482af402c008cdbaf3476414e122040973" dependencies = [ "filetime", "libc", From 5231ee40c8666561732a63cb043c3a4c08cd50c9 Mon Sep 17 00:00:00 2001 From: Yingwen Date: Tue, 24 Mar 2026 11:57:18 +0800 Subject: [PATCH 07/13] feat: add parquet pk prefilter helpers (#7850) * feat: extract parquet pk prefilter helpers Signed-off-by: evenyag * chore: fmt code Signed-off-by: evenyag * chore: fix warnings Signed-off-by: evenyag * chore: update todo Signed-off-by: evenyag --------- Signed-off-by: evenyag --- src/mito2/src/sst/parquet.rs | 1 + src/mito2/src/sst/parquet/prefilter.rs | 528 +++++++++++++++++++++++++ 2 files changed, 529 insertions(+) create mode 100644 src/mito2/src/sst/parquet/prefilter.rs diff --git a/src/mito2/src/sst/parquet.rs b/src/mito2/src/sst/parquet.rs index 26bed76fd6..fb8e1d1fc2 100644 --- a/src/mito2/src/sst/parquet.rs +++ b/src/mito2/src/sst/parquet.rs @@ -29,6 +29,7 @@ pub mod flat_format; pub mod format; pub(crate) mod helper; pub(crate) mod metadata; +pub mod prefilter; pub mod reader; pub mod row_group; pub mod row_selection; diff --git a/src/mito2/src/sst/parquet/prefilter.rs b/src/mito2/src/sst/parquet/prefilter.rs new file mode 100644 index 0000000000..5de2e3512f --- /dev/null +++ b/src/mito2/src/sst/parquet/prefilter.rs @@ -0,0 +1,528 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Helpers for parquet prefiltering. + +use std::ops::Range; + +use api::v1::SemanticType; +use common_recordbatch::filter::SimpleFilterEvaluator; +use datatypes::arrow::array::{BinaryArray, BooleanArray}; +use datatypes::arrow::record_batch::RecordBatch; +use mito_codec::primary_key_filter::is_partition_column; +use mito_codec::row_converter::PrimaryKeyFilter; +use snafu::{OptionExt, ResultExt}; +use store_api::metadata::{RegionMetadata, RegionMetadataRef}; + +use crate::error::{ComputeArrowSnafu, Result, UnexpectedSnafu}; +use crate::sst::parquet::flat_format::primary_key_column_index; +use crate::sst::parquet::format::PrimaryKeyArray; + +#[cfg_attr(not(test), allow(dead_code))] +pub(crate) fn matching_row_ranges_by_primary_key( + input: &RecordBatch, + pk_filter: &mut dyn PrimaryKeyFilter, +) -> Result>> { + let primary_key_index = primary_key_column_index(input.num_columns()); + let pk_dict_array = input + .column(primary_key_index) + .as_any() + .downcast_ref::() + .context(UnexpectedSnafu { + reason: "Primary key column is not a dictionary array", + })?; + let pk_values = pk_dict_array + .values() + .as_any() + .downcast_ref::() + .context(UnexpectedSnafu { + reason: "Primary key values are not binary array", + })?; + let keys = pk_dict_array.keys(); + let key_values = keys.values(); + + if key_values.is_empty() { + return Ok(std::iter::once(0..input.num_rows()).collect()); + } + + let mut matched_row_ranges: Vec> = Vec::new(); + let mut start = 0; + while start < key_values.len() { + let key = key_values[start]; + let mut end = start + 1; + while end < key_values.len() && key_values[end] == key { + end += 1; + } + + if pk_filter.matches(pk_values.value(key as usize)) { + if let Some(last) = matched_row_ranges.last_mut() + && last.end == start + { + last.end = end; + } else { + matched_row_ranges.push(start..end); + } + } + + start = end; + } + + Ok(matched_row_ranges) +} + +#[cfg_attr(not(test), allow(dead_code))] +pub(crate) fn prefilter_flat_batch_by_primary_key( + input: RecordBatch, + pk_filter: &mut dyn PrimaryKeyFilter, +) -> Result> { + if input.num_rows() == 0 { + return Ok(Some(input)); + } + + let matched_row_ranges = matching_row_ranges_by_primary_key(&input, pk_filter)?; + if matched_row_ranges.is_empty() { + return Ok(None); + } + + if matched_row_ranges.len() == 1 + && matched_row_ranges[0].start == 0 + && matched_row_ranges[0].end == input.num_rows() + { + return Ok(Some(input)); + } + + if matched_row_ranges.len() == 1 { + let span = &matched_row_ranges[0]; + return Ok(Some(input.slice(span.start, span.end - span.start))); + } + + let mut mask = vec![false; input.num_rows()]; + for span in matched_row_ranges { + mask[span].fill(true); + } + + let filtered = + datatypes::arrow::compute::filter_record_batch(&input, &BooleanArray::from(mask)) + .context(ComputeArrowSnafu)?; + if filtered.num_rows() == 0 { + Ok(None) + } else { + Ok(Some(filtered)) + } +} + +#[cfg_attr(not(test), allow(dead_code))] +pub(crate) fn retain_usable_primary_key_filters( + sst_metadata: &RegionMetadataRef, + expected_metadata: Option<&RegionMetadata>, + filters: &mut Vec, +) { + filters.retain(|filter| is_usable_primary_key_filter(sst_metadata, expected_metadata, filter)); +} + +#[cfg_attr(not(test), allow(dead_code))] +pub(crate) fn is_usable_primary_key_filter( + sst_metadata: &RegionMetadataRef, + expected_metadata: Option<&RegionMetadata>, + filter: &SimpleFilterEvaluator, +) -> bool { + // TODO(yingwen): The primary key filter always skips the partition column. Consider using a flag + // to control this behavior. We can remove this behavior after we remove the PartitionTreeMemtable. + if is_partition_column(filter.column_name()) { + return false; + } + + let sst_column = match expected_metadata { + Some(expected_metadata) => { + let Some(expected_column) = expected_metadata.column_by_name(filter.column_name()) + else { + return false; + }; + let Some(sst_column) = sst_metadata.column_by_id(expected_column.column_id) else { + return false; + }; + + if sst_column.column_schema.name != expected_column.column_schema.name + || sst_column.semantic_type != expected_column.semantic_type + || sst_column.column_schema.data_type != expected_column.column_schema.data_type + { + return false; + } + + sst_column + } + None => { + let Some(sst_column) = sst_metadata.column_by_name(filter.column_name()) else { + return false; + }; + sst_column + } + }; + + sst_column.semantic_type == SemanticType::Tag + && sst_metadata + .primary_key_index(sst_column.column_id) + .is_some() +} + +#[cfg_attr(not(test), allow(dead_code))] +pub(crate) struct CachedPrimaryKeyFilter { + inner: Box, + last_primary_key: Vec, + last_match: Option, +} + +impl CachedPrimaryKeyFilter { + #[cfg_attr(not(test), allow(dead_code))] + pub(crate) fn new(inner: Box) -> Self { + Self { + inner, + last_primary_key: Vec::new(), + last_match: None, + } + } +} + +impl PrimaryKeyFilter for CachedPrimaryKeyFilter { + fn matches(&mut self, pk: &[u8]) -> bool { + if let Some(last_match) = self.last_match + && self.last_primary_key == pk + { + return last_match; + } + + let matched = self.inner.matches(pk); + self.last_primary_key.clear(); + self.last_primary_key.extend_from_slice(pk); + self.last_match = Some(matched); + matched + } +} + +#[cfg_attr(not(test), allow(dead_code))] +pub(crate) fn batch_single_primary_key(batch: &RecordBatch) -> Result> { + let primary_key_index = primary_key_column_index(batch.num_columns()); + let pk_dict_array = batch + .column(primary_key_index) + .as_any() + .downcast_ref::() + .context(UnexpectedSnafu { + reason: "Primary key column is not a dictionary array", + })?; + let pk_values = pk_dict_array + .values() + .as_any() + .downcast_ref::() + .context(UnexpectedSnafu { + reason: "Primary key values are not binary array", + })?; + let keys = pk_dict_array.keys(); + if keys.is_empty() { + return Ok(None); + } + + let first_key = keys.value(0); + if first_key != keys.value(keys.len() - 1) { + return Ok(None); + } + + Ok(Some(pk_values.value(first_key as usize))) +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + use std::sync::atomic::{AtomicUsize, Ordering}; + + use api::v1::SemanticType; + use common_recordbatch::filter::SimpleFilterEvaluator; + use datafusion_expr::{col, lit}; + use datatypes::arrow::array::{ + ArrayRef, BinaryArray, DictionaryArray, TimestampMillisecondArray, UInt8Array, UInt32Array, + UInt64Array, + }; + use datatypes::arrow::datatypes::{Schema, UInt32Type}; + use datatypes::arrow::record_batch::RecordBatch; + use datatypes::prelude::ConcreteDataType; + use mito_codec::row_converter::{PrimaryKeyFilter, build_primary_key_codec}; + use store_api::codec::PrimaryKeyEncoding; + use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataBuilder}; + use store_api::storage::ColumnSchema; + + use super::*; + use crate::sst::internal_fields; + use crate::sst::parquet::format::ReadFormat; + use crate::test_util::sst_util::{ + new_primary_key, sst_region_metadata, sst_region_metadata_with_encoding, + }; + + fn new_test_filters(exprs: &[datafusion_expr::Expr]) -> Vec { + exprs + .iter() + .filter_map(SimpleFilterEvaluator::try_new) + .collect() + } + + fn expected_metadata_with_reused_tag_name( + old_metadata: &RegionMetadata, + ) -> Arc { + let mut builder = RegionMetadataBuilder::new(old_metadata.region_id); + builder + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "tag_0".to_string(), + ConcreteDataType::string_datatype(), + true, + ), + semantic_type: SemanticType::Tag, + column_id: 10, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "tag_1".to_string(), + ConcreteDataType::string_datatype(), + true, + ), + semantic_type: SemanticType::Tag, + column_id: 1, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "field_0".to_string(), + ConcreteDataType::uint64_datatype(), + true, + ), + semantic_type: SemanticType::Field, + column_id: 2, + }) + .push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "ts".to_string(), + ConcreteDataType::timestamp_millisecond_datatype(), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 3, + }) + .primary_key(vec![10, 1]); + + Arc::new(builder.build().unwrap()) + } + + fn new_raw_batch_with_metadata( + metadata: Arc, + primary_keys: &[&[u8]], + field_values: &[u64], + ) -> RecordBatch { + assert_eq!(primary_keys.len(), field_values.len()); + + let arrow_schema = metadata.schema.arrow_schema(); + let field_column = arrow_schema + .field(arrow_schema.index_of("field_0").unwrap()) + .clone(); + let time_index_column = arrow_schema + .field(arrow_schema.index_of("ts").unwrap()) + .clone(); + let mut fields = vec![field_column, time_index_column]; + fields.extend( + internal_fields() + .into_iter() + .map(|field| field.as_ref().clone()), + ); + let schema = Arc::new(Schema::new(fields)); + + let mut dict_values = Vec::new(); + let mut keys = Vec::with_capacity(primary_keys.len()); + for pk in primary_keys { + let key = dict_values + .iter() + .position(|existing: &&[u8]| existing == pk) + .unwrap_or_else(|| { + dict_values.push(*pk); + dict_values.len() - 1 + }); + keys.push(key as u32); + } + + let pk_array: ArrayRef = Arc::new(DictionaryArray::::new( + UInt32Array::from(keys), + Arc::new(BinaryArray::from_iter_values(dict_values.iter().copied())), + )); + + RecordBatch::try_new( + schema, + vec![ + Arc::new(UInt64Array::from(field_values.to_vec())), + Arc::new(TimestampMillisecondArray::from_iter_values( + 0..primary_keys.len() as i64, + )), + pk_array, + Arc::new(UInt64Array::from(vec![1; primary_keys.len()])), + Arc::new(UInt8Array::from(vec![1; primary_keys.len()])), + ], + ) + .unwrap() + } + + fn new_raw_batch(primary_keys: &[&[u8]], field_values: &[u64]) -> RecordBatch { + new_raw_batch_with_metadata(Arc::new(sst_region_metadata()), primary_keys, field_values) + } + + fn field_values(batch: &RecordBatch) -> Vec { + batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap() + .values() + .to_vec() + } + + #[test] + fn test_retain_usable_primary_key_filters_skips_non_tag_filters() { + let metadata = Arc::new(sst_region_metadata()); + let mut filters = + new_test_filters(&[col("field_0").eq(lit(1_u64)), col("ts").gt(lit(0_i64))]); + + retain_usable_primary_key_filters(&metadata, None, &mut filters); + + assert!(filters.is_empty()); + } + + #[test] + fn test_retain_usable_primary_key_filters_skips_reused_expected_tag_name() { + let metadata = Arc::new(sst_region_metadata()); + let expected_metadata = expected_metadata_with_reused_tag_name(&metadata); + let mut filters = new_test_filters(&[col("tag_0").eq(lit("b"))]); + + retain_usable_primary_key_filters( + &metadata, + Some(expected_metadata.as_ref()), + &mut filters, + ); + + assert!(filters.is_empty()); + } + + #[test] + fn test_is_usable_primary_key_filter_skips_legacy_primary_key_batches() { + let metadata = Arc::new(sst_region_metadata_with_encoding( + PrimaryKeyEncoding::Sparse, + )); + let read_format = ReadFormat::new_flat( + metadata.clone(), + metadata.column_metadatas.iter().map(|c| c.column_id), + None, + "test", + true, + ) + .unwrap(); + assert!(read_format.as_flat().is_some()); + + let filter = SimpleFilterEvaluator::try_new(&col("tag_0").eq(lit("b"))).unwrap(); + assert!(is_usable_primary_key_filter(&metadata, None, &filter)); + } + + #[test] + fn test_prefilter_primary_key_drops_single_dictionary_batch() { + let metadata = Arc::new(sst_region_metadata()); + let filters = Arc::new(new_test_filters(&[col("tag_0").eq(lit("b"))])); + let mut primary_key_filter = + build_primary_key_codec(metadata.as_ref()).primary_key_filter(&metadata, filters); + let pk_a = new_primary_key(&["a", "x"]); + let batch = new_raw_batch(&[pk_a.as_slice(), pk_a.as_slice()], &[10, 11]); + + let filtered = + prefilter_flat_batch_by_primary_key(batch, primary_key_filter.as_mut()).unwrap(); + + assert!(filtered.is_none()); + } + + #[test] + fn test_prefilter_primary_key_builds_mask_for_fragmented_matches() { + let metadata = Arc::new(sst_region_metadata()); + let filters = Arc::new(new_test_filters(&[col("tag_0") + .eq(lit("a")) + .or(col("tag_0").eq(lit("c")))])); + let mut primary_key_filter = + build_primary_key_codec(metadata.as_ref()).primary_key_filter(&metadata, filters); + let pk_a = new_primary_key(&["a", "x"]); + let pk_b = new_primary_key(&["b", "x"]); + let pk_c = new_primary_key(&["c", "x"]); + let pk_d = new_primary_key(&["d", "x"]); + let batch = new_raw_batch( + &[ + pk_a.as_slice(), + pk_a.as_slice(), + pk_b.as_slice(), + pk_b.as_slice(), + pk_c.as_slice(), + pk_c.as_slice(), + pk_d.as_slice(), + pk_d.as_slice(), + ], + &[10, 11, 12, 13, 14, 15, 16, 17], + ); + + let filtered = prefilter_flat_batch_by_primary_key(batch, primary_key_filter.as_mut()) + .unwrap() + .unwrap(); + + assert_eq!(filtered.num_rows(), 4); + assert_eq!(field_values(&filtered), vec![10, 11, 14, 15]); + } + + struct CountingPrimaryKeyFilter { + hits: Arc, + expected: Vec, + } + + impl PrimaryKeyFilter for CountingPrimaryKeyFilter { + fn matches(&mut self, pk: &[u8]) -> bool { + self.hits.fetch_add(1, Ordering::Relaxed); + pk == self.expected.as_slice() + } + } + + #[test] + fn test_cached_primary_key_filter_reuses_previous_result() { + let expected = new_primary_key(&["a", "x"]); + let hits = Arc::new(AtomicUsize::new(0)); + let mut filter = CachedPrimaryKeyFilter::new(Box::new(CountingPrimaryKeyFilter { + hits: Arc::clone(&hits), + expected: expected.clone(), + })); + + assert!(filter.matches(expected.as_slice())); + assert!(filter.matches(expected.as_slice())); + assert!(!filter.matches(new_primary_key(&["b", "x"]).as_slice())); + + assert_eq!(hits.load(Ordering::Relaxed), 2); + } + + #[test] + fn test_batch_single_primary_key() { + let pk_a = new_primary_key(&["a", "x"]); + let pk_b = new_primary_key(&["b", "x"]); + + let batch = new_raw_batch(&[pk_a.as_slice(), pk_a.as_slice()], &[10, 11]); + assert_eq!( + batch_single_primary_key(&batch).unwrap(), + Some(pk_a.as_slice()) + ); + + let batch = new_raw_batch(&[pk_a.as_slice(), pk_b.as_slice()], &[10, 11]); + assert_eq!(batch_single_primary_key(&batch).unwrap(), None); + } +} From 9bd983ea4063191679f82eda1523839746cb6aa4 Mon Sep 17 00:00:00 2001 From: Weny Xu Date: Tue, 24 Mar 2026 12:24:15 +0800 Subject: [PATCH 08/13] fix: prevent stale in-flight cache refill after invalidation in CacheContainer (#7825) * fix: prevent stale cache refill after invalidate Signed-off-by: WenyXu * chore: apply suggestions from CR Signed-off-by: WenyXu * feat: introduce `get_latest` Signed-off-by: WenyXu * chore: styling Signed-off-by: WenyXu * fix: enforce construction-time cache init strategy Make cache initialization behavior explicit via InitStrategy selected at construction and document dirty-vs-checked semantics. Keep latest-read call compatibility while partition manager uses strategy-driven get paths. Signed-off-by: WenyXu * test: rename get_by_ref freshness test Signed-off-by: WenyXu * feat: use `InitStrategy::VersionChecked` for table route cache Signed-off-by: WenyXu * chore: apply suggestions Signed-off-by: WenyXu * chore: apply suggestions from CR Signed-off-by: WenyXu * chore: apply suggestions from CR Signed-off-by: WenyXu --------- Signed-off-by: WenyXu --- src/catalog/src/kvbackend/table_cache.rs | 8 +- src/common/meta/Cargo.toml | 5 +- src/common/meta/src/cache/container.rs | 288 +++++++++++++++--- .../meta/src/cache/flow/table_flownode.rs | 24 +- src/common/meta/src/cache/table/schema.rs | 8 +- src/common/meta/src/cache/table/table_info.rs | 8 +- src/common/meta/src/cache/table/table_name.rs | 8 +- .../meta/src/cache/table/table_route.rs | 18 +- .../meta/src/cache/table/table_schema.rs | 2 +- src/common/meta/src/cache/table/view_info.rs | 8 +- src/common/meta/src/error.rs | 16 +- src/partition/src/cache.rs | 8 +- 12 files changed, 313 insertions(+), 88 deletions(-) diff --git a/src/catalog/src/kvbackend/table_cache.rs b/src/catalog/src/kvbackend/table_cache.rs index ea328c3e17..42b3fbc74b 100644 --- a/src/catalog/src/kvbackend/table_cache.rs +++ b/src/catalog/src/kvbackend/table_cache.rs @@ -65,11 +65,13 @@ fn init_factory( fn invalidator<'a>( cache: &'a Cache, - ident: &'a CacheIdent, + idents: &'a [&CacheIdent], ) -> BoxFuture<'a, MetaResult<()>> { Box::pin(async move { - if let CacheIdent::TableName(table_name) = ident { - cache.invalidate(table_name).await + for ident in idents { + if let CacheIdent::TableName(table_name) = ident { + cache.invalidate(table_name).await + } } Ok(()) }) diff --git a/src/common/meta/Cargo.toml b/src/common/meta/Cargo.toml index ec000c710d..f5ca9d2c09 100644 --- a/src/common/meta/Cargo.toml +++ b/src/common/meta/Cargo.toml @@ -8,7 +8,6 @@ license.workspace = true testing = [] pg_kvbackend = [ "dep:tokio-postgres", - "dep:backon", "dep:deadpool-postgres", "dep:deadpool", "dep:tokio-postgres-rustls", @@ -16,7 +15,7 @@ pg_kvbackend = [ "dep:rustls-native-certs", "dep:rustls", ] -mysql_kvbackend = ["dep:sqlx", "dep:backon"] +mysql_kvbackend = ["dep:sqlx"] enterprise = ["prost-types"] [lints] @@ -28,7 +27,7 @@ api.workspace = true async-recursion = "1.0" async-stream.workspace = true async-trait.workspace = true -backon = { workspace = true, optional = true } +backon.workspace = true base64.workspace = true bytes.workspace = true chrono.workspace = true diff --git a/src/common/meta/src/cache/container.rs b/src/common/meta/src/cache/container.rs index 0510476d15..e3a3e13a76 100644 --- a/src/common/meta/src/cache/container.rs +++ b/src/common/meta/src/cache/container.rs @@ -15,10 +15,14 @@ use std::borrow::Borrow; use std::hash::Hash; use std::sync::Arc; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::time::Duration; -use futures::future::{BoxFuture, join_all}; +use backon::{BackoffBuilder, ExponentialBuilder}; +use futures::future::BoxFuture; use moka::future::Cache; use snafu::{OptionExt, ResultExt}; +use tokio::time::sleep; use crate::cache_invalidator::{CacheInvalidator, Context}; use crate::error::{self, Error, Result}; @@ -29,12 +33,29 @@ use crate::metrics; pub type TokenFilter = Box bool + Send + Sync>; /// Invalidates cached values by [CacheToken]s. -pub type Invalidator = - Box Fn(&'a Cache, &'a CacheToken) -> BoxFuture<'a, Result<()>> + Send + Sync>; +pub type Invalidator = Box< + dyn for<'a> Fn(&'a Cache, &'a [&CacheToken]) -> BoxFuture<'a, Result<()>> + Send + Sync, +>; /// Initializes value (i.e., fetches from remote). pub type Initializer = Arc BoxFuture<'_, Result>> + Send + Sync>; +#[derive(Debug, Clone, Copy)] +/// Initialization strategy for cache-miss loading. +/// +/// This strategy is selected when building [CacheContainer] and remains immutable +/// for the lifetime of the container instance. +pub enum InitStrategy { + /// Fast path: load once without version conflict retry. + /// + /// Under concurrent invalidation, callers may observe stale/dirty value. + Unchecked, + /// Strict path: retry load when version changes during initialization. + /// + /// This avoids returning dirty value under invalidate/load races. + VersionChecked, +} + /// [CacheContainer] provides ability to: /// - Cache value loaded by [Initializer]. /// - Invalidate caches by [Invalidator]. @@ -44,6 +65,16 @@ pub struct CacheContainer { invalidator: Invalidator, initializer: Initializer, token_filter: fn(&CacheToken) -> bool, + version: Arc, + init_strategy: InitStrategy, +} + +fn latest_get_backoff() -> impl Iterator { + ExponentialBuilder::default() + .with_min_delay(Duration::from_millis(10)) + .with_max_delay(Duration::from_millis(100)) + .with_max_times(3) + .build() } impl CacheContainer @@ -52,13 +83,37 @@ where V: Send + Sync, CacheToken: Send + Sync, { - /// Constructs an [CacheContainer]. + /// Constructs an [CacheContainer] with [InitStrategy::Unchecked]. + /// + /// This keeps the historical behavior and can return stale/dirty value under + /// concurrent invalidation. pub fn new( name: String, cache: Cache, invalidator: Invalidator, initializer: Initializer, token_filter: fn(&CacheToken) -> bool, + ) -> Self { + Self::with_strategy( + name, + cache, + invalidator, + initializer, + token_filter, + InitStrategy::Unchecked, + ) + } + + /// Constructs an [CacheContainer] with explicit [InitStrategy]. + /// + /// The strategy is fixed at construction time and cannot be changed later. + pub fn with_strategy( + name: String, + cache: Cache, + invalidator: Invalidator, + initializer: Initializer, + token_filter: fn(&CacheToken) -> bool, + init_strategy: InitStrategy, ) -> Self { Self { name, @@ -66,6 +121,8 @@ where invalidator, initializer, token_filter, + version: Arc::new(AtomicUsize::new(0)), + init_strategy, } } @@ -75,6 +132,67 @@ where } } +impl CacheContainer { + fn inc_version(&self) { + self.version.fetch_add(1, Ordering::Relaxed); + } +} + +async fn init<'a, K, V>(init: Initializer, key: K, cache_name: &'a str) -> Result +where + K: Send + Sync + 'a, + V: Send + 'a, +{ + metrics::CACHE_CONTAINER_CACHE_MISS + .with_label_values(&[cache_name]) + .inc(); + let _timer = metrics::CACHE_CONTAINER_LOAD_CACHE + .with_label_values(&[cache_name]) + .start_timer(); + init(&key) + .await + .transpose() + .context(error::ValueNotExistSnafu)? +} + +async fn init_with_retry<'a, K, V>( + init: Initializer, + key: K, + mut backoff: impl Iterator + 'a, + version: Arc, + cache_name: &'a str, +) -> Result +where + K: Send + Sync + 'a, + V: Send + 'a, +{ + let mut attempts = 1usize; + loop { + let pre_version = version.load(Ordering::Relaxed); + metrics::CACHE_CONTAINER_CACHE_MISS + .with_label_values(&[cache_name]) + .inc(); + let _timer = metrics::CACHE_CONTAINER_LOAD_CACHE + .with_label_values(&[cache_name]) + .start_timer(); + let value = init(&key) + .await + .transpose() + .context(error::ValueNotExistSnafu)??; + + if pre_version == version.load(Ordering::Relaxed) { + return Ok(value); + } + + if let Some(duration) = backoff.next() { + sleep(duration).await; + attempts += 1; + } else { + return error::GetLatestCacheRetryExceededSnafu { attempts }.fail(); + } + } +} + #[async_trait::async_trait] impl CacheInvalidator for CacheContainer where @@ -82,14 +200,15 @@ where V: Send + Sync, { async fn invalidate(&self, _ctx: &Context, caches: &[CacheIdent]) -> Result<()> { - let tasks = caches + let idents = caches .iter() .filter(|token| (self.token_filter)(token)) - .map(|token| (self.invalidator)(&self.cache, token)); - join_all(tasks) - .await - .into_iter() - .collect::>>()?; + .collect::>(); + if !idents.is_empty() { + self.inc_version(); + (self.invalidator)(&self.cache, &idents).await?; + } + Ok(()) } } @@ -99,27 +218,39 @@ where K: Copy + Hash + Eq + Send + Sync + 'static, V: Clone + Send + Sync + 'static, { - /// Returns a _clone_ of the value corresponding to the key. + /// Returns a value from cache for copyable keys. + /// + /// With [InitStrategy::Unchecked], this method prioritizes latency and may + /// return stale/dirty value. With [InitStrategy::VersionChecked], this method + /// retries initialization on version change and avoids dirty returns. pub async fn get(&self, key: K) -> Result> { metrics::CACHE_CONTAINER_CACHE_GET .with_label_values(&[&self.name]) .inc(); - let moved_init = self.initializer.clone(); - let moved_key = key; - let init = async move { - metrics::CACHE_CONTAINER_CACHE_MISS - .with_label_values(&[&self.name]) - .inc(); - let _timer = metrics::CACHE_CONTAINER_LOAD_CACHE - .with_label_values(&[&self.name]) - .start_timer(); - moved_init(&moved_key) - .await - .transpose() - .context(error::ValueNotExistSnafu)? + + let result = match self.init_strategy { + InitStrategy::Unchecked => { + self.cache + .try_get_with(key, init(self.initializer.clone(), key, &self.name)) + .await + } + InitStrategy::VersionChecked => { + self.cache + .try_get_with( + key, + init_with_retry( + self.initializer.clone(), + key, + latest_get_backoff(), + self.version.clone(), + &self.name, + ), + ) + .await + } }; - match self.cache.try_get_with(key, init).await { + match result { Ok(value) => Ok(Some(value)), Err(err) => match err.as_ref() { Error::ValueNotExist { .. } => Ok(None), @@ -136,14 +267,15 @@ where { /// Invalidates cache by [CacheToken]. pub async fn invalidate(&self, caches: &[CacheToken]) -> Result<()> { - let tasks = caches + let idents = caches .iter() .filter(|token| (self.token_filter)(token)) - .map(|token| (self.invalidator)(&self.cache, token)); - join_all(tasks) - .await - .into_iter() - .collect::>>()?; + .collect::>(); + if !idents.is_empty() { + self.inc_version(); + (self.invalidator)(&self.cache, &idents).await?; + } + Ok(()) } @@ -156,7 +288,11 @@ where self.cache.contains_key(key) } - /// Returns a _clone_ of the value corresponding to the key. + /// Returns a value from cache by key reference. + /// + /// With [InitStrategy::Unchecked], this method prioritizes latency and may + /// return stale/dirty value. With [InitStrategy::VersionChecked], this method + /// retries initialization on version change and avoids dirty returns. pub async fn get_by_ref(&self, key: &Q) -> Result> where K: Borrow, @@ -165,24 +301,32 @@ where metrics::CACHE_CONTAINER_CACHE_GET .with_label_values(&[&self.name]) .inc(); - let moved_init = self.initializer.clone(); - let moved_key = key.to_owned(); - - let init = async move { - metrics::CACHE_CONTAINER_CACHE_MISS - .with_label_values(&[&self.name]) - .inc(); - let _timer = metrics::CACHE_CONTAINER_LOAD_CACHE - .with_label_values(&[&self.name]) - .start_timer(); - - moved_init(&moved_key) - .await - .transpose() - .context(error::ValueNotExistSnafu)? + let result = match self.init_strategy { + InitStrategy::Unchecked => { + self.cache + .try_get_with_by_ref( + key, + init(self.initializer.clone(), key.to_owned(), &self.name), + ) + .await + } + InitStrategy::VersionChecked => { + self.cache + .try_get_with_by_ref( + key, + init_with_retry( + self.initializer.clone(), + key.to_owned(), + latest_get_backoff(), + self.version.clone(), + &self.name, + ), + ) + .await + } }; - match self.cache.try_get_with_by_ref(key, init).await { + match result { Ok(value) => Ok(Some(value)), Err(err) => match err.as_ref() { Error::ValueNotExist { .. } => Ok(None), @@ -296,9 +440,11 @@ mod tests { moved_counter.fetch_add(1, Ordering::Relaxed); Box::pin(async { Ok(Some("hi".to_string())) }) }); - let invalidator: Invalidator = Box::new(|cache, key| { + let invalidator: Invalidator = Box::new(|cache, keys| { Box::pin(async move { - cache.invalidate(key).await; + for key in keys { + cache.invalidate(*key).await; + } Ok(()) }) }); @@ -323,4 +469,46 @@ mod tests { assert_eq!(value, "hi"); assert_eq!(counter.load(Ordering::Relaxed), 2); } + + #[tokio::test(flavor = "multi_thread")] + async fn test_get_by_ref_returns_fresh_value_after_invalidate() { + let cache: Cache = CacheBuilder::new(128).build(); + let counter = Arc::new(AtomicI32::new(0)); + let moved_counter = counter.clone(); + let init: Initializer = Arc::new(move |_| { + let counter = moved_counter.clone(); + Box::pin(async move { + let n = counter.fetch_add(1, Ordering::Relaxed) + 1; + sleep(Duration::from_millis(100)).await; + Ok(Some(format!("v{n}"))) + }) + }); + let invalidator: Invalidator = Box::new(|cache, keys| { + Box::pin(async move { + for key in keys { + cache.invalidate(*key).await; + } + Ok(()) + }) + }); + + let adv_cache = Arc::new(CacheContainer::with_strategy( + "test".to_string(), + cache, + invalidator, + init, + always_true_filter, + InitStrategy::VersionChecked, + )); + + let moved_cache = adv_cache.clone(); + let get_task = tokio::spawn(async move { moved_cache.get_by_ref("foo").await }); + + sleep(Duration::from_millis(50)).await; + adv_cache.invalidate(&["foo".to_string()]).await.unwrap(); + + let value = get_task.await.unwrap().unwrap().unwrap(); + assert_eq!(value, "v2"); + assert_eq!(counter.load(Ordering::Relaxed), 2); + } } diff --git a/src/common/meta/src/cache/flow/table_flownode.rs b/src/common/meta/src/cache/flow/table_flownode.rs index a7777f3361..ebe3664202 100644 --- a/src/common/meta/src/cache/flow/table_flownode.rs +++ b/src/common/meta/src/cache/flow/table_flownode.rs @@ -170,20 +170,22 @@ async fn handle_drop_flow( fn invalidator<'a>( cache: &'a Cache, - ident: &'a CacheIdent, + idents: &'a [&CacheIdent], ) -> BoxFuture<'a, Result<()>> { Box::pin(async move { - match ident { - CacheIdent::CreateFlow(create_flow) => handle_create_flow(cache, create_flow).await, - CacheIdent::DropFlow(drop_flow) => handle_drop_flow(cache, drop_flow).await, - CacheIdent::FlowNodeAddressChange(node_id) => { - info!( - "Invalidate flow node cache for node_id in table_flownode: {}", - node_id - ); - cache.invalidate_all(); + for ident in idents { + match ident { + CacheIdent::CreateFlow(create_flow) => handle_create_flow(cache, create_flow).await, + CacheIdent::DropFlow(drop_flow) => handle_drop_flow(cache, drop_flow).await, + CacheIdent::FlowNodeAddressChange(node_id) => { + info!( + "Invalidate flow node cache for node_id in table_flownode: {}", + node_id + ); + cache.invalidate_all(); + } + _ => {} } - _ => {} } Ok(()) }) diff --git a/src/common/meta/src/cache/table/schema.rs b/src/common/meta/src/cache/table/schema.rs index bcf81d4fe6..bd9e8e6dc1 100644 --- a/src/common/meta/src/cache/table/schema.rs +++ b/src/common/meta/src/cache/table/schema.rs @@ -58,11 +58,13 @@ fn init_factory(schema_manager: SchemaManager) -> Initializer( cache: &'a Cache>, - ident: &'a CacheIdent, + idents: &'a [&CacheIdent], ) -> BoxFuture<'a, crate::error::Result<()>> { Box::pin(async move { - if let CacheIdent::SchemaName(schema_name) = ident { - cache.invalidate(schema_name).await + for ident in idents { + if let CacheIdent::SchemaName(schema_name) = ident { + cache.invalidate(schema_name).await + } } Ok(()) }) diff --git a/src/common/meta/src/cache/table/table_info.rs b/src/common/meta/src/cache/table/table_info.rs index b853d908e8..97af5bcdb7 100644 --- a/src/common/meta/src/cache/table/table_info.rs +++ b/src/common/meta/src/cache/table/table_info.rs @@ -61,11 +61,13 @@ fn init_factory(table_info_manager: TableInfoManagerRef) -> Initializer( cache: &'a Cache>, - ident: &'a CacheIdent, + idents: &'a [&CacheIdent], ) -> BoxFuture<'a, Result<()>> { Box::pin(async move { - if let CacheIdent::TableId(table_id) = ident { - cache.invalidate(table_id).await + for ident in idents { + if let CacheIdent::TableId(table_id) = ident { + cache.invalidate(table_id).await + } } Ok(()) }) diff --git a/src/common/meta/src/cache/table/table_name.rs b/src/common/meta/src/cache/table/table_name.rs index 540da5e5f4..927a5b3480 100644 --- a/src/common/meta/src/cache/table/table_name.rs +++ b/src/common/meta/src/cache/table/table_name.rs @@ -71,11 +71,13 @@ fn init_factory(table_name_manager: TableNameManagerRef) -> Initializer( cache: &'a Cache, - ident: &'a CacheIdent, + idents: &'a [&CacheIdent], ) -> BoxFuture<'a, Result<()>> { Box::pin(async move { - if let CacheIdent::TableName(table_name) = ident { - cache.invalidate(table_name).await + for ident in idents { + if let CacheIdent::TableName(table_name) = ident { + cache.invalidate(table_name).await + } } Ok(()) }) diff --git a/src/common/meta/src/cache/table/table_route.rs b/src/common/meta/src/cache/table/table_route.rs index 47abdaa728..be820b0c52 100644 --- a/src/common/meta/src/cache/table/table_route.rs +++ b/src/common/meta/src/cache/table/table_route.rs @@ -19,6 +19,7 @@ use moka::future::Cache; use snafu::OptionExt; use store_api::storage::TableId; +use crate::cache::container::InitStrategy; use crate::cache::{CacheContainer, Initializer}; use crate::error; use crate::error::Result; @@ -65,7 +66,14 @@ pub fn new_table_route_cache( let table_info_manager = Arc::new(TableRouteManager::new(kv_backend)); let init = init_factory(table_info_manager); - CacheContainer::new(name, cache, Box::new(invalidator), init, filter) + CacheContainer::with_strategy( + name, + cache, + Box::new(invalidator), + init, + filter, + InitStrategy::VersionChecked, + ) } fn init_factory( @@ -92,11 +100,13 @@ fn init_factory( fn invalidator<'a>( cache: &'a Cache>, - ident: &'a CacheIdent, + idents: &'a [&CacheIdent], ) -> BoxFuture<'a, Result<()>> { Box::pin(async move { - if let CacheIdent::TableId(table_id) = ident { - cache.invalidate(table_id).await + for ident in idents { + if let CacheIdent::TableId(table_id) = ident { + cache.invalidate(table_id).await + } } Ok(()) }) diff --git a/src/common/meta/src/cache/table/table_schema.rs b/src/common/meta/src/cache/table/table_schema.rs index 99ece65683..33b1773f45 100644 --- a/src/common/meta/src/cache/table/table_schema.rs +++ b/src/common/meta/src/cache/table/table_schema.rs @@ -65,7 +65,7 @@ fn init_factory(table_info_manager: TableInfoManager) -> Initializer( _cache: &'a Cache>, - _ident: &'a CacheIdent, + _idents: &'a [&CacheIdent], ) -> BoxFuture<'a, error::Result<()>> { Box::pin(std::future::ready(Ok(()))) } diff --git a/src/common/meta/src/cache/table/view_info.rs b/src/common/meta/src/cache/table/view_info.rs index 6a85493d42..d0e1058a7e 100644 --- a/src/common/meta/src/cache/table/view_info.rs +++ b/src/common/meta/src/cache/table/view_info.rs @@ -60,11 +60,13 @@ fn init_factory(view_info_manager: ViewInfoManagerRef) -> Initializer( cache: &'a Cache>, - ident: &'a CacheIdent, + idents: &'a [&CacheIdent], ) -> BoxFuture<'a, Result<()>> { Box::pin(async move { - if let CacheIdent::TableId(view_id) = ident { - cache.invalidate(view_id).await + for ident in idents { + if let CacheIdent::TableId(view_id) = ident { + cache.invalidate(view_id).await + } } Ok(()) }) diff --git a/src/common/meta/src/error.rs b/src/common/meta/src/error.rs index c6613af828..b9fcbd6188 100644 --- a/src/common/meta/src/error.rs +++ b/src/common/meta/src/error.rs @@ -714,6 +714,16 @@ pub enum Error { #[snafu(display("Failed to get cache"))] GetCache { source: Arc }, + #[snafu(display( + "Failed to get latest cache value after {} attempts due to concurrent invalidation", + attempts + ))] + GetLatestCacheRetryExceeded { + attempts: usize, + #[snafu(implicit)] + location: Location, + }, + #[cfg(feature = "pg_kvbackend")] #[snafu(display("Failed to execute via Postgres, sql: {}", sql))] PostgresExecution { @@ -1063,6 +1073,7 @@ impl ErrorExt for Error { | ConnectEtcd { .. } | MoveValues { .. } | GetCache { .. } + | GetLatestCacheRetryExceeded { .. } | SerializeToJson { .. } | DeserializeFromJson { .. } => StatusCode::Internal, @@ -1243,7 +1254,10 @@ impl Error { /// Determine whether it is a retry later type through [StatusCode] pub fn is_retry_later(&self) -> bool { - matches!(self, Error::RetryLater { .. }) + matches!( + self, + Error::RetryLater { .. } | Error::GetLatestCacheRetryExceeded { .. } + ) } /// Determine whether it needs to clean poisons. diff --git a/src/partition/src/cache.rs b/src/partition/src/cache.rs index a886e1e08d..4066b69aa3 100644 --- a/src/partition/src/cache.rs +++ b/src/partition/src/cache.rs @@ -121,10 +121,12 @@ pub fn new_partition_info_cache( CacheContainer::new( name, cache, - Box::new(|cache, ident| { + Box::new(|cache, idents| { Box::pin(async move { - if let CacheIdent::TableId(table_id) = ident { - cache.invalidate(table_id).await + for ident in idents { + if let CacheIdent::TableId(table_id) = ident { + cache.invalidate(table_id).await + } } Ok(()) }) From 30e895abbef7ec63be7afd8dfdecf448ce88453e Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Tue, 24 Mar 2026 14:24:52 +0800 Subject: [PATCH 09/13] fix: prom cast to f64 (#7840) * fix: cast to f64 Signed-off-by: discord9 * test: div case Signed-off-by: discord9 * test: int test Signed-off-by: discord9 * chore: sqlness update Signed-off-by: discord9 * chore: test Signed-off-by: discord9 * chore: update test Signed-off-by: discord9 --------- Signed-off-by: discord9 --- src/query/src/promql/planner.rs | 55 ++-- tests-integration/src/tests/promql_test.rs | 238 +++++++++++++++++- .../explain/step_aggr_advance.result | 90 +++---- .../promql/anon_promql_ratio_repro.result | 106 ++++++++ .../common/promql/anon_promql_ratio_repro.sql | 63 +++++ .../standalone/common/tql/tql-cte.result | 4 +- 6 files changed, 494 insertions(+), 62 deletions(-) create mode 100644 tests/cases/standalone/common/promql/anon_promql_ratio_repro.result create mode 100644 tests/cases/standalone/common/promql/anon_promql_ratio_repro.sql diff --git a/src/query/src/promql/planner.rs b/src/query/src/promql/planner.rs index 427644e26a..b6f4f2d28f 100644 --- a/src/query/src/promql/planner.rs +++ b/src/query/src/promql/planner.rs @@ -3323,28 +3323,55 @@ impl PromPlanner { fn prom_token_to_binary_expr_builder( token: TokenType, ) -> Result Result>> { + let cast_float = |expr| { + if matches!( + &expr, + DfExpr::Cast(Cast { + data_type: ArrowDataType::Float64, + .. + }) + ) || matches!(&expr, DfExpr::Literal(ScalarValue::Float64(_), _)) + { + expr + } else { + DfExpr::Cast(Cast { + expr: Box::new(expr), + data_type: ArrowDataType::Float64, + }) + } + }; match token.id() { - token::T_ADD => Ok(Box::new(|lhs, rhs| Ok(lhs + rhs))), - token::T_SUB => Ok(Box::new(|lhs, rhs| Ok(lhs - rhs))), - token::T_MUL => Ok(Box::new(|lhs, rhs| Ok(lhs * rhs))), - token::T_DIV => Ok(Box::new(|lhs, rhs| Ok(lhs / rhs))), - token::T_MOD => Ok(Box::new(|lhs: DfExpr, rhs| Ok(lhs % rhs))), + token::T_ADD => Ok(Box::new(move |lhs, rhs| { + Ok(cast_float(lhs) + cast_float(rhs)) + })), + token::T_SUB => Ok(Box::new(move |lhs, rhs| { + Ok(cast_float(lhs) - cast_float(rhs)) + })), + token::T_MUL => Ok(Box::new(move |lhs, rhs| { + Ok(cast_float(lhs) * cast_float(rhs)) + })), + token::T_DIV => Ok(Box::new(move |lhs, rhs| { + Ok(cast_float(lhs) / cast_float(rhs)) + })), + token::T_MOD => Ok(Box::new(move |lhs: DfExpr, rhs| { + Ok(cast_float(lhs) % cast_float(rhs)) + })), token::T_EQLC => Ok(Box::new(|lhs, rhs| Ok(lhs.eq(rhs)))), token::T_NEQ => Ok(Box::new(|lhs, rhs| Ok(lhs.not_eq(rhs)))), token::T_GTR => Ok(Box::new(|lhs, rhs| Ok(lhs.gt(rhs)))), token::T_LSS => Ok(Box::new(|lhs, rhs| Ok(lhs.lt(rhs)))), token::T_GTE => Ok(Box::new(|lhs, rhs| Ok(lhs.gt_eq(rhs)))), token::T_LTE => Ok(Box::new(|lhs, rhs| Ok(lhs.lt_eq(rhs)))), - token::T_POW => Ok(Box::new(|lhs, rhs| { + token::T_POW => Ok(Box::new(move |lhs, rhs| { Ok(DfExpr::ScalarFunction(ScalarFunction { func: datafusion_functions::math::power(), - args: vec![lhs, rhs], + args: vec![cast_float(lhs), cast_float(rhs)], })) })), - token::T_ATAN2 => Ok(Box::new(|lhs, rhs| { + token::T_ATAN2 => Ok(Box::new(move |lhs, rhs| { Ok(DfExpr::ScalarFunction(ScalarFunction { func: datafusion_functions::math::atan2(), - args: vec![lhs, rhs], + args: vec![cast_float(lhs), cast_float(rhs)], })) })), _ => UnexpectedTokenSnafu { token }.fail(), @@ -5169,7 +5196,7 @@ mod test { .unwrap(); let expected = String::from( - "Projection: rhs.tag_0, rhs.timestamp, lhs.field_0 + rhs.field_0 AS lhs.field_0 + rhs.field_0 [tag_0:Utf8, timestamp:Timestamp(ms), lhs.field_0 + rhs.field_0:Float64;N]\ + "Projection: rhs.tag_0, rhs.timestamp, CAST(lhs.field_0 AS Float64) + CAST(rhs.field_0 AS Float64) AS lhs.field_0 + rhs.field_0 [tag_0:Utf8, timestamp:Timestamp(ms), lhs.field_0 + rhs.field_0:Float64;N]\ \n Inner Join: lhs.tag_0 = rhs.tag_0, lhs.timestamp = rhs.timestamp [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N, tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n SubqueryAlias: lhs [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ @@ -5224,7 +5251,7 @@ mod test { async fn binary_op_literal_column() { let query = r#"1 + some_metric{tag_0="bar"}"#; let expected = String::from( - "Projection: some_metric.tag_0, some_metric.timestamp, Float64(1) + some_metric.field_0 AS Float64(1) + field_0 [tag_0:Utf8, timestamp:Timestamp(ms), Float64(1) + field_0:Float64;N]\ + "Projection: some_metric.tag_0, some_metric.timestamp, Float64(1) + CAST(some_metric.field_0 AS Float64) AS Float64(1) + field_0 [tag_0:Utf8, timestamp:Timestamp(ms), Float64(1) + field_0:Float64;N]\ \n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n PromSeriesDivide: tags=[\"tag_0\"] [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n Sort: some_metric.tag_0 ASC NULLS FIRST, some_metric.timestamp ASC NULLS FIRST [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ @@ -5262,7 +5289,7 @@ mod test { async fn bool_with_additional_arithmetic() { let query = "some_metric + (1 == bool 2)"; let expected = String::from( - "Projection: some_metric.tag_0, some_metric.timestamp, some_metric.field_0 + CAST(Float64(1) = Float64(2) AS Float64) AS field_0 + Float64(1) = Float64(2) [tag_0:Utf8, timestamp:Timestamp(ms), field_0 + Float64(1) = Float64(2):Float64;N]\ + "Projection: some_metric.tag_0, some_metric.timestamp, CAST(some_metric.field_0 AS Float64) + CAST(Float64(1) = Float64(2) AS Float64) AS field_0 + Float64(1) = Float64(2) [tag_0:Utf8, timestamp:Timestamp(ms), field_0 + Float64(1) = Float64(2):Float64;N]\ \n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n PromSeriesDivide: tags=[\"tag_0\"] [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n Sort: some_metric.tag_0 ASC NULLS FIRST, some_metric.timestamp ASC NULLS FIRST [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ @@ -5372,7 +5399,7 @@ mod test { PromPlanner::stmt_to_plan(table_provider, &eval_stmt, &build_query_engine_state()) .await .unwrap(); - let expected = "Projection: http_server_requests_seconds_count.uri, http_server_requests_seconds_count.kubernetes_namespace, http_server_requests_seconds_count.kubernetes_pod_name, http_server_requests_seconds_count.greptime_timestamp, http_server_requests_seconds_sum.greptime_value / http_server_requests_seconds_count.greptime_value AS http_server_requests_seconds_sum.greptime_value / http_server_requests_seconds_count.greptime_value\ + let expected = "Projection: http_server_requests_seconds_count.uri, http_server_requests_seconds_count.kubernetes_namespace, http_server_requests_seconds_count.kubernetes_pod_name, http_server_requests_seconds_count.greptime_timestamp, CAST(http_server_requests_seconds_sum.greptime_value AS Float64) / CAST(http_server_requests_seconds_count.greptime_value AS Float64) AS http_server_requests_seconds_sum.greptime_value / http_server_requests_seconds_count.greptime_value\ \n Inner Join: http_server_requests_seconds_sum.greptime_timestamp = http_server_requests_seconds_count.greptime_timestamp, http_server_requests_seconds_sum.uri = http_server_requests_seconds_count.uri\ \n SubqueryAlias: http_server_requests_seconds_sum\ \n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[greptime_timestamp]\ @@ -5763,7 +5790,7 @@ mod test { let query = "some_alt_metric{__schema__=\"greptime_private\"} / some_metric"; let expected = String::from( - "Projection: some_metric.tag_0, some_metric.timestamp, greptime_private.some_alt_metric.field_0 / some_metric.field_0 AS greptime_private.some_alt_metric.field_0 / some_metric.field_0 [tag_0:Utf8, timestamp:Timestamp(ms), greptime_private.some_alt_metric.field_0 / some_metric.field_0:Float64;N]\ + "Projection: some_metric.tag_0, some_metric.timestamp, CAST(greptime_private.some_alt_metric.field_0 AS Float64) / CAST(some_metric.field_0 AS Float64) AS greptime_private.some_alt_metric.field_0 / some_metric.field_0 [tag_0:Utf8, timestamp:Timestamp(ms), greptime_private.some_alt_metric.field_0 / some_metric.field_0:Float64;N]\ \n Inner Join: greptime_private.some_alt_metric.tag_0 = some_metric.tag_0, greptime_private.some_alt_metric.timestamp = some_metric.timestamp [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N, tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n SubqueryAlias: greptime_private.some_alt_metric [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ \n PromInstantManipulate: range=[0..100000000], lookback=[1000], interval=[5000], time index=[timestamp] [tag_0:Utf8, timestamp:Timestamp(ms), field_0:Float64;N]\ diff --git a/tests-integration/src/tests/promql_test.rs b/tests-integration/src/tests/promql_test.rs index 7fbce91ea6..ede4663118 100644 --- a/tests-integration/src/tests/promql_test.rs +++ b/tests-integration/src/tests/promql_test.rs @@ -15,7 +15,9 @@ use std::sync::Arc; use std::time::{Duration, SystemTime, UNIX_EPOCH}; -use common_query::Output; +use common_query::{Output, OutputData}; +use common_recordbatch::util::collect_batches; +use datatypes::arrow::array::{Float64Array, Int64Array}; use frontend::instance::Instance; use query::parser::{PromQuery, QueryLanguageParser, QueryStatement}; use rstest::rstest; @@ -151,6 +153,103 @@ async fn create_insert_tql_assert( check_unordered_output_stream(query_output, expected).await; } +async fn execute_all(instance: &Arc, sql: &str, query_ctx: Arc) { + instance + .do_query(sql, query_ctx) + .await + .into_iter() + .for_each(|v| { + let _ = v.unwrap(); + }); +} + +#[allow(clippy::too_many_arguments)] +async fn promql_query_as_batches( + ins: Arc, + promql: &str, + alias: Option, + query_ctx: Arc, + start: SystemTime, + end: SystemTime, + interval: Duration, + lookback: Duration, +) -> common_recordbatch::RecordBatches { + let output = promql_query( + ins, promql, alias, query_ctx, start, end, interval, lookback, + ) + .await + .unwrap(); + match output.data { + OutputData::Stream(stream) => collect_batches(stream).await.unwrap(), + OutputData::RecordBatches(recordbatches) => recordbatches, + _ => unreachable!(), + } +} + +const ANON_PROMQL_RATIO_REPRO_DB: &str = "repro_db"; + +const ANON_PROMQL_RATIO_REPRO_CREATE: &str = r#" +CREATE TABLE phy ( + t TIMESTAMP TIME INDEX, + v DOUBLE +) ENGINE=metric WITH ("physical_metric_table" = ""); + +CREATE TABLE metric_a ( + l1 STRING NULL, + l2 STRING NULL, + l3 STRING NULL, + l4 STRING NULL, + l5 STRING NULL, + t TIMESTAMP NOT NULL, + v DOUBLE NULL, + TIME INDEX (t), + PRIMARY KEY (l1, l2, l3, l4, l5) +) ENGINE=metric WITH (on_physical_table = 'phy'); + +CREATE TABLE metric_b ( + l6 STRING NULL, + l1 STRING NULL, + l2 STRING NULL, + l3 STRING NULL, + l4 STRING NULL, + t TIMESTAMP NOT NULL, + v DOUBLE NULL, + TIME INDEX (t), + PRIMARY KEY (l6, l1, l2, l3, l4) +) ENGINE=metric WITH (on_physical_table = 'phy'); +"#; + +const ANON_PROMQL_RATIO_REPRO_INSERT: &str = r#" +INSERT INTO metric_a (l1, l2, l3, l4, l5, t, v) VALUES + ('v1', 'v2', 'v3', 'v4a', 'v5a', 1, 0), + ('v1', 'v2', 'v3', 'v4a', 'v5a', 180000, 120), + ('v1', 'v2', 'v3', 'v4a', 'v5a', 360000, 240), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 1, 0), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 180000, 30), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 360000, 60), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 1, 0), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 180000, 60), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 360000, 120); + +INSERT INTO metric_b (l6, l1, l2, l3, l4, t, v) VALUES + ('v6', 'v1', 'v2', 'v3', 'v4a', 1, 1), + ('v6', 'v1', 'v2', 'v3', 'v4a', 180000, 1), + ('v6', 'v1', 'v2', 'v3', 'v4a', 360000, 1), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 1, 2), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 180000, 2), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 360000, 2); +"#; + +const ANON_PROMQL_RATIO_REPRO_NUMERATOR: &str = r#"count(((rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)",__schema__="repro_db"}[3m]) / on(l3,l4) group_left metric_b{l6="v6",l1="v1",l2="v2",l3=~"v3(|-a|-b)",__schema__="repro_db"}) > 0.50))"#; + +const ANON_PROMQL_RATIO_REPRO_DENOMINATOR: &str = + r#"count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)",__schema__="repro_db"}[3m]))"#; + +const ANON_PROMQL_RATIO_REPRO_WHOLE: &str = r#"(count(((rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)",__schema__="repro_db"}[3m]) / on(l3,l4) group_left metric_b{l6="v6",l1="v1",l2="v2",l3=~"v3(|-a|-b)",__schema__="repro_db"}) > 0.50)) / count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)",__schema__="repro_db"}[3m]))) * 100"#; + +const ANON_PROMQL_RATIO_REPRO_SCALAR_DIV: &str = + r#"count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)",__schema__="repro_db"}[3m])) / 2"#; + #[apply(both_instances_cases)] async fn sql_insert_tql_query_ceil(instance: Arc) { let instance = instance.frontend(); @@ -709,3 +808,140 @@ async fn cross_schema_query(instance: Arc) { check_unordered_output_stream(query_output, expected).await; } + +#[apply(both_instances_cases)] +async fn anon_promql_ratio_repro(instance: Arc) { + let ins = instance.frontend(); + + execute_all( + &ins, + &format!("CREATE DATABASE {ANON_PROMQL_RATIO_REPRO_DB}"), + QueryContext::arc(), + ) + .await; + + let repro_ctx: Arc = + QueryContext::with_db_name(Some(ANON_PROMQL_RATIO_REPRO_DB)).into(); + execute_all(&ins, ANON_PROMQL_RATIO_REPRO_CREATE, repro_ctx.clone()).await; + execute_all(&ins, ANON_PROMQL_RATIO_REPRO_INSERT, repro_ctx).await; + + let start = UNIX_EPOCH.checked_add(Duration::from_secs(180)).unwrap(); + let end = UNIX_EPOCH.checked_add(Duration::from_secs(360)).unwrap(); + let interval = Duration::from_secs(180); + let lookback = Duration::from_secs(1); + + let numerator = promql_query_as_batches( + ins.clone(), + ANON_PROMQL_RATIO_REPRO_NUMERATOR, + Some("num".to_string()), + QueryContext::arc(), + start, + end, + interval, + lookback, + ) + .await; + let denominator = promql_query_as_batches( + ins.clone(), + ANON_PROMQL_RATIO_REPRO_DENOMINATOR, + Some("den".to_string()), + QueryContext::arc(), + start, + end, + interval, + lookback, + ) + .await; + let whole = promql_query_as_batches( + ins.clone(), + ANON_PROMQL_RATIO_REPRO_WHOLE, + Some("pct".to_string()), + QueryContext::arc(), + start, + end, + interval, + lookback, + ) + .await; + let scalar_div = promql_query_as_batches( + ins, + ANON_PROMQL_RATIO_REPRO_SCALAR_DIV, + Some("half_den".to_string()), + QueryContext::arc(), + start, + end, + interval, + lookback, + ) + .await; + + let numerator = numerator.iter().collect::>(); + let denominator = denominator.iter().collect::>(); + let whole = whole.iter().collect::>(); + let scalar_div = scalar_div.iter().collect::>(); + + let numerator_values = numerator[0] + .column_by_name("num") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + let denominator_values = denominator[0] + .column_by_name("den") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + let percentage_values = whole[0] + .column_by_name("pct") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + let scalar_div_values = scalar_div[0] + .column_by_name("half_den") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(numerator_values.len(), 1, "{}", numerator[0].pretty_print()); + assert_eq!( + denominator_values.len(), + 1, + "{}", + denominator[0].pretty_print() + ); + assert_eq!(percentage_values.len(), 1, "{}", whole[0].pretty_print()); + assert_eq!( + scalar_div_values.len(), + 1, + "{}", + scalar_div[0].pretty_print() + ); + + assert_eq!( + numerator_values.value(0), + 1, + "{}", + numerator[0].pretty_print() + ); + assert_eq!( + denominator_values.value(0), + 3, + "{}", + denominator[0].pretty_print() + ); + assert!( + (scalar_div_values.value(0) - 1.5).abs() < 1e-9, + "{}", + scalar_div[0].pretty_print() + ); + + let expected = 100.0 / 3.0; + assert!( + (percentage_values.value(0) - expected).abs() < 1e-9, + "{}", + whole[0].pretty_print() + ); +} diff --git a/tests/cases/distributed/explain/step_aggr_advance.result b/tests/cases/distributed/explain/step_aggr_advance.result index 4bd83b7afa..5938fa202d 100644 --- a/tests/cases/distributed/explain/step_aggr_advance.result +++ b/tests/cases/distributed/explain/step_aggr_advance.result @@ -442,54 +442,54 @@ Affected Rows: 0 -- SQLNESS REPLACE (Hash.*) REDACTED tql explain (1752591864, 1752592164, '30s') sum by (a, b, c) (rate(aggr_optimize_not [2m])) / sum by (a, b, c) (rate(aggr_optimize_not_count [2m])); -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| plan_type | plan | -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | Projection: aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.greptime_timestamp, aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) AS aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) | -| | Inner Join: aggr_optimize_not.a = aggr_optimize_not_count.a, aggr_optimize_not.b = aggr_optimize_not_count.b, aggr_optimize_not.c = aggr_optimize_not_count.c, aggr_optimize_not.greptime_timestamp = aggr_optimize_not_count.greptime_timestamp | -| | MergeSort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | -| | MergeScan [is_placeholder=false, remote_input=[ | -| | SubqueryAlias: aggr_optimize_not | -| | Sort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | -| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.greptime_timestamp]], aggr=[[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))]] | -| | Filter: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)) IS NOT NULL | -| | Projection: aggr_optimize_not.greptime_timestamp, prom_rate(greptime_timestamp_range, greptime_value, aggr_optimize_not.greptime_timestamp, Int64(120000)) AS prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | -| | PromRangeManipulate: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | -| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | -| | PromSeriesDivide: tags=["a", "b", "c", "d"] | -| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | -| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(1752591744001, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(1752592164000, None) | -| | TableScan: aggr_optimize_not | -| | ]] | -| | SubqueryAlias: aggr_optimize_not_count | -| | Sort: aggr_optimize_not_count.a ASC NULLS LAST, aggr_optimize_not_count.b ASC NULLS LAST, aggr_optimize_not_count.c ASC NULLS LAST, aggr_optimize_not_count.greptime_timestamp ASC NULLS LAST | -| | Aggregate: groupBy=[[aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.greptime_timestamp]], aggr=[[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))]] | -| | Filter: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)) IS NOT NULL | -| | Projection: aggr_optimize_not_count.greptime_timestamp, prom_rate(greptime_timestamp_range, greptime_value, aggr_optimize_not_count.greptime_timestamp, Int64(120000)) AS prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c | -| | PromRangeManipulate: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | -| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | -| | PromSeriesDivide: tags=["a", "b", "c", "d"] | -| | Sort: aggr_optimize_not_count.a ASC NULLS FIRST, aggr_optimize_not_count.b ASC NULLS FIRST, aggr_optimize_not_count.c ASC NULLS FIRST, aggr_optimize_not_count.d ASC NULLS FIRST, aggr_optimize_not_count.greptime_timestamp ASC NULLS FIRST | -| | MergeScan [is_placeholder=false, remote_input=[ | -| | Filter: aggr_optimize_not_count.greptime_timestamp >= TimestampMillisecond(1752591744001, None) AND aggr_optimize_not_count.greptime_timestamp <= TimestampMillisecond(1752592164000, None) | -| | TableScan: aggr_optimize_not_count | -| | ]] | -| physical_plan | ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp, sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@5 / sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@4 as aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))] | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.greptime_timestamp, CAST(aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) AS Float64) / CAST(aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) AS Float64) AS aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) | +| | Inner Join: aggr_optimize_not.a = aggr_optimize_not_count.a, aggr_optimize_not.b = aggr_optimize_not_count.b, aggr_optimize_not.c = aggr_optimize_not_count.c, aggr_optimize_not.greptime_timestamp = aggr_optimize_not_count.greptime_timestamp | +| | MergeSort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | SubqueryAlias: aggr_optimize_not | +| | Sort: aggr_optimize_not.a ASC NULLS LAST, aggr_optimize_not.b ASC NULLS LAST, aggr_optimize_not.c ASC NULLS LAST, aggr_optimize_not.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.greptime_timestamp]], aggr=[[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))]] | +| | Filter: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)) IS NOT NULL | +| | Projection: aggr_optimize_not.greptime_timestamp, prom_rate(greptime_timestamp_range, greptime_value, aggr_optimize_not.greptime_timestamp, Int64(120000)) AS prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.c, aggr_optimize_not.d | +| | PromRangeManipulate: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not.a ASC NULLS FIRST, aggr_optimize_not.b ASC NULLS FIRST, aggr_optimize_not.c ASC NULLS FIRST, aggr_optimize_not.d ASC NULLS FIRST, aggr_optimize_not.greptime_timestamp ASC NULLS FIRST | +| | Filter: aggr_optimize_not.greptime_timestamp >= TimestampMillisecond(1752591744001, None) AND aggr_optimize_not.greptime_timestamp <= TimestampMillisecond(1752592164000, None) | +| | TableScan: aggr_optimize_not | +| | ]] | +| | SubqueryAlias: aggr_optimize_not_count | +| | Sort: aggr_optimize_not_count.a ASC NULLS LAST, aggr_optimize_not_count.b ASC NULLS LAST, aggr_optimize_not_count.c ASC NULLS LAST, aggr_optimize_not_count.greptime_timestamp ASC NULLS LAST | +| | Aggregate: groupBy=[[aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c, aggr_optimize_not_count.greptime_timestamp]], aggr=[[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))]] | +| | Filter: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)) IS NOT NULL | +| | Projection: aggr_optimize_not_count.greptime_timestamp, prom_rate(greptime_timestamp_range, greptime_value, aggr_optimize_not_count.greptime_timestamp, Int64(120000)) AS prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), aggr_optimize_not_count.a, aggr_optimize_not_count.b, aggr_optimize_not_count.c | +| | PromRangeManipulate: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp], values=["greptime_value"] | +| | PromSeriesNormalize: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivide: tags=["a", "b", "c", "d"] | +| | Sort: aggr_optimize_not_count.a ASC NULLS FIRST, aggr_optimize_not_count.b ASC NULLS FIRST, aggr_optimize_not_count.c ASC NULLS FIRST, aggr_optimize_not_count.d ASC NULLS FIRST, aggr_optimize_not_count.greptime_timestamp ASC NULLS FIRST | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Filter: aggr_optimize_not_count.greptime_timestamp >= TimestampMillisecond(1752591744001, None) AND aggr_optimize_not_count.greptime_timestamp <= TimestampMillisecond(1752592164000, None) | +| | TableScan: aggr_optimize_not_count | +| | ]] | +| physical_plan | ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c, greptime_timestamp@3 as greptime_timestamp, sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@5 / sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))@4 as aggr_optimize_not.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))) / aggr_optimize_not_count.sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))] | | | REDACTED -| | CoalescePartitionsExec | -| | AggregateExec: mode=SinglePartitioned, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))] | -| | FilterExec: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))@1 IS NOT NULL | -| | ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_rate(greptime_timestamp_range@6, greptime_value@5, greptime_timestamp@4, 120000) as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@0 as a, b@1 as b, c@2 as c] | -| | PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] | -| | PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | -| | PromSeriesDivideExec: tags=["a", "b", "c", "d"] | -| | SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC, d@3 ASC, greptime_timestamp@4 ASC], preserve_partitioning=[true] | +| | CoalescePartitionsExec | +| | AggregateExec: mode=SinglePartitioned, gby=[a@2 as a, b@3 as b, c@4 as c, greptime_timestamp@0 as greptime_timestamp], aggr=[sum(prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)))] | +| | FilterExec: prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000))@1 IS NOT NULL | +| | ProjectionExec: expr=[greptime_timestamp@4 as greptime_timestamp, prom_rate(greptime_timestamp_range@6, greptime_value@5, greptime_timestamp@4, 120000) as prom_rate(greptime_timestamp_range,greptime_value,greptime_timestamp,Int64(120000)), a@0 as a, b@1 as b, c@2 as c] | +| | PromRangeManipulateExec: req range=[1752591864000..1752592164000], interval=[30000], eval range=[120000], time index=[greptime_timestamp] | +| | PromSeriesNormalizeExec: offset=[0], time index=[greptime_timestamp], filter NaN: [true] | +| | PromSeriesDivideExec: tags=["a", "b", "c", "d"] | +| | SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC, d@3 ASC, greptime_timestamp@4 ASC], preserve_partitioning=[true] | | | MergeScanExec: REDACTED -| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST], preserve_partitioning=[true] | -| | CooperativeExec | +| | SortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST, greptime_timestamp@3 ASC NULLS LAST], preserve_partitioning=[true] | +| | CooperativeExec | | | MergeScanExec: REDACTED -| | | -+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| | | ++---------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -- SQLNESS REPLACE (metrics.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED diff --git a/tests/cases/standalone/common/promql/anon_promql_ratio_repro.result b/tests/cases/standalone/common/promql/anon_promql_ratio_repro.result new file mode 100644 index 0000000000..ab3c4db715 --- /dev/null +++ b/tests/cases/standalone/common/promql/anon_promql_ratio_repro.result @@ -0,0 +1,106 @@ +CREATE TABLE phy ( + t TIMESTAMP TIME INDEX, + v DOUBLE +) ENGINE=metric WITH ("physical_metric_table" = ""); + +Affected Rows: 0 + +CREATE TABLE metric_a ( + l1 STRING NULL, + l2 STRING NULL, + l3 STRING NULL, + l4 STRING NULL, + l5 STRING NULL, + t TIMESTAMP NOT NULL, + v DOUBLE NULL, + TIME INDEX (t), + PRIMARY KEY (l1, l2, l3, l4, l5) +) ENGINE=metric WITH (on_physical_table = 'phy'); + +Affected Rows: 0 + +CREATE TABLE metric_b ( + l6 STRING NULL, + l1 STRING NULL, + l2 STRING NULL, + l3 STRING NULL, + l4 STRING NULL, + t TIMESTAMP NOT NULL, + v DOUBLE NULL, + TIME INDEX (t), + PRIMARY KEY (l6, l1, l2, l3, l4) +) ENGINE=metric WITH (on_physical_table = 'phy'); + +Affected Rows: 0 + +INSERT INTO metric_a (l1, l2, l3, l4, l5, t, v) VALUES + ('v1', 'v2', 'v3', 'v4a', 'v5a', 1, 0), + ('v1', 'v2', 'v3', 'v4a', 'v5a', 180000, 120), + ('v1', 'v2', 'v3', 'v4a', 'v5a', 360000, 240), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 1, 0), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 180000, 30), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 360000, 60), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 1, 0), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 180000, 60), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 360000, 120); + +Affected Rows: 9 + +INSERT INTO metric_b (l6, l1, l2, l3, l4, t, v) VALUES + ('v6', 'v1', 'v2', 'v3', 'v4a', 1, 1), + ('v6', 'v1', 'v2', 'v3', 'v4a', 180000, 1), + ('v6', 'v1', 'v2', 'v3', 'v4a', 360000, 1), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 1, 2), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 180000, 2), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 360000, 2); + +Affected Rows: 6 + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') count(((rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m]) / on(l3,l4) group_left metric_b{l6="v6",l1="v1",l2="v2",l3=~"v3(|-a|-b)"}) > 0.50)); + ++---------------------+-------------------------------------------------------------------+ +| t | count(metric_a.prom_rate(t_range,v,t,Int64(180000)) / metric_b.v) | ++---------------------+-------------------------------------------------------------------+ +| 1970-01-01T00:03:00 | 1 | ++---------------------+-------------------------------------------------------------------+ + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m])); + ++---------------------+---------------------------------------------+ +| t | count(prom_rate(t_range,v,t,Int64(180000))) | ++---------------------+---------------------------------------------+ +| 1970-01-01T00:03:00 | 3 | ++---------------------+---------------------------------------------+ + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m])) / 2; + ++---------------------+----------------------------------------------------------+ +| t | count(prom_rate(t_range,v,t,Int64(180000))) / Float64(2) | ++---------------------+----------------------------------------------------------+ +| 1970-01-01T00:03:00 | 1.5 | ++---------------------+----------------------------------------------------------+ + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') (count(((rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m]) / on(l3,l4) group_left metric_b{l6="v6",l1="v1",l2="v2",l3=~"v3(|-a|-b)"}) > 0.50)) / count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m]))) * 100; + ++---------------------+--------------------------------------------------------------------------------------------------------------------------------------------------+ +| t | metric_b.count(metric_a.prom_rate(t_range,v,t,Int64(180000)) / metric_b.v) / metric_a.count(prom_rate(t_range,v,t,Int64(180000))) * Float64(100) | ++---------------------+--------------------------------------------------------------------------------------------------------------------------------------------------+ +| 1970-01-01T00:03:00 | 33.33333333333333 | ++---------------------+--------------------------------------------------------------------------------------------------------------------------------------------------+ + +DROP TABLE metric_a; + +Affected Rows: 0 + +DROP TABLE metric_b; + +Affected Rows: 0 + +DROP TABLE phy; + +Affected Rows: 0 + diff --git a/tests/cases/standalone/common/promql/anon_promql_ratio_repro.sql b/tests/cases/standalone/common/promql/anon_promql_ratio_repro.sql new file mode 100644 index 0000000000..946d4f93a1 --- /dev/null +++ b/tests/cases/standalone/common/promql/anon_promql_ratio_repro.sql @@ -0,0 +1,63 @@ +CREATE TABLE phy ( + t TIMESTAMP TIME INDEX, + v DOUBLE +) ENGINE=metric WITH ("physical_metric_table" = ""); + +CREATE TABLE metric_a ( + l1 STRING NULL, + l2 STRING NULL, + l3 STRING NULL, + l4 STRING NULL, + l5 STRING NULL, + t TIMESTAMP NOT NULL, + v DOUBLE NULL, + TIME INDEX (t), + PRIMARY KEY (l1, l2, l3, l4, l5) +) ENGINE=metric WITH (on_physical_table = 'phy'); + +CREATE TABLE metric_b ( + l6 STRING NULL, + l1 STRING NULL, + l2 STRING NULL, + l3 STRING NULL, + l4 STRING NULL, + t TIMESTAMP NOT NULL, + v DOUBLE NULL, + TIME INDEX (t), + PRIMARY KEY (l6, l1, l2, l3, l4) +) ENGINE=metric WITH (on_physical_table = 'phy'); + +INSERT INTO metric_a (l1, l2, l3, l4, l5, t, v) VALUES + ('v1', 'v2', 'v3', 'v4a', 'v5a', 1, 0), + ('v1', 'v2', 'v3', 'v4a', 'v5a', 180000, 120), + ('v1', 'v2', 'v3', 'v4a', 'v5a', 360000, 240), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 1, 0), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 180000, 30), + ('v1', 'v2', 'v3', 'v4a', 'v5b', 360000, 60), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 1, 0), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 180000, 60), + ('v1', 'v2', 'v3-b', 'v4b', 'v5c', 360000, 120); + +INSERT INTO metric_b (l6, l1, l2, l3, l4, t, v) VALUES + ('v6', 'v1', 'v2', 'v3', 'v4a', 1, 1), + ('v6', 'v1', 'v2', 'v3', 'v4a', 180000, 1), + ('v6', 'v1', 'v2', 'v3', 'v4a', 360000, 1), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 1, 2), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 180000, 2), + ('v6', 'v1', 'v2', 'v3-b', 'v4b', 360000, 2); + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') count(((rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m]) / on(l3,l4) group_left metric_b{l6="v6",l1="v1",l2="v2",l3=~"v3(|-a|-b)"}) > 0.50)); + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m])); + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m])) / 2; + +-- SQLNESS SORT_RESULT 3 1 +TQL EVAL (180, 360, '180s') (count(((rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m]) / on(l3,l4) group_left metric_b{l6="v6",l1="v1",l2="v2",l3=~"v3(|-a|-b)"}) > 0.50)) / count(rate(metric_a{l1="v1",l2="v2",l3=~"v3(|-a|-b)"}[3m]))) * 100; + +DROP TABLE metric_a; +DROP TABLE metric_b; +DROP TABLE phy; diff --git a/tests/cases/standalone/common/tql/tql-cte.result b/tests/cases/standalone/common/tql/tql-cte.result index a8c0c45d5d..e8278e80bd 100644 --- a/tests/cases/standalone/common/tql/tql-cte.result +++ b/tests/cases/standalone/common/tql/tql-cte.result @@ -427,8 +427,8 @@ SELECT min(val) as min_computed, max(val) as max_computed FROM computed; | | Aggregate: groupBy=[[]], aggr=[[min(computed.val), max(computed.val)]] | | | SubqueryAlias: computed | | | Projection: metric.ts AS ts, val * Float64(2) + Float64(1) AS val | -| | Projection: metric.ts, val * Float64(2) + Float64(1) AS val * Float64(2) + Float64(1) | -| | Projection: metric.ts, metric.val * Float64(2) AS val * Float64(2) | +| | Projection: metric.ts, CAST(val * Float64(2) AS Float64) + Float64(1) AS val * Float64(2) + Float64(1) | +| | Projection: metric.ts, CAST(metric.val AS Float64) * Float64(2) AS val * Float64(2) | | | PromInstantManipulate: range=[0..40000], lookback=[300000], interval=[10000], time index=[ts] | | | PromSeriesDivide: tags=[] | | | Filter: metric.ts >= TimestampMillisecond(-299999, None) AND metric.ts <= TimestampMillisecond(40000, None) | From 187b8d3798b69d4881d2bb667e4facd16729b115 Mon Sep 17 00:00:00 2001 From: liyang Date: Tue, 24 Mar 2026 17:19:18 +0800 Subject: [PATCH 10/13] ci: remove redundant directory level when uploading artifacts to S3 (#7852) Signed-off-by: liyang --- .github/scripts/upload-artifacts-to-s3.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/scripts/upload-artifacts-to-s3.sh b/.github/scripts/upload-artifacts-to-s3.sh index 310575c069..1ddf32044b 100755 --- a/.github/scripts/upload-artifacts-to-s3.sh +++ b/.github/scripts/upload-artifacts-to-s3.sh @@ -33,7 +33,7 @@ function upload_artifacts() { # └── greptime-darwin-amd64-v0.2.0.tar.gz find "$ARTIFACTS_DIR" -type f \( -name "*.tar.gz" -o -name "*.sha256sum" \) | while IFS= read -r file; do filename=$(basename "$file") - TARGET_URL="$PROXY_URL/$RELEASE_DIRS/$VERSION/$filename" + TARGET_URL="$PROXY_URL/$RELEASE_DIRS/$VERSION" curl -X PUT \ -u "$PROXY_USERNAME:$PROXY_PASSWORD" \ @@ -49,7 +49,7 @@ function update_version_info() { if [[ "$VERSION" =~ ^v[0-9]+\.[0-9]+\.[0-9]+$ ]]; then echo "Updating latest-version.txt" echo "$VERSION" > latest-version.txt - TARGET_URL="$PROXY_URL/$RELEASE_DIRS/latest-version.txt" + TARGET_URL="$PROXY_URL/$RELEASE_DIRS" curl -X PUT \ -u "$PROXY_USERNAME:$PROXY_PASSWORD" \ @@ -62,7 +62,7 @@ function update_version_info() { echo "Updating latest-nightly-version.txt" echo "$VERSION" > latest-nightly-version.txt - TARGET_URL="$PROXY_URL/$RELEASE_DIRS/latest-nightly-version.txt" + TARGET_URL="$PROXY_URL/$RELEASE_DIRS" curl -X PUT \ -u "$PROXY_USERNAME:$PROXY_PASSWORD" \ -F "file=@latest-nightly-version.txt" \ From 0e22d6a72b7ee66b5e3c284a47da97ec6af2837e Mon Sep 17 00:00:00 2001 From: Yingwen Date: Tue, 24 Mar 2026 18:01:13 +0800 Subject: [PATCH 11/13] feat: implement partition range cache stream (#7842) * feat: add cache stream helpers, key construction, config wiring, and metrics for partition range cache Add range result cache size config field and wire it through cache builder chains. Implement cache key building (build_range_cache_key), stream replay/store helpers (cached_flat_range_stream, cache_flat_range_stream), dictionary compaction (compact_pk_dictionary), and partition range row group collection. Add range cache metrics (size, hit, miss) to ScanMetricsSet and PartitionMetrics. Move fingerprint tests from scan_region to range_cache module. These functions are not yet wired into scan execution. Signed-off-by: evenyag * feat: add benchmark for cache stream Signed-off-by: evenyag * refactor: move bench_util to test_util Signed-off-by: evenyag * feat: share dict Signed-off-by: evenyag * test: test ptr_eq Signed-off-by: evenyag * chore: fmt code Signed-off-by: evenyag * refactor: simplify value array handling Signed-off-by: evenyag * chore: add todo for estimate size Signed-off-by: evenyag * feat: simplify size calculation Signed-off-by: evenyag * chore: remove one test Signed-off-by: evenyag * test: update config test Signed-off-by: evenyag * chore: address review comment Only ignore exprs that can extract time ranges Signed-off-by: evenyag * test: fix tests Signed-off-by: evenyag --------- Signed-off-by: evenyag --- src/mito2/Cargo.toml | 5 + src/mito2/benches/bench_cache_stream.rs | 126 +++++ src/mito2/benches/memtable_bench.rs | 245 +------- src/mito2/src/cache.rs | 13 +- src/mito2/src/config.rs | 4 + src/mito2/src/memtable/bulk/part.rs | 11 +- src/mito2/src/memtable/bulk/part_reader.rs | 2 +- src/mito2/src/read.rs | 3 + src/mito2/src/read/range_cache.rs | 628 ++++++++++++++++++++- src/mito2/src/read/scan_region.rs | 39 +- src/mito2/src/read/scan_util.rs | 40 ++ src/mito2/src/test_util.rs | 1 + src/mito2/src/test_util/bench_util.rs | 259 +++++++++ src/mito2/src/test_util/memtable_util.rs | 2 +- src/mito2/src/worker.rs | 2 + src/table/src/predicate.rs | 2 +- tests-integration/tests/http.rs | 1 + 17 files changed, 1113 insertions(+), 270 deletions(-) create mode 100644 src/mito2/benches/bench_cache_stream.rs create mode 100644 src/mito2/src/test_util/bench_util.rs diff --git a/src/mito2/Cargo.toml b/src/mito2/Cargo.toml index 1d7cf7b6d7..a78bf079b0 100644 --- a/src/mito2/Cargo.toml +++ b/src/mito2/Cargo.toml @@ -108,6 +108,11 @@ name = "memtable_bench" harness = false required-features = ["test"] +[[bench]] +name = "bench_cache_stream" +harness = false +required-features = ["test"] + [[bench]] name = "bench_filter_time_partition" harness = false diff --git a/src/mito2/benches/bench_cache_stream.rs b/src/mito2/benches/bench_cache_stream.rs new file mode 100644 index 0000000000..f2314f2ccb --- /dev/null +++ b/src/mito2/benches/bench_cache_stream.rs @@ -0,0 +1,126 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Benchmarks for `cache_flat_range_stream` overhead. +//! +//! Compares consuming batches from a plain stream vs through the caching wrapper +//! that clones batches for the range cache. +//! +//! Run with: +//! ```sh +//! cargo bench -p mito2 --features test --bench bench_cache_stream +//! ``` + +use std::collections::VecDeque; +use std::sync::Arc; + +use criterion::{Criterion, criterion_group, criterion_main}; +use futures::TryStreamExt; +use mito_codec::row_converter::DensePrimaryKeyCodec; +use mito2::memtable::bulk::context::BulkIterContext; +use mito2::memtable::bulk::part::{BulkPartConverter, BulkPartEncoder}; +use mito2::memtable::bulk::part_reader::EncodedBulkPartIter; +use mito2::read::range_cache::bench_cache_flat_range_stream; +use mito2::sst::parquet::DEFAULT_ROW_GROUP_SIZE; +use mito2::sst::{FlatSchemaOptions, to_flat_sst_arrow_schema}; +use mito2::test_util::bench_util::{CpuDataGenerator, cpu_metadata}; + +fn cache_flat_range_stream_bench(c: &mut Criterion) { + let metadata = Arc::new(cpu_metadata()); + let region_id = metadata.region_id; + let start_sec = 1710043200; + // 2000 hosts × 51 steps = 102,000 rows ≈ DEFAULT_ROW_GROUP_SIZE + let num_hosts = 2000; + let end_sec = start_sec + 510; + let generator = CpuDataGenerator::new(metadata.clone(), num_hosts, start_sec, end_sec); + + // Build a BulkPart from all the generated data + let schema = to_flat_sst_arrow_schema(&metadata, &FlatSchemaOptions::default()); + let codec = Arc::new(DensePrimaryKeyCodec::new(&metadata)); + + let mut converter = BulkPartConverter::new( + &metadata, + schema, + DEFAULT_ROW_GROUP_SIZE, + codec, + true, // store_pk_columns + ); + for kvs in generator.iter() { + converter.append_key_values(&kvs).unwrap(); + } + let bulk_part = converter.convert().unwrap(); + + // Encode to parquet + let encoder = BulkPartEncoder::new(metadata.clone(), DEFAULT_ROW_GROUP_SIZE).unwrap(); + let encoded_part = encoder.encode_part(&bulk_part).unwrap().unwrap(); + + // Decode all record batches + let num_row_groups = encoded_part.metadata().parquet_metadata.num_row_groups(); + let context = Arc::new( + BulkIterContext::new( + metadata.clone(), + None, // No projection + None, // No predicate + false, + ) + .unwrap(), + ); + let row_groups: VecDeque = (0..num_row_groups).collect(); + + let rt = tokio::runtime::Runtime::new().unwrap(); + + let mut group = c.benchmark_group("cache_flat_range_stream"); + group.sample_size(10); + + group.bench_function("baseline_iter_stream", |b| { + b.iter(|| { + rt.block_on(async { + let iter = EncodedBulkPartIter::try_new( + &encoded_part, + context.clone(), + row_groups.clone(), + None, + None, + ) + .unwrap(); + let stream: mito2::read::BoxedRecordBatchStream = + Box::pin(futures::stream::iter(iter)); + let mut stream = stream; + while let Some(_batch) = stream.try_next().await.unwrap() {} + }); + }); + }); + + group.bench_function("cache_flat_range_stream", |b| { + b.iter(|| { + rt.block_on(async { + let iter = EncodedBulkPartIter::try_new( + &encoded_part, + context.clone(), + row_groups.clone(), + None, + None, + ) + .unwrap(); + let stream: mito2::read::BoxedRecordBatchStream = + Box::pin(futures::stream::iter(iter)); + let mut stream = bench_cache_flat_range_stream(stream, 64 * 1024 * 1024, region_id); + while let Some(_batch) = stream.try_next().await.unwrap() {} + }); + }); + }); +} + +criterion_group!(benches, cache_flat_range_stream_bench); +criterion_main!(benches); diff --git a/src/mito2/benches/memtable_bench.rs b/src/mito2/benches/memtable_bench.rs index df991f6f92..8336625e3c 100644 --- a/src/mito2/benches/memtable_bench.rs +++ b/src/mito2/benches/memtable_bench.rs @@ -12,15 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +//! Benchmarks for memtable operations: writes, full scans, filtered scans, +//! bulk part conversion, record batch iteration with filters, and flat merge. +//! +//! Run with: +//! ```sh +//! cargo bench -p mito2 --features test --bench memtable_bench +//! ``` + use std::sync::Arc; -use api::v1::value::ValueData; -use api::v1::{Row, Rows, SemanticType}; use criterion::{Criterion, criterion_group, criterion_main}; -use datafusion_common::Column; -use datafusion_expr::{Expr, lit}; -use datatypes::data_type::ConcreteDataType; -use datatypes::schema::ColumnSchema; use mito_codec::row_converter::DensePrimaryKeyCodec; use mito2::memtable::bulk::context::BulkIterContext; use mito2::memtable::bulk::part::BulkPartConverter; @@ -28,20 +30,13 @@ use mito2::memtable::bulk::part_reader::BulkPartBatchIter; use mito2::memtable::bulk::{BulkMemtable, BulkMemtableConfig}; use mito2::memtable::partition_tree::{PartitionTreeConfig, PartitionTreeMemtable}; use mito2::memtable::time_series::TimeSeriesMemtable; -use mito2::memtable::{IterBuilder, KeyValues, Memtable, RangesOptions}; +use mito2::memtable::{IterBuilder, Memtable, RangesOptions}; use mito2::read::flat_merge::FlatMergeIterator; use mito2::read::scan_region::PredicateGroup; use mito2::region::options::MergeMode; use mito2::sst::{FlatSchemaOptions, to_flat_sst_arrow_schema}; -use mito2::test_util::memtable_util::{self, region_metadata_to_row_schema}; -use rand::Rng; -use rand::rngs::ThreadRng; -use rand::seq::IndexedRandom; -use store_api::metadata::{ - ColumnMetadata, RegionMetadata, RegionMetadataBuilder, RegionMetadataRef, -}; -use store_api::storage::RegionId; -use table::predicate::Predicate; +use mito2::test_util::bench_util::{CpuDataGenerator, cpu_metadata}; +use mito2::test_util::memtable_util; /// Writes rows. fn write_rows(c: &mut Criterion) { @@ -216,224 +211,6 @@ fn filter_1_host(c: &mut Criterion) { }); } -struct Host { - hostname: String, - region: String, - datacenter: String, - rack: String, - os: String, - arch: String, - team: String, - service: String, - service_version: String, - service_environment: String, -} - -impl Host { - fn random_with_id(id: usize) -> Host { - let mut rng = rand::rng(); - let region = format!("ap-southeast-{}", rng.random_range(0..10)); - let datacenter = format!( - "{}{}", - region, - ['a', 'b', 'c', 'd', 'e'].choose(&mut rng).unwrap() - ); - Host { - hostname: format!("host_{id}"), - region, - datacenter, - rack: rng.random_range(0..100).to_string(), - os: "Ubuntu16.04LTS".to_string(), - arch: "x86".to_string(), - team: "CHI".to_string(), - service: rng.random_range(0..100).to_string(), - service_version: rng.random_range(0..10).to_string(), - service_environment: "test".to_string(), - } - } - - fn fill_values(&self, values: &mut Vec) { - let tags = [ - api::v1::Value { - value_data: Some(ValueData::StringValue(self.hostname.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.region.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.datacenter.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.rack.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.os.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.arch.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.team.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.service.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.service_version.clone())), - }, - api::v1::Value { - value_data: Some(ValueData::StringValue(self.service_environment.clone())), - }, - ]; - for tag in tags { - values.push(tag); - } - } -} - -struct CpuDataGenerator { - metadata: RegionMetadataRef, - column_schemas: Vec, - hosts: Vec, - start_sec: i64, - end_sec: i64, -} - -impl CpuDataGenerator { - fn new(metadata: RegionMetadataRef, num_hosts: usize, start_sec: i64, end_sec: i64) -> Self { - let column_schemas = region_metadata_to_row_schema(&metadata); - Self { - metadata, - column_schemas, - hosts: Self::generate_hosts(num_hosts), - start_sec, - end_sec, - } - } - - fn iter(&self) -> impl Iterator + '_ { - // point per 10s. - (self.start_sec..self.end_sec) - .step_by(10) - .enumerate() - .map(|(seq, ts)| self.build_key_values(seq, ts)) - } - - fn build_key_values(&self, seq: usize, current_sec: i64) -> KeyValues { - let rows = self - .hosts - .iter() - .map(|host| { - let mut rng = rand::rng(); - let mut values = Vec::with_capacity(21); - values.push(api::v1::Value { - value_data: Some(ValueData::TimestampMillisecondValue(current_sec * 1000)), - }); - host.fill_values(&mut values); - for _ in 0..10 { - values.push(api::v1::Value { - value_data: Some(ValueData::F64Value(Self::random_f64(&mut rng))), - }); - } - Row { values } - }) - .collect(); - let mutation = api::v1::Mutation { - op_type: api::v1::OpType::Put as i32, - sequence: seq as u64, - rows: Some(Rows { - schema: self.column_schemas.clone(), - rows, - }), - write_hint: None, - }; - - KeyValues::new(&self.metadata, mutation).unwrap() - } - - fn random_host_filter(&self) -> Predicate { - let host = self.random_hostname(); - let expr = Expr::Column(Column::from_name("hostname")).eq(lit(host)); - Predicate::new(vec![expr]) - } - - fn random_host_filter_exprs(&self) -> Vec { - let host = self.random_hostname(); - vec![Expr::Column(Column::from_name("hostname")).eq(lit(host))] - } - - fn random_hostname(&self) -> String { - let mut rng = rand::rng(); - self.hosts.choose(&mut rng).unwrap().hostname.clone() - } - - fn random_f64(rng: &mut ThreadRng) -> f64 { - let base: u32 = rng.random_range(30..95); - base as f64 - } - - fn generate_hosts(num_hosts: usize) -> Vec { - (0..num_hosts).map(Host::random_with_id).collect() - } -} - -/// Creates a metadata for TSBS cpu-like table. -fn cpu_metadata() -> RegionMetadata { - let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 1)); - builder.push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new( - "ts", - ConcreteDataType::timestamp_millisecond_datatype(), - false, - ), - semantic_type: SemanticType::Timestamp, - column_id: 0, - }); - let mut column_id = 1; - let tags = [ - "hostname", - "region", - "datacenter", - "rack", - "os", - "arch", - "team", - "service", - "service_version", - "service_environment", - ]; - for tag in tags { - builder.push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new(tag, ConcreteDataType::string_datatype(), true), - semantic_type: SemanticType::Tag, - column_id, - }); - column_id += 1; - } - let fields = [ - "usage_user", - "usage_system", - "usage_idle", - "usage_nice", - "usage_iowait", - "usage_irq", - "usage_softirq", - "usage_steal", - "usage_guest", - "usage_guest_nice", - ]; - for field in fields { - builder.push_column_metadata(ColumnMetadata { - column_schema: ColumnSchema::new(field, ConcreteDataType::float64_datatype(), true), - semantic_type: SemanticType::Field, - column_id, - }); - column_id += 1; - } - builder.primary_key(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); - builder.build().unwrap() -} - fn bulk_part_converter(c: &mut Criterion) { let metadata = Arc::new(cpu_metadata()); let start_sec = 1710043200; diff --git a/src/mito2/src/cache.rs b/src/mito2/src/cache.rs index c9a8b99166..35db74eee6 100644 --- a/src/mito2/src/cache.rs +++ b/src/mito2/src/cache.rs @@ -350,7 +350,7 @@ impl CacheStrategy { /// Calls [CacheManager::get_range_result()]. /// It returns None if the strategy is [CacheStrategy::Compaction] or [CacheStrategy::Disabled]. - #[cfg_attr(not(test), allow(dead_code))] + #[allow(dead_code)] pub(crate) fn get_range_result( &self, key: &RangeScanCacheKey, @@ -363,7 +363,6 @@ impl CacheStrategy { /// Calls [CacheManager::put_range_result()]. /// It does nothing if the strategy isn't [CacheStrategy::EnableAll]. - #[cfg_attr(not(test), allow(dead_code))] pub(crate) fn put_range_result( &self, key: RangeScanCacheKey, @@ -476,7 +475,6 @@ pub struct CacheManager { /// Cache for time series selectors. selector_result_cache: Option, /// Cache for range scan outputs in flat format. - #[cfg_attr(not(test), allow(dead_code))] range_result_cache: Option, /// Cache for index result. index_result_cache: Option, @@ -713,7 +711,7 @@ impl CacheManager { } /// Gets cached result for range scan. - #[cfg_attr(not(test), allow(dead_code))] + #[allow(dead_code)] pub(crate) fn get_range_result( &self, key: &RangeScanCacheKey, @@ -723,8 +721,7 @@ impl CacheManager { .and_then(|cache| update_hit_miss(cache.get(key), RANGE_RESULT_TYPE)) } - /// Puts range scan result into the cache. - #[cfg_attr(not(test), allow(dead_code))] + /// Puts range scan result into cache. pub(crate) fn put_range_result( &self, key: RangeScanCacheKey, @@ -949,7 +946,7 @@ impl CacheManagerBuilder { Cache::builder() .max_capacity(self.range_result_cache_size) .weigher(range_result_cache_weight) - .eviction_listener(|k, v, cause| { + .eviction_listener(move |k, v, cause| { let size = range_result_cache_weight(&k, &v); CACHE_BYTES .with_label_values(&[RANGE_RESULT_TYPE]) @@ -1361,7 +1358,7 @@ mod tests { } .build(), }; - let value = Arc::new(RangeScanCacheValue::new(Vec::new())); + let value = Arc::new(RangeScanCacheValue::new(Vec::new(), 0)); assert!(cache.get_range_result(&key).is_none()); cache.put_range_result(key.clone(), value.clone()); diff --git a/src/mito2/src/config.rs b/src/mito2/src/config.rs index 602f5508ba..0eee067ab6 100644 --- a/src/mito2/src/config.rs +++ b/src/mito2/src/config.rs @@ -116,6 +116,8 @@ pub struct MitoConfig { pub page_cache_size: ReadableSize, /// Cache size for time series selector (e.g. `last_value()`). Setting it to 0 to disable the cache. pub selector_result_cache_size: ReadableSize, + /// Cache size for flat range scan results. Setting it to 0 to disable the cache. + pub range_result_cache_size: ReadableSize, /// Whether to enable the write cache. pub enable_write_cache: bool, /// File system path for write cache dir's root, defaults to `{data_home}`. @@ -200,6 +202,7 @@ impl Default for MitoConfig { vector_cache_size: ReadableSize::mb(512), page_cache_size: ReadableSize::mb(512), selector_result_cache_size: ReadableSize::mb(512), + range_result_cache_size: ReadableSize::mb(512), enable_write_cache: false, write_cache_path: String::new(), write_cache_size: ReadableSize::gb(5), @@ -336,6 +339,7 @@ impl MitoConfig { self.vector_cache_size = mem_cache_size; self.page_cache_size = page_cache_size; self.selector_result_cache_size = mem_cache_size; + self.range_result_cache_size = mem_cache_size; self.index.adjust_buffer_and_cache_size(sys_memory); } diff --git a/src/mito2/src/memtable/bulk/part.rs b/src/mito2/src/memtable/bulk/part.rs index 71e49776c0..bf345c038e 100644 --- a/src/mito2/src/memtable/bulk/part.rs +++ b/src/mito2/src/memtable/bulk/part.rs @@ -967,7 +967,7 @@ impl EncodedBulkPart { Self { data, metadata } } - pub(crate) fn metadata(&self) -> &BulkPartMeta { + pub fn metadata(&self) -> &BulkPartMeta { &self.metadata } @@ -977,7 +977,7 @@ impl EncodedBulkPart { } /// Returns the encoded data. - pub(crate) fn data(&self) -> &Bytes { + pub fn data(&self) -> &Bytes { &self.data } @@ -1121,10 +1121,7 @@ pub struct BulkPartEncoder { } impl BulkPartEncoder { - pub(crate) fn new( - metadata: RegionMetadataRef, - row_group_size: usize, - ) -> Result { + pub fn new(metadata: RegionMetadataRef, row_group_size: usize) -> Result { // TODO(yingwen): Skip arrow schema if needed. let json = metadata.to_json().context(InvalidMetadataSnafu)?; let key_value_meta = @@ -1216,7 +1213,7 @@ impl BulkPartEncoder { } /// Encodes bulk part to a [EncodedBulkPart], returns the encoded data. - fn encode_part(&self, part: &BulkPart) -> Result> { + pub fn encode_part(&self, part: &BulkPart) -> Result> { if part.batch.num_rows() == 0 { return Ok(None); } diff --git a/src/mito2/src/memtable/bulk/part_reader.rs b/src/mito2/src/memtable/bulk/part_reader.rs index 1e9d955321..904aae8c90 100644 --- a/src/mito2/src/memtable/bulk/part_reader.rs +++ b/src/mito2/src/memtable/bulk/part_reader.rs @@ -50,7 +50,7 @@ pub struct EncodedBulkPartIter { impl EncodedBulkPartIter { /// Creates a new [BulkPartIter]. - pub(crate) fn try_new( + pub fn try_new( encoded_part: &EncodedBulkPart, context: BulkIterContextRef, mut row_groups_to_read: VecDeque, diff --git a/src/mito2/src/read.rs b/src/mito2/src/read.rs index 240a99c247..84931b9f37 100644 --- a/src/mito2/src/read.rs +++ b/src/mito2/src/read.rs @@ -27,6 +27,9 @@ pub mod projection; pub(crate) mod prune; pub(crate) mod pruner; pub mod range; +#[cfg(feature = "test")] +pub mod range_cache; +#[cfg(not(feature = "test"))] pub(crate) mod range_cache; pub mod scan_region; pub mod scan_util; diff --git a/src/mito2/src/read/range_cache.rs b/src/mito2/src/read/range_cache.rs index 5b90e68bae..5fc8931691 100644 --- a/src/mito2/src/read/range_cache.rs +++ b/src/mito2/src/read/range_cache.rs @@ -17,12 +17,23 @@ use std::mem; use std::sync::Arc; +use async_stream::try_stream; +use common_time::range::TimestampRange; +use datatypes::arrow::array::{Array, AsArray, DictionaryArray}; +use datatypes::arrow::datatypes::UInt32Type; use datatypes::arrow::record_batch::RecordBatch; use datatypes::prelude::ConcreteDataType; +use futures::TryStreamExt; +use store_api::region_engine::PartitionRange; use store_api::storage::{ColumnId, FileId, RegionId, TimeSeriesRowSelector}; -use crate::memtable::record_batch_estimated_size; +use crate::cache::CacheStrategy; +use crate::read::BoxedRecordBatchStream; +use crate::read::scan_region::StreamContext; +use crate::read::scan_util::PartitionMetrics; use crate::region::options::MergeMode; +use crate::sst::file::FileTimeRange; +use crate::sst::parquet::flat_format::primary_key_column_index; /// Fingerprint of the scan request fields that affect partition range cache reuse. /// @@ -124,7 +135,6 @@ impl ScanRequestFingerprint { .unwrap_or(&[]) } - #[cfg(test)] pub(crate) fn without_time_filters(&self) -> Self { Self { inner: Arc::clone(&self.inner), @@ -163,7 +173,7 @@ impl ScanRequestFingerprint { #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub(crate) struct RangeScanCacheKey { pub(crate) region_id: RegionId, - /// Sorted (file_id, row_group_index) pairs that uniquely identify the covered data. + /// Sorted (file_id, row_group_index) pairs that uniquely identify the data this range covers. pub(crate) row_groups: Vec<(FileId, i64)>, pub(crate) scan: ScanRequestFingerprint, } @@ -179,30 +189,458 @@ impl RangeScanCacheKey { /// Cached result for one range scan. pub(crate) struct RangeScanCacheValue { pub(crate) batches: Vec, + /// Precomputed size of all batches, accounting for shared dictionary values. + estimated_batches_size: usize, } impl RangeScanCacheValue { - #[cfg_attr(not(test), allow(dead_code))] - pub(crate) fn new(batches: Vec) -> Self { - Self { batches } + pub(crate) fn new(batches: Vec, estimated_batches_size: usize) -> Self { + Self { + batches, + estimated_batches_size, + } } pub(crate) fn estimated_size(&self) -> usize { mem::size_of::() + self.batches.capacity() * mem::size_of::() - + self - .batches - .iter() - .map(record_batch_estimated_size) - .sum::() + + self.estimated_batches_size } } +/// Row groups and whether all sources are file-only for a partition range. +#[allow(dead_code)] +pub(crate) struct PartitionRangeRowGroups { + /// Sorted (file_id, row_group_index) pairs. + pub(crate) row_groups: Vec<(FileId, i64)>, + pub(crate) only_file_sources: bool, +} + +/// Collects (file_id, row_group_index) pairs from a partition range's row group indices. +#[allow(dead_code)] +pub(crate) fn collect_partition_range_row_groups( + stream_ctx: &StreamContext, + part_range: &PartitionRange, +) -> PartitionRangeRowGroups { + let range_meta = &stream_ctx.ranges[part_range.identifier]; + let mut row_groups = Vec::new(); + let mut only_file_sources = true; + + for index in &range_meta.row_group_indices { + if stream_ctx.is_file_range_index(*index) { + let file_id = stream_ctx.input.file_from_index(*index).file_id().file_id(); + row_groups.push((file_id, index.row_group_index)); + } else { + only_file_sources = false; + } + } + + row_groups.sort_unstable_by(|a, b| a.0.as_bytes().cmp(b.0.as_bytes()).then(a.1.cmp(&b.1))); + + PartitionRangeRowGroups { + row_groups, + only_file_sources, + } +} + +/// Builds a cache key for the given partition range if it is eligible for caching. +#[allow(dead_code)] +pub(crate) fn build_range_cache_key( + stream_ctx: &StreamContext, + part_range: &PartitionRange, +) -> Option { + let fingerprint = stream_ctx.scan_fingerprint.as_ref()?; + + // Dyn filters can change at runtime, so we can't cache when they're present. + let has_dyn_filters = stream_ctx + .input + .predicate_group() + .predicate_without_region() + .is_some_and(|p| !p.dyn_filters().is_empty()); + if has_dyn_filters { + return None; + } + + let rg = collect_partition_range_row_groups(stream_ctx, part_range); + if !rg.only_file_sources || rg.row_groups.is_empty() { + return None; + } + + let range_meta = &stream_ctx.ranges[part_range.identifier]; + let scan = if query_time_range_covers_partition_range( + stream_ctx.input.time_range.as_ref(), + range_meta.time_range, + ) { + fingerprint.without_time_filters() + } else { + fingerprint.clone() + }; + + Some(RangeScanCacheKey { + region_id: stream_ctx.input.region_metadata().region_id, + row_groups: rg.row_groups, + scan, + }) +} + +#[allow(dead_code)] +fn query_time_range_covers_partition_range( + query_time_range: Option<&TimestampRange>, + partition_time_range: FileTimeRange, +) -> bool { + let Some(query_time_range) = query_time_range else { + return true; + }; + + let (part_start, part_end) = partition_time_range; + query_time_range.contains(&part_start) && query_time_range.contains(&part_end) +} + +/// Returns a stream that replays cached record batches. +#[allow(dead_code)] +pub(crate) fn cached_flat_range_stream(value: Arc) -> BoxedRecordBatchStream { + Box::pin(futures::stream::iter( + value.batches.clone().into_iter().map(Ok), + )) +} + +/// Returns true if two primary key dictionary arrays share the same underlying +/// values buffers by pointer comparison. +/// +/// The primary key column is always `DictionaryArray` with `Binary` values. +fn pk_values_ptr_eq(a: &DictionaryArray, b: &DictionaryArray) -> bool { + let a = a.values().as_binary::(); + let b = b.values().as_binary::(); + let values_eq = a.values().ptr_eq(b.values()) && a.offsets().ptr_eq(b.offsets()); + match (a.nulls(), b.nulls()) { + (Some(a), Some(b)) => values_eq && a.inner().ptr_eq(b.inner()), + (None, None) => values_eq, + _ => false, + } +} + +/// Buffers record batches for caching, tracking memory size while deduplicating +/// shared dictionary values across batches. +/// +/// Uses the primary key column as a proxy to detect dictionary sharing: if the PK +/// column's dictionary values are pointer-equal across batches, we assume all +/// dictionary columns share their values and deduct the total dictionary values size. +struct CacheBatchBuffer { + batches: Vec, + /// Running total of batch memory. + total_size: usize, + /// The first batch's PK dictionary array, for pointer comparison. + /// `None` if no dictionary PK column exists or no batch has been added yet. + first_pk_dict: Option>, + /// Sum of `get_array_memory_size()` of all dictionary value arrays from the first batch. + total_dict_values_size: usize, + /// Whether the PK dictionary is still shared across all batches seen so far. + shared: bool, +} + +impl CacheBatchBuffer { + fn new() -> Self { + Self { + batches: Vec::new(), + total_size: 0, + first_pk_dict: None, + total_dict_values_size: 0, + shared: true, + } + } + + fn push(&mut self, batch: RecordBatch) { + if self.batches.is_empty() { + self.init_first_batch(&batch); + } else { + self.add_subsequent_batch(&batch); + } + self.batches.push(batch); + } + + fn init_first_batch(&mut self, batch: &RecordBatch) { + self.total_size += batch.get_array_memory_size(); + + let pk_col_idx = primary_key_column_index(batch.num_columns()); + let mut total_dict_values_size = 0; + for col_idx in 0..batch.num_columns() { + let col = batch.column(col_idx); + if let Some(dict) = col.as_any().downcast_ref::>() { + total_dict_values_size += dict.values().get_array_memory_size(); + if col_idx == pk_col_idx { + self.first_pk_dict = Some(dict.clone()); + } + } + } + self.total_dict_values_size = total_dict_values_size; + } + + fn add_subsequent_batch(&mut self, batch: &RecordBatch) { + let batch_size = batch.get_array_memory_size(); + + if self.shared + && let Some(first_pk_dict) = &self.first_pk_dict + { + let pk_col_idx = primary_key_column_index(batch.num_columns()); + let col = batch.column(pk_col_idx); + if let Some(dict) = col.as_any().downcast_ref::>() + && pk_values_ptr_eq(first_pk_dict, dict) + { + // PK dict is shared, deduct all dict values sizes. + self.total_size += batch_size - self.total_dict_values_size; + return; + } + // Dictionary diverged. + self.shared = false; + } + + self.total_size += batch_size; + } + + fn estimated_batches_size(&self) -> usize { + self.total_size + } + + fn into_batches(self) -> Vec { + self.batches + } +} + +/// Wraps a stream to cache its output for future range cache hits. +#[allow(dead_code)] +pub(crate) fn cache_flat_range_stream( + mut stream: BoxedRecordBatchStream, + cache_strategy: CacheStrategy, + key: RangeScanCacheKey, + part_metrics: PartitionMetrics, +) -> BoxedRecordBatchStream { + Box::pin(try_stream! { + let mut buffer = CacheBatchBuffer::new(); + while let Some(batch) = stream.try_next().await? { + buffer.push(batch.clone()); + yield batch; + } + + let estimated_size = buffer.estimated_batches_size(); + let batches = buffer.into_batches(); + let value = Arc::new(RangeScanCacheValue::new(batches, estimated_size)); + part_metrics.inc_range_cache_size(key.estimated_size() + value.estimated_size()); + cache_strategy.put_range_result(key, value); + }) +} + +/// Creates a `cache_flat_range_stream` with dummy internals for benchmarking. +/// +/// This avoids exposing `RangeScanCacheKey`, `ScanRequestFingerprint`, and +/// `PartitionMetrics` publicly. +#[cfg(feature = "test")] +pub fn bench_cache_flat_range_stream( + stream: BoxedRecordBatchStream, + cache_size_bytes: u64, + region_id: RegionId, +) -> BoxedRecordBatchStream { + use std::time::Instant; + + use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; + + use crate::region::options::MergeMode; + + let cache_manager = Arc::new( + crate::cache::CacheManager::builder() + .range_result_cache_size(cache_size_bytes) + .build(), + ); + let cache_strategy = CacheStrategy::EnableAll(cache_manager); + + let fingerprint = ScanRequestFingerprintBuilder { + read_column_ids: vec![], + read_column_types: vec![], + filters: vec![], + time_filters: vec![], + series_row_selector: None, + append_mode: false, + filter_deleted: false, + merge_mode: MergeMode::LastRow, + partition_expr_version: 0, + } + .build(); + + let key = RangeScanCacheKey { + region_id, + row_groups: vec![], + scan: fingerprint, + }; + + let metrics_set = ExecutionPlanMetricsSet::new(); + let part_metrics = + PartitionMetrics::new(region_id, 0, "bench", Instant::now(), false, &metrics_set); + + cache_flat_range_stream(stream, cache_strategy, key, part_metrics) +} + #[cfg(test)] mod tests { - use store_api::storage::TimeSeriesRowSelector; + use std::sync::Arc; + use std::time::Instant; + + use common_time::Timestamp; + use common_time::range::TimestampRange; + use common_time::timestamp::TimeUnit; + use datafusion_common::ScalarValue; + use datafusion_expr::{Expr, col, lit}; + use smallvec::smallvec; + use store_api::storage::FileId; use super::*; + use crate::cache::CacheManager; + use crate::read::projection::ProjectionMapper; + use crate::read::range::{RangeMeta, RowGroupIndex, SourceIndex}; + use crate::read::scan_region::{PredicateGroup, ScanInput}; + use crate::test_util::memtable_util::metadata_with_primary_key; + use crate::test_util::scheduler_util::SchedulerEnv; + use crate::test_util::sst_util::sst_file_handle_with_file_id; + + fn test_cache_strategy() -> CacheStrategy { + CacheStrategy::EnableAll(Arc::new( + CacheManager::builder() + .range_result_cache_size(1024) + .build(), + )) + } + + async fn new_stream_context( + filters: Vec, + query_time_range: Option, + partition_time_range: FileTimeRange, + ) -> (StreamContext, PartitionRange) { + let env = SchedulerEnv::new().await; + let metadata = Arc::new(metadata_with_primary_key(vec![0, 1], false)); + let mapper = ProjectionMapper::new(&metadata, [0, 2, 3].into_iter(), true).unwrap(); + let predicate = PredicateGroup::new(metadata.as_ref(), &filters).unwrap(); + let file_id = FileId::random(); + let file = sst_file_handle_with_file_id( + file_id, + partition_time_range.0.value(), + partition_time_range.1.value(), + ); + let input = ScanInput::new(env.access_layer.clone(), mapper) + .with_predicate(predicate) + .with_time_range(query_time_range) + .with_files(vec![file]) + .with_cache(test_cache_strategy()) + .with_flat_format(true); + let range_meta = RangeMeta { + time_range: partition_time_range, + indices: smallvec![SourceIndex { + index: 0, + num_row_groups: 1, + }], + row_group_indices: smallvec![RowGroupIndex { + index: 0, + row_group_index: 0, + }], + num_rows: 10, + }; + let partition_range = range_meta.new_partition_range(0); + let scan_fingerprint = crate::read::scan_region::build_scan_fingerprint(&input); + let stream_ctx = StreamContext { + input, + ranges: vec![range_meta], + scan_fingerprint, + query_start: Instant::now(), + }; + + (stream_ctx, partition_range) + } + + /// Helper to create a timestamp millisecond literal. + fn ts_lit(val: i64) -> Expr { + lit(ScalarValue::TimestampMillisecond(Some(val), None)) + } + + #[tokio::test] + async fn strips_time_only_filters_when_query_covers_partition_range() { + let (stream_ctx, part_range) = new_stream_context( + vec![ + col("ts").gt_eq(ts_lit(1000)), + col("ts").lt(ts_lit(2001)), + col("ts").is_not_null(), + col("k0").eq(lit("foo")), + ], + TimestampRange::with_unit(1000, 2002, TimeUnit::Millisecond), + ( + Timestamp::new_millisecond(1000), + Timestamp::new_millisecond(2000), + ), + ) + .await; + + let key = build_range_cache_key(&stream_ctx, &part_range).unwrap(); + + // Range-reducible time filters should be cleared when query covers partition range. + assert!(key.scan.time_filters().is_empty()); + // Non-range time predicates stay in filters. + let mut expected_filters = [ + col("k0").eq(lit("foo")).to_string(), + col("ts").is_not_null().to_string(), + ]; + expected_filters.sort_unstable(); + assert_eq!(key.scan.filters(), expected_filters.as_slice()); + } + + #[tokio::test] + async fn preserves_time_filters_when_query_does_not_cover_partition_range() { + let (stream_ctx, part_range) = new_stream_context( + vec![col("ts").gt_eq(ts_lit(1000)), col("k0").eq(lit("foo"))], + TimestampRange::with_unit(1000, 1500, TimeUnit::Millisecond), + ( + Timestamp::new_millisecond(1000), + Timestamp::new_millisecond(2000), + ), + ) + .await; + + let key = build_range_cache_key(&stream_ctx, &part_range).unwrap(); + + // Time filters should be preserved when query does not cover partition range. + assert_eq!( + key.scan.time_filters(), + [col("ts").gt_eq(ts_lit(1000)).to_string()].as_slice() + ); + assert_eq!( + key.scan.filters(), + [col("k0").eq(lit("foo")).to_string()].as_slice() + ); + } + + #[tokio::test] + async fn strips_time_only_filters_when_query_has_no_time_range_limit() { + let (stream_ctx, part_range) = new_stream_context( + vec![ + col("ts").gt_eq(ts_lit(1000)), + col("ts").is_not_null(), + col("k0").eq(lit("foo")), + ], + None, + ( + Timestamp::new_millisecond(1000), + Timestamp::new_millisecond(2000), + ), + ) + .await; + + let key = build_range_cache_key(&stream_ctx, &part_range).unwrap(); + + // Range-reducible time filters should be cleared when query has no time range limit. + assert!(key.scan.time_filters().is_empty()); + // Non-range time predicates stay in filters. + let mut expected_filters = [ + col("k0").eq(lit("foo")).to_string(), + col("ts").is_not_null().to_string(), + ]; + expected_filters.sort_unstable(); + assert_eq!(key.scan.filters(), expected_filters.as_slice()); + } #[test] fn normalizes_and_clears_time_filters() { @@ -249,4 +687,170 @@ mod tests { fingerprint.partition_expr_version ); } + + /// Creates a test schema with 5 columns where the primary key dictionary column + /// is at index 2 (`num_columns - 3`), matching the flat format layout. + /// + /// Layout: `[field0: Int64, field1: Int64, pk: Dictionary, ts: Int64, seq: Int64]` + fn dict_test_schema() -> Arc { + use datatypes::arrow::datatypes::{DataType as ArrowDataType, Field, Schema}; + Arc::new(Schema::new(vec![ + Field::new("field0", ArrowDataType::Int64, false), + Field::new("field1", ArrowDataType::Int64, false), + Field::new( + "pk", + ArrowDataType::Dictionary( + Box::new(ArrowDataType::UInt32), + Box::new(ArrowDataType::Binary), + ), + false, + ), + Field::new("ts", ArrowDataType::Int64, false), + Field::new("seq", ArrowDataType::Int64, false), + ])) + } + + /// Helper to create a record batch with a dictionary column at the primary key position. + fn make_dict_batch( + schema: Arc, + dict_values: &datatypes::arrow::array::BinaryArray, + keys: &[u32], + int_values: &[i64], + ) -> RecordBatch { + use datatypes::arrow::array::{Int64Array, UInt32Array}; + + let key_array = UInt32Array::from(keys.to_vec()); + let dict_array: DictionaryArray = + DictionaryArray::new(key_array, Arc::new(dict_values.clone())); + let int_array = Int64Array::from(int_values.to_vec()); + let zeros = Int64Array::from(vec![0i64; int_values.len()]); + RecordBatch::try_new( + schema, + vec![ + Arc::new(zeros.clone()), + Arc::new(int_array), + Arc::new(dict_array), + Arc::new(zeros.clone()), + Arc::new(zeros), + ], + ) + .unwrap() + } + + /// Computes the total `get_array_memory_size()` of all dictionary value arrays in a batch. + fn compute_total_dict_values_size(batch: &RecordBatch) -> usize { + batch + .columns() + .iter() + .filter_map(|col| { + col.as_any() + .downcast_ref::>() + .map(|dict| dict.values().get_array_memory_size()) + }) + .sum() + } + + #[test] + fn cache_batch_buffer_empty() { + let buffer = CacheBatchBuffer::new(); + assert_eq!(buffer.estimated_batches_size(), 0); + assert!(buffer.into_batches().is_empty()); + } + + #[test] + fn cache_batch_buffer_single_batch() { + use datatypes::arrow::array::BinaryArray; + + let schema = dict_test_schema(); + let dict_values = BinaryArray::from_vec(vec![b"a", b"b", b"c"]); + let batch = make_dict_batch(schema, &dict_values, &[0, 1, 2], &[10, 20, 30]); + + let full_size = batch.get_array_memory_size(); + + let mut buffer = CacheBatchBuffer::new(); + buffer.push(batch); + assert_eq!(buffer.estimated_batches_size(), full_size); + assert_eq!(buffer.into_batches().len(), 1); + } + + #[test] + fn cache_batch_buffer_shared_dictionary() { + use datatypes::arrow::array::BinaryArray; + + let schema = dict_test_schema(); + let dict_values = BinaryArray::from_vec(vec![b"alpha", b"beta", b"gamma"]); + + // Two batches sharing the same dictionary values array. + let batch1 = make_dict_batch(schema.clone(), &dict_values, &[0, 1], &[10, 20]); + let batch2 = make_dict_batch(schema, &dict_values, &[1, 2], &[30, 40]); + + let batch1_full = batch1.get_array_memory_size(); + let batch2_full = batch2.get_array_memory_size(); + + // The total dictionary values size that should be deduplicated for the second batch. + let dict_values_size = compute_total_dict_values_size(&batch2); + + let mut buffer = CacheBatchBuffer::new(); + buffer.push(batch1); + buffer.push(batch2); + + // Second batch's dict values should not be counted again. + assert_eq!( + buffer.estimated_batches_size(), + batch1_full + batch2_full - dict_values_size + ); + assert_eq!(buffer.into_batches().len(), 2); + } + + #[test] + fn cache_batch_buffer_non_shared_dictionary() { + use datatypes::arrow::array::BinaryArray; + + let schema = dict_test_schema(); + let dict_values1 = BinaryArray::from_vec(vec![b"a", b"b"]); + let dict_values2 = BinaryArray::from_vec(vec![b"x", b"y"]); + + let batch1 = make_dict_batch(schema.clone(), &dict_values1, &[0, 1], &[10, 20]); + let batch2 = make_dict_batch(schema, &dict_values2, &[0, 1], &[30, 40]); + + let batch1_full = batch1.get_array_memory_size(); + let batch2_full = batch2.get_array_memory_size(); + + let mut buffer = CacheBatchBuffer::new(); + buffer.push(batch1); + buffer.push(batch2); + + // Different dictionaries: full size for both. + assert_eq!(buffer.estimated_batches_size(), batch1_full + batch2_full); + } + + #[test] + fn cache_batch_buffer_shared_then_diverged() { + use datatypes::arrow::array::BinaryArray; + + let schema = dict_test_schema(); + let shared_values = BinaryArray::from_vec(vec![b"a", b"b", b"c"]); + let different_values = BinaryArray::from_vec(vec![b"x", b"y"]); + + let batch1 = make_dict_batch(schema.clone(), &shared_values, &[0], &[1]); + let batch2 = make_dict_batch(schema.clone(), &shared_values, &[1], &[2]); + let batch3 = make_dict_batch(schema, &different_values, &[0], &[3]); + + let size1 = batch1.get_array_memory_size(); + let size2 = batch2.get_array_memory_size(); + let size3 = batch3.get_array_memory_size(); + + let dict_values_size = compute_total_dict_values_size(&batch2); + + let mut buffer = CacheBatchBuffer::new(); + buffer.push(batch1); + buffer.push(batch2); + buffer.push(batch3); + + // batch2 shares dict with batch1 (dedup), batch3 does not (full size). + assert_eq!( + buffer.estimated_batches_size(), + size1 + (size2 - dict_values_size) + size3 + ); + } } diff --git a/src/mito2/src/read/scan_region.rs b/src/mito2/src/read/scan_region.rs index 5cb2d75e25..e7cae7e7b8 100644 --- a/src/mito2/src/read/scan_region.rs +++ b/src/mito2/src/read/scan_region.rs @@ -40,7 +40,7 @@ use store_api::region_engine::{PartitionRange, RegionScannerRef}; use store_api::storage::{ ColumnId, RegionId, ScanRequest, SequenceRange, TimeSeriesDistribution, TimeSeriesRowSelector, }; -use table::predicate::{Predicate, build_time_range_predicate}; +use table::predicate::{Predicate, build_time_range_predicate, extract_time_range_from_expr}; use tokio::sync::{Semaphore, mpsc}; use tokio_stream::wrappers::ReceiverStream; @@ -1420,7 +1420,6 @@ fn pre_filter_mode(append_mode: bool, merge_mode: MergeMode) -> PreFilterMode { /// Builds a [ScanRequestFingerprint] from a [ScanInput] if the scan is eligible /// for partition range caching. -#[cfg_attr(not(test), allow(dead_code))] pub(crate) fn build_scan_fingerprint(input: &ScanInput) -> Option { let eligible = input.flat_format && !input.compaction @@ -1439,7 +1438,14 @@ pub(crate) fn build_scan_fingerprint(input: &ScanInput) -> Option Option false, }; - if is_time_only { + if is_time_only + && extract_time_range_from_expr(&time_index_name, ts_col_unit, expr).is_some() + { + // Range-reducible time predicates can be safely dropped from the + // cache key when the query time range covers the partition range. time_filters.push(expr.to_string()); } else { + // Non-time filters and non-range time predicates (those that + // extract_time_range_from_expr cannot convert to a TimestampRange) + // always stay in the cache key. filters.push(expr.to_string()); } } @@ -1511,6 +1524,10 @@ pub struct StreamContext { pub input: ScanInput, /// Metadata for partition ranges. pub(crate) ranges: Vec, + /// Precomputed scan fingerprint for partition range caching. + /// `None` when the scan is not eligible for caching. + #[allow(dead_code)] + pub(crate) scan_fingerprint: Option, // Metrics: /// The start time of the query. @@ -1523,10 +1540,12 @@ impl StreamContext { let query_start = input.query_start.unwrap_or_else(Instant::now); let ranges = RangeMeta::seq_scan_ranges(&input); READ_SST_COUNT.observe(input.num_files() as f64); + let scan_fingerprint = build_scan_fingerprint(&input); Self { input, ranges, + scan_fingerprint, query_start, } } @@ -1536,10 +1555,12 @@ impl StreamContext { let query_start = input.query_start.unwrap_or_else(Instant::now); let ranges = RangeMeta::unordered_scan_ranges(&input); READ_SST_COUNT.observe(input.num_files() as f64); + let scan_fingerprint = build_scan_fingerprint(&input); Self { input, ranges, + scan_fingerprint, query_start, } } @@ -1849,6 +1870,7 @@ mod tests { use std::sync::Arc; use datafusion::physical_plan::expressions::lit as physical_lit; + use datafusion_common::ScalarValue; use datafusion_expr::{col, lit}; use datatypes::value::Value; use partition::expr::col as partition_col; @@ -2035,13 +2057,18 @@ mod tests { assert!(scan_region.use_flat_format()); } + /// Helper to create a timestamp millisecond literal. + fn ts_lit(val: i64) -> datafusion_expr::Expr { + lit(ScalarValue::TimestampMillisecond(Some(val), None)) + } + #[tokio::test] async fn test_build_scan_fingerprint_for_eligible_scan() { let metadata = Arc::new(metadata_with_primary_key(vec![0, 1], false)); let input = new_scan_input( metadata.clone(), vec![ - col("ts").gt_eq(lit(1000)), + col("ts").gt_eq(ts_lit(1000)), col("k0").eq(lit("foo")), col("v0").gt(lit(1)), ], @@ -2071,7 +2098,7 @@ mod tests { col("k0").eq(lit("foo")).to_string(), col("v0").gt(lit(1)).to_string(), ], - time_filters: vec![col("ts").gt_eq(lit(1000)).to_string()], + time_filters: vec![col("ts").gt_eq(ts_lit(1000)).to_string()], series_row_selector: Some(TimeSeriesRowSelector::LastRow), append_mode: false, filter_deleted: false, diff --git a/src/mito2/src/read/scan_util.rs b/src/mito2/src/read/scan_util.rs index 0ee6a4437d..6f68616709 100644 --- a/src/mito2/src/read/scan_util.rs +++ b/src/mito2/src/read/scan_util.rs @@ -247,6 +247,12 @@ pub(crate) struct ScanMetricsSet { num_range_builders: isize, /// Peak number of file range builders. num_peak_range_builders: isize, + /// Total bytes added to the range cache during this scan. + range_cache_size: usize, + /// Number of range cache hits during this scan. + range_cache_hit: usize, + /// Number of range cache misses during this scan. + range_cache_miss: usize, } /// Wrapper for file metrics that compares by total cost in reverse order. @@ -345,6 +351,9 @@ impl fmt::Debug for ScanMetricsSet { build_ranges_peak_mem_size, num_range_builders: _, num_peak_range_builders, + range_cache_size, + range_cache_hit, + range_cache_miss, } = self; // Write core metrics @@ -590,6 +599,16 @@ impl fmt::Debug for ScanMetricsSet { write!(f, "}}")?; } + if *range_cache_size > 0 { + write!(f, ", \"range_cache_size\":{range_cache_size}")?; + } + if *range_cache_hit > 0 { + write!(f, ", \"range_cache_hit\":{range_cache_hit}")?; + } + if *range_cache_miss > 0 { + write!(f, ", \"range_cache_miss\":{range_cache_miss}")?; + } + write!( f, ", \"build_ranges_peak_mem_size\":{build_ranges_peak_mem_size}, \ @@ -1097,6 +1116,27 @@ impl PartitionMetrics { pub(crate) fn dedup_metrics_reporter(&self) -> Arc { self.0.clone() } + + /// Increments the total bytes added to the range cache. + #[allow(dead_code)] + pub(crate) fn inc_range_cache_size(&self, size: usize) { + let mut metrics = self.0.metrics.lock().unwrap(); + metrics.range_cache_size += size; + } + + /// Increments the range cache hit counter. + #[allow(dead_code)] + pub(crate) fn inc_range_cache_hit(&self) { + let mut metrics = self.0.metrics.lock().unwrap(); + metrics.range_cache_hit += 1; + } + + /// Increments the range cache miss counter. + #[allow(dead_code)] + pub(crate) fn inc_range_cache_miss(&self) { + let mut metrics = self.0.metrics.lock().unwrap(); + metrics.range_cache_miss += 1; + } } impl fmt::Debug for PartitionMetrics { diff --git a/src/mito2/src/test_util.rs b/src/mito2/src/test_util.rs index 842689bba6..350195bfa9 100644 --- a/src/mito2/src/test_util.rs +++ b/src/mito2/src/test_util.rs @@ -15,6 +15,7 @@ //! Utilities for testing. pub mod batch_util; +pub mod bench_util; pub mod memtable_util; pub mod scheduler_util; pub mod sst_util; diff --git a/src/mito2/src/test_util/bench_util.rs b/src/mito2/src/test_util/bench_util.rs new file mode 100644 index 0000000000..8f182e4157 --- /dev/null +++ b/src/mito2/src/test_util/bench_util.rs @@ -0,0 +1,259 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Shared utilities for mito2 benchmarks. +//! +//! Provides a TSBS cpu-like data generator ([`CpuDataGenerator`]) and schema +//! ([`cpu_metadata`]) used by multiple benchmark binaries in this directory. + +use api::v1::value::ValueData; +use api::v1::{Row, Rows, SemanticType}; +use datafusion_common::Column; +use datafusion_expr::{Expr, lit}; +use datatypes::data_type::ConcreteDataType; +use datatypes::schema::ColumnSchema; +use rand::Rng; +use rand::rngs::ThreadRng; +use rand::seq::IndexedRandom; +use store_api::metadata::{ + ColumnMetadata, RegionMetadata, RegionMetadataBuilder, RegionMetadataRef, +}; +use store_api::storage::RegionId; +use table::predicate::Predicate; + +use crate::memtable::KeyValues; +use crate::test_util::memtable_util::region_metadata_to_row_schema; + +pub struct Host { + pub hostname: String, + pub region: String, + pub datacenter: String, + pub rack: String, + pub os: String, + pub arch: String, + pub team: String, + pub service: String, + pub service_version: String, + pub service_environment: String, +} + +impl Host { + pub fn random_with_id(id: usize) -> Host { + let mut rng = rand::rng(); + let region = format!("ap-southeast-{}", rng.random_range(0..10)); + let datacenter = format!( + "{}{}", + region, + ['a', 'b', 'c', 'd', 'e'].choose(&mut rng).unwrap() + ); + Host { + hostname: format!("host_{id}"), + region, + datacenter, + rack: rng.random_range(0..100).to_string(), + os: "Ubuntu16.04LTS".to_string(), + arch: "x86".to_string(), + team: "CHI".to_string(), + service: rng.random_range(0..100).to_string(), + service_version: rng.random_range(0..10).to_string(), + service_environment: "test".to_string(), + } + } + + pub fn fill_values(&self, values: &mut Vec) { + let tags = [ + api::v1::Value { + value_data: Some(ValueData::StringValue(self.hostname.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.region.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.datacenter.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.rack.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.os.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.arch.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.team.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.service.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.service_version.clone())), + }, + api::v1::Value { + value_data: Some(ValueData::StringValue(self.service_environment.clone())), + }, + ]; + for tag in tags { + values.push(tag); + } + } +} + +pub struct CpuDataGenerator { + pub metadata: RegionMetadataRef, + column_schemas: Vec, + hosts: Vec, + start_sec: i64, + end_sec: i64, +} + +impl CpuDataGenerator { + pub fn new( + metadata: RegionMetadataRef, + num_hosts: usize, + start_sec: i64, + end_sec: i64, + ) -> Self { + let column_schemas = region_metadata_to_row_schema(&metadata); + Self { + metadata, + column_schemas, + hosts: Self::generate_hosts(num_hosts), + start_sec, + end_sec, + } + } + + pub fn iter(&self) -> impl Iterator + '_ { + // point per 10s. + (self.start_sec..self.end_sec) + .step_by(10) + .enumerate() + .map(|(seq, ts)| self.build_key_values(seq, ts)) + } + + pub fn build_key_values(&self, seq: usize, current_sec: i64) -> KeyValues { + let rows = self + .hosts + .iter() + .map(|host| { + let mut rng = rand::rng(); + let mut values = Vec::with_capacity(21); + values.push(api::v1::Value { + value_data: Some(ValueData::TimestampMillisecondValue(current_sec * 1000)), + }); + host.fill_values(&mut values); + for _ in 0..10 { + values.push(api::v1::Value { + value_data: Some(ValueData::F64Value(Self::random_f64(&mut rng))), + }); + } + Row { values } + }) + .collect(); + let mutation = api::v1::Mutation { + op_type: api::v1::OpType::Put as i32, + sequence: seq as u64, + rows: Some(Rows { + schema: self.column_schemas.clone(), + rows, + }), + write_hint: None, + }; + + KeyValues::new(&self.metadata, mutation).unwrap() + } + + pub fn random_host_filter(&self) -> Predicate { + let host = self.random_hostname(); + let expr = Expr::Column(Column::from_name("hostname")).eq(lit(host)); + Predicate::new(vec![expr]) + } + + pub fn random_host_filter_exprs(&self) -> Vec { + let host = self.random_hostname(); + vec![Expr::Column(Column::from_name("hostname")).eq(lit(host))] + } + + pub fn random_hostname(&self) -> String { + let mut rng = rand::rng(); + self.hosts.choose(&mut rng).unwrap().hostname.clone() + } + + pub fn random_f64(rng: &mut ThreadRng) -> f64 { + let base: u32 = rng.random_range(30..95); + base as f64 + } + + pub fn generate_hosts(num_hosts: usize) -> Vec { + (0..num_hosts).map(Host::random_with_id).collect() + } +} + +/// Creates a metadata for TSBS cpu-like table. +pub fn cpu_metadata() -> RegionMetadata { + let mut builder = RegionMetadataBuilder::new(RegionId::new(1, 1)); + builder.push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new( + "ts", + ConcreteDataType::timestamp_millisecond_datatype(), + false, + ), + semantic_type: SemanticType::Timestamp, + column_id: 0, + }); + let mut column_id = 1; + let tags = [ + "hostname", + "region", + "datacenter", + "rack", + "os", + "arch", + "team", + "service", + "service_version", + "service_environment", + ]; + for tag in tags { + builder.push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new(tag, ConcreteDataType::string_datatype(), true), + semantic_type: SemanticType::Tag, + column_id, + }); + column_id += 1; + } + let fields = [ + "usage_user", + "usage_system", + "usage_idle", + "usage_nice", + "usage_iowait", + "usage_irq", + "usage_softirq", + "usage_steal", + "usage_guest", + "usage_guest_nice", + ]; + for field in fields { + builder.push_column_metadata(ColumnMetadata { + column_schema: ColumnSchema::new(field, ConcreteDataType::float64_datatype(), true), + semantic_type: SemanticType::Field, + column_id, + }); + column_id += 1; + } + builder.primary_key(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]); + builder.build().unwrap() +} diff --git a/src/mito2/src/test_util/memtable_util.rs b/src/mito2/src/test_util/memtable_util.rs index 8917875250..25ab9bb8b4 100644 --- a/src/mito2/src/test_util/memtable_util.rs +++ b/src/mito2/src/test_util/memtable_util.rs @@ -30,7 +30,7 @@ use mito_codec::row_converter::{DensePrimaryKeyCodec, PrimaryKeyCodecExt, SortFi use store_api::metadata::{ ColumnMetadata, RegionMetadata, RegionMetadataBuilder, RegionMetadataRef, }; -use store_api::storage::{ColumnId, RegionId, SequenceNumber, SequenceRange}; +use store_api::storage::{ColumnId, RegionId, SequenceNumber}; use crate::error::Result; use crate::memtable::bulk::part::BulkPart; diff --git a/src/mito2/src/worker.rs b/src/mito2/src/worker.rs index 71896b3d5d..fd5ad82f3f 100644 --- a/src/mito2/src/worker.rs +++ b/src/mito2/src/worker.rs @@ -207,6 +207,7 @@ impl WorkerGroup { .vector_cache_size(config.vector_cache_size.as_bytes()) .page_cache_size(config.page_cache_size.as_bytes()) .selector_result_cache_size(config.selector_result_cache_size.as_bytes()) + .range_result_cache_size(config.range_result_cache_size.as_bytes()) .index_metadata_size(config.index.metadata_cache_size.as_bytes()) .index_content_size(config.index.content_cache_size.as_bytes()) .index_content_page_size(config.index.content_cache_page_size.as_bytes()) @@ -421,6 +422,7 @@ impl WorkerGroup { .vector_cache_size(config.vector_cache_size.as_bytes()) .page_cache_size(config.page_cache_size.as_bytes()) .selector_result_cache_size(config.selector_result_cache_size.as_bytes()) + .range_result_cache_size(config.range_result_cache_size.as_bytes()) .write_cache(write_cache) .build(), ); diff --git a/src/table/src/predicate.rs b/src/table/src/predicate.rs index f9be7be16e..2c9ac41560 100644 --- a/src/table/src/predicate.rs +++ b/src/table/src/predicate.rs @@ -203,7 +203,7 @@ pub fn build_time_range_predicate( /// Extract time range filter from `WHERE`/`IN (...)`/`BETWEEN` clauses. /// Return None if no time range can be found in expr. -fn extract_time_range_from_expr( +pub fn extract_time_range_from_expr( ts_col_name: &str, ts_col_unit: TimeUnit, expr: &Expr, diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 65e56fa15e..7ae59ae9fc 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -1642,6 +1642,7 @@ fn drop_lines_with_inconsistent_results(input: String) -> String { "metadata_cache_size =", "content_cache_size =", "result_cache_size =", + "range_result_cache_size =", "name =", "recovery_parallelism =", "max_background_index_builds =", From c8c2e09eedd5a2f42acd599d76d4301e29abae53 Mon Sep 17 00:00:00 2001 From: shuiyisong <113876041+shuiyisong@users.noreply.github.com> Date: Tue, 24 Mar 2026 18:21:31 +0800 Subject: [PATCH 12/13] refactor: move election trait and implementations to the `common-meta` crate (#7820) * refactor: move election impl to common-meta Signed-off-by: shuiyisong * fix: adding back comment Signed-off-by: shuiyisong --------- Signed-off-by: shuiyisong --- src/{meta-srv => common/meta}/src/election.rs | 76 ++++++++++++++++++- .../meta}/src/election/etcd.rs | 14 ++-- .../meta}/src/election/rds.rs | 4 +- .../meta}/src/election/rds/mysql.rs | 42 +++++----- .../meta}/src/election/rds/postgres.rs | 45 ++++++----- src/common/meta/src/error.rs | 70 +++++++++++++++-- src/common/meta/src/lib.rs | 1 + src/meta-srv/src/bootstrap.rs | 21 ++--- src/meta-srv/src/cluster.rs | 4 +- src/meta-srv/src/lib.rs | 1 - src/meta-srv/src/metasrv.rs | 74 +----------------- src/meta-srv/src/service/admin/leader.rs | 2 +- src/meta-srv/src/service/cluster.rs | 5 +- src/meta-srv/src/service/heartbeat.rs | 4 +- 14 files changed, 218 insertions(+), 145 deletions(-) rename src/{meta-srv => common/meta}/src/election.rs (67%) rename src/{meta-srv => common/meta}/src/election/etcd.rs (94%) rename src/{meta-srv => common/meta}/src/election/rds.rs (96%) rename src/{meta-srv => common/meta}/src/election/rds/mysql.rs (97%) rename src/{meta-srv => common/meta}/src/election/rds/postgres.rs (97%) diff --git a/src/meta-srv/src/election.rs b/src/common/meta/src/election.rs similarity index 67% rename from src/meta-srv/src/election.rs rename to src/common/meta/src/election.rs index 2d2826b286..12173beda8 100644 --- a/src/meta-srv/src/election.rs +++ b/src/common/meta/src/election.rs @@ -21,15 +21,85 @@ use std::sync::Arc; use std::sync::atomic::{AtomicBool, Ordering}; use common_telemetry::{error, info, warn}; +use serde::{Deserialize, Serialize}; use tokio::sync::broadcast::error::RecvError; use tokio::sync::broadcast::{self, Receiver, Sender}; use crate::error::Result; -use crate::metasrv::MetasrvNodeInfo; -pub(crate) const CANDIDATE_LEASE_SECS: u64 = 600; +pub const CANDIDATE_LEASE_SECS: u64 = 600; const KEEP_ALIVE_INTERVAL_SECS: u64 = CANDIDATE_LEASE_SECS / 2; +/// The value of the leader. It is used to store the leader's address. +pub struct LeaderValue(pub String); + +impl> From for LeaderValue { + fn from(value: T) -> Self { + let string = String::from_utf8_lossy(value.as_ref()); + Self(string.to_string()) + } +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct MetasrvNodeInfo { + // The metasrv's address + pub addr: String, + // The node build version + pub version: String, + // The node build git commit hash + pub git_commit: String, + // The node start timestamp in milliseconds + pub start_time_ms: u64, + // The node total cpu millicores + #[serde(default)] + pub total_cpu_millicores: i64, + // The node total memory bytes + #[serde(default)] + pub total_memory_bytes: i64, + /// The node build cpu usage millicores + #[serde(default)] + pub cpu_usage_millicores: i64, + /// The node build memory usage bytes + #[serde(default)] + pub memory_usage_bytes: i64, + // The node hostname + #[serde(default)] + pub hostname: String, +} + +// TODO(zyy17): Allow deprecated fields for backward compatibility. Remove this when the deprecated top-level fields are removed from the proto. +#[allow(deprecated)] +impl From for api::v1::meta::MetasrvNodeInfo { + fn from(node_info: MetasrvNodeInfo) -> Self { + Self { + peer: Some(api::v1::meta::Peer { + addr: node_info.addr, + ..Default::default() + }), + // TODO(zyy17): The following top-level fields are deprecated. They are kept for backward compatibility and will be removed in a future version. + // New code should use the fields in `info.NodeInfo` instead. + version: node_info.version.clone(), + git_commit: node_info.git_commit.clone(), + start_time_ms: node_info.start_time_ms, + cpus: node_info.total_cpu_millicores as u32, + memory_bytes: node_info.total_memory_bytes as u64, + // The canonical location for node information. + info: Some(api::v1::meta::NodeInfo { + version: node_info.version, + git_commit: node_info.git_commit, + start_time_ms: node_info.start_time_ms, + total_cpu_millicores: node_info.total_cpu_millicores, + total_memory_bytes: node_info.total_memory_bytes, + cpu_usage_millicores: node_info.cpu_usage_millicores, + memory_usage_bytes: node_info.memory_usage_bytes, + cpus: node_info.total_cpu_millicores as u32, + memory_bytes: node_info.total_memory_bytes as u64, + hostname: node_info.hostname, + }), + } + } +} + /// Messages sent when the leader changes. #[derive(Debug, Clone)] pub enum LeaderChangeMessage { @@ -168,3 +238,5 @@ pub trait Election: Send + Sync { fn subscribe_leader_change(&self) -> Receiver; } + +pub type ElectionRef = Arc>; diff --git a/src/meta-srv/src/election/etcd.rs b/src/common/meta/src/election/etcd.rs similarity index 94% rename from src/meta-srv/src/election/etcd.rs rename to src/common/meta/src/election/etcd.rs index 883f723d74..affad31ef4 100644 --- a/src/meta-srv/src/election/etcd.rs +++ b/src/common/meta/src/election/etcd.rs @@ -16,8 +16,6 @@ use std::sync::Arc; use std::sync::atomic::{AtomicBool, Ordering}; use std::time::Duration; -use common_meta::distributed_time_constants::{META_KEEP_ALIVE_INTERVAL_SECS, META_LEASE_SECS}; -use common_meta::key::{CANDIDATES_ROOT, ELECTION_KEY}; use common_telemetry::{error, info, warn}; use etcd_client::{ Client, GetOptions, LeaderKey as EtcdLeaderKey, LeaseKeepAliveStream, LeaseKeeper, PutOptions, @@ -27,13 +25,15 @@ use tokio::sync::broadcast; use tokio::sync::broadcast::Receiver; use tokio::time::{MissedTickBehavior, timeout}; +use crate::distributed_time_constants::{META_KEEP_ALIVE_INTERVAL_SECS, META_LEASE_SECS}; use crate::election::{ - CANDIDATE_LEASE_SECS, Election, KEEP_ALIVE_INTERVAL_SECS, LeaderChangeMessage, LeaderKey, - listen_leader_change, send_leader_change_and_set_flags, + CANDIDATE_LEASE_SECS, Election, ElectionRef, KEEP_ALIVE_INTERVAL_SECS, LeaderChangeMessage, + LeaderKey, LeaderValue, MetasrvNodeInfo, listen_leader_change, + send_leader_change_and_set_flags, }; use crate::error; use crate::error::Result; -use crate::metasrv::{ElectionRef, LeaderValue, MetasrvNodeInfo}; +use crate::key::{CANDIDATES_ROOT, ELECTION_KEY}; impl LeaderKey for EtcdLeaderKey { fn name(&self) -> &[u8] { @@ -253,7 +253,7 @@ impl Election for EtcdElection { .leader(self.election_key()) .await .context(error::EtcdFailedSnafu)?; - let leader_value = res.kv().context(error::NoLeaderSnafu)?.value(); + let leader_value = res.kv().context(error::ElectionNoLeaderSnafu)?.value(); Ok(leader_value.into()) } } @@ -279,7 +279,7 @@ impl EtcdElection { ensure!( res.ttl() > 0, error::UnexpectedSnafu { - violated: "Failed to refresh the lease", + err_msg: "Failed to refresh the lease".to_string(), } ); diff --git a/src/meta-srv/src/election/rds.rs b/src/common/meta/src/election/rds.rs similarity index 96% rename from src/meta-srv/src/election/rds.rs rename to src/common/meta/src/election/rds.rs index 16e113415a..6ee529ee02 100644 --- a/src/meta-srv/src/election/rds.rs +++ b/src/common/meta/src/election/rds.rs @@ -36,7 +36,7 @@ fn parse_value_and_expire_time(value: &str) -> Result<(String, Timestamp)> { .split(LEASE_SEP) .collect_tuple() .with_context(|| UnexpectedSnafu { - violated: format!( + err_msg: format!( "Invalid value {}, expect node info || {} || expire time", value, LEASE_SEP ), @@ -45,7 +45,7 @@ fn parse_value_and_expire_time(value: &str) -> Result<(String, Timestamp)> { let expire_time = match Timestamp::from_str(expire_time, None) { Ok(ts) => ts, Err(_) => UnexpectedSnafu { - violated: format!("Invalid timestamp: {}", expire_time), + err_msg: format!("Invalid timestamp: {}", expire_time), } .fail()?, }; diff --git a/src/meta-srv/src/election/rds/mysql.rs b/src/common/meta/src/election/rds/mysql.rs similarity index 97% rename from src/meta-srv/src/election/rds/mysql.rs rename to src/common/meta/src/election/rds/mysql.rs index 20051a2610..80f3d8ca7c 100644 --- a/src/meta-srv/src/election/rds/mysql.rs +++ b/src/common/meta/src/election/rds/mysql.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use std::sync::atomic::{AtomicBool, Ordering}; use std::time::Duration; -use common_meta::key::{CANDIDATES_ROOT, ELECTION_KEY}; use common_telemetry::{error, info, warn}; use common_time::Timestamp; use snafu::{OptionExt, ResultExt, ensure}; @@ -29,14 +28,15 @@ use tokio::time::MissedTickBehavior; use crate::election::rds::{LEASE_SEP, Lease, RdsLeaderKey, parse_value_and_expire_time}; use crate::election::{ - Election, LeaderChangeMessage, listen_leader_change, send_leader_change_and_set_flags, + Election, ElectionRef, LeaderChangeMessage, LeaderValue, MetasrvNodeInfo, listen_leader_change, + send_leader_change_and_set_flags, }; use crate::error::{ AcquireMySqlClientSnafu, DecodeSqlValueSnafu, DeserializeFromJsonSnafu, - LeaderLeaseChangedSnafu, LeaderLeaseExpiredSnafu, MySqlExecutionSnafu, NoLeaderSnafu, Result, - SerializeToJsonSnafu, SqlExecutionTimeoutSnafu, UnexpectedSnafu, + ElectionLeaderLeaseChangedSnafu, ElectionLeaderLeaseExpiredSnafu, ElectionNoLeaderSnafu, + MySqlExecutionSnafu, Result, SerializeToJsonSnafu, SqlExecutionTimeoutSnafu, UnexpectedSnafu, }; -use crate::metasrv::{ElectionRef, LeaderValue, MetasrvNodeInfo}; +use crate::key::{CANDIDATES_ROOT, ELECTION_KEY}; struct ElectionSqlFactory<'a> { table_name: &'a str, @@ -592,7 +592,7 @@ impl Election for MySqlElection { ensure!( lease.expire_time > lease.current, UnexpectedSnafu { - violated: format!( + err_msg: format!( "Candidate lease expired at {:?} (current time: {:?}), key: {:?}", lease.expire_time, lease.current, @@ -667,10 +667,10 @@ impl Election for MySqlElection { let client = self.client.lock().await; let mut executor = Executor::Default(client); if let Some(lease) = self.get_value_with_lease(&key, &mut executor).await? { - ensure!(lease.expire_time > lease.current, NoLeaderSnafu); + ensure!(lease.expire_time > lease.current, ElectionNoLeaderSnafu); Ok(lease.leader_value.as_bytes().into()) } else { - NoLeaderSnafu.fail() + ElectionNoLeaderSnafu.fail() } } } @@ -705,7 +705,7 @@ impl MySqlElection { let current_time = match Timestamp::from_str(¤t_time_str, None) { Ok(ts) => ts, Err(_) => UnexpectedSnafu { - violated: format!("Invalid timestamp: {}", current_time_str), + err_msg: format!("Invalid timestamp: {}", current_time_str), } .fail()?, }; @@ -740,7 +740,7 @@ impl MySqlElection { current = match Timestamp::from_str(current_time_str, None) { Ok(ts) => ts, Err(_) => UnexpectedSnafu { - violated: format!("Invalid timestamp: {}", current_time_str), + err_msg: format!("Invalid timestamp: {}", current_time_str), } .fail()?, }; @@ -777,7 +777,7 @@ impl MySqlElection { ensure!( res == 1, UnexpectedSnafu { - violated: format!("Failed to update key: {}", String::from_utf8_lossy(key)), + err_msg: format!("Failed to update key: {}", String::from_utf8_lossy(key)), } ); @@ -920,9 +920,12 @@ impl MySqlElection { /// will be released. /// - **Case 2**: If all checks pass, the function returns without performing any actions. fn lease_check(&self, lease: &Option) -> Result { - let lease = lease.as_ref().context(NoLeaderSnafu)?; + let lease = lease.as_ref().context(ElectionNoLeaderSnafu)?; // Case 1: Lease expired - ensure!(lease.expire_time > lease.current, LeaderLeaseExpiredSnafu); + ensure!( + lease.expire_time > lease.current, + ElectionLeaderLeaseExpiredSnafu + ); // Case 2: Everything is fine Ok(lease.clone()) } @@ -960,7 +963,7 @@ impl MySqlElection { let remote_lease = self.get_value_with_lease(&key, &mut executor).await?; ensure!( expected_lease.map(|lease| lease.origin) == remote_lease.map(|lease| lease.origin), - LeaderLeaseChangedSnafu + ElectionLeaderLeaseChangedSnafu ); self.delete_value(&key, &mut executor).await?; self.put_value_with_lease( @@ -987,12 +990,11 @@ mod tests { use std::assert_matches::assert_matches; use std::env; - use common_meta::maybe_skip_mysql_integration_test; use common_telemetry::init_default_ut_logging; + use sqlx::MySqlPool; use super::*; - use crate::error; - use crate::utils::mysql::create_mysql_pool; + use crate::{error, maybe_skip_mysql_integration_test}; async fn create_mysql_client( table_name: Option<&str>, @@ -1003,11 +1005,11 @@ mod tests { let endpoint = env::var("GT_MYSQL_ENDPOINTS").unwrap_or_default(); if endpoint.is_empty() { return UnexpectedSnafu { - violated: "MySQL endpoint is empty".to_string(), + err_msg: "MySQL endpoint is empty".to_string(), } .fail(); } - let pool = create_mysql_pool(&[endpoint], None).await.unwrap(); + let pool = MySqlPool::connect(&endpoint).await.unwrap(); let mut client = ElectionMysqlClient::new( pool, execution_timeout, @@ -1302,7 +1304,7 @@ mod tests { let err = elected(&leader_mysql_election, table_name, Some(incorrect_lease)) .await .unwrap_err(); - assert_matches!(err, error::Error::LeaderLeaseChanged { .. }); + assert_matches!(err, error::Error::ElectionLeaderLeaseChanged { .. }); let lease = get_lease(&leader_mysql_election).await; assert!(lease.is_none()); drop_table(&leader_mysql_election.client, table_name).await; diff --git a/src/meta-srv/src/election/rds/postgres.rs b/src/common/meta/src/election/rds/postgres.rs similarity index 97% rename from src/meta-srv/src/election/rds/postgres.rs rename to src/common/meta/src/election/rds/postgres.rs index c21efd780b..01910335a0 100644 --- a/src/meta-srv/src/election/rds/postgres.rs +++ b/src/common/meta/src/election/rds/postgres.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use std::sync::atomic::{AtomicBool, Ordering}; use std::time::Duration; -use common_meta::key::{CANDIDATES_ROOT, ELECTION_KEY}; use common_telemetry::{error, info, warn}; use common_time::Timestamp; use deadpool_postgres::{Manager, Pool}; @@ -28,13 +27,15 @@ use tokio_postgres::types::ToSql; use crate::election::rds::{LEASE_SEP, Lease, RdsLeaderKey, parse_value_and_expire_time}; use crate::election::{ - Election, LeaderChangeMessage, listen_leader_change, send_leader_change_and_set_flags, + Election, ElectionRef, LeaderChangeMessage, LeaderValue, MetasrvNodeInfo, listen_leader_change, + send_leader_change_and_set_flags, }; use crate::error::{ - DeserializeFromJsonSnafu, GetPostgresClientSnafu, NoLeaderSnafu, PostgresExecutionSnafu, - Result, SerializeToJsonSnafu, SqlExecutionTimeoutSnafu, UnexpectedSnafu, + DeserializeFromJsonSnafu, ElectionNoLeaderSnafu, GetPostgresClientSnafu, + PostgresExecutionSnafu, Result, SerializeToJsonSnafu, SqlExecutionTimeoutSnafu, + UnexpectedSnafu, }; -use crate::metasrv::{ElectionRef, LeaderValue, MetasrvNodeInfo}; +use crate::key::{CANDIDATES_ROOT, ELECTION_KEY}; struct ElectionSqlFactory<'a> { lock_id: u64, @@ -404,13 +405,13 @@ impl Election for PgElection { .get_value_with_lease(&key) .await? .context(UnexpectedSnafu { - violated: format!("Failed to get lease for key: {:?}", key), + err_msg: format!("Failed to get lease for key: {:?}", key), })?; ensure!( lease.expire_time > lease.current, UnexpectedSnafu { - violated: format!( + err_msg: format!( "Candidate lease expired at {:?} (current time {:?}), key: {:?}", lease.expire_time, lease.current, key ), @@ -464,11 +465,11 @@ impl Election for PgElection { .query(&self.sql_set.campaign, &[]) .await?; let row = res.first().context(UnexpectedSnafu { - violated: "Failed to get the result of acquiring advisory lock", + err_msg: "Failed to get the result of acquiring advisory lock".to_string(), })?; let is_leader = row.try_get(0).map_err(|_| { UnexpectedSnafu { - violated: "Failed to get the result of get lock", + err_msg: "Failed to get the result of get lock".to_string(), } .build() })?; @@ -500,10 +501,10 @@ impl Election for PgElection { } else { let key = self.election_key(); if let Some(lease) = self.get_value_with_lease(&key).await? { - ensure!(lease.expire_time > lease.current, NoLeaderSnafu); + ensure!(lease.expire_time > lease.current, ElectionNoLeaderSnafu); Ok(lease.leader_value.as_bytes().into()) } else { - NoLeaderSnafu.fail() + ElectionNoLeaderSnafu.fail() } } } @@ -537,7 +538,7 @@ impl PgElection { let current_time = match Timestamp::from_str(current_time_str, None) { Ok(ts) => ts, Err(_) => UnexpectedSnafu { - violated: format!("Invalid timestamp: {}", current_time_str), + err_msg: format!("Invalid timestamp: {}", current_time_str), } .fail()?, }; @@ -576,7 +577,7 @@ impl PgElection { current = match Timestamp::from_str(current_time_str, None) { Ok(ts) => ts, Err(_) => UnexpectedSnafu { - violated: format!("Invalid timestamp: {}", current_time_str), + err_msg: format!("Invalid timestamp: {}", current_time_str), } .fail()?, }; @@ -613,7 +614,7 @@ impl PgElection { ensure!( res == 1, UnexpectedSnafu { - violated: format!("Failed to update key: {}", String::from_utf8_lossy(key)), + err_msg: format!("Failed to update key: {}", String::from_utf8_lossy(key)), } ); @@ -742,9 +743,9 @@ impl PgElection { let lease = self .get_value_with_lease(&key) .await? - .context(NoLeaderSnafu)?; + .context(ElectionNoLeaderSnafu)?; // Case 2 - ensure!(lease.expire_time > lease.current, NoLeaderSnafu); + ensure!(lease.expire_time > lease.current, ElectionNoLeaderSnafu); // Case 3 Ok(()) } @@ -831,11 +832,11 @@ mod tests { use std::assert_matches::assert_matches; use std::env; - use common_meta::maybe_skip_postgres_integration_test; + use deadpool_postgres::{Config, Runtime}; + use tokio_postgres::NoTls; use super::*; - use crate::error; - use crate::utils::postgres::create_postgres_pool; + use crate::{error, maybe_skip_postgres_integration_test}; async fn create_postgres_client( table_name: Option<&str>, @@ -846,11 +847,13 @@ mod tests { let endpoint = env::var("GT_POSTGRES_ENDPOINTS").unwrap_or_default(); if endpoint.is_empty() { return UnexpectedSnafu { - violated: "Postgres endpoint is empty".to_string(), + err_msg: "Postgres endpoint is empty".to_string(), } .fail(); } - let pool = create_postgres_pool(&[endpoint], None, None).await.unwrap(); + let mut cfg = Config::new(); + cfg.url = Some(endpoint); + let pool = cfg.create_pool(Some(Runtime::Tokio1), NoTls).unwrap(); let mut pg_client = ElectionPgClient::new( pool, execution_timeout, diff --git a/src/common/meta/src/error.rs b/src/common/meta/src/error.rs index b9fcbd6188..05b5af393b 100644 --- a/src/common/meta/src/error.rs +++ b/src/common/meta/src/error.rs @@ -338,6 +338,24 @@ pub enum Error { location: Location, }, + #[snafu(display("Metasrv election has no leader at this moment"))] + ElectionNoLeader { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Metasrv election leader lease expired"))] + ElectionLeaderLeaseExpired { + #[snafu(implicit)] + location: Location, + }, + + #[snafu(display("Metasrv election leader lease changed during election"))] + ElectionLeaderLeaseChanged { + #[snafu(implicit)] + location: Location, + }, + #[snafu(display("Table already exists, table: {}", table_name))] TableAlreadyExists { table_name: String, @@ -751,6 +769,15 @@ pub enum Error { location: Location, }, + #[cfg(feature = "pg_kvbackend")] + #[snafu(display("Failed to get Postgres client"))] + GetPostgresClient { + #[snafu(source)] + error: deadpool::managed::PoolError, + #[snafu(implicit)] + location: Location, + }, + #[cfg(feature = "pg_kvbackend")] #[snafu(display("Failed to {} Postgres transaction", operation))] PostgresTransaction { @@ -805,6 +832,24 @@ pub enum Error { location: Location, }, + #[cfg(feature = "mysql_kvbackend")] + #[snafu(display("Failed to decode sql value"))] + DecodeSqlValue { + #[snafu(source)] + error: sqlx::error::Error, + #[snafu(implicit)] + location: Location, + }, + + #[cfg(feature = "mysql_kvbackend")] + #[snafu(display("Failed to acquire mysql client from pool"))] + AcquireMySqlClient { + #[snafu(source)] + error: sqlx::Error, + #[snafu(implicit)] + location: Location, + }, + #[cfg(feature = "mysql_kvbackend")] #[snafu(display("Failed to {} MySql transaction", operation))] MySqlTransaction { @@ -822,6 +867,15 @@ pub enum Error { location: Location, }, + #[cfg(any(feature = "pg_kvbackend", feature = "mysql_kvbackend"))] + #[snafu(display("Sql execution timeout, sql: {}, duration: {:?}", sql, duration))] + SqlExecutionTimeout { + sql: String, + duration: std::time::Duration, + #[snafu(implicit)] + location: Location, + }, + #[snafu(display( "Datanode table info not found, table id: {}, datanode id: {}", table_id, @@ -1075,7 +1129,10 @@ impl ErrorExt for Error { | GetCache { .. } | GetLatestCacheRetryExceeded { .. } | SerializeToJson { .. } - | DeserializeFromJson { .. } => StatusCode::Internal, + | DeserializeFromJson { .. } + | ElectionNoLeader { .. } + | ElectionLeaderLeaseExpired { .. } + | ElectionLeaderLeaseChanged { .. } => StatusCode::Internal, NoLeader { .. } => StatusCode::TableUnavailable, ValueNotExist { .. } @@ -1198,15 +1255,18 @@ impl ErrorExt for Error { PostgresExecution { .. } | CreatePostgresPool { .. } | GetPostgresConnection { .. } + | GetPostgresClient { .. } | PostgresTransaction { .. } | PostgresTlsConfig { .. } | InvalidTlsConfig { .. } => StatusCode::Internal, #[cfg(feature = "mysql_kvbackend")] - MySqlExecution { .. } | CreateMySqlPool { .. } | MySqlTransaction { .. } => { - StatusCode::Internal - } + MySqlExecution { .. } + | CreateMySqlPool { .. } + | DecodeSqlValue { .. } + | AcquireMySqlClient { .. } + | MySqlTransaction { .. } => StatusCode::Internal, #[cfg(any(feature = "pg_kvbackend", feature = "mysql_kvbackend"))] - RdsTransactionRetryFailed { .. } => StatusCode::Internal, + RdsTransactionRetryFailed { .. } | SqlExecutionTimeout { .. } => StatusCode::Internal, DatanodeTableInfoNotFound { .. } => StatusCode::Internal, } } diff --git a/src/common/meta/src/lib.rs b/src/common/meta/src/lib.rs index 93cd229b16..36aae1026e 100644 --- a/src/common/meta/src/lib.rs +++ b/src/common/meta/src/lib.rs @@ -22,6 +22,7 @@ pub mod datanode; pub mod ddl; pub mod ddl_manager; pub mod distributed_time_constants; +pub mod election; pub mod error; pub mod flow_name; pub mod heartbeat; diff --git a/src/meta-srv/src/bootstrap.rs b/src/meta-srv/src/bootstrap.rs index 2cfe7d2f7d..eadb7cdc75 100644 --- a/src/meta-srv/src/bootstrap.rs +++ b/src/meta-srv/src/bootstrap.rs @@ -24,6 +24,8 @@ use common_base::Plugins; use common_config::Configurable; #[cfg(any(feature = "pg_kvbackend", feature = "mysql_kvbackend"))] use common_meta::distributed_time_constants::META_LEASE_SECS; +use common_meta::election::CANDIDATE_LEASE_SECS; +use common_meta::election::etcd::EtcdElection; use common_meta::kv_backend::chroot::ChrootKvBackend; use common_meta::kv_backend::etcd::EtcdStore; use common_meta::kv_backend::memory::MemoryKvBackend; @@ -42,9 +44,6 @@ use tonic::codec::CompressionEncoding; use tonic::transport::server::{Router, TcpIncoming}; use crate::cluster::{MetaPeerClientBuilder, MetaPeerClientRef}; -#[cfg(any(feature = "pg_kvbackend", feature = "mysql_kvbackend"))] -use crate::election::CANDIDATE_LEASE_SECS; -use crate::election::etcd::EtcdElection; use crate::error::OtherSnafu; use crate::metasrv::builder::MetasrvBuilder; use crate::metasrv::{ @@ -281,7 +280,8 @@ pub async fn metasrv_builder( etcd_client, opts.store_key_prefix.clone(), ) - .await?; + .await + .context(error::KvBackendSnafu)?; (kv_backend, Some(election)) } @@ -290,10 +290,10 @@ pub async fn metasrv_builder( use std::time::Duration; use common_meta::distributed_time_constants::POSTGRES_KEEP_ALIVE_SECS; + use common_meta::election::rds::postgres::{ElectionPgClient, PgElection}; use common_meta::kv_backend::rds::PgStore; use deadpool_postgres::{Config, ManagerConfig, RecyclingMethod}; - use crate::election::rds::postgres::{ElectionPgClient, PgElection}; use crate::utils::postgres::create_postgres_pool; let candidate_lease_ttl = Duration::from_secs(CANDIDATE_LEASE_SECS); @@ -321,7 +321,8 @@ pub async fn metasrv_builder( execution_timeout, idle_session_timeout, statement_timeout, - )?; + ) + .context(error::KvBackendSnafu)?; let election = PgElection::with_pg_client( opts.grpc.server_addr.clone(), election_client, @@ -332,7 +333,8 @@ pub async fn metasrv_builder( &opts.meta_table_name, opts.meta_election_lock_id, ) - .await?; + .await + .context(error::KvBackendSnafu)?; let pool = create_postgres_pool(&opts.store_addrs, Some(cfg), opts.backend_tls.clone()) .await?; @@ -352,9 +354,9 @@ pub async fn metasrv_builder( (None, BackendImpl::MysqlStore) => { use std::time::Duration; + use common_meta::election::rds::mysql::{ElectionMysqlClient, MySqlElection}; use common_meta::kv_backend::rds::MySqlStore; - use crate::election::rds::mysql::{ElectionMysqlClient, MySqlElection}; use crate::utils::mysql::create_mysql_pool; let pool = create_mysql_pool(&opts.store_addrs, opts.backend_tls.as_ref()).await?; @@ -389,7 +391,8 @@ pub async fn metasrv_builder( meta_lease_ttl, &election_table_name, ) - .await?; + .await + .context(error::KvBackendSnafu)?; (kv_backend, Some(election)) } }; diff --git a/src/meta-srv/src/cluster.rs b/src/meta-srv/src/cluster.rs index 35b15b3b29..ef3ba07702 100644 --- a/src/meta-srv/src/cluster.rs +++ b/src/meta-srv/src/cluster.rs @@ -247,7 +247,7 @@ impl MetaPeerClient { // Safety: when self.is_leader() == false, election must not empty. let election = self.election.as_ref().unwrap(); - let leader_addr = election.leader().await?.0; + let leader_addr = election.leader().await.context(error::KvBackendSnafu)?.0; let channel = self .channel_manager @@ -279,7 +279,7 @@ impl MetaPeerClient { // Safety: when self.is_leader() == false, election must not empty. let election = self.election.as_ref().unwrap(); - let leader_addr = election.leader().await?.0; + let leader_addr = election.leader().await.context(error::KvBackendSnafu)?.0; let channel = self .channel_manager diff --git a/src/meta-srv/src/lib.rs b/src/meta-srv/src/lib.rs index c67bc32b40..0e87d4421a 100644 --- a/src/meta-srv/src/lib.rs +++ b/src/meta-srv/src/lib.rs @@ -21,7 +21,6 @@ pub mod bootstrap; pub mod cache_invalidator; pub mod cluster; pub mod discovery; -pub mod election; pub mod error; pub mod events; mod failure_detector; diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index 165efd0555..a1515d897e 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -32,6 +32,8 @@ use common_meta::ddl_manager::DdlManagerRef; use common_meta::distributed_time_constants::{ self, BASE_HEARTBEAT_INTERVAL, default_distributed_time_constants, frontend_heartbeat_interval, }; +use common_meta::election::LeaderChangeMessage; +pub use common_meta::election::{ElectionRef, MetasrvNodeInfo}; use common_meta::key::TableMetadataManagerRef; use common_meta::key::runtime_switch::RuntimeSwitchManagerRef; use common_meta::kv_backend::{KvBackendRef, ResettableKvBackend, ResettableKvBackendRef}; @@ -64,7 +66,6 @@ use tokio::sync::broadcast::error::RecvError; use crate::cluster::MetaPeerClientRef; use crate::discovery; -use crate::election::{Election, LeaderChangeMessage}; use crate::error::{ self, InitMetadataSnafu, KvBackendSnafu, Result, StartProcedureManagerSnafu, StartTelemetryTaskSnafu, StopProcedureManagerSnafu, @@ -459,76 +460,6 @@ impl Context { } } -/// The value of the leader. It is used to store the leader's address. -pub struct LeaderValue(pub String); - -impl> From for LeaderValue { - fn from(value: T) -> Self { - let string = String::from_utf8_lossy(value.as_ref()); - Self(string.to_string()) - } -} - -#[derive(Debug, Clone, Serialize, Deserialize)] -pub struct MetasrvNodeInfo { - // The metasrv's address - pub addr: String, - // The node build version - pub version: String, - // The node build git commit hash - pub git_commit: String, - // The node start timestamp in milliseconds - pub start_time_ms: u64, - // The node total cpu millicores - #[serde(default)] - pub total_cpu_millicores: i64, - // The node total memory bytes - #[serde(default)] - pub total_memory_bytes: i64, - /// The node build cpu usage millicores - #[serde(default)] - pub cpu_usage_millicores: i64, - /// The node build memory usage bytes - #[serde(default)] - pub memory_usage_bytes: i64, - // The node hostname - #[serde(default)] - pub hostname: String, -} - -// TODO(zyy17): Allow deprecated fields for backward compatibility. Remove this when the deprecated top-level fields are removed from the proto. -#[allow(deprecated)] -impl From for api::v1::meta::MetasrvNodeInfo { - fn from(node_info: MetasrvNodeInfo) -> Self { - Self { - peer: Some(api::v1::meta::Peer { - addr: node_info.addr, - ..Default::default() - }), - // TODO(zyy17): The following top-level fields are deprecated. They are kept for backward compatibility and will be removed in a future version. - // New code should use the fields in `info.NodeInfo` instead. - version: node_info.version.clone(), - git_commit: node_info.git_commit.clone(), - start_time_ms: node_info.start_time_ms, - cpus: node_info.total_cpu_millicores as u32, - memory_bytes: node_info.total_memory_bytes as u64, - // The canonical location for node information. - info: Some(api::v1::meta::NodeInfo { - version: node_info.version, - git_commit: node_info.git_commit, - start_time_ms: node_info.start_time_ms, - total_cpu_millicores: node_info.total_cpu_millicores, - total_memory_bytes: node_info.total_memory_bytes, - cpu_usage_millicores: node_info.cpu_usage_millicores, - memory_usage_bytes: node_info.memory_usage_bytes, - cpus: node_info.total_cpu_millicores as u32, - memory_bytes: node_info.total_memory_bytes as u64, - hostname: node_info.hostname, - }), - } - } -} - #[derive(Clone, Copy)] pub enum SelectTarget { Datanode, @@ -552,7 +483,6 @@ pub struct SelectorContext { pub type SelectorRef = Arc>>; pub type RegionStatAwareSelectorRef = Arc>>; -pub type ElectionRef = Arc>; pub struct MetaStateHandler { subscribe_manager: Option, diff --git a/src/meta-srv/src/service/admin/leader.rs b/src/meta-srv/src/service/admin/leader.rs index 1fadb4a3ef..17329e7b47 100644 --- a/src/meta-srv/src/service/admin/leader.rs +++ b/src/meta-srv/src/service/admin/leader.rs @@ -32,7 +32,7 @@ pub struct LeaderHandler { impl LeaderHandler { async fn get_leader(&self) -> Result> { if let Some(election) = &self.election { - let leader_addr = election.leader().await?.0; + let leader_addr = election.leader().await.context(error::KvBackendSnafu)?.0; return Ok(Some(leader_addr)); } Ok(None) diff --git a/src/meta-srv/src/service/cluster.rs b/src/meta-srv/src/service/cluster.rs index 5c0ae4c71f..366a8aa5fb 100644 --- a/src/meta-srv/src/service/cluster.rs +++ b/src/meta-srv/src/service/cluster.rs @@ -63,7 +63,10 @@ impl cluster_server::Cluster for Metasrv { let leader_addr = &self.options().grpc.server_addr; let (leader, followers) = match self.election() { Some(election) => { - let nodes = election.all_candidates().await?; + let nodes = election + .all_candidates() + .await + .context(error::KvBackendSnafu)?; let followers = nodes .into_iter() .filter(|node_info| &node_info.addr != leader_addr) diff --git a/src/meta-srv/src/service/heartbeat.rs b/src/meta-srv/src/service/heartbeat.rs index e09073546a..238ed99df2 100644 --- a/src/meta-srv/src/service/heartbeat.rs +++ b/src/meta-srv/src/service/heartbeat.rs @@ -23,7 +23,7 @@ use api::v1::meta::{ use common_telemetry::{debug, error, info, warn}; use futures::StreamExt; use once_cell::sync::OnceCell; -use snafu::OptionExt; +use snafu::{OptionExt, ResultExt}; use tokio::sync::mpsc; use tokio::sync::mpsc::Sender; use tokio_stream::wrappers::ReceiverStream; @@ -148,7 +148,7 @@ async fn handle_ask_leader(_req: AskLeaderRequest, ctx: Context) -> Result ctx.server_addr, From 13cdfa9b59a2d2b7a1f166c3993f50597807a368 Mon Sep 17 00:00:00 2001 From: Ning Sun Date: Tue, 24 Mar 2026 20:16:38 +0800 Subject: [PATCH 13/13] fix: update 8-bit int to smallint in postgres (#7854) --- src/servers/src/postgres/types.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/servers/src/postgres/types.rs b/src/servers/src/postgres/types.rs index a95890e78c..d4d15ef64a 100644 --- a/src/servers/src/postgres/types.rs +++ b/src/servers/src/postgres/types.rs @@ -235,7 +235,7 @@ pub(super) fn type_gt_to_pg(origin: &ConcreteDataType) -> Result { match origin { &ConcreteDataType::Null(_) => Ok(Type::UNKNOWN), &ConcreteDataType::Boolean(_) => Ok(Type::BOOL), - &ConcreteDataType::Int8(_) => Ok(Type::CHAR), + &ConcreteDataType::Int8(_) => Ok(Type::INT2), &ConcreteDataType::Int16(_) | &ConcreteDataType::UInt8(_) => Ok(Type::INT2), &ConcreteDataType::Int32(_) | &ConcreteDataType::UInt16(_) => Ok(Type::INT4), &ConcreteDataType::Int64(_) | &ConcreteDataType::UInt32(_) => Ok(Type::INT8), @@ -253,7 +253,7 @@ pub(super) fn type_gt_to_pg(origin: &ConcreteDataType) -> Result { ConcreteDataType::List(list) => match list.item_type() { &ConcreteDataType::Null(_) => Ok(Type::UNKNOWN), &ConcreteDataType::Boolean(_) => Ok(Type::BOOL_ARRAY), - &ConcreteDataType::Int8(_) => Ok(Type::CHAR_ARRAY), + &ConcreteDataType::Int8(_) => Ok(Type::INT2_ARRAY), &ConcreteDataType::Int16(_) | &ConcreteDataType::UInt8(_) => Ok(Type::INT2_ARRAY), &ConcreteDataType::Int32(_) | &ConcreteDataType::UInt16(_) => Ok(Type::INT4_ARRAY), &ConcreteDataType::Int64(_) | &ConcreteDataType::UInt32(_) => Ok(Type::INT8_ARRAY), @@ -1151,7 +1151,7 @@ mod test { let pg_field_info = vec![ FieldInfo::new("nulls".into(), None, None, Type::UNKNOWN, FieldFormat::Text), FieldInfo::new("bools".into(), None, None, Type::BOOL, FieldFormat::Text), - FieldInfo::new("int8s".into(), None, None, Type::CHAR, FieldFormat::Text), + FieldInfo::new("int8s".into(), None, None, Type::INT2, FieldFormat::Text), FieldInfo::new("int16s".into(), None, None, Type::INT2, FieldFormat::Text), FieldInfo::new("int32s".into(), None, None, Type::INT4, FieldFormat::Text), FieldInfo::new("int64s".into(), None, None, Type::INT8, FieldFormat::Text), @@ -1230,7 +1230,7 @@ mod test { Type::NUMERIC, FieldFormat::Text, ), - FieldInfo::new("int8s".into(), None, None, Type::CHAR, FieldFormat::Text), + FieldInfo::new("int8s".into(), None, None, Type::INT2, FieldFormat::Text), FieldInfo::new("int16s".into(), None, None, Type::INT2, FieldFormat::Text), FieldInfo::new("int32s".into(), None, None, Type::INT4, FieldFormat::Text), FieldInfo::new("int64s".into(), None, None, Type::INT8, FieldFormat::Text),