From 9ea6d1226737da77db6a1c67b7296367e751bdf5 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Mon, 13 Oct 2025 16:51:28 +0800 Subject: [PATCH 01/19] feat: query mem limiter --- Cargo.lock | 1 + config/datanode.example.toml | 12 +++ config/flownode.example.toml | 8 ++ config/standalone.example.toml | 12 +++ src/cmd/tests/load_config_test.rs | 2 + src/common/recordbatch/src/error.rs | 10 ++ src/common/recordbatch/src/lib.rs | 114 ++++++++++++++++++++++ src/common/recordbatch/src/recordbatch.rs | 11 +++ src/flow/src/adapter.rs | 2 + src/mito2/src/config.rs | 3 + src/mito2/src/engine.rs | 23 ++++- src/query/Cargo.toml | 1 + src/query/src/options.rs | 5 + src/query/src/query_engine/state.rs | 15 ++- 14 files changed, 214 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index aec4ad866aff..17bba1a7bad8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10127,6 +10127,7 @@ dependencies = [ "chrono", "common-base", "common-catalog", + "common-config", "common-datasource", "common-error", "common-function", diff --git a/config/datanode.example.toml b/config/datanode.example.toml index e283967680d5..82ce35b022c8 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -256,6 +256,14 @@ overwrite_entry_start_id = false ## Default to 0, which means the number of CPU cores. parallelism = 0 +## Memory pool size for query execution operators (aggregation, sorting, join). +## Setting it to 0 disables the limit (unbounded, default behavior). +## When this limit is reached, queries will fail with ResourceExhausted error. +## NOTE: This does NOT limit memory used by table scans. +## Example: "2GB", "4GB" +## @toml2docs:none-default +#+ memory_pool_size = "2GB" + ## The data storage options. [storage] ## The working home directory. @@ -496,6 +504,10 @@ max_concurrent_scan_files = 384 ## Whether to allow stale WAL entries read during replay. allow_stale_entries = false +## Memory limit for table scans across all queries. Setting it to 0 disables the limit. +## @toml2docs:none-default +#+ scan_memory_limit = "2GB" + ## Minimum time interval between two compactions. ## To align with the old behavior, the default value is 0 (no restrictions). min_compaction_interval = "0m" diff --git a/config/flownode.example.toml b/config/flownode.example.toml index 81ff25f28366..008bf5039c0a 100644 --- a/config/flownode.example.toml +++ b/config/flownode.example.toml @@ -158,6 +158,14 @@ default_ratio = 1.0 ## Default to 1, so it won't use too much cpu or memory parallelism = 1 +## Memory pool size for query execution operators (aggregation, sorting, join). +## Setting it to 0 disables the limit (unbounded, default behavior). +## When this limit is reached, queries will fail with ResourceExhausted error. +## NOTE: This does NOT limit memory used by table scans. +## Example: "1GB", "2GB" +## @toml2docs:none-default +#+ memory_pool_size = "1GB" + ## The memory options. [memory] ## Whether to enable heap profiling activation during startup. diff --git a/config/standalone.example.toml b/config/standalone.example.toml index 5fae0f444fda..07e0f74bb8cf 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -353,6 +353,14 @@ max_running_procedures = 128 ## Default to 0, which means the number of CPU cores. parallelism = 0 +## Memory pool size for query execution operators (aggregation, sorting, join). +## Setting it to 0 disables the limit (unbounded, default behavior). +## When this limit is reached, queries will fail with ResourceExhausted error. +## NOTE: This does NOT limit memory used by table scans. +## Example: "2GB", "4GB" +## @toml2docs:none-default +#+ memory_pool_size = "2GB" + ## The data storage options. [storage] ## The working home directory. @@ -580,6 +588,10 @@ max_concurrent_scan_files = 384 ## Whether to allow stale WAL entries read during replay. allow_stale_entries = false +## Memory limit for table scans across all queries. Setting it to 0 disables the limit. +## @toml2docs:none-default +#+ scan_memory_limit = "2GB" + ## Minimum time interval between two compactions. ## To align with the old behavior, the default value is 0 (no restrictions). min_compaction_interval = "0m" diff --git a/src/cmd/tests/load_config_test.rs b/src/cmd/tests/load_config_test.rs index b92cf9631d46..9ebc3e55d747 100644 --- a/src/cmd/tests/load_config_test.rs +++ b/src/cmd/tests/load_config_test.rs @@ -15,6 +15,7 @@ use std::time::Duration; use cmd::options::GreptimeOptions; +use common_base::readable_size::ReadableSize; use common_config::{Configurable, DEFAULT_DATA_HOME}; use common_options::datanode::{ClientOptions, DatanodeClientOptions}; use common_telemetry::logging::{DEFAULT_LOGGING_DIR, DEFAULT_OTLP_HTTP_ENDPOINT, LoggingOptions}; @@ -240,6 +241,7 @@ fn test_load_flownode_example_config() { query: QueryOptions { parallelism: 1, allow_query_fallback: false, + memory_pool_size: ReadableSize(0), }, meta_client: Some(MetaClientOptions { metasrv_addrs: vec!["127.0.0.1:3002".to_string()], diff --git a/src/common/recordbatch/src/error.rs b/src/common/recordbatch/src/error.rs index e07d152d2dc4..5e628a739771 100644 --- a/src/common/recordbatch/src/error.rs +++ b/src/common/recordbatch/src/error.rs @@ -193,6 +193,14 @@ pub enum Error { #[snafu(implicit)] location: Location, }, + + #[snafu(display("Exceeded memory limit: {} bytes used, {} bytes limit", used, limit))] + ExceedMemoryLimit { + used: usize, + limit: usize, + #[snafu(implicit)] + location: Location, + }, } impl ErrorExt for Error { @@ -229,6 +237,8 @@ impl ErrorExt for Error { Error::StreamTimeout { .. } => StatusCode::Cancelled, Error::StreamCancelled { .. } => StatusCode::Cancelled, + + Error::ExceedMemoryLimit { .. } => StatusCode::RuntimeResourcesExhausted, } } diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index 7ae4a419d6fa..d807593ae8d7 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -23,6 +23,7 @@ pub mod util; use std::pin::Pin; use std::sync::Arc; +use std::sync::atomic::{AtomicUsize, Ordering}; use adapter::RecordBatchMetrics; use arc_swap::ArcSwapOption; @@ -406,6 +407,119 @@ impl> + Unpin> Stream for RecordBatchStream } } +/// Memory tracker for RecordBatch streams. Clone to share the same limit across queries. +#[derive(Debug, Clone)] +pub struct QueryMemoryTracker { + current: Arc, + limit: usize, +} + +impl QueryMemoryTracker { + /// Create a new memory tracker with the given limit (in bytes). + /// If limit is 0, no limit is enforced. + pub fn new(limit: usize) -> Self { + Self { + current: Arc::new(AtomicUsize::new(0)), + limit, + } + } + + /// Track memory usage. Returns error if limit is exceeded. + pub fn track(&self, size: usize) -> Result<()> { + if self.limit == 0 { + return Ok(()); + } + + let new_total = self.current.fetch_add(size, Ordering::Relaxed) + size; + if new_total > self.limit { + self.release(size); + return error::ExceedMemoryLimitSnafu { + used: new_total, + limit: self.limit, + } + .fail(); + } + Ok(()) + } + + /// Release tracked memory. + pub fn release(&self, size: usize) { + if self.limit == 0 { + return; + } + + self.current + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { + Some(current.saturating_sub(size)) + }) + .ok(); + } + + /// Get current memory usage. + pub fn current(&self) -> usize { + self.current.load(Ordering::Relaxed) + } +} + +/// A wrapper stream that tracks memory usage of RecordBatches. +pub struct MemoryTrackedStream { + inner: SendableRecordBatchStream, + tracker: QueryMemoryTracker, + total_tracked: usize, +} + +impl MemoryTrackedStream { + pub fn new(inner: SendableRecordBatchStream, tracker: QueryMemoryTracker) -> Self { + Self { + inner, + tracker, + total_tracked: 0, + } + } +} + +impl Stream for MemoryTrackedStream { + type Item = Result; + + fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + match Pin::new(&mut self.inner).poll_next(cx) { + Poll::Ready(Some(Ok(batch))) => { + let size = batch.estimated_size(); + + if let Err(e) = self.tracker.track(size) { + return Poll::Ready(Some(Err(e))); + } + + self.total_tracked += size; + Poll::Ready(Some(Ok(batch))) + } + Poll::Ready(Some(Err(e))) => Poll::Ready(Some(Err(e))), + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + } + } +} + +impl Drop for MemoryTrackedStream { + fn drop(&mut self) { + self.tracker.release(self.total_tracked); + } +} + +impl RecordBatchStream for MemoryTrackedStream { + fn schema(&self) -> SchemaRef { + self.inner.schema() + } + + fn output_ordering(&self) -> Option<&[OrderOption]> { + self.inner.output_ordering() + } + + fn metrics(&self) -> Option { + self.inner.metrics() + } +} + #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/src/common/recordbatch/src/recordbatch.rs b/src/common/recordbatch/src/recordbatch.rs index 3cc30ce1ba78..3664af93eb18 100644 --- a/src/common/recordbatch/src/recordbatch.rs +++ b/src/common/recordbatch/src/recordbatch.rs @@ -241,6 +241,17 @@ impl RecordBatch { .unwrap_or("failed to pretty display a record batch".to_string()) } + /// Estimate the memory size of this record batch in bytes. + /// This uses the slice memory size which represents the actual memory used. + pub fn estimated_size(&self) -> usize { + self.df_record_batch + .columns() + .iter() + // If cannot get slice memory size, assume 0 + .map(|c| c.to_data().get_slice_memory_size().unwrap_or(0)) + .sum() + } + /// Return a slice record batch starts from offset, with len rows pub fn slice(&self, offset: usize, len: usize) -> Result { ensure!( diff --git a/src/flow/src/adapter.rs b/src/flow/src/adapter.rs index 9721d490405f..7f3c4f404f96 100644 --- a/src/flow/src/adapter.rs +++ b/src/flow/src/adapter.rs @@ -21,6 +21,7 @@ use std::sync::Arc; use std::time::{Duration, Instant, SystemTime}; use api::v1::{RowDeleteRequest, RowDeleteRequests, RowInsertRequest, RowInsertRequests}; +use common_base::readable_size::ReadableSize; use common_config::Configurable; use common_error::ext::BoxedError; use common_meta::key::TableMetadataManagerRef; @@ -132,6 +133,7 @@ impl Default for FlownodeOptions { query: QueryOptions { parallelism: 1, allow_query_fallback: false, + memory_pool_size: ReadableSize(0), }, user_provider: None, memory: MemoryOptions::default(), diff --git a/src/mito2/src/config.rs b/src/mito2/src/config.rs index edf070996020..934a0a132c1f 100644 --- a/src/mito2/src/config.rs +++ b/src/mito2/src/config.rs @@ -127,6 +127,8 @@ pub struct MitoConfig { pub max_concurrent_scan_files: usize, /// Whether to allow stale entries read during replay. pub allow_stale_entries: bool, + /// Memory limit for table scans across all queries. Setting it to 0 disables the limit. + pub scan_memory_limit: ReadableSize, /// Index configs. pub index: IndexConfig, @@ -179,6 +181,7 @@ impl Default for MitoConfig { parallel_scan_channel_size: DEFAULT_SCAN_CHANNEL_SIZE, max_concurrent_scan_files: DEFAULT_MAX_CONCURRENT_SCAN_FILES, allow_stale_entries: false, + scan_memory_limit: ReadableSize(0), index: IndexConfig::default(), inverted_index: InvertedIndexConfig::default(), fulltext_index: FulltextIndexConfig::default(), diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index d48f75fa91a8..cb645ffc31c2 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -79,7 +79,7 @@ use async_trait::async_trait; use common_base::Plugins; use common_error::ext::BoxedError; use common_meta::key::SchemaMetadataManagerRef; -use common_recordbatch::SendableRecordBatchStream; +use common_recordbatch::{MemoryTrackedStream, QueryMemoryTracker, SendableRecordBatchStream}; use common_telemetry::{info, tracing, warn}; use common_wal::options::{WAL_OPTIONS_KEY, WalOptions}; use futures::future::{join_all, try_join_all}; @@ -197,10 +197,13 @@ impl<'a, S: LogStore> MitoEngineBuilder<'a, S> { ) .await?; let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(self.log_store)); + let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize); + let inner = EngineInner { workers, config, wal_raw_entry_reader, + scan_memory_tracker, #[cfg(feature = "enterprise")] extension_range_provider_factory: None, }; @@ -251,6 +254,10 @@ impl MitoEngine { &self.inner.config } + pub fn scan_memory_tracker(&self) -> QueryMemoryTracker { + self.inner.scan_memory_tracker.clone() + } + pub fn cache_manager(&self) -> CacheManagerRef { self.inner.workers.cache_manager() } @@ -291,11 +298,17 @@ impl MitoEngine { region_id: RegionId, request: ScanRequest, ) -> Result { - self.scanner(region_id, request) + let stream = self + .scanner(region_id, request) .await .map_err(BoxedError::new)? .scan() - .await + .await?; + + Ok(Box::pin(MemoryTrackedStream::new( + stream, + self.inner.scan_memory_tracker.clone(), + ))) } /// Scan [`Batch`]es by [`ScanRequest`]. @@ -573,6 +586,8 @@ struct EngineInner { config: Arc, /// The Wal raw entry reader. wal_raw_entry_reader: Arc, + /// Memory tracker for table scans. + scan_memory_tracker: QueryMemoryTracker, #[cfg(feature = "enterprise")] extension_range_provider_factory: Option, } @@ -1069,6 +1084,7 @@ impl MitoEngine { let config = Arc::new(config); let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(log_store.clone())); + let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize); Ok(MitoEngine { inner: Arc::new(EngineInner { workers: WorkerGroup::start_for_test( @@ -1085,6 +1101,7 @@ impl MitoEngine { .await?, config, wal_raw_entry_reader, + scan_memory_tracker, #[cfg(feature = "enterprise")] extension_range_provider_factory: None, }), diff --git a/src/query/Cargo.toml b/src/query/Cargo.toml index 0c6c2e033f9a..e92eabb142fd 100644 --- a/src/query/Cargo.toml +++ b/src/query/Cargo.toml @@ -24,6 +24,7 @@ catalog.workspace = true chrono.workspace = true common-base.workspace = true common-catalog.workspace = true +common-config.workspace = true common-datasource.workspace = true common-error.workspace = true common-function.workspace = true diff --git a/src/query/src/options.rs b/src/query/src/options.rs index 25e1a0a2a054..f4a5b9aa8d7c 100644 --- a/src/query/src/options.rs +++ b/src/query/src/options.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use common_base::readable_size::ReadableSize; use serde::{Deserialize, Serialize}; /// Query engine config @@ -22,6 +23,9 @@ pub struct QueryOptions { pub parallelism: usize, /// Whether to allow query fallback when push down fails. pub allow_query_fallback: bool, + /// Memory pool size for query execution. Setting it to 0 disables the limit (unbounded). + /// When this limit is reached, queries will fail with ResourceExhausted error. + pub memory_pool_size: ReadableSize, } #[allow(clippy::derivable_impls)] @@ -30,6 +34,7 @@ impl Default for QueryOptions { Self { parallelism: 0, allow_query_fallback: false, + memory_pool_size: ReadableSize(0), } } } diff --git a/src/query/src/query_engine/state.rs b/src/query/src/query_engine/state.rs index 7b53f385e874..ff947a979fb8 100644 --- a/src/query/src/query_engine/state.rs +++ b/src/query/src/query_engine/state.rs @@ -31,7 +31,8 @@ use datafusion::dataframe::DataFrame; use datafusion::error::Result as DfResult; use datafusion::execution::SessionStateBuilder; use datafusion::execution::context::{QueryPlanner, SessionConfig, SessionContext, SessionState}; -use datafusion::execution::runtime_env::RuntimeEnv; +use datafusion::execution::memory_pool::GreedyMemoryPool; +use datafusion::execution::runtime_env::{RuntimeEnv, RuntimeEnvBuilder}; use datafusion::physical_optimizer::PhysicalOptimizerRule; use datafusion::physical_optimizer::optimizer::PhysicalOptimizer; use datafusion::physical_optimizer::sanity_checker::SanityCheckPlan; @@ -100,7 +101,17 @@ impl QueryEngineState { plugins: Plugins, options: QueryOptionsNew, ) -> Self { - let runtime_env = Arc::new(RuntimeEnv::default()); + let memory_pool_size = options.memory_pool_size.as_bytes() as usize; + let runtime_env = if memory_pool_size > 0 { + Arc::new( + RuntimeEnvBuilder::new() + .with_memory_pool(Arc::new(GreedyMemoryPool::new(memory_pool_size))) + .build() + .expect("Failed to build RuntimeEnv"), + ) + } else { + Arc::new(RuntimeEnv::default()) + }; let mut session_config = SessionConfig::new().with_create_default_catalog_and_schema(false); if options.parallelism > 0 { session_config = session_config.with_target_partitions(options.parallelism); From bc8e4ab4ea2ca1371b5ebff3a6347bba563705f8 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Mon, 13 Oct 2025 17:08:24 +0800 Subject: [PATCH 02/19] feat: config docs --- config/config.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/config/config.md b/config/config.md index 92f192e6df6a..5306f752ee10 100644 --- a/config/config.md +++ b/config/config.md @@ -101,6 +101,7 @@ | `flow.num_workers` | Integer | `0` | The number of flow worker in flownode.
Not setting(or set to 0) this value will use the number of CPU cores divided by 2. | | `query` | -- | -- | The query engine options. | | `query.parallelism` | Integer | `0` | Parallelism of the query engine.
Default to 0, which means the number of CPU cores. | +| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans.
Example: "2GB", "4GB" | | `storage` | -- | -- | The data storage options. | | `storage.data_home` | String | `./greptimedb_data` | The working home directory. | | `storage.type` | String | `File` | The storage type used to store the data.
- `File`: the data is stored in the local file system.
- `S3`: the data is stored in the S3 object storage.
- `Gcs`: the data is stored in the Google Cloud Storage.
- `Azblob`: the data is stored in the Azure Blob Storage.
- `Oss`: the data is stored in the Aliyun OSS. | @@ -152,6 +153,7 @@ | `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. | | `region_engine.mito.max_concurrent_scan_files` | Integer | `384` | Maximum number of SST files to scan concurrently. | | `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. | +| `region_engine.mito.scan_memory_limit` | String | Unset | Memory limit for table scans across all queries. Setting it to 0 disables the limit. | | `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.
To align with the old behavior, the default value is 0 (no restrictions). | | `region_engine.mito.default_experimental_flat_format` | Bool | `false` | Whether to enable experimental flat format as the default format. | | `region_engine.mito.index` | -- | -- | The options for index in Mito engine. | @@ -493,6 +495,7 @@ | `wal.overwrite_entry_start_id` | Bool | `false` | Ignore missing entries during read WAL.
**It's only used when the provider is `kafka`**.

This option ensures that when Kafka messages are deleted, the system
can still successfully replay memtable data without throwing an
out-of-range error.
However, enabling this option might lead to unexpected data loss,
as the system will skip over missing entries instead of treating
them as critical errors. | | `query` | -- | -- | The query engine options. | | `query.parallelism` | Integer | `0` | Parallelism of the query engine.
Default to 0, which means the number of CPU cores. | +| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans.
Example: "2GB", "4GB" | | `storage` | -- | -- | The data storage options. | | `storage.data_home` | String | `./greptimedb_data` | The working home directory. | | `storage.type` | String | `File` | The storage type used to store the data.
- `File`: the data is stored in the local file system.
- `S3`: the data is stored in the S3 object storage.
- `Gcs`: the data is stored in the Google Cloud Storage.
- `Azblob`: the data is stored in the Azure Blob Storage.
- `Oss`: the data is stored in the Aliyun OSS. | @@ -546,6 +549,7 @@ | `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. | | `region_engine.mito.max_concurrent_scan_files` | Integer | `384` | Maximum number of SST files to scan concurrently. | | `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. | +| `region_engine.mito.scan_memory_limit` | String | Unset | Memory limit for table scans across all queries. Setting it to 0 disables the limit. | | `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.
To align with the old behavior, the default value is 0 (no restrictions). | | `region_engine.mito.default_experimental_flat_format` | Bool | `false` | Whether to enable experimental flat format as the default format. | | `region_engine.mito.index` | -- | -- | The options for index in Mito engine. | @@ -666,5 +670,6 @@ | `tracing.tokio_console_addr` | String | Unset | The tokio console address. | | `query` | -- | -- | -- | | `query.parallelism` | Integer | `1` | Parallelism of the query engine for query sent by flownode.
Default to 1, so it won't use too much cpu or memory | +| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans.
Example: "1GB", "2GB" | | `memory` | -- | -- | The memory options. | | `memory.enable_heap_profiling` | Bool | `true` | Whether to enable heap profiling activation during startup.
When enabled, heap profiling will be activated if the `MALLOC_CONF` environment variable
is set to "prof:true,prof_active:false". The official image adds this env variable.
Default is true. | From 0105d4270a896116a5efc220ab9721c29375e969 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Mon, 13 Oct 2025 18:21:39 +0800 Subject: [PATCH 03/19] feat: frontend query limit config --- config/config.md | 1 + config/frontend.example.toml | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/config/config.md b/config/config.md index 5306f752ee10..84e4f5be6e75 100644 --- a/config/config.md +++ b/config/config.md @@ -304,6 +304,7 @@ | `query` | -- | -- | The query engine options. | | `query.parallelism` | Integer | `0` | Parallelism of the query engine.
Default to 0, which means the number of CPU cores. | | `query.allow_query_fallback` | Bool | `false` | Whether to allow query fallback when push down optimize fails.
Default to false, meaning when push down optimize failed, return error msg | +| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans (only applies to datanodes).
Example: "4GB", "8GB" | | `datanode` | -- | -- | Datanode options. | | `datanode.client` | -- | -- | Datanode client options. | | `datanode.client.connect_timeout` | String | `10s` | -- | diff --git a/config/frontend.example.toml b/config/frontend.example.toml index b26d88323e49..9977d1102131 100644 --- a/config/frontend.example.toml +++ b/config/frontend.example.toml @@ -244,6 +244,14 @@ parallelism = 0 ## Default to false, meaning when push down optimize failed, return error msg allow_query_fallback = false +## Memory pool size for query execution operators (aggregation, sorting, join). +## Setting it to 0 disables the limit (unbounded, default behavior). +## When this limit is reached, queries will fail with ResourceExhausted error. +## NOTE: This does NOT limit memory used by table scans (only applies to datanodes). +## Example: "4GB", "8GB" +## @toml2docs:none-default +#+ memory_pool_size = "4GB" + ## Datanode options. [datanode] ## Datanode client options. From 6c4fb8dbcb6033d35cdad79b34bfa87e579420db Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Tue, 14 Oct 2025 10:40:14 +0800 Subject: [PATCH 04/19] fix: unused imports Signed-off-by: jeremyhi --- Cargo.lock | 1 - src/query/Cargo.toml | 1 - 2 files changed, 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 17bba1a7bad8..aec4ad866aff 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10127,7 +10127,6 @@ dependencies = [ "chrono", "common-base", "common-catalog", - "common-config", "common-datasource", "common-error", "common-function", diff --git a/src/query/Cargo.toml b/src/query/Cargo.toml index e92eabb142fd..0c6c2e033f9a 100644 --- a/src/query/Cargo.toml +++ b/src/query/Cargo.toml @@ -24,7 +24,6 @@ catalog.workspace = true chrono.workspace = true common-base.workspace = true common-catalog.workspace = true -common-config.workspace = true common-datasource.workspace = true common-error.workspace = true common-function.workspace = true From fa0380fdd5e68909609b8b2e05ac13e18fb4f399 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Wed, 15 Oct 2025 15:44:02 +0800 Subject: [PATCH 05/19] feat: add metrics for query memory tracker Signed-off-by: jeremyhi --- src/common/recordbatch/src/lib.rs | 86 ++++++++++++++++++++++++----- src/mito2/src/engine.rs | 20 ++++++- src/mito2/src/lib.rs | 1 + src/mito2/src/metrics.rs | 10 ++++ src/query/src/metrics.rs | 12 ++++ src/query/src/query_engine/state.rs | 70 ++++++++++++++++++++++- tests-integration/tests/http.rs | 2 + 7 files changed, 181 insertions(+), 20 deletions(-) diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index d807593ae8d7..d28b478ccd65 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -21,6 +21,7 @@ pub mod filter; mod recordbatch; pub mod util; +use std::fmt; use std::pin::Pin; use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; @@ -408,10 +409,23 @@ impl> + Unpin> Stream for RecordBatchStream } /// Memory tracker for RecordBatch streams. Clone to share the same limit across queries. -#[derive(Debug, Clone)] +#[derive(Clone)] pub struct QueryMemoryTracker { current: Arc, limit: usize, + on_update: Option>, + on_reject: Option>, +} + +impl fmt::Debug for QueryMemoryTracker { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("QueryMemoryTracker") + .field("current", &self.current.load(Ordering::Relaxed)) + .field("limit", &self.limit) + .field("on_update", &self.on_update.is_some()) + .field("on_reject", &self.on_reject.is_some()) + .finish() + } } impl QueryMemoryTracker { @@ -421,43 +435,85 @@ impl QueryMemoryTracker { Self { current: Arc::new(AtomicUsize::new(0)), limit, + on_update: None, + on_reject: None, } } + /// Set a callback to be called whenever the usage changes successfully. + /// The callback receives the new total usage in bytes. + /// + /// # Note + /// The callback is called after both successful `track()` and `release()` operations. + /// It is called even when `limit == 0` (unlimited mode) to track actual usage. + pub fn with_update_callback(mut self, callback: F) -> Self + where + F: Fn(usize) + Send + Sync + 'static, + { + self.on_update = Some(Arc::new(callback)); + self + } + + /// Set a callback to be called when memory allocation is rejected. + /// + /// # Note + /// This is only called when `track()` fails due to exceeding the limit. + /// It is never called when `limit == 0` (unlimited mode). + pub fn with_reject_callback(mut self, callback: F) -> Self + where + F: Fn() + Send + Sync + 'static, + { + self.on_reject = Some(Arc::new(callback)); + self + } + /// Track memory usage. Returns error if limit is exceeded. pub fn track(&self, size: usize) -> Result<()> { + let new_total = self.current.fetch_add(size, Ordering::Relaxed) + size; + if self.limit == 0 { + // Unlimited mode: still track usage but never reject + if let Some(callback) = &self.on_update { + callback(new_total); + } return Ok(()); } - let new_total = self.current.fetch_add(size, Ordering::Relaxed) + size; if new_total > self.limit { - self.release(size); + self.release_internal(size, false); + if let Some(callback) = &self.on_reject { + callback(); + } return error::ExceedMemoryLimitSnafu { used: new_total, limit: self.limit, } .fail(); } + + if let Some(callback) = &self.on_update { + callback(new_total); + } + Ok(()) } /// Release tracked memory. pub fn release(&self, size: usize) { - if self.limit == 0 { - return; - } - - self.current - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { - Some(current.saturating_sub(size)) - }) - .ok(); + self.release_internal(size, true); } - /// Get current memory usage. - pub fn current(&self) -> usize { - self.current.load(Ordering::Relaxed) + fn release_internal(&self, size: usize, trigger_callback: bool) { + if let Ok(old_value) = + self.current + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { + Some(current.saturating_sub(size)) + }) + && trigger_callback + && let Some(callback) = &self.on_update + { + callback(old_value.saturating_sub(size)); + } } } diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index cb645ffc31c2..b03caad346d3 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -115,7 +115,9 @@ use crate::error::{ use crate::extension::BoxedExtensionRangeProviderFactory; use crate::manifest::action::RegionEdit; use crate::memtable::MemtableStats; -use crate::metrics::HANDLE_REQUEST_ELAPSED; +use crate::metrics::{ + HANDLE_REQUEST_ELAPSED, SCAN_MEMORY_USAGE_BYTES, SCAN_REQUESTS_REJECTED_TOTAL, +}; use crate::read::scan_region::{ScanRegion, Scanner}; use crate::read::stream::ScanBatchStream; use crate::region::MitoRegionRef; @@ -197,7 +199,13 @@ impl<'a, S: LogStore> MitoEngineBuilder<'a, S> { ) .await?; let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(self.log_store)); - let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize); + let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize) + .with_update_callback(|usage| { + SCAN_MEMORY_USAGE_BYTES.set(usage as i64); + }) + .with_reject_callback(|| { + SCAN_REQUESTS_REJECTED_TOTAL.inc(); + }); let inner = EngineInner { workers, @@ -1084,7 +1092,13 @@ impl MitoEngine { let config = Arc::new(config); let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(log_store.clone())); - let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize); + let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize) + .with_update_callback(|usage| { + SCAN_MEMORY_USAGE_BYTES.set(usage as i64); + }) + .with_reject_callback(|| { + SCAN_REQUESTS_REJECTED_TOTAL.inc(); + }); Ok(MitoEngine { inner: Arc::new(EngineInner { workers: WorkerGroup::start_for_test( diff --git a/src/mito2/src/lib.rs b/src/mito2/src/lib.rs index 45ce635148d4..5ddc8d0b14af 100644 --- a/src/mito2/src/lib.rs +++ b/src/mito2/src/lib.rs @@ -18,6 +18,7 @@ #![feature(assert_matches)] #![feature(int_roundings)] +#![recursion_limit = "256"] #![feature(debug_closure_helpers)] #![feature(duration_constructors)] diff --git a/src/mito2/src/metrics.rs b/src/mito2/src/metrics.rs index 0f923f60a6eb..fdcbf18e494d 100644 --- a/src/mito2/src/metrics.rs +++ b/src/mito2/src/metrics.rs @@ -207,6 +207,16 @@ lazy_static! { "Number of rows returned in a scan task", exponential_buckets(100.0, 10.0, 7).unwrap(), ).unwrap(); + /// Gauge for scan memory usage in bytes. + pub static ref SCAN_MEMORY_USAGE_BYTES: IntGauge = register_int_gauge!( + "greptime_mito_scan_memory_usage_bytes", + "current scan memory usage in bytes" + ).unwrap(); + /// Counter of rejected scan requests due to memory limit. + pub static ref SCAN_REQUESTS_REJECTED_TOTAL: IntCounter = register_int_counter!( + "greptime_mito_scan_requests_rejected_total", + "total number of scan requests rejected due to memory limit" + ).unwrap(); // ------- End of query metrics. // Cache related metrics. diff --git a/src/query/src/metrics.rs b/src/query/src/metrics.rs index 290f368a8fdd..e0d02e9a3d1f 100644 --- a/src/query/src/metrics.rs +++ b/src/query/src/metrics.rs @@ -62,6 +62,18 @@ lazy_static! { "query push down fallback errors total" ) .unwrap(); + + pub static ref QUERY_MEMORY_POOL_USAGE_BYTES: IntGauge = register_int_gauge!( + "greptime_query_memory_pool_usage_bytes", + "current query memory pool usage in bytes" + ) + .unwrap(); + + pub static ref QUERY_MEMORY_POOL_REJECTED_TOTAL: IntCounter = register_int_counter!( + "greptime_query_memory_pool_rejected_total", + "total number of query memory allocations rejected" + ) + .unwrap(); } /// A stream to call the callback once a RecordBatch stream is done. diff --git a/src/query/src/query_engine/state.rs b/src/query/src/query_engine/state.rs index ff947a979fb8..e83d37504a4a 100644 --- a/src/query/src/query_engine/state.rs +++ b/src/query/src/query_engine/state.rs @@ -31,7 +31,9 @@ use datafusion::dataframe::DataFrame; use datafusion::error::Result as DfResult; use datafusion::execution::SessionStateBuilder; use datafusion::execution::context::{QueryPlanner, SessionConfig, SessionContext, SessionState}; -use datafusion::execution::memory_pool::GreedyMemoryPool; +use datafusion::execution::memory_pool::{ + GreedyMemoryPool, MemoryConsumer, MemoryLimit, MemoryPool, MemoryReservation, +}; use datafusion::execution::runtime_env::{RuntimeEnv, RuntimeEnvBuilder}; use datafusion::physical_optimizer::PhysicalOptimizerRule; use datafusion::physical_optimizer::optimizer::PhysicalOptimizer; @@ -50,6 +52,7 @@ use crate::QueryEngineContext; use crate::dist_plan::{ DistExtensionPlanner, DistPlannerAnalyzer, DistPlannerOptions, MergeSortExtensionPlanner, }; +use crate::metrics::{QUERY_MEMORY_POOL_REJECTED_TOTAL, QUERY_MEMORY_POOL_USAGE_BYTES}; use crate::optimizer::ExtensionAnalyzerRule; use crate::optimizer::constant_term::MatchesConstantTermOptimizer; use crate::optimizer::count_wildcard::CountWildcardToTimeIndexRule; @@ -105,7 +108,7 @@ impl QueryEngineState { let runtime_env = if memory_pool_size > 0 { Arc::new( RuntimeEnvBuilder::new() - .with_memory_pool(Arc::new(GreedyMemoryPool::new(memory_pool_size))) + .with_memory_pool(Arc::new(MetricsMemoryPool::new(memory_pool_size))) .build() .expect("Failed to build RuntimeEnv"), ) @@ -431,3 +434,66 @@ impl DfQueryPlanner { } } } + +/// A wrapper around GreedyMemoryPool that records metrics. +/// +/// This wrapper intercepts all memory pool operations and updates +/// Prometheus metrics for monitoring query memory usage and rejections. +#[derive(Debug)] +struct MetricsMemoryPool { + inner: Arc, +} + +impl MetricsMemoryPool { + fn new(limit: usize) -> Self { + Self { + inner: Arc::new(GreedyMemoryPool::new(limit)), + } + } + + #[inline] + fn update_metrics(&self) { + QUERY_MEMORY_POOL_USAGE_BYTES.set(self.inner.reserved() as i64); + } +} + +impl MemoryPool for MetricsMemoryPool { + fn register(&self, consumer: &MemoryConsumer) { + self.inner.register(consumer); + } + + fn unregister(&self, consumer: &MemoryConsumer) { + self.inner.unregister(consumer); + } + + fn grow(&self, reservation: &MemoryReservation, additional: usize) { + self.inner.grow(reservation, additional); + self.update_metrics(); + } + + fn shrink(&self, reservation: &MemoryReservation, shrink: usize) { + self.inner.shrink(reservation, shrink); + self.update_metrics(); + } + + fn try_grow( + &self, + reservation: &MemoryReservation, + additional: usize, + ) -> datafusion_common::Result<()> { + let result = self.inner.try_grow(reservation, additional); + if result.is_err() { + QUERY_MEMORY_POOL_REJECTED_TOTAL.inc(); + } + self.update_metrics(); + result + } + + fn reserved(&self) -> usize { + self.inner.reserved() + } + + fn memory_limit(&self) -> MemoryLimit { + self.inner.memory_limit() + } +} diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 538392e4371a..1a8de5f41671 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -1577,6 +1577,8 @@ fn drop_lines_with_inconsistent_results(input: String) -> String { "enable_virtual_host_style =", "cache_path =", "cache_capacity =", + "memory_pool_size =", + "scan_memory_limit =", "sas_token =", "scope =", "num_workers =", From 1acb31030475634be424a058025d254e68d55240 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Wed, 15 Oct 2025 21:31:33 +0800 Subject: [PATCH 06/19] fix: right postion for tracker Signed-off-by: jeremyhi --- src/mito2/src/compaction.rs | 6 ++-- src/mito2/src/engine.rs | 15 ++++------ src/mito2/src/read/scan_region.rs | 42 ++++++++++++++++++++-------- src/mito2/src/read/seq_scan.rs | 20 +++++++++---- src/mito2/src/read/series_scan.rs | 20 +++++++++---- src/mito2/src/read/unordered_scan.rs | 23 +++++++++++---- 6 files changed, 88 insertions(+), 38 deletions(-) diff --git a/src/mito2/src/compaction.rs b/src/mito2/src/compaction.rs index d83ed7ab7dfc..ceab3d150a79 100644 --- a/src/mito2/src/compaction.rs +++ b/src/mito2/src/compaction.rs @@ -640,14 +640,16 @@ impl CompactionSstReaderBuilder<'_> { async fn build_sst_reader(self) -> Result { let scan_input = self.build_scan_input(false)?.with_compaction(true); - SeqScan::new(scan_input).build_reader_for_compaction().await + SeqScan::new(scan_input, None) + .build_reader_for_compaction() + .await } /// Builds [BoxedRecordBatchStream] that reads all SST files and yields batches in flat format for compaction. async fn build_flat_sst_reader(self) -> Result { let scan_input = self.build_scan_input(true)?.with_compaction(true); - SeqScan::new(scan_input) + SeqScan::new(scan_input, None) .build_flat_reader_for_compaction() .await } diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index b03caad346d3..54f40c355eec 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -79,7 +79,7 @@ use async_trait::async_trait; use common_base::Plugins; use common_error::ext::BoxedError; use common_meta::key::SchemaMetadataManagerRef; -use common_recordbatch::{MemoryTrackedStream, QueryMemoryTracker, SendableRecordBatchStream}; +use common_recordbatch::{QueryMemoryTracker, SendableRecordBatchStream}; use common_telemetry::{info, tracing, warn}; use common_wal::options::{WAL_OPTIONS_KEY, WalOptions}; use futures::future::{join_all, try_join_all}; @@ -306,17 +306,11 @@ impl MitoEngine { region_id: RegionId, request: ScanRequest, ) -> Result { - let stream = self - .scanner(region_id, request) + self.scanner(region_id, request) .await .map_err(BoxedError::new)? .scan() - .await?; - - Ok(Box::pin(MemoryTrackedStream::new( - stream, - self.inner.scan_memory_tracker.clone(), - ))) + .await } /// Scan [`Batch`]es by [`ScanRequest`]. @@ -805,7 +799,8 @@ impl EngineInner { .with_ignore_fulltext_index(self.config.fulltext_index.apply_on_query.disabled()) .with_ignore_bloom_filter(self.config.bloom_filter_index.apply_on_query.disabled()) .with_start_time(query_start) - .with_flat_format(self.config.default_experimental_flat_format); + .with_flat_format(self.config.default_experimental_flat_format) + .with_query_memory_tracker(self.scan_memory_tracker.clone()); #[cfg(feature = "enterprise")] let scan_region = self.maybe_fill_extension_range_provider(scan_region, region); diff --git a/src/mito2/src/read/scan_region.rs b/src/mito2/src/read/scan_region.rs index 536c48e2488a..d90833c6d97d 100644 --- a/src/mito2/src/read/scan_region.rs +++ b/src/mito2/src/read/scan_region.rs @@ -22,8 +22,8 @@ use std::time::Instant; use api::v1::SemanticType; use common_error::ext::BoxedError; -use common_recordbatch::SendableRecordBatchStream; use common_recordbatch::filter::SimpleFilterEvaluator; +use common_recordbatch::{QueryMemoryTracker, SendableRecordBatchStream}; use common_telemetry::{debug, error, tracing, warn}; use common_time::range::TimestampRange; use datafusion_common::Column; @@ -222,6 +222,8 @@ pub(crate) struct ScanRegion { filter_deleted: bool, /// Whether to use flat format. flat_format: bool, + /// Memory tracker for the query. + query_memory_tracker: Option, #[cfg(feature = "enterprise")] extension_range_provider: Option, } @@ -247,6 +249,7 @@ impl ScanRegion { start_time: None, filter_deleted: true, flat_format: false, + query_memory_tracker: None, #[cfg(feature = "enterprise")] extension_range_provider: None, } @@ -310,6 +313,13 @@ impl ScanRegion { self } + /// Sets memory tracker for the query. + #[must_use] + pub(crate) fn with_query_memory_tracker(mut self, tracker: QueryMemoryTracker) -> Self { + self.query_memory_tracker = Some(tracker); + self + } + #[cfg(feature = "enterprise")] pub(crate) fn set_extension_range_provider( &mut self, @@ -349,20 +359,20 @@ impl ScanRegion { /// Scan sequentially. pub(crate) async fn seq_scan(self) -> Result { - let input = self.scan_input().await?.with_compaction(false); - Ok(SeqScan::new(input)) + let (input, tracker) = self.scan_input().await?; + Ok(SeqScan::new(input.with_compaction(false), tracker)) } /// Unordered scan. pub(crate) async fn unordered_scan(self) -> Result { - let input = self.scan_input().await?; - Ok(UnorderedScan::new(input)) + let (input, tracker) = self.scan_input().await?; + Ok(UnorderedScan::new(input, tracker)) } /// Scans by series. pub(crate) async fn series_scan(self) -> Result { - let input = self.scan_input().await?; - Ok(SeriesScan::new(input)) + let (input, tracker) = self.scan_input().await?; + Ok(SeriesScan::new(input, tracker)) } /// Returns true if the region can use unordered scan for current request. @@ -385,7 +395,7 @@ impl ScanRegion { } /// Creates a scan input. - async fn scan_input(mut self) -> Result { + async fn scan_input(mut self) -> Result<(ScanInput, Option)> { let sst_min_sequence = self.request.sst_min_sequence.and_then(NonZeroU64::new); let time_range = self.build_time_range_predicate(); let predicate = PredicateGroup::new(&self.version.metadata, &self.request.filters)?; @@ -508,7 +518,7 @@ impl ScanRegion { } else { input }; - Ok(input) + Ok((input, self.query_memory_tracker)) } fn region_id(&self) -> RegionId { @@ -1167,11 +1177,16 @@ pub struct StreamContext { // Metrics: /// The start time of the query. pub(crate) query_start: Instant, + /// Memory tracker for the query. + pub(crate) query_memory_tracker: Option, } impl StreamContext { /// Creates a new [StreamContext] for [SeqScan]. - pub(crate) fn seq_scan_ctx(input: ScanInput) -> Self { + pub(crate) fn seq_scan_ctx( + input: ScanInput, + query_memory_tracker: Option, + ) -> Self { 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); @@ -1180,11 +1195,15 @@ impl StreamContext { input, ranges, query_start, + query_memory_tracker, } } /// Creates a new [StreamContext] for [UnorderedScan]. - pub(crate) fn unordered_scan_ctx(input: ScanInput) -> Self { + pub(crate) fn unordered_scan_ctx( + input: ScanInput, + query_memory_tracker: Option, + ) -> Self { 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); @@ -1193,6 +1212,7 @@ impl StreamContext { input, ranges, query_start, + query_memory_tracker, } } diff --git a/src/mito2/src/read/seq_scan.rs b/src/mito2/src/read/seq_scan.rs index 631c40b42a4f..150ec3b009cd 100644 --- a/src/mito2/src/read/seq_scan.rs +++ b/src/mito2/src/read/seq_scan.rs @@ -21,7 +21,9 @@ use std::time::Instant; use async_stream::try_stream; use common_error::ext::BoxedError; use common_recordbatch::util::ChainedRecordBatchStream; -use common_recordbatch::{RecordBatchStreamWrapper, SendableRecordBatchStream}; +use common_recordbatch::{ + MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, SendableRecordBatchStream, +}; use common_telemetry::tracing; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::{DisplayAs, DisplayFormatType}; @@ -71,11 +73,11 @@ pub struct SeqScan { impl SeqScan { /// Creates a new [SeqScan] with the given input. /// If `input.compaction` is true, the scanner will not attempt to split ranges. - pub(crate) fn new(input: ScanInput) -> Self { + pub(crate) fn new(input: ScanInput, query_memory_tracker: Option) -> Self { let mut properties = ScannerProperties::default() .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input)); + let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input, query_memory_tracker)); properties.partitions = vec![stream_ctx.partition_ranges()]; Self { @@ -350,10 +352,18 @@ impl SeqScan { metrics, ); - Ok(Box::pin(RecordBatchStreamWrapper::new( + let stream = Box::pin(RecordBatchStreamWrapper::new( input.mapper.output_schema(), Box::pin(record_batch_stream), - ))) + )); + + Ok( + if let Some(tracker) = &self.stream_ctx.query_memory_tracker { + Box::pin(MemoryTrackedStream::new(stream, tracker.clone())) + } else { + stream + }, + ) } fn scan_batch_in_partition( diff --git a/src/mito2/src/read/series_scan.rs b/src/mito2/src/read/series_scan.rs index 3a006dcb6768..32a19797a6b7 100644 --- a/src/mito2/src/read/series_scan.rs +++ b/src/mito2/src/read/series_scan.rs @@ -21,7 +21,9 @@ use std::time::{Duration, Instant}; use async_stream::try_stream; use common_error::ext::BoxedError; use common_recordbatch::util::ChainedRecordBatchStream; -use common_recordbatch::{RecordBatchStreamWrapper, SendableRecordBatchStream}; +use common_recordbatch::{ + MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, SendableRecordBatchStream, +}; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::{DisplayAs, DisplayFormatType}; use datatypes::arrow::array::BinaryArray; @@ -76,11 +78,11 @@ pub struct SeriesScan { impl SeriesScan { /// Creates a new [SeriesScan]. - pub(crate) fn new(input: ScanInput) -> Self { + pub(crate) fn new(input: ScanInput, query_memory_tracker: Option) -> Self { let mut properties = ScannerProperties::default() .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input)); + let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input, query_memory_tracker)); properties.partitions = vec![stream_ctx.partition_ranges()]; Self { @@ -116,10 +118,18 @@ impl SeriesScan { metrics, ); - Ok(Box::pin(RecordBatchStreamWrapper::new( + let stream = Box::pin(RecordBatchStreamWrapper::new( input.mapper.output_schema(), Box::pin(record_batch_stream), - ))) + )); + + Ok( + if let Some(tracker) = &self.stream_ctx.query_memory_tracker { + Box::pin(MemoryTrackedStream::new(stream, tracker.clone())) + } else { + stream + }, + ) } fn scan_batch_in_partition( diff --git a/src/mito2/src/read/unordered_scan.rs b/src/mito2/src/read/unordered_scan.rs index 4dc5d59b987e..1522e8bcd311 100644 --- a/src/mito2/src/read/unordered_scan.rs +++ b/src/mito2/src/read/unordered_scan.rs @@ -20,7 +20,9 @@ use std::time::Instant; use async_stream::{stream, try_stream}; use common_error::ext::BoxedError; -use common_recordbatch::{RecordBatchStreamWrapper, SendableRecordBatchStream}; +use common_recordbatch::{ + MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, SendableRecordBatchStream, +}; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::{DisplayAs, DisplayFormatType}; use datatypes::arrow::record_batch::RecordBatch; @@ -56,11 +58,14 @@ pub struct UnorderedScan { impl UnorderedScan { /// Creates a new [UnorderedScan]. - pub(crate) fn new(input: ScanInput) -> Self { + pub(crate) fn new(input: ScanInput, query_memory_tracker: Option) -> Self { let mut properties = ScannerProperties::default() .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - let stream_ctx = Arc::new(StreamContext::unordered_scan_ctx(input)); + let stream_ctx = Arc::new(StreamContext::unordered_scan_ctx( + input, + query_memory_tracker, + )); properties.partitions = vec![stream_ctx.partition_ranges()]; Self { @@ -246,10 +251,18 @@ impl UnorderedScan { metrics, ); - Ok(Box::pin(RecordBatchStreamWrapper::new( + let stream = Box::pin(RecordBatchStreamWrapper::new( input.mapper.output_schema(), Box::pin(record_batch_stream), - ))) + )); + + Ok( + if let Some(tracker) = &self.stream_ctx.query_memory_tracker { + Box::pin(MemoryTrackedStream::new(stream, tracker.clone())) + } else { + stream + }, + ) } fn scan_batch_in_partition( From e53acbf04181694ecae7482239db73d1875fb55f Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Thu, 16 Oct 2025 14:18:42 +0800 Subject: [PATCH 07/19] fix: avoid race condition Signed-off-by: jeremyhi --- src/common/recordbatch/src/lib.rs | 54 +++++++++++++++---------------- 1 file changed, 26 insertions(+), 28 deletions(-) diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index d28b478ccd65..ac810cf542cf 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -469,47 +469,45 @@ impl QueryMemoryTracker { /// Track memory usage. Returns error if limit is exceeded. pub fn track(&self, size: usize) -> Result<()> { - let new_total = self.current.fetch_add(size, Ordering::Relaxed) + size; - - if self.limit == 0 { - // Unlimited mode: still track usage but never reject - if let Some(callback) = &self.on_update { - callback(new_total); - } - return Ok(()); - } + let mut new_total = 0; + let result = self + .current + .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { + new_total = current.saturating_add(size); + if self.limit == 0 || new_total <= self.limit { + Some(new_total) + } else { + None + } + }); - if new_total > self.limit { - self.release_internal(size, false); - if let Some(callback) = &self.on_reject { - callback(); + match result { + Ok(_) => { + if let Some(callback) = &self.on_update { + callback(new_total); + } + Ok(()) } - return error::ExceedMemoryLimitSnafu { - used: new_total, - limit: self.limit, + Err(current) => { + if let Some(callback) = &self.on_reject { + callback(); + } + error::ExceedMemoryLimitSnafu { + used: current, + limit: self.limit, + } + .fail() } - .fail(); - } - - if let Some(callback) = &self.on_update { - callback(new_total); } - - Ok(()) } /// Release tracked memory. pub fn release(&self, size: usize) { - self.release_internal(size, true); - } - - fn release_internal(&self, size: usize, trigger_callback: bool) { if let Ok(old_value) = self.current .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { Some(current.saturating_sub(size)) }) - && trigger_callback && let Some(callback) = &self.on_update { callback(old_value.saturating_sub(size)); From 151324424ad98f24d0bd4ba1999af6fe657415ce Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Thu, 16 Oct 2025 16:45:10 +0800 Subject: [PATCH 08/19] feat: soft and hard limit Signed-off-by: jeremyhi --- Cargo.lock | 3 + config/datanode.example.toml | 17 ++- config/flownode.example.toml | 5 +- config/frontend.example.toml | 5 +- config/standalone.example.toml | 17 ++- src/cmd/Cargo.toml | 1 + src/cmd/tests/load_config_test.rs | 21 ++- src/common/base/src/lib.rs | 1 + src/common/base/src/memory_limit.rs | 216 ++++++++++++++++++++++++++++ src/common/recordbatch/src/lib.rs | 56 ++++++-- src/flow/src/adapter.rs | 4 +- src/mito2/Cargo.toml | 1 + src/mito2/src/config.rs | 12 +- src/mito2/src/engine.rs | 35 +++-- src/query/Cargo.toml | 1 + src/query/src/options.rs | 9 +- src/query/src/query_engine/state.rs | 4 +- tests-integration/tests/http.rs | 1 + 18 files changed, 355 insertions(+), 54 deletions(-) create mode 100644 src/common/base/src/memory_limit.rs diff --git a/Cargo.lock b/Cargo.lock index aec4ad866aff..627a9c7de079 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1940,6 +1940,7 @@ dependencies = [ "moka", "nu-ansi-term", "object-store", + "ordered-float 4.6.0", "plugins", "prometheus", "prost 0.13.5", @@ -7642,6 +7643,7 @@ dependencies = [ "mito-codec", "moka", "object-store", + "ordered-float 4.6.0", "parquet", "partition", "paste", @@ -10127,6 +10129,7 @@ dependencies = [ "chrono", "common-base", "common-catalog", + "common-config", "common-datasource", "common-error", "common-function", diff --git a/config/datanode.example.toml b/config/datanode.example.toml index 82ce35b022c8..47d3e6bd5ddd 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -257,12 +257,11 @@ overwrite_entry_start_id = false parallelism = 0 ## Memory pool size for query execution operators (aggregation, sorting, join). +## Supports absolute size (e.g., "2GB", "4GB") or percentage of system memory (e.g., "20%"). ## Setting it to 0 disables the limit (unbounded, default behavior). ## When this limit is reached, queries will fail with ResourceExhausted error. ## NOTE: This does NOT limit memory used by table scans. -## Example: "2GB", "4GB" -## @toml2docs:none-default -#+ memory_pool_size = "2GB" +memory_pool_size = "50%" ## The data storage options. [storage] @@ -504,9 +503,15 @@ max_concurrent_scan_files = 384 ## Whether to allow stale WAL entries read during replay. allow_stale_entries = false -## Memory limit for table scans across all queries. Setting it to 0 disables the limit. -## @toml2docs:none-default -#+ scan_memory_limit = "2GB" +## Memory limit for table scans across all queries. +## Supports absolute size (e.g., "2GB") or percentage of system memory (e.g., "20%"). +## Setting it to 0 disables the limit. +scan_memory_limit = "50%" + +## Soft limit ratio for scan memory (0.0 to 1.0). +## When memory usage exceeds soft limit, new queries are rejected but existing queries continue. +## This prevents thundering herd in high concurrency scenarios. +scan_memory_soft_limit_ratio = 0.7 ## Minimum time interval between two compactions. ## To align with the old behavior, the default value is 0 (no restrictions). diff --git a/config/flownode.example.toml b/config/flownode.example.toml index 008bf5039c0a..4e44c1ecbb04 100644 --- a/config/flownode.example.toml +++ b/config/flownode.example.toml @@ -159,12 +159,11 @@ default_ratio = 1.0 parallelism = 1 ## Memory pool size for query execution operators (aggregation, sorting, join). +## Supports absolute size (e.g., "1GB", "2GB") or percentage of system memory (e.g., "20%"). ## Setting it to 0 disables the limit (unbounded, default behavior). ## When this limit is reached, queries will fail with ResourceExhausted error. ## NOTE: This does NOT limit memory used by table scans. -## Example: "1GB", "2GB" -## @toml2docs:none-default -#+ memory_pool_size = "1GB" +memory_pool_size = "50%" ## The memory options. [memory] diff --git a/config/frontend.example.toml b/config/frontend.example.toml index 9977d1102131..4e1ef02e24ad 100644 --- a/config/frontend.example.toml +++ b/config/frontend.example.toml @@ -245,12 +245,11 @@ parallelism = 0 allow_query_fallback = false ## Memory pool size for query execution operators (aggregation, sorting, join). +## Supports absolute size (e.g., "4GB", "8GB") or percentage of system memory (e.g., "30%"). ## Setting it to 0 disables the limit (unbounded, default behavior). ## When this limit is reached, queries will fail with ResourceExhausted error. ## NOTE: This does NOT limit memory used by table scans (only applies to datanodes). -## Example: "4GB", "8GB" -## @toml2docs:none-default -#+ memory_pool_size = "4GB" +memory_pool_size = "50%" ## Datanode options. [datanode] diff --git a/config/standalone.example.toml b/config/standalone.example.toml index 07e0f74bb8cf..1646345dd556 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -354,12 +354,11 @@ max_running_procedures = 128 parallelism = 0 ## Memory pool size for query execution operators (aggregation, sorting, join). +## Supports absolute size (e.g., "2GB", "4GB") or percentage of system memory (e.g., "20%"). ## Setting it to 0 disables the limit (unbounded, default behavior). ## When this limit is reached, queries will fail with ResourceExhausted error. ## NOTE: This does NOT limit memory used by table scans. -## Example: "2GB", "4GB" -## @toml2docs:none-default -#+ memory_pool_size = "2GB" +memory_pool_size = "50%" ## The data storage options. [storage] @@ -588,9 +587,15 @@ max_concurrent_scan_files = 384 ## Whether to allow stale WAL entries read during replay. allow_stale_entries = false -## Memory limit for table scans across all queries. Setting it to 0 disables the limit. -## @toml2docs:none-default -#+ scan_memory_limit = "2GB" +## Memory limit for table scans across all queries. +## Supports absolute size (e.g., "2GB") or percentage of system memory (e.g., "20%"). +## Setting it to 0 disables the limit. +scan_memory_limit = "50%" + +## Soft limit ratio for scan memory (0.0 to 1.0). +## When memory usage exceeds soft limit, new queries are rejected but existing queries continue. +## This prevents thundering herd in high concurrency scenarios. +scan_memory_soft_limit_ratio = 0.7 ## Minimum time interval between two compactions. ## To align with the old behavior, the default value is 0 (no restrictions). diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 8baa1dc50d4e..105b7d19b008 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -96,6 +96,7 @@ tikv-jemallocator = "0.6" client = { workspace = true, features = ["testing"] } common-test-util.workspace = true common-version.workspace = true +ordered-float.workspace = true serde.workspace = true temp-env = "0.3" tempfile.workspace = true diff --git a/src/cmd/tests/load_config_test.rs b/src/cmd/tests/load_config_test.rs index 9ebc3e55d747..fd265272be22 100644 --- a/src/cmd/tests/load_config_test.rs +++ b/src/cmd/tests/load_config_test.rs @@ -15,7 +15,7 @@ use std::time::Duration; use cmd::options::GreptimeOptions; -use common_base::readable_size::ReadableSize; +use common_base::memory_limit::MemoryLimit; use common_config::{Configurable, DEFAULT_DATA_HOME}; use common_options::datanode::{ClientOptions, DatanodeClientOptions}; use common_telemetry::logging::{DEFAULT_LOGGING_DIR, DEFAULT_OTLP_HTTP_ENDPOINT, LoggingOptions}; @@ -74,6 +74,8 @@ fn test_load_datanode_example_config() { RegionEngineConfig::Mito(MitoConfig { auto_flush_interval: Duration::from_secs(3600), write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), + scan_memory_limit: MemoryLimit::Percentage(0.5), + scan_memory_soft_limit_ratio: ordered_float::OrderedFloat(0.7), ..Default::default() }), RegionEngineConfig::File(FileEngineConfig {}), @@ -82,6 +84,10 @@ fn test_load_datanode_example_config() { flush_metadata_region_interval: Duration::from_secs(30), }), ], + query: QueryOptions { + memory_pool_size: MemoryLimit::Percentage(0.5), + ..Default::default() + }, logging: LoggingOptions { level: Some("info".to_string()), dir: format!("{}/{}", DEFAULT_DATA_HOME, DEFAULT_LOGGING_DIR), @@ -154,6 +160,10 @@ fn test_load_frontend_example_config() { cors_allowed_origins: vec!["https://example.com".to_string()], ..Default::default() }, + query: QueryOptions { + memory_pool_size: MemoryLimit::Percentage(0.5), + ..Default::default() + }, ..Default::default() }, ..Default::default() @@ -241,7 +251,7 @@ fn test_load_flownode_example_config() { query: QueryOptions { parallelism: 1, allow_query_fallback: false, - memory_pool_size: ReadableSize(0), + memory_pool_size: MemoryLimit::Percentage(0.5), }, meta_client: Some(MetaClientOptions { metasrv_addrs: vec!["127.0.0.1:3002".to_string()], @@ -285,6 +295,8 @@ fn test_load_standalone_example_config() { RegionEngineConfig::Mito(MitoConfig { auto_flush_interval: Duration::from_secs(3600), write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), + scan_memory_limit: MemoryLimit::Percentage(0.5), + scan_memory_soft_limit_ratio: ordered_float::OrderedFloat(0.7), ..Default::default() }), RegionEngineConfig::File(FileEngineConfig {}), @@ -313,7 +325,10 @@ fn test_load_standalone_example_config() { cors_allowed_origins: vec!["https://example.com".to_string()], ..Default::default() }, - + query: QueryOptions { + memory_pool_size: MemoryLimit::Percentage(0.5), + ..Default::default() + }, ..Default::default() }, ..Default::default() diff --git a/src/common/base/src/lib.rs b/src/common/base/src/lib.rs index cc5acdbf47af..681efe28d71f 100644 --- a/src/common/base/src/lib.rs +++ b/src/common/base/src/lib.rs @@ -15,6 +15,7 @@ pub mod bit_vec; pub mod bytes; pub mod cancellation; +pub mod memory_limit; pub mod plugins; pub mod range_read; #[allow(clippy::all)] diff --git a/src/common/base/src/memory_limit.rs b/src/common/base/src/memory_limit.rs new file mode 100644 index 000000000000..ccd3fd6081a7 --- /dev/null +++ b/src/common/base/src/memory_limit.rs @@ -0,0 +1,216 @@ +// 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. + +use std::fmt::{self, Display}; +use std::str::FromStr; + +use serde::{Deserialize, Deserializer, Serialize, Serializer}; + +use crate::readable_size::ReadableSize; + +/// Memory limit configuration that supports both absolute size and percentage. +/// +/// Examples: +/// - Absolute size: "2GB", "4GiB", "512MB" +/// - Percentage: "50%", "75%" +/// - Unlimited: "0" +#[derive(Debug, Clone, Copy, PartialEq)] +pub enum MemoryLimit { + /// Absolute memory size. + Size(ReadableSize), + /// Percentage of total system memory (0.0 to 1.0). + Percentage(f64), +} + +// Safe to implement Eq because percentage values are always in valid range (0.0-1.0) +// and won't be NaN or Infinity. +impl Eq for MemoryLimit {} + +impl MemoryLimit { + /// Resolve the memory limit to bytes based on total system memory. + /// Returns 0 if the limit is disabled (Size(0) or Percentage(0)). + pub fn resolve(&self, total_memory_bytes: u64) -> u64 { + match self { + MemoryLimit::Size(size) => size.as_bytes(), + MemoryLimit::Percentage(pct) => { + if *pct <= 0.0 { + 0 + } else { + (total_memory_bytes as f64 * pct) as u64 + } + } + } + } + + /// Returns true if this limit is disabled (0 bytes or 0%). + pub fn is_unlimited(&self) -> bool { + match self { + MemoryLimit::Size(size) => size.as_bytes() == 0, + MemoryLimit::Percentage(pct) => *pct <= 0.0, + } + } +} + +impl Default for MemoryLimit { + fn default() -> Self { + MemoryLimit::Size(ReadableSize(0)) + } +} + +impl FromStr for MemoryLimit { + type Err = String; + + fn from_str(s: &str) -> Result { + let s = s.trim(); + + if let Some(pct_str) = s.strip_suffix('%') { + let pct = pct_str + .trim() + .parse::() + .map_err(|e| format!("invalid percentage value '{}': {}", pct_str, e))?; + + if !(0.0..=100.0).contains(&pct) { + return Err(format!("percentage must be between 0 and 100, got {}", pct)); + } + + Ok(MemoryLimit::Percentage(pct / 100.0)) + } else { + let size = ReadableSize::from_str(s)?; + Ok(MemoryLimit::Size(size)) + } + } +} + +impl Display for MemoryLimit { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + MemoryLimit::Size(size) => write!(f, "{}", size), + MemoryLimit::Percentage(pct) => write!(f, "{}%", pct * 100.0), + } + } +} + +impl Serialize for MemoryLimit { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + serializer.serialize_str(&self.to_string()) + } +} + +impl<'de> Deserialize<'de> for MemoryLimit { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + MemoryLimit::from_str(&s).map_err(serde::de::Error::custom) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_parse_absolute_size() { + assert_eq!( + "2GB".parse::().unwrap(), + MemoryLimit::Size(ReadableSize(2 * 1024 * 1024 * 1024)) + ); + assert_eq!( + "512MB".parse::().unwrap(), + MemoryLimit::Size(ReadableSize(512 * 1024 * 1024)) + ); + assert_eq!( + "0".parse::().unwrap(), + MemoryLimit::Size(ReadableSize(0)) + ); + } + + #[test] + fn test_parse_percentage() { + assert_eq!( + "50%".parse::().unwrap(), + MemoryLimit::Percentage(0.5) + ); + assert_eq!( + "75%".parse::().unwrap(), + MemoryLimit::Percentage(0.75) + ); + assert_eq!( + "0%".parse::().unwrap(), + MemoryLimit::Percentage(0.0) + ); + } + + #[test] + fn test_parse_invalid() { + assert!("150%".parse::().is_err()); + assert!("-10%".parse::().is_err()); + assert!("invalid".parse::().is_err()); + } + + #[test] + fn test_resolve() { + let total = 8 * 1024 * 1024 * 1024; // 8GB + + assert_eq!( + MemoryLimit::Size(ReadableSize(2 * 1024 * 1024 * 1024)).resolve(total), + 2 * 1024 * 1024 * 1024 + ); + assert_eq!( + MemoryLimit::Percentage(0.5).resolve(total), + 4 * 1024 * 1024 * 1024 + ); + assert_eq!(MemoryLimit::Size(ReadableSize(0)).resolve(total), 0); + assert_eq!(MemoryLimit::Percentage(0.0).resolve(total), 0); + } + + #[test] + fn test_is_unlimited() { + assert!(MemoryLimit::Size(ReadableSize(0)).is_unlimited()); + assert!(MemoryLimit::Percentage(0.0).is_unlimited()); + assert!(!MemoryLimit::Size(ReadableSize(1024)).is_unlimited()); + assert!(!MemoryLimit::Percentage(0.5).is_unlimited()); + } + + #[test] + fn test_parse_100_percent() { + assert_eq!( + "100%".parse::().unwrap(), + MemoryLimit::Percentage(1.0) + ); + } + + #[test] + fn test_parse_decimal_percentage() { + assert_eq!( + "20.5%".parse::().unwrap(), + MemoryLimit::Percentage(0.205) + ); + } + + #[test] + fn test_display_integer_percentage() { + assert_eq!(MemoryLimit::Percentage(0.2).to_string(), "20%"); + assert_eq!(MemoryLimit::Percentage(0.5).to_string(), "50%"); + } + + #[test] + fn test_display_decimal_percentage() { + assert_eq!(MemoryLimit::Percentage(0.205).to_string(), "20.5%"); + } +} diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index ac810cf542cf..8b5bf3b19660 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -409,10 +409,15 @@ impl> + Unpin> Stream for RecordBatchStream } /// Memory tracker for RecordBatch streams. Clone to share the same limit across queries. +/// +/// Uses soft and hard limit strategy to prevent thundering herd in high concurrency: +/// - Soft limit: Reject new allocations but allow existing streams to continue +/// - Hard limit: Reject all allocations #[derive(Clone)] pub struct QueryMemoryTracker { current: Arc, - limit: usize, + soft_limit: usize, + hard_limit: usize, on_update: Option>, on_reject: Option>, } @@ -421,7 +426,8 @@ impl fmt::Debug for QueryMemoryTracker { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("QueryMemoryTracker") .field("current", &self.current.load(Ordering::Relaxed)) - .field("limit", &self.limit) + .field("soft_limit", &self.soft_limit) + .field("hard_limit", &self.hard_limit) .field("on_update", &self.on_update.is_some()) .field("on_reject", &self.on_reject.is_some()) .finish() @@ -429,12 +435,23 @@ impl fmt::Debug for QueryMemoryTracker { } impl QueryMemoryTracker { - /// Create a new memory tracker with the given limit (in bytes). - /// If limit is 0, no limit is enforced. - pub fn new(limit: usize) -> Self { + /// Create a new memory tracker with the given hard limit (in bytes). + /// + /// # Arguments + /// * `hard_limit` - Maximum memory usage in bytes. 0 means unlimited. + /// * `soft_limit_ratio` - Ratio of soft limit to hard limit (0.0 to 1.0). + /// When current usage exceeds soft limit, new allocations are rejected + /// but existing streams can continue until hard limit. + pub fn new(hard_limit: usize, soft_limit_ratio: f64) -> Self { + let soft_limit = if hard_limit > 0 { + (hard_limit as f64 * soft_limit_ratio.clamp(0.0, 1.0)) as usize + } else { + 0 + }; Self { current: Arc::new(AtomicUsize::new(0)), - limit, + soft_limit, + hard_limit, on_update: None, on_reject: None, } @@ -468,13 +485,31 @@ impl QueryMemoryTracker { } /// Track memory usage. Returns error if limit is exceeded. - pub fn track(&self, size: usize) -> Result<()> { + /// + /// # Arguments + /// * `size` - Memory size to track in bytes + /// * `is_initial` - True for first allocation of a stream, false for subsequent allocations + /// + /// When `is_initial` is true, soft limit is checked to prevent thundering herd. + /// When `is_initial` is false, hard limit is checked to allow existing streams to continue. + pub fn track(&self, size: usize, is_initial: bool) -> Result<()> { let mut new_total = 0; let result = self .current .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { new_total = current.saturating_add(size); - if self.limit == 0 || new_total <= self.limit { + let limit = if self.hard_limit == 0 { + // Unlimited mode + return Some(new_total); + } else if is_initial { + // New allocation: check soft limit + self.soft_limit + } else { + // Existing stream: check hard limit + self.hard_limit + }; + + if new_total <= limit { Some(new_total) } else { None @@ -494,7 +529,7 @@ impl QueryMemoryTracker { } error::ExceedMemoryLimitSnafu { used: current, - limit: self.limit, + limit: self.hard_limit, } .fail() } @@ -539,8 +574,9 @@ impl Stream for MemoryTrackedStream { match Pin::new(&mut self.inner).poll_next(cx) { Poll::Ready(Some(Ok(batch))) => { let size = batch.estimated_size(); + let is_initial = self.total_tracked == 0; - if let Err(e) = self.tracker.track(size) { + if let Err(e) = self.tracker.track(size, is_initial) { return Poll::Ready(Some(Err(e))); } diff --git a/src/flow/src/adapter.rs b/src/flow/src/adapter.rs index 7f3c4f404f96..a8d2482faf0e 100644 --- a/src/flow/src/adapter.rs +++ b/src/flow/src/adapter.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use std::time::{Duration, Instant, SystemTime}; use api::v1::{RowDeleteRequest, RowDeleteRequests, RowInsertRequest, RowInsertRequests}; -use common_base::readable_size::ReadableSize; +use common_base::memory_limit::MemoryLimit; use common_config::Configurable; use common_error::ext::BoxedError; use common_meta::key::TableMetadataManagerRef; @@ -133,7 +133,7 @@ impl Default for FlownodeOptions { query: QueryOptions { parallelism: 1, allow_query_fallback: false, - memory_pool_size: ReadableSize(0), + memory_pool_size: MemoryLimit::default(), }, user_provider: None, memory: MemoryOptions::default(), diff --git a/src/mito2/Cargo.toml b/src/mito2/Cargo.toml index 4cc1efb8bc1e..4ae8b107f90b 100644 --- a/src/mito2/Cargo.toml +++ b/src/mito2/Cargo.toml @@ -56,6 +56,7 @@ log-store = { workspace = true } mito-codec.workspace = true moka = { workspace = true, features = ["sync", "future"] } object-store.workspace = true +ordered-float.workspace = true parquet = { workspace = true, features = ["async"] } paste.workspace = true pin-project.workspace = true diff --git a/src/mito2/src/config.rs b/src/mito2/src/config.rs index 934a0a132c1f..188295e3e52e 100644 --- a/src/mito2/src/config.rs +++ b/src/mito2/src/config.rs @@ -18,9 +18,11 @@ use std::cmp; use std::path::Path; use std::time::Duration; +use common_base::memory_limit::MemoryLimit; use common_base::readable_size::ReadableSize; use common_stat::{get_total_cpu_cores, get_total_memory_readable}; use common_telemetry::warn; +use ordered_float::OrderedFloat; use serde::{Deserialize, Serialize}; use serde_with::serde_as; @@ -128,7 +130,12 @@ pub struct MitoConfig { /// Whether to allow stale entries read during replay. pub allow_stale_entries: bool, /// Memory limit for table scans across all queries. Setting it to 0 disables the limit. - pub scan_memory_limit: ReadableSize, + /// Supports absolute size (e.g., "2GB") or percentage (e.g., "50%"). + pub scan_memory_limit: MemoryLimit, + /// Soft limit ratio for scan memory (0.0 to 1.0). + /// When memory usage exceeds soft limit, new queries are rejected but existing queries continue. + /// This helps prevent thundering herd in high concurrency scenarios. + pub scan_memory_soft_limit_ratio: OrderedFloat, /// Index configs. pub index: IndexConfig, @@ -181,7 +188,8 @@ impl Default for MitoConfig { parallel_scan_channel_size: DEFAULT_SCAN_CHANNEL_SIZE, max_concurrent_scan_files: DEFAULT_MAX_CONCURRENT_SCAN_FILES, allow_stale_entries: false, - scan_memory_limit: ReadableSize(0), + scan_memory_limit: MemoryLimit::default(), + scan_memory_soft_limit_ratio: OrderedFloat(0.7), index: IndexConfig::default(), inverted_index: InvertedIndexConfig::default(), fulltext_index: FulltextIndexConfig::default(), diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index 54f40c355eec..b42ce75846e7 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -77,6 +77,7 @@ use std::time::Instant; use api::region::RegionResponse; use async_trait::async_trait; use common_base::Plugins; +use common_config::utils::get_sys_total_memory; use common_error::ext::BoxedError; use common_meta::key::SchemaMetadataManagerRef; use common_recordbatch::{QueryMemoryTracker, SendableRecordBatchStream}; @@ -199,13 +200,16 @@ impl<'a, S: LogStore> MitoEngineBuilder<'a, S> { ) .await?; let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(self.log_store)); - let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize) - .with_update_callback(|usage| { - SCAN_MEMORY_USAGE_BYTES.set(usage as i64); - }) - .with_reject_callback(|| { - SCAN_REQUESTS_REJECTED_TOTAL.inc(); - }); + let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); + let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize; + let scan_memory_tracker = + QueryMemoryTracker::new(scan_memory_limit, config.scan_memory_soft_limit_ratio.0) + .with_update_callback(|usage| { + SCAN_MEMORY_USAGE_BYTES.set(usage as i64); + }) + .with_reject_callback(|| { + SCAN_REQUESTS_REJECTED_TOTAL.inc(); + }); let inner = EngineInner { workers, @@ -1087,13 +1091,16 @@ impl MitoEngine { let config = Arc::new(config); let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(log_store.clone())); - let scan_memory_tracker = QueryMemoryTracker::new(config.scan_memory_limit.0 as usize) - .with_update_callback(|usage| { - SCAN_MEMORY_USAGE_BYTES.set(usage as i64); - }) - .with_reject_callback(|| { - SCAN_REQUESTS_REJECTED_TOTAL.inc(); - }); + let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); + let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize; + let scan_memory_tracker = + QueryMemoryTracker::new(scan_memory_limit, config.scan_memory_soft_limit_ratio.0) + .with_update_callback(|usage| { + SCAN_MEMORY_USAGE_BYTES.set(usage as i64); + }) + .with_reject_callback(|| { + SCAN_REQUESTS_REJECTED_TOTAL.inc(); + }); Ok(MitoEngine { inner: Arc::new(EngineInner { workers: WorkerGroup::start_for_test( diff --git a/src/query/Cargo.toml b/src/query/Cargo.toml index 0c6c2e033f9a..e92eabb142fd 100644 --- a/src/query/Cargo.toml +++ b/src/query/Cargo.toml @@ -24,6 +24,7 @@ catalog.workspace = true chrono.workspace = true common-base.workspace = true common-catalog.workspace = true +common-config.workspace = true common-datasource.workspace = true common-error.workspace = true common-function.workspace = true diff --git a/src/query/src/options.rs b/src/query/src/options.rs index f4a5b9aa8d7c..d9e9c7ebdfd4 100644 --- a/src/query/src/options.rs +++ b/src/query/src/options.rs @@ -12,11 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use common_base::readable_size::ReadableSize; +use common_base::memory_limit::MemoryLimit; use serde::{Deserialize, Serialize}; /// Query engine config -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] #[serde(default)] pub struct QueryOptions { /// Parallelism of query engine. Default to 0, which implies the number of logical CPUs. @@ -24,8 +24,9 @@ pub struct QueryOptions { /// Whether to allow query fallback when push down fails. pub allow_query_fallback: bool, /// Memory pool size for query execution. Setting it to 0 disables the limit (unbounded). + /// Supports absolute size (e.g., "2GB") or percentage (e.g., "50%"). /// When this limit is reached, queries will fail with ResourceExhausted error. - pub memory_pool_size: ReadableSize, + pub memory_pool_size: MemoryLimit, } #[allow(clippy::derivable_impls)] @@ -34,7 +35,7 @@ impl Default for QueryOptions { Self { parallelism: 0, allow_query_fallback: false, - memory_pool_size: ReadableSize(0), + memory_pool_size: MemoryLimit::default(), } } } diff --git a/src/query/src/query_engine/state.rs b/src/query/src/query_engine/state.rs index e83d37504a4a..17dde0a3a251 100644 --- a/src/query/src/query_engine/state.rs +++ b/src/query/src/query_engine/state.rs @@ -19,6 +19,7 @@ use std::sync::{Arc, RwLock}; use async_trait::async_trait; use catalog::CatalogManagerRef; use common_base::Plugins; +use common_config::utils::get_sys_total_memory; use common_function::aggrs::aggr_wrapper::fix_order::FixStateUdafOrderingAnalyzer; use common_function::function_factory::ScalarFunctionFactory; use common_function::handlers::{ @@ -104,7 +105,8 @@ impl QueryEngineState { plugins: Plugins, options: QueryOptionsNew, ) -> Self { - let memory_pool_size = options.memory_pool_size.as_bytes() as usize; + let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); + let memory_pool_size = options.memory_pool_size.resolve(total_memory) as usize; let runtime_env = if memory_pool_size > 0 { Arc::new( RuntimeEnvBuilder::new() diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 1a8de5f41671..7beb09ffb4c4 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -1579,6 +1579,7 @@ fn drop_lines_with_inconsistent_results(input: String) -> String { "cache_capacity =", "memory_pool_size =", "scan_memory_limit =", + "scan_memory_soft_limit_ratio =", "sas_token =", "scope =", "num_workers =", From 799fdf06f3c28dd2fb95b15c9953cbc95a41c776 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Thu, 16 Oct 2025 16:46:30 +0800 Subject: [PATCH 09/19] feat: docs Signed-off-by: jeremyhi --- config/config.md | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/config/config.md b/config/config.md index 84e4f5be6e75..c6f15c7e60b8 100644 --- a/config/config.md +++ b/config/config.md @@ -101,7 +101,7 @@ | `flow.num_workers` | Integer | `0` | The number of flow worker in flownode.
Not setting(or set to 0) this value will use the number of CPU cores divided by 2. | | `query` | -- | -- | The query engine options. | | `query.parallelism` | Integer | `0` | Parallelism of the query engine.
Default to 0, which means the number of CPU cores. | -| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans.
Example: "2GB", "4GB" | +| `query.memory_pool_size` | String | `50%` | Memory pool size for query execution operators (aggregation, sorting, join).
Supports absolute size (e.g., "2GB", "4GB") or percentage of system memory (e.g., "20%").
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans. | | `storage` | -- | -- | The data storage options. | | `storage.data_home` | String | `./greptimedb_data` | The working home directory. | | `storage.type` | String | `File` | The storage type used to store the data.
- `File`: the data is stored in the local file system.
- `S3`: the data is stored in the S3 object storage.
- `Gcs`: the data is stored in the Google Cloud Storage.
- `Azblob`: the data is stored in the Azure Blob Storage.
- `Oss`: the data is stored in the Aliyun OSS. | @@ -153,7 +153,8 @@ | `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. | | `region_engine.mito.max_concurrent_scan_files` | Integer | `384` | Maximum number of SST files to scan concurrently. | | `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. | -| `region_engine.mito.scan_memory_limit` | String | Unset | Memory limit for table scans across all queries. Setting it to 0 disables the limit. | +| `region_engine.mito.scan_memory_limit` | String | `50%` | Memory limit for table scans across all queries.
Supports absolute size (e.g., "2GB") or percentage of system memory (e.g., "20%").
Setting it to 0 disables the limit. | +| `region_engine.mito.scan_memory_soft_limit_ratio` | Float | `0.7` | Soft limit ratio for scan memory (0.0 to 1.0).
When memory usage exceeds soft limit, new queries are rejected but existing queries continue.
This prevents thundering herd in high concurrency scenarios. | | `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.
To align with the old behavior, the default value is 0 (no restrictions). | | `region_engine.mito.default_experimental_flat_format` | Bool | `false` | Whether to enable experimental flat format as the default format. | | `region_engine.mito.index` | -- | -- | The options for index in Mito engine. | @@ -304,7 +305,7 @@ | `query` | -- | -- | The query engine options. | | `query.parallelism` | Integer | `0` | Parallelism of the query engine.
Default to 0, which means the number of CPU cores. | | `query.allow_query_fallback` | Bool | `false` | Whether to allow query fallback when push down optimize fails.
Default to false, meaning when push down optimize failed, return error msg | -| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans (only applies to datanodes).
Example: "4GB", "8GB" | +| `query.memory_pool_size` | String | `50%` | Memory pool size for query execution operators (aggregation, sorting, join).
Supports absolute size (e.g., "4GB", "8GB") or percentage of system memory (e.g., "30%").
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans (only applies to datanodes). | | `datanode` | -- | -- | Datanode options. | | `datanode.client` | -- | -- | Datanode client options. | | `datanode.client.connect_timeout` | String | `10s` | -- | @@ -496,7 +497,7 @@ | `wal.overwrite_entry_start_id` | Bool | `false` | Ignore missing entries during read WAL.
**It's only used when the provider is `kafka`**.

This option ensures that when Kafka messages are deleted, the system
can still successfully replay memtable data without throwing an
out-of-range error.
However, enabling this option might lead to unexpected data loss,
as the system will skip over missing entries instead of treating
them as critical errors. | | `query` | -- | -- | The query engine options. | | `query.parallelism` | Integer | `0` | Parallelism of the query engine.
Default to 0, which means the number of CPU cores. | -| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans.
Example: "2GB", "4GB" | +| `query.memory_pool_size` | String | `50%` | Memory pool size for query execution operators (aggregation, sorting, join).
Supports absolute size (e.g., "2GB", "4GB") or percentage of system memory (e.g., "20%").
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans. | | `storage` | -- | -- | The data storage options. | | `storage.data_home` | String | `./greptimedb_data` | The working home directory. | | `storage.type` | String | `File` | The storage type used to store the data.
- `File`: the data is stored in the local file system.
- `S3`: the data is stored in the S3 object storage.
- `Gcs`: the data is stored in the Google Cloud Storage.
- `Azblob`: the data is stored in the Azure Blob Storage.
- `Oss`: the data is stored in the Aliyun OSS. | @@ -550,7 +551,8 @@ | `region_engine.mito.parallel_scan_channel_size` | Integer | `32` | Capacity of the channel to send data from parallel scan tasks to the main task. | | `region_engine.mito.max_concurrent_scan_files` | Integer | `384` | Maximum number of SST files to scan concurrently. | | `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. | -| `region_engine.mito.scan_memory_limit` | String | Unset | Memory limit for table scans across all queries. Setting it to 0 disables the limit. | +| `region_engine.mito.scan_memory_limit` | String | `50%` | Memory limit for table scans across all queries.
Supports absolute size (e.g., "2GB") or percentage of system memory (e.g., "20%").
Setting it to 0 disables the limit. | +| `region_engine.mito.scan_memory_soft_limit_ratio` | Float | `0.7` | Soft limit ratio for scan memory (0.0 to 1.0).
When memory usage exceeds soft limit, new queries are rejected but existing queries continue.
This prevents thundering herd in high concurrency scenarios. | | `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.
To align with the old behavior, the default value is 0 (no restrictions). | | `region_engine.mito.default_experimental_flat_format` | Bool | `false` | Whether to enable experimental flat format as the default format. | | `region_engine.mito.index` | -- | -- | The options for index in Mito engine. | @@ -671,6 +673,6 @@ | `tracing.tokio_console_addr` | String | Unset | The tokio console address. | | `query` | -- | -- | -- | | `query.parallelism` | Integer | `1` | Parallelism of the query engine for query sent by flownode.
Default to 1, so it won't use too much cpu or memory | -| `query.memory_pool_size` | String | Unset | Memory pool size for query execution operators (aggregation, sorting, join).
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans.
Example: "1GB", "2GB" | +| `query.memory_pool_size` | String | `50%` | Memory pool size for query execution operators (aggregation, sorting, join).
Supports absolute size (e.g., "1GB", "2GB") or percentage of system memory (e.g., "20%").
Setting it to 0 disables the limit (unbounded, default behavior).
When this limit is reached, queries will fail with ResourceExhausted error.
NOTE: This does NOT limit memory used by table scans. | | `memory` | -- | -- | The memory options. | | `memory.enable_heap_profiling` | Bool | `true` | Whether to enable heap profiling activation during startup.
When enabled, heap profiling will be activated if the `MALLOC_CONF` environment variable
is set to "prof:true,prof_active:false". The official image adds this env variable.
Default is true. | From 01e6f3b20e37d452d9fd8ca472a2fbf844cb4ff5 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Thu, 16 Oct 2025 17:01:59 +0800 Subject: [PATCH 10/19] fix: when soft_limit == 0 Signed-off-by: jeremyhi --- src/common/recordbatch/src/lib.rs | 61 +++++++++++++++++++++++++++++-- 1 file changed, 57 insertions(+), 4 deletions(-) diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index 8b5bf3b19660..522fa891d345 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -440,10 +440,11 @@ impl QueryMemoryTracker { /// # Arguments /// * `hard_limit` - Maximum memory usage in bytes. 0 means unlimited. /// * `soft_limit_ratio` - Ratio of soft limit to hard limit (0.0 to 1.0). + /// When ratio <= 0, soft limit is disabled (all queries use hard limit). /// When current usage exceeds soft limit, new allocations are rejected /// but existing streams can continue until hard limit. pub fn new(hard_limit: usize, soft_limit_ratio: f64) -> Self { - let soft_limit = if hard_limit > 0 { + let soft_limit = if hard_limit > 0 && soft_limit_ratio > 0.0 { (hard_limit as f64 * soft_limit_ratio.clamp(0.0, 1.0)) as usize } else { 0 @@ -484,6 +485,11 @@ impl QueryMemoryTracker { self } + /// Get the current memory usage in bytes. + pub fn current(&self) -> usize { + self.current.load(Ordering::Relaxed) + } + /// Track memory usage. Returns error if limit is exceeded. /// /// # Arguments @@ -501,11 +507,11 @@ impl QueryMemoryTracker { let limit = if self.hard_limit == 0 { // Unlimited mode return Some(new_total); - } else if is_initial { - // New allocation: check soft limit + } else if is_initial && self.soft_limit > 0 { + // New allocation: check soft limit (if enabled) self.soft_limit } else { - // Existing stream: check hard limit + // Existing stream or soft limit disabled: check hard limit self.hard_limit }; @@ -700,4 +706,51 @@ mod tests { assert_eq!(collected[0], batch1); assert_eq!(collected[1], batch2); } + + #[test] + fn test_query_memory_tracker_zero_soft_limit_ratio() { + // When soft_limit_ratio is 0, soft limit should be disabled + // and all queries should use hard limit + let tracker = QueryMemoryTracker::new(1000, 0.0); + + // First allocation (is_initial=true) should succeed up to hard limit + assert!(tracker.track(500, true).is_ok()); + assert_eq!(tracker.current(), 500); + + // Second allocation should also succeed + assert!(tracker.track(400, true).is_ok()); + assert_eq!(tracker.current(), 900); + + // Exceeding hard limit should fail + assert!(tracker.track(200, true).is_err()); + assert_eq!(tracker.current(), 900); + + tracker.release(900); + assert_eq!(tracker.current(), 0); + } + + #[test] + fn test_query_memory_tracker_with_soft_limit() { + // With soft_limit_ratio = 0.5, soft limit is 500 + let tracker = QueryMemoryTracker::new(1000, 0.5); + + // First allocation (is_initial=true) checks soft limit + assert!(tracker.track(400, true).is_ok()); + assert_eq!(tracker.current(), 400); + + // New allocation exceeding soft limit should fail + assert!(tracker.track(200, true).is_err()); + assert_eq!(tracker.current(), 400); + + // Existing stream (is_initial=false) can use up to hard limit + assert!(tracker.track(500, false).is_ok()); + assert_eq!(tracker.current(), 900); + + // Exceeding hard limit should fail even for existing stream + assert!(tracker.track(200, false).is_err()); + assert_eq!(tracker.current(), 900); + + tracker.release(900); + assert_eq!(tracker.current(), 0); + } } From 8ca80f89b70e3bc8a98ca28b4dec7b685979189f Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Thu, 16 Oct 2025 18:50:44 +0800 Subject: [PATCH 11/19] feat: upgrade limit algorithm Signed-off-by: jeremyhi --- src/common/recordbatch/src/lib.rs | 87 ++++++++++++++++++++++--------- 1 file changed, 63 insertions(+), 24 deletions(-) diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index 522fa891d345..04de666fb29d 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -21,6 +21,7 @@ pub mod filter; mod recordbatch; pub mod util; +use std::collections::VecDeque; use std::fmt; use std::pin::Pin; use std::sync::Arc; @@ -496,26 +497,34 @@ impl QueryMemoryTracker { /// * `size` - Memory size to track in bytes /// * `is_initial` - True for first allocation of a stream, false for subsequent allocations /// - /// When `is_initial` is true, soft limit is checked to prevent thundering herd. - /// When `is_initial` is false, hard limit is checked to allow existing streams to continue. + /// # Soft Limit Behavior + /// When `is_initial` is true and soft limit is enabled: + /// - If current usage < soft limit: allow the allocation (checked against hard limit) + /// - If current usage >= soft limit: reject the allocation immediately + /// + /// This ensures that new queries are only accepted when memory pressure is low, + /// while existing queries can continue until hard limit is reached. pub fn track(&self, size: usize, is_initial: bool) -> Result<()> { let mut new_total = 0; let result = self .current .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { new_total = current.saturating_add(size); - let limit = if self.hard_limit == 0 { + + if self.hard_limit == 0 { // Unlimited mode return Some(new_total); - } else if is_initial && self.soft_limit > 0 { - // New allocation: check soft limit (if enabled) - self.soft_limit - } else { - // Existing stream or soft limit disabled: check hard limit - self.hard_limit - }; + } + + if is_initial && self.soft_limit > 0 { + // New query: check if current usage is below soft limit + if current >= self.soft_limit { + return None; // Reject immediately + } + } - if new_total <= limit { + // Check against hard limit + if new_total <= self.hard_limit { Some(new_total) } else { None @@ -533,9 +542,15 @@ impl QueryMemoryTracker { if let Some(callback) = &self.on_reject { callback(); } + // Distinguish soft vs hard limit rejection in error message + let limit = if is_initial && self.soft_limit > 0 && current >= self.soft_limit { + self.soft_limit + } else { + self.hard_limit + }; error::ExceedMemoryLimitSnafu { used: current, - limit: self.hard_limit, + limit, } .fail() } @@ -556,11 +571,19 @@ impl QueryMemoryTracker { } } +/// Window size for staged memory release. +/// Keeps the most recent N batches tracked, releases older ones progressively. +const MEMORY_RELEASE_WINDOW_SIZE: usize = 3; + /// A wrapper stream that tracks memory usage of RecordBatches. pub struct MemoryTrackedStream { inner: SendableRecordBatchStream, tracker: QueryMemoryTracker, - total_tracked: usize, + // Whether this is the first batch (for soft limit check). + is_initial: bool, + // Window of recent batch sizes for staged release. + // When the window is full, the oldest batch is released from tracking. + batch_window: VecDeque, } impl MemoryTrackedStream { @@ -568,7 +591,8 @@ impl MemoryTrackedStream { Self { inner, tracker, - total_tracked: 0, + is_initial: true, + batch_window: VecDeque::with_capacity(MEMORY_RELEASE_WINDOW_SIZE), } } } @@ -580,13 +604,21 @@ impl Stream for MemoryTrackedStream { match Pin::new(&mut self.inner).poll_next(cx) { Poll::Ready(Some(Ok(batch))) => { let size = batch.estimated_size(); - let is_initial = self.total_tracked == 0; - if let Err(e) = self.tracker.track(size, is_initial) { + if let Err(e) = self.tracker.track(size, self.is_initial) { return Poll::Ready(Some(Err(e))); } - self.total_tracked += size; + self.is_initial = false; + self.batch_window.push_back(size); + + // Staged release: if window is full, release the oldest batch + if self.batch_window.len() > MEMORY_RELEASE_WINDOW_SIZE + && let Some(old_size) = self.batch_window.pop_front() + { + self.tracker.release(old_size); + } + Poll::Ready(Some(Ok(batch))) } Poll::Ready(Some(Err(e))) => Poll::Ready(Some(Err(e))), @@ -598,7 +630,10 @@ impl Stream for MemoryTrackedStream { impl Drop for MemoryTrackedStream { fn drop(&mut self) { - self.tracker.release(self.total_tracked); + let remaining: usize = self.batch_window.iter().sum(); + if remaining > 0 { + self.tracker.release(remaining); + } } } @@ -734,16 +769,20 @@ mod tests { // With soft_limit_ratio = 0.5, soft limit is 500 let tracker = QueryMemoryTracker::new(1000, 0.5); - // First allocation (is_initial=true) checks soft limit + // First new query (is_initial=true): current=0 < soft_limit=500, allowed assert!(tracker.track(400, true).is_ok()); assert_eq!(tracker.current(), 400); - // New allocation exceeding soft limit should fail - assert!(tracker.track(200, true).is_err()); - assert_eq!(tracker.current(), 400); + // Second new query: current=400 < soft_limit=500, can allocate up to hard limit + assert!(tracker.track(400, true).is_ok()); + assert_eq!(tracker.current(), 800); + + // Third new query: current=800 >= soft_limit=500, rejected + assert!(tracker.track(100, true).is_err()); + assert_eq!(tracker.current(), 800); - // Existing stream (is_initial=false) can use up to hard limit - assert!(tracker.track(500, false).is_ok()); + // Existing stream (is_initial=false) can still allocate up to hard limit + assert!(tracker.track(100, false).is_ok()); assert_eq!(tracker.current(), 900); // Exceeding hard limit should fail even for existing stream From ed0427f614734454041c7a94becd23c94a5382a7 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Thu, 16 Oct 2025 21:27:14 +0800 Subject: [PATCH 12/19] fix: remove batch window Signed-off-by: jeremyhi --- src/common/recordbatch/src/error.rs | 8 ++++++- src/common/recordbatch/src/lib.rs | 34 ++++++++--------------------- 2 files changed, 16 insertions(+), 26 deletions(-) diff --git a/src/common/recordbatch/src/error.rs b/src/common/recordbatch/src/error.rs index 5e628a739771..8aa480c2eec9 100644 --- a/src/common/recordbatch/src/error.rs +++ b/src/common/recordbatch/src/error.rs @@ -194,8 +194,14 @@ pub enum Error { location: Location, }, - #[snafu(display("Exceeded memory limit: {} bytes used, {} bytes limit", used, limit))] + #[snafu(display( + "Exceeded memory limit: {} bytes requested, {} bytes used, {} bytes limit", + requested, + used, + limit + ))] ExceedMemoryLimit { + requested: usize, used: usize, limit: usize, #[snafu(implicit)] diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index 04de666fb29d..e839062117cc 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -21,7 +21,6 @@ pub mod filter; mod recordbatch; pub mod util; -use std::collections::VecDeque; use std::fmt; use std::pin::Pin; use std::sync::Arc; @@ -549,6 +548,7 @@ impl QueryMemoryTracker { self.hard_limit }; error::ExceedMemoryLimitSnafu { + requested: size, used: current, limit, } @@ -571,19 +571,12 @@ impl QueryMemoryTracker { } } -/// Window size for staged memory release. -/// Keeps the most recent N batches tracked, releases older ones progressively. -const MEMORY_RELEASE_WINDOW_SIZE: usize = 3; - /// A wrapper stream that tracks memory usage of RecordBatches. pub struct MemoryTrackedStream { inner: SendableRecordBatchStream, tracker: QueryMemoryTracker, - // Whether this is the first batch (for soft limit check). - is_initial: bool, - // Window of recent batch sizes for staged release. - // When the window is full, the oldest batch is released from tracking. - batch_window: VecDeque, + // Total tracked size, released when stream drops. + total_tracked: usize, } impl MemoryTrackedStream { @@ -591,8 +584,7 @@ impl MemoryTrackedStream { Self { inner, tracker, - is_initial: true, - batch_window: VecDeque::with_capacity(MEMORY_RELEASE_WINDOW_SIZE), + total_tracked: 0, } } } @@ -604,20 +596,13 @@ impl Stream for MemoryTrackedStream { match Pin::new(&mut self.inner).poll_next(cx) { Poll::Ready(Some(Ok(batch))) => { let size = batch.estimated_size(); + let is_initial = self.total_tracked == 0; - if let Err(e) = self.tracker.track(size, self.is_initial) { + if let Err(e) = self.tracker.track(size, is_initial) { return Poll::Ready(Some(Err(e))); } - self.is_initial = false; - self.batch_window.push_back(size); - - // Staged release: if window is full, release the oldest batch - if self.batch_window.len() > MEMORY_RELEASE_WINDOW_SIZE - && let Some(old_size) = self.batch_window.pop_front() - { - self.tracker.release(old_size); - } + self.total_tracked += size; Poll::Ready(Some(Ok(batch))) } @@ -630,9 +615,8 @@ impl Stream for MemoryTrackedStream { impl Drop for MemoryTrackedStream { fn drop(&mut self) { - let remaining: usize = self.batch_window.iter().sum(); - if remaining > 0 { - self.tracker.release(remaining); + if self.total_tracked > 0 { + self.tracker.release(self.total_tracked); } } } From c0e980230646ea627738841fa3286d00fa3b52ef Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Thu, 16 Oct 2025 22:03:42 +0800 Subject: [PATCH 13/19] chore: batch mem size Signed-off-by: jeremyhi --- src/common/recordbatch/src/lib.rs | 6 +++++- src/common/recordbatch/src/recordbatch.rs | 11 ----------- 2 files changed, 5 insertions(+), 12 deletions(-) diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index e839062117cc..9da2649c4e96 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -595,7 +595,11 @@ impl Stream for MemoryTrackedStream { fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { match Pin::new(&mut self.inner).poll_next(cx) { Poll::Ready(Some(Ok(batch))) => { - let size = batch.estimated_size(); + let size = batch + .columns() + .iter() + .map(|vec_ref| vec_ref.memory_size()) + .sum::(); let is_initial = self.total_tracked == 0; if let Err(e) = self.tracker.track(size, is_initial) { diff --git a/src/common/recordbatch/src/recordbatch.rs b/src/common/recordbatch/src/recordbatch.rs index 3664af93eb18..3cc30ce1ba78 100644 --- a/src/common/recordbatch/src/recordbatch.rs +++ b/src/common/recordbatch/src/recordbatch.rs @@ -241,17 +241,6 @@ impl RecordBatch { .unwrap_or("failed to pretty display a record batch".to_string()) } - /// Estimate the memory size of this record batch in bytes. - /// This uses the slice memory size which represents the actual memory used. - pub fn estimated_size(&self) -> usize { - self.df_record_batch - .columns() - .iter() - // If cannot get slice memory size, assume 0 - .map(|c| c.to_data().get_slice_memory_size().unwrap_or(0)) - .sum() - } - /// Return a slice record batch starts from offset, with len rows pub fn slice(&self, offset: usize, len: usize) -> Result { ensure!( From add9f2e80a8cd84346f04376fb9c908963318d94 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Fri, 17 Oct 2025 15:12:09 +0800 Subject: [PATCH 14/19] feat: refine limit algorithm Signed-off-by: jeremyhi --- Cargo.lock | 1 - config/config.md | 2 - config/datanode.example.toml | 5 - config/standalone.example.toml | 5 - src/cmd/tests/load_config_test.rs | 2 - src/common/recordbatch/src/error.rs | 14 +- src/common/recordbatch/src/lib.rs | 432 +++++++++++++++++++++------ src/mito2/Cargo.toml | 1 - src/mito2/src/config.rs | 6 - src/mito2/src/engine.rs | 30 +- src/mito2/src/read/seq_scan.rs | 3 +- src/mito2/src/read/series_scan.rs | 3 +- src/mito2/src/read/unordered_scan.rs | 3 +- 13 files changed, 367 insertions(+), 140 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 627a9c7de079..2945761e0938 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7643,7 +7643,6 @@ dependencies = [ "mito-codec", "moka", "object-store", - "ordered-float 4.6.0", "parquet", "partition", "paste", diff --git a/config/config.md b/config/config.md index c6f15c7e60b8..48427be475c0 100644 --- a/config/config.md +++ b/config/config.md @@ -154,7 +154,6 @@ | `region_engine.mito.max_concurrent_scan_files` | Integer | `384` | Maximum number of SST files to scan concurrently. | | `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. | | `region_engine.mito.scan_memory_limit` | String | `50%` | Memory limit for table scans across all queries.
Supports absolute size (e.g., "2GB") or percentage of system memory (e.g., "20%").
Setting it to 0 disables the limit. | -| `region_engine.mito.scan_memory_soft_limit_ratio` | Float | `0.7` | Soft limit ratio for scan memory (0.0 to 1.0).
When memory usage exceeds soft limit, new queries are rejected but existing queries continue.
This prevents thundering herd in high concurrency scenarios. | | `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.
To align with the old behavior, the default value is 0 (no restrictions). | | `region_engine.mito.default_experimental_flat_format` | Bool | `false` | Whether to enable experimental flat format as the default format. | | `region_engine.mito.index` | -- | -- | The options for index in Mito engine. | @@ -552,7 +551,6 @@ | `region_engine.mito.max_concurrent_scan_files` | Integer | `384` | Maximum number of SST files to scan concurrently. | | `region_engine.mito.allow_stale_entries` | Bool | `false` | Whether to allow stale WAL entries read during replay. | | `region_engine.mito.scan_memory_limit` | String | `50%` | Memory limit for table scans across all queries.
Supports absolute size (e.g., "2GB") or percentage of system memory (e.g., "20%").
Setting it to 0 disables the limit. | -| `region_engine.mito.scan_memory_soft_limit_ratio` | Float | `0.7` | Soft limit ratio for scan memory (0.0 to 1.0).
When memory usage exceeds soft limit, new queries are rejected but existing queries continue.
This prevents thundering herd in high concurrency scenarios. | | `region_engine.mito.min_compaction_interval` | String | `0m` | Minimum time interval between two compactions.
To align with the old behavior, the default value is 0 (no restrictions). | | `region_engine.mito.default_experimental_flat_format` | Bool | `false` | Whether to enable experimental flat format as the default format. | | `region_engine.mito.index` | -- | -- | The options for index in Mito engine. | diff --git a/config/datanode.example.toml b/config/datanode.example.toml index 47d3e6bd5ddd..bbf16963ae20 100644 --- a/config/datanode.example.toml +++ b/config/datanode.example.toml @@ -508,11 +508,6 @@ allow_stale_entries = false ## Setting it to 0 disables the limit. scan_memory_limit = "50%" -## Soft limit ratio for scan memory (0.0 to 1.0). -## When memory usage exceeds soft limit, new queries are rejected but existing queries continue. -## This prevents thundering herd in high concurrency scenarios. -scan_memory_soft_limit_ratio = 0.7 - ## Minimum time interval between two compactions. ## To align with the old behavior, the default value is 0 (no restrictions). min_compaction_interval = "0m" diff --git a/config/standalone.example.toml b/config/standalone.example.toml index 1646345dd556..41acb4d265e6 100644 --- a/config/standalone.example.toml +++ b/config/standalone.example.toml @@ -592,11 +592,6 @@ allow_stale_entries = false ## Setting it to 0 disables the limit. scan_memory_limit = "50%" -## Soft limit ratio for scan memory (0.0 to 1.0). -## When memory usage exceeds soft limit, new queries are rejected but existing queries continue. -## This prevents thundering herd in high concurrency scenarios. -scan_memory_soft_limit_ratio = 0.7 - ## Minimum time interval between two compactions. ## To align with the old behavior, the default value is 0 (no restrictions). min_compaction_interval = "0m" diff --git a/src/cmd/tests/load_config_test.rs b/src/cmd/tests/load_config_test.rs index fd265272be22..43d69e6c18ad 100644 --- a/src/cmd/tests/load_config_test.rs +++ b/src/cmd/tests/load_config_test.rs @@ -75,7 +75,6 @@ fn test_load_datanode_example_config() { auto_flush_interval: Duration::from_secs(3600), write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), scan_memory_limit: MemoryLimit::Percentage(0.5), - scan_memory_soft_limit_ratio: ordered_float::OrderedFloat(0.7), ..Default::default() }), RegionEngineConfig::File(FileEngineConfig {}), @@ -296,7 +295,6 @@ fn test_load_standalone_example_config() { auto_flush_interval: Duration::from_secs(3600), write_cache_ttl: Some(Duration::from_secs(60 * 60 * 8)), scan_memory_limit: MemoryLimit::Percentage(0.5), - scan_memory_soft_limit_ratio: ordered_float::OrderedFloat(0.7), ..Default::default() }), RegionEngineConfig::File(FileEngineConfig {}), diff --git a/src/common/recordbatch/src/error.rs b/src/common/recordbatch/src/error.rs index 8aa480c2eec9..729becbbe02f 100644 --- a/src/common/recordbatch/src/error.rs +++ b/src/common/recordbatch/src/error.rs @@ -195,14 +195,22 @@ pub enum Error { }, #[snafu(display( - "Exceeded memory limit: {} bytes requested, {} bytes used, {} bytes limit", + "Exceeded memory limit: {} bytes requested, {} bytes used globally ({}%), {} bytes used by this stream (privileged: {}), effective limit: {} bytes ({}%), hard limit: {} bytes", requested, - used, + global_used, + if *limit > 0 { global_used * 100 / limit } else { 0 }, + stream_used, + is_privileged, + effective_limit, + if *limit > 0 { effective_limit * 100 / limit } else { 0 }, limit ))] ExceedMemoryLimit { requested: usize, - used: usize, + global_used: usize, + stream_used: usize, + is_privileged: bool, + effective_limit: usize, limit: usize, #[snafu(implicit)] location: Location, diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index 9da2649c4e96..d4edcfa8bde0 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -24,7 +24,7 @@ pub mod util; use std::fmt; use std::pin::Pin; use std::sync::Arc; -use std::sync::atomic::{AtomicUsize, Ordering}; +use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use adapter::RecordBatchMetrics; use arc_swap::ArcSwapOption; @@ -408,16 +408,99 @@ impl> + Unpin> Stream for RecordBatchStream } } +/// Memory permit for a stream, providing privileged access or rate limiting. +/// +/// The permit tracks whether this stream has privileged Top-K status. +/// When dropped, it automatically releases any privileged slot it holds. +pub struct MemoryPermit { + tracker: QueryMemoryTracker, + is_privileged: AtomicBool, +} + +impl MemoryPermit { + /// Check if this permit currently has privileged status. + pub fn is_privileged(&self) -> bool { + self.is_privileged.load(Ordering::Acquire) + } + + /// Ensure this permit has privileged status by acquiring a slot if available. + /// Returns true if privileged (either already privileged or just acquired privilege). + fn ensure_privileged(&self) -> bool { + if self.is_privileged.load(Ordering::Acquire) { + return true; + } + + // Try to claim a privileged slot + self.tracker + .privileged_count + .fetch_update(Ordering::AcqRel, Ordering::Acquire, |count| { + if count < self.tracker.capacity { + Some(count + 1) + } else { + None + } + }) + .map(|_| { + self.is_privileged.store(true, Ordering::Release); + true + }) + .unwrap_or(false) + } + + /// Track additional memory usage with this permit. + /// Returns error if limit is exceeded. + /// + /// # Arguments + /// * `additional` - Additional memory size to track in bytes + /// * `stream_tracked` - Total memory already tracked by this stream + /// + /// # Top-K Behavior + /// - Privileged streams (Top-K): Can push global memory usage up to full limit + /// - Non-privileged streams: Can push global memory usage up to limit * non_privileged_ratio (default: 0.5) + /// - Non-privileged streams automatically attempt to acquire privilege if slots become available + /// - The configured limit is absolute hard limit - no stream can exceed it + pub fn track(&self, additional: usize, stream_tracked: usize) -> Result<()> { + // Ensure privileged status if possible + let is_privileged = self.ensure_privileged(); + + self.tracker + .track_internal(additional, is_privileged, stream_tracked) + } + + /// Release tracked memory. + /// + /// # Arguments + /// * `amount` - Amount of memory to release in bytes + pub fn release(&self, amount: usize) { + self.tracker.release(amount); + } +} + +impl Drop for MemoryPermit { + fn drop(&mut self) { + // Release privileged slot if we had one + if self.is_privileged.load(Ordering::Acquire) { + self.tracker + .privileged_count + .fetch_sub(1, Ordering::Release); + } + } +} + /// Memory tracker for RecordBatch streams. Clone to share the same limit across queries. /// -/// Uses soft and hard limit strategy to prevent thundering herd in high concurrency: -/// - Soft limit: Reject new allocations but allow existing streams to continue -/// - Hard limit: Reject all allocations +/// Uses Top-K first-come-first-served strategy: +/// - First K streams (default: 10) get privileged status and can use full limit +/// - Non-privileged streams use discounted limit (default: 0.5x, configurable) +/// - The configured limit is absolute hard limit - no stream can exceed it +/// - When privileged streams drop, waiting streams can promote to Top-K #[derive(Clone)] pub struct QueryMemoryTracker { current: Arc, - soft_limit: usize, - hard_limit: usize, + limit: usize, + non_privileged_ratio: f64, + privileged_count: Arc, + capacity: usize, on_update: Option>, on_reject: Option>, } @@ -425,9 +508,14 @@ pub struct QueryMemoryTracker { impl fmt::Debug for QueryMemoryTracker { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("QueryMemoryTracker") - .field("current", &self.current.load(Ordering::Relaxed)) - .field("soft_limit", &self.soft_limit) - .field("hard_limit", &self.hard_limit) + .field("current", &self.current.load(Ordering::Acquire)) + .field("limit", &self.limit) + .field("non_privileged_ratio", &self.non_privileged_ratio) + .field( + "privileged_count", + &self.privileged_count.load(Ordering::Acquire), + ) + .field("capacity", &self.capacity) .field("on_update", &self.on_update.is_some()) .field("on_reject", &self.on_reject.is_some()) .finish() @@ -435,29 +523,78 @@ impl fmt::Debug for QueryMemoryTracker { } impl QueryMemoryTracker { - /// Create a new memory tracker with the given hard limit (in bytes). + /// Default capacity for Top-K privileged streams. + pub const DEFAULT_CAPACITY: usize = 10; + /// Default memory ratio for non-privileged streams. + pub const DEFAULT_NON_PRIVILEGED_RATIO: f64 = 0.5; + + /// Create a new memory tracker with the given limit (in bytes). /// /// # Arguments - /// * `hard_limit` - Maximum memory usage in bytes. 0 means unlimited. - /// * `soft_limit_ratio` - Ratio of soft limit to hard limit (0.0 to 1.0). - /// When ratio <= 0, soft limit is disabled (all queries use hard limit). - /// When current usage exceeds soft limit, new allocations are rejected - /// but existing streams can continue until hard limit. - pub fn new(hard_limit: usize, soft_limit_ratio: f64) -> Self { - let soft_limit = if hard_limit > 0 && soft_limit_ratio > 0.0 { - (hard_limit as f64 * soft_limit_ratio.clamp(0.0, 1.0)) as usize - } else { - 0 - }; + /// * `limit` - Maximum memory usage in bytes (hard limit for all streams). 0 means unlimited. + /// * `non_privileged_ratio` - Memory ratio for non-privileged streams (default: 0.5). + /// Non-privileged streams can only use `limit * non_privileged_ratio`. + /// Privileged streams can use up to the full `limit`. + pub fn new(limit: usize) -> Self { + Self::with_capacity_and_ratio( + limit, + Self::DEFAULT_CAPACITY, + Self::DEFAULT_NON_PRIVILEGED_RATIO, + ) + } + + /// Create a new memory tracker with custom Top-K capacity. + pub fn with_capacity(limit: usize, capacity: usize) -> Self { + Self::with_capacity_and_ratio(limit, capacity, Self::DEFAULT_NON_PRIVILEGED_RATIO) + } + + /// Create a new memory tracker with custom Top-K capacity and non-privileged ratio. + /// + /// # Panics + /// Panics if `non_privileged_ratio` is not in the range [0.0, 1.0]. + pub fn with_capacity_and_ratio( + limit: usize, + capacity: usize, + non_privileged_ratio: f64, + ) -> Self { + assert!( + (0.0..=1.0).contains(&non_privileged_ratio), + "non_privileged_ratio must be in [0.0, 1.0], got {}", + non_privileged_ratio + ); + Self { current: Arc::new(AtomicUsize::new(0)), - soft_limit, - hard_limit, + limit, + non_privileged_ratio, + privileged_count: Arc::new(AtomicUsize::new(0)), + capacity, on_update: None, on_reject: None, } } + /// Register a new stream and get a memory permit. + /// The first `capacity` streams get privileged status automatically. + pub fn register_stream(&self) -> MemoryPermit { + // Try to claim a privileged slot + let is_privileged = self + .privileged_count + .fetch_update(Ordering::AcqRel, Ordering::Acquire, |count| { + if count < self.capacity { + Some(count + 1) + } else { + None + } + }) + .is_ok(); + + MemoryPermit { + tracker: self.clone(), + is_privileged: AtomicBool::new(is_privileged), + } + } + /// Set a callback to be called whenever the usage changes successfully. /// The callback receives the new total usage in bytes. /// @@ -487,43 +624,41 @@ impl QueryMemoryTracker { /// Get the current memory usage in bytes. pub fn current(&self) -> usize { - self.current.load(Ordering::Relaxed) + self.current.load(Ordering::Acquire) } - /// Track memory usage. Returns error if limit is exceeded. - /// - /// # Arguments - /// * `size` - Memory size to track in bytes - /// * `is_initial` - True for first allocation of a stream, false for subsequent allocations - /// - /// # Soft Limit Behavior - /// When `is_initial` is true and soft limit is enabled: - /// - If current usage < soft limit: allow the allocation (checked against hard limit) - /// - If current usage >= soft limit: reject the allocation immediately + /// Internal method to track additional memory usage. /// - /// This ensures that new queries are only accepted when memory pressure is low, - /// while existing queries can continue until hard limit is reached. - pub fn track(&self, size: usize, is_initial: bool) -> Result<()> { + /// Called by `MemoryPermit::track()`. Use `MemoryPermit::track()` instead of calling this directly. + fn track_internal( + &self, + additional: usize, + is_privileged: bool, + stream_tracked: usize, + ) -> Result<()> { + // Calculate effective global limit based on stream privilege + // Privileged streams: can push global usage up to full limit + // Non-privileged streams: can only push global usage up to discounted limit + let effective_limit = if is_privileged { + self.limit + } else { + (self.limit as f64 * self.non_privileged_ratio) as usize + }; + let mut new_total = 0; let result = self .current - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { - new_total = current.saturating_add(size); + .fetch_update(Ordering::AcqRel, Ordering::Acquire, |current| { + new_total = current.saturating_add(additional); - if self.hard_limit == 0 { + if self.limit == 0 { // Unlimited mode return Some(new_total); } - if is_initial && self.soft_limit > 0 { - // New query: check if current usage is below soft limit - if current >= self.soft_limit { - return None; // Reject immediately - } - } - - // Check against hard limit - if new_total <= self.hard_limit { + // Check if new global total exceeds effective limit + // The configured limit is absolute hard limit - no stream can exceed it + if new_total <= effective_limit { Some(new_total) } else { None @@ -541,16 +676,13 @@ impl QueryMemoryTracker { if let Some(callback) = &self.on_reject { callback(); } - // Distinguish soft vs hard limit rejection in error message - let limit = if is_initial && self.soft_limit > 0 && current >= self.soft_limit { - self.soft_limit - } else { - self.hard_limit - }; error::ExceedMemoryLimitSnafu { - requested: size, - used: current, - limit, + requested: additional, + global_used: current, + stream_used: stream_tracked, + is_privileged, + effective_limit, + limit: self.limit, } .fail() } @@ -558,15 +690,18 @@ impl QueryMemoryTracker { } /// Release tracked memory. - pub fn release(&self, size: usize) { + /// + /// # Arguments + /// * `amount` - Amount of memory to release in bytes + pub fn release(&self, amount: usize) { if let Ok(old_value) = self.current - .fetch_update(Ordering::Relaxed, Ordering::Relaxed, |current| { - Some(current.saturating_sub(size)) + .fetch_update(Ordering::AcqRel, Ordering::Acquire, |current| { + Some(current.saturating_sub(amount)) }) && let Some(callback) = &self.on_update { - callback(old_value.saturating_sub(size)); + callback(old_value.saturating_sub(amount)); } } } @@ -574,16 +709,16 @@ impl QueryMemoryTracker { /// A wrapper stream that tracks memory usage of RecordBatches. pub struct MemoryTrackedStream { inner: SendableRecordBatchStream, - tracker: QueryMemoryTracker, + permit: MemoryPermit, // Total tracked size, released when stream drops. total_tracked: usize, } impl MemoryTrackedStream { - pub fn new(inner: SendableRecordBatchStream, tracker: QueryMemoryTracker) -> Self { + pub fn new(inner: SendableRecordBatchStream, permit: MemoryPermit) -> Self { Self { inner, - tracker, + permit, total_tracked: 0, } } @@ -600,9 +735,8 @@ impl Stream for MemoryTrackedStream { .iter() .map(|vec_ref| vec_ref.memory_size()) .sum::(); - let is_initial = self.total_tracked == 0; - if let Err(e) = self.tracker.track(size, is_initial) { + if let Err(e) = self.permit.track(size, self.total_tracked) { return Poll::Ready(Some(Err(e))); } @@ -615,12 +749,16 @@ impl Stream for MemoryTrackedStream { Poll::Pending => Poll::Pending, } } + + fn size_hint(&self) -> (usize, Option) { + self.inner.size_hint() + } } impl Drop for MemoryTrackedStream { fn drop(&mut self) { if self.total_tracked > 0 { - self.tracker.release(self.total_tracked); + self.permit.release(self.total_tracked); } } } @@ -731,53 +869,155 @@ mod tests { } #[test] - fn test_query_memory_tracker_zero_soft_limit_ratio() { - // When soft_limit_ratio is 0, soft limit should be disabled - // and all queries should use hard limit - let tracker = QueryMemoryTracker::new(1000, 0.0); + fn test_query_memory_tracker_basic() { + let tracker = Arc::new(QueryMemoryTracker::new(1000)); + + // Register first stream - should get privileged status + let permit1 = tracker.register_stream(); + assert!(permit1.is_privileged()); - // First allocation (is_initial=true) should succeed up to hard limit - assert!(tracker.track(500, true).is_ok()); + // Privileged stream can use up to limit + assert!(permit1.track(500, 0).is_ok()); assert_eq!(tracker.current(), 500); - // Second allocation should also succeed - assert!(tracker.track(400, true).is_ok()); + // Register second stream - also privileged + let permit2 = tracker.register_stream(); + assert!(permit2.is_privileged()); + // Can add more but cannot exceed hard limit (1000) + assert!(permit2.track(400, 0).is_ok()); assert_eq!(tracker.current(), 900); - // Exceeding hard limit should fail - assert!(tracker.track(200, true).is_err()); - assert_eq!(tracker.current(), 900); + permit1.release(500); + permit2.release(400); + assert_eq!(tracker.current(), 0); + } - tracker.release(900); + #[test] + fn test_query_memory_tracker_top_k_limit() { + // Capacity = 2 for easy testing + // Limit: 1000, non-privileged ratio: 0.5 + // Privileged can push global to 1000, non-privileged can push global to 500 + let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 2)); + + // First 2 streams are privileged + let permit1 = tracker.register_stream(); + let permit2 = tracker.register_stream(); + assert!(permit1.is_privileged()); + assert!(permit2.is_privileged()); + + // Third stream is not privileged + let permit3 = tracker.register_stream(); + assert!(!permit3.is_privileged()); + + // Privileged stream uses some memory + assert!(permit1.track(300, 0).is_ok()); + assert_eq!(tracker.current(), 300); + + // Non-privileged can add up to 200 (total becomes 500, its effective limit) + assert!(permit3.track(200, 0).is_ok()); + assert_eq!(tracker.current(), 500); + + // Non-privileged stream cannot push global beyond 500 + let err = permit3.track(100, 200).unwrap_err(); + let err_msg = err.to_string(); + assert!(err_msg.contains("200 bytes used by this stream")); + assert!(err_msg.contains("effective limit: 500 bytes (50%)")); + assert!(err_msg.contains("500 bytes used globally (50%)")); + assert_eq!(tracker.current(), 500); + + permit1.release(300); + permit3.release(200); assert_eq!(tracker.current(), 0); } #[test] - fn test_query_memory_tracker_with_soft_limit() { - // With soft_limit_ratio = 0.5, soft limit is 500 - let tracker = QueryMemoryTracker::new(1000, 0.5); + fn test_query_memory_tracker_promotion() { + // Capacity = 1 for easy testing + let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 1)); + + // First stream is privileged + let permit1 = tracker.register_stream(); + assert!(permit1.is_privileged()); - // First new query (is_initial=true): current=0 < soft_limit=500, allowed - assert!(tracker.track(400, true).is_ok()); + // Second stream is not privileged (can only use 500) + let permit2 = tracker.register_stream(); + assert!(!permit2.is_privileged()); + + // Non-privileged can only track 500 + assert!(permit2.track(400, 0).is_ok()); assert_eq!(tracker.current(), 400); - // Second new query: current=400 < soft_limit=500, can allocate up to hard limit - assert!(tracker.track(400, true).is_ok()); - assert_eq!(tracker.current(), 800); + // Drop first permit to release privileged slot + drop(permit1); + + // Second stream can now be promoted and use more memory + assert!(permit2.track(500, 400).is_ok()); + assert!(permit2.is_privileged()); + assert_eq!(tracker.current(), 900); - // Third new query: current=800 >= soft_limit=500, rejected - assert!(tracker.track(100, true).is_err()); - assert_eq!(tracker.current(), 800); + permit2.release(900); + assert_eq!(tracker.current(), 0); + } + + #[test] + fn test_query_memory_tracker_privileged_hard_limit() { + // Test that the configured limit is absolute hard limit for all streams + // Privileged: can use full limit (1000) + // Non-privileged: can use 0.5x limit (500) + let tracker = Arc::new(QueryMemoryTracker::new(1000)); - // Existing stream (is_initial=false) can still allocate up to hard limit - assert!(tracker.track(100, false).is_ok()); + let permit1 = tracker.register_stream(); + assert!(permit1.is_privileged()); + + // Privileged can use up to full limit (1000) + assert!(permit1.track(900, 0).is_ok()); assert_eq!(tracker.current(), 900); - // Exceeding hard limit should fail even for existing stream - assert!(tracker.track(200, false).is_err()); + // Privileged cannot exceed hard limit (1000) + assert!(permit1.track(200, 900).is_err()); assert_eq!(tracker.current(), 900); - tracker.release(900); + // Can add within hard limit + assert!(permit1.track(100, 900).is_ok()); + assert_eq!(tracker.current(), 1000); + + // Cannot exceed even by 1 byte + assert!(permit1.track(1, 1000).is_err()); + assert_eq!(tracker.current(), 1000); + + permit1.release(1000); + assert_eq!(tracker.current(), 0); + } + + #[test] + fn test_query_memory_tracker_non_privileged_ratio() { + // Test non-privileged streams use discounted limit + // Limit: 1000, ratio: 0.5, so non-privileged can use 500 + let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 1)); + + let permit1 = tracker.register_stream(); + assert!(permit1.is_privileged()); + + let permit2 = tracker.register_stream(); + assert!(!permit2.is_privileged()); + + // Non-privileged can use up to 500 (1000 * 0.5) + assert!(permit2.track(400, 0).is_ok()); + assert_eq!(tracker.current(), 400); + + // Cannot exceed discounted limit (500) + assert!(permit2.track(200, 400).is_err()); + assert_eq!(tracker.current(), 400); + + // Can add within discounted limit + assert!(permit2.track(100, 400).is_ok()); + assert_eq!(tracker.current(), 500); + + // Cannot exceed discounted limit + assert!(permit2.track(1, 500).is_err()); + assert_eq!(tracker.current(), 500); + + permit2.release(500); assert_eq!(tracker.current(), 0); } } diff --git a/src/mito2/Cargo.toml b/src/mito2/Cargo.toml index 4ae8b107f90b..4cc1efb8bc1e 100644 --- a/src/mito2/Cargo.toml +++ b/src/mito2/Cargo.toml @@ -56,7 +56,6 @@ log-store = { workspace = true } mito-codec.workspace = true moka = { workspace = true, features = ["sync", "future"] } object-store.workspace = true -ordered-float.workspace = true parquet = { workspace = true, features = ["async"] } paste.workspace = true pin-project.workspace = true diff --git a/src/mito2/src/config.rs b/src/mito2/src/config.rs index 188295e3e52e..ad895fa9cdf9 100644 --- a/src/mito2/src/config.rs +++ b/src/mito2/src/config.rs @@ -22,7 +22,6 @@ use common_base::memory_limit::MemoryLimit; use common_base::readable_size::ReadableSize; use common_stat::{get_total_cpu_cores, get_total_memory_readable}; use common_telemetry::warn; -use ordered_float::OrderedFloat; use serde::{Deserialize, Serialize}; use serde_with::serde_as; @@ -132,10 +131,6 @@ pub struct MitoConfig { /// Memory limit for table scans across all queries. Setting it to 0 disables the limit. /// Supports absolute size (e.g., "2GB") or percentage (e.g., "50%"). pub scan_memory_limit: MemoryLimit, - /// Soft limit ratio for scan memory (0.0 to 1.0). - /// When memory usage exceeds soft limit, new queries are rejected but existing queries continue. - /// This helps prevent thundering herd in high concurrency scenarios. - pub scan_memory_soft_limit_ratio: OrderedFloat, /// Index configs. pub index: IndexConfig, @@ -189,7 +184,6 @@ impl Default for MitoConfig { max_concurrent_scan_files: DEFAULT_MAX_CONCURRENT_SCAN_FILES, allow_stale_entries: false, scan_memory_limit: MemoryLimit::default(), - scan_memory_soft_limit_ratio: OrderedFloat(0.7), index: IndexConfig::default(), inverted_index: InvertedIndexConfig::default(), fulltext_index: FulltextIndexConfig::default(), diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index b42ce75846e7..3b5b5acdbc4e 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -202,14 +202,13 @@ impl<'a, S: LogStore> MitoEngineBuilder<'a, S> { let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(self.log_store)); let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize; - let scan_memory_tracker = - QueryMemoryTracker::new(scan_memory_limit, config.scan_memory_soft_limit_ratio.0) - .with_update_callback(|usage| { - SCAN_MEMORY_USAGE_BYTES.set(usage as i64); - }) - .with_reject_callback(|| { - SCAN_REQUESTS_REJECTED_TOTAL.inc(); - }); + let scan_memory_tracker = QueryMemoryTracker::new(scan_memory_limit) + .with_update_callback(|usage| { + SCAN_MEMORY_USAGE_BYTES.set(usage as i64); + }) + .with_reject_callback(|| { + SCAN_REQUESTS_REJECTED_TOTAL.inc(); + }); let inner = EngineInner { workers, @@ -1093,14 +1092,13 @@ impl MitoEngine { let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(log_store.clone())); let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize; - let scan_memory_tracker = - QueryMemoryTracker::new(scan_memory_limit, config.scan_memory_soft_limit_ratio.0) - .with_update_callback(|usage| { - SCAN_MEMORY_USAGE_BYTES.set(usage as i64); - }) - .with_reject_callback(|| { - SCAN_REQUESTS_REJECTED_TOTAL.inc(); - }); + let scan_memory_tracker = QueryMemoryTracker::new(scan_memory_limit) + .with_update_callback(|usage| { + SCAN_MEMORY_USAGE_BYTES.set(usage as i64); + }) + .with_reject_callback(|| { + SCAN_REQUESTS_REJECTED_TOTAL.inc(); + }); Ok(MitoEngine { inner: Arc::new(EngineInner { workers: WorkerGroup::start_for_test( diff --git a/src/mito2/src/read/seq_scan.rs b/src/mito2/src/read/seq_scan.rs index 150ec3b009cd..4c9f225712a7 100644 --- a/src/mito2/src/read/seq_scan.rs +++ b/src/mito2/src/read/seq_scan.rs @@ -359,7 +359,8 @@ impl SeqScan { Ok( if let Some(tracker) = &self.stream_ctx.query_memory_tracker { - Box::pin(MemoryTrackedStream::new(stream, tracker.clone())) + let permit = tracker.register_stream(); + Box::pin(MemoryTrackedStream::new(stream, permit)) } else { stream }, diff --git a/src/mito2/src/read/series_scan.rs b/src/mito2/src/read/series_scan.rs index 32a19797a6b7..1b9358f88b55 100644 --- a/src/mito2/src/read/series_scan.rs +++ b/src/mito2/src/read/series_scan.rs @@ -125,7 +125,8 @@ impl SeriesScan { Ok( if let Some(tracker) = &self.stream_ctx.query_memory_tracker { - Box::pin(MemoryTrackedStream::new(stream, tracker.clone())) + let permit = tracker.register_stream(); + Box::pin(MemoryTrackedStream::new(stream, permit)) } else { stream }, diff --git a/src/mito2/src/read/unordered_scan.rs b/src/mito2/src/read/unordered_scan.rs index 1522e8bcd311..406e64b30a2c 100644 --- a/src/mito2/src/read/unordered_scan.rs +++ b/src/mito2/src/read/unordered_scan.rs @@ -258,7 +258,8 @@ impl UnorderedScan { Ok( if let Some(tracker) = &self.stream_ctx.query_memory_tracker { - Box::pin(MemoryTrackedStream::new(stream, tracker.clone())) + let permit = tracker.register_stream(); + Box::pin(MemoryTrackedStream::new(stream, permit)) } else { stream }, From 6b0dfce6e921dc65bc5959679c98d7476ae255b8 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Fri, 17 Oct 2025 15:43:59 +0800 Subject: [PATCH 15/19] fix: get sys mem Signed-off-by: jeremyhi --- Cargo.lock | 1 + src/mito2/src/engine.rs | 6 +++--- src/query/Cargo.toml | 1 + src/query/src/query_engine/state.rs | 4 ++-- 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2945761e0938..3e36dbd0bcdc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10138,6 +10138,7 @@ dependencies = [ "common-query", "common-recordbatch", "common-runtime", + "common-stat", "common-telemetry", "common-time", "datafusion", diff --git a/src/mito2/src/engine.rs b/src/mito2/src/engine.rs index 3b5b5acdbc4e..2383a092f76b 100644 --- a/src/mito2/src/engine.rs +++ b/src/mito2/src/engine.rs @@ -77,10 +77,10 @@ use std::time::Instant; use api::region::RegionResponse; use async_trait::async_trait; use common_base::Plugins; -use common_config::utils::get_sys_total_memory; use common_error::ext::BoxedError; use common_meta::key::SchemaMetadataManagerRef; use common_recordbatch::{QueryMemoryTracker, SendableRecordBatchStream}; +use common_stat::get_total_memory_bytes; use common_telemetry::{info, tracing, warn}; use common_wal::options::{WAL_OPTIONS_KEY, WalOptions}; use futures::future::{join_all, try_join_all}; @@ -200,7 +200,7 @@ impl<'a, S: LogStore> MitoEngineBuilder<'a, S> { ) .await?; let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(self.log_store)); - let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); + let total_memory = get_total_memory_bytes().max(0) as u64; let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize; let scan_memory_tracker = QueryMemoryTracker::new(scan_memory_limit) .with_update_callback(|usage| { @@ -1090,7 +1090,7 @@ impl MitoEngine { let config = Arc::new(config); let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(log_store.clone())); - let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); + let total_memory = get_total_memory_bytes().max(0) as u64; let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize; let scan_memory_tracker = QueryMemoryTracker::new(scan_memory_limit) .with_update_callback(|usage| { diff --git a/src/query/Cargo.toml b/src/query/Cargo.toml index e92eabb142fd..d5f61409a4e1 100644 --- a/src/query/Cargo.toml +++ b/src/query/Cargo.toml @@ -34,6 +34,7 @@ common-plugins.workspace = true common-query.workspace = true common-recordbatch.workspace = true common-runtime.workspace = true +common-stat.workspace = true common-telemetry.workspace = true common-time.workspace = true datafusion.workspace = true diff --git a/src/query/src/query_engine/state.rs b/src/query/src/query_engine/state.rs index 17dde0a3a251..ffe64e700575 100644 --- a/src/query/src/query_engine/state.rs +++ b/src/query/src/query_engine/state.rs @@ -19,13 +19,13 @@ use std::sync::{Arc, RwLock}; use async_trait::async_trait; use catalog::CatalogManagerRef; use common_base::Plugins; -use common_config::utils::get_sys_total_memory; use common_function::aggrs::aggr_wrapper::fix_order::FixStateUdafOrderingAnalyzer; use common_function::function_factory::ScalarFunctionFactory; use common_function::handlers::{ FlowServiceHandlerRef, ProcedureServiceHandlerRef, TableMutationHandlerRef, }; use common_function::state::FunctionState; +use common_stat::get_total_memory_bytes; use common_telemetry::warn; use datafusion::catalog::TableFunction; use datafusion::dataframe::DataFrame; @@ -105,7 +105,7 @@ impl QueryEngineState { plugins: Plugins, options: QueryOptionsNew, ) -> Self { - let total_memory = get_sys_total_memory().map(|s| s.as_bytes()).unwrap_or(0); + let total_memory = get_total_memory_bytes().max(0) as u64; let memory_pool_size = options.memory_pool_size.resolve(total_memory) as usize; let runtime_env = if memory_pool_size > 0 { Arc::new( From 690a90916507a9144e6025206a5134030f612253 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Fri, 17 Oct 2025 16:12:53 +0800 Subject: [PATCH 16/19] chore: minor change --- Cargo.lock | 1 - src/query/Cargo.toml | 1 - src/query/src/options.rs | 2 +- tests-integration/tests/http.rs | 1 - 4 files changed, 1 insertion(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3e36dbd0bcdc..6a85892b8ffe 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10128,7 +10128,6 @@ dependencies = [ "chrono", "common-base", "common-catalog", - "common-config", "common-datasource", "common-error", "common-function", diff --git a/src/query/Cargo.toml b/src/query/Cargo.toml index d5f61409a4e1..0da5535bfbdb 100644 --- a/src/query/Cargo.toml +++ b/src/query/Cargo.toml @@ -24,7 +24,6 @@ catalog.workspace = true chrono.workspace = true common-base.workspace = true common-catalog.workspace = true -common-config.workspace = true common-datasource.workspace = true common-error.workspace = true common-function.workspace = true diff --git a/src/query/src/options.rs b/src/query/src/options.rs index d9e9c7ebdfd4..50ca1177a56f 100644 --- a/src/query/src/options.rs +++ b/src/query/src/options.rs @@ -16,7 +16,7 @@ use common_base::memory_limit::MemoryLimit; use serde::{Deserialize, Serialize}; /// Query engine config -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] #[serde(default)] pub struct QueryOptions { /// Parallelism of query engine. Default to 0, which implies the number of logical CPUs. diff --git a/tests-integration/tests/http.rs b/tests-integration/tests/http.rs index 7beb09ffb4c4..1a8de5f41671 100644 --- a/tests-integration/tests/http.rs +++ b/tests-integration/tests/http.rs @@ -1579,7 +1579,6 @@ fn drop_lines_with_inconsistent_results(input: String) -> String { "cache_capacity =", "memory_pool_size =", "scan_memory_limit =", - "scan_memory_soft_limit_ratio =", "sas_token =", "scope =", "num_workers =", From 7c6de18ced740ca2ca84e9d7fcff70335697142a Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Fri, 17 Oct 2025 16:30:50 +0800 Subject: [PATCH 17/19] feat: up tracker to the top stream --- src/common/recordbatch/src/error.rs | 13 +++++----- src/common/recordbatch/src/lib.rs | 37 ++++++++++++++-------------- src/mito2/src/read/scan_region.rs | 14 ++--------- src/mito2/src/read/seq_scan.rs | 24 ++++++++++-------- src/mito2/src/read/series_scan.rs | 24 ++++++++++-------- src/mito2/src/read/unordered_scan.rs | 27 ++++++++++---------- 6 files changed, 70 insertions(+), 69 deletions(-) diff --git a/src/common/recordbatch/src/error.rs b/src/common/recordbatch/src/error.rs index 729becbbe02f..88d79d570e24 100644 --- a/src/common/recordbatch/src/error.rs +++ b/src/common/recordbatch/src/error.rs @@ -15,6 +15,7 @@ //! Error of record batch. use std::any::Any; +use common_base::readable_size::ReadableSize; use common_error::ext::{BoxedError, ErrorExt}; use common_error::status_code::StatusCode; use common_macro::stack_trace_debug; @@ -195,15 +196,15 @@ pub enum Error { }, #[snafu(display( - "Exceeded memory limit: {} bytes requested, {} bytes used globally ({}%), {} bytes used by this stream (privileged: {}), effective limit: {} bytes ({}%), hard limit: {} bytes", - requested, - global_used, + "Exceeded memory limit: {} requested, {} used globally ({}%), {} used by this stream (privileged: {}), effective limit: {} ({}%), hard limit: {}", + ReadableSize(*requested as u64), + ReadableSize(*global_used as u64), if *limit > 0 { global_used * 100 / limit } else { 0 }, - stream_used, + ReadableSize(*stream_used as u64), is_privileged, - effective_limit, + ReadableSize(*effective_limit as u64), if *limit > 0 { effective_limit * 100 / limit } else { 0 }, - limit + ReadableSize(*limit as u64) ))] ExceedMemoryLimit { requested: usize, diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index d4edcfa8bde0..9dce99c03d9e 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -574,9 +574,10 @@ impl QueryMemoryTracker { } } - /// Register a new stream and get a memory permit. - /// The first `capacity` streams get privileged status automatically. - pub fn register_stream(&self) -> MemoryPermit { + /// Register a new permit for memory tracking. + /// The first `capacity` permits get privileged status automatically. + /// The returned permit can be shared across multiple streams of the same query. + pub fn register_permit(&self) -> MemoryPermit { // Try to claim a privileged slot let is_privileged = self .privileged_count @@ -709,13 +710,13 @@ impl QueryMemoryTracker { /// A wrapper stream that tracks memory usage of RecordBatches. pub struct MemoryTrackedStream { inner: SendableRecordBatchStream, - permit: MemoryPermit, + permit: Arc, // Total tracked size, released when stream drops. total_tracked: usize, } impl MemoryTrackedStream { - pub fn new(inner: SendableRecordBatchStream, permit: MemoryPermit) -> Self { + pub fn new(inner: SendableRecordBatchStream, permit: Arc) -> Self { Self { inner, permit, @@ -873,7 +874,7 @@ mod tests { let tracker = Arc::new(QueryMemoryTracker::new(1000)); // Register first stream - should get privileged status - let permit1 = tracker.register_stream(); + let permit1 = tracker.register_permit(); assert!(permit1.is_privileged()); // Privileged stream can use up to limit @@ -881,7 +882,7 @@ mod tests { assert_eq!(tracker.current(), 500); // Register second stream - also privileged - let permit2 = tracker.register_stream(); + let permit2 = tracker.register_permit(); assert!(permit2.is_privileged()); // Can add more but cannot exceed hard limit (1000) assert!(permit2.track(400, 0).is_ok()); @@ -900,13 +901,13 @@ mod tests { let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 2)); // First 2 streams are privileged - let permit1 = tracker.register_stream(); - let permit2 = tracker.register_stream(); + let permit1 = tracker.register_permit(); + let permit2 = tracker.register_permit(); assert!(permit1.is_privileged()); assert!(permit2.is_privileged()); // Third stream is not privileged - let permit3 = tracker.register_stream(); + let permit3 = tracker.register_permit(); assert!(!permit3.is_privileged()); // Privileged stream uses some memory @@ -920,9 +921,9 @@ mod tests { // Non-privileged stream cannot push global beyond 500 let err = permit3.track(100, 200).unwrap_err(); let err_msg = err.to_string(); - assert!(err_msg.contains("200 bytes used by this stream")); - assert!(err_msg.contains("effective limit: 500 bytes (50%)")); - assert!(err_msg.contains("500 bytes used globally (50%)")); + assert!(err_msg.contains("200B used by this stream")); + assert!(err_msg.contains("effective limit: 500B (50%)")); + assert!(err_msg.contains("500B used globally (50%)")); assert_eq!(tracker.current(), 500); permit1.release(300); @@ -936,11 +937,11 @@ mod tests { let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 1)); // First stream is privileged - let permit1 = tracker.register_stream(); + let permit1 = tracker.register_permit(); assert!(permit1.is_privileged()); // Second stream is not privileged (can only use 500) - let permit2 = tracker.register_stream(); + let permit2 = tracker.register_permit(); assert!(!permit2.is_privileged()); // Non-privileged can only track 500 @@ -966,7 +967,7 @@ mod tests { // Non-privileged: can use 0.5x limit (500) let tracker = Arc::new(QueryMemoryTracker::new(1000)); - let permit1 = tracker.register_stream(); + let permit1 = tracker.register_permit(); assert!(permit1.is_privileged()); // Privileged can use up to full limit (1000) @@ -995,10 +996,10 @@ mod tests { // Limit: 1000, ratio: 0.5, so non-privileged can use 500 let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 1)); - let permit1 = tracker.register_stream(); + let permit1 = tracker.register_permit(); assert!(permit1.is_privileged()); - let permit2 = tracker.register_stream(); + let permit2 = tracker.register_permit(); assert!(!permit2.is_privileged()); // Non-privileged can use up to 500 (1000 * 0.5) diff --git a/src/mito2/src/read/scan_region.rs b/src/mito2/src/read/scan_region.rs index d90833c6d97d..8e4139374462 100644 --- a/src/mito2/src/read/scan_region.rs +++ b/src/mito2/src/read/scan_region.rs @@ -1177,16 +1177,11 @@ pub struct StreamContext { // Metrics: /// The start time of the query. pub(crate) query_start: Instant, - /// Memory tracker for the query. - pub(crate) query_memory_tracker: Option, } impl StreamContext { /// Creates a new [StreamContext] for [SeqScan]. - pub(crate) fn seq_scan_ctx( - input: ScanInput, - query_memory_tracker: Option, - ) -> Self { + pub(crate) fn seq_scan_ctx(input: ScanInput) -> Self { 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); @@ -1195,15 +1190,11 @@ impl StreamContext { input, ranges, query_start, - query_memory_tracker, } } /// Creates a new [StreamContext] for [UnorderedScan]. - pub(crate) fn unordered_scan_ctx( - input: ScanInput, - query_memory_tracker: Option, - ) -> Self { + pub(crate) fn unordered_scan_ctx(input: ScanInput) -> Self { 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); @@ -1212,7 +1203,6 @@ impl StreamContext { input, ranges, query_start, - query_memory_tracker, } } diff --git a/src/mito2/src/read/seq_scan.rs b/src/mito2/src/read/seq_scan.rs index 4c9f225712a7..b8cec43f4ad8 100644 --- a/src/mito2/src/read/seq_scan.rs +++ b/src/mito2/src/read/seq_scan.rs @@ -22,7 +22,8 @@ use async_stream::try_stream; use common_error::ext::BoxedError; use common_recordbatch::util::ChainedRecordBatchStream; use common_recordbatch::{ - MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, SendableRecordBatchStream, + MemoryPermit, MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, + SendableRecordBatchStream, }; use common_telemetry::tracing; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; @@ -68,6 +69,8 @@ pub struct SeqScan { /// Metrics for each partition. /// The scanner only sets in query and keeps it empty during compaction. metrics_list: PartitionMetricsList, + /// Memory permit for the entire query (shared across all partitions). + query_memory_permit: Option>, } impl SeqScan { @@ -77,13 +80,17 @@ impl SeqScan { let mut properties = ScannerProperties::default() .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input, query_memory_tracker)); + let query_memory_permit = query_memory_tracker + .as_ref() + .map(|t| Arc::new(t.register_permit())); + let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input)); properties.partitions = vec![stream_ctx.partition_ranges()]; Self { properties, stream_ctx, metrics_list: PartitionMetricsList::default(), + query_memory_permit, } } @@ -357,14 +364,11 @@ impl SeqScan { Box::pin(record_batch_stream), )); - Ok( - if let Some(tracker) = &self.stream_ctx.query_memory_tracker { - let permit = tracker.register_stream(); - Box::pin(MemoryTrackedStream::new(stream, permit)) - } else { - stream - }, - ) + Ok(if let Some(permit) = &self.query_memory_permit { + Box::pin(MemoryTrackedStream::new(stream, permit.clone())) + } else { + stream + }) } fn scan_batch_in_partition( diff --git a/src/mito2/src/read/series_scan.rs b/src/mito2/src/read/series_scan.rs index 1b9358f88b55..c5d160717d90 100644 --- a/src/mito2/src/read/series_scan.rs +++ b/src/mito2/src/read/series_scan.rs @@ -22,7 +22,8 @@ use async_stream::try_stream; use common_error::ext::BoxedError; use common_recordbatch::util::ChainedRecordBatchStream; use common_recordbatch::{ - MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, SendableRecordBatchStream, + MemoryPermit, MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, + SendableRecordBatchStream, }; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::{DisplayAs, DisplayFormatType}; @@ -74,6 +75,8 @@ pub struct SeriesScan { /// Metrics for each partition. /// The scanner only sets in query and keeps it empty during compaction. metrics_list: Arc, + /// Memory permit for the entire query (shared across all partitions). + query_memory_permit: Option>, } impl SeriesScan { @@ -82,7 +85,10 @@ impl SeriesScan { let mut properties = ScannerProperties::default() .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input, query_memory_tracker)); + let query_memory_permit = query_memory_tracker + .as_ref() + .map(|t| Arc::new(t.register_permit())); + let stream_ctx = Arc::new(StreamContext::seq_scan_ctx(input)); properties.partitions = vec![stream_ctx.partition_ranges()]; Self { @@ -90,6 +96,7 @@ impl SeriesScan { stream_ctx, receivers: Mutex::new(Vec::new()), metrics_list: Arc::new(PartitionMetricsList::default()), + query_memory_permit, } } @@ -123,14 +130,11 @@ impl SeriesScan { Box::pin(record_batch_stream), )); - Ok( - if let Some(tracker) = &self.stream_ctx.query_memory_tracker { - let permit = tracker.register_stream(); - Box::pin(MemoryTrackedStream::new(stream, permit)) - } else { - stream - }, - ) + Ok(if let Some(permit) = &self.query_memory_permit { + Box::pin(MemoryTrackedStream::new(stream, permit.clone())) + } else { + stream + }) } fn scan_batch_in_partition( diff --git a/src/mito2/src/read/unordered_scan.rs b/src/mito2/src/read/unordered_scan.rs index 406e64b30a2c..309d156b00a9 100644 --- a/src/mito2/src/read/unordered_scan.rs +++ b/src/mito2/src/read/unordered_scan.rs @@ -21,7 +21,8 @@ use std::time::Instant; use async_stream::{stream, try_stream}; use common_error::ext::BoxedError; use common_recordbatch::{ - MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, SendableRecordBatchStream, + MemoryPermit, MemoryTrackedStream, QueryMemoryTracker, RecordBatchStreamWrapper, + SendableRecordBatchStream, }; use datafusion::physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion::physical_plan::{DisplayAs, DisplayFormatType}; @@ -54,6 +55,8 @@ pub struct UnorderedScan { stream_ctx: Arc, /// Metrics for each partition. metrics_list: PartitionMetricsList, + /// Memory permit for the entire query (shared across all partitions). + query_memory_permit: Option>, } impl UnorderedScan { @@ -62,16 +65,17 @@ impl UnorderedScan { let mut properties = ScannerProperties::default() .with_append_mode(input.append_mode) .with_total_rows(input.total_rows()); - let stream_ctx = Arc::new(StreamContext::unordered_scan_ctx( - input, - query_memory_tracker, - )); + let query_memory_permit = query_memory_tracker + .as_ref() + .map(|t| Arc::new(t.register_permit())); + let stream_ctx = Arc::new(StreamContext::unordered_scan_ctx(input)); properties.partitions = vec![stream_ctx.partition_ranges()]; Self { properties, stream_ctx, metrics_list: PartitionMetricsList::default(), + query_memory_permit, } } @@ -256,14 +260,11 @@ impl UnorderedScan { Box::pin(record_batch_stream), )); - Ok( - if let Some(tracker) = &self.stream_ctx.query_memory_tracker { - let permit = tracker.register_stream(); - Box::pin(MemoryTrackedStream::new(stream, permit)) - } else { - stream - }, - ) + Ok(if let Some(permit) = &self.query_memory_permit { + Box::pin(MemoryTrackedStream::new(stream, permit.clone())) + } else { + stream + }) } fn scan_batch_in_partition( From 2e195e4ba4d2ed491c5ea6e6bd4ccc64c5f699a3 Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Fri, 17 Oct 2025 19:28:37 +0800 Subject: [PATCH 18/19] feat: estimated_size for batch Signed-off-by: jeremyhi --- src/common/recordbatch/src/lib.rs | 10 +++------- src/common/recordbatch/src/recordbatch.rs | 9 +++++++++ 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index 9dce99c03d9e..b344f27f1417 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -524,9 +524,9 @@ impl fmt::Debug for QueryMemoryTracker { impl QueryMemoryTracker { /// Default capacity for Top-K privileged streams. - pub const DEFAULT_CAPACITY: usize = 10; + pub const DEFAULT_CAPACITY: usize = 15; /// Default memory ratio for non-privileged streams. - pub const DEFAULT_NON_PRIVILEGED_RATIO: f64 = 0.5; + pub const DEFAULT_NON_PRIVILEGED_RATIO: f64 = 0.7; /// Create a new memory tracker with the given limit (in bytes). /// @@ -731,11 +731,7 @@ impl Stream for MemoryTrackedStream { fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { match Pin::new(&mut self.inner).poll_next(cx) { Poll::Ready(Some(Ok(batch))) => { - let size = batch - .columns() - .iter() - .map(|vec_ref| vec_ref.memory_size()) - .sum::(); + let size = batch.estimated_size(); if let Err(e) = self.permit.track(size, self.total_tracked) { return Poll::Ready(Some(Err(e))); diff --git a/src/common/recordbatch/src/recordbatch.rs b/src/common/recordbatch/src/recordbatch.rs index 3cc30ce1ba78..7d6f55cc418e 100644 --- a/src/common/recordbatch/src/recordbatch.rs +++ b/src/common/recordbatch/src/recordbatch.rs @@ -253,6 +253,15 @@ impl RecordBatch { let columns = self.columns.iter().map(|vector| vector.slice(offset, len)); RecordBatch::new(self.schema.clone(), columns) } + + pub(crate) fn estimated_size(&self) -> usize { + self.df_record_batch + .columns() + .iter() + // If can not get slice memory size, assume 0 here. + .map(|c| c.to_data().get_slice_memory_size().unwrap_or(0)) + .sum() + } } impl Serialize for RecordBatch { From e3007ffe1149423db8c17f51ce6e0ee62c9ca9ab Mon Sep 17 00:00:00 2001 From: jeremyhi Date: Fri, 17 Oct 2025 19:31:48 +0800 Subject: [PATCH 19/19] chore: minor refactor --- src/common/recordbatch/src/lib.rs | 185 ++++++++++++---------- src/common/recordbatch/src/recordbatch.rs | 9 -- 2 files changed, 103 insertions(+), 91 deletions(-) diff --git a/src/common/recordbatch/src/lib.rs b/src/common/recordbatch/src/lib.rs index b344f27f1417..8cca133dd026 100644 --- a/src/common/recordbatch/src/lib.rs +++ b/src/common/recordbatch/src/lib.rs @@ -434,7 +434,7 @@ impl MemoryPermit { self.tracker .privileged_count .fetch_update(Ordering::AcqRel, Ordering::Acquire, |count| { - if count < self.tracker.capacity { + if count < self.tracker.privileged_slots { Some(count + 1) } else { None @@ -454,10 +454,10 @@ impl MemoryPermit { /// * `additional` - Additional memory size to track in bytes /// * `stream_tracked` - Total memory already tracked by this stream /// - /// # Top-K Behavior - /// - Privileged streams (Top-K): Can push global memory usage up to full limit - /// - Non-privileged streams: Can push global memory usage up to limit * non_privileged_ratio (default: 0.5) - /// - Non-privileged streams automatically attempt to acquire privilege if slots become available + /// # Behavior + /// - Privileged streams: Can push global memory usage up to full limit + /// - Standard-tier streams: Can push global memory usage up to limit * standard_tier_memory_fraction (default: 0.7) + /// - Standard-tier streams automatically attempt to acquire privilege if slots become available /// - The configured limit is absolute hard limit - no stream can exceed it pub fn track(&self, additional: usize, stream_tracked: usize) -> Result<()> { // Ensure privileged status if possible @@ -489,18 +489,18 @@ impl Drop for MemoryPermit { /// Memory tracker for RecordBatch streams. Clone to share the same limit across queries. /// -/// Uses Top-K first-come-first-served strategy: -/// - First K streams (default: 10) get privileged status and can use full limit -/// - Non-privileged streams use discounted limit (default: 0.5x, configurable) -/// - The configured limit is absolute hard limit - no stream can exceed it -/// - When privileged streams drop, waiting streams can promote to Top-K +/// Implements a two-tier memory allocation strategy: +/// - **Privileged tier**: First N streams (default: 20) can use up to the full memory limit +/// - **Standard tier**: Remaining streams are restricted to a fraction of the limit (default: 70%) +/// - Privilege is granted on a first-come-first-served basis +/// - The configured limit is an absolute hard cap - no stream can exceed it #[derive(Clone)] pub struct QueryMemoryTracker { current: Arc, limit: usize, - non_privileged_ratio: f64, + standard_tier_memory_fraction: f64, privileged_count: Arc, - capacity: usize, + privileged_slots: usize, on_update: Option>, on_reject: Option>, } @@ -510,12 +510,15 @@ impl fmt::Debug for QueryMemoryTracker { f.debug_struct("QueryMemoryTracker") .field("current", &self.current.load(Ordering::Acquire)) .field("limit", &self.limit) - .field("non_privileged_ratio", &self.non_privileged_ratio) + .field( + "standard_tier_memory_fraction", + &self.standard_tier_memory_fraction, + ) .field( "privileged_count", &self.privileged_count.load(Ordering::Acquire), ) - .field("capacity", &self.capacity) + .field("privileged_slots", &self.privileged_slots) .field("on_update", &self.on_update.is_some()) .field("on_reject", &self.on_reject.is_some()) .finish() @@ -523,66 +526,80 @@ impl fmt::Debug for QueryMemoryTracker { } impl QueryMemoryTracker { - /// Default capacity for Top-K privileged streams. - pub const DEFAULT_CAPACITY: usize = 15; - /// Default memory ratio for non-privileged streams. - pub const DEFAULT_NON_PRIVILEGED_RATIO: f64 = 0.7; + /// Default maximum number of streams that can get privileged memory access. + /// Privileged streams can use up to the full memory limit. + /// Privilege is granted on a first-come-first-served basis. + pub const DEFAULT_PRIVILEGED_SLOTS: usize = 20; + /// Default memory fraction available to standard-tier (non-privileged) streams. + /// Standard-tier streams can only use up to `limit * this_fraction`. + /// Value range: [0.0, 1.0]. + pub const DEFAULT_STANDARD_TIER_MEMORY_FRACTION: f64 = 0.7; /// Create a new memory tracker with the given limit (in bytes). /// + /// Uses default configuration: + /// - Privileged slots: 20 + /// - Standard-tier memory fraction: 0.7 (70%) + /// /// # Arguments /// * `limit` - Maximum memory usage in bytes (hard limit for all streams). 0 means unlimited. - /// * `non_privileged_ratio` - Memory ratio for non-privileged streams (default: 0.5). - /// Non-privileged streams can only use `limit * non_privileged_ratio`. - /// Privileged streams can use up to the full `limit`. pub fn new(limit: usize) -> Self { - Self::with_capacity_and_ratio( + Self::with_config( limit, - Self::DEFAULT_CAPACITY, - Self::DEFAULT_NON_PRIVILEGED_RATIO, + Self::DEFAULT_PRIVILEGED_SLOTS, + Self::DEFAULT_STANDARD_TIER_MEMORY_FRACTION, ) } - /// Create a new memory tracker with custom Top-K capacity. - pub fn with_capacity(limit: usize, capacity: usize) -> Self { - Self::with_capacity_and_ratio(limit, capacity, Self::DEFAULT_NON_PRIVILEGED_RATIO) + /// Create a new memory tracker with custom privileged slots limit. + pub fn with_privileged_slots(limit: usize, privileged_slots: usize) -> Self { + Self::with_config( + limit, + privileged_slots, + Self::DEFAULT_STANDARD_TIER_MEMORY_FRACTION, + ) } - /// Create a new memory tracker with custom Top-K capacity and non-privileged ratio. + /// Create a new memory tracker with full configuration. + /// + /// # Arguments + /// * `limit` - Maximum memory usage in bytes (hard limit for all streams). 0 means unlimited. + /// * `privileged_slots` - Maximum number of streams that can get privileged status. + /// * `standard_tier_memory_fraction` - Memory fraction for standard-tier streams (range: [0.0, 1.0]). /// /// # Panics - /// Panics if `non_privileged_ratio` is not in the range [0.0, 1.0]. - pub fn with_capacity_and_ratio( + /// Panics if `standard_tier_memory_fraction` is not in the range [0.0, 1.0]. + pub fn with_config( limit: usize, - capacity: usize, - non_privileged_ratio: f64, + privileged_slots: usize, + standard_tier_memory_fraction: f64, ) -> Self { assert!( - (0.0..=1.0).contains(&non_privileged_ratio), - "non_privileged_ratio must be in [0.0, 1.0], got {}", - non_privileged_ratio + (0.0..=1.0).contains(&standard_tier_memory_fraction), + "standard_tier_memory_fraction must be in [0.0, 1.0], got {}", + standard_tier_memory_fraction ); Self { current: Arc::new(AtomicUsize::new(0)), limit, - non_privileged_ratio, + standard_tier_memory_fraction, privileged_count: Arc::new(AtomicUsize::new(0)), - capacity, + privileged_slots, on_update: None, on_reject: None, } } /// Register a new permit for memory tracking. - /// The first `capacity` permits get privileged status automatically. + /// The first `privileged_slots` permits get privileged status automatically. /// The returned permit can be shared across multiple streams of the same query. pub fn register_permit(&self) -> MemoryPermit { // Try to claim a privileged slot let is_privileged = self .privileged_count .fetch_update(Ordering::AcqRel, Ordering::Acquire, |count| { - if count < self.capacity { + if count < self.privileged_slots { Some(count + 1) } else { None @@ -639,11 +656,11 @@ impl QueryMemoryTracker { ) -> Result<()> { // Calculate effective global limit based on stream privilege // Privileged streams: can push global usage up to full limit - // Non-privileged streams: can only push global usage up to discounted limit + // Standard-tier streams: can only push global usage up to fraction of limit let effective_limit = if is_privileged { self.limit } else { - (self.limit as f64 * self.non_privileged_ratio) as usize + (self.limit as f64 * self.standard_tier_memory_fraction) as usize }; let mut new_total = 0; @@ -731,13 +748,17 @@ impl Stream for MemoryTrackedStream { fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { match Pin::new(&mut self.inner).poll_next(cx) { Poll::Ready(Some(Ok(batch))) => { - let size = batch.estimated_size(); + let additional = batch + .columns() + .iter() + .map(|c| c.memory_size()) + .sum::(); - if let Err(e) = self.permit.track(size, self.total_tracked) { + if let Err(e) = self.permit.track(additional, self.total_tracked) { return Poll::Ready(Some(Err(e))); } - self.total_tracked += size; + self.total_tracked += additional; Poll::Ready(Some(Ok(batch))) } @@ -890,11 +911,11 @@ mod tests { } #[test] - fn test_query_memory_tracker_top_k_limit() { - // Capacity = 2 for easy testing - // Limit: 1000, non-privileged ratio: 0.5 - // Privileged can push global to 1000, non-privileged can push global to 500 - let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 2)); + fn test_query_memory_tracker_privileged_limit() { + // Privileged slots = 2 for easy testing + // Limit: 1000, standard-tier fraction: 0.7 (default) + // Privileged can push global to 1000, standard-tier can push global to 700 + let tracker = Arc::new(QueryMemoryTracker::with_privileged_slots(1000, 2)); // First 2 streams are privileged let permit1 = tracker.register_permit(); @@ -902,7 +923,7 @@ mod tests { assert!(permit1.is_privileged()); assert!(permit2.is_privileged()); - // Third stream is not privileged + // Third stream is standard-tier (not privileged) let permit3 = tracker.register_permit(); assert!(!permit3.is_privileged()); @@ -910,37 +931,37 @@ mod tests { assert!(permit1.track(300, 0).is_ok()); assert_eq!(tracker.current(), 300); - // Non-privileged can add up to 200 (total becomes 500, its effective limit) - assert!(permit3.track(200, 0).is_ok()); - assert_eq!(tracker.current(), 500); + // Standard-tier can add up to 400 (total becomes 700, its effective limit) + assert!(permit3.track(400, 0).is_ok()); + assert_eq!(tracker.current(), 700); - // Non-privileged stream cannot push global beyond 500 - let err = permit3.track(100, 200).unwrap_err(); + // Standard-tier stream cannot push global beyond 700 + let err = permit3.track(100, 400).unwrap_err(); let err_msg = err.to_string(); - assert!(err_msg.contains("200B used by this stream")); - assert!(err_msg.contains("effective limit: 500B (50%)")); - assert!(err_msg.contains("500B used globally (50%)")); - assert_eq!(tracker.current(), 500); + assert!(err_msg.contains("400B used by this stream")); + assert!(err_msg.contains("effective limit: 700B (70%)")); + assert!(err_msg.contains("700B used globally (70%)")); + assert_eq!(tracker.current(), 700); permit1.release(300); - permit3.release(200); + permit3.release(400); assert_eq!(tracker.current(), 0); } #[test] fn test_query_memory_tracker_promotion() { - // Capacity = 1 for easy testing - let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 1)); + // Privileged slots = 1 for easy testing + let tracker = Arc::new(QueryMemoryTracker::with_privileged_slots(1000, 1)); // First stream is privileged let permit1 = tracker.register_permit(); assert!(permit1.is_privileged()); - // Second stream is not privileged (can only use 500) + // Second stream is standard-tier (can only use 500) let permit2 = tracker.register_permit(); assert!(!permit2.is_privileged()); - // Non-privileged can only track 500 + // Standard-tier can only track 500 assert!(permit2.track(400, 0).is_ok()); assert_eq!(tracker.current(), 400); @@ -960,7 +981,7 @@ mod tests { fn test_query_memory_tracker_privileged_hard_limit() { // Test that the configured limit is absolute hard limit for all streams // Privileged: can use full limit (1000) - // Non-privileged: can use 0.5x limit (500) + // Standard-tier: can use 0.7x limit (700 with defaults) let tracker = Arc::new(QueryMemoryTracker::new(1000)); let permit1 = tracker.register_permit(); @@ -987,10 +1008,10 @@ mod tests { } #[test] - fn test_query_memory_tracker_non_privileged_ratio() { - // Test non-privileged streams use discounted limit - // Limit: 1000, ratio: 0.5, so non-privileged can use 500 - let tracker = Arc::new(QueryMemoryTracker::with_capacity(1000, 1)); + fn test_query_memory_tracker_standard_tier_fraction() { + // Test standard-tier streams use fraction of limit + // Limit: 1000, default fraction: 0.7, so standard-tier can use 700 + let tracker = Arc::new(QueryMemoryTracker::with_privileged_slots(1000, 1)); let permit1 = tracker.register_permit(); assert!(permit1.is_privileged()); @@ -998,23 +1019,23 @@ mod tests { let permit2 = tracker.register_permit(); assert!(!permit2.is_privileged()); - // Non-privileged can use up to 500 (1000 * 0.5) - assert!(permit2.track(400, 0).is_ok()); - assert_eq!(tracker.current(), 400); + // Standard-tier can use up to 700 (1000 * 0.7 default) + assert!(permit2.track(600, 0).is_ok()); + assert_eq!(tracker.current(), 600); - // Cannot exceed discounted limit (500) - assert!(permit2.track(200, 400).is_err()); - assert_eq!(tracker.current(), 400); + // Cannot exceed standard-tier limit (700) + assert!(permit2.track(200, 600).is_err()); + assert_eq!(tracker.current(), 600); - // Can add within discounted limit - assert!(permit2.track(100, 400).is_ok()); - assert_eq!(tracker.current(), 500); + // Can add within standard-tier limit + assert!(permit2.track(100, 600).is_ok()); + assert_eq!(tracker.current(), 700); - // Cannot exceed discounted limit - assert!(permit2.track(1, 500).is_err()); - assert_eq!(tracker.current(), 500); + // Cannot exceed standard-tier limit + assert!(permit2.track(1, 700).is_err()); + assert_eq!(tracker.current(), 700); - permit2.release(500); + permit2.release(700); assert_eq!(tracker.current(), 0); } } diff --git a/src/common/recordbatch/src/recordbatch.rs b/src/common/recordbatch/src/recordbatch.rs index 7d6f55cc418e..3cc30ce1ba78 100644 --- a/src/common/recordbatch/src/recordbatch.rs +++ b/src/common/recordbatch/src/recordbatch.rs @@ -253,15 +253,6 @@ impl RecordBatch { let columns = self.columns.iter().map(|vector| vector.slice(offset, len)); RecordBatch::new(self.schema.clone(), columns) } - - pub(crate) fn estimated_size(&self) -> usize { - self.df_record_batch - .columns() - .iter() - // If can not get slice memory size, assume 0 here. - .map(|c| c.to_data().get_slice_memory_size().unwrap_or(0)) - .sum() - } } impl Serialize for RecordBatch {