Skip to content

Commit a6c8c86

Browse files
authored
feat: user configurable memory control policy (risingwavelabs#8475)
1 parent 28c539c commit a6c8c86

File tree

6 files changed

+63
-16
lines changed

6 files changed

+63
-16
lines changed

risedev.yml

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -737,6 +737,13 @@ template:
737737
# Total available memory for the compute node in bytes
738738
total-memory-bytes: 8589934592
739739

740+
# The policy for compute node memory control.
741+
memory-control-policy: streaming-only
742+
743+
# The proportion of streaming memory to all available memory for computing. Only works when
744+
# `memory_control_policy` is set to "streaming-batch".
745+
streaming-memory-proportion: 0.7
746+
740747
# Parallelism of tasks per compute node
741748
parallelism: 4
742749

src/compute/src/lib.rs

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -93,6 +93,22 @@ pub struct ComputeNodeOpts {
9393
#[clap(long, env = "RW_PARALLELISM", default_value_t = default_parallelism())]
9494
pub parallelism: usize,
9595

96+
/// The policy for compute node memory control. Valid values:
97+
/// - streaming-only
98+
/// - streaming-batch
99+
#[clap(
100+
long,
101+
env = "RW_MEMORY_CONTROL_POLICY",
102+
default_value = "streaming-only"
103+
)]
104+
pub memory_control_policy: String,
105+
106+
/// The proportion of streaming memory to all available memory for computing. Only works when
107+
/// `memory_control_policy` is set to "streaming-batch". Ignored otherwise. See
108+
/// [`FixedProportionPolicy`] for more details.
109+
#[clap(long, env = "RW_STREAMING_MEMORY_PROPORTION", default_value_t = 0.7)]
110+
pub streaming_memory_proportion: f64,
111+
96112
#[clap(flatten)]
97113
override_config: OverrideConfigOpts,
98114
}

src/compute/src/memory_management/policy.rs

Lines changed: 25 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,8 @@ use risingwave_common::error::Result;
2222
use risingwave_common::util::epoch::Epoch;
2323
use risingwave_stream::task::LocalStreamManager;
2424

25+
use crate::ComputeNodeOpts;
26+
2527
/// `MemoryControlStats` contains the necessary information for memory control, including both batch
2628
/// and streaming.
2729
pub struct MemoryControlStats {
@@ -49,6 +51,27 @@ pub trait MemoryControl: Send + Sync {
4951
fn describe(&self, total_compute_memory_bytes: usize) -> String;
5052
}
5153

54+
pub fn memory_control_policy_from_config(opts: &ComputeNodeOpts) -> Result<MemoryControlPolicy> {
55+
let input_policy = &opts.memory_control_policy;
56+
if input_policy == FixedProportionPolicy::CONFIG_STR {
57+
Ok(Box::new(FixedProportionPolicy::new(
58+
opts.streaming_memory_proportion,
59+
)?))
60+
} else if input_policy == StreamingOnlyPolicy::CONFIG_STR {
61+
Ok(Box::new(StreamingOnlyPolicy {}))
62+
} else {
63+
let valid_values = [
64+
FixedProportionPolicy::CONFIG_STR,
65+
StreamingOnlyPolicy::CONFIG_STR,
66+
];
67+
Err(anyhow!(format!(
68+
"invalid memory control policy in configuration: {}, valid values: {:?}",
69+
input_policy, valid_values,
70+
))
71+
.into())
72+
}
73+
}
74+
5275
/// `FixedProportionPolicy` performs memory control by limiting the memory usage of both batch and
5376
/// streaming to a fixed proportion.
5477
pub struct FixedProportionPolicy {
@@ -60,6 +83,7 @@ pub struct FixedProportionPolicy {
6083

6184
impl FixedProportionPolicy {
6285
const BATCH_KILL_QUERY_THRESHOLD: f64 = 0.8;
86+
const CONFIG_STR: &str = "streaming-batch";
6387
const STREAM_EVICTION_THRESHOLD_AGGRESSIVE: f64 = 0.9;
6488
const STREAM_EVICTION_THRESHOLD_GRACEFUL: f64 = 0.7;
6589

@@ -73,16 +97,6 @@ impl FixedProportionPolicy {
7397
}
7498
}
7599

76-
impl Default for FixedProportionPolicy {
77-
fn default() -> Self {
78-
Self {
79-
// The default streaming memory proportion is 70%. That for batch is correspondingly
80-
// 30%.
81-
streaming_memory_proportion: 0.7,
82-
}
83-
}
84-
}
85-
86100
impl MemoryControl for FixedProportionPolicy {
87101
fn apply(
88102
&self,
@@ -158,10 +172,10 @@ impl MemoryControl for FixedProportionPolicy {
158172
/// `FixedProportionPolicy` in that it calculates the memory usage based on jemalloc statistics,
159173
/// which actually contains system usage other than computing tasks. This is the default memory
160174
/// control policy.
161-
#[derive(Default)]
162175
pub struct StreamingOnlyPolicy {}
163176

164177
impl StreamingOnlyPolicy {
178+
const CONFIG_STR: &str = "streaming-only";
165179
const STREAM_EVICTION_THRESHOLD_AGGRESSIVE: f64 = 0.9;
166180
const STREAM_EVICTION_THRESHOLD_GRACEFUL: f64 = 0.7;
167181
}

src/compute/src/server.rs

Lines changed: 8 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -60,7 +60,7 @@ use tokio::task::JoinHandle;
6060
use crate::memory_management::memory_manager::{
6161
GlobalMemoryManager, MIN_COMPUTE_MEMORY_MB, SYSTEM_RESERVED_MEMORY_MB,
6262
};
63-
use crate::memory_management::policy::StreamingOnlyPolicy;
63+
use crate::memory_management::policy::memory_control_policy_from_config;
6464
use crate::observer::observer_manager::ComputeObserverNode;
6565
use crate::rpc::service::config_service::ConfigServiceImpl;
6666
use crate::rpc::service::exchange_metrics::ExchangeServiceMetrics;
@@ -79,7 +79,7 @@ pub async fn compute_node_serve(
7979
opts: ComputeNodeOpts,
8080
) -> (Vec<JoinHandle<()>>, Sender<()>) {
8181
// Load the configuration.
82-
let config = load_config(&opts.config_path, Some(opts.override_config));
82+
let config = load_config(&opts.config_path, Some(opts.override_config.clone()));
8383

8484
info!("Starting compute node",);
8585
info!("> config: {:?}", config);
@@ -105,7 +105,10 @@ pub async fn compute_node_serve(
105105
let storage_opts = Arc::new(StorageOpts::from((&config, &system_params)));
106106

107107
let state_store_url = {
108-
let from_local = opts.state_store.unwrap_or("hummock+memory".to_string());
108+
let from_local = opts
109+
.state_store
110+
.clone()
111+
.unwrap_or_else(|| "hummock+memory".to_string());
109112
system_params.state_store(from_local)
110113
};
111114

@@ -115,6 +118,7 @@ pub async fn compute_node_serve(
115118
total_storage_memory_limit_bytes(&config.storage, embedded_compactor_enabled);
116119
let compute_memory_bytes =
117120
validate_compute_node_memory_config(opts.total_memory_bytes, storage_memory_bytes);
121+
let memory_control_policy = memory_control_policy_from_config(&opts).unwrap();
118122

119123
let worker_id = meta_client.worker_id();
120124
info!("Assigned worker node id {}", worker_id);
@@ -239,7 +243,7 @@ pub async fn compute_node_serve(
239243
compute_memory_bytes,
240244
system_params.barrier_interval_ms(),
241245
streaming_metrics.clone(),
242-
Box::new(StreamingOnlyPolicy {}),
246+
memory_control_policy,
243247
);
244248
// Run a background memory monitor
245249
tokio::spawn(memory_mgr.clone().run(batch_mgr_clone, stream_mgr_clone));

src/risedevtool/src/service_config.rs

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,8 @@ pub struct ComputeNodeConfig {
4242
pub connector_rpc_endpoint: String,
4343

4444
pub total_memory_bytes: usize,
45+
pub memory_control_policy: String,
46+
pub streaming_memory_proportion: f64,
4547
pub parallelism: usize,
4648
}
4749

src/risedevtool/src/task/compute_node_service.rs

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,11 @@ impl ComputeNodeService {
6565
.arg("--parallelism")
6666
.arg(&config.parallelism.to_string())
6767
.arg("--total-memory-bytes")
68-
.arg(&config.total_memory_bytes.to_string());
68+
.arg(&config.total_memory_bytes.to_string())
69+
.arg("--memory-control-policy")
70+
.arg(&config.memory_control_policy)
71+
.arg("--streaming-memory-proportion")
72+
.arg(&config.streaming_memory_proportion.to_string());
6973

7074
let provide_jaeger = config.provide_jaeger.as_ref().unwrap();
7175
match provide_jaeger.len() {

0 commit comments

Comments
 (0)