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