risingwave_common/session_config/mod.rs
1// Copyright 2022 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
15mod non_zero64;
16mod opt;
17pub mod parallelism;
18mod query_mode;
19mod search_path;
20pub mod sink_decouple;
21mod transaction_isolation_level;
22mod visibility_mode;
23
24use chrono_tz::Tz;
25use itertools::Itertools;
26pub use opt::OptionConfig;
27pub use query_mode::QueryMode;
28use risingwave_common_proc_macro::{ConfigDoc, SessionConfig};
29pub use search_path::{SearchPath, USER_NAME_WILD_CARD};
30use serde::{Deserialize, Serialize};
31use thiserror::Error;
32
33use self::non_zero64::ConfigNonZeroU64;
34use crate::config::mutate::TomlTableMutateExt;
35use crate::config::streaming::{JoinEncodingType, OverWindowCachePolicy};
36use crate::config::{ConfigMergeError, StreamingConfig, merge_streaming_config_section};
37use crate::hash::VirtualNode;
38use crate::session_config::parallelism::ConfigParallelism;
39use crate::session_config::sink_decouple::SinkDecouple;
40use crate::session_config::transaction_isolation_level::IsolationLevel;
41pub use crate::session_config::visibility_mode::VisibilityMode;
42use crate::{PG_VERSION, SERVER_ENCODING, SERVER_VERSION_NUM, STANDARD_CONFORMING_STRINGS};
43
44pub const SESSION_CONFIG_LIST_SEP: &str = ", ";
45
46#[derive(Error, Debug)]
47pub enum SessionConfigError {
48 #[error("Invalid value `{value}` for `{entry}`")]
49 InvalidValue {
50 entry: &'static str,
51 value: String,
52 source: anyhow::Error,
53 },
54
55 #[error("Unrecognized config entry `{0}`")]
56 UnrecognizedEntry(String),
57}
58
59type SessionConfigResult<T> = std::result::Result<T, SessionConfigError>;
60
61// NOTE(kwannoel): We declare it separately as a constant,
62// otherwise seems like it can't infer the type of -1 when written inline.
63const DISABLE_BACKFILL_RATE_LIMIT: i32 = -1;
64const DISABLE_SOURCE_RATE_LIMIT: i32 = -1;
65const DISABLE_DML_RATE_LIMIT: i32 = -1;
66const DISABLE_SINK_RATE_LIMIT: i32 = -1;
67
68/// Default to bypass cluster limits iff in debug mode.
69const BYPASS_CLUSTER_LIMITS: bool = cfg!(debug_assertions);
70
71/// This is the Session Config of RisingWave.
72///
73/// All config entries implement `Display` and `FromStr` for getter and setter, to be read and
74/// altered within a session.
75///
76/// Users can change the default value of a configuration entry using `ALTER SYSTEM SET`. To
77/// facilitate this, a `serde` implementation is used as the wire format for retrieving initial
78/// configurations and updates from the meta service. It's important to note that the meta
79/// service stores the overridden value of each configuration entry per row with `Display` in
80/// the meta store, rather than using the `serde` format. However, we still delegate the `serde`
81/// impl of all fields to `Display`/`FromStr` to make it consistent.
82#[serde_with::apply(_ => #[serde_as(as = "serde_with::DisplayFromStr")] )]
83#[serde_with::serde_as]
84#[derive(Clone, Debug, Deserialize, Serialize, SessionConfig, ConfigDoc, PartialEq)]
85pub struct SessionConfig {
86 /// If `RW_IMPLICIT_FLUSH` is on, then every INSERT/UPDATE/DELETE statement will block
87 /// until the entire dataflow is refreshed. In other words, every related table & MV will
88 /// be able to see the write.
89 #[parameter(default = false, alias = "rw_implicit_flush")]
90 implicit_flush: bool,
91
92 /// If `CREATE_COMPACTION_GROUP_FOR_MV` is on, dedicated compaction groups will be created in
93 /// MV creation.
94 #[parameter(default = false)]
95 create_compaction_group_for_mv: bool,
96
97 /// A temporary config variable to force query running in either local or distributed mode.
98 /// The default value is auto which means let the system decide to run batch queries in local
99 /// or distributed mode automatically.
100 #[parameter(default = QueryMode::default())]
101 query_mode: QueryMode,
102
103 /// Sets the number of digits displayed for floating-point values.
104 /// See <https://www.postgresql.org/docs/current/runtime-config-client.html#:~:text=for%20more%20information.-,extra_float_digits,-(integer)>
105 #[parameter(default = 1)]
106 extra_float_digits: i32,
107
108 /// Sets the application name to be reported in statistics and logs.
109 /// See <https://www.postgresql.org/docs/14/runtime-config-logging.html#:~:text=What%20to%20Log-,application_name,-(string)>
110 #[parameter(default = "", flags = "REPORT")]
111 application_name: String,
112
113 /// It is typically set by an application upon connection to the server.
114 /// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-DATESTYLE>
115 #[parameter(default = "", rename = "datestyle")]
116 date_style: String,
117
118 /// Force the use of lookup join instead of hash join when possible for local batch execution.
119 #[parameter(default = true, alias = "rw_batch_enable_lookup_join")]
120 batch_enable_lookup_join: bool,
121
122 /// Enable usage of sortAgg instead of hash agg when order property is satisfied in batch
123 /// execution
124 #[parameter(default = true, alias = "rw_batch_enable_sort_agg")]
125 batch_enable_sort_agg: bool,
126
127 /// Enable distributed DML, so an insert, delete, and update statement can be executed in a distributed way (e.g. running in multiple compute nodes).
128 /// No atomicity guarantee in this mode. Its goal is to gain the best ingestion performance for initial batch ingestion where users always can drop their table when failure happens.
129 #[parameter(default = false, rename = "batch_enable_distributed_dml")]
130 batch_enable_distributed_dml: bool,
131
132 /// Evaluate expression in strict mode for batch queries.
133 /// If set to false, an expression failure will not cause an error but leave a null value
134 /// on the result set.
135 #[parameter(default = true)]
136 batch_expr_strict_mode: bool,
137
138 /// The max gap allowed to transform small range scan into multi point lookup.
139 #[parameter(default = 8)]
140 max_split_range_gap: i32,
141
142 /// Sets the order in which schemas are searched when an object (table, data type, function, etc.)
143 /// is referenced by a simple name with no schema specified.
144 /// See <https://www.postgresql.org/docs/14/runtime-config-client.html#GUC-SEARCH-PATH>
145 #[parameter(default = SearchPath::default())]
146 search_path: SearchPath,
147
148 /// If `VISIBILITY_MODE` is all, we will support querying data without checkpoint.
149 #[parameter(default = VisibilityMode::default())]
150 visibility_mode: VisibilityMode,
151
152 /// See <https://www.postgresql.org/docs/current/transaction-iso.html>
153 #[parameter(default = IsolationLevel::default())]
154 transaction_isolation: IsolationLevel,
155
156 /// Select as of specific epoch.
157 /// Sets the historical epoch for querying data. If 0, querying latest data.
158 #[parameter(default = ConfigNonZeroU64::default())]
159 query_epoch: ConfigNonZeroU64,
160
161 /// Session timezone. Defaults to UTC.
162 #[parameter(default = "UTC", check_hook = check_timezone)]
163 timezone: String,
164
165 /// The execution parallelism for streaming queries, including tables, materialized views, indexes,
166 /// and sinks. Defaults to 0, which means they will be scheduled adaptively based on the cluster size.
167 ///
168 /// If a non-zero value is set, streaming queries will be scheduled to use a fixed number of parallelism.
169 /// Note that the value will be bounded at `STREAMING_MAX_PARALLELISM`.
170 #[parameter(default = ConfigParallelism::default())]
171 streaming_parallelism: ConfigParallelism,
172
173 /// Specific parallelism for backfill. By default, it will fall back to `STREAMING_PARALLELISM`.
174 #[parameter(default = ConfigParallelism::default())]
175 streaming_parallelism_for_backfill: ConfigParallelism,
176
177 /// Specific parallelism for table. By default, it will fall back to `STREAMING_PARALLELISM`.
178 #[parameter(default = ConfigParallelism::default())]
179 streaming_parallelism_for_table: ConfigParallelism,
180
181 /// Specific parallelism for sink. By default, it will fall back to `STREAMING_PARALLELISM`.
182 #[parameter(default = ConfigParallelism::default())]
183 streaming_parallelism_for_sink: ConfigParallelism,
184
185 /// Specific parallelism for index. By default, it will fall back to `STREAMING_PARALLELISM`.
186 #[parameter(default = ConfigParallelism::default())]
187 streaming_parallelism_for_index: ConfigParallelism,
188
189 /// Specific parallelism for source. By default, it will fall back to `STREAMING_PARALLELISM`.
190 #[parameter(default = ConfigParallelism::default())]
191 streaming_parallelism_for_source: ConfigParallelism,
192
193 /// Specific parallelism for materialized view. By default, it will fall back to `STREAMING_PARALLELISM`.
194 #[parameter(default = ConfigParallelism::default())]
195 streaming_parallelism_for_materialized_view: ConfigParallelism,
196
197 /// Enable delta join for streaming queries. Defaults to false.
198 #[parameter(default = false, alias = "rw_streaming_enable_delta_join")]
199 streaming_enable_delta_join: bool,
200
201 /// Enable bushy join for streaming queries. Defaults to true.
202 #[parameter(default = true, alias = "rw_streaming_enable_bushy_join")]
203 streaming_enable_bushy_join: bool,
204
205 /// Force filtering to be done inside the join whenever there's a choice between optimizations.
206 /// Defaults to false.
207 #[parameter(default = false, alias = "rw_streaming_force_filter_inside_join")]
208 streaming_force_filter_inside_join: bool,
209
210 /// Enable arrangement backfill for streaming queries. Defaults to true.
211 /// When set to true, the parallelism of the upstream fragment will be
212 /// decoupled from the parallelism of the downstream scan fragment.
213 /// Or more generally, the parallelism of the upstream table / index / mv
214 /// will be decoupled from the parallelism of the downstream table / index / mv / sink.
215 #[parameter(default = true)]
216 streaming_use_arrangement_backfill: bool,
217
218 #[parameter(default = false)]
219 streaming_use_snapshot_backfill: bool,
220
221 /// Allow `jsonb` in stream key
222 #[parameter(default = false, alias = "rw_streaming_allow_jsonb_in_stream_key")]
223 streaming_allow_jsonb_in_stream_key: bool,
224
225 /// Enable materialized expressions for impure functions (typically UDF).
226 #[parameter(default = true)]
227 streaming_enable_materialized_expressions: bool,
228
229 /// Separate consecutive `StreamHashJoin` by no-shuffle `StreamExchange`
230 #[parameter(default = false)]
231 streaming_separate_consecutive_join: bool,
232
233 /// Separate `StreamSink` by no-shuffle `StreamExchange`
234 #[parameter(default = false)]
235 streaming_separate_sink: bool,
236
237 /// Determine which encoding will be used to encode join rows in operator cache.
238 ///
239 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
240 /// taking effect for new streaming jobs created in the current session.
241 #[parameter(default = None)]
242 streaming_join_encoding: OptionConfig<JoinEncodingType>,
243
244 /// Enable join ordering for streaming and batch queries. Defaults to true.
245 #[parameter(default = true, alias = "rw_enable_join_ordering")]
246 enable_join_ordering: bool,
247
248 /// Enable two phase agg optimization. Defaults to true.
249 /// Setting this to true will always set `FORCE_TWO_PHASE_AGG` to false.
250 #[parameter(default = true, flags = "SETTER", alias = "rw_enable_two_phase_agg")]
251 enable_two_phase_agg: bool,
252
253 /// Force two phase agg optimization whenever there's a choice between
254 /// optimizations. Defaults to false.
255 /// Setting this to true will always set `ENABLE_TWO_PHASE_AGG` to false.
256 #[parameter(default = false, flags = "SETTER", alias = "rw_force_two_phase_agg")]
257 force_two_phase_agg: bool,
258
259 /// Enable sharing of common sub-plans.
260 /// This means that DAG structured query plans can be constructed,
261 #[parameter(default = true, alias = "rw_enable_share_plan")]
262 /// rather than only tree structured query plans.
263 enable_share_plan: bool,
264
265 /// Enable split distinct agg
266 #[parameter(default = false, alias = "rw_force_split_distinct_agg")]
267 force_split_distinct_agg: bool,
268
269 /// See <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-INTERVALSTYLE>
270 #[parameter(default = "", rename = "intervalstyle")]
271 interval_style: String,
272
273 /// If `BATCH_PARALLELISM` is non-zero, batch queries will use this parallelism.
274 #[parameter(default = ConfigNonZeroU64::default())]
275 batch_parallelism: ConfigNonZeroU64,
276
277 /// The version of PostgreSQL that Risingwave claims to be.
278 #[parameter(default = PG_VERSION)]
279 server_version: String,
280
281 /// The version of PostgreSQL that Risingwave claims to be.
282 #[parameter(default = SERVER_VERSION_NUM)]
283 server_version_num: i32,
284
285 /// see <https://www.postgresql.org/docs/15/runtime-config-client.html#GUC-CLIENT-MIN-MESSAGES>
286 #[parameter(default = "notice")]
287 client_min_messages: String,
288
289 /// see <https://www.postgresql.org/docs/15/runtime-config-client.html#GUC-CLIENT-ENCODING>
290 #[parameter(default = SERVER_ENCODING, check_hook = check_client_encoding)]
291 client_encoding: String,
292
293 /// Enable decoupling sink and internal streaming graph or not
294 #[parameter(default = SinkDecouple::default())]
295 sink_decouple: SinkDecouple,
296
297 /// See <https://www.postgresql.org/docs/current/runtime-config-compatible.html#RUNTIME-CONFIG-COMPATIBLE-VERSION>
298 /// Unused in RisingWave, support for compatibility.
299 #[parameter(default = false)]
300 synchronize_seqscans: bool,
301
302 /// Abort query statement that takes more than the specified amount of time in sec. If
303 /// `log_min_error_statement` is set to ERROR or lower, the statement that timed out will also be
304 /// logged. If this value is specified without units, it is taken as milliseconds. A value of
305 /// zero (the default) disables the timeout.
306 #[parameter(default = 0u32)]
307 statement_timeout: u32,
308
309 /// Terminate any session that has been idle (that is, waiting for a client query) within an open transaction for longer than the specified amount of time in milliseconds.
310 #[parameter(default = 60000u32)]
311 idle_in_transaction_session_timeout: u32,
312
313 /// See <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-LOCK-TIMEOUT>
314 /// Unused in RisingWave, support for compatibility.
315 #[parameter(default = 0)]
316 lock_timeout: i32,
317
318 /// For limiting the startup time of a shareable CDC streaming source when the source is being created. Unit: seconds.
319 #[parameter(default = 60)]
320 cdc_source_wait_streaming_start_timeout: i32,
321
322 /// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-ROW-SECURITY>.
323 /// Unused in RisingWave, support for compatibility.
324 #[parameter(default = true)]
325 row_security: bool,
326
327 /// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-STANDARD-CONFORMING-STRINGS>
328 #[parameter(default = STANDARD_CONFORMING_STRINGS)]
329 standard_conforming_strings: String,
330
331 /// Set streaming rate limit (rows per second) for each parallelism for mv / source / sink backfilling
332 /// If set to -1, disable rate limit.
333 /// If set to 0, this pauses the snapshot read / source read.
334 #[parameter(default = DISABLE_BACKFILL_RATE_LIMIT)]
335 backfill_rate_limit: i32,
336
337 /// Set streaming rate limit (rows per second) for each parallelism for mv / source backfilling, source reads.
338 /// If set to -1, disable rate limit.
339 /// If set to 0, this pauses the snapshot read / source read.
340 #[parameter(default = DISABLE_SOURCE_RATE_LIMIT)]
341 source_rate_limit: i32,
342
343 /// Set streaming rate limit (rows per second) for each parallelism for table DML.
344 /// If set to -1, disable rate limit.
345 /// If set to 0, this pauses the DML.
346 #[parameter(default = DISABLE_DML_RATE_LIMIT)]
347 dml_rate_limit: i32,
348
349 /// Set sink rate limit (rows per second) for each parallelism for external sink.
350 /// If set to -1, disable rate limit.
351 /// If set to 0, this pauses the sink.
352 #[parameter(default = DISABLE_SINK_RATE_LIMIT)]
353 sink_rate_limit: i32,
354
355 /// Cache policy for partition cache in streaming over window.
356 /// Can be `full`, `recent`, `recent_first_n` or `recent_last_n`.
357 ///
358 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
359 /// taking effect for new streaming jobs created in the current session.
360 #[parameter(default = None, alias = "rw_streaming_over_window_cache_policy")]
361 streaming_over_window_cache_policy: OptionConfig<OverWindowCachePolicy>,
362
363 /// Run DDL statements in background
364 #[parameter(default = false)]
365 background_ddl: bool,
366
367 /// Enable shared source. Currently only for Kafka.
368 ///
369 /// When enabled, `CREATE SOURCE` will create a source streaming job, and `CREATE MATERIALIZED VIEWS` from the source
370 /// will forward the data from the same source streaming job, and also backfill prior data from the external source.
371 #[parameter(default = true)]
372 streaming_use_shared_source: bool,
373
374 /// Shows the server-side character set encoding. At present, this parameter can be shown but not set, because the encoding is determined at database creation time.
375 #[parameter(default = SERVER_ENCODING)]
376 server_encoding: String,
377
378 #[parameter(default = "hex", check_hook = check_bytea_output)]
379 bytea_output: String,
380
381 /// Bypass checks on cluster limits
382 ///
383 /// When enabled, `CREATE MATERIALIZED VIEW` will not fail if the cluster limit is hit.
384 #[parameter(default = BYPASS_CLUSTER_LIMITS)]
385 bypass_cluster_limits: bool,
386
387 /// The maximum number of parallelism a streaming query can use. Defaults to 256.
388 ///
389 /// Compared to `STREAMING_PARALLELISM`, which configures the initial parallelism, this configures
390 /// the maximum parallelism a streaming query can use in the future, if the cluster size changes or
391 /// users manually change the parallelism with `ALTER .. SET PARALLELISM`.
392 ///
393 /// It's not always a good idea to set this to a very large number, as it may cause performance
394 /// degradation when performing range scans on the table or the materialized view.
395 // a.k.a. vnode count
396 #[parameter(default = VirtualNode::COUNT_FOR_COMPAT, check_hook = check_streaming_max_parallelism)]
397 streaming_max_parallelism: usize,
398
399 /// Used to provide the connection information for the iceberg engine.
400 /// Format: `iceberg_engine_connection` = `schema_name.connection_name`.
401 #[parameter(default = "", check_hook = check_iceberg_engine_connection)]
402 iceberg_engine_connection: String,
403
404 /// Whether the streaming join should be unaligned or not.
405 #[parameter(default = false)]
406 streaming_enable_unaligned_join: bool,
407
408 /// The timeout for reading from the buffer of the sync log store on barrier.
409 /// Every epoch we will attempt to read the full buffer of the sync log store.
410 /// If we hit the timeout, we will stop reading and continue.
411 ///
412 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
413 /// taking effect for new streaming jobs created in the current session.
414 #[parameter(default = None)]
415 streaming_sync_log_store_pause_duration_ms: OptionConfig<usize>,
416
417 /// The max buffer size for sync logstore, before we start flushing.
418 ///
419 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
420 /// taking effect for new streaming jobs created in the current session.
421 #[parameter(default = None)]
422 streaming_sync_log_store_buffer_size: OptionConfig<usize>,
423
424 /// Whether to disable purifying the definition of the table or source upon retrieval.
425 /// Only set this if encountering issues with functionalities like `SHOW` or `ALTER TABLE/SOURCE`.
426 /// This config may be removed in the future.
427 #[parameter(default = false, flags = "NO_ALTER_SYS")]
428 disable_purify_definition: bool,
429
430 /// The `ef_search` used in querying hnsw vector index
431 #[parameter(default = 40_usize)] // default value borrowed from pg_vector
432 batch_hnsw_ef_search: usize,
433
434 /// Enable index selection for queries
435 #[parameter(default = true)]
436 enable_index_selection: bool,
437
438 /// Enable locality backfill for streaming queries. Defaults to false.
439 #[parameter(default = false)]
440 enable_locality_backfill: bool,
441
442 /// Duration in seconds before notifying the user that a long-running DDL operation (e.g., DROP TABLE, CANCEL JOBS)
443 /// is still running. Set to 0 to disable notifications. Defaults to 30 seconds.
444 #[parameter(default = 30u32)]
445 slow_ddl_notification_secs: u32,
446
447 /// Unsafe: Enable storage retention for non-append-only tables.
448 /// Enabling this can lead to streaming inconsistency and node panic
449 /// if there is any row INSERT/UPDATE/DELETE operation corresponding to the ttled primary key.
450 #[parameter(default = false)]
451 unsafe_enable_storage_retention_for_non_append_only_tables: bool,
452
453 /// Enable DataFusion Engine
454 /// When enabled, queries involving Iceberg tables will be executed using the DataFusion engine.
455 #[parameter(default = false)]
456 enable_datafusion_engine: bool,
457
458 /// Emit chunks in upsert format for `UPDATE` and `DELETE` DMLs.
459 /// May lead to undefined behavior if the table is created with `ON CONFLICT DO NOTHING`.
460 ///
461 /// When enabled:
462 /// - `UPDATE` will only emit `Insert` records for new rows, instead of `Update` records.
463 /// - `DELETE` will only include key columns and pad the rest with NULL, instead of emitting complete rows.
464 #[parameter(default = false)]
465 upsert_dml: bool,
466}
467
468fn check_iceberg_engine_connection(val: &str) -> Result<(), String> {
469 if val.is_empty() {
470 return Ok(());
471 }
472
473 let parts: Vec<&str> = val.split('.').collect();
474 if parts.len() != 2 {
475 return Err("Invalid iceberg engine connection format, Should be set to this format: schema_name.connection_name.".to_owned());
476 }
477
478 Ok(())
479}
480
481fn check_timezone(val: &str) -> Result<(), String> {
482 // Check if the provided string is a valid timezone.
483 Tz::from_str_insensitive(val).map_err(|_e| "Not a valid timezone")?;
484 Ok(())
485}
486
487fn check_client_encoding(val: &str) -> Result<(), String> {
488 // https://github.com/postgres/postgres/blob/REL_15_3/src/common/encnames.c#L525
489 let clean = val.replace(|c: char| !c.is_ascii_alphanumeric(), "");
490 if !clean.eq_ignore_ascii_case("UTF8") {
491 Err("Only support 'UTF8' for CLIENT_ENCODING".to_owned())
492 } else {
493 Ok(())
494 }
495}
496
497fn check_bytea_output(val: &str) -> Result<(), String> {
498 if val == "hex" {
499 Ok(())
500 } else {
501 Err("Only support 'hex' for BYTEA_OUTPUT".to_owned())
502 }
503}
504
505/// Check if the provided value is a valid max parallelism.
506fn check_streaming_max_parallelism(val: &usize) -> Result<(), String> {
507 match val {
508 // TODO(var-vnode): this is to prevent confusion with singletons, after we distinguish
509 // them better, we may allow 1 as the max parallelism (though not much point).
510 0 | 1 => Err("STREAMING_MAX_PARALLELISM must be greater than 1".to_owned()),
511 2..=VirtualNode::MAX_COUNT => Ok(()),
512 _ => Err(format!(
513 "STREAMING_MAX_PARALLELISM must be less than or equal to {}",
514 VirtualNode::MAX_COUNT
515 )),
516 }
517}
518
519impl SessionConfig {
520 pub fn set_force_two_phase_agg(
521 &mut self,
522 val: bool,
523 reporter: &mut impl ConfigReporter,
524 ) -> SessionConfigResult<bool> {
525 let set_val = self.set_force_two_phase_agg_inner(val, reporter)?;
526 if self.force_two_phase_agg {
527 self.set_enable_two_phase_agg(true, reporter)
528 } else {
529 Ok(set_val)
530 }
531 }
532
533 pub fn set_enable_two_phase_agg(
534 &mut self,
535 val: bool,
536 reporter: &mut impl ConfigReporter,
537 ) -> SessionConfigResult<bool> {
538 let set_val = self.set_enable_two_phase_agg_inner(val, reporter)?;
539 if !self.force_two_phase_agg {
540 self.set_force_two_phase_agg(false, reporter)
541 } else {
542 Ok(set_val)
543 }
544 }
545}
546
547pub struct VariableInfo {
548 pub name: String,
549 pub setting: String,
550 pub description: String,
551}
552
553/// Report status or notice to caller.
554pub trait ConfigReporter {
555 fn report_status(&mut self, key: &str, new_val: String);
556}
557
558// Report nothing.
559impl ConfigReporter for () {
560 fn report_status(&mut self, _key: &str, _new_val: String) {}
561}
562
563def_anyhow_newtype! {
564 pub SessionConfigToOverrideError,
565 toml::ser::Error => "failed to serialize session config",
566 ConfigMergeError => transparent,
567}
568
569impl SessionConfig {
570 /// Generate an initial override for the streaming config from the session config.
571 pub fn to_initial_streaming_config_override(
572 &self,
573 ) -> Result<String, SessionConfigToOverrideError> {
574 let mut table = toml::Table::new();
575
576 // TODO: make this more type safe.
577 // We `unwrap` here to assert the hard-coded keys are correct.
578 if let Some(v) = self.streaming_join_encoding.as_ref() {
579 table
580 .upsert("streaming.developer.join_encoding_type", v)
581 .unwrap();
582 }
583 if let Some(v) = self.streaming_sync_log_store_pause_duration_ms.as_ref() {
584 table
585 .upsert("streaming.developer.sync_log_store_pause_duration_ms", v)
586 .unwrap();
587 }
588 if let Some(v) = self.streaming_sync_log_store_buffer_size.as_ref() {
589 table
590 .upsert("streaming.developer.sync_log_store_buffer_size", v)
591 .unwrap();
592 }
593 if let Some(v) = self.streaming_over_window_cache_policy.as_ref() {
594 table
595 .upsert("streaming.developer.over_window_cache_policy", v)
596 .unwrap();
597 }
598
599 let res = toml::to_string(&table)?;
600
601 // Validate all fields are valid by trying to merge it to the default config.
602 if !res.is_empty() {
603 let merged =
604 merge_streaming_config_section(&StreamingConfig::default(), res.as_str())?.unwrap();
605
606 let unrecognized_keys = merged.unrecognized_keys().collect_vec();
607 if !unrecognized_keys.is_empty() {
608 bail!("unrecognized configs: {:?}", unrecognized_keys);
609 }
610 }
611
612 Ok(res)
613 }
614}
615
616#[cfg(test)]
617mod test {
618 use expect_test::expect;
619
620 use super::*;
621
622 #[derive(SessionConfig)]
623 struct TestConfig {
624 #[parameter(default = 1, flags = "NO_ALTER_SYS", alias = "test_param_alias" | "alias_param_test")]
625 test_param: i32,
626 }
627
628 #[test]
629 fn test_session_config_alias() {
630 let mut config = TestConfig::default();
631 config.set("test_param", "2".to_owned(), &mut ()).unwrap();
632 assert_eq!(config.get("test_param_alias").unwrap(), "2");
633 config
634 .set("alias_param_test", "3".to_owned(), &mut ())
635 .unwrap();
636 assert_eq!(config.get("test_param_alias").unwrap(), "3");
637 assert!(TestConfig::check_no_alter_sys("test_param").unwrap());
638 }
639
640 #[test]
641 fn test_initial_streaming_config_override() {
642 let mut config = SessionConfig::default();
643 config
644 .set_streaming_join_encoding(Some(JoinEncodingType::Cpu).into(), &mut ())
645 .unwrap();
646 config
647 .set_streaming_over_window_cache_policy(
648 Some(OverWindowCachePolicy::RecentFirstN).into(),
649 &mut (),
650 )
651 .unwrap();
652
653 // Check the converted config override string.
654 let override_str = config.to_initial_streaming_config_override().unwrap();
655 expect![[r#"
656 [streaming.developer]
657 join_encoding_type = "cpu_optimized"
658 over_window_cache_policy = "recent_first_n"
659 "#]]
660 .assert_eq(&override_str);
661
662 // Try merging it to the default streaming config.
663 let merged = merge_streaming_config_section(&StreamingConfig::default(), &override_str)
664 .unwrap()
665 .unwrap();
666 assert_eq!(merged.developer.join_encoding_type, JoinEncodingType::Cpu);
667 assert_eq!(
668 merged.developer.over_window_cache_policy,
669 OverWindowCachePolicy::RecentFirstN
670 );
671 }
672}