From 1cbb87b0b776c2f6cf7c909a5dca37e242c5eb23 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 19 Feb 2024 17:03:56 +0800 Subject: [PATCH] feat(frontend): support tab-completion for `[ALTER SYSTEM] SET` (#15123) Signed-off-by: Bugen Zhao --- e2e_test/batch/catalog/pg_settings.slt.part | 134 ++++++++++++------ .../src/catalog/system_catalog/mod.rs | 5 + .../system_catalog/pg_catalog/pg_settings.rs | 72 ++++++++-- src/frontend/src/scheduler/task_context.rs | 1 + src/frontend/src/session.rs | 13 +- 5 files changed, 169 insertions(+), 56 deletions(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 09e2546a856d8..5f37db11fcb91 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -1,50 +1,94 @@ query TT -SELECT name FROM pg_catalog.pg_settings order by name; +SELECT context, name FROM pg_catalog.pg_settings ORDER BY (context, name); ---- -application_name -background_ddl -batch_enable_distributed_dml -batch_parallelism -bytea_output -client_encoding -client_min_messages -create_compaction_group_for_mv -datestyle -extra_float_digits -idle_in_transaction_session_timeout -intervalstyle -lock_timeout -max_split_range_gap -query_epoch -query_mode -row_security -rw_batch_enable_lookup_join -rw_batch_enable_sort_agg -rw_enable_join_ordering -rw_enable_share_plan -rw_enable_two_phase_agg -rw_force_split_distinct_agg -rw_force_two_phase_agg -rw_implicit_flush -rw_streaming_allow_jsonb_in_stream_key -rw_streaming_enable_bushy_join -rw_streaming_enable_delta_join -rw_streaming_over_window_cache_policy -search_path -server_encoding -server_version -server_version_num -sink_decouple -standard_conforming_strings -statement_timeout -streaming_enable_arrangement_backfill -streaming_parallelism -streaming_rate_limit -synchronize_seqscans -timezone -transaction_isolation -visibility_mode +internal block_size_kb +internal bloom_false_positive +internal data_directory +internal parallel_compact_size_mb +internal sstable_size_mb +internal state_store +internal wasm_storage_url +postmaster backup_storage_directory +postmaster backup_storage_url +postmaster barrier_interval_ms +postmaster checkpoint_frequency +postmaster enable_tracing +postmaster max_concurrent_creating_streaming_jobs +postmaster pause_on_next_bootstrap +user application_name +user background_ddl +user batch_enable_distributed_dml +user batch_parallelism +user bytea_output +user client_encoding +user client_min_messages +user create_compaction_group_for_mv +user datestyle +user extra_float_digits +user idle_in_transaction_session_timeout +user intervalstyle +user lock_timeout +user max_split_range_gap +user query_epoch +user query_mode +user row_security +user rw_batch_enable_lookup_join +user rw_batch_enable_sort_agg +user rw_enable_join_ordering +user rw_enable_share_plan +user rw_enable_two_phase_agg +user rw_force_split_distinct_agg +user rw_force_two_phase_agg +user rw_implicit_flush +user rw_streaming_allow_jsonb_in_stream_key +user rw_streaming_enable_bushy_join +user rw_streaming_enable_delta_join +user rw_streaming_over_window_cache_policy +user search_path +user server_encoding +user server_version +user server_version_num +user sink_decouple +user standard_conforming_strings +user statement_timeout +user streaming_enable_arrangement_backfill +user streaming_parallelism +user streaming_rate_limit +user synchronize_seqscans +user timezone +user transaction_isolation +user visibility_mode query TT SELECT * FROM pg_catalog.pg_settings where name='dummy'; ----- \ No newline at end of file +---- + +# Tab-completion of `SET` command +query T +SELECT name +FROM + (SELECT pg_catalog.lower(name) AS name + FROM pg_catalog.pg_settings + WHERE context IN ('user', + 'superuser') + UNION ALL SELECT 'constraints' + UNION ALL SELECT 'transaction' + UNION ALL SELECT 'session' + UNION ALL SELECT 'role' + UNION ALL SELECT 'tablespace' + UNION ALL SELECT 'all') ss +WHERE substring(name, 1, 8)='search_p'; +---- +search_path + +# Tab-completion of `ALTER SYSTEM SET` command +query T +SELECT name +FROM + (SELECT pg_catalog.lower(name) AS name + FROM pg_catalog.pg_settings + WHERE context != 'internal' + UNION ALL SELECT 'all') ss +WHERE substring(name, 1, 7)='checkpo'; +---- +checkpoint_frequency diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 61ec69b77ae5a..18a4757601aed 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -31,6 +31,7 @@ use risingwave_common::catalog::{ }; use risingwave_common::error::BoxedError; use risingwave_common::session_config::ConfigMap; +use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::types::DataType; use risingwave_pb::meta::list_table_fragment_states_response::TableFragmentState; use risingwave_pb::meta::table_parallelism::{PbFixedParallelism, PbParallelism}; @@ -110,6 +111,8 @@ pub struct SysCatalogReaderImpl { auth_context: Arc, // Read config. config: Arc>, + // Read system params. + system_params: SystemParamsReaderRef, } impl SysCatalogReaderImpl { @@ -120,6 +123,7 @@ impl SysCatalogReaderImpl { meta_client: Arc, auth_context: Arc, config: Arc>, + system_params: SystemParamsReaderRef, ) -> Self { Self { catalog_reader, @@ -128,6 +132,7 @@ impl SysCatalogReaderImpl { meta_client, auth_context, config, + system_params, } } } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs index 4fc0fb057108f..0f079ca3f6452 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs @@ -12,7 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::Fields; +use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_common::types::{DataType, Datum, Fields, ToOwnedDatum, WithDataType}; use risingwave_frontend_macro::system_catalog; use crate::catalog::system_catalog::SysCatalogReaderImpl; @@ -24,19 +25,70 @@ struct PgSetting { name: String, setting: String, short_desc: String, + context: Context, +} + +/// Context required to set the parameter's value. +/// +/// Note that we do not strictly follow the PostgreSQL's semantics for each variant +/// but only pick the minimum set of variants required for features like tab-completion. +#[derive(Clone, Copy)] +enum Context { + /// Used for immutable system parameters. + Internal, + + /// Used for mutable system parameters. + // TODO: `postmaster` means that changes require a restart of the server. This is + // not accurate for all system parameters. Use lower contexts once we guarantee about + // the behavior of each parameter. + Postmaster, + + /// Used for session variables. + // TODO: There might be variables that can only be set by superusers in the future. + // Should use `superuser` context then. + User, +} + +impl WithDataType for Context { + fn default_data_type() -> DataType { + DataType::Varchar + } +} + +impl ToOwnedDatum for Context { + fn to_owned_datum(self) -> Datum { + match self { + Context::Internal => "internal", + Context::Postmaster => "postmaster", + Context::User => "user", + } + .to_owned_datum() + } } #[system_catalog(table, "pg_catalog.pg_settings")] fn read_pg_settings(reader: &SysCatalogReaderImpl) -> Vec { - let config_reader = reader.config.read(); - let all_variables = config_reader.show_all(); + let variables = (reader.config.read().show_all()) + .into_iter() + .map(|info| PgSetting { + name: info.name, + setting: info.setting, + short_desc: info.description, + context: Context::User, + }); - all_variables - .iter() + let system_params = (reader.system_params.load().get_all()) + .into_iter() .map(|info| PgSetting { - name: info.name.clone(), - setting: info.setting.clone(), - short_desc: info.description.clone(), - }) - .collect() + name: info.name.to_owned(), + setting: info.value, + short_desc: info.description.to_owned(), + context: if info.mutable { + Context::Postmaster + } else { + Context::Internal + }, + }); + + variables.chain(system_params).collect() } diff --git a/src/frontend/src/scheduler/task_context.rs b/src/frontend/src/scheduler/task_context.rs index dfb2496dad556..dcfbf30a215a1 100644 --- a/src/frontend/src/scheduler/task_context.rs +++ b/src/frontend/src/scheduler/task_context.rs @@ -52,6 +52,7 @@ impl BatchTaskContext for FrontendBatchTaskContext { self.session.env().meta_client_ref(), self.session.auth_context(), self.session.shared_config(), + self.session.env().system_params_manager().get_params(), )) } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 9419999f8e479..460c978535833 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -43,7 +43,9 @@ use risingwave_common::catalog::{ }; use risingwave_common::config::{load_config, BatchConfig, MetaConfig, MetricLevel}; use risingwave_common::session_config::{ConfigMap, ConfigReporter, VisibilityMode}; -use risingwave_common::system_param::local_manager::LocalSystemParamsManager; +use risingwave_common::system_param::local_manager::{ + LocalSystemParamsManager, LocalSystemParamsManagerRef, +}; use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common::types::DataType; @@ -119,6 +121,8 @@ pub struct FrontendEnv { worker_node_manager: WorkerNodeManagerRef, query_manager: QueryManager, hummock_snapshot_manager: HummockSnapshotManagerRef, + system_params_manager: LocalSystemParamsManagerRef, + server_addr: HostAddr, client_pool: ComputeClientPoolRef, @@ -159,6 +163,7 @@ impl FrontendEnv { let worker_node_manager = Arc::new(WorkerNodeManager::mock(vec![])); let meta_client = Arc::new(MockFrontendMetaClient {}); let hummock_snapshot_manager = Arc::new(HummockSnapshotManager::new(meta_client.clone())); + let system_params_manager = Arc::new(LocalSystemParamsManager::for_test()); let compute_client_pool = Arc::new(ComputeClientPool::default()); let query_manager = QueryManager::new( worker_node_manager.clone(), @@ -191,6 +196,7 @@ impl FrontendEnv { worker_node_manager, query_manager, hummock_snapshot_manager, + system_params_manager, server_addr, client_pool, sessions_map: Arc::new(RwLock::new(HashMap::new())), @@ -383,6 +389,7 @@ impl FrontendEnv { meta_client: frontend_meta_client, query_manager, hummock_snapshot_manager, + system_params_manager, server_addr: frontend_address, client_pool, frontend_metrics, @@ -448,6 +455,10 @@ impl FrontendEnv { &self.hummock_snapshot_manager } + pub fn system_params_manager(&self) -> &LocalSystemParamsManagerRef { + &self.system_params_manager + } + pub fn server_address(&self) -> &HostAddr { &self.server_addr }