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