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 /// Unsafe: allow impure expressions on non-append-only streams without materialization.
226 ///
227 /// This may lead to inconsistent results or panics due to re-evaluation on updates/retracts.
228 #[parameter(default = false)]
229 streaming_unsafe_allow_unmaterialized_impure_expr: bool,
230
231 /// Separate consecutive `StreamHashJoin` by no-shuffle `StreamExchange`
232 #[parameter(default = false)]
233 streaming_separate_consecutive_join: bool,
234
235 /// Separate `StreamSink` by no-shuffle `StreamExchange`
236 #[parameter(default = false)]
237 streaming_separate_sink: bool,
238
239 /// Determine which encoding will be used to encode join rows in operator cache.
240 ///
241 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
242 /// taking effect for new streaming jobs created in the current session.
243 #[parameter(default = None)]
244 streaming_join_encoding: OptionConfig<JoinEncodingType>,
245
246 /// Enable join ordering for streaming and batch queries. Defaults to true.
247 #[parameter(default = true, alias = "rw_enable_join_ordering")]
248 enable_join_ordering: bool,
249
250 /// Enable two phase agg optimization. Defaults to true.
251 /// Setting this to true will always set `FORCE_TWO_PHASE_AGG` to false.
252 #[parameter(default = true, flags = "SETTER", alias = "rw_enable_two_phase_agg")]
253 enable_two_phase_agg: bool,
254
255 /// Force two phase agg optimization whenever there's a choice between
256 /// optimizations. Defaults to false.
257 /// Setting this to true will always set `ENABLE_TWO_PHASE_AGG` to false.
258 #[parameter(default = false, flags = "SETTER", alias = "rw_force_two_phase_agg")]
259 force_two_phase_agg: bool,
260
261 /// Enable sharing of common sub-plans.
262 /// This means that DAG structured query plans can be constructed,
263 #[parameter(default = true, alias = "rw_enable_share_plan")]
264 /// rather than only tree structured query plans.
265 enable_share_plan: bool,
266
267 /// Enable split distinct agg
268 #[parameter(default = false, alias = "rw_force_split_distinct_agg")]
269 force_split_distinct_agg: bool,
270
271 /// See <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-INTERVALSTYLE>
272 #[parameter(default = "", rename = "intervalstyle")]
273 interval_style: String,
274
275 /// If `BATCH_PARALLELISM` is non-zero, batch queries will use this parallelism.
276 #[parameter(default = ConfigNonZeroU64::default())]
277 batch_parallelism: ConfigNonZeroU64,
278
279 /// The version of PostgreSQL that Risingwave claims to be.
280 #[parameter(default = PG_VERSION)]
281 server_version: String,
282
283 /// The version of PostgreSQL that Risingwave claims to be.
284 #[parameter(default = SERVER_VERSION_NUM)]
285 server_version_num: i32,
286
287 /// see <https://www.postgresql.org/docs/15/runtime-config-client.html#GUC-CLIENT-MIN-MESSAGES>
288 #[parameter(default = "notice")]
289 client_min_messages: String,
290
291 /// see <https://www.postgresql.org/docs/15/runtime-config-client.html#GUC-CLIENT-ENCODING>
292 #[parameter(default = SERVER_ENCODING, check_hook = check_client_encoding)]
293 client_encoding: String,
294
295 /// Enable decoupling sink and internal streaming graph or not
296 #[parameter(default = SinkDecouple::default())]
297 sink_decouple: SinkDecouple,
298
299 /// See <https://www.postgresql.org/docs/current/runtime-config-compatible.html#RUNTIME-CONFIG-COMPATIBLE-VERSION>
300 /// Unused in RisingWave, support for compatibility.
301 #[parameter(default = false)]
302 synchronize_seqscans: bool,
303
304 /// Abort query statement that takes more than the specified amount of time in sec. If
305 /// `log_min_error_statement` is set to ERROR or lower, the statement that timed out will also be
306 /// logged. If this value is specified without units, it is taken as milliseconds. A value of
307 /// zero (the default) disables the timeout.
308 #[parameter(default = 0u32)]
309 statement_timeout: u32,
310
311 /// 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.
312 #[parameter(default = 60000u32)]
313 idle_in_transaction_session_timeout: u32,
314
315 /// See <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-LOCK-TIMEOUT>
316 /// Unused in RisingWave, support for compatibility.
317 #[parameter(default = 0)]
318 lock_timeout: i32,
319
320 /// For limiting the startup time of a shareable CDC streaming source when the source is being created. Unit: seconds.
321 #[parameter(default = 60)]
322 cdc_source_wait_streaming_start_timeout: i32,
323
324 /// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-ROW-SECURITY>.
325 /// Unused in RisingWave, support for compatibility.
326 #[parameter(default = true)]
327 row_security: bool,
328
329 /// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-STANDARD-CONFORMING-STRINGS>
330 #[parameter(default = STANDARD_CONFORMING_STRINGS)]
331 standard_conforming_strings: String,
332
333 /// Set streaming rate limit (rows per second) for each parallelism for mv / source / sink backfilling
334 /// If set to -1, disable rate limit.
335 /// If set to 0, this pauses the snapshot read / source read.
336 #[parameter(default = DISABLE_BACKFILL_RATE_LIMIT)]
337 backfill_rate_limit: i32,
338
339 /// Set streaming rate limit (rows per second) for each parallelism for mv / source backfilling, source reads.
340 /// If set to -1, disable rate limit.
341 /// If set to 0, this pauses the snapshot read / source read.
342 #[parameter(default = DISABLE_SOURCE_RATE_LIMIT)]
343 source_rate_limit: i32,
344
345 /// Set streaming rate limit (rows per second) for each parallelism for table DML.
346 /// If set to -1, disable rate limit.
347 /// If set to 0, this pauses the DML.
348 #[parameter(default = DISABLE_DML_RATE_LIMIT)]
349 dml_rate_limit: i32,
350
351 /// Set sink rate limit (rows per second) for each parallelism for external sink.
352 /// If set to -1, disable rate limit.
353 /// If set to 0, this pauses the sink.
354 #[parameter(default = DISABLE_SINK_RATE_LIMIT)]
355 sink_rate_limit: i32,
356
357 /// Cache policy for partition cache in streaming over window.
358 /// Can be `full`, `recent`, `recent_first_n` or `recent_last_n`.
359 ///
360 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
361 /// taking effect for new streaming jobs created in the current session.
362 #[parameter(default = None, alias = "rw_streaming_over_window_cache_policy")]
363 streaming_over_window_cache_policy: OptionConfig<OverWindowCachePolicy>,
364
365 /// Run DDL statements in background
366 #[parameter(default = false)]
367 background_ddl: bool,
368
369 /// Enable shared source. Currently only for Kafka.
370 ///
371 /// When enabled, `CREATE SOURCE` will create a source streaming job, and `CREATE MATERIALIZED VIEWS` from the source
372 /// will forward the data from the same source streaming job, and also backfill prior data from the external source.
373 #[parameter(default = true)]
374 streaming_use_shared_source: bool,
375
376 /// 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.
377 #[parameter(default = SERVER_ENCODING)]
378 server_encoding: String,
379
380 #[parameter(default = "hex", check_hook = check_bytea_output)]
381 bytea_output: String,
382
383 /// Bypass checks on cluster limits
384 ///
385 /// When enabled, `CREATE MATERIALIZED VIEW` will not fail if the cluster limit is hit.
386 #[parameter(default = BYPASS_CLUSTER_LIMITS)]
387 bypass_cluster_limits: bool,
388
389 /// The maximum number of parallelism a streaming query can use. Defaults to 256.
390 ///
391 /// Compared to `STREAMING_PARALLELISM`, which configures the initial parallelism, this configures
392 /// the maximum parallelism a streaming query can use in the future, if the cluster size changes or
393 /// users manually change the parallelism with `ALTER .. SET PARALLELISM`.
394 ///
395 /// It's not always a good idea to set this to a very large number, as it may cause performance
396 /// degradation when performing range scans on the table or the materialized view.
397 // a.k.a. vnode count
398 #[parameter(default = VirtualNode::COUNT_FOR_COMPAT, check_hook = check_streaming_max_parallelism)]
399 streaming_max_parallelism: usize,
400
401 /// Used to provide the connection information for the iceberg engine.
402 /// Format: `iceberg_engine_connection` = `schema_name.connection_name`.
403 #[parameter(default = "", check_hook = check_iceberg_engine_connection)]
404 iceberg_engine_connection: String,
405
406 /// Whether the streaming join should be unaligned or not.
407 #[parameter(default = false)]
408 streaming_enable_unaligned_join: bool,
409
410 /// The timeout for reading from the buffer of the sync log store on barrier.
411 /// Every epoch we will attempt to read the full buffer of the sync log store.
412 /// If we hit the timeout, we will stop reading and continue.
413 ///
414 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
415 /// taking effect for new streaming jobs created in the current session.
416 #[parameter(default = None)]
417 streaming_sync_log_store_pause_duration_ms: OptionConfig<usize>,
418
419 /// The max buffer size for sync logstore, before we start flushing.
420 ///
421 /// This overrides the corresponding entry from the `[streaming.developer]` section in the config file,
422 /// taking effect for new streaming jobs created in the current session.
423 #[parameter(default = None)]
424 streaming_sync_log_store_buffer_size: OptionConfig<usize>,
425
426 /// Whether to disable purifying the definition of the table or source upon retrieval.
427 /// Only set this if encountering issues with functionalities like `SHOW` or `ALTER TABLE/SOURCE`.
428 /// This config may be removed in the future.
429 #[parameter(default = false, flags = "NO_ALTER_SYS")]
430 disable_purify_definition: bool,
431
432 /// The `ef_search` used in querying hnsw vector index
433 #[parameter(default = 40_usize)] // default value borrowed from pg_vector
434 batch_hnsw_ef_search: usize,
435
436 /// Enable index selection for queries
437 #[parameter(default = true)]
438 enable_index_selection: bool,
439
440 /// Enable locality backfill for streaming queries. Defaults to false.
441 #[parameter(default = false)]
442 enable_locality_backfill: bool,
443
444 /// Duration in seconds before notifying the user that a long-running DDL operation (e.g., DROP TABLE, CANCEL JOBS)
445 /// is still running. Set to 0 to disable notifications. Defaults to 30 seconds.
446 #[parameter(default = 30u32)]
447 slow_ddl_notification_secs: u32,
448
449 /// Unsafe: Enable storage retention for non-append-only tables.
450 /// Enabling this can lead to streaming inconsistency and node panic
451 /// if there is any row INSERT/UPDATE/DELETE operation corresponding to the ttled primary key.
452 #[parameter(default = false)]
453 unsafe_enable_storage_retention_for_non_append_only_tables: bool,
454
455 /// Enable DataFusion Engine
456 /// When enabled, queries involving Iceberg tables will be executed using the DataFusion engine.
457 #[parameter(default = false)]
458 enable_datafusion_engine: bool,
459
460 /// Emit chunks in upsert format for `UPDATE` and `DELETE` DMLs.
461 /// May lead to undefined behavior if the table is created with `ON CONFLICT DO NOTHING`.
462 ///
463 /// When enabled:
464 /// - `UPDATE` will only emit `Insert` records for new rows, instead of `Update` records.
465 /// - `DELETE` will only include key columns and pad the rest with NULL, instead of emitting complete rows.
466 #[parameter(default = false)]
467 upsert_dml: bool,
468}
469
470fn check_iceberg_engine_connection(val: &str) -> Result<(), String> {
471 if val.is_empty() {
472 return Ok(());
473 }
474
475 let parts: Vec<&str> = val.split('.').collect();
476 if parts.len() != 2 {
477 return Err("Invalid iceberg engine connection format, Should be set to this format: schema_name.connection_name.".to_owned());
478 }
479
480 Ok(())
481}
482
483fn check_timezone(val: &str) -> Result<(), String> {
484 // Check if the provided string is a valid timezone.
485 Tz::from_str_insensitive(val).map_err(|_e| "Not a valid timezone")?;
486 Ok(())
487}
488
489fn check_client_encoding(val: &str) -> Result<(), String> {
490 // https://github.com/postgres/postgres/blob/REL_15_3/src/common/encnames.c#L525
491 let clean = val.replace(|c: char| !c.is_ascii_alphanumeric(), "");
492 if !clean.eq_ignore_ascii_case("UTF8") {
493 Err("Only support 'UTF8' for CLIENT_ENCODING".to_owned())
494 } else {
495 Ok(())
496 }
497}
498
499fn check_bytea_output(val: &str) -> Result<(), String> {
500 if val == "hex" {
501 Ok(())
502 } else {
503 Err("Only support 'hex' for BYTEA_OUTPUT".to_owned())
504 }
505}
506
507/// Check if the provided value is a valid max parallelism.
508fn check_streaming_max_parallelism(val: &usize) -> Result<(), String> {
509 match val {
510 // TODO(var-vnode): this is to prevent confusion with singletons, after we distinguish
511 // them better, we may allow 1 as the max parallelism (though not much point).
512 0 | 1 => Err("STREAMING_MAX_PARALLELISM must be greater than 1".to_owned()),
513 2..=VirtualNode::MAX_COUNT => Ok(()),
514 _ => Err(format!(
515 "STREAMING_MAX_PARALLELISM must be less than or equal to {}",
516 VirtualNode::MAX_COUNT
517 )),
518 }
519}
520
521impl SessionConfig {
522 pub fn set_force_two_phase_agg(
523 &mut self,
524 val: bool,
525 reporter: &mut impl ConfigReporter,
526 ) -> SessionConfigResult<bool> {
527 let set_val = self.set_force_two_phase_agg_inner(val, reporter)?;
528 if self.force_two_phase_agg {
529 self.set_enable_two_phase_agg(true, reporter)
530 } else {
531 Ok(set_val)
532 }
533 }
534
535 pub fn set_enable_two_phase_agg(
536 &mut self,
537 val: bool,
538 reporter: &mut impl ConfigReporter,
539 ) -> SessionConfigResult<bool> {
540 let set_val = self.set_enable_two_phase_agg_inner(val, reporter)?;
541 if !self.force_two_phase_agg {
542 self.set_force_two_phase_agg(false, reporter)
543 } else {
544 Ok(set_val)
545 }
546 }
547}
548
549pub struct VariableInfo {
550 pub name: String,
551 pub setting: String,
552 pub description: String,
553}
554
555/// Report status or notice to caller.
556pub trait ConfigReporter {
557 fn report_status(&mut self, key: &str, new_val: String);
558}
559
560// Report nothing.
561impl ConfigReporter for () {
562 fn report_status(&mut self, _key: &str, _new_val: String) {}
563}
564
565def_anyhow_newtype! {
566 pub SessionConfigToOverrideError,
567 toml::ser::Error => "failed to serialize session config",
568 ConfigMergeError => transparent,
569}
570
571impl SessionConfig {
572 /// Generate an initial override for the streaming config from the session config.
573 pub fn to_initial_streaming_config_override(
574 &self,
575 ) -> Result<String, SessionConfigToOverrideError> {
576 let mut table = toml::Table::new();
577
578 // TODO: make this more type safe.
579 // We `unwrap` here to assert the hard-coded keys are correct.
580 if let Some(v) = self.streaming_join_encoding.as_ref() {
581 table
582 .upsert("streaming.developer.join_encoding_type", v)
583 .unwrap();
584 }
585 if let Some(v) = self.streaming_sync_log_store_pause_duration_ms.as_ref() {
586 table
587 .upsert("streaming.developer.sync_log_store_pause_duration_ms", v)
588 .unwrap();
589 }
590 if let Some(v) = self.streaming_sync_log_store_buffer_size.as_ref() {
591 table
592 .upsert("streaming.developer.sync_log_store_buffer_size", v)
593 .unwrap();
594 }
595 if let Some(v) = self.streaming_over_window_cache_policy.as_ref() {
596 table
597 .upsert("streaming.developer.over_window_cache_policy", v)
598 .unwrap();
599 }
600
601 let res = toml::to_string(&table)?;
602
603 // Validate all fields are valid by trying to merge it to the default config.
604 if !res.is_empty() {
605 let merged =
606 merge_streaming_config_section(&StreamingConfig::default(), res.as_str())?.unwrap();
607
608 let unrecognized_keys = merged.unrecognized_keys().collect_vec();
609 if !unrecognized_keys.is_empty() {
610 bail!("unrecognized configs: {:?}", unrecognized_keys);
611 }
612 }
613
614 Ok(res)
615 }
616}
617
618#[cfg(test)]
619mod test {
620 use expect_test::expect;
621
622 use super::*;
623
624 #[derive(SessionConfig)]
625 struct TestConfig {
626 #[parameter(default = 1, flags = "NO_ALTER_SYS", alias = "test_param_alias" | "alias_param_test")]
627 test_param: i32,
628 }
629
630 #[test]
631 fn test_session_config_alias() {
632 let mut config = TestConfig::default();
633 config.set("test_param", "2".to_owned(), &mut ()).unwrap();
634 assert_eq!(config.get("test_param_alias").unwrap(), "2");
635 config
636 .set("alias_param_test", "3".to_owned(), &mut ())
637 .unwrap();
638 assert_eq!(config.get("test_param_alias").unwrap(), "3");
639 assert!(TestConfig::check_no_alter_sys("test_param").unwrap());
640 }
641
642 #[test]
643 fn test_initial_streaming_config_override() {
644 let mut config = SessionConfig::default();
645 config
646 .set_streaming_join_encoding(Some(JoinEncodingType::Cpu).into(), &mut ())
647 .unwrap();
648 config
649 .set_streaming_over_window_cache_policy(
650 Some(OverWindowCachePolicy::RecentFirstN).into(),
651 &mut (),
652 )
653 .unwrap();
654
655 // Check the converted config override string.
656 let override_str = config.to_initial_streaming_config_override().unwrap();
657 expect![[r#"
658 [streaming.developer]
659 join_encoding_type = "cpu_optimized"
660 over_window_cache_policy = "recent_first_n"
661 "#]]
662 .assert_eq(&override_str);
663
664 // Try merging it to the default streaming config.
665 let merged = merge_streaming_config_section(&StreamingConfig::default(), &override_str)
666 .unwrap()
667 .unwrap();
668 assert_eq!(merged.developer.join_encoding_type, JoinEncodingType::Cpu);
669 assert_eq!(
670 merged.developer.over_window_cache_policy,
671 OverWindowCachePolicy::RecentFirstN
672 );
673 }
674}