risingwave_common/config/
streaming.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::time::Duration;
16
17use risingwave_common_proc_macro::serde_prefix_all;
18
19use super::*;
20
21mod async_stack_trace;
22mod join_encoding_type;
23mod over_window;
24
25pub use async_stack_trace::*;
26pub use join_encoding_type::*;
27pub use over_window::*;
28
29/// The section `[streaming]` in `risingwave.toml`.
30#[serde_with::apply(Option => #[serde(with = "none_as_empty_string")])]
31#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)]
32pub struct StreamingConfig {
33    /// The maximum number of barriers in-flight in the compute nodes.
34    #[serde(default = "default::streaming::in_flight_barrier_nums")]
35    pub in_flight_barrier_nums: usize,
36
37    /// The thread number of the streaming actor runtime in the compute node. The default value is
38    /// decided by `tokio`.
39    #[serde(default)]
40    pub actor_runtime_worker_threads_num: Option<usize>,
41
42    /// Enable async stack tracing through `await-tree` for risectl.
43    #[serde(default = "default::streaming::async_stack_trace")]
44    pub async_stack_trace: AsyncStackTraceOption,
45
46    #[serde(default)]
47    #[config_doc(nested)]
48    pub developer: StreamingDeveloperConfig,
49
50    /// Max unique user stream errors per actor
51    #[serde(default = "default::streaming::unique_user_stream_errors")]
52    pub unique_user_stream_errors: usize,
53
54    /// Disable strict stream consistency checks.
55    #[serde(default = "default::streaming::unsafe_disable_strict_consistency")]
56    pub unsafe_disable_strict_consistency: bool,
57
58    #[serde(default, flatten)]
59    #[config_doc(omitted)]
60    pub unrecognized: Unrecognized<Self>,
61}
62
63/// The subsections `[streaming.developer]`.
64///
65/// It is put at [`StreamingConfig::developer`].
66#[serde_prefix_all("stream_", mode = "alias")]
67#[serde_with::apply(Option => #[serde(with = "none_as_empty_string")])]
68#[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde, ConfigDoc)]
69pub struct StreamingDeveloperConfig {
70    /// Set to true to enable per-executor row count metrics. This will produce a lot of timeseries
71    /// and might affect the prometheus performance. If you only need actor input and output
72    /// rows data, see `stream_actor_in_record_cnt` and `stream_actor_out_record_cnt` instead.
73    #[serde(default = "default::developer::stream_enable_executor_row_count")]
74    pub enable_executor_row_count: bool,
75
76    /// The capacity of the chunks in the channel that connects between `ConnectorSource` and
77    /// `SourceExecutor`.
78    #[serde(default = "default::developer::connector_message_buffer_size")]
79    pub connector_message_buffer_size: usize,
80
81    /// Limit number of the cached entries in an extreme aggregation call.
82    #[serde(default = "default::developer::unsafe_stream_extreme_cache_size")]
83    pub unsafe_extreme_cache_size: usize,
84
85    /// Minimum cache size for TopN cache per group key.
86    #[serde(default = "default::developer::stream_topn_cache_min_capacity")]
87    pub topn_cache_min_capacity: usize,
88
89    /// The maximum size of the chunk produced by executor at a time.
90    #[serde(default = "default::developer::stream_chunk_size")]
91    pub chunk_size: usize,
92
93    /// The initial permits that a channel holds, i.e., the maximum row count can be buffered in
94    /// the channel.
95    #[serde(default = "default::developer::stream_exchange_initial_permits")]
96    pub exchange_initial_permits: usize,
97
98    /// The permits that are batched to add back, for reducing the backward `AddPermits` messages
99    /// in remote exchange.
100    #[serde(default = "default::developer::stream_exchange_batched_permits")]
101    pub exchange_batched_permits: usize,
102
103    /// The maximum number of concurrent barriers in an exchange channel.
104    #[serde(default = "default::developer::stream_exchange_concurrent_barriers")]
105    pub exchange_concurrent_barriers: usize,
106
107    /// The concurrency for dispatching messages to different downstream jobs.
108    ///
109    /// - `1` means no concurrency, i.e., dispatch messages to downstream jobs one by one.
110    /// - `0` means unlimited concurrency.
111    #[serde(default = "default::developer::stream_exchange_concurrent_dispatchers")]
112    pub exchange_concurrent_dispatchers: usize,
113
114    /// The initial permits for a dml channel, i.e., the maximum row count can be buffered in
115    /// the channel.
116    #[serde(default = "default::developer::stream_dml_channel_initial_permits")]
117    pub dml_channel_initial_permits: usize,
118
119    /// The max heap size of dirty groups of `HashAggExecutor`.
120    #[serde(default = "default::developer::stream_hash_agg_max_dirty_groups_heap_size")]
121    pub hash_agg_max_dirty_groups_heap_size: usize,
122
123    #[serde(default = "default::developer::memory_controller_threshold_aggressive")]
124    pub memory_controller_threshold_aggressive: f64,
125
126    #[serde(default = "default::developer::memory_controller_threshold_graceful")]
127    pub memory_controller_threshold_graceful: f64,
128
129    #[serde(default = "default::developer::memory_controller_threshold_stable")]
130    pub memory_controller_threshold_stable: f64,
131
132    #[serde(default = "default::developer::memory_controller_eviction_factor_aggressive")]
133    pub memory_controller_eviction_factor_aggressive: f64,
134
135    #[serde(default = "default::developer::memory_controller_eviction_factor_graceful")]
136    pub memory_controller_eviction_factor_graceful: f64,
137
138    #[serde(default = "default::developer::memory_controller_eviction_factor_stable")]
139    pub memory_controller_eviction_factor_stable: f64,
140
141    #[serde(default = "default::developer::memory_controller_update_interval_ms")]
142    pub memory_controller_update_interval_ms: usize,
143
144    #[serde(default = "default::developer::memory_controller_sequence_tls_step")]
145    pub memory_controller_sequence_tls_step: u64,
146
147    #[serde(default = "default::developer::memory_controller_sequence_tls_lag")]
148    pub memory_controller_sequence_tls_lag: u64,
149
150    #[serde(default = "default::developer::stream_enable_arrangement_backfill")]
151    /// Enable arrangement backfill
152    /// If false, the arrangement backfill will be disabled,
153    /// even if session variable set.
154    /// If true, it's decided by session variable `streaming_use_arrangement_backfill` (default true)
155    pub enable_arrangement_backfill: bool,
156
157    #[serde(default = "default::developer::stream_enable_snapshot_backfill")]
158    /// Enable snapshot backfill
159    /// If false, the snapshot backfill will be disabled,
160    /// even if session variable set.
161    /// If true, it's decided by session variable `streaming_use_snapshot_backfill` (default true)
162    pub enable_snapshot_backfill: bool,
163
164    #[serde(default = "default::developer::stream_high_join_amplification_threshold")]
165    /// If number of hash join matches exceeds this threshold number,
166    /// it will be logged.
167    pub high_join_amplification_threshold: usize,
168
169    /// Actor tokio metrics is enabled if `enable_actor_tokio_metrics` is set or metrics level >= Debug.
170    #[serde(default = "default::developer::enable_actor_tokio_metrics")]
171    pub enable_actor_tokio_metrics: bool,
172
173    /// The number of the connections for streaming remote exchange between two nodes.
174    /// If not specified, the value of `server.connection_pool_size` will be used.
175    #[serde(default = "default::developer::stream_exchange_connection_pool_size")]
176    pub(super) exchange_connection_pool_size: Option<u16>,
177
178    /// A flag to allow disabling the auto schema change handling
179    #[serde(default = "default::developer::stream_enable_auto_schema_change")]
180    pub enable_auto_schema_change: bool,
181
182    #[serde(default = "default::developer::enable_shared_source")]
183    /// Enable shared source
184    /// If false, the shared source will be disabled,
185    /// even if session variable set.
186    /// If true, it's decided by session variable `streaming_use_shared_source` (default true)
187    pub enable_shared_source: bool,
188
189    #[serde(default = "default::developer::switch_jdbc_pg_to_native")]
190    /// When true, all jdbc sinks with connector='jdbc' and jdbc.url="jdbc:postgresql://..."
191    /// will be switched from jdbc postgresql sinks to rust native (connector='postgres') sinks.
192    pub switch_jdbc_pg_to_native: bool,
193
194    /// The maximum number of consecutive barriers allowed in a message when sent between actors.
195    #[serde(default = "default::developer::stream_max_barrier_batch_size")]
196    pub max_barrier_batch_size: u32,
197
198    /// Configure the system-wide cache row cardinality of hash join.
199    /// For example, if this is set to 1000, it means we can have at most 1000 rows in cache.
200    #[serde(default = "default::developer::streaming_hash_join_entry_state_max_rows")]
201    pub hash_join_entry_state_max_rows: usize,
202
203    /// Number of processed rows between periodic join cache evictions.
204    /// Values smaller than 1 will be clamped to 1 by the executor.
205    #[serde(default = "default::developer::streaming_join_hash_map_evict_interval_rows")]
206    pub join_hash_map_evict_interval_rows: u32,
207
208    #[serde(default = "default::developer::streaming_now_progress_ratio")]
209    pub now_progress_ratio: Option<f32>,
210
211    /// Enable / Disable profiling stats used by `EXPLAIN ANALYZE`
212    #[serde(default = "default::developer::enable_explain_analyze_stats")]
213    pub enable_explain_analyze_stats: bool,
214
215    #[serde(default)]
216    pub compute_client_config: RpcClientConfig,
217
218    /// The interval in seconds to rebuild snapshot iterators during snapshot backfill.
219    #[serde(default = "default::developer::stream_snapshot_iter_rebuild_interval_secs")]
220    pub snapshot_iter_rebuild_interval_secs: u64,
221
222    /// `IcebergListExecutor`: The interval in seconds for Iceberg source to list new files.
223    #[serde(default = "default::developer::iceberg_list_interval_sec")]
224    pub iceberg_list_interval_sec: u64,
225
226    /// `IcebergFetchExecutor`: The number of files the executor will fetch concurrently in a batch.
227    #[serde(default = "default::developer::iceberg_fetch_batch_size")]
228    pub iceberg_fetch_batch_size: u64,
229
230    /// `IcebergSink`: The size of the cache for positional delete in the sink.
231    #[serde(default = "default::developer::iceberg_sink_positional_delete_cache_size")]
232    pub iceberg_sink_positional_delete_cache_size: usize,
233
234    /// `IcebergSink`: The maximum number of rows in a row group when writing Parquet files.
235    #[serde(default = "default::developer::iceberg_sink_write_parquet_max_row_group_rows")]
236    pub iceberg_sink_write_parquet_max_row_group_rows: usize,
237
238    /// When enabled, materialized views using default `NoCheck` conflict behavior will be forced
239    /// to use `Overwrite`. Useful to avoid propagating inconsistent changelog downstream.
240    #[serde(default = "default::developer::materialize_force_overwrite_on_no_check")]
241    pub materialize_force_overwrite_on_no_check: bool,
242
243    /// Whether by default enable preloading all rows in memory for state table.
244    /// If true, all capable state tables will preload its state to memory
245    #[serde(default = "default::streaming::default_enable_mem_preload_state_table")]
246    pub default_enable_mem_preload_state_table: bool,
247
248    /// The list of state table ids to *enable* preloading all rows in memory for state table.
249    /// Only takes effect when `default_enable_mem_preload_state_table` is false.
250    #[serde(default)]
251    pub mem_preload_state_table_ids_whitelist: Vec<u32>,
252
253    /// The list of state table ids to *disable* preloading all rows in memory for state table.
254    /// Only takes effect when `default_enable_mem_preload_state_table` is true.
255    #[serde(default)]
256    pub mem_preload_state_table_ids_blacklist: Vec<u32>,
257
258    /// Eliminate unnecessary updates aggressively, even if it impacts performance. Enable this
259    /// only if it's confirmed that no-op updates are causing significant streaming amplification.
260    #[serde(default)]
261    pub aggressive_noop_update_elimination: bool,
262
263    /// The interval in seconds for the refresh scheduler to check and trigger scheduled refreshes.
264    #[serde(default = "default::developer::refresh_scheduler_interval_sec")]
265    pub refresh_scheduler_interval_sec: u64,
266
267    /// Determine which encoding will be used to encode join rows in operator cache.
268    #[serde(default)]
269    pub join_encoding_type: JoinEncodingType,
270
271    /// The timeout for reading from the buffer of the sync log store on barrier.
272    /// Every epoch we will attempt to read the full buffer of the sync log store.
273    /// If we hit the timeout, we will stop reading and continue.
274    #[serde(default = "default::developer::sync_log_store_pause_duration_ms")]
275    pub sync_log_store_pause_duration_ms: usize,
276
277    /// The max buffer size for sync logstore, before we start flushing.
278    #[serde(default = "default::developer::sync_log_store_buffer_size")]
279    pub sync_log_store_buffer_size: usize,
280
281    /// Cache policy for partition cache in streaming over window.
282    /// Can be `full`, `recent`, `recent_first_n` or `recent_last_n`.
283    #[serde(default)]
284    pub over_window_cache_policy: OverWindowCachePolicy,
285
286    /// When enabled, vnode stats pruning is applied in production.
287    /// When disabled, vnode stats pruning is in dry-run mode: we still maintain vnode stats
288    /// and verify that pruning would be correct, but we don't actually use the pruning
289    /// results — we still use cache and storage to fulfill the read. This is useful for
290    /// validating the correctness of vnode stats pruning before enabling it in production.
291    #[serde(default = "default::developer::enable_state_table_vnode_stats_pruning")]
292    pub enable_state_table_vnode_stats_pruning: bool,
293
294    /// Whether `MaterializeExecutor` enables vnode key stats for its state table.
295    #[serde(default = "default::developer::enable_vnode_key_stats_for_materialize")]
296    pub enable_vnode_key_stats_for_materialize: bool,
297
298    /// The maximum number of kv log store readers that can concurrently read historical data
299    /// (i.e., from the state store) during initialization. A reader is considered "initializing"
300    /// until it has read at least one row from the historical stream or the stream returns empty.
301    /// Set to 0 to disable the limit (unlimited concurrency).
302    #[serde(default = "default::developer::max_concurrent_kv_log_store_historical_read")]
303    pub max_concurrent_kv_log_store_historical_read: usize,
304
305    #[serde(default, flatten)]
306    #[serde_prefix_all(skip)]
307    #[config_doc(omitted)]
308    pub unrecognized: Unrecognized<Self>,
309}
310
311impl StreamingDeveloperConfig {
312    pub fn snapshot_iter_rebuild_interval(&self) -> Duration {
313        let rebuild_interval = if self.snapshot_iter_rebuild_interval_secs < 10 {
314            tracing::warn!(
315                "too small rebuild_interval {} second. rewrite to 10",
316                self.snapshot_iter_rebuild_interval_secs
317            );
318            10
319        } else {
320            self.snapshot_iter_rebuild_interval_secs
321        };
322        Duration::from_secs(rebuild_interval)
323    }
324}
325
326impl StreamingConfig {
327    /// Returns the dot-separated keys of all unrecognized fields, including those in `developer` section.
328    pub fn unrecognized_keys(&self) -> impl Iterator<Item = String> {
329        std::iter::from_coroutine(
330            #[coroutine]
331            || {
332                for k in self.unrecognized.inner().keys() {
333                    yield format!("streaming.{k}");
334                }
335                for k in self.developer.unrecognized.inner().keys() {
336                    yield format!("streaming.developer.{k}");
337                }
338            },
339        )
340    }
341}
342
343pub mod default {
344    pub use crate::config::default::developer;
345
346    pub mod streaming {
347        use tracing::info;
348
349        use crate::config::AsyncStackTraceOption;
350        use crate::util::env_var::env_var_is_true;
351
352        pub fn in_flight_barrier_nums() -> usize {
353            // quick fix
354            // TODO: remove this limitation from code
355            10000
356        }
357
358        pub fn async_stack_trace() -> AsyncStackTraceOption {
359            AsyncStackTraceOption::default()
360        }
361
362        pub fn unique_user_stream_errors() -> usize {
363            10
364        }
365
366        pub fn unsafe_disable_strict_consistency() -> bool {
367            false
368        }
369
370        pub fn default_enable_mem_preload_state_table() -> bool {
371            if env_var_is_true("DEFAULT_ENABLE_MEM_PRELOAD_STATE_TABLE") {
372                info!("enabled mem_preload_state_table globally by env var");
373                true
374            } else {
375                false
376            }
377        }
378    }
379}