risingwave_frontend/handler/
variable.rs1use anyhow::Context;
16use itertools::Itertools;
17use pgwire::pg_field_descriptor::PgFieldDescriptor;
18use pgwire::pg_protocol::ParameterStatus;
19use pgwire::pg_response::{PgResponse, StatementType};
20use risingwave_common::session_config::{ConfigReporter, SESSION_CONFIG_LIST_SEP};
21use risingwave_common::system_param::reader::SystemParamsRead;
22use risingwave_common::types::Fields;
23use risingwave_sqlparser::ast::{Ident, SetTimeZoneValue, SetVariableValue, Value};
24
25use super::{RwPgResponse, RwPgResponseBuilderExt, fields_to_descriptors};
26use crate::error::Result;
27use crate::handler::HandlerArgs;
28
29pub(crate) fn set_var_to_param_str(value: &SetVariableValue) -> Option<String> {
31 match value {
32 SetVariableValue::Single(var) => Some(var.to_string_unquoted()),
33 SetVariableValue::List(list) => Some(
34 list.iter()
35 .map(|var| var.to_string_unquoted())
36 .join(SESSION_CONFIG_LIST_SEP),
37 ),
38 SetVariableValue::Default => None,
39 }
40}
41
42pub fn handle_set(
43 handler_args: HandlerArgs,
44 name: Ident,
45 value: SetVariableValue,
46) -> Result<RwPgResponse> {
47 let string_val = set_var_to_param_str(&value);
49
50 let param_name = name.real_value().to_lowercase();
52 if param_name.eq_ignore_ascii_case("iceberg_engine_connection")
53 && let Some(val) = string_val.as_deref()
54 && !val.is_empty()
55 && let Some((schema_name, connection_name)) = val.split_once('.')
56 {
57 handler_args
58 .session
59 .get_connection_by_name(Some(schema_name.to_owned()), connection_name)?;
60 }
61
62 let mut status = ParameterStatus::default();
63
64 struct Reporter<'a> {
65 status: &'a mut ParameterStatus,
66 }
67
68 impl ConfigReporter for Reporter<'_> {
69 fn report_status(&mut self, key: &str, new_val: String) {
70 if key == "APPLICATION_NAME" {
71 self.status.application_name = Some(new_val);
72 }
73 }
74 }
75
76 handler_args.session.set_config_report(
80 ¶m_name,
81 string_val,
82 Reporter {
83 status: &mut status,
84 },
85 )?;
86
87 Ok(PgResponse::builder(StatementType::SET_VARIABLE)
88 .status(status)
89 .into())
90}
91
92pub(super) fn handle_set_time_zone(
93 handler_args: HandlerArgs,
94 value: SetTimeZoneValue,
95) -> Result<RwPgResponse> {
96 let tz_info = match value {
97 SetTimeZoneValue::Local => {
98 iana_time_zone::get_timezone().context("Failed to get local time zone")
99 }
100 SetTimeZoneValue::Default => Ok("UTC".to_owned()),
101 SetTimeZoneValue::Ident(ident) => Ok(ident.real_value()),
102 SetTimeZoneValue::Literal(Value::DoubleQuotedString(s))
103 | SetTimeZoneValue::Literal(Value::SingleQuotedString(s)) => Ok(s),
104 _ => Ok(value.to_string()),
105 }?;
106
107 handler_args.session.set_config("timezone", tz_info)?;
108
109 Ok(PgResponse::empty_result(StatementType::SET_VARIABLE))
110}
111
112pub(super) fn handle_show(handler_args: HandlerArgs, variable: Vec<Ident>) -> Result<RwPgResponse> {
113 let name = variable.iter().map(|e| e.real_value()).join(" ");
115 if name.eq_ignore_ascii_case("PARAMETERS") {
116 handle_show_system_params(handler_args)
117 } else if name.eq_ignore_ascii_case("ALL") {
118 handle_show_all(handler_args)
119 } else {
120 let config_reader = handler_args.session.config();
121 Ok(PgResponse::builder(StatementType::SHOW_VARIABLE)
122 .rows([ShowVariableRow {
123 name: config_reader.get(&name)?,
124 }])
125 .into())
126 }
127}
128
129fn handle_show_all(handler_args: HandlerArgs) -> Result<RwPgResponse> {
130 let config_reader = handler_args.session.config();
131
132 let all_variables = config_reader.show_all();
133
134 let rows = all_variables.iter().map(|info| ShowVariableAllRow {
135 name: info.name.clone(),
136 setting: info.setting.clone(),
137 description: info.description.clone(),
138 });
139 Ok(PgResponse::builder(StatementType::SHOW_VARIABLE)
140 .rows(rows)
141 .into())
142}
143
144fn handle_show_system_params(handler_args: HandlerArgs) -> Result<RwPgResponse> {
145 let params = handler_args
146 .session
147 .env()
148 .system_params_manager()
149 .get_params()
150 .load();
151 let rows = params
152 .get_all()
153 .into_iter()
154 .map(|info| ShowVariableParamsRow {
155 name: info.name.into(),
156 value: info.value,
157 description: info.description.into(),
158 mutable: info.mutable,
159 });
160 Ok(PgResponse::builder(StatementType::SHOW_VARIABLE)
161 .rows(rows)
162 .into())
163}
164
165pub fn infer_show_variable(name: &str) -> Vec<PgFieldDescriptor> {
166 fields_to_descriptors(if name.eq_ignore_ascii_case("ALL") {
167 ShowVariableAllRow::fields()
168 } else if name.eq_ignore_ascii_case("PARAMETERS") {
169 ShowVariableParamsRow::fields()
170 } else {
171 ShowVariableRow::fields()
172 })
173}
174
175#[derive(Fields)]
176#[fields(style = "Title Case")]
177struct ShowVariableRow {
178 name: String,
179}
180
181#[derive(Fields)]
182#[fields(style = "Title Case")]
183struct ShowVariableAllRow {
184 name: String,
185 setting: String,
186 description: String,
187}
188
189#[derive(Fields)]
190#[fields(style = "Title Case")]
191struct ShowVariableParamsRow {
192 name: String,
193 value: String,
194 description: String,
195 mutable: bool,
196}