Skip to content

Commit c82698a

Browse files
feat(catalog): implement pg_settings (#15108)
Signed-off-by: Runji Wang <[email protected]>
1 parent 0975be2 commit c82698a

File tree

7 files changed

+90
-27
lines changed

7 files changed

+90
-27
lines changed
+4-4
Original file line numberDiff line numberDiff line change
@@ -1,16 +1,16 @@
11
# UNION and other complex queries should also be in local mode
22
query I
3-
SELECT name FROM pg_catalog.pg_settings union select 'a';
3+
SELECT amname FROM pg_catalog.pg_am union select 'a';
44
----
55
a
66

77
query T
8-
SELECT name FROM (SELECT pg_catalog.lower(name) AS name FROM pg_catalog.pg_settings UNION ALL SELECT 'session authorization' UNION ALL SELECT 'all') ss WHERE substring(name,1,0)=''
9-
LIMIT 1000
8+
SELECT amname FROM (SELECT pg_catalog.lower(amname) AS amname FROM pg_catalog.pg_am UNION ALL SELECT 'session authorization' UNION ALL SELECT 'all') ss WHERE substring(amname,1,0)=''
9+
LIMIT 1000;
1010
----
1111
session authorization
1212
all
1313

1414
query I
15-
with q as ( select name FROM pg_catalog.pg_settings ) select * from q;
15+
with q as ( select amname FROM pg_catalog.pg_am ) select * from q;
1616
----

e2e_test/batch/catalog/pg_settings.slt.part

+44-1
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,49 @@
11
query TT
2-
SELECT * FROM pg_catalog.pg_settings;
2+
SELECT name FROM pg_catalog.pg_settings order by name;
33
----
4+
application_name
5+
background_ddl
6+
batch_enable_distributed_dml
7+
batch_parallelism
8+
bytea_output
9+
client_encoding
10+
client_min_messages
11+
create_compaction_group_for_mv
12+
datestyle
13+
extra_float_digits
14+
idle_in_transaction_session_timeout
15+
intervalstyle
16+
lock_timeout
17+
max_split_range_gap
18+
query_epoch
19+
query_mode
20+
row_security
21+
rw_batch_enable_lookup_join
22+
rw_batch_enable_sort_agg
23+
rw_enable_join_ordering
24+
rw_enable_share_plan
25+
rw_enable_two_phase_agg
26+
rw_force_split_distinct_agg
27+
rw_force_two_phase_agg
28+
rw_implicit_flush
29+
rw_streaming_allow_jsonb_in_stream_key
30+
rw_streaming_enable_bushy_join
31+
rw_streaming_enable_delta_join
32+
rw_streaming_over_window_cache_policy
33+
search_path
34+
server_encoding
35+
server_version
36+
server_version_num
37+
sink_decouple
38+
standard_conforming_strings
39+
statement_timeout
40+
streaming_enable_arrangement_backfill
41+
streaming_parallelism
42+
streaming_rate_limit
43+
synchronize_seqscans
44+
timezone
45+
transaction_isolation
46+
visibility_mode
447

548
query TT
649
SELECT * FROM pg_catalog.pg_settings where name='dummy';

src/frontend/src/catalog/system_catalog/mod.rs

+7
Original file line numberDiff line numberDiff line change
@@ -22,13 +22,15 @@ use std::sync::{Arc, LazyLock};
2222
use async_trait::async_trait;
2323
use futures::future::BoxFuture;
2424
use itertools::Itertools;
25+
use parking_lot::RwLock;
2526
use risingwave_common::acl::AclMode;
2627
use risingwave_common::array::DataChunk;
2728
use risingwave_common::catalog::{
2829
ColumnCatalog, ColumnDesc, Field, SysCatalogReader, TableDesc, TableId, DEFAULT_SUPER_USER_ID,
2930
NON_RESERVED_SYS_CATALOG_ID,
3031
};
3132
use risingwave_common::error::BoxedError;
33+
use risingwave_common::session_config::ConfigMap;
3234
use risingwave_common::types::DataType;
3335
use risingwave_pb::meta::list_table_fragment_states_response::TableFragmentState;
3436
use risingwave_pb::meta::table_parallelism::{PbFixedParallelism, PbParallelism};
@@ -104,7 +106,10 @@ pub struct SysCatalogReaderImpl {
104106
worker_node_manager: WorkerNodeManagerRef,
105107
// Read from meta.
106108
meta_client: Arc<dyn FrontendMetaClient>,
109+
// Read auth context.
107110
auth_context: Arc<AuthContext>,
111+
// Read config.
112+
config: Arc<RwLock<ConfigMap>>,
108113
}
109114

110115
impl SysCatalogReaderImpl {
@@ -114,13 +119,15 @@ impl SysCatalogReaderImpl {
114119
worker_node_manager: WorkerNodeManagerRef,
115120
meta_client: Arc<dyn FrontendMetaClient>,
116121
auth_context: Arc<AuthContext>,
122+
config: Arc<RwLock<ConfigMap>>,
117123
) -> Self {
118124
Self {
119125
catalog_reader,
120126
user_info_reader,
121127
worker_node_manager,
122128
meta_client,
123129
auth_context,
130+
config,
124131
}
125132
}
126133
}

src/frontend/src/catalog/system_catalog/pg_catalog/pg_settings.rs

+18-1
Original file line numberDiff line numberDiff line change
@@ -15,11 +15,28 @@
1515
use risingwave_common::types::Fields;
1616
use risingwave_frontend_macro::system_catalog;
1717

18+
use crate::catalog::system_catalog::SysCatalogReaderImpl;
19+
1820
/// The catalog `pg_settings` stores settings.
1921
/// Ref: [`https://www.postgresql.org/docs/current/view-pg-settings.html`]
20-
#[system_catalog(view, "pg_catalog.pg_settings")]
2122
#[derive(Fields)]
2223
struct PgSetting {
2324
name: String,
2425
setting: String,
26+
short_desc: String,
27+
}
28+
29+
#[system_catalog(table, "pg_catalog.pg_settings")]
30+
fn read_pg_settings(reader: &SysCatalogReaderImpl) -> Vec<PgSetting> {
31+
let config_reader = reader.config.read();
32+
let all_variables = config_reader.show_all();
33+
34+
all_variables
35+
.iter()
36+
.map(|info| PgSetting {
37+
name: info.name.clone(),
38+
setting: info.setting.clone(),
39+
short_desc: info.description.clone(),
40+
})
41+
.collect()
2542
}

src/frontend/src/scheduler/local.rs

+2-6
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,7 @@ use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId};
5252
use crate::scheduler::task_context::FrontendBatchTaskContext;
5353
use crate::scheduler::worker_node_manager::WorkerNodeSelector;
5454
use crate::scheduler::{ReadSnapshot, SchedulerError, SchedulerResult};
55-
use crate::session::{AuthContext, FrontendEnv, SessionImpl};
55+
use crate::session::{FrontendEnv, SessionImpl};
5656

5757
pub type LocalQueryStream = ReceiverStream<Result<DataChunk, BoxedError>>;
5858

@@ -94,10 +94,6 @@ impl LocalQueryExecution {
9494
}
9595
}
9696

97-
fn auth_context(&self) -> Arc<AuthContext> {
98-
self.session.auth_context()
99-
}
100-
10197
fn shutdown_rx(&self) -> ShutdownToken {
10298
self.session.reset_cancel_query_flag()
10399
}
@@ -106,7 +102,7 @@ impl LocalQueryExecution {
106102
pub async fn run_inner(self) {
107103
debug!(%self.query.query_id, self.sql, "Starting to run query");
108104

109-
let context = FrontendBatchTaskContext::new(self.front_env.clone(), self.auth_context());
105+
let context = FrontendBatchTaskContext::new(self.session.clone());
110106

111107
let task_id = TaskId {
112108
query_id: self.query.query_id.id.clone(),

src/frontend/src/scheduler/mod.rs

+1-1
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,6 @@ impl ExecutionContext {
6666
}
6767

6868
pub fn to_batch_task_context(&self) -> FrontendBatchTaskContext {
69-
FrontendBatchTaskContext::new(self.session.env().clone(), self.session.auth_context())
69+
FrontendBatchTaskContext::new(self.session.clone())
7070
}
7171
}

src/frontend/src/scheduler/task_context.rs

+14-14
Original file line numberDiff line numberDiff line change
@@ -25,18 +25,17 @@ use risingwave_connector::source::monitor::SourceMetrics;
2525
use risingwave_rpc_client::ComputeClientPoolRef;
2626

2727
use crate::catalog::system_catalog::SysCatalogReaderImpl;
28-
use crate::session::{AuthContext, FrontendEnv};
28+
use crate::session::SessionImpl;
2929

3030
/// Batch task execution context in frontend.
3131
#[derive(Clone)]
3232
pub struct FrontendBatchTaskContext {
33-
env: FrontendEnv,
34-
auth_context: Arc<AuthContext>,
33+
session: Arc<SessionImpl>,
3534
}
3635

3736
impl FrontendBatchTaskContext {
38-
pub fn new(env: FrontendEnv, auth_context: Arc<AuthContext>) -> Self {
39-
Self { env, auth_context }
37+
pub fn new(session: Arc<SessionImpl>) -> Self {
38+
Self { session }
4039
}
4140
}
4241

@@ -47,16 +46,17 @@ impl BatchTaskContext for FrontendBatchTaskContext {
4746

4847
fn catalog_reader(&self) -> SysCatalogReaderRef {
4948
Arc::new(SysCatalogReaderImpl::new(
50-
self.env.catalog_reader().clone(),
51-
self.env.user_info_reader().clone(),
52-
self.env.worker_node_manager_ref(),
53-
self.env.meta_client_ref(),
54-
self.auth_context.clone(),
49+
self.session.env().catalog_reader().clone(),
50+
self.session.env().user_info_reader().clone(),
51+
self.session.env().worker_node_manager_ref(),
52+
self.session.env().meta_client_ref(),
53+
self.session.auth_context(),
54+
self.session.shared_config(),
5555
))
5656
}
5757

5858
fn is_local_addr(&self, peer_addr: &HostAddr) -> bool {
59-
is_local_address(self.env.server_address(), peer_addr)
59+
is_local_address(self.session.env().server_address(), peer_addr)
6060
}
6161

6262
fn state_store(&self) -> risingwave_storage::store_impl::StateStoreImpl {
@@ -68,19 +68,19 @@ impl BatchTaskContext for FrontendBatchTaskContext {
6868
}
6969

7070
fn client_pool(&self) -> ComputeClientPoolRef {
71-
self.env.client_pool()
71+
self.session.env().client_pool()
7272
}
7373

7474
fn get_config(&self) -> &BatchConfig {
75-
self.env.batch_config()
75+
self.session.env().batch_config()
7676
}
7777

7878
fn dml_manager(&self) -> risingwave_dml::dml_manager::DmlManagerRef {
7979
unimplemented!("not supported in local mode")
8080
}
8181

8282
fn source_metrics(&self) -> Arc<SourceMetrics> {
83-
self.env.source_metrics()
83+
self.session.env().source_metrics()
8484
}
8585

8686
fn store_mem_usage(&self, _val: usize) {

0 commit comments

Comments
 (0)