risingwave_common/session_config/mod.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
mod non_zero64;
mod over_window;
mod query_mode;
mod search_path;
pub mod sink_decouple;
mod transaction_isolation_level;
mod visibility_mode;
use chrono_tz::Tz;
pub use over_window::OverWindowCachePolicy;
pub use query_mode::QueryMode;
use risingwave_common_proc_macro::{ConfigDoc, SessionConfig};
pub use search_path::{SearchPath, USER_NAME_WILD_CARD};
use serde::{Deserialize, Serialize};
use serde_with::{serde_as, DisplayFromStr};
use thiserror::Error;
use self::non_zero64::ConfigNonZeroU64;
use crate::hash::VirtualNode;
use crate::session_config::sink_decouple::SinkDecouple;
use crate::session_config::transaction_isolation_level::IsolationLevel;
pub use crate::session_config::visibility_mode::VisibilityMode;
use crate::{PG_VERSION, SERVER_ENCODING, SERVER_VERSION_NUM, STANDARD_CONFORMING_STRINGS};
pub const SESSION_CONFIG_LIST_SEP: &str = ", ";
#[derive(Error, Debug)]
pub enum SessionConfigError {
#[error("Invalid value `{value}` for `{entry}`")]
InvalidValue {
entry: &'static str,
value: String,
source: anyhow::Error,
},
#[error("Unrecognized config entry `{0}`")]
UnrecognizedEntry(String),
}
type SessionConfigResult<T> = std::result::Result<T, SessionConfigError>;
// NOTE(kwannoel): We declare it separately as a constant,
// otherwise seems like it can't infer the type of -1 when written inline.
const DISABLE_BACKFILL_RATE_LIMIT: i32 = -1;
const DISABLE_SOURCE_RATE_LIMIT: i32 = -1;
#[serde_as]
/// This is the Session Config of RisingWave.
#[derive(Clone, Debug, Deserialize, Serialize, SessionConfig, ConfigDoc, PartialEq)]
pub struct SessionConfig {
/// If `RW_IMPLICIT_FLUSH` is on, then every INSERT/UPDATE/DELETE statement will block
/// until the entire dataflow is refreshed. In other words, every related table & MV will
/// be able to see the write.
#[parameter(default = false, alias = "rw_implicit_flush")]
implicit_flush: bool,
/// If `CREATE_COMPACTION_GROUP_FOR_MV` is on, dedicated compaction groups will be created in
/// MV creation.
#[parameter(default = false)]
create_compaction_group_for_mv: bool,
/// A temporary config variable to force query running in either local or distributed mode.
/// The default value is auto which means let the system decide to run batch queries in local
/// or distributed mode automatically.
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = QueryMode::default())]
query_mode: QueryMode,
/// Sets the number of digits displayed for floating-point values.
/// See <https://www.postgresql.org/docs/current/runtime-config-client.html#:~:text=for%20more%20information.-,extra_float_digits,-(integer)>
#[parameter(default = 1)]
extra_float_digits: i32,
/// Sets the application name to be reported in statistics and logs.
/// See <https://www.postgresql.org/docs/14/runtime-config-logging.html#:~:text=What%20to%20Log-,application_name,-(string)>
#[parameter(default = "", flags = "REPORT")]
application_name: String,
/// It is typically set by an application upon connection to the server.
/// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-DATESTYLE>
#[parameter(default = "", rename = "datestyle")]
date_style: String,
/// Force the use of lookup join instead of hash join when possible for local batch execution.
#[parameter(default = true, alias = "rw_batch_enable_lookup_join")]
batch_enable_lookup_join: bool,
/// Enable usage of sortAgg instead of hash agg when order property is satisfied in batch
/// execution
#[parameter(default = true, alias = "rw_batch_enable_sort_agg")]
batch_enable_sort_agg: bool,
/// Enable distributed DML, so an insert, delete, and update statement can be executed in a distributed way (e.g. running in multiple compute nodes).
/// 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.
#[parameter(default = false, rename = "batch_enable_distributed_dml")]
batch_enable_distributed_dml: bool,
/// The max gap allowed to transform small range scan into multi point lookup.
#[parameter(default = 8)]
max_split_range_gap: i32,
/// Sets the order in which schemas are searched when an object (table, data type, function, etc.)
/// is referenced by a simple name with no schema specified.
/// See <https://www.postgresql.org/docs/14/runtime-config-client.html#GUC-SEARCH-PATH>
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = SearchPath::default())]
search_path: SearchPath,
/// If `VISIBILITY_MODE` is all, we will support querying data without checkpoint.
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = VisibilityMode::default())]
visibility_mode: VisibilityMode,
/// See <https://www.postgresql.org/docs/current/transaction-iso.html>
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = IsolationLevel::default())]
transaction_isolation: IsolationLevel,
/// Select as of specific epoch.
/// Sets the historical epoch for querying data. If 0, querying latest data.
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = ConfigNonZeroU64::default())]
query_epoch: ConfigNonZeroU64,
/// Session timezone. Defaults to UTC.
#[parameter(default = "UTC", check_hook = check_timezone)]
timezone: String,
/// The execution parallelism for streaming queries, including tables, materialized views, indexes,
/// and sinks. Defaults to 0, which means they will be scheduled adaptively based on the cluster size.
///
/// If a non-zero value is set, streaming queries will be scheduled to use a fixed number of parallelism.
/// Note that the value will be bounded at `STREAMING_MAX_PARALLELISM`.
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = ConfigNonZeroU64::default())]
streaming_parallelism: ConfigNonZeroU64,
/// Enable delta join for streaming queries. Defaults to false.
#[parameter(default = false, alias = "rw_streaming_enable_delta_join")]
streaming_enable_delta_join: bool,
/// Enable bushy join for streaming queries. Defaults to true.
#[parameter(default = true, alias = "rw_streaming_enable_bushy_join")]
streaming_enable_bushy_join: bool,
/// Enable arrangement backfill for streaming queries. Defaults to true.
/// When set to true, the parallelism of the upstream fragment will be
/// decoupled from the parallelism of the downstream scan fragment.
/// Or more generally, the parallelism of the upstream table / index / mv
/// will be decoupled from the parallelism of the downstream table / index / mv / sink.
#[parameter(default = true)]
streaming_use_arrangement_backfill: bool,
#[parameter(default = false)]
streaming_use_snapshot_backfill: bool,
/// Allow `jsonb` in stream key
#[parameter(default = false, alias = "rw_streaming_allow_jsonb_in_stream_key")]
streaming_allow_jsonb_in_stream_key: bool,
/// Enable join ordering for streaming and batch queries. Defaults to true.
#[parameter(default = true, alias = "rw_enable_join_ordering")]
enable_join_ordering: bool,
/// Enable two phase agg optimization. Defaults to true.
/// Setting this to true will always set `FORCE_TWO_PHASE_AGG` to false.
#[parameter(default = true, flags = "SETTER", alias = "rw_enable_two_phase_agg")]
enable_two_phase_agg: bool,
/// Force two phase agg optimization whenever there's a choice between
/// optimizations. Defaults to false.
/// Setting this to true will always set `ENABLE_TWO_PHASE_AGG` to false.
#[parameter(default = false, flags = "SETTER", alias = "rw_force_two_phase_agg")]
force_two_phase_agg: bool,
/// Enable sharing of common sub-plans.
/// This means that DAG structured query plans can be constructed,
#[parameter(default = true, alias = "rw_enable_share_plan")]
/// rather than only tree structured query plans.
enable_share_plan: bool,
/// Enable split distinct agg
#[parameter(default = false, alias = "rw_force_split_distinct_agg")]
force_split_distinct_agg: bool,
/// See <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-INTERVALSTYLE>
#[parameter(default = "", rename = "intervalstyle")]
interval_style: String,
/// If `BATCH_PARALLELISM` is non-zero, batch queries will use this parallelism.
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = ConfigNonZeroU64::default())]
batch_parallelism: ConfigNonZeroU64,
/// The version of PostgreSQL that Risingwave claims to be.
#[parameter(default = PG_VERSION)]
server_version: String,
/// The version of PostgreSQL that Risingwave claims to be.
#[parameter(default = SERVER_VERSION_NUM)]
server_version_num: i32,
/// see <https://www.postgresql.org/docs/15/runtime-config-client.html#GUC-CLIENT-MIN-MESSAGES>
#[parameter(default = "notice")]
client_min_messages: String,
/// see <https://www.postgresql.org/docs/15/runtime-config-client.html#GUC-CLIENT-ENCODING>
#[parameter(default = SERVER_ENCODING, check_hook = check_client_encoding)]
client_encoding: String,
/// Enable decoupling sink and internal streaming graph or not
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = SinkDecouple::default())]
sink_decouple: SinkDecouple,
/// See <https://www.postgresql.org/docs/current/runtime-config-compatible.html#RUNTIME-CONFIG-COMPATIBLE-VERSION>
/// Unused in RisingWave, support for compatibility.
#[parameter(default = false)]
synchronize_seqscans: bool,
/// Abort query statement that takes more than the specified amount of time in sec. If
/// `log_min_error_statement` is set to ERROR or lower, the statement that timed out will also be
/// logged. If this value is specified without units, it is taken as milliseconds. A value of
/// zero (the default) disables the timeout.
#[parameter(default = 0u32)]
statement_timeout: u32,
/// 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.
#[parameter(default = 60000u32)]
idle_in_transaction_session_timeout: u32,
/// See <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-LOCK-TIMEOUT>
/// Unused in RisingWave, support for compatibility.
#[parameter(default = 0)]
lock_timeout: i32,
/// For limiting the startup time of a shareable CDC streaming source when the source is being created. Unit: seconds.
#[parameter(default = 30)]
cdc_source_wait_streaming_start_timeout: i32,
/// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-ROW-SECURITY>.
/// Unused in RisingWave, support for compatibility.
#[parameter(default = true)]
row_security: bool,
/// see <https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-STANDARD-CONFORMING-STRINGS>
#[parameter(default = STANDARD_CONFORMING_STRINGS)]
standard_conforming_strings: String,
/// Set streaming rate limit (rows per second) for each parallelism for mv / source / sink backfilling
/// If set to -1, disable rate limit.
/// If set to 0, this pauses the snapshot read / source read.
#[parameter(default = DISABLE_BACKFILL_RATE_LIMIT)]
backfill_rate_limit: i32,
/// Set streaming rate limit (rows per second) for each parallelism for mv / source backfilling, source reads.
/// If set to -1, disable rate limit.
/// If set to 0, this pauses the snapshot read / source read.
#[parameter(default = DISABLE_SOURCE_RATE_LIMIT)]
source_rate_limit: i32,
/// Cache policy for partition cache in streaming over window.
/// Can be "full", "recent", "`recent_first_n`" or "`recent_last_n`".
#[serde_as(as = "DisplayFromStr")]
#[parameter(default = OverWindowCachePolicy::default(), alias = "rw_streaming_over_window_cache_policy")]
streaming_over_window_cache_policy: OverWindowCachePolicy,
/// Run DDL statements in background
#[parameter(default = false)]
background_ddl: bool,
/// Enable shared source. Currently only for Kafka.
///
/// When enabled, `CREATE SOURCE` will create a source streaming job, and `CREATE MATERIALIZED VIEWS` from the source
/// will forward the data from the same source streaming job, and also backfill prior data from the external source.
#[parameter(default = true)]
streaming_use_shared_source: bool,
/// 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.
#[parameter(default = SERVER_ENCODING)]
server_encoding: String,
#[parameter(default = "hex", check_hook = check_bytea_output)]
bytea_output: String,
/// Bypass checks on cluster limits
///
/// When enabled, `CREATE MATERIALIZED VIEW` will not fail if the cluster limit is hit.
#[parameter(default = false)]
bypass_cluster_limits: bool,
/// The maximum number of parallelism a streaming query can use. Defaults to 256.
///
/// Compared to `STREAMING_PARALLELISM`, which configures the initial parallelism, this configures
/// the maximum parallelism a streaming query can use in the future, if the cluster size changes or
/// users manually change the parallelism with `ALTER .. SET PARALLELISM`.
///
/// It's not always a good idea to set this to a very large number, as it may cause performance
/// degradation when performing range scans on the table or the materialized view.
// a.k.a. vnode count
#[parameter(default = VirtualNode::COUNT_FOR_COMPAT, check_hook = check_streaming_max_parallelism)]
streaming_max_parallelism: usize,
}
fn check_timezone(val: &str) -> Result<(), String> {
// Check if the provided string is a valid timezone.
Tz::from_str_insensitive(val).map_err(|_e| "Not a valid timezone")?;
Ok(())
}
fn check_client_encoding(val: &str) -> Result<(), String> {
// https://github.com/postgres/postgres/blob/REL_15_3/src/common/encnames.c#L525
let clean = val.replace(|c: char| !c.is_ascii_alphanumeric(), "");
if !clean.eq_ignore_ascii_case("UTF8") {
Err("Only support 'UTF8' for CLIENT_ENCODING".to_string())
} else {
Ok(())
}
}
fn check_bytea_output(val: &str) -> Result<(), String> {
if val == "hex" {
Ok(())
} else {
Err("Only support 'hex' for BYTEA_OUTPUT".to_string())
}
}
/// Check if the provided value is a valid max parallelism.
fn check_streaming_max_parallelism(val: &usize) -> Result<(), String> {
match val {
// TODO(var-vnode): this is to prevent confusion with singletons, after we distinguish
// them better, we may allow 1 as the max parallelism (though not much point).
0 | 1 => Err("STREAMING_MAX_PARALLELISM must be greater than 1".to_owned()),
2..=VirtualNode::MAX_COUNT => Ok(()),
_ => Err(format!(
"STREAMING_MAX_PARALLELISM must be less than or equal to {}",
VirtualNode::MAX_COUNT
)),
}
}
impl SessionConfig {
pub fn set_force_two_phase_agg(
&mut self,
val: bool,
reporter: &mut impl ConfigReporter,
) -> SessionConfigResult<bool> {
let set_val = self.set_force_two_phase_agg_inner(val, reporter)?;
if self.force_two_phase_agg {
self.set_enable_two_phase_agg(true, reporter)
} else {
Ok(set_val)
}
}
pub fn set_enable_two_phase_agg(
&mut self,
val: bool,
reporter: &mut impl ConfigReporter,
) -> SessionConfigResult<bool> {
let set_val = self.set_enable_two_phase_agg_inner(val, reporter)?;
if !self.force_two_phase_agg {
self.set_force_two_phase_agg(false, reporter)
} else {
Ok(set_val)
}
}
}
pub struct VariableInfo {
pub name: String,
pub setting: String,
pub description: String,
}
/// Report status or notice to caller.
pub trait ConfigReporter {
fn report_status(&mut self, key: &str, new_val: String);
}
// Report nothing.
impl ConfigReporter for () {
fn report_status(&mut self, _key: &str, _new_val: String) {}
}
#[cfg(test)]
mod test {
use super::*;
#[derive(SessionConfig)]
struct TestConfig {
#[parameter(default = 1, flags = "NO_ALTER_SYS", alias = "test_param_alias" | "alias_param_test")]
test_param: i32,
}
#[test]
fn test_session_config_alias() {
let mut config = TestConfig::default();
config.set("test_param", "2".to_string(), &mut ()).unwrap();
assert_eq!(config.get("test_param_alias").unwrap(), "2");
config
.set("alias_param_test", "3".to_string(), &mut ())
.unwrap();
assert_eq!(config.get("test_param_alias").unwrap(), "3");
assert!(TestConfig::check_no_alter_sys("test_param").unwrap());
}
}