From 16ae65b69c775777c9260eb4835906e08007fb0a Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 18 Feb 2024 15:00:47 +0800 Subject: [PATCH 1/4] refactor(catalog): refactor system catalog using proc macro (#14947) Signed-off-by: Runji Wang --- Cargo.lock | 10 + Cargo.toml | 1 + e2e_test/batch/catalog/pg_class.slt.part | 30 +- src/batch/src/executor/sys_row_seq_scan.rs | 36 +- src/common/src/catalog/mod.rs | 4 +- src/common/src/util/chunk_coalesce.rs | 5 + src/frontend/Cargo.toml | 1 + src/frontend/macro/Cargo.toml | 16 + src/frontend/macro/src/lib.rs | 166 ++++++++++ .../tests/testdata/output/pg_catalog.yaml | 178 +--------- src/frontend/src/catalog/root_catalog.rs | 28 +- .../information_schema/columns.rs | 93 +++--- .../system_catalog/information_schema/mod.rs | 10 +- .../information_schema/tables.rs | 67 ++-- .../information_schema/views.rs | 42 ++- .../src/catalog/system_catalog/mod.rs | 237 ++++--------- .../catalog/system_catalog/pg_catalog/mod.rs | 97 ++---- .../system_catalog/pg_catalog/pg_am.rs | 29 +- .../system_catalog/pg_catalog/pg_attrdef.rs | 33 +- .../system_catalog/pg_catalog/pg_attribute.rs | 71 ++-- .../pg_catalog/pg_auth_members.rs | 35 +- .../system_catalog/pg_catalog/pg_cast.rs | 58 ++-- .../system_catalog/pg_catalog/pg_class.rs | 74 ++--- .../system_catalog/pg_catalog/pg_collation.rs | 43 +-- .../pg_catalog/pg_constraint.rs | 74 ++--- .../pg_catalog/pg_conversion.rs | 37 +-- .../system_catalog/pg_catalog/pg_database.rs | 67 ++-- .../system_catalog/pg_catalog/pg_depend.rs | 35 +- .../pg_catalog/pg_description.rs | 47 ++- .../system_catalog/pg_catalog/pg_enum.rs | 29 +- .../system_catalog/pg_catalog/pg_extension.rs | 39 +-- .../system_catalog/pg_catalog/pg_index.rs | 74 ++--- .../system_catalog/pg_catalog/pg_indexes.rs | 48 ++- .../system_catalog/pg_catalog/pg_inherits.rs | 29 +- .../system_catalog/pg_catalog/pg_keywords.rs | 26 +- .../system_catalog/pg_catalog/pg_locks.rs | 53 ++- .../system_catalog/pg_catalog/pg_matviews.rs | 64 ++-- .../system_catalog/pg_catalog/pg_namespace.rs | 36 +- .../system_catalog/pg_catalog/pg_opclass.rs | 39 +-- .../system_catalog/pg_catalog/pg_operator.rs | 51 ++- .../system_catalog/pg_catalog/pg_proc.rs | 34 +- .../system_catalog/pg_catalog/pg_roles.rs | 72 ++-- .../system_catalog/pg_catalog/pg_settings.rs | 23 +- .../system_catalog/pg_catalog/pg_shadow.rs | 77 ++--- .../pg_catalog/pg_shdescription.rs | 27 +- .../pg_catalog/pg_stat_activity.rs | 59 ++-- .../system_catalog/pg_catalog/pg_tables.rs | 64 ++-- .../pg_catalog/pg_tablespace.rs | 31 +- .../system_catalog/pg_catalog/pg_type.rs | 106 +++--- .../system_catalog/pg_catalog/pg_user.rs | 46 ++- .../system_catalog/pg_catalog/pg_views.rs | 44 ++- .../catalog/system_catalog/rw_catalog/mod.rs | 40 --- .../system_catalog/rw_catalog/rw_actors.rs | 54 ++- .../system_catalog/rw_catalog/rw_columns.rs | 275 +++++++--------- .../rw_catalog/rw_connections.rs | 68 ++-- .../system_catalog/rw_catalog/rw_databases.rs | 63 ++-- .../rw_catalog/rw_ddl_progress.rs | 81 +++-- .../rw_catalog/rw_description.rs | 106 +++--- .../rw_catalog/rw_event_logs.rs | 69 ++-- .../rw_catalog/rw_fragment_parallelism.rs | 106 +++--- .../system_catalog/rw_catalog/rw_fragments.rs | 127 +++---- .../system_catalog/rw_catalog/rw_functions.rs | 110 +++---- .../rw_catalog/rw_hummock_branched_objects.rs | 50 ++- .../rw_hummock_compact_task_assignment.rs | 95 +++--- .../rw_hummock_compact_task_progress.rs | 82 ++--- .../rw_hummock_compaction_group_configs.rs | 95 +++--- .../rw_catalog/rw_hummock_meta_configs.rs | 53 ++- .../rw_catalog/rw_hummock_pinned_snapshots.rs | 51 ++- .../rw_catalog/rw_hummock_pinned_versions.rs | 51 ++- .../rw_catalog/rw_hummock_version.rs | 188 +++++------ .../rw_catalog/rw_hummock_version_deltas.rs | 67 ++-- .../system_catalog/rw_catalog/rw_indexes.rs | 126 +++---- .../rw_catalog/rw_internal_tables.rs | 105 +++--- .../rw_catalog/rw_materialized_views.rs | 105 +++--- .../rw_catalog/rw_meta_snapshot.rs | 102 +++--- .../rw_catalog/rw_parallel_units.rs | 48 ++- .../rw_catalog/rw_relation_info.rs | 311 ++++++++---------- .../system_catalog/rw_catalog/rw_relations.rs | 53 ++- .../system_catalog/rw_catalog/rw_schemas.rs | 63 ++-- .../system_catalog/rw_catalog/rw_sinks.rs | 130 ++++---- .../system_catalog/rw_catalog/rw_sources.rs | 175 ++++------ .../rw_catalog/rw_streaming_parallelism.rs | 83 ++--- .../rw_catalog/rw_system_tables.rs | 85 ++--- .../rw_catalog/rw_table_fragments.rs | 59 ++-- .../rw_catalog/rw_table_stats.rs | 58 ++-- .../system_catalog/rw_catalog/rw_tables.rs | 105 +++--- .../system_catalog/rw_catalog/rw_types.rs | 69 ++-- .../rw_catalog/rw_user_secrets.rs | 75 ++--- .../system_catalog/rw_catalog/rw_users.rs | 64 ++-- .../system_catalog/rw_catalog/rw_views.rs | 73 ++-- .../rw_catalog/rw_worker_nodes.rs | 107 +++--- 91 files changed, 2756 insertions(+), 3766 deletions(-) create mode 100644 src/frontend/macro/Cargo.toml create mode 100644 src/frontend/macro/src/lib.rs diff --git a/Cargo.lock b/Cargo.lock index 75fc3bbb2a509..98467d29c0211 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9306,6 +9306,7 @@ dependencies = [ "risingwave_dml", "risingwave_expr", "risingwave_expr_impl", + "risingwave_frontend_macro", "risingwave_hummock_sdk", "risingwave_object_store", "risingwave_pb", @@ -9328,6 +9329,15 @@ dependencies = [ "workspace-hack", ] +[[package]] +name = "risingwave_frontend_macro" +version = "1.7.0-alpha" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.48", +] + [[package]] name = "risingwave_hummock_sdk" version = "1.7.0-alpha" diff --git a/Cargo.toml b/Cargo.toml index 98844e0f36f44..dbc4cb88b89c3 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -20,6 +20,7 @@ members = [ "src/expr/macro", "src/expr/udf", "src/frontend", + "src/frontend/macro", "src/frontend/planner_test", "src/java_binding", "src/jni_core", diff --git a/e2e_test/batch/catalog/pg_class.slt.part b/e2e_test/batch/catalog/pg_class.slt.part index 3b8a5162536e6..2f2ffbe016e3a 100644 --- a/e2e_test/batch/catalog/pg_class.slt.part +++ b/e2e_test/batch/catalog/pg_class.slt.part @@ -1,23 +1,23 @@ query ITIT SELECT oid,relname,relowner,relkind FROM pg_catalog.pg_class ORDER BY oid limit 15; ---- -1 pg_type 1 v -2 pg_namespace 1 v -3 pg_cast 1 v -4 pg_matviews 1 v -5 pg_user 1 v -6 pg_class 1 v -7 pg_index 1 v -8 pg_opclass 1 v -9 pg_collation 1 v -10 pg_am 1 v -11 pg_operator 1 v -12 pg_views 1 v -13 pg_attribute 1 v -14 pg_database 1 v +1 columns 1 v +2 tables 1 v +3 views 1 v +4 pg_am 1 v +5 pg_attrdef 1 v +6 pg_attribute 1 v +7 pg_auth_members 1 v +8 pg_cast 1 r +9 pg_class 1 v +10 pg_collation 1 v +11 pg_constraint 1 v +12 pg_conversion 1 v +13 pg_database 1 v +14 pg_depend 1 v 15 pg_description 1 v query ITIT SELECT oid,relname,relowner,relkind FROM pg_catalog.pg_class WHERE oid = 'pg_namespace'::regclass; ---- -2 pg_namespace 1 v +24 pg_namespace 1 v diff --git a/src/batch/src/executor/sys_row_seq_scan.rs b/src/batch/src/executor/sys_row_seq_scan.rs index 6bebf862e5ea8..fd427b132909f 100644 --- a/src/batch/src/executor/sys_row_seq_scan.rs +++ b/src/batch/src/executor/sys_row_seq_scan.rs @@ -15,9 +15,7 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::DataChunk; -use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, SysCatalogReaderRef, TableId}; -use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::ToOwnedDatum; +use risingwave_common::catalog::{ColumnDesc, Schema, SysCatalogReaderRef, TableId}; use risingwave_pb::batch_plan::plan_node::NodeBody; use crate::error::{BatchError, Result}; @@ -29,7 +27,7 @@ use crate::task::BatchTaskContext; pub struct SysRowSeqScanExecutor { table_id: TableId, schema: Schema, - column_ids: Vec, + column_indices: Vec, identity: String, sys_catalog_reader: SysCatalogReaderRef, @@ -39,14 +37,14 @@ impl SysRowSeqScanExecutor { pub fn new( table_id: TableId, schema: Schema, - column_id: Vec, + column_indices: Vec, identity: String, sys_catalog_reader: SysCatalogReaderRef, ) -> Self { Self { table_id, schema, - column_ids: column_id, + column_indices, identity, sys_catalog_reader, } @@ -78,12 +76,15 @@ impl BoxedExecutorBuilder for SysRowSeqScanExecutorBuilder { .map(|column_desc| ColumnDesc::from(column_desc.clone())) .collect_vec(); - let column_ids = column_descs.iter().map(|d| d.column_id).collect_vec(); + let column_indices = column_descs + .iter() + .map(|d| d.column_id.get_id() as usize) + .collect_vec(); let schema = Schema::new(column_descs.iter().map(Into::into).collect_vec()); Ok(Box::new(SysRowSeqScanExecutor::new( table_id, schema, - column_ids, + column_indices, source.plan_node().get_identity().clone(), sys_catalog_reader, ))) @@ -107,26 +108,13 @@ impl Executor for SysRowSeqScanExecutor { impl SysRowSeqScanExecutor { #[try_stream(boxed, ok = DataChunk, error = BatchError)] async fn do_executor(self: Box) { - let rows = self + let chunk = self .sys_catalog_reader .read_table(&self.table_id) .await .map_err(BatchError::SystemTable)?; - let filtered_rows = rows - .iter() - .map(|row| { - let datums = self - .column_ids - .iter() - .map(|column_id| row.datum_at(column_id.get_id() as usize).to_owned_datum()) - .collect_vec(); - OwnedRow::new(datums) - }) - .collect_vec(); - - if !filtered_rows.is_empty() { - let chunk = DataChunk::from_rows(&filtered_rows, &self.schema.data_types()); - yield chunk + if chunk.cardinality() != 0 { + yield chunk.project(&self.column_indices); } } } diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index c2dcb183d6308..5e11860ca180b 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -31,9 +31,9 @@ use risingwave_pb::catalog::HandleConflictBehavior as PbHandleConflictBehavior; use risingwave_pb::plan_common::ColumnDescVersion; pub use schema::{test_utils as schema_test_utils, Field, FieldDisplay, Schema}; +use crate::array::DataChunk; pub use crate::constants::hummock; use crate::error::BoxedError; -use crate::row::OwnedRow; use crate::types::DataType; /// The global version of the catalog. @@ -146,7 +146,7 @@ pub fn cdc_table_name_column_desc() -> ColumnDesc { /// The local system catalog reader in the frontend node. #[async_trait] pub trait SysCatalogReader: Sync + Send + 'static { - async fn read_table(&self, table_id: &TableId) -> Result, BoxedError>; + async fn read_table(&self, table_id: &TableId) -> Result; } pub type SysCatalogReaderRef = Arc; diff --git a/src/common/src/util/chunk_coalesce.rs b/src/common/src/util/chunk_coalesce.rs index 9e2c6754c57e9..5cdeb8026a0f1 100644 --- a/src/common/src/util/chunk_coalesce.rs +++ b/src/common/src/util/chunk_coalesce.rs @@ -143,6 +143,11 @@ impl DataChunkBuilder { } } + /// Build a data chunk from the current buffer. + pub fn finish(mut self) -> DataChunk { + self.build_data_chunk() + } + fn append_one_row_internal(&mut self, data_chunk: &DataChunk, row_idx: usize) { self.do_append_one_row_from_datums(data_chunk.row_at(row_idx).0.iter()); } diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index f6254f2a4b172..5b9a6afcc6106 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -62,6 +62,7 @@ risingwave_common_service = { workspace = true } risingwave_connector = { workspace = true } risingwave_dml = { workspace = true } risingwave_expr = { workspace = true } +risingwave_frontend_macro = { path = "macro" } risingwave_hummock_sdk = { workspace = true } risingwave_object_store = { workspace = true } risingwave_pb = { workspace = true } diff --git a/src/frontend/macro/Cargo.toml b/src/frontend/macro/Cargo.toml new file mode 100644 index 0000000000000..79f551acbaee2 --- /dev/null +++ b/src/frontend/macro/Cargo.toml @@ -0,0 +1,16 @@ +[package] +name = "risingwave_frontend_macro" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +keywords = { workspace = true } +license = { workspace = true } +repository = { workspace = true } + +[lib] +proc-macro = true + +[dependencies] +proc-macro2 = "1" +quote = "1" +syn = { version = "2", features = ["full", "extra-traits"] } diff --git a/src/frontend/macro/src/lib.rs b/src/frontend/macro/src/lib.rs new file mode 100644 index 0000000000000..8ba10a9f4454a --- /dev/null +++ b/src/frontend/macro/src/lib.rs @@ -0,0 +1,166 @@ +// 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. + +use proc_macro::TokenStream; +use proc_macro2::TokenStream as TokenStream2; +use quote::{format_ident, quote}; +use syn::parse::{Parse, ParseStream}; +use syn::{parse_macro_input, Ident, ItemFn, ItemStruct, LitStr, Result, Token}; + +#[proc_macro_attribute] +pub fn system_catalog(attr: TokenStream, item: TokenStream) -> TokenStream { + let mut input = item.clone(); + let attr = parse_macro_input!(attr as Attr); + let item = parse_macro_input!(item as syn::Item); + + match system_catalog_inner(attr, item) { + // concat item and generated code + Ok(output) => { + input.extend(TokenStream::from(output)); + input + } + Err(err) => err.to_compile_error().into(), + } +} + +fn system_catalog_inner(attr: Attr, item: syn::Item) -> Result { + match item { + syn::Item::Fn(item_fn) => gen_sys_table(attr, item_fn), + syn::Item::Struct(item_struct) => gen_sys_view(attr, item_struct), + _ => Err(syn::Error::new_spanned(item, "expect function or struct")), + } +} + +struct Attr { + kind: Ident, + schema_name: String, + table_name: String, + sql: Option, +} + +impl Parse for Attr { + fn parse(input: ParseStream<'_>) -> Result { + let kind = input.parse::()?; + input.parse::()?; + let name = input.parse::()?; + let full_name = name.value(); + let (schema_name, table_name) = full_name + .split_once('.') + .ok_or_else(|| syn::Error::new_spanned(name, "expect \"schema.table\""))?; + let sql = if input.parse::().is_ok() { + Some(input.parse::()?.value()) + } else { + None + }; + Ok(Attr { + kind, + schema_name: schema_name.to_string(), + table_name: table_name.to_string(), + sql, + }) + } +} + +/// Check if the type is `type_` and return `T`. +fn strip_outer_type<'a>(ty: &'a syn::Type, type_: &str) -> Option<&'a syn::Type> { + let syn::Type::Path(path) = ty else { + return None; + }; + let seg = path.path.segments.last()?; + if seg.ident != type_ { + return None; + } + let syn::PathArguments::AngleBracketed(args) = &seg.arguments else { + return None; + }; + let Some(syn::GenericArgument::Type(ty)) = args.args.first() else { + return None; + }; + Some(ty) +} + +fn gen_sys_table(attr: Attr, item_fn: ItemFn) -> Result { + if attr.kind != "table" { + return Err(syn::Error::new_spanned(attr.kind, "expect `table`")); + } + + let schema_name = &attr.schema_name; + let table_name = &attr.table_name; + let gen_fn_name = format_ident!("{}_{}", attr.schema_name, attr.table_name); + let user_fn_name = item_fn.sig.ident; + + let return_type_error = + || syn::Error::new_spanned(&item_fn.sig.output, "expect `-> Result>`"); + let syn::ReturnType::Type(_, ty) = &item_fn.sig.output else { + return Err(return_type_error()); + }; + let (return_result, ty) = match strip_outer_type(ty, "Result") { + Some(ty) => (true, ty), + None => (false, ty.as_ref()), + }; + let struct_type = strip_outer_type(ty, "Vec").ok_or_else(return_type_error)?; + let _await = item_fn.sig.asyncness.map(|_| quote!(.await)); + let handle_error = return_result.then(|| quote!(?)); + + Ok(quote! { + #[linkme::distributed_slice(crate::catalog::system_catalog::SYS_CATALOGS_SLICE)] + #[no_mangle] // to prevent duplicate schema.table name + fn #gen_fn_name() -> crate::catalog::system_catalog::BuiltinCatalog { + crate::catalog::system_catalog::BuiltinCatalog::Table(crate::catalog::system_catalog::BuiltinTable { + name: #table_name, + schema: #schema_name, + columns: #struct_type::fields(), + pk: #struct_type::primary_key(), + function: |reader| std::boxed::Box::pin(async { + let rows = #user_fn_name(reader) #_await #handle_error; + let mut builder = #struct_type::data_chunk_builder(rows.len() + 1); + for row in rows { + _ = builder.append_one_row(row.into_owned_row()); + } + Ok(builder.finish()) + }), + }) + } + }) +} + +fn gen_sys_view(attr: Attr, item_struct: ItemStruct) -> Result { + if attr.kind != "view" { + return Err(syn::Error::new_spanned(attr.kind, "expect `view`")); + } + let schema_name = &attr.schema_name; + let table_name = &attr.table_name; + let gen_fn_name = format_ident!("{}_{}", attr.schema_name, attr.table_name); + let struct_type = &item_struct.ident; + + let sql = if let Some(sql) = attr.sql { + quote! { #sql.into() } + } else { + quote! { crate::catalog::system_catalog::infer_dummy_view_sql(&fields) } + }; + + Ok(quote! { + #[linkme::distributed_slice(crate::catalog::system_catalog::SYS_CATALOGS_SLICE)] + #[no_mangle] // to prevent duplicate schema.table name + fn #gen_fn_name() -> crate::catalog::system_catalog::BuiltinCatalog { + let fields = #struct_type::fields(); + crate::catalog::system_catalog::BuiltinCatalog::View(crate::catalog::system_catalog::BuiltinView { + name: #table_name, + schema: #schema_name, + sql: #sql, + columns: fields, + }) + } + }) +} diff --git a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml index b369b85de69f5..cca1da2ee47ee 100644 --- a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml @@ -26,181 +26,9 @@ - sql: | select * from pg_catalog.pg_cast logical_plan: |- - LogicalProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_2, *VALUES*_0.column_3] } - └─LogicalShare { id: 2 } - └─LogicalProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_2, *VALUES*_0.column_3] } - └─LogicalValues - ├─rows: - │ ┌─[0:Int32, 16:Int32, 23:Int32, 'e':Varchar] - │ ├─[1:Int32, 16:Int32, 1043:Int32, 'a':Varchar] - │ ├─[2:Int32, 21:Int32, 23:Int32, 'i':Varchar] - │ ├─[3:Int32, 21:Int32, 20:Int32, 'i':Varchar] - │ ├─[4:Int32, 21:Int32, 700:Int32, 'i':Varchar] - │ ├─[5:Int32, 21:Int32, 701:Int32, 'i':Varchar] - │ ├─[6:Int32, 21:Int32, 1700:Int32, 'i':Varchar] - │ ├─[7:Int32, 21:Int32, 1043:Int32, 'a':Varchar] - │ ├─[8:Int32, 21:Int32, 1301:Int32, 'i':Varchar] - │ ├─[9:Int32, 23:Int32, 16:Int32, 'e':Varchar] - │ ├─[10:Int32, 23:Int32, 21:Int32, 'a':Varchar] - │ ├─[11:Int32, 23:Int32, 20:Int32, 'i':Varchar] - │ ├─[12:Int32, 23:Int32, 700:Int32, 'i':Varchar] - │ ├─[13:Int32, 23:Int32, 701:Int32, 'i':Varchar] - │ ├─[14:Int32, 23:Int32, 1700:Int32, 'i':Varchar] - │ ├─[15:Int32, 23:Int32, 1043:Int32, 'a':Varchar] - │ ├─[16:Int32, 23:Int32, 1301:Int32, 'i':Varchar] - │ ├─[17:Int32, 20:Int32, 21:Int32, 'a':Varchar] - │ ├─[18:Int32, 20:Int32, 23:Int32, 'a':Varchar] - │ ├─[19:Int32, 20:Int32, 700:Int32, 'i':Varchar] - │ ├─[20:Int32, 20:Int32, 701:Int32, 'i':Varchar] - │ ├─[21:Int32, 20:Int32, 1700:Int32, 'i':Varchar] - │ ├─[22:Int32, 20:Int32, 1043:Int32, 'a':Varchar] - │ ├─[23:Int32, 20:Int32, 1301:Int32, 'i':Varchar] - │ ├─[24:Int32, 700:Int32, 21:Int32, 'a':Varchar] - │ ├─[25:Int32, 700:Int32, 23:Int32, 'a':Varchar] - │ ├─[26:Int32, 700:Int32, 20:Int32, 'a':Varchar] - │ ├─[27:Int32, 700:Int32, 701:Int32, 'i':Varchar] - │ ├─[28:Int32, 700:Int32, 1700:Int32, 'a':Varchar] - │ ├─[29:Int32, 700:Int32, 1043:Int32, 'a':Varchar] - │ ├─[30:Int32, 701:Int32, 21:Int32, 'a':Varchar] - │ ├─[31:Int32, 701:Int32, 23:Int32, 'a':Varchar] - │ ├─[32:Int32, 701:Int32, 20:Int32, 'a':Varchar] - │ ├─[33:Int32, 701:Int32, 700:Int32, 'a':Varchar] - │ ├─[34:Int32, 701:Int32, 1700:Int32, 'a':Varchar] - │ ├─[35:Int32, 701:Int32, 1043:Int32, 'a':Varchar] - │ ├─[36:Int32, 1700:Int32, 21:Int32, 'a':Varchar] - │ ├─[37:Int32, 1700:Int32, 23:Int32, 'a':Varchar] - │ ├─[38:Int32, 1700:Int32, 20:Int32, 'a':Varchar] - │ ├─[39:Int32, 1700:Int32, 700:Int32, 'i':Varchar] - │ ├─[40:Int32, 1700:Int32, 701:Int32, 'i':Varchar] - │ ├─[41:Int32, 1700:Int32, 1043:Int32, 'a':Varchar] - │ ├─[42:Int32, 1082:Int32, 1043:Int32, 'a':Varchar] - │ ├─[43:Int32, 1082:Int32, 1114:Int32, 'i':Varchar] - │ ├─[44:Int32, 1082:Int32, 1184:Int32, 'i':Varchar] - │ ├─[45:Int32, 1043:Int32, 16:Int32, 'e':Varchar] - │ ├─[46:Int32, 1043:Int32, 21:Int32, 'e':Varchar] - │ ├─[47:Int32, 1043:Int32, 23:Int32, 'e':Varchar] - │ ├─[48:Int32, 1043:Int32, 20:Int32, 'e':Varchar] - │ ├─[49:Int32, 1043:Int32, 700:Int32, 'e':Varchar] - │ ├─[50:Int32, 1043:Int32, 701:Int32, 'e':Varchar] - │ ├─[51:Int32, 1043:Int32, 1700:Int32, 'e':Varchar] - │ ├─[52:Int32, 1043:Int32, 1082:Int32, 'e':Varchar] - │ ├─[53:Int32, 1043:Int32, 1083:Int32, 'e':Varchar] - │ ├─[54:Int32, 1043:Int32, 1114:Int32, 'e':Varchar] - │ ├─[55:Int32, 1043:Int32, 1184:Int32, 'e':Varchar] - │ ├─[56:Int32, 1043:Int32, 1186:Int32, 'e':Varchar] - │ ├─[57:Int32, 1043:Int32, 17:Int32, 'e':Varchar] - │ ├─[58:Int32, 1043:Int32, 3802:Int32, 'e':Varchar] - │ ├─[59:Int32, 1043:Int32, 1301:Int32, 'e':Varchar] - │ ├─[60:Int32, 1083:Int32, 1043:Int32, 'a':Varchar] - │ ├─[61:Int32, 1083:Int32, 1186:Int32, 'i':Varchar] - │ ├─[62:Int32, 1114:Int32, 1082:Int32, 'a':Varchar] - │ ├─[63:Int32, 1114:Int32, 1043:Int32, 'a':Varchar] - │ ├─[64:Int32, 1114:Int32, 1083:Int32, 'a':Varchar] - │ ├─[65:Int32, 1114:Int32, 1184:Int32, 'i':Varchar] - │ ├─[66:Int32, 1184:Int32, 1082:Int32, 'a':Varchar] - │ ├─[67:Int32, 1184:Int32, 1043:Int32, 'a':Varchar] - │ ├─[68:Int32, 1184:Int32, 1083:Int32, 'a':Varchar] - │ ├─[69:Int32, 1184:Int32, 1114:Int32, 'a':Varchar] - │ ├─[70:Int32, 1186:Int32, 1043:Int32, 'a':Varchar] - │ ├─[71:Int32, 1186:Int32, 1083:Int32, 'a':Varchar] - │ ├─[72:Int32, 17:Int32, 1043:Int32, 'a':Varchar] - │ ├─[73:Int32, 3802:Int32, 16:Int32, 'e':Varchar] - │ ├─[74:Int32, 3802:Int32, 21:Int32, 'e':Varchar] - │ ├─[75:Int32, 3802:Int32, 23:Int32, 'e':Varchar] - │ ├─[76:Int32, 3802:Int32, 20:Int32, 'e':Varchar] - │ ├─[77:Int32, 3802:Int32, 700:Int32, 'e':Varchar] - │ ├─[78:Int32, 3802:Int32, 701:Int32, 'e':Varchar] - │ ├─[79:Int32, 3802:Int32, 1700:Int32, 'e':Varchar] - │ ├─[80:Int32, 3802:Int32, 1043:Int32, 'a':Varchar] - │ ├─[81:Int32, 1301:Int32, 701:Int32, 'e':Varchar] - │ └─[82:Int32, 1301:Int32, 1043:Int32, 'a':Varchar] - └─schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32, *VALUES*_0.column_2:Int32, *VALUES*_0.column_3:Varchar] } - batch_plan: |- - BatchValues - └─rows: - ┌─[0:Int32, 16:Int32, 23:Int32, 'e':Varchar] - ├─[1:Int32, 16:Int32, 1043:Int32, 'a':Varchar] - ├─[2:Int32, 21:Int32, 23:Int32, 'i':Varchar] - ├─[3:Int32, 21:Int32, 20:Int32, 'i':Varchar] - ├─[4:Int32, 21:Int32, 700:Int32, 'i':Varchar] - ├─[5:Int32, 21:Int32, 701:Int32, 'i':Varchar] - ├─[6:Int32, 21:Int32, 1700:Int32, 'i':Varchar] - ├─[7:Int32, 21:Int32, 1043:Int32, 'a':Varchar] - ├─[8:Int32, 21:Int32, 1301:Int32, 'i':Varchar] - ├─[9:Int32, 23:Int32, 16:Int32, 'e':Varchar] - ├─[10:Int32, 23:Int32, 21:Int32, 'a':Varchar] - ├─[11:Int32, 23:Int32, 20:Int32, 'i':Varchar] - ├─[12:Int32, 23:Int32, 700:Int32, 'i':Varchar] - ├─[13:Int32, 23:Int32, 701:Int32, 'i':Varchar] - ├─[14:Int32, 23:Int32, 1700:Int32, 'i':Varchar] - ├─[15:Int32, 23:Int32, 1043:Int32, 'a':Varchar] - ├─[16:Int32, 23:Int32, 1301:Int32, 'i':Varchar] - ├─[17:Int32, 20:Int32, 21:Int32, 'a':Varchar] - ├─[18:Int32, 20:Int32, 23:Int32, 'a':Varchar] - ├─[19:Int32, 20:Int32, 700:Int32, 'i':Varchar] - ├─[20:Int32, 20:Int32, 701:Int32, 'i':Varchar] - ├─[21:Int32, 20:Int32, 1700:Int32, 'i':Varchar] - ├─[22:Int32, 20:Int32, 1043:Int32, 'a':Varchar] - ├─[23:Int32, 20:Int32, 1301:Int32, 'i':Varchar] - ├─[24:Int32, 700:Int32, 21:Int32, 'a':Varchar] - ├─[25:Int32, 700:Int32, 23:Int32, 'a':Varchar] - ├─[26:Int32, 700:Int32, 20:Int32, 'a':Varchar] - ├─[27:Int32, 700:Int32, 701:Int32, 'i':Varchar] - ├─[28:Int32, 700:Int32, 1700:Int32, 'a':Varchar] - ├─[29:Int32, 700:Int32, 1043:Int32, 'a':Varchar] - ├─[30:Int32, 701:Int32, 21:Int32, 'a':Varchar] - ├─[31:Int32, 701:Int32, 23:Int32, 'a':Varchar] - ├─[32:Int32, 701:Int32, 20:Int32, 'a':Varchar] - ├─[33:Int32, 701:Int32, 700:Int32, 'a':Varchar] - ├─[34:Int32, 701:Int32, 1700:Int32, 'a':Varchar] - ├─[35:Int32, 701:Int32, 1043:Int32, 'a':Varchar] - ├─[36:Int32, 1700:Int32, 21:Int32, 'a':Varchar] - ├─[37:Int32, 1700:Int32, 23:Int32, 'a':Varchar] - ├─[38:Int32, 1700:Int32, 20:Int32, 'a':Varchar] - ├─[39:Int32, 1700:Int32, 700:Int32, 'i':Varchar] - ├─[40:Int32, 1700:Int32, 701:Int32, 'i':Varchar] - ├─[41:Int32, 1700:Int32, 1043:Int32, 'a':Varchar] - ├─[42:Int32, 1082:Int32, 1043:Int32, 'a':Varchar] - ├─[43:Int32, 1082:Int32, 1114:Int32, 'i':Varchar] - ├─[44:Int32, 1082:Int32, 1184:Int32, 'i':Varchar] - ├─[45:Int32, 1043:Int32, 16:Int32, 'e':Varchar] - ├─[46:Int32, 1043:Int32, 21:Int32, 'e':Varchar] - ├─[47:Int32, 1043:Int32, 23:Int32, 'e':Varchar] - ├─[48:Int32, 1043:Int32, 20:Int32, 'e':Varchar] - ├─[49:Int32, 1043:Int32, 700:Int32, 'e':Varchar] - ├─[50:Int32, 1043:Int32, 701:Int32, 'e':Varchar] - ├─[51:Int32, 1043:Int32, 1700:Int32, 'e':Varchar] - ├─[52:Int32, 1043:Int32, 1082:Int32, 'e':Varchar] - ├─[53:Int32, 1043:Int32, 1083:Int32, 'e':Varchar] - ├─[54:Int32, 1043:Int32, 1114:Int32, 'e':Varchar] - ├─[55:Int32, 1043:Int32, 1184:Int32, 'e':Varchar] - ├─[56:Int32, 1043:Int32, 1186:Int32, 'e':Varchar] - ├─[57:Int32, 1043:Int32, 17:Int32, 'e':Varchar] - ├─[58:Int32, 1043:Int32, 3802:Int32, 'e':Varchar] - ├─[59:Int32, 1043:Int32, 1301:Int32, 'e':Varchar] - ├─[60:Int32, 1083:Int32, 1043:Int32, 'a':Varchar] - ├─[61:Int32, 1083:Int32, 1186:Int32, 'i':Varchar] - ├─[62:Int32, 1114:Int32, 1082:Int32, 'a':Varchar] - ├─[63:Int32, 1114:Int32, 1043:Int32, 'a':Varchar] - ├─[64:Int32, 1114:Int32, 1083:Int32, 'a':Varchar] - ├─[65:Int32, 1114:Int32, 1184:Int32, 'i':Varchar] - ├─[66:Int32, 1184:Int32, 1082:Int32, 'a':Varchar] - ├─[67:Int32, 1184:Int32, 1043:Int32, 'a':Varchar] - ├─[68:Int32, 1184:Int32, 1083:Int32, 'a':Varchar] - ├─[69:Int32, 1184:Int32, 1114:Int32, 'a':Varchar] - ├─[70:Int32, 1186:Int32, 1043:Int32, 'a':Varchar] - ├─[71:Int32, 1186:Int32, 1083:Int32, 'a':Varchar] - ├─[72:Int32, 17:Int32, 1043:Int32, 'a':Varchar] - ├─[73:Int32, 3802:Int32, 16:Int32, 'e':Varchar] - ├─[74:Int32, 3802:Int32, 21:Int32, 'e':Varchar] - ├─[75:Int32, 3802:Int32, 23:Int32, 'e':Varchar] - ├─[76:Int32, 3802:Int32, 20:Int32, 'e':Varchar] - ├─[77:Int32, 3802:Int32, 700:Int32, 'e':Varchar] - ├─[78:Int32, 3802:Int32, 701:Int32, 'e':Varchar] - ├─[79:Int32, 3802:Int32, 1700:Int32, 'e':Varchar] - ├─[80:Int32, 3802:Int32, 1043:Int32, 'a':Varchar] - ├─[81:Int32, 1301:Int32, 701:Int32, 'e':Varchar] - └─[82:Int32, 1301:Int32, 1043:Int32, 'a':Varchar] + LogicalProject { exprs: [pg_cast.oid, pg_cast.castsource, pg_cast.casttarget, pg_cast.castcontext] } + └─LogicalSysScan { table: pg_cast, columns: [pg_cast.oid, pg_cast.castsource, pg_cast.casttarget, pg_cast.castcontext] } + batch_plan: 'BatchScan { table: pg_cast, columns: [pg_cast.oid, pg_cast.castsource, pg_cast.casttarget, pg_cast.castcontext], distribution: Single }' - sql: | select pg_catalog.pg_get_userbyid(1) logical_plan: |- diff --git a/src/frontend/src/catalog/root_catalog.rs b/src/frontend/src/catalog/root_catalog.rs index 3b9a9722a1f8f..fd3fd3020b3c7 100644 --- a/src/frontend/src/catalog/root_catalog.rs +++ b/src/frontend/src/catalog/root_catalog.rs @@ -141,23 +141,19 @@ impl Catalog { .unwrap() .create_schema(proto); - if let Some(sys_tables) = get_sys_tables_in_schema(proto.name.as_str()) { - sys_tables.into_iter().for_each(|sys_table| { - self.get_database_mut(proto.database_id) - .unwrap() - .get_schema_mut(proto.id) - .unwrap() - .create_sys_table(sys_table); - }); + for sys_table in get_sys_tables_in_schema(proto.name.as_str()) { + self.get_database_mut(proto.database_id) + .unwrap() + .get_schema_mut(proto.id) + .unwrap() + .create_sys_table(sys_table); } - if let Some(sys_views) = get_sys_views_in_schema(proto.name.as_str()) { - sys_views.into_iter().for_each(|sys_view| { - self.get_database_mut(proto.database_id) - .unwrap() - .get_schema_mut(proto.id) - .unwrap() - .create_sys_view(sys_view); - }); + for sys_view in get_sys_views_in_schema(proto.name.as_str()) { + self.get_database_mut(proto.database_id) + .unwrap() + .get_schema_mut(proto.id) + .unwrap() + .create_sys_view(sys_view); } } diff --git a/src/frontend/src/catalog/system_catalog/information_schema/columns.rs b/src/frontend/src/catalog/system_catalog/information_schema/columns.rs index 74cc04fa74187..074b772ca0bb8 100644 --- a/src/frontend/src/catalog/system_catalog/information_schema/columns.rs +++ b/src/frontend/src/catalog/system_catalog/information_schema/columns.rs @@ -12,12 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::INFORMATION_SCHEMA_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The view `columns` contains information about all table columns (or view columns) in the /// database. System columns (ctid, etc.) are not included. Only those columns are shown that the @@ -25,46 +21,45 @@ use crate::catalog::system_catalog::BuiltinView; /// Ref: [`https://www.postgresql.org/docs/current/infoschema-columns.html`] /// /// In RisingWave, `columns` also contains all materialized views' columns. -pub static INFORMATION_SCHEMA_COLUMNS: LazyLock = LazyLock::new(|| BuiltinView { - name: "columns", - schema: INFORMATION_SCHEMA_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "table_catalog"), - (DataType::Varchar, "table_schema"), - (DataType::Varchar, "table_name"), - (DataType::Varchar, "column_name"), - (DataType::Varchar, "column_default"), - (DataType::Int32, "character_maximum_length"), - (DataType::Int32, "numeric_precision"), - (DataType::Int32, "numeric_scale"), - (DataType::Int32, "ordinal_position"), - (DataType::Varchar, "is_nullable"), - (DataType::Varchar, "collation_name"), - (DataType::Varchar, "udt_schema"), - (DataType::Varchar, "data_type"), - (DataType::Varchar, "udt_name"), - ], - sql: "SELECT CURRENT_DATABASE() AS table_catalog, \ - s.name AS table_schema, \ - r.name AS table_name, \ - c.name AS column_name, \ - NULL AS column_default, \ - NULL::integer AS character_maximum_length, \ - NULL::integer AS numeric_precision, \ - NULL::integer AS numeric_scale, \ - c.position AS ordinal_position, \ - 'YES' AS is_nullable, \ - NULL AS collation_name, \ - 'pg_catalog' AS udt_schema, \ - CASE \ - WHEN c.data_type = 'varchar' THEN 'character varying' \ - ELSE c.data_type \ - END AS data_type, \ - c.udt_type AS udt_name \ - FROM rw_catalog.rw_columns c \ - LEFT JOIN rw_catalog.rw_relations r ON c.relation_id = r.id \ - JOIN rw_catalog.rw_schemas s ON s.id = r.schema_id \ - WHERE c.is_hidden = false\ - " - .to_string(), -}); +#[system_catalog( + view, + "information_schema.columns", + "SELECT CURRENT_DATABASE() AS table_catalog, + s.name AS table_schema, + r.name AS table_name, + c.name AS column_name, + NULL AS column_default, + NULL::integer AS character_maximum_length, + NULL::integer AS numeric_precision, + NULL::integer AS numeric_scale, + c.position AS ordinal_position, + 'YES' AS is_nullable, + NULL AS collation_name, + 'pg_catalog' AS udt_schema, + CASE + WHEN c.data_type = 'varchar' THEN 'character varying' + ELSE c.data_type + END AS data_type, + c.udt_type AS udt_name + FROM rw_catalog.rw_columns c + LEFT JOIN rw_catalog.rw_relations r ON c.relation_id = r.id + JOIN rw_catalog.rw_schemas s ON s.id = r.schema_id + WHERE c.is_hidden = false" +)] +#[derive(Fields)] +struct Column { + table_catalog: String, + table_schema: String, + table_name: String, + column_name: String, + column_default: String, + character_maximum_length: i32, + numeric_precision: i32, + numeric_scale: i32, + ordinal_position: i32, + is_nullable: String, + collation_name: String, + udt_schema: String, + data_type: String, + udt_name: String, +} diff --git a/src/frontend/src/catalog/system_catalog/information_schema/mod.rs b/src/frontend/src/catalog/system_catalog/information_schema/mod.rs index a87dd0a00d4c1..c618bee476c4d 100644 --- a/src/frontend/src/catalog/system_catalog/information_schema/mod.rs +++ b/src/frontend/src/catalog/system_catalog/information_schema/mod.rs @@ -12,10 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod columns; -pub mod tables; -pub mod views; - -pub use columns::*; -pub use tables::*; -pub use views::*; +mod columns; +mod tables; +mod views; diff --git a/src/frontend/src/catalog/system_catalog/information_schema/tables.rs b/src/frontend/src/catalog/system_catalog/information_schema/tables.rs index cb5e159dc3ad1..356cb0741c807 100644 --- a/src/frontend/src/catalog/system_catalog/information_schema/tables.rs +++ b/src/frontend/src/catalog/system_catalog/information_schema/tables.rs @@ -12,13 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::string::ToString; -use std::sync::LazyLock; - -use risingwave_common::catalog::INFORMATION_SCHEMA_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The view tables contains all tables and views defined in the current database. Only those tables /// and views are shown that the current user has access to (by way of being the owner or having @@ -26,33 +21,33 @@ use crate::catalog::system_catalog::BuiltinView; /// Ref: [`https://www.postgresql.org/docs/current/infoschema-tables.html`] /// /// In RisingWave, `tables` contains all relations. -pub static INFORMATION_SCHEMA_TABLES: LazyLock = LazyLock::new(|| BuiltinView { - name: "tables", - schema: INFORMATION_SCHEMA_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "table_catalog"), - (DataType::Varchar, "table_schema"), - (DataType::Varchar, "table_name"), - (DataType::Varchar, "table_type"), - (DataType::Varchar, "is_insertable_into"), - ], - sql: "SELECT CURRENT_DATABASE() AS table_catalog, \ - s.name AS table_schema, \ - r.name AS table_name, \ - CASE r.relation_type \ - WHEN 'materialized view' THEN 'MATERIALIZED VIEW' \ - WHEN 'table' THEN 'BASE TABLE' \ - WHEN 'system table' THEN 'SYSTEM TABLE' \ - WHEN 'view' THEN 'VIEW' \ - ELSE UPPER(r.relation_type) \ - END AS table_type, \ - CASE \ - WHEN r.relation_type = 'table' \ - THEN 'YES' \ - ELSE 'NO' \ - END AS is_insertable_into \ - FROM rw_catalog.rw_relations r \ - JOIN rw_catalog.rw_schemas s ON r.schema_id = s.id \ +#[system_catalog( + view, + "information_schema.tables", + "SELECT CURRENT_DATABASE() AS table_catalog, + s.name AS table_schema, + r.name AS table_name, + CASE r.relation_type + WHEN 'materialized view' THEN 'MATERIALIZED VIEW' + WHEN 'table' THEN 'BASE TABLE' + WHEN 'system table' THEN 'SYSTEM TABLE' + WHEN 'view' THEN 'VIEW' + ELSE UPPER(r.relation_type) + END AS table_type, + CASE + WHEN r.relation_type = 'table' + THEN 'YES' + ELSE 'NO' + END AS is_insertable_into + FROM rw_catalog.rw_relations r + JOIN rw_catalog.rw_schemas s ON r.schema_id = s.id ORDER BY table_schema, table_name" - .to_string(), -}); +)] +#[derive(Fields)] +struct Table { + table_catalog: String, + table_schema: String, + table_name: String, + table_type: String, + is_insertable_into: String, +} diff --git a/src/frontend/src/catalog/system_catalog/information_schema/views.rs b/src/frontend/src/catalog/system_catalog/information_schema/views.rs index aff86ca614628..38ea74e05b430 100644 --- a/src/frontend/src/catalog/system_catalog/information_schema/views.rs +++ b/src/frontend/src/catalog/system_catalog/information_schema/views.rs @@ -12,12 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::INFORMATION_SCHEMA_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The view `views` contains all views defined in the current database. Only those views /// are shown that the current user has access to (by way of being the owner or having @@ -25,21 +21,21 @@ use crate::catalog::system_catalog::BuiltinView; /// Ref: [`https://www.postgresql.org/docs/current/infoschema-views.html`] /// /// In RisingWave, `views` contains information about defined views. -pub static INFORMATION_SCHEMA_VIEWS: LazyLock = LazyLock::new(|| BuiltinView { - name: "views", - schema: INFORMATION_SCHEMA_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "table_catalog"), - (DataType::Varchar, "table_schema"), - (DataType::Varchar, "table_name"), - (DataType::Varchar, "view_definition"), - ], - sql: "SELECT CURRENT_DATABASE() AS table_catalog, \ - s.name AS table_schema, \ - v.name AS table_name, \ - v.definition AS view_definition \ - FROM rw_catalog.rw_views v \ - JOIN rw_catalog.rw_schemas s ON v.schema_id = s.id \ +#[system_catalog( + view, + "information_schema.views", + "SELECT CURRENT_DATABASE() AS table_catalog, + s.name AS table_schema, + v.name AS table_name, + v.definition AS view_definition + FROM rw_catalog.rw_views v + JOIN rw_catalog.rw_schemas s ON v.schema_id = s.id ORDER BY table_schema, table_name" - .to_string(), -}); +)] +#[derive(Fields)] +struct View { + table_catalog: String, + table_schema: String, + table_name: String, + view_definition: String, +} diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index 52462d9111ad9..a0be5af42fd36 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -20,23 +20,21 @@ use std::collections::HashMap; use std::sync::{Arc, LazyLock}; use async_trait::async_trait; +use futures::future::BoxFuture; use itertools::Itertools; use risingwave_common::acl::AclMode; +use risingwave_common::array::DataChunk; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, Field, SysCatalogReader, TableDesc, TableId, DEFAULT_SUPER_USER_ID, NON_RESERVED_SYS_CATALOG_ID, }; use risingwave_common::error::BoxedError; -use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; use risingwave_pb::meta::list_table_fragment_states_response::TableFragmentState; use risingwave_pb::meta::table_parallelism::{PbFixedParallelism, PbParallelism}; use risingwave_pb::user::grant_privilege::Object; use crate::catalog::catalog_service::CatalogReader; -use crate::catalog::system_catalog::information_schema::*; -use crate::catalog::system_catalog::pg_catalog::*; -use crate::catalog::system_catalog::rw_catalog::*; use crate::catalog::view_catalog::ViewCatalog; use crate::meta_client::FrontendMetaClient; use crate::scheduler::worker_node_manager::WorkerNodeManagerRef; @@ -130,27 +128,28 @@ impl SysCatalogReaderImpl { pub struct BuiltinTable { name: &'static str, schema: &'static str, - columns: &'static [SystemCatalogColumnsDef<'static>], + columns: Vec>, pk: &'static [usize], + function: for<'a> fn(&'a SysCatalogReaderImpl) -> BoxFuture<'a, Result>, } pub struct BuiltinView { name: &'static str, schema: &'static str, - columns: &'static [SystemCatalogColumnsDef<'static>], + columns: Vec>, sql: String, } pub enum BuiltinCatalog { - Table(&'static BuiltinTable), - View(&'static BuiltinView), + Table(BuiltinTable), + View(BuiltinView), } impl BuiltinCatalog { - fn name(&self) -> &'static str { + fn full_name(&self) -> String { match self { - BuiltinCatalog::Table(t) => t.name, - BuiltinCatalog::View(v) => v.name, + BuiltinCatalog::Table(t) => format!("{}.{}", t.schema, t.name), + BuiltinCatalog::View(t) => format!("{}.{}", t.schema, t.name), } } } @@ -164,8 +163,8 @@ impl From<&BuiltinTable> for SystemTableCatalog { .columns .iter() .enumerate() - .map(|(idx, c)| ColumnCatalog { - column_desc: ColumnDesc::new_atomic(c.0.clone(), c.1, idx as i32), + .map(|(idx, (name, ty))| ColumnCatalog { + column_desc: ColumnDesc::new_atomic(ty.clone(), name, idx as i32), is_hidden: false, }) .collect(), @@ -184,9 +183,9 @@ impl From<&BuiltinView> for ViewCatalog { columns: val .columns .iter() - .map(|c| Field::with_name(c.0.clone(), c.1.to_string())) + .map(|(name, ty)| Field::with_name(ty.clone(), name.to_string())) .collect(), - sql: val.sql.to_string(), + sql: val.sql.clone(), owner: DEFAULT_SUPER_USER_ID, properties: Default::default(), } @@ -194,19 +193,19 @@ impl From<&BuiltinView> for ViewCatalog { } // TODO: support struct column and type name when necessary. -pub(super) type SystemCatalogColumnsDef<'a> = (DataType, &'a str); +pub(super) type SystemCatalogColumnsDef<'a> = (&'a str, DataType); /// `infer_dummy_view_sql` returns a dummy SQL statement for a view with the given columns that /// returns no rows. For example, with columns `a` and `b`, it returns `SELECT NULL::integer AS a, /// NULL::varchar AS b WHERE 1 != 1`. // FIXME(noel): Tracked by #[inline(always)] -fn infer_dummy_view_sql(columns: &[SystemCatalogColumnsDef<'_>]) -> String { +pub fn infer_dummy_view_sql(columns: &[SystemCatalogColumnsDef<'_>]) -> String { format!( "SELECT {} WHERE 1 != 1", columns .iter() - .map(|(ty, name)| format!("NULL::{} AS {}", ty, name)) + .map(|(name, ty)| format!("NULL::{} AS {}", ty, name)) .join(", ") ) } @@ -281,183 +280,81 @@ fn get_acl_items( } pub struct SystemCatalog { - table_by_schema_name: HashMap<&'static str, Vec>>, - table_name_by_id: HashMap, - view_by_schema_name: HashMap<&'static str, Vec>>, + // table id = index + 1 + catalogs: Vec, } -pub fn get_sys_tables_in_schema(schema_name: &str) -> Option>> { - SYS_CATALOGS.table_by_schema_name.get(schema_name).cloned() +pub fn get_sys_tables_in_schema(schema_name: &str) -> Vec> { + SYS_CATALOGS + .catalogs + .iter() + .enumerate() + .filter_map(|(idx, c)| match c { + BuiltinCatalog::Table(t) if t.schema == schema_name => Some(Arc::new( + SystemTableCatalog::from(t).with_id((idx as u32 + 1).into()), + )), + _ => None, + }) + .collect() } -pub fn get_sys_views_in_schema(schema_name: &str) -> Option>> { - SYS_CATALOGS.view_by_schema_name.get(schema_name).cloned() +pub fn get_sys_views_in_schema(schema_name: &str) -> Vec> { + SYS_CATALOGS + .catalogs + .iter() + .enumerate() + .filter_map(|(idx, c)| match c { + BuiltinCatalog::View(v) if v.schema == schema_name => { + Some(Arc::new(ViewCatalog::from(v).with_id(idx as u32 + 1))) + } + _ => None, + }) + .collect() } /// The global registry of all builtin catalogs. pub static SYS_CATALOGS: LazyLock = LazyLock::new(|| { - let mut table_by_schema_name = HashMap::new(); - let mut table_name_by_id = HashMap::new(); - let mut view_by_schema_name = HashMap::new(); tracing::info!("found {} catalogs", SYS_CATALOGS_SLICE.len()); - for catalog in SYS_CATALOGS_SLICE { - let (id, catalog) = catalog(); - assert!(id < NON_RESERVED_SYS_CATALOG_ID as u32); - match catalog { - BuiltinCatalog::Table(table) => { - let sys_table: SystemTableCatalog = table.into(); - table_by_schema_name - .entry(table.schema) - .or_insert(vec![]) - .push(Arc::new(sys_table.with_id(id.into()))); - table_name_by_id.insert(id.into(), table.name); - } - BuiltinCatalog::View(view) => { - let sys_view: ViewCatalog = view.into(); - view_by_schema_name - .entry(view.schema) - .or_insert(vec![]) - .push(Arc::new(sys_view.with_id(id))); - } - } - } - SystemCatalog { - table_by_schema_name, - table_name_by_id, - view_by_schema_name, - } + assert!(SYS_CATALOGS_SLICE.len() + 1 < NON_RESERVED_SYS_CATALOG_ID as usize); + let catalogs = SYS_CATALOGS_SLICE + .iter() + .map(|f| f()) + .sorted_by_key(|c| c.full_name()) + .collect(); + SystemCatalog { catalogs } }); #[linkme::distributed_slice] -pub static SYS_CATALOGS_SLICE: [fn() -> (u32, BuiltinCatalog)]; - -macro_rules! prepare_sys_catalog { - ($( { $builtin_catalog:expr $(, $func:ident $($await:ident)?)? } ),* $(,)?) => { - $( - const _: () = { - #[linkme::distributed_slice(crate::catalog::system_catalog::SYS_CATALOGS_SLICE)] - fn catalog() -> (u32, BuiltinCatalog) { - (${index()} as u32 + 1, $builtin_catalog) - } - }; - )* - - #[async_trait] - impl SysCatalogReader for SysCatalogReaderImpl { - async fn read_table(&self, table_id: &TableId) -> Result, BoxedError> { - let table_name = SYS_CATALOGS.table_name_by_id.get(table_id).unwrap(); - $( - if $builtin_catalog.name() == *table_name { - $( - let rows = self.$func(); - $(let rows = rows.$await;)? - return Ok(rows?); - )? - } - )* - unreachable!() +pub static SYS_CATALOGS_SLICE: [fn() -> BuiltinCatalog]; + +#[async_trait] +impl SysCatalogReader for SysCatalogReaderImpl { + async fn read_table(&self, table_id: &TableId) -> Result { + let table_name = SYS_CATALOGS + .catalogs + .get(table_id.table_id as usize - 1) + .unwrap(); + match table_name { + BuiltinCatalog::Table(t) => (t.function)(self).await, + BuiltinCatalog::View(_) => { + panic!("read_table should not be called on a view") } } } } -// `prepare_sys_catalog!` macro is used to generate all builtin system catalogs. -prepare_sys_catalog! { - { BuiltinCatalog::View(&PG_TYPE) }, - { BuiltinCatalog::View(&PG_NAMESPACE) }, - { BuiltinCatalog::View(&PG_CAST) }, - { BuiltinCatalog::View(&PG_MATVIEWS) }, - { BuiltinCatalog::View(&PG_USER) }, - { BuiltinCatalog::View(&PG_CLASS) }, - { BuiltinCatalog::View(&PG_INDEX) }, - { BuiltinCatalog::View(&PG_OPCLASS) }, - { BuiltinCatalog::View(&PG_COLLATION) }, - { BuiltinCatalog::View(&PG_AM) }, - { BuiltinCatalog::View(&PG_OPERATOR) }, - { BuiltinCatalog::View(&PG_VIEWS) }, - { BuiltinCatalog::View(&PG_ATTRIBUTE) }, - { BuiltinCatalog::View(&PG_DATABASE) }, - { BuiltinCatalog::View(&PG_DESCRIPTION) }, - { BuiltinCatalog::View(&PG_SETTINGS) }, - { BuiltinCatalog::View(&PG_KEYWORDS) }, - { BuiltinCatalog::View(&PG_ATTRDEF) }, - { BuiltinCatalog::View(&PG_ROLES) }, - { BuiltinCatalog::View(&PG_AUTH_MEMBERS) }, - { BuiltinCatalog::View(&PG_SHDESCRIPTION) }, - { BuiltinCatalog::View(&PG_TABLESPACE) }, - { BuiltinCatalog::View(&PG_STAT_ACTIVITY) }, - { BuiltinCatalog::View(&PG_ENUM) }, - { BuiltinCatalog::View(&PG_CONVERSION) }, - { BuiltinCatalog::View(&PG_INDEXES) }, - { BuiltinCatalog::View(&PG_INHERITS) }, - { BuiltinCatalog::View(&PG_CONSTRAINT) }, - { BuiltinCatalog::View(&PG_TABLES) }, - { BuiltinCatalog::View(&PG_PROC) }, - { BuiltinCatalog::View(&PG_SHADOW) }, - { BuiltinCatalog::View(&PG_LOCKS) }, - { BuiltinCatalog::View(&PG_EXTENSION) }, - { BuiltinCatalog::View(&PG_DEPEND) }, - { BuiltinCatalog::View(&INFORMATION_SCHEMA_COLUMNS) }, - { BuiltinCatalog::View(&INFORMATION_SCHEMA_TABLES) }, - { BuiltinCatalog::View(&INFORMATION_SCHEMA_VIEWS) }, - { BuiltinCatalog::Table(&RW_DATABASES), read_rw_database_info }, - { BuiltinCatalog::Table(&RW_SCHEMAS), read_rw_schema_info }, - { BuiltinCatalog::Table(&RW_USERS), read_rw_user_info }, - { BuiltinCatalog::Table(&RW_USER_SECRETS), read_rw_user_secrets_info }, - { BuiltinCatalog::Table(&RW_TABLES), read_rw_table_info }, - { BuiltinCatalog::Table(&RW_INTERNAL_TABLES), read_rw_internal_table_info }, - { BuiltinCatalog::Table(&RW_MATERIALIZED_VIEWS), read_rw_mview_info }, - { BuiltinCatalog::Table(&RW_INDEXES), read_rw_indexes_info }, - { BuiltinCatalog::Table(&RW_SOURCES), read_rw_sources_info }, - { BuiltinCatalog::Table(&RW_SINKS), read_rw_sinks_info }, - { BuiltinCatalog::Table(&RW_CONNECTIONS), read_rw_connections_info }, - { BuiltinCatalog::Table(&RW_FUNCTIONS), read_rw_functions_info }, - { BuiltinCatalog::Table(&RW_VIEWS), read_rw_views_info }, - { BuiltinCatalog::Table(&RW_WORKER_NODES), read_rw_worker_nodes_info await }, - { BuiltinCatalog::Table(&RW_PARALLEL_UNITS), read_rw_parallel_units_info }, - { BuiltinCatalog::Table(&RW_TABLE_FRAGMENTS), read_rw_table_fragments_info await }, - { BuiltinCatalog::Table(&RW_FRAGMENTS), read_rw_fragment_distributions_info await }, - { BuiltinCatalog::Table(&RW_ACTORS), read_rw_actor_states_info await }, - { BuiltinCatalog::Table(&RW_META_SNAPSHOT), read_meta_snapshot await }, - { BuiltinCatalog::Table(&RW_DDL_PROGRESS), read_ddl_progress await }, - { BuiltinCatalog::Table(&RW_TABLE_STATS), read_table_stats }, - { BuiltinCatalog::Table(&RW_RELATION_INFO), read_relation_info await }, - { BuiltinCatalog::Table(&RW_SYSTEM_TABLES), read_system_table_info }, - { BuiltinCatalog::View(&RW_RELATIONS) }, - { BuiltinCatalog::View(&RW_STREAMING_PARALLELISM) }, - { BuiltinCatalog::View(&RW_FRAGMENT_PARALLELISM) }, - { BuiltinCatalog::Table(&RW_COLUMNS), read_rw_columns_info }, - { BuiltinCatalog::Table(&RW_TYPES), read_rw_types }, - { BuiltinCatalog::Table(&RW_HUMMOCK_PINNED_VERSIONS), read_hummock_pinned_versions await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_PINNED_SNAPSHOTS), read_hummock_pinned_snapshots await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_CURRENT_VERSION), read_hummock_current_version await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_CHECKPOINT_VERSION), read_hummock_checkpoint_version await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_SSTABLES), read_hummock_sstables await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_VERSION_DELTAS), read_hummock_version_deltas await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_BRANCHED_OBJECTS), read_hummock_branched_objects await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_COMPACTION_GROUP_CONFIGS), read_hummock_compaction_group_configs await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_META_CONFIGS), read_hummock_meta_configs await}, - { BuiltinCatalog::Table(&RW_HUMMOCK_COMPACT_TASK_ASSIGNMENT), read_hummock_compact_task_assignments await }, - { BuiltinCatalog::Table(&RW_HUMMOCK_COMPACT_TASK_PROGRESS), read_hummock_compact_task_progress await }, - { BuiltinCatalog::Table(&RW_EVENT_LOGS), read_event_logs await}, - { BuiltinCatalog::Table(&RW_DESCRIPTION), read_rw_description }, -} - #[cfg(test)] mod tests { - use itertools::Itertools; - use crate::catalog::system_catalog::SYS_CATALOGS; use crate::test_utils::LocalFrontend; #[tokio::test] async fn test_builtin_view_definition() { let frontend = LocalFrontend::new(Default::default()).await; - let sqls = SYS_CATALOGS - .view_by_schema_name - .values() - .flat_map(|v| v.iter().map(|v| v.sql.clone())) - .collect_vec(); + let sqls = SYS_CATALOGS.catalogs.iter().filter_map(|c| match c { + super::BuiltinCatalog::View(v) => Some(v.sql.clone()), + _ => None, + }); for sql in sqls { frontend.query_formatted_result(sql).await; } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs index cf83731530ac4..c1a935803f9f4 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs @@ -12,72 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod pg_am; -pub mod pg_attrdef; -pub mod pg_attribute; -pub mod pg_auth_members; -pub mod pg_cast; -pub mod pg_class; -pub mod pg_collation; -pub mod pg_constraint; -pub mod pg_conversion; -pub mod pg_database; +mod pg_am; +mod pg_attrdef; +mod pg_attribute; +mod pg_auth_members; +mod pg_cast; +mod pg_class; +mod pg_collation; +mod pg_constraint; +mod pg_conversion; +mod pg_database; mod pg_depend; -pub mod pg_description; -pub mod pg_enum; -pub mod pg_extension; -pub mod pg_index; -pub mod pg_indexes; -pub mod pg_inherits; -pub mod pg_keywords; -pub mod pg_locks; -pub mod pg_matviews; -pub mod pg_namespace; -pub mod pg_opclass; -pub mod pg_operator; -pub mod pg_proc; -pub mod pg_roles; -pub mod pg_settings; +mod pg_description; +mod pg_enum; +mod pg_extension; +mod pg_index; +mod pg_indexes; +mod pg_inherits; +mod pg_keywords; +mod pg_locks; +mod pg_matviews; +mod pg_namespace; +mod pg_opclass; +mod pg_operator; +mod pg_proc; +mod pg_roles; +mod pg_settings; mod pg_shadow; -pub mod pg_shdescription; -pub mod pg_stat_activity; -pub mod pg_tables; -pub mod pg_tablespace; -pub mod pg_type; -pub mod pg_user; -pub mod pg_views; +mod pg_shdescription; +mod pg_stat_activity; +mod pg_tables; +mod pg_tablespace; +mod pg_type; +mod pg_user; +mod pg_views; -pub use pg_am::*; -pub use pg_attrdef::*; -pub use pg_attribute::*; -pub use pg_auth_members::*; -pub use pg_cast::*; -pub use pg_class::*; -pub use pg_collation::*; -pub use pg_constraint::*; -pub use pg_conversion::*; -pub use pg_database::*; -pub use pg_depend::*; -pub use pg_description::*; -pub use pg_enum::*; -pub use pg_extension::*; -pub use pg_index::*; -pub use pg_indexes::*; -pub use pg_inherits::*; pub use pg_keywords::*; -pub use pg_locks::*; -pub use pg_matviews::*; -pub use pg_namespace::*; -pub use pg_opclass::*; -pub use pg_operator::*; -pub use pg_proc::*; -pub use pg_roles::*; -pub use pg_settings::*; -pub use pg_shadow::*; -pub use pg_shdescription::*; -pub use pg_stat_activity::*; -pub use pg_tables::*; -pub use pg_tablespace::*; -pub use pg_type::*; -pub use pg_user::*; -pub use pg_views::*; diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs index 9451860ea35fc..946eb0aa3d4a4 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_am.rs @@ -12,25 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_AM_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "amname"), - (DataType::Int32, "amhandler"), - (DataType::Varchar, "amtype"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// Stores information about relation access methods. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-am.html`] -pub static PG_AM: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_am", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_AM_COLUMNS, - sql: infer_dummy_view_sql(PG_AM_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_am")] +#[derive(Fields)] +struct PgAmColumn { + oid: i32, + amname: String, + amhandler: i32, + amtype: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs index a566324593cee..5f6e8122ed050 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attrdef.rs @@ -12,28 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_ATTRDEF_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Int32, "adrelid"), - (DataType::Int16, "adnum"), - // The column default value, use pg_get_expr(adbin, adrelid) to convert it to an SQL - // expression. We don't have `pg_node_tree` type yet, so we use `text` instead. - (DataType::Varchar, "adbin"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_attrdef` stores column default values. The main information about columns is /// stored in `pg_attribute`. Only columns for which a default value has been explicitly set will /// have an entry here. Ref: [`https://www.postgresql.org/docs/current/catalog-pg-attrdef.html`] -pub static PG_ATTRDEF: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_attrdef", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_ATTRDEF_COLUMNS, - sql: infer_dummy_view_sql(PG_ATTRDEF_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_attrdef")] +#[derive(Fields)] +struct PgAttrdef { + oid: i32, + adrelid: i32, + adnum: i16, + // The column default value, use pg_get_expr(adbin, adrelid) to convert it to an SQL + // expression. We don't have `pg_node_tree` type yet, so we use `text` instead. + adbin: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs index c8920d5458d33..e2534a6dbc592 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs @@ -12,12 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_attribute` stores information about table columns. There will be exactly one /// `pg_attribute` row for every column in every table in the database. (There will also be @@ -26,35 +22,34 @@ use crate::catalog::system_catalog::BuiltinView; /// /// In RisingWave, we simply make it contain the columns of the view and all the columns of the /// tables that are not internal tables. -pub static PG_ATTRIBUTE: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_attribute", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "attrelid"), - (DataType::Varchar, "attname"), - (DataType::Int32, "atttypid"), - (DataType::Int16, "attlen"), - (DataType::Int16, "attnum"), - (DataType::Boolean, "attnotnull"), - (DataType::Boolean, "atthasdef"), - (DataType::Boolean, "attisdropped"), - (DataType::Varchar, "attidentity"), - (DataType::Varchar, "attgenerated"), - (DataType::Int32, "atttypmod"), - ], - sql: "SELECT c.relation_id AS attrelid, \ - c.name AS attname, \ - c.type_oid AS atttypid, \ - c.type_len AS attlen, \ - c.position::smallint AS attnum, \ - false AS attnotnull, \ - false AS atthasdef, \ - false AS attisdropped, \ - ''::varchar AS attidentity, \ - ''::varchar AS attgenerated, \ - -1 AS atttypmod \ - FROM rw_catalog.rw_columns c \ - WHERE c.is_hidden = false\ - " - .to_string(), -}); +#[system_catalog( + view, + "pg_catalog.pg_attribute", + "SELECT c.relation_id AS attrelid, + c.name AS attname, + c.type_oid AS atttypid, + c.type_len AS attlen, + c.position::smallint AS attnum, + false AS attnotnull, + false AS atthasdef, + false AS attisdropped, + ''::varchar AS attidentity, + ''::varchar AS attgenerated, + -1 AS atttypmod + FROM rw_catalog.rw_columns c + WHERE c.is_hidden = false" +)] +#[derive(Fields)] +struct PgAttribute { + attrelid: i32, + attname: String, + atttypid: i32, + attlen: i16, + attnum: i16, + attnotnull: bool, + atthasdef: bool, + attisdropped: bool, + attidentity: String, + attgenerated: String, + atttypmod: i32, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_auth_members.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_auth_members.rs index 6ba65700a67ba..c39e3a0662ed4 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_auth_members.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_auth_members.rs @@ -12,28 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_AUTH_MEMBERS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Int32, "roleid"), - (DataType::Int32, "member"), - (DataType::Int32, "grantor"), - (DataType::Boolean, "admin_option"), - (DataType::Boolean, "inherit_option"), - (DataType::Boolean, "set_option"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_auth_members` shows the membership relations between roles. Any non-circular set of relationships is allowed. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-auth-members.html`] -pub static PG_AUTH_MEMBERS: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_auth_members", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_AUTH_MEMBERS_COLUMNS, - sql: infer_dummy_view_sql(PG_AUTH_MEMBERS_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_auth_members")] +#[derive(Fields)] +struct PgAuthMember { + oid: i32, + roleid: i32, + member: i32, + grantor: i32, + admin_option: bool, + inherit_option: bool, + set_option: bool, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs index 7057ca75d9a3b..c13e87f162afe 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs @@ -12,48 +12,34 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; +use risingwave_common::types::{DataType, Fields}; +use risingwave_frontend_macro::system_catalog; -use itertools::Itertools; -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::expr::cast_map_array; -pub static PG_CAST_DATA: LazyLock> = LazyLock::new(|| { +/// The catalog `pg_cast` stores data type conversion paths. +/// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-cast.html`] +#[derive(Fields)] +struct PgCast { + oid: i32, + castsource: i32, + casttarget: i32, + castcontext: String, +} + +#[system_catalog(table, "pg_catalog.pg_cast")] +fn read_pg_cast(_: &SysCatalogReaderImpl) -> Vec { let mut cast_array = cast_map_array(); cast_array.sort(); cast_array .iter() .enumerate() - .map(|(idx, (src, target, ctx))| { - format!( - "({}, {}, {}, \'{}\')", - idx, - DataType::from(*src).to_oid(), - DataType::from(*target).to_oid(), - ctx - ) + .map(|(idx, (src, target, ctx))| PgCast { + oid: idx as i32, + castsource: DataType::from(*src).to_oid(), + casttarget: DataType::from(*target).to_oid(), + castcontext: ctx.to_string(), }) - .collect_vec() -}); - -/// The catalog `pg_cast` stores data type conversion paths. -/// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-cast.html`] -pub static PG_CAST: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_cast", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "oid"), - (DataType::Int32, "castsource"), - (DataType::Int32, "casttarget"), - (DataType::Varchar, "castcontext"), - ], - sql: format!( - "SELECT oid, castsource, casttarget, castcontext \ - FROM (VALUES {}) AS _(oid, castsource, casttarget, castcontext)\ - ", - PG_CAST_DATA.join(",") - ), -}); + .collect() +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs index 6ff7abac7187a..0d2dc8c8a41a7 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs @@ -12,52 +12,38 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{BuiltinView, SystemCatalogColumnsDef}; - -pub static PG_CLASS_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Int32, "oid"), - (DataType::Varchar, "relname"), - (DataType::Int32, "relnamespace"), - (DataType::Int32, "relowner"), - (DataType::Varchar, "relpersistence"), /* p = permanent table, u = unlogged table, t = - * temporary table */ - (DataType::Varchar, "relkind"), /* r = ordinary table, i = index, S = sequence, t = - * TOAST table, v = view, m = materialized view, c = - * composite type, f = foreign table, p = partitioned - * table, I = partitioned index */ - (DataType::Int32, "relam"), - (DataType::Int32, "reltablespace"), - (DataType::List(Box::new(DataType::Varchar)), "reloptions"), - ] -}); +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_class` catalogs tables and most everything else that has columns or is otherwise /// similar to a table. Ref: [`https://www.postgresql.org/docs/current/catalog-pg-class.html`] /// todo: should we add internal tables as well? -pub static PG_CLASS: LazyLock = LazyLock::new(|| { - BuiltinView { - name: "pg_class", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &PG_CLASS_COLUMNS, - sql: "SELECT id AS oid, name AS relname, schema_id AS relnamespace, owner AS relowner, 'p' as relpersistence, \ - CASE \ - WHEN relation_type = 'table' THEN 'r' \ - WHEN relation_type = 'system table' THEN 'r' \ - WHEN relation_type = 'index' THEN 'i' \ - WHEN relation_type = 'view' THEN 'v' \ - WHEN relation_type = 'materialized view' THEN 'm' \ - END relkind, \ - 0 AS relam, \ - 0 AS reltablespace, \ - ARRAY[]::varchar[] AS reloptions \ - FROM rw_catalog.rw_relations\ - " - .to_string(), +#[system_catalog(view, "pg_catalog.pg_class", + "SELECT id AS oid, name AS relname, schema_id AS relnamespace, owner AS relowner, 'p' as relpersistence, + CASE + WHEN relation_type = 'table' THEN 'r' + WHEN relation_type = 'system table' THEN 'r' + WHEN relation_type = 'index' THEN 'i' + WHEN relation_type = 'view' THEN 'v' + WHEN relation_type = 'materialized view' THEN 'm' + END relkind, + 0 AS relam, + 0 AS reltablespace, + ARRAY[]::varchar[] AS reloptions + FROM rw_catalog.rw_relations +")] +#[derive(Fields)] +struct PgClass { + oid: i32, + relname: String, + relnamespace: i32, + relowner: i32, + // p = permanent table, u = unlogged table, t = temporary table + relpersistence: String, + // r = ordinary table, i = index, S = sequence, t = TOAST table, v = view, m = materialized view, + // c = composite type, f = foreign table, p = partitioned table, I = partitioned index + relkind: String, + relam: i32, + reltablespace: i32, + reloptions: Vec, } -}); diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs index 4900d7d867fc9..cffe57b32350e 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_collation.rs @@ -12,32 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_COLLATION_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "collname"), - (DataType::Int32, "collnamespace"), - (DataType::Int32, "collowner"), - (DataType::Int32, "collprovider"), - (DataType::Boolean, "collisdeterministic"), - (DataType::Int32, "collencoding"), - (DataType::Varchar, "collcollate"), - (DataType::Varchar, "collctype"), - (DataType::Varchar, "colliculocale"), - (DataType::Varchar, "collversion"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// Mapping from sql name to system locale groups. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-collation.html`]. -pub static PG_COLLATION: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_collation", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_COLLATION_COLUMNS, - sql: infer_dummy_view_sql(PG_COLLATION_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_collation")] +#[derive(Fields)] +struct PgCollation { + oid: i32, + collname: String, + collnamespace: i32, + collowner: i32, + collprovider: i32, + collisdeterministic: bool, + collencoding: i32, + collcollate: String, + collctype: String, + colliculocale: String, + collversion: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs index 14dfee2dfdb8b..d69a4b881570e 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs @@ -12,50 +12,38 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub static PG_CONSTRAINT_COLUMNS: LazyLock>> = - LazyLock::new(|| { - vec![ - (DataType::Int32, "oid"), - (DataType::Varchar, "conname"), - (DataType::Int32, "connamespace"), - (DataType::Varchar, "contype"), - (DataType::Boolean, "condeferrable"), - (DataType::Boolean, "convalidated"), - (DataType::Int32, "conrelid"), - (DataType::Int32, "contypid"), - (DataType::Int32, "conindid"), - (DataType::Int32, "conparentid"), - (DataType::Int32, "confrelid"), - (DataType::Varchar, "confupdtype"), - (DataType::Varchar, "confdeltype"), - (DataType::Varchar, "confmatchtype"), - (DataType::Boolean, "conislocal"), - (DataType::Int32, "coninhcount"), - (DataType::Boolean, "connoinherit"), - (DataType::List(Box::new(DataType::Int16)), "conkey"), - (DataType::List(Box::new(DataType::Int16)), "confkey"), - (DataType::List(Box::new(DataType::Int32)), "conpfeqop"), - (DataType::List(Box::new(DataType::Int32)), "conppeqop"), - (DataType::List(Box::new(DataType::Int32)), "conffeqop"), - (DataType::List(Box::new(DataType::Int16)), "confdelsetcols"), - (DataType::List(Box::new(DataType::Int32)), "conexclop"), - (DataType::Varchar, "conbin"), - ] - }); +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_constraint` records information about table and index inheritance hierarchies. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-constraint.html`] /// This is introduced only for pg compatibility and is not used in our system. -pub static PG_CONSTRAINT: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_constraint", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &PG_CONSTRAINT_COLUMNS, - sql: infer_dummy_view_sql(&PG_CONSTRAINT_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_constraint")] +#[derive(Fields)] +struct PgConstraint { + oid: i32, + conname: String, + connamespace: i32, + contype: String, + condeferrable: bool, + convalidated: bool, + conrelid: i32, + contypid: i32, + conindid: i32, + conparentid: i32, + confrelid: i32, + confupdtype: String, + confdeltype: String, + confmatchtype: String, + conislocal: bool, + coninhcount: i32, + connoinherit: bool, + conkey: Vec, + confkey: Vec, + conpfeqop: Vec, + conppeqop: Vec, + conffeqop: Vec, + confdelsetcols: Vec, + conexclop: Vec, + conbin: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs index a0595780094dc..7e06c45ec34bb 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_conversion.rs @@ -12,29 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_CONVERSION_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "conname"), - (DataType::Int32, "connamespace"), - (DataType::Int32, "conowner"), - (DataType::Int16, "conforencoding"), - (DataType::Int16, "contoencoding"), - (DataType::Int32, "conproc"), - (DataType::Boolean, "condefault"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_conversion` describes encoding conversion functions. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-conversion.html`] -pub static PG_CONVERSION: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_conversion", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_CONVERSION_COLUMNS, - sql: infer_dummy_view_sql(PG_CONVERSION_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_conversion")] +#[derive(Fields)] +struct PgConversion { + oid: i32, + conname: String, + connamespace: i32, + conowner: i32, + conforencoding: i16, + contoencoding: i16, + conproc: i32, + condefault: bool, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs index 80b5166140e3c..b0510d0244345 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_database.rs @@ -12,13 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::convert::Into; -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_database` stores database. /// @@ -36,32 +31,32 @@ use crate::catalog::system_catalog::BuiltinView; /// ``` /// /// Ref: [`pg_database`](https://www.postgresql.org/docs/current/catalog-pg-database.html) -pub static PG_DATABASE: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_database", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "datname"), - (DataType::Int32, "datdba"), - (DataType::Int32, "encoding"), - (DataType::Varchar, "datcollate"), - (DataType::Varchar, "datctype"), - (DataType::Boolean, "datistemplate"), - (DataType::Boolean, "datallowconn"), - (DataType::Int32, "datconnlimit"), - (DataType::Int32, "dattablespace"), - (DataType::Varchar, "datacl"), - ], - sql: "SELECT id AS oid, \ - name AS datname, \ - owner AS datdba, \ - 6 AS encoding, \ - 'C' AS datcollate, \ - 'C' AS datctype, \ - false AS datistemplate, \ - true AS datallowconn, \ - -1 AS datconnlimit, \ - 1663 AS dattablespace, \ - acl AS datacl FROM rw_catalog.rw_databases" - .into(), -}); +#[system_catalog( + view, + "pg_catalog.pg_database", + "SELECT id AS oid, + name AS datname, + owner AS datdba, + 6 AS encoding, + 'C' AS datcollate, + 'C' AS datctype, + false AS datistemplate, + true AS datallowconn, + -1 AS datconnlimit, + 1663 AS dattablespace, + acl AS datacl FROM rw_catalog.rw_databases" +)] +#[derive(Fields)] +struct PgDatabase { + oid: i32, + datname: String, + datdba: i32, + encoding: i32, + datcollate: String, + datctype: String, + datistemplate: bool, + datallowconn: bool, + datconnlimit: i32, + dattablespace: i32, + datacl: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_depend.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_depend.rs index ba52a8fcd23cb..d1ce940d83522 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_depend.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_depend.rs @@ -12,28 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_DEPEND_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "classid"), - (DataType::Int32, "objid"), - (DataType::Int16, "objsubid"), - (DataType::Int32, "refclassid"), - (DataType::Int32, "refobjid"), - (DataType::Int16, "refobjsubid"), - (DataType::Varchar, "deptype"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_depend` records the dependency relationships between database objects. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-depend.html`] -pub static PG_DEPEND: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_depend", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_DEPEND_COLUMNS, - sql: infer_dummy_view_sql(PG_DEPEND_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_depend")] +#[derive(Fields)] +struct PgDepend { + classid: i32, + objid: i32, + objsubid: i16, + refclassid: i32, + refobjid: i32, + refobjsubid: i16, + deptype: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs index 0a1749300b96d..668450361c2f3 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_description.rs @@ -12,34 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_description` stores description. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-description.html`] -pub static PG_DESCRIPTION: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_description", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "objoid"), - (DataType::Int32, "classoid"), - (DataType::Int32, "objsubid"), - (DataType::Varchar, "description"), - ], +#[system_catalog(view, "pg_catalog.pg_description", // objsubid = 0 => _row_id (hidden column) // objsubid is NULL => table self - sql: "SELECT objoid, \ - classoid, \ - CASE \ - WHEN objsubid = 0 THEN -1 \ - WHEN objsubid IS NULL THEN 0 \ - ELSE objsubid \ - END AS objsubid, \ - description FROM rw_catalog.rw_description \ - WHERE description IS NOT NULL;" - .into(), -}); + "SELECT objoid, + classoid, + CASE + WHEN objsubid = 0 THEN -1 + WHEN objsubid IS NULL THEN 0 + ELSE objsubid + END AS objsubid, + description + FROM rw_catalog.rw_description + WHERE description IS NOT NULL;" +)] +#[derive(Fields)] +struct PgDescription { + objoid: i32, + classoid: i32, + objsubid: i32, + description: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs index 484a9cfce4835..b25f27aeb5ef8 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_enum.rs @@ -12,26 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_ENUM_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Int32, "enumtypid"), - (DataType::Float32, "enumsortorder"), - (DataType::Varchar, "enumlabel"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The `pg_enum` catalog contains entries showing the values and labels for each enum type. /// The internal representation of a given enum value is actually the OID of its associated row in /// `pg_enum`. Reference: [`https://www.postgresql.org/docs/current/catalog-pg-enum.html`] -pub static PG_ENUM: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_enum", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_ENUM_COLUMNS, - sql: infer_dummy_view_sql(PG_ENUM_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_enum")] +#[derive(Fields)] +struct PgEnum { + oid: i32, + enumtypid: i32, + enumsortorder: f32, + enumlabel: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_extension.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_extension.rs index b03209077ef45..c241022fdf371 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_extension.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_extension.rs @@ -12,34 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub static PG_EXTENSION_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Int32, "oid"), // oid - (DataType::Varchar, "extname"), - (DataType::Int32, "extowner"), // oid - (DataType::Int32, "extnamespace"), // oid - (DataType::Boolean, "extrelocatable"), - (DataType::Varchar, "extversion"), - (DataType::List(Box::new(DataType::Int32)), "extconfig"), // []oid - (DataType::List(Box::new(DataType::Varchar)), "extcondition"), - ] -}); +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_extension` stores information about the installed extensions. See Section 38.17 /// for details about extensions. /// /// Reference: . /// Currently, we don't have any type of extension. -pub static PG_EXTENSION: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_extension", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &PG_EXTENSION_COLUMNS, - sql: infer_dummy_view_sql(&PG_EXTENSION_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_extension")] +#[derive(Fields)] +struct PgExtension { + oid: i32, + extname: String, + extowner: i32, + extnamespace: i32, + extrelocatable: bool, + extversion: String, + extconfig: Vec, + extcondition: Vec, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs index b68d080fd11ba..196c36ec7f1af 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs @@ -12,48 +12,38 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::convert::Into; -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_INDEX_TABLE_NAME: &str = "pg_index"; -pub static PG_INDEX_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Int32, "indexrelid"), - (DataType::Int32, "indrelid"), - (DataType::Int16, "indnatts"), - // We return false as default to indicate that this is NOT a unique index - (DataType::Boolean, "indisunique"), - (DataType::List(Box::new(DataType::Int16)), "indkey"), - (DataType::List(Box::new(DataType::Int16)), "indoption"), - // None. We don't have `pg_node_tree` type yet, so we use `text` instead. - (DataType::Varchar, "indexprs"), - // None. We don't have `pg_node_tree` type yet, so we use `text` instead. - (DataType::Varchar, "indpred"), - // TODO: we return false as the default value. - (DataType::Boolean, "indisprimary"), - ] -}); +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_index` contains part of the information about indexes. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-index.html`] -pub static PG_INDEX: LazyLock = LazyLock::new(|| BuiltinView { - name: PG_INDEX_TABLE_NAME, - schema: PG_CATALOG_SCHEMA_NAME, - columns: &PG_INDEX_COLUMNS, - sql: "SELECT id AS indexrelid, \ - primary_table_id AS indrelid, \ - ARRAY_LENGTH(indkey)::smallint AS indnatts, \ - false AS indisunique, \ - indkey, \ - ARRAY[]::smallint[] as indoption, \ - NULL AS indexprs, \ - NULL AS indpred, \ - FALSE AS indisprimary \ - FROM rw_catalog.rw_indexes" - .into(), -}); +#[system_catalog( + view, + "pg_catalog.pg_index", + "SELECT id AS indexrelid, + primary_table_id AS indrelid, + ARRAY_LENGTH(indkey)::smallint AS indnatts, + false AS indisunique, + indkey, + ARRAY[]::smallint[] as indoption, + NULL AS indexprs, + NULL AS indpred, + FALSE AS indisprimary + FROM rw_catalog.rw_indexes" +)] +#[derive(Fields)] +struct PgIndex { + indexrelid: i32, + indrelid: i32, + indnatts: i16, + // We return false as default to indicate that this is NOT a unique index + indisunique: bool, + indkey: Vec, + indoption: Vec, + // None. We don't have `pg_node_tree` type yet, so we use `text` instead. + indexprs: Option, + // None. We don't have `pg_node_tree` type yet, so we use `text` instead. + indpred: Option, + // TODO: we return false as the default value. + indisprimary: bool, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs index 9fb8b06cedf87..e35aba9567ae2 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs @@ -12,33 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The view `pg_indexes` provides access to useful information about each index in the database. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-indexes.html`] -pub static PG_INDEXES: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_indexes", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "schemaname"), - (DataType::Varchar, "tablename"), - (DataType::Varchar, "indexname"), - (DataType::Varchar, "tablespace"), - (DataType::Varchar, "indexdef"), - ], - sql: "SELECT s.name AS schemaname, \ - t.name AS tablename, \ - i.name AS indexname, \ - NULL AS tablespace, \ - i.definition AS indexdef \ - FROM rw_catalog.rw_indexes i \ - JOIN rw_catalog.rw_tables t ON i.primary_table_id = t.id \ - JOIN rw_catalog.rw_schemas s ON i.schema_id = s.id\ +#[system_catalog( + view, + "pg_catalog.pg_indexes", + "SELECT s.name AS schemaname, + t.name AS tablename, + i.name AS indexname, + NULL AS tablespace, + i.definition AS indexdef + FROM rw_catalog.rw_indexes i + JOIN rw_catalog.rw_tables t ON i.primary_table_id = t.id + JOIN rw_catalog.rw_schemas s ON i.schema_id = s.id " - .to_string(), -}); +)] +#[derive(Fields)] +struct PgIndexes { + schemaname: String, + tablename: String, + indexname: String, + tablespace: String, + indexdef: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs index 08a92f9c13fb3..9c3b7067cbbb1 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_inherits.rs @@ -12,26 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_INHERITS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "inhrelid"), - (DataType::Int32, "inhparent"), - (DataType::Int32, "inhseqno"), - (DataType::Boolean, "inhdetachpending"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_inherits` records information about table and index inheritance hierarchies. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-inherits.html`] /// This is introduced only for pg compatibility and is not used in our system. -pub static PG_INHERITS: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_inherits", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_INHERITS_COLUMNS, - sql: infer_dummy_view_sql(PG_INHERITS_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_inherits")] +#[derive(Fields)] +struct PgInherits { + inhrelid: i32, + inhparent: i32, + inhseqno: i32, + inhdetachpending: bool, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs index 269b1f701ebf9..c3bb43ef0a842 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_keywords.rs @@ -12,27 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; pub const PG_KEYWORDS_TABLE_NAME: &str = "pg_keywords"; pub const PG_GET_KEYWORDS_FUNC_NAME: &str = "pg_get_keywords"; -pub const PG_KEYWORDS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Varchar, "word"), - (DataType::Varchar, "catcode"), - (DataType::Varchar, "catdesc"), -]; /// The catalog `pg_keywords` stores keywords. `pg_get_keywords` returns the content of this table. /// Ref: [`https://www.postgresql.org/docs/15/functions-info.html`] // TODO: change to read reserved keywords here -pub static PG_KEYWORDS: LazyLock = LazyLock::new(|| BuiltinView { - name: PG_KEYWORDS_TABLE_NAME, - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_KEYWORDS_COLUMNS, - sql: infer_dummy_view_sql(PG_KEYWORDS_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_keywords")] +#[derive(Fields)] +struct PgKeywords { + word: String, + catcode: String, + catdesc: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_locks.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_locks.rs index c47ac8533daa0..f231c39d3d57c 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_locks.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_locks.rs @@ -12,39 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_LOCKS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Varchar, "locktype"), - (DataType::Int32, "database"), // oid - (DataType::Int32, "relation"), // oid - (DataType::Int32, "page"), - (DataType::Int16, "tuple"), - (DataType::Varchar, "virtualxid"), - (DataType::Int32, "transactionid"), // xid - (DataType::Int32, "classid"), // oid - (DataType::Int32, "objid"), // oid - (DataType::Int16, "objsubid"), - (DataType::Varchar, "virtualtransaction"), - (DataType::Int32, "pid"), - (DataType::Varchar, "mode"), - (DataType::Boolean, "granted"), - (DataType::Boolean, "fastpath"), - (DataType::Timestamptz, "waitstart"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_locks` provides access to information about the locks held by active processes /// within the database server. /// Reference: [`https://www.postgresql.org/docs/current/view-pg-locks.html`]. /// Currently, we don't have any type of lock. -pub static PG_LOCKS: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_locks", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_LOCKS_COLUMNS, - sql: infer_dummy_view_sql(PG_LOCKS_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_locks")] +#[derive(Fields)] +struct PgLock { + locktype: String, + database: i32, // oid + relation: i32, // oid + page: i32, + tuple: i16, + virtualxid: String, + transactionid: i32, + classid: i32, // oid + objid: i32, // oid + objsubid: i16, + virtualtransaction: String, + pid: i32, + mode: String, + granted: bool, + fastpath: bool, + waitstart: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs index 5197dfbece7a2..f214ac756b667 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_matviews.rs @@ -12,41 +12,37 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The view `pg_matviews` provides access to useful information about each materialized view in the /// database. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-matviews.html`] -pub static PG_MATVIEWS: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_matviews", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "schemaname"), - (DataType::Varchar, "matviewname"), - (DataType::Int32, "matviewowner"), - (DataType::Varchar, "definition"), - // Below are some columns that PostgreSQL doesn't have. - // TODO: these field is only exist in RW and used by cloud side, need to remove it and let - // cloud switch to use `rw_catalog.rw_relation_info`. - (DataType::Int32, "matviewid"), - (DataType::Varchar, "matviewtimezone"), /* The timezone used to interpret ambiguous - * dates/timestamps as tstz */ - (DataType::Varchar, "matviewgraph"), /* materialized view graph is json encoded fragment - * infos. */ - ], - sql: "SELECT schemaname, \ - i.relationname AS matviewname, \ - i.relationowner AS matviewowner, \ - definition, \ - i.relationid AS matviewid, \ - i.relationtimezone AS matviewtimezone, \ - i.fragments AS matviewgraph \ - FROM rw_catalog.rw_relation_info i \ - WHERE i.relationtype = 'MATERIALIZED VIEW'" - .to_string(), -}); +#[system_catalog( + view, + "pg_catalog.pg_matviews", + "SELECT schemaname, + i.relationname AS matviewname, + i.relationowner AS matviewowner, + definition, + i.relationid AS matviewid, + i.relationtimezone AS matviewtimezone, + i.fragments AS matviewgraph + FROM rw_catalog.rw_relation_info i + WHERE i.relationtype = 'MATERIALIZED VIEW'" +)] +#[derive(Fields)] +struct PgMatview { + schemaname: String, + matviewname: String, + matviewowner: i32, + definition: String, + // Below are some columns that PostgreSQL doesn't have. + // TODO: these field is only exist in RW and used by cloud side, need to remove it and let + // cloud switch to use `rw_catalog.rw_relation_info`. + matviewid: i32, + // The timezone used to interpret ambiguous dates/timestamps as tstz + matviewtimezone: String, + // materialized view graph is json encoded fragment infos. + matviewgraph: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs index f5b57ef7f98aa..69506da1ecd67 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_namespace.rs @@ -12,29 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::convert::Into; -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; - -pub const PG_NAMESPACE_TABLE_NAME: &str = "pg_namespace"; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_namespace` stores namespaces. A namespace is the structure underlying SQL /// schemas: each namespace can have a separate collection of relations, types, etc. without name /// conflicts. Ref: [`https://www.postgresql.org/docs/current/catalog-pg-namespace.html`] -pub static PG_NAMESPACE: LazyLock = LazyLock::new(|| { - BuiltinView { - name: PG_NAMESPACE_TABLE_NAME, - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "nspname"), - (DataType::Int32, "nspowner"), - (DataType::Varchar, "nspacl"), - ], - sql: "SELECT id AS oid, name AS nspname, owner AS nspowner, acl AS nspacl FROM rw_catalog.rw_schemas".into(), - } -}); +#[system_catalog( + view, + "pg_catalog.pg_namespace", + "SELECT id AS oid, name AS nspname, owner AS nspowner, acl AS nspacl FROM rw_catalog.rw_schemas" +)] +#[derive(Fields)] +struct PgNamespace { + oid: i32, + nspname: String, + nspowner: i32, + nspacl: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs index 0e473fe743f36..efdc72b09453a 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_opclass.rs @@ -12,30 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_OPCLASS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Int32, "opcmethod"), - (DataType::Varchar, "opcname"), - (DataType::Int32, "opcnamespace"), - (DataType::Int32, "opcowner"), - (DataType::Int32, "opcfamily"), - (DataType::Int32, "opcintype"), - (DataType::Int32, "opcdefault"), - (DataType::Int32, "opckeytype"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_opclass` defines index access method operator classes. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-opclass.html`]. -pub static PG_OPCLASS: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_opclass", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_OPCLASS_COLUMNS, - sql: infer_dummy_view_sql(PG_OPCLASS_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_opclass")] +#[derive(Fields)] +struct PgOpclass { + oid: i32, + opcmethod: i32, + opcname: String, + opcnamespace: i32, + opcowner: i32, + opcfamily: i32, + opcintype: i32, + opcdefault: bool, + opckeytype: i32, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs index 34b31e78c98ec..2947995657465 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_operator.rs @@ -12,36 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_OPERATOR_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "oprname"), - (DataType::Int32, "oprnamespace"), - (DataType::Int32, "oprowner"), - (DataType::Varchar, "oprkind"), - (DataType::Boolean, "oprcanmerge"), - (DataType::Boolean, "oprcanhash"), - (DataType::Int32, "oprleft"), - (DataType::Int32, "oprright"), - (DataType::Int32, "oprresult"), - (DataType::Int32, "oprcom"), - (DataType::Int32, "oprnegate"), - (DataType::Int32, "oprcode"), - (DataType::Int32, "oprrest"), - (DataType::Int32, "oprjoin"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_operator` stores operator info. /// Reference: [`https://www.postgresql.org/docs/current/catalog-pg-operator.html`] -pub static PG_OPERATOR: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_operator", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_OPERATOR_COLUMNS, - sql: infer_dummy_view_sql(PG_OPERATOR_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_operator")] +#[derive(Fields)] +struct PgOpclass { + oid: i32, + oprname: String, + oprnamespace: i32, + oprowner: i32, + oprkind: String, + oprcanmerge: bool, + oprcanhash: bool, + oprleft: i32, + oprright: i32, + oprresult: i32, + oprcom: i32, + oprnegate: i32, + oprcode: i32, + oprrest: i32, + oprjoin: i32, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs index 988fca53ca3d5..259f42376abe9 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_proc.rs @@ -12,30 +12,22 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_PROC_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "proname"), - (DataType::Int32, "pronamespace"), - (DataType::Int32, "proowner"), - (DataType::Int32, "proargdefaults"), - (DataType::Int32, "prorettype"), // Data type of the return value, refer to pg_type. -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_proc` stores information about functions, procedures, aggregate functions, and /// window functions (collectively also known as routines). /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-proc.html`] // TODO: read real data including oid etc in rw, currently there are no such data in rw. // more details can be found here: https://github.com/postgres/postgres/blob/master/src/include/catalog/pg_proc.dat -pub static PG_PROC: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_proc", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_PROC_COLUMNS, - sql: infer_dummy_view_sql(PG_PROC_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_proc")] +#[derive(Fields)] +struct PgProc { + oid: i32, + proname: String, + pronamespace: i32, + proowner: i32, + proargdefaults: i32, + // Data type of the return value, refer to pg_type. + prorettype: i32, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs index e52ba53f85ab7..bcef20a74920a 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_roles.rs @@ -12,45 +12,41 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_roles` provides access to information about database roles. This is simply a /// publicly readable view of `pg_authid` that blanks out the password field. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-roles.html`] -pub static PG_ROLES: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_roles", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "rolname"), - (DataType::Boolean, "rolsuper"), - (DataType::Boolean, "rolinherit"), - (DataType::Boolean, "rolcreaterole"), - (DataType::Boolean, "rolcreatedb"), - (DataType::Boolean, "rolcanlogin"), - (DataType::Boolean, "rolreplication"), - (DataType::Int32, "rolconnlimit"), - (DataType::Timestamptz, "rolvaliduntil"), - (DataType::Boolean, "rolbypassrls"), - (DataType::Varchar, "rolpassword"), - ], - sql: "SELECT id AS oid, \ - name AS rolname, \ - is_super AS rolsuper, \ - true AS rolinherit, \ - create_user AS rolcreaterole, \ - create_db AS rolcreatedb, \ - can_login AS rolcanlogin, \ - true AS rolreplication, \ - -1 AS rolconnlimit, \ - NULL::timestamptz AS rolvaliduntil, \ - true AS rolbypassrls, \ - '********' AS rolpassword \ - FROM rw_catalog.rw_users" - .into(), -}); +#[system_catalog( + view, + "pg_catalog.pg_roles", + "SELECT id AS oid, + name AS rolname, + is_super AS rolsuper, + true AS rolinherit, + create_user AS rolcreaterole, + create_db AS rolcreatedb, + can_login AS rolcanlogin, + true AS rolreplication, + -1 AS rolconnlimit, + NULL::timestamptz AS rolvaliduntil, + true AS rolbypassrls, + '********' AS rolpassword + FROM rw_catalog.rw_users" +)] +#[derive(Fields)] +struct PgRule { + oid: i32, + rolname: String, + rolsuper: bool, + rolinherit: bool, + rolcreaterole: bool, + rolcreatedb: bool, + rolcanlogin: bool, + rolreplication: bool, + rolconnlimit: i32, + rolvaliduntil: Timestamptz, + rolbypassrls: bool, + rolpassword: String, +} 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 e2955bd9f8ab1..5acd1dab05a7d 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,21 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_SETTINGS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = - &[(DataType::Varchar, "name"), (DataType::Varchar, "setting")]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_settings` stores settings. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-settings.html`] -pub static PG_SETTINGS: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_settings", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_SETTINGS_COLUMNS, - sql: infer_dummy_view_sql(PG_SETTINGS_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_settings")] +#[derive(Fields)] +struct PgSetting { + name: String, + setting: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs index 3dde2bdfafbc2..3f4a8f68b4edf 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shadow.rs @@ -12,49 +12,40 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{BuiltinView, SystemCatalogColumnsDef}; - -pub static PG_SHADOW_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Varchar, "usename"), - (DataType::Int32, "usesysid"), - (DataType::Boolean, "usecreatedb"), - (DataType::Boolean, "usesuper"), - // User can initiate streaming replication and put the system in and out of backup mode. - (DataType::Boolean, "userepl"), - // User can bypass row level security. - (DataType::Boolean, "usebypassrls"), - (DataType::Varchar, "passwd"), - // Password expiry time (only used for password authentication) - (DataType::Timestamptz, "valuntil"), - // Session defaults for run-time configuration variables - (DataType::List(Box::new(DataType::Varchar)), "useconfig"), - ] -}); - +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; /// The view `pg_shadow` exists for backwards compatibility: it emulates a catalog that existed in /// PostgreSQL before version 8.1. It shows properties of all roles that are marked as rolcanlogin /// in `pg_authid`. Ref: [`https://www.postgresql.org/docs/current/view-pg-shadow.html`] -pub static PG_SHADOW: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_shadow", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &PG_SHADOW_COLUMNS, - sql: "SELECT u.name AS usename,\ - u.id AS usesysid, \ - u.create_db AS usecreatedb, \ - u.is_super AS usesuper, \ - false AS userepl, \ - false AS usebypassrls, \ - s.password AS passwd, \ - NULL::timestamptz AS valuntil, \ - NULL::text[] AS useconfig \ - FROM rw_catalog.rw_users u \ - JOIN rw_catalog.rw_user_secrets s \ - ON u.id = s.id" - .to_string(), -}); +#[system_catalog( + view, + "pg_catalog.pg_shadow", + "SELECT u.name AS usename, + u.id AS usesysid, + u.create_db AS usecreatedb, + u.is_super AS usesuper, + false AS userepl, + false AS usebypassrls, + s.password AS passwd, + NULL::timestamptz AS valuntil, + NULL::text[] AS useconfig + FROM rw_catalog.rw_users u + JOIN rw_catalog.rw_user_secrets s + ON u.id = s.id" +)] +#[derive(Fields)] +struct PgShadow { + usename: String, + usesysid: i32, + usecreatedb: bool, + usesuper: bool, + // User can initiate streaming replication and put the system in and out of backup mode. + userepl: bool, + // User can bypass row level security. + usebypassrls: bool, + passwd: String, + // Password expiry time (only used for password authentication) + valuntil: Timestamptz, + // Session defaults for run-time configuration variables + useconfig: Vec, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs index eee6e8fead496..cc8bd37d33068 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_shdescription.rs @@ -12,24 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_SHDESCRIPTION_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "objoid"), - (DataType::Int32, "classoid"), - (DataType::Varchar, "description"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_shdescription` stores optional descriptions (comments) for shared database /// objects. Ref: [`https://www.postgresql.org/docs/current/catalog-pg-shdescription.html`] -pub static PG_SHDESCRIPTION: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_shdescription", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_SHDESCRIPTION_COLUMNS, - sql: infer_dummy_view_sql(PG_SHDESCRIPTION_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_shdescription")] +#[derive(Fields)] +struct PgShdescription { + objoid: i32, + classoid: i32, + description: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs index 15ea7aa4095df..9b72190386292 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_stat_activity.rs @@ -12,39 +12,34 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_STAT_ACTIVITY_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "pid"), // Process ID of this backend. - (DataType::Int32, "datid"), // OID of the database this backend is connected to. - (DataType::Varchar, "datname"), // Name of the database this backend is connected to. - (DataType::Int32, "leader_pid"), /* Process ID of the parallel group leader, if this process - * is a parallel query worker. NULL if this process is a - * parallel group leader or does not participate in - * parallel query. */ - (DataType::Int32, "usesysid"), // OID of the user logged into this backend. - (DataType::Varchar, "usename"), // Name of the user logged into this backend. - (DataType::Varchar, "application_name"), /* Name of the application that is connected to - * this backend. */ - (DataType::Varchar, "client_addr"), // IP address of the client connected to this backend. - (DataType::Varchar, "client_hostname"), /* Host name of the connected client, as reported by a - * reverse DNS lookup of client_addr. */ - (DataType::Int16, "client_port"), /* TCP port number that the client is using for - * communication with this backend, or -1 if a Unix socket - * is used. */ -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The `pg_stat_activity` view will have one row per server process, showing information related to /// the current activity of that process. /// Ref: [`https://www.postgresql.org/docs/current/monitoring-stats.html#MONITORING-PG-STAT-ACTIVITY-VIEW`] -pub static PG_STAT_ACTIVITY: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_stat_activity", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_STAT_ACTIVITY_COLUMNS, - sql: infer_dummy_view_sql(PG_STAT_ACTIVITY_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_stat_activity")] +#[derive(Fields)] +struct PgStatActivity { + /// Process ID of this backend. + pid: i32, + /// OID of the database this backend is connected to. + datid: i32, + /// Name of the database this backend is connected to. + datname: String, + /// Process ID of the parallel group leader, if this process is a parallel query worker. + /// NULL if this process is a parallel group leader or does not participate in parallel query. + leader_pid: i32, + /// OID of the user logged into this backend. + usesysid: i32, + /// Name of the user logged into this backend. + usename: String, + /// Name of the application that is connected to this backend. + application_name: String, + /// IP address of the client connected to this backend. + client_addr: String, + /// Host name of the connected client, as reported by a reverse DNS lookup of client_addr. + client_hostname: String, + /// TCP port number that the client is using for communication with this backend, or -1 if a Unix socket is used. + client_port: i16, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs index 569373864fd8a..07b3a15e09b1b 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tables.rs @@ -12,41 +12,35 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_TABLES_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Varchar, "schemaname"), - (DataType::Varchar, "tablename"), - (DataType::Varchar, "tableowner"), - (DataType::Varchar, "tablespace"), /* Since we don't have any concept of tablespace, we will - * set this to null. */ -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The view `pg_tables` provides access to useful information about each table in the database. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-tables.html`] -pub static PG_TABLES: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_tables", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_TABLES_COLUMNS, - sql: "SELECT s.name AS schemaname, \ - t.tablename, \ - pg_catalog.pg_get_userbyid(t.owner) AS tableowner, \ - NULL AS tablespace \ - FROM \ - (SELECT name AS tablename, \ - schema_id, \ - owner \ - FROM rw_catalog.rw_tables \ - UNION \ - SELECT name AS tablename, \ - schema_id, \ - owner \ - FROM rw_catalog.rw_system_tables) AS t \ - JOIN rw_catalog.rw_schemas s ON t.schema_id = s.id" - .into(), -}); +#[system_catalog( + view, + "pg_catalog.pg_tables", + "SELECT s.name AS schemaname, + t.tablename, + pg_catalog.pg_get_userbyid(t.owner) AS tableowner, + NULL AS tablespace + FROM + (SELECT name AS tablename, + schema_id, + owner + FROM rw_catalog.rw_tables + UNION + SELECT name AS tablename, + schema_id, + owner + FROM rw_catalog.rw_system_tables) AS t + JOIN rw_catalog.rw_schemas s ON t.schema_id = s.id" +)] +#[derive(Fields)] +struct PgTable { + schemaname: String, + tablename: String, + tableowner: String, + // Since we don't have any concept of tablespace, we will set this to null. + tablespace: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs index 605c82c720a2f..81ef923bfa24e 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_tablespace.rs @@ -12,27 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{infer_dummy_view_sql, BuiltinView, SystemCatalogColumnsDef}; - -pub const PG_TABLESPACE_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "spcname"), - (DataType::Int32, "spcowner"), - (DataType::Varchar, "spcacl"), - (DataType::Varchar, "spcoptions"), -]; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_tablespace` stores information about the available tablespaces. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-tablespace.html`] /// This is introduced only for pg compatibility and is not used in our system. -pub static PG_TABLESPACE: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_tablespace", - schema: PG_CATALOG_SCHEMA_NAME, - columns: PG_TABLESPACE_COLUMNS, - sql: infer_dummy_view_sql(PG_TABLESPACE_COLUMNS), -}); +#[system_catalog(view, "pg_catalog.pg_tablespace")] +#[derive(Fields)] +struct PgTablespace { + oid: i32, + spcname: String, + spcowner: i32, + spcacl: String, + spcoptions: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs index 8a59e8add8477..65c71bb8fb0e7 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs @@ -12,68 +12,50 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_type` stores information about data types. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-type.html`] -pub static PG_TYPE: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_type", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "oid"), - (DataType::Varchar, "typname"), - // 0 - (DataType::Int32, "typelem"), - // 0 - (DataType::Int32, "typarray"), - // FIXME: Should be regproc type - (DataType::Varchar, "typinput"), - // false - (DataType::Boolean, "typnotnull"), - // 0 - (DataType::Int32, "typbasetype"), - // -1 - (DataType::Int32, "typtypmod"), - // 0 - (DataType::Int32, "typcollation"), - // 0 - (DataType::Int32, "typlen"), - // should be pg_catalog oid. - (DataType::Int32, "typnamespace"), - // 'b' - (DataType::Varchar, "typtype"), - // 0 - (DataType::Int32, "typrelid"), - // None - (DataType::Varchar, "typdefault"), - // None - (DataType::Varchar, "typcategory"), - // None - (DataType::Int32, "typreceive"), - ], - sql: "SELECT t.id AS oid, \ - t.name AS typname, \ - t.typelem AS typelem, \ - t.typarray AS typarray, \ - t.input_oid AS typinput, \ - false AS typnotnull, \ - 0 AS typbasetype, \ - -1 AS typtypmod, \ - 0 AS typcollation, \ - 0 AS typlen, \ - s.id AS typnamespace, \ - 'b' AS typtype, \ - 0 AS typrelid, \ - NULL AS typdefault, \ - NULL AS typcategory, \ - NULL::integer AS typreceive \ - FROM rw_catalog.rw_types t \ - JOIN rw_catalog.rw_schemas s \ - ON s.name = 'pg_catalog'" - .to_string(), -}); +#[system_catalog( + view, + "pg_catalog.pg_type", + "SELECT t.id AS oid, + t.name AS typname, + t.typelem AS typelem, + t.typarray AS typarray, + t.input_oid AS typinput, + false AS typnotnull, + 0 AS typbasetype, + -1 AS typtypmod, + 0 AS typcollation, + 0 AS typlen, + s.id AS typnamespace, + 'b' AS typtype, + 0 AS typrelid, + NULL AS typdefault, + NULL AS typcategory, + NULL::integer AS typreceive + FROM rw_catalog.rw_types t + JOIN rw_catalog.rw_schemas s + ON s.name = 'pg_catalog'" +)] +#[derive(Fields)] +struct PgType { + oid: i32, + typname: String, + typelem: i32, + typarray: i32, + typinput: String, + typnotnull: bool, + typbasetype: i32, + typtypmod: i32, + typcollation: i32, + typlen: i32, + typnamespace: i32, + typtype: String, + typrelid: i32, + typdefault: String, + typcategory: String, + typreceive: i32, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs index 20b9c7a04d9cb..c47d849458527 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_user.rs @@ -12,32 +12,26 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The catalog `pg_user` provides access to information about database users. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-user.html`] -pub const PG_USER_TABLE_NAME: &str = "pg_user"; - -pub static PG_USER: LazyLock = LazyLock::new(|| BuiltinView { - name: PG_USER_TABLE_NAME, - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "usesysid"), - (DataType::Varchar, "name"), - (DataType::Boolean, "usecreatedb"), - (DataType::Boolean, "usesuper"), - (DataType::Varchar, "passwd"), - ], - sql: "SELECT id AS usesysid, \ - name, \ - create_db AS usecreatedb, \ - is_super AS usesuper, \ - '********' AS passwd \ - FROM rw_catalog.rw_users" - .into(), -}); +#[system_catalog( + view, + "pg_catalog.pg_user", + "SELECT id AS usesysid, + name, + create_db AS usecreatedb, + is_super AS usesuper, + '********' AS passwd + FROM rw_catalog.rw_users" +)] +#[derive(Fields)] +struct PgUser { + usesysid: i32, + usename: String, + usecreatedb: bool, + usesuper: bool, + passwd: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs index e0da7a954a6af..5e32e48990f24 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_views.rs @@ -12,30 +12,26 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::PG_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// The view `pg_views` provides access to useful information about each view in the database. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-views.html`] -pub static PG_VIEWS: LazyLock = LazyLock::new(|| BuiltinView { - name: "pg_views", - schema: PG_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "schemaname"), - (DataType::Varchar, "viewname"), - (DataType::Varchar, "viewowner"), - (DataType::Varchar, "definition"), - ], - sql: "SELECT s.name AS schemaname, \ - v.name AS viewname, \ - pg_catalog.pg_get_userbyid(v.owner) AS viewowner, \ - v.definition AS definition \ - FROM rw_catalog.rw_views v \ - JOIN rw_catalog.rw_schemas s \ - ON v.schema_id = s.id" - .into(), -}); +#[system_catalog( + view, + "pg_catalog.pg_views", + "SELECT s.name AS schemaname, + v.name AS viewname, + pg_catalog.pg_get_userbyid(v.owner) AS viewowner, + v.definition AS definition + FROM rw_catalog.rw_views v + JOIN rw_catalog.rw_schemas s + ON v.schema_id = s.id" +)] +#[derive(Fields)] +struct PgView { + schemaname: String, + viewname: String, + viewowner: String, + definition: String, +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs index 1961a8c928834..9422fd3ec1c57 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs @@ -51,43 +51,3 @@ mod rw_user_secrets; mod rw_users; mod rw_views; mod rw_worker_nodes; - -pub use rw_actors::*; -pub use rw_columns::*; -pub use rw_connections::*; -pub use rw_databases::*; -pub use rw_ddl_progress::*; -pub use rw_description::*; -pub use rw_event_logs::*; -pub use rw_fragment_parallelism::*; -pub use rw_fragments::*; -pub use rw_functions::*; -pub use rw_hummock_branched_objects::*; -pub use rw_hummock_compact_task_assignment::*; -pub use rw_hummock_compact_task_progress::*; -pub use rw_hummock_compaction_group_configs::*; -pub use rw_hummock_meta_configs::*; -pub use rw_hummock_pinned_snapshots::*; -pub use rw_hummock_pinned_versions::*; -pub use rw_hummock_version::*; -pub use rw_hummock_version_deltas::*; -pub use rw_indexes::*; -pub use rw_internal_tables::*; -pub use rw_materialized_views::*; -pub use rw_meta_snapshot::*; -pub use rw_parallel_units::*; -pub use rw_relation_info::*; -pub use rw_relations::*; -pub use rw_schemas::*; -pub use rw_sinks::*; -pub use rw_sources::*; -pub use rw_streaming_parallelism::*; -pub use rw_system_tables::*; -pub use rw_table_fragments::*; -pub use rw_table_stats::*; -pub use rw_tables::*; -pub use rw_types::*; -pub use rw_user_secrets::*; -pub use rw_users::*; -pub use rw_views::*; -pub use rw_worker_nodes::*; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs index f4703b3148768..9769c7cafbf6c 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs @@ -12,40 +12,32 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_ACTORS: BuiltinTable = BuiltinTable { - name: "rw_actors", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "actor_id"), - (DataType::Int32, "fragment_id"), - (DataType::Int32, "parallel_unit_id"), - (DataType::Varchar, "status"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwActor { + #[primary_key] + actor_id: i32, + fragment_id: i32, + parallel_unit_id: i32, + state: String, +} -impl SysCatalogReaderImpl { - pub async fn read_rw_actor_states_info(&self) -> Result> { - let states = self.meta_client.list_actor_states().await?; +#[system_catalog(table, "rw_catalog.rw_actors")] +async fn read_rw_actors(reader: &SysCatalogReaderImpl) -> Result> { + let states = reader.meta_client.list_actor_states().await?; - Ok(states - .into_iter() - .map(|state| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(state.actor_id as i32)), - Some(ScalarImpl::Int32(state.fragment_id as i32)), - Some(ScalarImpl::Int32(state.parallel_unit_id as i32)), - Some(ScalarImpl::Utf8(state.state().as_str_name().into())), - ]) - }) - .collect_vec()) - } + Ok(states + .into_iter() + .map(|state| RwActor { + actor_id: state.actor_id as i32, + fragment_id: state.fragment_id as i32, + parallel_unit_id: state.parallel_unit_id as i32, + state: state.state().as_str_name().into(), + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs index 25184f48ae7c1..40760df81a492 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs @@ -12,164 +12,139 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; - -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub static RW_COLUMNS: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_columns", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "relation_id"), // belonged relation id - (DataType::Varchar, "name"), // column name - (DataType::Int32, "position"), // 1-indexed position - (DataType::Boolean, "is_hidden"), - (DataType::Boolean, "is_primary_key"), - (DataType::Boolean, "is_distribution_key"), - (DataType::Varchar, "data_type"), - (DataType::Int32, "type_oid"), - (DataType::Int16, "type_len"), - (DataType::Varchar, "udt_type"), - ], - pk: &[0, 1], -}); +#[derive(Fields)] +#[primary_key(relation_id, name)] +struct RwColumn { + relation_id: i32, // belonged relation id + name: String, // column name + position: i32, // 1-indexed position + is_hidden: bool, + is_primary_key: bool, + is_distribution_key: bool, + data_type: String, + type_oid: i32, + type_len: i16, + udt_type: String, +} -impl SysCatalogReaderImpl { - pub fn read_rw_columns_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; +#[system_catalog(table, "rw_catalog.rw_columns")] +fn read_rw_columns(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; - Ok(schemas - .flat_map(|schema| { - // view columns - let view_rows = schema.iter_view().flat_map(|view| { - view.columns.iter().enumerate().map(|(index, column)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(view.id as i32)), - Some(ScalarImpl::Utf8(column.name.clone().into())), - Some(ScalarImpl::Int32(index as i32 + 1)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Utf8(column.data_type().to_string().into())), - Some(ScalarImpl::Int32(column.data_type().to_oid())), - Some(ScalarImpl::Int16(column.data_type().type_len())), - Some(ScalarImpl::Utf8(column.data_type().pg_name().into())), - ]) + Ok(schemas + .flat_map(|schema| { + let view_rows = schema.iter_view().flat_map(|view| { + view.columns + .iter() + .enumerate() + .map(|(index, column)| RwColumn { + relation_id: view.id as i32, + name: column.name.clone(), + position: index as i32 + 1, + is_hidden: false, + is_primary_key: false, + is_distribution_key: false, + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), }) - }); + }); - // sink columns - let sink_rows = schema - .iter_sink() - .flat_map(|sink| { - sink.full_columns() - .iter() - .enumerate() - .map(|(index, column)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(sink.id.sink_id as i32)), - Some(ScalarImpl::Utf8(column.name().into())), - Some(ScalarImpl::Int32(index as i32 + 1)), - Some(ScalarImpl::Bool(column.is_hidden)), - Some(ScalarImpl::Bool(sink.downstream_pk.contains(&index))), - Some(ScalarImpl::Bool(sink.distribution_key.contains(&index))), - Some(ScalarImpl::Utf8(column.data_type().to_string().into())), - Some(ScalarImpl::Int32(column.data_type().to_oid())), - Some(ScalarImpl::Int16(column.data_type().type_len())), - Some(ScalarImpl::Utf8(column.data_type().pg_name().into())), - ]) - }) - }) - .chain(view_rows); + let sink_rows = schema + .iter_sink() + .flat_map(|sink| { + sink.full_columns() + .iter() + .enumerate() + .map(|(index, column)| RwColumn { + relation_id: sink.id.sink_id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: sink.downstream_pk.contains(&index), + is_distribution_key: sink.distribution_key.contains(&index), + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }) + .chain(view_rows); - // pg_catalog columns - let catalog_rows = schema - .iter_system_tables() - .flat_map(|table| { - table - .columns - .iter() - .enumerate() - .map(move |(index, column)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(column.name().into())), - Some(ScalarImpl::Int32(index as i32 + 1)), - Some(ScalarImpl::Bool(column.is_hidden)), - Some(ScalarImpl::Bool(table.pk.contains(&index))), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Utf8(column.data_type().to_string().into())), - Some(ScalarImpl::Int32(column.data_type().to_oid())), - Some(ScalarImpl::Int16(column.data_type().type_len())), - Some(ScalarImpl::Utf8(column.data_type().pg_name().into())), - ]) - }) - }) - .chain(sink_rows); + let catalog_rows = schema + .iter_system_tables() + .flat_map(|table| { + table + .columns + .iter() + .enumerate() + .map(move |(index, column)| RwColumn { + relation_id: table.id.table_id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: table.pk.contains(&index), + is_distribution_key: false, + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }) + .chain(sink_rows); - // table columns - let table_rows = schema - .iter_valid_table() - .flat_map(|table| { - table - .columns - .iter() - .enumerate() - .map(move |(index, column)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(column.name().into())), - Some(ScalarImpl::Int32(index as i32 + 1)), - Some(ScalarImpl::Bool(column.is_hidden)), - Some(ScalarImpl::Bool( - table.pk().iter().any(|idx| idx.column_index == index), - )), - Some(ScalarImpl::Bool( - table.distribution_key().contains(&index), - )), - Some(ScalarImpl::Utf8(column.data_type().to_string().into())), - Some(ScalarImpl::Int32(column.data_type().to_oid())), - Some(ScalarImpl::Int16(column.data_type().type_len())), - Some(ScalarImpl::Utf8(column.data_type().pg_name().into())), - ]) - }) - }) - .chain(catalog_rows); + let table_rows = schema + .iter_valid_table() + .flat_map(|table| { + table + .columns + .iter() + .enumerate() + .map(move |(index, column)| RwColumn { + relation_id: table.id.table_id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: table.pk().iter().any(|idx| idx.column_index == index), + is_distribution_key: table.distribution_key.contains(&index), + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }) + .chain(catalog_rows); - // source columns - schema - .iter_source() - .flat_map(|source| { - source - .columns - .iter() - .enumerate() - .map(move |(index, column)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(source.id as i32)), - Some(ScalarImpl::Utf8(column.name().into())), - Some(ScalarImpl::Int32(index as i32 + 1)), - Some(ScalarImpl::Bool(column.is_hidden)), - Some(ScalarImpl::Bool( - source.pk_col_ids.contains(&column.column_id()), - )), - Some(ScalarImpl::Bool(false)), - Some(ScalarImpl::Utf8(column.data_type().to_string().into())), - Some(ScalarImpl::Int32(column.data_type().to_oid())), - Some(ScalarImpl::Int16(column.data_type().type_len())), - Some(ScalarImpl::Utf8(column.data_type().pg_name().into())), - ]) - }) - }) - .chain(table_rows) - }) - .collect_vec()) - } + // source columns + schema + .iter_source() + .flat_map(|source| { + source + .columns + .iter() + .enumerate() + .map(move |(index, column)| RwColumn { + relation_id: source.id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: source.pk_col_ids.contains(&column.column_id()), + is_distribution_key: false, + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }) + .chain(table_rows) + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs index 528847be10436..2af0b29b16f76 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_connections.rs @@ -12,48 +12,40 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_CONNECTIONS: BuiltinTable = BuiltinTable { - name: "rw_connections", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "type"), - (DataType::Varchar, "provider"), - (DataType::Varchar, "acl"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwConnection { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + type_: String, + provider: String, + acl: String, +} -impl SysCatalogReaderImpl { - pub fn read_rw_connections_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; +#[system_catalog(table, "rw_catalog.rw_connections")] +fn read_rw_connections(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; - Ok(schemas - .flat_map(|schema| { - schema.iter_connections().map(|conn| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(conn.id as i32)), - Some(ScalarImpl::Utf8(conn.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(conn.owner as i32)), - Some(ScalarImpl::Utf8(conn.connection_type().into())), - Some(ScalarImpl::Utf8(conn.provider().into())), - Some(ScalarImpl::Utf8("".into())), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_connections().map(|conn| RwConnection { + id: conn.id as i32, + name: conn.name.clone(), + schema_id: schema.id() as i32, + owner: conn.owner as i32, + type_: conn.connection_type().into(), + provider: conn.provider().into(), + acl: "".into(), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs index 85f19afdb1640..472333e4bbb53 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_databases.rs @@ -12,48 +12,37 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::catalog::OwnedByUserCatalog; use crate::error::Result; -pub const RW_DATABASES: BuiltinTable = BuiltinTable { - name: "rw_databases", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "owner"), - (DataType::Varchar, "acl"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwDatabases { + #[primary_key] + id: i32, + name: String, + owner: i32, + acl: String, +} -impl SysCatalogReaderImpl { - pub fn read_rw_database_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_databases")] +fn read(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(reader - .iter_databases() - .map(|db| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(db.id() as i32)), - Some(ScalarImpl::Utf8(db.name().into())), - Some(ScalarImpl::Int32(db.owner() as i32)), - Some(ScalarImpl::Utf8( - get_acl_items(&Object::DatabaseId(db.id()), false, &users, username_map) - .into(), - )), - ]) - }) - .collect_vec()) - } + Ok(catalog_reader + .iter_databases() + .map(|db| RwDatabases { + id: db.id() as i32, + name: db.name().into(), + owner: db.owner() as i32, + acl: get_acl_items(&Object::DatabaseId(db.id()), false, &users, username_map), + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs index 7efa1edb2d10a..9f592d4e4f6b3 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_ddl_progress.rs @@ -13,52 +13,47 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, Timestamptz}; use risingwave_common::util::epoch::Epoch; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_DDL_PROGRESS: BuiltinTable = BuiltinTable { - name: "rw_ddl_progress", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "ddl_id"), - (DataType::Varchar, "ddl_statement"), - (DataType::Varchar, "progress"), - (DataType::Timestamptz, "initialized_at"), - ], - pk: &[0], -}; - -impl SysCatalogReaderImpl { - pub async fn read_ddl_progress(&self) -> Result> { - let ddl_progresses = self.meta_client.list_ddl_progress().await?; - - let table_ids = ddl_progresses - .iter() - .map(|progress| progress.id as u32) - .collect_vec(); - - let tables = self.meta_client.get_tables(&table_ids).await?; - - let ddl_progress = ddl_progresses - .into_iter() - .map(|s| { - let initialized_at = tables - .get(&(s.id as u32)) - .and_then(|table| table.initialized_at_epoch.map(Epoch::from)); +#[derive(Fields)] +struct RwDdlProgress { + #[primary_key] + ddl_id: i64, + ddl_statement: String, + progress: String, + initialized_at: Option, +} - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(s.id as i64)), - Some(ScalarImpl::Utf8(s.statement.into())), - Some(ScalarImpl::Utf8(s.progress.into())), - initialized_at.map(|e| e.as_scalar()), - ]) - }) - .collect_vec(); - Ok(ddl_progress) - } +#[system_catalog(table, "rw_catalog.rw_ddl_progress")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let ddl_progresses = reader.meta_client.list_ddl_progress().await?; + + let table_ids = ddl_progresses + .iter() + .map(|progress| progress.id as u32) + .collect_vec(); + + let tables = reader.meta_client.get_tables(&table_ids).await?; + + let ddl_progress = ddl_progresses + .into_iter() + .map(|s| { + let initialized_at = tables + .get(&(s.id as u32)) + .and_then(|table| table.initialized_at_epoch.map(Epoch::from)); + + RwDdlProgress { + ddl_id: s.id as i64, + ddl_statement: s.statement.clone(), + progress: s.progress.clone(), + initialized_at: initialized_at.map(|e| *e.as_scalar().as_timestamptz()), + } + }) + .collect(); + Ok(ddl_progress) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs index eb1e7d6fd6214..d99618e490822 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_description.rs @@ -15,70 +15,64 @@ use std::iter; use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_DESCRIPTION: BuiltinTable = BuiltinTable { - name: "rw_description", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - // table_id, view_id, function_id, etc. - (DataType::Int32, "objoid"), - // rw_tables, rw_views, rw_functions, etc. - (DataType::Int32, "classoid"), - // If `objoid` is `table_id`, then non-null `objsubid` is column number. - (DataType::Int32, "objsubid"), - (DataType::Varchar, "description"), - ], - pk: &[0, 1, 2], -}; +#[derive(Fields)] +#[primary_key(objoid, classoid, objsubid)] +struct RwDescription { + // table_id, view_id, function_id, etc. + objoid: i32, + // rw_tables, rw_views, rw_functions, etc. + classoid: i32, + // If `objoid` is `table_id`, then non-null `objsubid` is column number. + objsubid: Option, + description: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_description(&self) -> Result> { - let build_row = - |table_id, catalog_id, index: Option, description: Option>| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table_id)), - Some(ScalarImpl::Int32(catalog_id)), - index.map(ScalarImpl::Int32), - description.map(ScalarImpl::Utf8), - ]) - }; +#[system_catalog(table, "rw_catalog.rw_description")] +fn read(reader: &SysCatalogReaderImpl) -> Result> { + let build_row = + |table_id, catalog_id, index: Option, description: Option>| RwDescription { + objoid: table_id, + classoid: catalog_id, + objsubid: index, + description: description.map(|s| s.into()), + }; - let reader = self.catalog_reader.read_guard(); - let rw_catalog = - reader.get_schema_by_name(&self.auth_context.database, RW_CATALOG_SCHEMA_NAME)?; - let schemas = reader - .iter_schemas(&self.auth_context.database)? - .filter(|schema| schema.id() != rw_catalog.id()); + let catalog_reader = reader.catalog_reader.read_guard(); + let rw_catalog = + catalog_reader.get_schema_by_name(&reader.auth_context.database, RW_CATALOG_SCHEMA_NAME)?; + let schemas = catalog_reader + .iter_schemas(&reader.auth_context.database)? + .filter(|schema| schema.id() != rw_catalog.id()); - let rw_tables_id: i32 = rw_catalog - .get_system_table_by_name("rw_tables") - .map(|st| st.id.table_id) - .unwrap_or_default() as _; + let rw_tables_id: i32 = rw_catalog + .get_system_table_by_name("rw_tables") + .map(|st| st.id.table_id) + .unwrap_or_default() as _; - Ok(schemas - .flat_map(|schema| { - schema.iter_table().flat_map(|table| { - iter::once(build_row( + Ok(schemas + .flat_map(|schema| { + schema.iter_table().flat_map(|table| { + iter::once(build_row( + table.id.table_id as _, + rw_tables_id, + None, + table.description.as_deref().map(Into::into), + )) + .chain(table.columns.iter().map(|col| { + build_row( table.id.table_id as _, rw_tables_id, - None, - table.description.as_deref().map(Into::into), - )) - .chain(table.columns.iter().map(|col| { - build_row( - table.id.table_id as _, - rw_tables_id, - Some(col.column_id().get_id() as _), - col.column_desc.description.as_deref().map(Into::into), - ) - })) - }) + Some(col.column_id().get_id() as _), + col.column_desc.description.as_deref().map(Into::into), + ) + })) }) - .collect()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs index 9572ecfbc2617..bc83a5220757f 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_event_logs.rs @@ -13,49 +13,44 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl, Timestamptz}; +use risingwave_common::types::{Fields, JsonbVal, Timestamptz}; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::meta::event_log::Event; use serde_json::json; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_EVENT_LOGS: BuiltinTable = BuiltinTable { - name: "rw_event_logs", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "unique_id"), - (DataType::Timestamptz, "timestamp"), - (DataType::Varchar, "event_type"), - (DataType::Jsonb, "info"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwEventLog { + #[primary_key] + unique_id: String, + timestamp: Timestamptz, + event_type: String, + info: JsonbVal, +} -impl SysCatalogReaderImpl { - pub async fn read_event_logs(&self) -> Result> { - let configs = self - .meta_client - .list_event_log() - .await? - .into_iter() - .sorted_by(|a, b| a.timestamp.cmp(&b.timestamp)) - .map(|mut e| { - let id = e.unique_id.take().unwrap().into(); - let ts = Timestamptz::from_millis(e.timestamp.take().unwrap() as i64).unwrap(); - let event_type = event_type(e.event.as_ref().unwrap()); - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(id)), - Some(ScalarImpl::Timestamptz(ts)), - Some(ScalarImpl::Utf8(event_type.into())), - Some(ScalarImpl::Jsonb(json!(e).into())), - ]) - }) - .collect_vec(); - Ok(configs) - } +#[system_catalog(table, "rw_catalog.rw_event_logs")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let configs = reader + .meta_client + .list_event_log() + .await? + .into_iter() + .sorted_by(|a, b| a.timestamp.cmp(&b.timestamp)) + .map(|mut e| { + let id = e.unique_id.take().unwrap(); + let ts = Timestamptz::from_millis(e.timestamp.take().unwrap() as i64).unwrap(); + let event_type = event_type(e.event.as_ref().unwrap()); + RwEventLog { + unique_id: id, + timestamp: ts, + event_type: event_type.clone(), + info: json!(e).into(), + } + }) + .collect(); + Ok(configs) } fn event_type(e: &Event) -> String { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_parallelism.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_parallelism.rs index 9a0c50bedd37a..9a544ebaa8621 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_parallelism.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragment_parallelism.rs @@ -12,70 +12,44 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Copyright 2023 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. - -use std::sync::LazyLock; - -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{BuiltinView, SystemCatalogColumnsDef}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -pub static RW_FRAGMENT_PARALLELISM_COLUMNS: LazyLock>> = - LazyLock::new(|| { - vec![ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Varchar, "relation_type"), - (DataType::Int32, "fragment_id"), - (DataType::Varchar, "distribution_type"), - (DataType::List(Box::new(DataType::Int32)), "state_table_ids"), - ( - DataType::List(Box::new(DataType::Int32)), - "upstream_fragment_ids", - ), - (DataType::List(Box::new(DataType::Varchar)), "flags"), - (DataType::Int32, "parallelism"), - ] - }); -pub static RW_FRAGMENT_PARALLELISM: LazyLock = LazyLock::new(|| BuiltinView { - name: "rw_fragment_parallelism", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &RW_FRAGMENT_PARALLELISM_COLUMNS, - sql: "WITH all_streaming_jobs AS ( \ - SELECT id, name, 'table' as relation_type FROM rw_tables \ - UNION ALL \ - SELECT id, name, 'materialized view' as relation_type FROM rw_materialized_views \ - UNION ALL \ - SELECT id, name, 'sink' as relation_type FROM rw_sinks \ - UNION ALL \ - SELECT id, name, 'index' as relation_type FROM rw_indexes \ - ) \ - SELECT \ - job.id, \ - job.name, \ - job.relation_type, \ - f.fragment_id, \ - f.distribution_type, \ - f.state_table_ids, \ - f.upstream_fragment_ids, \ - f.flags, \ - f.parallelism \ - FROM all_streaming_jobs job \ - INNER JOIN rw_fragments f ON job.id = f.table_id \ - ORDER BY job.id\ - " - .to_string(), -}); +#[system_catalog( + view, + "rw_catalog.rw_fragment_parallelism", + "WITH all_streaming_jobs AS ( + SELECT id, name, 'table' as relation_type FROM rw_tables + UNION ALL + SELECT id, name, 'materialized view' as relation_type FROM rw_materialized_views + UNION ALL + SELECT id, name, 'sink' as relation_type FROM rw_sinks + UNION ALL + SELECT id, name, 'index' as relation_type FROM rw_indexes + ) + SELECT + job.id, + job.name, + job.relation_type, + f.fragment_id, + f.distribution_type, + f.state_table_ids, + f.upstream_fragment_ids, + f.flags, + f.parallelism + FROM all_streaming_jobs job + INNER JOIN rw_fragments f ON job.id = f.table_id + ORDER BY job.id" +)] +#[derive(Fields)] +struct RwEventLog { + id: i32, + name: String, + relation_type: String, + fragment_id: i32, + distribution_type: String, + state_table_ids: Vec, + upstream_fragment_ids: Vec, + flags: Vec, + parallelism: i32, +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs index d18bac5a9b1f4..10acbde7c7c8f 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_fragments.rs @@ -12,98 +12,79 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use itertools::Itertools; -use risingwave_common::array::ListValue; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::stream_plan::FragmentTypeFlag; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub static RW_FRAGMENTS_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Int32, "fragment_id"), - (DataType::Int32, "table_id"), - (DataType::Varchar, "distribution_type"), - (DataType::List(Box::new(DataType::Int32)), "state_table_ids"), - ( - DataType::List(Box::new(DataType::Int32)), - "upstream_fragment_ids", - ), - (DataType::List(Box::new(DataType::Varchar)), "flags"), - (DataType::Int32, "parallelism"), - ] -}); - -pub static RW_FRAGMENTS: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_fragments", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &RW_FRAGMENTS_COLUMNS, - pk: &[0], -}); +#[derive(Fields)] +struct RwFragment { + #[primary_key] + fragment_id: i32, + table_id: i32, + distribution_type: String, + state_table_ids: Vec, + upstream_fragment_ids: Vec, + flags: Vec, + parallelism: i32, +} -impl SysCatalogReaderImpl { - fn extract_fragment_type_flag(mask: u32) -> Vec { - let mut result = vec![]; - for i in 0..32 { - let bit = 1 << i; - if mask & bit != 0 { - match FragmentTypeFlag::try_from(bit as i32) { - Err(_) => continue, - Ok(flag) => result.push(flag), - }; - } +fn extract_fragment_type_flag(mask: u32) -> Vec { + let mut result = vec![]; + for i in 0..32 { + let bit = 1 << i; + if mask & bit != 0 { + match FragmentTypeFlag::try_from(bit as i32) { + Err(_) => continue, + Ok(flag) => result.push(flag), + }; } - result } + result +} - pub async fn read_rw_fragment_distributions_info(&self) -> Result> { - let distributions = self.meta_client.list_fragment_distribution().await?; +#[system_catalog(table, "rw_catalog.rw_fragments")] +async fn read_rw_fragment(reader: &SysCatalogReaderImpl) -> Result> { + let distributions = reader.meta_client.list_fragment_distribution().await?; - Ok(distributions - .into_iter() - .map(|distribution| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(distribution.fragment_id as i32)), - Some(ScalarImpl::Int32(distribution.table_id as i32)), - Some(ScalarImpl::Utf8( - distribution.distribution_type().as_str_name().into(), - )), - Some(ScalarImpl::List(ListValue::from_iter( - distribution.state_table_ids.into_iter().map(|id| id as i32), - ))), - Some(ScalarImpl::List(ListValue::from_iter( - distribution - .upstream_fragment_ids - .into_iter() - .map(|id| id as i32), - ))), - Some(ScalarImpl::List(ListValue::from_iter( - Self::extract_fragment_type_flag(distribution.fragment_type_mask) - .into_iter() - .flat_map(|t| t.as_str_name().strip_prefix("FRAGMENT_TYPE_FLAG_")), - ))), - Some(ScalarImpl::Int32(distribution.parallelism as i32)), - ]) - }) - .collect_vec()) - } + Ok(distributions + .into_iter() + .map(|distribution| RwFragment { + fragment_id: distribution.fragment_id as i32, + table_id: distribution.table_id as i32, + distribution_type: distribution.distribution_type().as_str_name().into(), + state_table_ids: distribution + .state_table_ids + .into_iter() + .map(|id| id as i32) + .collect(), + upstream_fragment_ids: distribution + .upstream_fragment_ids + .into_iter() + .map(|id| id as i32) + .collect(), + flags: extract_fragment_type_flag(distribution.fragment_type_mask) + .into_iter() + .flat_map(|t| t.as_str_name().strip_prefix("FRAGMENT_TYPE_FLAG_")) + .map(|s| s.into()) + .collect(), + parallelism: distribution.parallelism as i32, + }) + .collect()) } #[cfg(test)] mod tests { use risingwave_pb::stream_plan::FragmentTypeFlag; - use crate::catalog::system_catalog::SysCatalogReaderImpl; + use super::extract_fragment_type_flag; #[test] fn test_extract_mask() { let mask = (FragmentTypeFlag::Source as u32) | (FragmentTypeFlag::StreamScan as u32); - let result = SysCatalogReaderImpl::extract_fragment_type_flag(mask); + let result = extract_fragment_type_flag(mask); assert_eq!(result.len(), 2); assert!(result.contains(&FragmentTypeFlag::Source)); assert!(result.contains(&FragmentTypeFlag::StreamScan)) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs index d31784d55989d..6107dc8c66d39 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_functions.rs @@ -12,77 +12,55 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use itertools::Itertools; -use risingwave_common::array::ListValue; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{ - get_acl_items, BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef, -}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; -pub static RW_FUNCTIONS_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "type"), - (DataType::List(Box::new(DataType::Int32)), "arg_type_ids"), - (DataType::Int32, "return_type_id"), - (DataType::Varchar, "language"), - (DataType::Varchar, "link"), - (DataType::Varchar, "acl"), - ] -}); - -pub static RW_FUNCTIONS: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_functions", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &RW_FUNCTIONS_COLUMNS, - pk: &[0], -}); +#[derive(Fields)] +struct RwFunction { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + r#type: String, + arg_type_ids: Vec, + return_type_id: i32, + language: String, + link: Option, + acl: String, +} -impl SysCatalogReaderImpl { - pub fn read_rw_functions_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_functions")] +fn read(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema.iter_function().map(|function| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(function.id.function_id() as i32)), - Some(ScalarImpl::Utf8(function.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(function.owner as i32)), - Some(ScalarImpl::Utf8(function.kind.to_string().into())), - Some(ScalarImpl::List(ListValue::from_iter( - function.arg_types.iter().map(|t| t.to_oid()), - ))), - Some(ScalarImpl::Int32(function.return_type.to_oid())), - Some(ScalarImpl::Utf8(function.language.clone().into())), - function.link.clone().map(|s| ScalarImpl::Utf8(s.into())), - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::FunctionId(function.id.function_id()), - false, - &users, - username_map, - ) - .into(), - )), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_function().map(|function| RwFunction { + id: function.id.function_id() as i32, + name: function.name.clone(), + schema_id: schema.id() as i32, + owner: function.owner as i32, + r#type: function.kind.to_string(), + arg_type_ids: function.arg_types.iter().map(|t| t.to_oid()).collect(), + return_type_id: function.return_type.to_oid(), + language: function.language.clone(), + link: function.link.clone(), + acl: get_acl_items( + &Object::FunctionId(function.id.function_id()), + false, + &users, + username_map, + ), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs index c63c5c6872faa..2699503a2fdd5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_branched_objects.rs @@ -12,37 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_BRANCHED_OBJECTS: BuiltinTable = BuiltinTable { - name: "rw_hummock_branched_objects", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "object_id"), - (DataType::Int64, "sst_id"), - (DataType::Int64, "compaction_group_id"), - ], - pk: &[], -}; +#[derive(Fields)] +struct RwHummockBranchedObject { + object_id: i64, + sst_id: i64, + compaction_group_id: i64, +} -impl SysCatalogReaderImpl { - pub async fn read_hummock_branched_objects(&self) -> Result> { - let branched_objects = self.meta_client.list_branched_objects().await?; - let rows = branched_objects - .into_iter() - .map(|o| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(o.object_id as _)), - Some(ScalarImpl::Int64(o.sst_id as _)), - Some(ScalarImpl::Int64(o.compaction_group_id as _)), - ]) - }) - .collect(); - Ok(rows) - } +#[system_catalog(table, "rw_catalog.rw_hummock_branched_objects")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let branched_objects = reader.meta_client.list_branched_objects().await?; + let rows = branched_objects + .into_iter() + .map(|o| RwHummockBranchedObject { + object_id: o.object_id as _, + sst_id: o.sst_id as _, + compaction_group_id: o.compaction_group_id as _, + }) + .collect(); + Ok(rows) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs index 47cea4b382816..e37dcf29a308e 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_assignment.rs @@ -12,73 +12,58 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; -use risingwave_pb::hummock::CompactTaskAssignment; +use risingwave_common::types::{Fields, JsonbVal}; +use risingwave_frontend_macro::system_catalog; use serde_json::json; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_COMPACT_TASK_ASSIGNMENT: BuiltinTable = BuiltinTable { - name: "rw_hummock_compact_task_assignment", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "compaction_group_id"), - (DataType::Int64, "task_id"), - (DataType::Int32, "select_level"), - (DataType::Int32, "target_level"), - (DataType::Int32, "task_type"), - (DataType::Int32, "task_status"), - (DataType::Int64, "watermark"), - (DataType::Int32, "base_level"), - (DataType::Boolean, "gc_delete_keys"), - (DataType::Int64, "target_file_size"), - (DataType::Int64, "target_sub_level_id"), - (DataType::Int32, "compression_algorithm"), - (DataType::Int32, "task_type"), - (DataType::Jsonb, "table_ids"), - ], - pk: &[0], -}; - -impl SysCatalogReaderImpl { - pub async fn read_hummock_compact_task_assignments(&self) -> Result> { - // The naming of compact_task_assignment is due to versioning; now compact_task_assignment only records the state of the compact task - let compact_task_assignments = self.meta_client.list_compact_task_assignment().await?; - Ok(compact_task_assignments_to_rows(compact_task_assignments)) - } +#[derive(Fields)] +struct RwHummockCompactTaskAssignment { + #[primary_key] + compaction_group_id: i64, + task_id: i64, + select_level: i32, + target_level: i32, + task_type: i32, + task_status: i32, + watermark: i64, + base_level: i32, + gc_delete_keys: bool, + target_file_size: i64, + target_sub_level_id: i64, + compression_algorithm: i32, + table_ids: JsonbVal, } -fn compact_task_assignments_to_rows( - compact_task_assignments: Vec, -) -> Vec { +#[system_catalog(table, "rw_catalog.rw_hummock_compact_task_assignment")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + // The naming of compact_task_assignment is due to versioning; now compact_task_assignment only records the state of the compact task + let compact_task_assignments = reader.meta_client.list_compact_task_assignment().await?; + let mut rows = vec![]; for compact_task_assignment in compact_task_assignments { let compact_task = compact_task_assignment.compact_task.unwrap(); let select_level = compact_task.input_ssts[0].level_idx; - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Int64(compact_task.compaction_group_id as _)), - Some(ScalarImpl::Int64(compact_task.task_id as _)), - Some(ScalarImpl::Int32(select_level as _)), - Some(ScalarImpl::Int32(compact_task.target_level as _)), - Some(ScalarImpl::Int32(compact_task.task_type as _)), - Some(ScalarImpl::Int32(compact_task.task_status as _)), - Some(ScalarImpl::Int64(compact_task.watermark as _)), - Some(ScalarImpl::Int32(compact_task.base_level as _)), - Some(ScalarImpl::Bool(compact_task.gc_delete_keys as _)), - Some(ScalarImpl::Int64(compact_task.target_file_size as _)), - Some(ScalarImpl::Int64(compact_task.target_sub_level_id as _)), - Some(ScalarImpl::Int32(compact_task.compression_algorithm as _)), - Some(ScalarImpl::Int32(compact_task.task_type as _)), - Some(ScalarImpl::Jsonb( - json!(compact_task.existing_table_ids).into(), - )), - ])); + rows.push(RwHummockCompactTaskAssignment { + compaction_group_id: compact_task.compaction_group_id as _, + task_id: compact_task.task_id as _, + select_level: select_level as _, + target_level: compact_task.target_level as _, + task_type: compact_task.task_type as _, + task_status: compact_task.task_status as _, + watermark: compact_task.watermark as _, + base_level: compact_task.base_level as _, + gc_delete_keys: compact_task.gc_delete_keys as _, + target_file_size: compact_task.target_file_size as _, + target_sub_level_id: compact_task.target_sub_level_id as _, + compression_algorithm: compact_task.compression_algorithm as _, + table_ids: json!(compact_task.existing_table_ids).into(), + }); } - rows + Ok(rows) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs index 0f16d9f3b7ab9..60eee538ae7dc 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compact_task_progress.rs @@ -12,65 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; -use risingwave_pb::hummock::CompactTaskProgress; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_COMPACT_TASK_PROGRESS: BuiltinTable = BuiltinTable { - name: "rw_hummock_compact_task_progress", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "compaction_group_id"), - (DataType::Int64, "task_id"), - (DataType::Int32, "num_ssts_sealed"), - (DataType::Int32, "num_ssts_uploaded"), - (DataType::Int64, "num_progress_key"), - (DataType::Int64, "num_pending_read_io"), - (DataType::Int64, "num_pending_write_io"), - ], - pk: &[0], -}; - -impl SysCatalogReaderImpl { - pub async fn read_hummock_compact_task_progress(&self) -> Result> { - let compact_task_progress = self.meta_client.list_compact_task_progress().await?; - Ok(compact_task_progress_to_rows(compact_task_progress)) - } +#[derive(Fields)] +struct RwHummockCompactTaskProgress { + #[primary_key] + compaction_group_id: i64, + task_id: i64, + num_ssts_sealed: i32, + num_ssts_uploaded: i32, + num_progress_key: i64, + num_pending_read_io: i64, + num_pending_write_io: i64, } -fn compact_task_progress_to_rows( - compact_task_progress_vec: Vec, -) -> Vec { +#[system_catalog(table, "rw_catalog.rw_hummock_compact_task_progress")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let compact_task_progress = reader.meta_client.list_compact_task_progress().await?; + let mut rows = vec![]; - for compact_task_progress in compact_task_progress_vec { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Int64( - compact_task_progress - .compaction_group_id - .unwrap_or_default() as _, - )), - Some(ScalarImpl::Int64(compact_task_progress.task_id as _)), - Some(ScalarImpl::Int32( - compact_task_progress.num_ssts_sealed as _, - )), - Some(ScalarImpl::Int32( - compact_task_progress.num_ssts_uploaded as _, - )), - Some(ScalarImpl::Int64( - compact_task_progress.num_progress_key as _, - )), - Some(ScalarImpl::Int64( - compact_task_progress.num_pending_read_io as _, - )), - Some(ScalarImpl::Int64( - compact_task_progress.num_pending_write_io as _, - )), - ])); + for p in compact_task_progress { + rows.push(RwHummockCompactTaskProgress { + compaction_group_id: p.compaction_group_id.unwrap_or_default() as _, + task_id: p.task_id as _, + num_ssts_sealed: p.num_ssts_sealed as _, + num_ssts_uploaded: p.num_ssts_uploaded as _, + num_progress_key: p.num_progress_key as _, + num_pending_read_io: p.num_pending_read_io as _, + num_pending_write_io: p.num_pending_write_io as _, + }); } - - rows + Ok(rows) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs index 2ffad904c9eca..ff4d8c9e34d42 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_compaction_group_configs.rs @@ -13,60 +13,55 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, JsonbVal}; +use risingwave_frontend_macro::system_catalog; use serde_json::json; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_COMPACTION_GROUP_CONFIGS: BuiltinTable = BuiltinTable { - name: "rw_hummock_compaction_group_configs", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "id"), - (DataType::Int64, "parent_id"), - (DataType::Jsonb, "member_tables"), - (DataType::Jsonb, "compaction_config"), - (DataType::Jsonb, "active_write_limit"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwHummockCompactionGroupConfig { + #[primary_key] + id: i64, + parent_id: Option, + member_tables: Option, + compaction_config: Option, + active_write_limit: Option, +} -impl SysCatalogReaderImpl { - pub async fn read_hummock_compaction_group_configs(&self) -> Result> { - let info = self - .meta_client - .list_hummock_compaction_group_configs() - .await?; - let mut write_limits = self.meta_client.list_hummock_active_write_limits().await?; - let mut rows = info +#[system_catalog(table, "rw_catalog.rw_hummock_compaction_group_configs")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let info = reader + .meta_client + .list_hummock_compaction_group_configs() + .await?; + let mut write_limits = reader + .meta_client + .list_hummock_active_write_limits() + .await?; + let mut rows = info + .into_iter() + .map(|i| RwHummockCompactionGroupConfig { + id: i.id as _, + parent_id: Some(i.parent_id as _), + member_tables: Some(json!(i.member_table_ids).into()), + compaction_config: Some(json!(i.compaction_config).into()), + active_write_limit: write_limits.remove(&i.id).map(|w| json!(w).into()), + }) + .collect_vec(); + // As compaction group configs and active write limits are fetched via two RPCs, it's possible there's inconsistency. + // Just leave unknown field blank. + rows.extend( + write_limits .into_iter() - .map(|i| { - let active_write_limit = write_limits - .remove(&i.id) - .map(|w| ScalarImpl::Jsonb(json!(w).into())); - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(i.id as _)), - Some(ScalarImpl::Int64(i.parent_id as _)), - Some(ScalarImpl::Jsonb(json!(i.member_table_ids).into())), - Some(ScalarImpl::Jsonb(json!(i.compaction_config).into())), - active_write_limit, - ]) - }) - .collect_vec(); - // As compaction group configs and active write limits are fetched via two RPCs, it's possible there's inconsistency. - // Just leave unknown field blank. - rows.extend(write_limits.into_iter().map(|(cg, w)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(cg as _)), - None, - None, - None, - Some(ScalarImpl::Jsonb(json!(w).into())), - ]) - })); - Ok(rows) - } + .map(|(cg, w)| RwHummockCompactionGroupConfig { + id: cg as _, + parent_id: None, + member_tables: None, + compaction_config: None, + active_write_limit: Some(json!(w).into()), + }), + ); + Ok(rows) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs index 9bf0303e2e8ca..40569e84d918e 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_meta_configs.rs @@ -13,38 +13,31 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_META_CONFIGS: BuiltinTable = BuiltinTable { - name: "rw_hummock_meta_configs", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "config_name"), - (DataType::Varchar, "config_value"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwHummockMetaConfig { + #[primary_key] + config_name: String, + config_value: String, +} -impl SysCatalogReaderImpl { - pub async fn read_hummock_meta_configs(&self) -> Result> { - let configs = self - .meta_client - .list_hummock_meta_configs() - .await? - .into_iter() - .sorted() - .map(|(k, v)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(k.into())), - Some(ScalarImpl::Utf8(v.into())), - ]) - }) - .collect_vec(); - Ok(configs) - } +#[system_catalog(table, "rw_catalog.rw_hummock_meta_configs")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let configs = reader + .meta_client + .list_hummock_meta_configs() + .await? + .into_iter() + .sorted() + .map(|(k, v)| RwHummockMetaConfig { + config_name: k, + config_value: v, + }) + .collect(); + Ok(configs) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs index 48a22923087a6..ac2b96bdc0023 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_snapshots.rs @@ -12,38 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_PINNED_SNAPSHOTS: BuiltinTable = BuiltinTable { - name: "rw_hummock_pinned_snapshots", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "worker_node_id"), - (DataType::Int64, "min_pinned_snapshot_id"), - ], - pk: &[], -}; +#[derive(Fields)] +struct RwHummockPinnedSnapshot { + worker_node_id: i32, + min_pinned_snapshot_id: i64, +} -impl SysCatalogReaderImpl { - pub async fn read_hummock_pinned_snapshots(&self) -> Result> { - let pinned_snapshots = self - .meta_client - .list_hummock_pinned_snapshots() - .await? - .into_iter() - .map(|s| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(s.0 as i32)), - Some(ScalarImpl::Int64(s.1 as i64)), - ]) - }) - .collect_vec(); - Ok(pinned_snapshots) - } +#[system_catalog(table, "rw_catalog.rw_hummock_pinned_snapshots")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let pinned_snapshots = reader + .meta_client + .list_hummock_pinned_snapshots() + .await? + .into_iter() + .map(|s| RwHummockPinnedSnapshot { + worker_node_id: s.0 as _, + min_pinned_snapshot_id: s.1 as _, + }) + .collect(); + Ok(pinned_snapshots) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs index 07f0d786f4206..45a8e23f0ecc5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_pinned_versions.rs @@ -12,38 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_PINNED_VERSIONS: BuiltinTable = BuiltinTable { - name: "rw_hummock_pinned_versions", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "worker_node_id"), - (DataType::Int64, "min_pinned_version_id"), - ], - pk: &[], -}; +#[derive(Fields)] +struct RwHummockPinnedVersion { + worker_node_id: i32, + min_pinned_version_id: i64, +} -impl SysCatalogReaderImpl { - pub async fn read_hummock_pinned_versions(&self) -> Result> { - let pinned_versions = self - .meta_client - .list_hummock_pinned_versions() - .await? - .into_iter() - .map(|s| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(s.0 as i32)), - Some(ScalarImpl::Int64(s.1 as i64)), - ]) - }) - .collect_vec(); - Ok(pinned_versions) - } +#[system_catalog(table, "rw_catalog.rw_hummock_pinned_versions")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let pinned_versions = reader + .meta_client + .list_hummock_pinned_versions() + .await? + .into_iter() + .map(|s| RwHummockPinnedVersion { + worker_node_id: s.0 as _, + min_pinned_version_id: s.1 as _, + }) + .collect(); + Ok(pinned_versions) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs index 6b0e68d7590b5..5551170e57a6f 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs @@ -12,85 +12,70 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, JsonbVal}; +use risingwave_frontend_macro::system_catalog; use risingwave_hummock_sdk::version::HummockVersion; use serde_json::json; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_CURRENT_VERSION: BuiltinTable = BuiltinTable { - name: "rw_hummock_current_version", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "version_id"), - (DataType::Int64, "max_committed_epoch"), - (DataType::Int64, "safe_epoch"), - (DataType::Jsonb, "compaction_group"), - ], - pk: &[], -}; - -pub const RW_HUMMOCK_CHECKPOINT_VERSION: BuiltinTable = BuiltinTable { - name: "rw_hummock_checkpoint_version", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "version_id"), - (DataType::Int64, "max_committed_epoch"), - (DataType::Int64, "safe_epoch"), - (DataType::Jsonb, "compaction_group"), - ], - pk: &[], -}; +#[derive(Fields)] +struct RwHummockVersion { + version_id: i64, + max_committed_epoch: i64, + safe_epoch: i64, + compaction_group: JsonbVal, +} -pub const RW_HUMMOCK_SSTABLES: BuiltinTable = BuiltinTable { - name: "rw_hummock_sstables", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "sstable_id"), - (DataType::Int64, "object_id"), - (DataType::Int64, "compaction_group_id"), - (DataType::Int32, "level_id"), - (DataType::Int64, "sub_level_id"), - (DataType::Int32, "level_type"), - (DataType::Bytea, "key_range_left"), - (DataType::Bytea, "key_range_right"), - (DataType::Boolean, "right_exclusive"), - (DataType::Int64, "file_size"), - (DataType::Int64, "meta_offset"), - (DataType::Int64, "stale_key_count"), - (DataType::Int64, "total_key_count"), - (DataType::Int64, "min_epoch"), - (DataType::Int64, "max_epoch"), - (DataType::Int64, "uncompressed_file_size"), - (DataType::Int64, "range_tombstone_count"), - (DataType::Int32, "bloom_filter_kind"), - (DataType::Jsonb, "table_ids"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwHummockSstable { + #[primary_key] + sstable_id: i64, + object_id: i64, + compaction_group_id: i64, + level_id: i32, + sub_level_id: Option, + level_type: i32, + key_range_left: Vec, + key_range_right: Vec, + right_exclusive: bool, + file_size: i64, + meta_offset: i64, + stale_key_count: i64, + total_key_count: i64, + min_epoch: i64, + max_epoch: i64, + uncompressed_file_size: i64, + range_tombstone_count: i64, + bloom_filter_kind: i32, + table_ids: JsonbVal, +} -impl SysCatalogReaderImpl { - pub async fn read_hummock_current_version(&self) -> Result> { - let version = self.meta_client.get_hummock_current_version().await?; - Ok(version_to_compaction_group_rows( - &remove_key_range_from_version(version), - )) - } +#[system_catalog(table, "rw_catalog.rw_hummock_current_version")] +async fn read_hummock_current_version( + reader: &SysCatalogReaderImpl, +) -> Result> { + let version = reader.meta_client.get_hummock_current_version().await?; + Ok(version_to_compaction_group_rows( + &remove_key_range_from_version(version), + )) +} - pub async fn read_hummock_checkpoint_version(&self) -> Result> { - let version = self.meta_client.get_hummock_checkpoint_version().await?; - Ok(version_to_compaction_group_rows( - &remove_key_range_from_version(version), - )) - } +#[system_catalog(table, "rw_catalog.rw_hummock_checkpoint_version")] +async fn read_hummock_checkpoint_version( + reader: &SysCatalogReaderImpl, +) -> Result> { + let version = reader.meta_client.get_hummock_checkpoint_version().await?; + Ok(version_to_compaction_group_rows( + &remove_key_range_from_version(version), + )) +} - pub async fn read_hummock_sstables(&self) -> Result> { - let version = self.meta_client.get_hummock_current_version().await?; - Ok(version_to_sstable_rows(version)) - } +#[system_catalog(table, "rw_catalog.rw_hummock_sstables")] +async fn read_hummock_sstables(reader: &SysCatalogReaderImpl) -> Result> { + let version = reader.meta_client.get_hummock_current_version().await?; + Ok(version_to_sstable_rows(version)) } fn remove_key_range_from_version(mut version: HummockVersion) -> HummockVersion { @@ -109,53 +94,46 @@ fn remove_key_range_from_version(mut version: HummockVersion) -> HummockVersion version } -fn version_to_compaction_group_rows(version: &HummockVersion) -> Vec { +fn version_to_compaction_group_rows(version: &HummockVersion) -> Vec { version .levels .values() - .map(|cg| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(version.id as _)), - Some(ScalarImpl::Int64(version.max_committed_epoch as _)), - Some(ScalarImpl::Int64(version.safe_epoch as _)), - Some(ScalarImpl::Jsonb(json!(cg).into())), - ]) + .map(|cg| RwHummockVersion { + version_id: version.id as _, + max_committed_epoch: version.max_committed_epoch as _, + safe_epoch: version.safe_epoch as _, + compaction_group: json!(cg).into(), }) .collect() } -fn version_to_sstable_rows(version: HummockVersion) -> Vec { +fn version_to_sstable_rows(version: HummockVersion) -> Vec { let mut sstables = vec![]; for cg in version.levels.into_values() { for level in cg.levels.into_iter().chain(cg.l0.unwrap().sub_levels) { for sst in level.table_infos { let key_range = sst.key_range.unwrap(); - let sub_level_id = if level.level_idx > 0 { - None - } else { - Some(ScalarImpl::Int64(level.sub_level_id as _)) - }; - sstables.push(OwnedRow::new(vec![ - Some(ScalarImpl::Int64(sst.sst_id as _)), - Some(ScalarImpl::Int64(sst.object_id as _)), - Some(ScalarImpl::Int64(cg.group_id as _)), - Some(ScalarImpl::Int32(level.level_idx as _)), - sub_level_id, - Some(ScalarImpl::Int32(level.level_type as _)), - Some(ScalarImpl::Bytea(key_range.left.into())), - Some(ScalarImpl::Bytea(key_range.right.into())), - Some(ScalarImpl::Bool(key_range.right_exclusive)), - Some(ScalarImpl::Int64(sst.file_size as _)), - Some(ScalarImpl::Int64(sst.meta_offset as _)), - Some(ScalarImpl::Int64(sst.stale_key_count as _)), - Some(ScalarImpl::Int64(sst.total_key_count as _)), - Some(ScalarImpl::Int64(sst.min_epoch as _)), - Some(ScalarImpl::Int64(sst.max_epoch as _)), - Some(ScalarImpl::Int64(sst.uncompressed_file_size as _)), - Some(ScalarImpl::Int64(sst.range_tombstone_count as _)), - Some(ScalarImpl::Int32(sst.bloom_filter_kind as _)), - Some(ScalarImpl::Jsonb(json!(sst.table_ids).into())), - ])); + sstables.push(RwHummockSstable { + sstable_id: sst.sst_id as _, + object_id: sst.object_id as _, + compaction_group_id: cg.group_id as _, + level_id: level.level_idx as _, + sub_level_id: (level.level_idx > 0).then_some(level.sub_level_id as _), + level_type: level.level_type as _, + key_range_left: key_range.left, + key_range_right: key_range.right, + right_exclusive: key_range.right_exclusive, + file_size: sst.file_size as _, + meta_offset: sst.meta_offset as _, + stale_key_count: sst.stale_key_count as _, + total_key_count: sst.total_key_count as _, + min_epoch: sst.min_epoch as _, + max_epoch: sst.max_epoch as _, + uncompressed_file_size: sst.uncompressed_file_size as _, + range_tombstone_count: sst.range_tombstone_count as _, + bloom_filter_kind: sst.bloom_filter_kind as _, + table_ids: json!(sst.table_ids).into(), + }); } } } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs index 7e99e4040f981..052cb5fae170e 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs @@ -12,46 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, JsonbVal}; +use risingwave_frontend_macro::system_catalog; use serde_json::json; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_HUMMOCK_VERSION_DELTAS: BuiltinTable = BuiltinTable { - name: "rw_hummock_version_deltas", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "id"), - (DataType::Int64, "prev_id"), - (DataType::Int64, "max_committed_epoch"), - (DataType::Int64, "safe_epoch"), - (DataType::Boolean, "trivial_move"), - (DataType::Jsonb, "gc_object_ids"), - (DataType::Jsonb, "group_deltas"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwHummockVersionDelta { + #[primary_key] + id: i64, + prev_id: i64, + max_committed_epoch: i64, + safe_epoch: i64, + trivial_move: bool, + gc_object_ids: JsonbVal, + group_deltas: JsonbVal, +} -impl SysCatalogReaderImpl { - pub async fn read_hummock_version_deltas(&self) -> Result> { - let deltas = self.meta_client.list_version_deltas().await?; - let rows = deltas - .into_iter() - .map(|d| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(d.id as _)), - Some(ScalarImpl::Int64(d.prev_id as _)), - Some(ScalarImpl::Int64(d.max_committed_epoch as _)), - Some(ScalarImpl::Int64(d.safe_epoch as _)), - Some(ScalarImpl::Bool(d.trivial_move)), - Some(ScalarImpl::Jsonb(json!(d.gc_object_ids).into())), - Some(ScalarImpl::Jsonb(json!(d.group_deltas).into())), - ]) - }) - .collect(); - Ok(rows) - } +#[system_catalog(table, "rw_catalog.rw_hummock_version_deltas")] +async fn read(reader: &SysCatalogReaderImpl) -> Result> { + let deltas = reader.meta_client.list_version_deltas().await?; + let rows = deltas + .into_iter() + .map(|d| RwHummockVersionDelta { + id: d.id as _, + prev_id: d.prev_id as _, + max_committed_epoch: d.max_committed_epoch as _, + safe_epoch: d.safe_epoch as _, + trivial_move: d.trivial_move, + gc_object_ids: json!(d.gc_object_ids).into(), + group_deltas: json!(d.group_deltas).into(), + }) + .collect(); + Ok(rows) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs index 7c82b0aa86df3..a383219421583 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs @@ -12,84 +12,62 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; -use itertools::Itertools; -use risingwave_common::array::ListValue; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; - -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub static RW_INDEXES_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "primary_table_id"), - (DataType::List(Box::new(DataType::Int16)), "indkey"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - (DataType::Timestamptz, "initialized_at"), - (DataType::Timestamptz, "created_at"), - (DataType::Varchar, "initialized_at_cluster_version"), - (DataType::Varchar, "created_at_cluster_version"), - ] -}); - -pub static RW_INDEXES: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_indexes", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &RW_INDEXES_COLUMNS, - pk: &[0], -}); +#[derive(Fields)] +struct RwIndex { + #[primary_key] + id: i32, + name: String, + primary_table_id: i32, + indkey: Vec, + schema_id: i32, + owner: i32, + definition: String, + acl: String, + initialized_at: Option, + created_at: Option, + initialized_at_cluster_version: Option, + created_at_cluster_version: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_indexes_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; +#[system_catalog(table, "rw_catalog.rw_indexes")] +fn read_rw_indexes(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; - Ok(schemas - .flat_map(|schema| { - schema.iter_index().map(|index| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(index.id.index_id as i32)), - Some(ScalarImpl::Utf8(index.name.clone().into())), - Some(ScalarImpl::Int32(index.primary_table.id().table_id as i32)), - Some(ScalarImpl::List(ListValue::from_iter( - index - .index_item - .iter() - .take(index.index_columns_len as usize) - .map(|index| { - let ind = if let Some(input_ref) = index.as_input_ref() { - input_ref.index() + 1 - } else { - 0 - }; - ind as i16 - }), - ))), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(index.index_table.owner as i32)), - Some(ScalarImpl::Utf8(index.index_table.create_sql().into())), - Some(ScalarImpl::Utf8("".into())), - index.initialized_at_epoch.map(|e| e.as_scalar()), - index.created_at_epoch.map(|e| e.as_scalar()), - index - .initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - index - .created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_index().map(|index| RwIndex { + id: index.id.index_id as i32, + name: index.name.clone(), + primary_table_id: index.primary_table.id().table_id as i32, + indkey: index + .index_item + .iter() + .take(index.index_columns_len as usize) + .map(|index| { + let ind = if let Some(input_ref) = index.as_input_ref() { + input_ref.index() + 1 + } else { + 0 + }; + ind as i16 + }) + .collect(), + schema_id: schema.id() as i32, + owner: index.index_table.owner as i32, + definition: index.index_table.create_sql(), + acl: "".into(), + initialized_at: index.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: index.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: index.initialized_at_cluster_version.clone(), + created_at_cluster_version: index.created_at_cluster_version.clone(), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs index d7a9412a5943d..9ea91bfa50731 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_internal_tables.rs @@ -12,72 +12,55 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; -pub const RW_INTERNAL_TABLES: BuiltinTable = BuiltinTable { - name: "rw_internal_tables", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - (DataType::Timestamptz, "initialized_at"), - (DataType::Timestamptz, "created_at"), - (DataType::Varchar, "initialized_at_cluster_version"), - (DataType::Varchar, "created_at_cluster_version"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwInternalTable { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + definition: String, + acl: String, + initialized_at: Option, + created_at: Option, + initialized_at_cluster_version: Option, + created_at_cluster_version: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_internal_table_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_internal_tables")] +fn read_rw_internal_tables(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema.iter_internal_table().map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8(table.create_sql().into())), - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::TableId(table.id.table_id), - true, - &users, - username_map, - ) - .into(), - )), - table.initialized_at_epoch.map(|e| e.as_scalar()), - table.created_at_epoch.map(|e| e.as_scalar()), - table - .initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - table - .created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_internal_table().map(|table| RwInternalTable { + id: table.id.table_id as i32, + name: table.name().into(), + schema_id: schema.id() as i32, + owner: table.owner as i32, + definition: table.create_sql(), + acl: get_acl_items( + &Object::TableId(table.id.table_id), + true, + &users, + username_map, + ), + initialized_at: table.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: table.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: table.initialized_at_cluster_version.clone(), + created_at_cluster_version: table.created_at_cluster_version.clone(), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs index 9b9b7de05a844..16a1d7cf19d21 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_materialized_views.rs @@ -12,72 +12,55 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; -pub const RW_MATERIALIZED_VIEWS: BuiltinTable = BuiltinTable { - name: "rw_materialized_views", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - (DataType::Timestamptz, "initialized_at"), - (DataType::Timestamptz, "created_at"), - (DataType::Varchar, "initialized_at_cluster_version"), - (DataType::Varchar, "created_at_cluster_version"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwMaterializedView { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + definition: String, + acl: String, + initialized_at: Option, + created_at: Option, + initialized_at_cluster_version: Option, + created_at_cluster_version: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_mview_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_materialized_views")] +fn read_rw_materialized_views(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema.iter_mv().map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8(table.create_sql().into())), - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::TableId(table.id.table_id), - false, - &users, - username_map, - ) - .into(), - )), - table.initialized_at_epoch.map(|e| e.as_scalar()), - table.created_at_epoch.map(|e| e.as_scalar()), - table - .initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - table - .created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_mv().map(|table| RwMaterializedView { + id: table.id.table_id as i32, + name: table.name().into(), + schema_id: schema.id() as i32, + owner: table.owner as i32, + definition: table.create_sql(), + acl: get_acl_items( + &Object::TableId(table.id.table_id), + true, + &users, + username_map, + ), + initialized_at: table.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: table.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: table.initialized_at_cluster_version.clone(), + created_at_cluster_version: table.created_at_cluster_version.clone(), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs index e79e0870fc673..ebb969cac462f 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_meta_snapshot.rs @@ -12,65 +12,55 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl, Timestamp}; +use risingwave_common::types::{Fields, Timestamp}; use risingwave_common::util::epoch::Epoch; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_META_SNAPSHOT: BuiltinTable = BuiltinTable { - name: "rw_meta_snapshot", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int64, "meta_snapshot_id"), - (DataType::Int64, "hummock_version_id"), - // the smallest epoch this meta snapshot includes - (DataType::Int64, "safe_epoch"), - // human-readable timestamp of safe_epoch - (DataType::Timestamp, "safe_epoch_ts"), - // the largest epoch this meta snapshot includes - (DataType::Int64, "max_committed_epoch"), - // human-readable timestamp of max_committed_epoch - (DataType::Timestamp, "max_committed_epoch_ts"), - (DataType::Varchar, "remarks"), - ], - pk: &[], -}; +#[derive(Fields)] +struct RwMetaSnapshot { + meta_snapshot_id: i64, + hummock_version_id: i64, + // the smallest epoch this meta snapshot includes + safe_epoch: i64, + // human-readable timestamp of safe_epoch + safe_epoch_ts: Option, + // the largest epoch this meta snapshot includes + max_committed_epoch: i64, + // human-readable timestamp of max_committed_epoch + max_committed_epoch_ts: Option, + remarks: Option, +} -impl SysCatalogReaderImpl { - pub async fn read_meta_snapshot(&self) -> Result> { - let try_get_date_time = |epoch: u64| { - if epoch == 0 { - return None; - } - let time_millis = Epoch::from(epoch).as_unix_millis(); - Timestamp::with_secs_nsecs( - (time_millis / 1000) as i64, - (time_millis % 1000 * 1_000_000) as u32, - ) - .map(ScalarImpl::Timestamp) - .ok() - }; - let meta_snapshots = self - .meta_client - .list_meta_snapshots() - .await? - .into_iter() - .map(|s| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int64(s.id as i64)), - Some(ScalarImpl::Int64(s.hummock_version_id as i64)), - Some(ScalarImpl::Int64(s.safe_epoch as i64)), - try_get_date_time(s.safe_epoch), - Some(ScalarImpl::Int64(s.max_committed_epoch as i64)), - try_get_date_time(s.max_committed_epoch), - s.remarks.map(|r| ScalarImpl::Utf8(r.into())), - ]) - }) - .collect_vec(); - Ok(meta_snapshots) - } +#[system_catalog(table, "rw_catalog.rw_meta_snapshot")] +async fn read_meta_snapshot(reader: &SysCatalogReaderImpl) -> Result> { + let try_get_date_time = |epoch: u64| { + if epoch == 0 { + return None; + } + let time_millis = Epoch::from(epoch).as_unix_millis(); + Timestamp::with_secs_nsecs( + (time_millis / 1000) as i64, + (time_millis % 1000 * 1_000_000) as u32, + ) + .ok() + }; + let meta_snapshots = reader + .meta_client + .list_meta_snapshots() + .await? + .into_iter() + .map(|s| RwMetaSnapshot { + meta_snapshot_id: s.id as _, + hummock_version_id: s.hummock_version_id as _, + safe_epoch: s.safe_epoch as _, + safe_epoch_ts: try_get_date_time(s.safe_epoch), + max_committed_epoch: s.max_committed_epoch as _, + max_committed_epoch_ts: try_get_date_time(s.max_committed_epoch), + remarks: s.remarks, + }) + .collect(); + Ok(meta_snapshots) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs index d883c45d83a77..0a46e10ec1754 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs @@ -12,35 +12,33 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_PARALLEL_UNITS: BuiltinTable = BuiltinTable { - name: "rw_parallel_units", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[(DataType::Int32, "id"), (DataType::Int32, "worker_id")], - pk: &[0], -}; +#[derive(Fields)] +struct RwParallelUnit { + #[primary_key] + id: i32, + worker_id: i32, +} -impl SysCatalogReaderImpl { - pub fn read_rw_parallel_units_info(&self) -> Result> { - let workers = self.worker_node_manager.list_worker_nodes(); +#[system_catalog(table, "rw_catalog.rw_parallel_units")] +fn read_rw_parallel_units(reader: &SysCatalogReaderImpl) -> Result> { + let workers = reader.worker_node_manager.list_worker_nodes(); - Ok(workers - .into_iter() - .flat_map(|worker| { - worker.parallel_units.into_iter().map(|unit| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(unit.id as i32)), - Some(ScalarImpl::Int32(unit.worker_node_id as i32)), - ]) + Ok(workers + .into_iter() + .flat_map(|worker| { + worker + .parallel_units + .into_iter() + .map(move |unit| RwParallelUnit { + id: unit.id as i32, + worker_id: worker.id as i32, }) - }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs index 3040213bc91af..930c50382c8ef 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs @@ -12,203 +12,162 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; use serde_json::json; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; // TODO: `rw_relation_info` contains some extra streaming meta info that's only meaningful for // streaming jobs, we'd better query relation infos from `rw_relations` and move these streaming // infos into anther system table. -pub const RW_RELATION_INFO: BuiltinTable = BuiltinTable { - name: "rw_relation_info", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Varchar, "schemaname"), - (DataType::Varchar, "relationname"), - (DataType::Int32, "relationowner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "relationtype"), - (DataType::Int32, "relationid"), - (DataType::Varchar, "relationtimezone"), /* The timezone used to interpret ambiguous - * dates/timestamps as tstz */ - (DataType::Varchar, "fragments"), // fragments is json encoded fragment infos. - (DataType::Timestamptz, "initialized_at"), - (DataType::Timestamptz, "created_at"), - (DataType::Varchar, "initialized_at_cluster_version"), - (DataType::Varchar, "created_at_cluster_version"), - ], - pk: &[0, 1], -}; - -impl SysCatalogReaderImpl { - pub async fn read_relation_info(&self) -> Result> { - let mut table_ids = Vec::new(); - { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.get_all_schema_names(&self.auth_context.database)?; - for schema in &schemas { - let schema_catalog = - reader.get_schema_by_name(&self.auth_context.database, schema)?; - - schema_catalog.iter_mv().for_each(|t| { - table_ids.push(t.id.table_id); - }); - - schema_catalog.iter_table().for_each(|t| { - table_ids.push(t.id.table_id); - }); - - schema_catalog.iter_sink().for_each(|t| { - table_ids.push(t.id.sink_id); - }); - - schema_catalog.iter_index().for_each(|t| { - table_ids.push(t.index_table.id.table_id); - }); - } - } +#[derive(Fields)] +#[primary_key(schemaname, relationname)] +struct RwRelationInfo { + schemaname: String, + relationname: String, + relationowner: i32, + definition: String, + relationtype: String, + relationid: i32, + relationtimezone: String, // The timezone used to interpret ambiguous dates/timestamps as tstz + fragments: Option, // fragments is json encoded fragment infos. + initialized_at: Option, + created_at: Option, + initialized_at_cluster_version: Option, + created_at_cluster_version: Option, +} - let table_fragments = self.meta_client.list_table_fragments(&table_ids).await?; - let mut rows = Vec::new(); - let reader = self.catalog_reader.read_guard(); - let schemas = reader.get_all_schema_names(&self.auth_context.database)?; +#[system_catalog(table, "rw_catalog.rw_relation_info")] +async fn read_relation_info(reader: &SysCatalogReaderImpl) -> Result> { + let mut table_ids = Vec::new(); + { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.get_all_schema_names(&reader.auth_context.database)?; for schema in &schemas { - let schema_catalog = reader.get_schema_by_name(&self.auth_context.database, schema)?; + let schema_catalog = + catalog_reader.get_schema_by_name(&reader.auth_context.database, schema)?; + schema_catalog.iter_mv().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.id.table_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("MATERIALIZED VIEW".into())), - Some(ScalarImpl::Int32(t.id.table_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_ctx().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - t.initialized_at_epoch.map(|e| e.as_scalar()), - t.created_at_epoch.map(|e| e.as_scalar()), - t.initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - t.created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ])); - } + table_ids.push(t.id.table_id); }); schema_catalog.iter_table().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.id.table_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("TABLE".into())), - Some(ScalarImpl::Int32(t.id.table_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_ctx().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - t.initialized_at_epoch.map(|e| e.as_scalar()), - t.created_at_epoch.map(|e| e.as_scalar()), - t.initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - t.created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ])); - } + table_ids.push(t.id.table_id); }); schema_catalog.iter_sink().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.id.sink_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner.user_id as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("SINK".into())), - Some(ScalarImpl::Int32(t.id.sink_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_ctx().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - t.initialized_at_epoch.map(|e| e.as_scalar()), - t.created_at_epoch.map(|e| e.as_scalar()), - t.initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - t.created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ])); - } + table_ids.push(t.id.sink_id); }); schema_catalog.iter_index().for_each(|t| { - if let Some(fragments) = table_fragments.get(&t.index_table.id.table_id) { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.index_table.owner as i32)), - Some(ScalarImpl::Utf8(t.index_table.definition.clone().into())), - Some(ScalarImpl::Utf8("INDEX".into())), - Some(ScalarImpl::Int32(t.index_table.id.table_id as i32)), - Some(ScalarImpl::Utf8( - fragments.get_ctx().unwrap().get_timezone().clone().into(), - )), - Some(ScalarImpl::Utf8( - json!(fragments.get_fragments()).to_string().into(), - )), - t.initialized_at_epoch.map(|e| e.as_scalar()), - t.created_at_epoch.map(|e| e.as_scalar()), - t.initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - t.created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ])); - } - }); - - // Sources have no fragments. - schema_catalog.iter_source().for_each(|t| { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Utf8(schema.clone().into())), - Some(ScalarImpl::Utf8(t.name.clone().into())), - Some(ScalarImpl::Int32(t.owner as i32)), - Some(ScalarImpl::Utf8(t.definition.clone().into())), - Some(ScalarImpl::Utf8("SOURCE".into())), - Some(ScalarImpl::Int32(t.id as i32)), - Some(ScalarImpl::Utf8("".into())), - None, - t.initialized_at_epoch.map(|e| e.as_scalar()), - t.created_at_epoch.map(|e| e.as_scalar()), - t.initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - t.created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ])); + table_ids.push(t.index_table.id.table_id); }); } + } + + let table_fragments = reader.meta_client.list_table_fragments(&table_ids).await?; + let mut rows = Vec::new(); + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.get_all_schema_names(&reader.auth_context.database)?; + for schema in &schemas { + let schema_catalog = + catalog_reader.get_schema_by_name(&reader.auth_context.database, schema)?; + schema_catalog.iter_mv().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.id.table_id) { + rows.push(RwRelationInfo { + schemaname: schema.clone(), + relationname: t.name.clone(), + relationowner: t.owner as i32, + definition: t.definition.clone(), + relationtype: "MATERIALIZED VIEW".into(), + relationid: t.id.table_id as i32, + relationtimezone: fragments.get_ctx().unwrap().get_timezone().clone(), + fragments: Some(json!(fragments.get_fragments()).to_string()), + initialized_at: t.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: t.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: t.initialized_at_cluster_version.clone(), + created_at_cluster_version: t.created_at_cluster_version.clone(), + }); + } + }); - Ok(rows) + schema_catalog.iter_table().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.id.table_id) { + rows.push(RwRelationInfo { + schemaname: schema.clone(), + relationname: t.name.clone(), + relationowner: t.owner as i32, + definition: t.definition.clone(), + relationtype: "TABLE".into(), + relationid: t.id.table_id as i32, + relationtimezone: fragments.get_ctx().unwrap().get_timezone().clone(), + fragments: Some(json!(fragments.get_fragments()).to_string()), + initialized_at: t.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: t.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: t.initialized_at_cluster_version.clone(), + created_at_cluster_version: t.created_at_cluster_version.clone(), + }); + } + }); + + schema_catalog.iter_sink().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.id.sink_id) { + rows.push(RwRelationInfo { + schemaname: schema.clone(), + relationname: t.name.clone(), + relationowner: t.owner.user_id as i32, + definition: t.definition.clone(), + relationtype: "SINK".into(), + relationid: t.id.sink_id as i32, + relationtimezone: fragments.get_ctx().unwrap().get_timezone().clone(), + fragments: Some(json!(fragments.get_fragments()).to_string()), + initialized_at: t.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: t.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: t.initialized_at_cluster_version.clone(), + created_at_cluster_version: t.created_at_cluster_version.clone(), + }); + } + }); + + schema_catalog.iter_index().for_each(|t| { + if let Some(fragments) = table_fragments.get(&t.index_table.id.table_id) { + rows.push(RwRelationInfo { + schemaname: schema.clone(), + relationname: t.name.clone(), + relationowner: t.index_table.owner as i32, + definition: t.index_table.definition.clone(), + relationtype: "INDEX".into(), + relationid: t.index_table.id.table_id as i32, + relationtimezone: fragments.get_ctx().unwrap().get_timezone().clone(), + fragments: Some(json!(fragments.get_fragments()).to_string()), + initialized_at: t.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: t.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: t.initialized_at_cluster_version.clone(), + created_at_cluster_version: t.created_at_cluster_version.clone(), + }); + } + }); + + // Sources have no fragments. + schema_catalog.iter_source().for_each(|t| { + rows.push(RwRelationInfo { + schemaname: schema.clone(), + relationname: t.name.clone(), + relationowner: t.owner as i32, + definition: t.definition.clone(), + relationtype: "SOURCE".into(), + relationid: t.id as i32, + relationtimezone: "".into(), + fragments: None, + initialized_at: t.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: t.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: t.initialized_at_cluster_version.clone(), + created_at_cluster_version: t.created_at_cluster_version.clone(), + }); + }); } + + Ok(rows) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs index 855e65fd62ca3..50cd2a7e82d51 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relations.rs @@ -12,34 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::BuiltinView; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; /// `rw_relations` is a view that shows all relations in the database. -pub static RW_RELATIONS: LazyLock = LazyLock::new(|| { - BuiltinView { - name: "rw_relations", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Varchar, "relation_type"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - ], - sql: "SELECT id, name, 'table' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_tables \ - UNION ALL SELECT id, name, 'system table' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_system_tables \ - UNION ALL SELECT id, name, 'source' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_sources \ - UNION ALL SELECT id, name, 'index' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_indexes \ - UNION ALL SELECT id, name, 'sink' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_sinks \ - UNION ALL SELECT id, name, 'materialized view' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_materialized_views \ - UNION ALL SELECT id, name, 'view' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_views \ - ".to_string(), - } -}); +#[system_catalog( + view, + "rw_catalog.rw_relations", + "SELECT id, name, 'table' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_tables + UNION ALL SELECT id, name, 'system table' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_system_tables + UNION ALL SELECT id, name, 'source' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_sources + UNION ALL SELECT id, name, 'index' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_indexes + UNION ALL SELECT id, name, 'sink' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_sinks + UNION ALL SELECT id, name, 'materialized view' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_materialized_views + UNION ALL SELECT id, name, 'view' AS relation_type, schema_id, owner, definition, acl FROM rw_catalog.rw_views + " +)] +#[derive(Fields)] +struct RwRelation { + id: i32, + name: String, + relation_type: String, + schema_id: i32, + owner: i32, + definition: String, + acl: String, +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs index e603fab62a565..8d8786e0b1098 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_schemas.rs @@ -12,48 +12,37 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::catalog::OwnedByUserCatalog; use crate::error::Result; -pub const RW_SCHEMAS: BuiltinTable = BuiltinTable { - name: "rw_schemas", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "owner"), - (DataType::Varchar, "acl"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwSchema { + #[primary_key] + id: i32, + name: String, + owner: i32, + acl: String, +} -impl SysCatalogReaderImpl { - pub fn read_rw_schema_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_schemas")] +fn read_rw_schema_info(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .map(|schema| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Utf8(schema.name().into())), - Some(ScalarImpl::Int32(schema.owner() as i32)), - Some(ScalarImpl::Utf8( - get_acl_items(&Object::SchemaId(schema.id()), false, &users, username_map) - .into(), - )), - ]) - }) - .collect_vec()) - } + Ok(schemas + .map(|schema| RwSchema { + id: schema.id() as i32, + name: schema.name(), + owner: schema.owner() as i32, + acl: get_acl_items(&Object::SchemaId(schema.id()), false, &users, username_map), + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs index 9735820c9ad9f..68488a384e1e6 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sinks.rs @@ -12,87 +12,67 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; use crate::handler::create_source::UPSTREAM_SOURCE_KEY; -pub const RW_SINKS: BuiltinTable = BuiltinTable { - name: "rw_sinks", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "connector"), - (DataType::Varchar, "sink_type"), - (DataType::Int32, "connection_id"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - (DataType::Timestamptz, "initialized_at"), - (DataType::Timestamptz, "created_at"), - (DataType::Varchar, "initialized_at_cluster_version"), - (DataType::Varchar, "created_at_cluster_version"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwSink { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + connector: String, + sink_type: String, + connection_id: Option, + definition: String, + acl: String, + initialized_at: Option, + created_at: Option, + initialized_at_cluster_version: Option, + created_at_cluster_version: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_sinks_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_sinks")] +fn read_rw_sinks_info(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema.iter_sink().map(|sink| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(sink.id.sink_id as i32)), - Some(ScalarImpl::Utf8(sink.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(sink.owner.user_id as i32)), - Some(ScalarImpl::Utf8( - sink.properties - .get(UPSTREAM_SOURCE_KEY) - .cloned() - .unwrap_or("".to_string()) - .to_uppercase() - .into(), - )), - Some(ScalarImpl::Utf8( - sink.sink_type.to_proto().as_str_name().into(), - )), - sink.connection_id - .map(|id| ScalarImpl::Int32(id.connection_id() as i32)), - Some(ScalarImpl::Utf8(sink.create_sql().into())), - Some( - get_acl_items( - &Object::SinkId(sink.id.sink_id), - false, - &users, - username_map, - ) - .into(), - ), - sink.initialized_at_epoch.map(|e| e.as_scalar()), - sink.created_at_epoch.map(|e| e.as_scalar()), - sink.initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - sink.created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_sink().map(|sink| RwSink { + id: sink.id.sink_id as i32, + name: sink.name.clone(), + schema_id: schema.id() as i32, + owner: sink.owner.user_id as i32, + connector: sink + .properties + .get(UPSTREAM_SOURCE_KEY) + .cloned() + .unwrap_or("".to_string()) + .to_uppercase(), + sink_type: sink.sink_type.to_proto().as_str_name().into(), + connection_id: sink.connection_id.map(|id| id.connection_id() as i32), + definition: sink.create_sql(), + acl: get_acl_items( + &Object::SinkId(sink.id.sink_id), + false, + &users, + username_map, + ), + initialized_at: sink.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: sink.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: sink.initialized_at_cluster_version.clone(), + created_at_cluster_version: sink.created_at_cluster_version.clone(), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs index 43f5ddf630d31..5a05f69750caf 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs @@ -12,118 +12,79 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use itertools::Itertools; -use risingwave_common::array::ListValue; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{ - get_acl_items, BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef, -}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; use crate::handler::create_source::UPSTREAM_SOURCE_KEY; -pub static RW_SOURCES_COLUMNS: LazyLock>> = LazyLock::new(|| { - vec![ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "connector"), - // [col1, col2] - (DataType::List(Box::new(DataType::Varchar)), "columns"), - (DataType::Varchar, "format"), - (DataType::Varchar, "row_encode"), - (DataType::Boolean, "append_only"), - (DataType::Int32, "connection_id"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - (DataType::Timestamptz, "initialized_at"), - (DataType::Timestamptz, "created_at"), - (DataType::Varchar, "initialized_at_cluster_version"), - (DataType::Varchar, "created_at_cluster_version"), - ] -}); - -pub static RW_SOURCES: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_sources", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &RW_SOURCES_COLUMNS, - pk: &[0], -}); +#[derive(Fields)] +struct RwSource { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + connector: String, + columns: Vec, + format: Option, + row_encode: Option, + append_only: bool, + connection_id: Option, + definition: String, + acl: String, + initialized_at: Option, + created_at: Option, + initialized_at_cluster_version: Option, + created_at_cluster_version: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_sources_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_sources")] +fn read_rw_sources_info(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema - .iter_source() - .filter(|s| s.associated_table_id.is_none()) - .map(|source| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(source.id as i32)), - Some(ScalarImpl::Utf8(source.name.clone().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(source.owner as i32)), - Some(ScalarImpl::Utf8( - source - .with_properties - .get(UPSTREAM_SOURCE_KEY) - .cloned() - .unwrap_or("".to_string()) - .to_uppercase() - .into(), - )), - Some(ScalarImpl::List(ListValue::from_iter( - source.columns.iter().map(|c| c.name()), - ))), - source - .info - .get_format() - .map(|format| Some(ScalarImpl::Utf8(format.as_str_name().into()))) - .unwrap_or(None), - source - .info - .get_row_encode() - .map(|row_encode| { - Some(ScalarImpl::Utf8(row_encode.as_str_name().into())) - }) - .unwrap_or(None), - Some(ScalarImpl::Bool(source.append_only)), - source.connection_id.map(|id| ScalarImpl::Int32(id as i32)), - Some(ScalarImpl::Utf8(source.create_sql().into())), - Some( - get_acl_items( - &Object::SourceId(source.id), - false, - &users, - username_map, - ) - .into(), - ), - source.initialized_at_epoch.map(|e| e.as_scalar()), - source.created_at_epoch.map(|e| e.as_scalar()), - source - .initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - source - .created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ]) - }) - }) - .collect_vec()) - } + Ok(schemas + .flat_map(|schema| { + schema + .iter_source() + .filter(|s| s.associated_table_id.is_none()) + .map(|source| RwSource { + id: source.id as i32, + name: source.name.clone(), + schema_id: schema.id() as i32, + owner: source.owner as i32, + connector: source + .with_properties + .get(UPSTREAM_SOURCE_KEY) + .cloned() + .unwrap_or("".to_string()) + .to_uppercase(), + columns: source.columns.iter().map(|c| c.name().into()).collect(), + format: source + .info + .get_format() + .ok() + .map(|format| format.as_str_name().into()), + row_encode: source + .info + .get_row_encode() + .ok() + .map(|row_encode| row_encode.as_str_name().into()), + append_only: source.append_only, + connection_id: source.connection_id.map(|id| id as i32), + definition: source.create_sql(), + acl: get_acl_items(&Object::SourceId(source.id), false, &users, username_map), + initialized_at: source.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: source.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: source.initialized_at_cluster_version.clone(), + created_at_cluster_version: source.created_at_cluster_version.clone(), + }) + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_streaming_parallelism.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_streaming_parallelism.rs index e159a2c30fb52..1d4a44d3b47c5 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_streaming_parallelism.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_streaming_parallelism.rs @@ -12,57 +12,34 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Copyright 2023 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. - -use std::sync::LazyLock; - -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::types::DataType; - -use crate::catalog::system_catalog::{BuiltinView, SystemCatalogColumnsDef}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -pub static RW_STREAMING_PARALLELISM_COLUMNS: LazyLock>> = - LazyLock::new(|| { - vec![ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Varchar, "relation_type"), - (DataType::Varchar, "parallelism"), - ] - }); -pub static RW_STREAMING_PARALLELISM: LazyLock = LazyLock::new(|| BuiltinView { - name: "rw_streaming_parallelism", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &RW_STREAMING_PARALLELISM_COLUMNS, - sql: "WITH all_streaming_jobs AS ( \ - SELECT id, name, 'table' as relation_type FROM rw_tables \ - UNION ALL \ - SELECT id, name, 'materialized view' as relation_type FROM rw_materialized_views \ - UNION ALL \ - SELECT id, name, 'sink' as relation_type FROM rw_sinks \ - UNION ALL \ - SELECT id, name, 'index' as relation_type FROM rw_indexes \ - ) \ - SELECT \ - job.id, \ - job.name, \ - job.relation_type, \ - tf.parallelism \ - FROM all_streaming_jobs job \ - INNER JOIN rw_table_fragments tf ON job.id = tf.table_id \ - ORDER BY job.id\ - " - .to_string(), -}); +#[system_catalog( + view, + "rw_catalog.rw_streaming_parallelism", + "WITH all_streaming_jobs AS ( + SELECT id, name, 'table' as relation_type FROM rw_tables + UNION ALL + SELECT id, name, 'materialized view' as relation_type FROM rw_materialized_views + UNION ALL + SELECT id, name, 'sink' as relation_type FROM rw_sinks + UNION ALL + SELECT id, name, 'index' as relation_type FROM rw_indexes + ) + SELECT + job.id, + job.name, + job.relation_type, + tf.parallelism + FROM all_streaming_jobs job + INNER JOIN rw_table_fragments tf ON job.id = tf.table_id + ORDER BY job.id" +)] +#[derive(Fields)] +struct RwStreamingParallelism { + id: i32, + name: String, + relation_type: String, + parallelism: String, +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs index 79176bc96ec36..0fda14d726876 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_system_tables.rs @@ -12,61 +12,48 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; /// `rw_system_tables` stores all system tables in the database. -pub static RW_SYSTEM_TABLES: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_system_tables", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - ], - pk: &[0], -}); +#[derive(Fields)] +struct SystemTable { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + definition: Option, + acl: String, +} -impl SysCatalogReaderImpl { - pub fn read_system_table_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_system_tables")] +fn read_system_table_info(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema.iter_system_tables().map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - None, - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::TableId(table.id.table_id), - false, - &users, - username_map, - ) - .into(), - )), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_system_tables().map(|table| SystemTable { + id: table.id.table_id as i32, + name: table.name().to_string(), + schema_id: schema.id() as i32, + owner: table.owner as i32, + definition: None, + acl: get_acl_items( + &Object::TableId(table.id.table_id), + false, + &users, + username_map, + ), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs index 85de321f344b5..6bfb835f1d530 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_fragments.rs @@ -12,42 +12,35 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{ - extract_parallelism_from_table_state, BuiltinTable, SysCatalogReaderImpl, -}; +use crate::catalog::system_catalog::{extract_parallelism_from_table_state, SysCatalogReaderImpl}; use crate::error::Result; -pub const RW_TABLE_FRAGMENTS: BuiltinTable = BuiltinTable { - name: "rw_table_fragments", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "table_id"), - (DataType::Varchar, "status"), - (DataType::Varchar, "parallelism"), - ], - pk: &[0], -}; - -impl SysCatalogReaderImpl { - pub async fn read_rw_table_fragments_info(&self) -> Result> { - let states = self.meta_client.list_table_fragment_states().await?; +#[derive(Fields)] +struct RwTableFragment { + #[primary_key] + table_id: i32, + status: String, + parallelism: String, +} - Ok(states - .into_iter() - .map(|state| { - let parallelism = extract_parallelism_from_table_state(&state); +#[system_catalog(table, "rw_catalog.rw_table_fragments")] +async fn read_rw_table_fragments_info( + reader: &SysCatalogReaderImpl, +) -> Result> { + let states = reader.meta_client.list_table_fragment_states().await?; - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(state.table_id as i32)), - Some(ScalarImpl::Utf8(state.state().as_str_name().into())), - Some(ScalarImpl::Utf8(parallelism.to_uppercase().into())), - ]) - }) - .collect_vec()) - } + Ok(states + .into_iter() + .map(|state| { + let parallelism = extract_parallelism_from_table_state(&state); + RwTableFragment { + table_id: state.table_id as i32, + status: state.state().as_str_name().into(), + parallelism: parallelism.to_uppercase(), + } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs index 189ecdb4a1725..3d81f480660b8 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_table_stats.rs @@ -12,43 +12,33 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl, SystemCatalogColumnsDef}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_TABLE_STATS_TABLE_NAME: &str = "rw_table_stats"; -pub const RW_TABLE_STATS_TABLE_ID_INDEX: usize = 0; - -pub const RW_TABLE_STATS_COLUMNS: &[SystemCatalogColumnsDef<'_>] = &[ - (DataType::Int32, "id"), - (DataType::Int64, "total_key_count"), - (DataType::Int64, "total_key_size"), - (DataType::Int64, "total_value_size"), -]; - -pub const RW_TABLE_STATS: BuiltinTable = BuiltinTable { - name: RW_TABLE_STATS_TABLE_NAME, - schema: RW_CATALOG_SCHEMA_NAME, - columns: RW_TABLE_STATS_COLUMNS, - pk: &[RW_TABLE_STATS_TABLE_ID_INDEX], -}; +#[derive(Fields)] +struct RwTableStats { + #[primary_key] + id: i32, + total_key_count: i64, + total_key_size: i64, + total_value_size: i64, +} -impl SysCatalogReaderImpl { - pub fn read_table_stats(&self) -> Result> { - let catalog = self.catalog_reader.read_guard(); - let table_stats = catalog.table_stats(); - let mut rows = vec![]; - for (id, stats) in &table_stats.table_stats { - rows.push(OwnedRow::new(vec![ - Some(ScalarImpl::Int32(*id as i32)), - Some(ScalarImpl::Int64(stats.total_key_count)), - Some(ScalarImpl::Int64(stats.total_key_size)), - Some(ScalarImpl::Int64(stats.total_value_size)), - ])); - } - Ok(rows) +#[system_catalog(table, "rw_catalog.rw_table_stats")] +fn read_table_stats(reader: &SysCatalogReaderImpl) -> Result> { + let catalog = reader.catalog_reader.read_guard(); + let table_stats = catalog.table_stats(); + let mut rows = vec![]; + for (id, stats) in &table_stats.table_stats { + rows.push(RwTableStats { + id: *id as i32, + total_key_count: stats.total_key_count, + total_key_size: stats.total_key_size, + total_value_size: stats.total_value_size, + }); } + Ok(rows) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs index 45db055e6f8f6..d86690eddfe16 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_tables.rs @@ -12,72 +12,55 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; -pub const RW_TABLES: BuiltinTable = BuiltinTable { - name: "rw_tables", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - (DataType::Timestamptz, "initialized_at"), - (DataType::Timestamptz, "created_at"), - (DataType::Varchar, "initialized_at_cluster_version"), - (DataType::Varchar, "created_at_cluster_version"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwTable { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + definition: String, + acl: String, + initialized_at: Option, + created_at: Option, + initialized_at_cluster_version: Option, + created_at_cluster_version: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_table_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_tables")] +fn read_rw_table_info(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema.iter_table().map(|table| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(table.id.table_id as i32)), - Some(ScalarImpl::Utf8(table.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(table.owner as i32)), - Some(ScalarImpl::Utf8(table.create_sql().into())), - Some(ScalarImpl::Utf8( - get_acl_items( - &Object::TableId(table.id.table_id), - true, - &users, - username_map, - ) - .into(), - )), - table.initialized_at_epoch.map(|e| e.as_scalar()), - table.created_at_epoch.map(|e| e.as_scalar()), - table - .initialized_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - table - .created_at_cluster_version - .clone() - .map(|v| ScalarImpl::Utf8(v.into())), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_table().map(|table| RwTable { + id: table.id.table_id as i32, + name: table.name().to_string(), + schema_id: schema.id() as i32, + owner: table.owner as i32, + definition: table.create_sql(), + acl: get_acl_items( + &Object::TableId(table.id.table_id), + false, + &users, + username_map, + ), + initialized_at: table.initialized_at_epoch.map(|e| e.as_timestamptz()), + created_at: table.created_at_epoch.map(|e| e.as_timestamptz()), + initialized_at_cluster_version: table.initialized_at_cluster_version.clone(), + created_at_cluster_version: table.created_at_cluster_version.clone(), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs index d207d222578c5..041af546a9980 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs @@ -12,17 +12,39 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; - -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; use risingwave_common::for_all_base_types; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; +/// `rw_types` stores all supported types in the database. +#[derive(Fields)] +struct RwType { + #[primary_key] + id: i32, + name: String, + input_oid: String, + typelem: i32, + typarray: i32, +} + +#[system_catalog(table, "rw_catalog.rw_types")] +fn read_rw_types(_: &SysCatalogReaderImpl) -> Result> { + let mut rows = vec![]; + for (id, name, input_oid, typelem, typarray) in RW_TYPE_DATA { + rows.push(RwType { + id: *id, + name: name.to_string(), + input_oid: input_oid.to_string(), + typelem: *typelem, + typarray: *typarray, + }); + } + Ok(rows) +} + macro_rules! impl_pg_type_data { ($( { $enum:ident | $oid:literal | $oid_array:literal | $name:ident | $input:ident | $len:literal } )*) => { &[ @@ -40,35 +62,4 @@ macro_rules! impl_pg_type_data { ] } } -pub const RW_TYPE_DATA: &[(i32, &str, &str, i32, i32)] = for_all_base_types! { impl_pg_type_data }; - -/// `rw_types` stores all supported types in the database. -pub static RW_TYPES: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_types", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Varchar, "input_oid"), - (DataType::Int32, "typelem"), - (DataType::Int32, "typarray"), - ], - pk: &[0], -}); - -impl SysCatalogReaderImpl { - pub fn read_rw_types(&self) -> Result> { - Ok(RW_TYPE_DATA - .iter() - .map(|(id, name, input, typelem, typarray)| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(*id)), - Some(ScalarImpl::Utf8(name.to_string().into())), - Some(ScalarImpl::Utf8(input.to_string().into())), - Some(ScalarImpl::Int32(*typelem)), - Some(ScalarImpl::Int32(*typarray)), - ]) - }) - .collect_vec()) - } -} +const RW_TYPE_DATA: &[(i32, &str, &str, i32, i32)] = for_all_base_types! { impl_pg_type_data }; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs index b481d60a89b4b..93b2fe361f667 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_user_secrets.rs @@ -12,58 +12,49 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::LazyLock; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; - -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::{ErrorCode, Result}; use crate::user::user_authentication::encrypted_raw_password; /// `rw_user_secret` stores all user encrypted passwords in the database, which is only readable by /// super users. -pub static RW_USER_SECRETS: LazyLock = LazyLock::new(|| BuiltinTable { - name: "rw_user_secrets", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[(DataType::Int32, "id"), (DataType::Varchar, "password")], - pk: &[0], -}); +#[derive(Fields)] +struct RwUserSecret { + #[primary_key] + id: i32, + password: Option, +} -impl SysCatalogReaderImpl { - pub fn read_rw_user_secrets_info(&self) -> Result> { - let reader = self.user_info_reader.read_guard(); - // Since this catalog contains passwords, it must not be publicly readable. - match reader.get_user_by_name(&self.auth_context.user_name) { - None => { - return Err(ErrorCode::CatalogError( - format!("user {} not found", self.auth_context.user_name).into(), +#[system_catalog(table, "rw_catalog.rw_user_secrets")] +fn read_rw_user_secrets_info(reader: &SysCatalogReaderImpl) -> Result> { + let user_info_reader = reader.user_info_reader.read_guard(); + // Since this catalog contains passwords, it must not be publicly readable. + match user_info_reader.get_user_by_name(&reader.auth_context.user_name) { + None => { + return Err(ErrorCode::CatalogError( + format!("user {} not found", reader.auth_context.user_name).into(), + ) + .into()); + } + Some(user) => { + if !user.is_super { + return Err(ErrorCode::PermissionDenied( + "permission denied for table rw_user_secrets".to_string(), ) .into()); } - Some(user) => { - if !user.is_super { - return Err(ErrorCode::PermissionDenied( - "permission denied for table rw_user_secrets".to_string(), - ) - .into()); - } - } } - let users = reader.get_all_users(); - - Ok(users - .into_iter() - .map(|user| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(user.id as i32)), - user.auth_info - .as_ref() - .map(|info| ScalarImpl::Utf8(encrypted_raw_password(info).into())), - ]) - }) - .collect_vec()) } + let users = user_info_reader.get_all_users(); + + Ok(users + .into_iter() + .map(|user| RwUserSecret { + id: user.id as i32, + password: user.auth_info.as_ref().map(encrypted_raw_password), + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs index fb59dbab44da7..048e48d6de514 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_users.rs @@ -12,45 +12,37 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; -pub const RW_USERS: BuiltinTable = BuiltinTable { - name: "rw_users", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Boolean, "is_super"), - (DataType::Boolean, "create_db"), - (DataType::Boolean, "create_user"), - (DataType::Boolean, "can_login"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwUser { + #[primary_key] + id: i32, + name: String, + is_super: bool, + create_db: bool, + create_user: bool, + can_login: bool, +} -impl SysCatalogReaderImpl { - pub fn read_rw_user_info(&self) -> Result> { - let reader = self.user_info_reader.read_guard(); - let users = reader.get_all_users(); +#[system_catalog(table, "rw_catalog.rw_users")] +fn read_rw_user_info(reader: &SysCatalogReaderImpl) -> Result> { + let reader = reader.user_info_reader.read_guard(); + let users = reader.get_all_users(); - Ok(users - .into_iter() - .map(|user| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(user.id as i32)), - Some(ScalarImpl::Utf8(user.name.into())), - Some(ScalarImpl::Bool(user.is_super)), - Some(ScalarImpl::Bool(user.can_create_db)), - Some(ScalarImpl::Bool(user.can_create_user)), - Some(ScalarImpl::Bool(user.can_login)), - ]) - }) - .collect_vec()) - } + Ok(users + .into_iter() + .map(|user| RwUser { + id: user.id as i32, + name: user.name, + is_super: user.is_super, + create_db: user.can_create_db, + create_user: user.can_create_user, + can_login: user.can_login, + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs index c9a21e0f7e81b..6ebf16ddd40ac 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_views.rs @@ -12,53 +12,42 @@ // See the License for the specific language governing permissions and // limitations under the License. -use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; use risingwave_pb::user::grant_privilege::Object; -use crate::catalog::system_catalog::{get_acl_items, BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::{get_acl_items, SysCatalogReaderImpl}; use crate::error::Result; -pub const RW_VIEWS: BuiltinTable = BuiltinTable { - name: "rw_views", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "name"), - (DataType::Int32, "schema_id"), - (DataType::Int32, "owner"), - (DataType::Varchar, "definition"), - (DataType::Varchar, "acl"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwView { + #[primary_key] + id: i32, + name: String, + schema_id: i32, + owner: i32, + definition: String, + acl: String, +} -impl SysCatalogReaderImpl { - pub fn read_rw_views_info(&self) -> Result> { - let reader = self.catalog_reader.read_guard(); - let schemas = reader.iter_schemas(&self.auth_context.database)?; - let user_reader = self.user_info_reader.read_guard(); - let users = user_reader.get_all_users(); - let username_map = user_reader.get_user_name_map(); +#[system_catalog(table, "rw_catalog.rw_views")] +fn read_rw_view_info(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + let user_reader = reader.user_info_reader.read_guard(); + let users = user_reader.get_all_users(); + let username_map = user_reader.get_user_name_map(); - Ok(schemas - .flat_map(|schema| { - schema.iter_view().map(|view| { - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(view.id as i32)), - Some(ScalarImpl::Utf8(view.name().into())), - Some(ScalarImpl::Int32(schema.id() as i32)), - Some(ScalarImpl::Int32(view.owner as i32)), - Some(ScalarImpl::Utf8(view.create_sql().into())), - Some(ScalarImpl::Utf8( - get_acl_items(&Object::ViewId(view.id), false, &users, username_map) - .into(), - )), - ]) - }) + Ok(schemas + .flat_map(|schema| { + schema.iter_view().map(|view| RwView { + id: view.id as i32, + name: view.name().to_string(), + schema_id: schema.id() as i32, + owner: view.owner as i32, + definition: view.create_sql(), + acl: get_acl_items(&Object::ViewId(view.id), false, &users, username_map), }) - .collect_vec()) - } + }) + .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs index c1bbd92478895..226b0230e3f21 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs @@ -13,68 +13,59 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; -use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl, Timestamptz}; +use risingwave_common::types::{Fields, Timestamptz}; +use risingwave_frontend_macro::system_catalog; -use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; +use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; /// `rw_worker_nodes` contains all information about the compute nodes in the cluster. -pub const RW_WORKER_NODES: BuiltinTable = BuiltinTable { - name: "rw_worker_nodes", - schema: RW_CATALOG_SCHEMA_NAME, - columns: &[ - (DataType::Int32, "id"), - (DataType::Varchar, "host"), - (DataType::Varchar, "port"), - (DataType::Varchar, "type"), - (DataType::Varchar, "state"), - (DataType::Int32, "parallelism"), - (DataType::Boolean, "is_streaming"), - (DataType::Boolean, "is_serving"), - (DataType::Boolean, "is_unschedulable"), - (DataType::Varchar, "rw_version"), - (DataType::Int64, "system_total_memory_bytes"), - (DataType::Int64, "system_total_cpu_cores"), - (DataType::Timestamptz, "started_at"), - ], - pk: &[0], -}; +#[derive(Fields)] +struct RwWorkerNode { + #[primary_key] + id: i32, + host: Option, + port: Option, + r#type: String, + state: String, + parallelism: i32, + is_streaming: Option, + is_serving: Option, + is_unschedulable: Option, + rw_version: Option, + system_total_memory_bytes: Option, + system_total_cpu_cores: Option, + started_at: Option, +} -impl SysCatalogReaderImpl { - pub async fn read_rw_worker_nodes_info(&self) -> Result> { - let workers = self.meta_client.list_all_nodes().await?; +#[system_catalog(table, "rw_catalog.rw_worker_nodes")] +async fn read_rw_worker_nodes_info(reader: &SysCatalogReaderImpl) -> Result> { + let workers = reader.meta_client.list_all_nodes().await?; - Ok(workers - .into_iter() - .sorted_by_key(|w| w.id) - .map(|worker| { - let host = worker.host.as_ref(); - let property = worker.property.as_ref(); - let resource = worker.resource.as_ref(); - OwnedRow::new(vec![ - Some(ScalarImpl::Int32(worker.id as i32)), - host.map(|h| ScalarImpl::Utf8(h.host.clone().into())), - host.map(|h| ScalarImpl::Utf8(h.port.to_string().into())), - Some(ScalarImpl::Utf8( - worker.get_type().unwrap().as_str_name().into(), - )), - Some(ScalarImpl::Utf8( - worker.get_state().unwrap().as_str_name().into(), - )), - Some(ScalarImpl::Int32(worker.parallel_units.len() as i32)), - property.map(|p| ScalarImpl::Bool(p.is_streaming)), - property.map(|p| ScalarImpl::Bool(p.is_serving)), - property.map(|p| ScalarImpl::Bool(p.is_unschedulable)), - resource.map(|r| ScalarImpl::Utf8(r.rw_version.to_owned().into())), - resource.map(|r| ScalarImpl::Int64(r.total_memory_bytes as _)), - resource.map(|r| ScalarImpl::Int64(r.total_cpu_cores as _)), - worker.started_at.map(|ts| { - ScalarImpl::Timestamptz(Timestamptz::from_secs(ts as i64).unwrap()) - }), - ]) - }) - .collect_vec()) - } + Ok(workers + .into_iter() + .sorted_by_key(|w| w.id) + .map(|worker| { + let host = worker.host.as_ref(); + let property = worker.property.as_ref(); + let resource = worker.resource.as_ref(); + RwWorkerNode { + id: worker.id as i32, + host: host.map(|h| h.host.clone()), + port: host.map(|h| h.port.to_string()), + r#type: worker.get_type().unwrap().as_str_name().into(), + state: worker.get_state().unwrap().as_str_name().into(), + parallelism: worker.parallel_units.len() as i32, + is_streaming: property.map(|p| p.is_streaming), + is_serving: property.map(|p| p.is_serving), + is_unschedulable: property.map(|p| p.is_unschedulable), + rw_version: resource.map(|r| r.rw_version.to_owned()), + system_total_memory_bytes: resource.map(|r| r.total_memory_bytes as _), + system_total_cpu_cores: resource.map(|r| r.total_cpu_cores as _), + started_at: worker + .started_at + .map(|ts| Timestamptz::from_secs(ts as i64).unwrap()), + } + }) + .collect()) } From 0975be2bb8d1fca8ed5ad553e209aa7394b8dee1 Mon Sep 17 00:00:00 2001 From: August Date: Sun, 18 Feb 2024 16:19:37 +0800 Subject: [PATCH 2/4] fix(sql-backend): clean up dirty source catalogs for failed table with connectors and recovery (#15110) --- .../model_v2/migration/src/m20230908_072257_init.rs | 8 ++++++++ src/meta/src/controller/catalog.rs | 1 + src/meta/src/controller/streaming_job.rs | 11 +++++++++++ 3 files changed, 20 insertions(+) diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index 2da479531abf0..0c3d9d36c4853 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -519,6 +519,14 @@ impl MigrationTrait for Migration { .to(Connection::Table, Connection::ConnectionId) .to_owned(), ) + .foreign_key( + &mut ForeignKey::create() + .name("FK_source_optional_associated_table_id") + .from(Source::Table, Source::OptionalAssociatedTableId) + .to(Object::Table, Object::Oid) + .on_delete(ForeignKeyAction::Cascade) + .to_owned(), + ) .to_owned(), ) .await?; diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 76eab54f631a0..6077efa7f88c1 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -468,6 +468,7 @@ impl CatalogController { .clone() .into_iter() .chain(state_table_ids.clone().into_iter()) + .chain(associated_source_ids.clone().into_iter()) .collect(); let res = Object::delete_many() diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index f4004e25b500c..4fc84dda21d55 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -401,6 +401,14 @@ impl CatalogController { .all(&txn) .await?; + let associated_source_id: Option = Table::find_by_id(job_id) + .select_only() + .column(table::Column::OptionalAssociatedSourceId) + .filter(table::Column::OptionalAssociatedSourceId.is_not_null()) + .into_tuple() + .one(&txn) + .await?; + Object::delete_by_id(job_id).exec(&txn).await?; if !internal_table_ids.is_empty() { Object::delete_many() @@ -408,6 +416,9 @@ impl CatalogController { .exec(&txn) .await?; } + if let Some(source_id) = associated_source_id { + Object::delete_by_id(source_id).exec(&txn).await?; + } txn.commit().await?; Ok(true) From c82698a4452f6a7cb472f3acfaba35d612adeec0 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Sun, 18 Feb 2024 16:45:50 +0800 Subject: [PATCH 3/4] feat(catalog): implement `pg_settings` (#15108) Signed-off-by: Runji Wang --- e2e_test/batch/catalog/issue_8791.slt.part | 8 ++-- e2e_test/batch/catalog/pg_settings.slt.part | 45 ++++++++++++++++++- .../src/catalog/system_catalog/mod.rs | 7 +++ .../system_catalog/pg_catalog/pg_settings.rs | 19 +++++++- src/frontend/src/scheduler/local.rs | 8 +--- src/frontend/src/scheduler/mod.rs | 2 +- src/frontend/src/scheduler/task_context.rs | 28 ++++++------ 7 files changed, 90 insertions(+), 27 deletions(-) diff --git a/e2e_test/batch/catalog/issue_8791.slt.part b/e2e_test/batch/catalog/issue_8791.slt.part index b8339c44a23f4..482d0965ba543 100644 --- a/e2e_test/batch/catalog/issue_8791.slt.part +++ b/e2e_test/batch/catalog/issue_8791.slt.part @@ -1,16 +1,16 @@ # UNION and other complex queries should also be in local mode query I -SELECT name FROM pg_catalog.pg_settings union select 'a'; +SELECT amname FROM pg_catalog.pg_am union select 'a'; ---- a query T -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)='' -LIMIT 1000 +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)='' +LIMIT 1000; ---- session authorization all query I -with q as ( select name FROM pg_catalog.pg_settings ) select * from q; +with q as ( select amname FROM pg_catalog.pg_am ) select * from q; ---- diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 0481ab1a1dd5b..09e2546a856d8 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -1,6 +1,49 @@ query TT -SELECT * FROM pg_catalog.pg_settings; +SELECT name FROM pg_catalog.pg_settings order by 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 query TT SELECT * FROM pg_catalog.pg_settings where name='dummy'; diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index a0be5af42fd36..61ec69b77ae5a 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -22,6 +22,7 @@ use std::sync::{Arc, LazyLock}; use async_trait::async_trait; use futures::future::BoxFuture; use itertools::Itertools; +use parking_lot::RwLock; use risingwave_common::acl::AclMode; use risingwave_common::array::DataChunk; use risingwave_common::catalog::{ @@ -29,6 +30,7 @@ use risingwave_common::catalog::{ NON_RESERVED_SYS_CATALOG_ID, }; use risingwave_common::error::BoxedError; +use risingwave_common::session_config::ConfigMap; use risingwave_common::types::DataType; use risingwave_pb::meta::list_table_fragment_states_response::TableFragmentState; use risingwave_pb::meta::table_parallelism::{PbFixedParallelism, PbParallelism}; @@ -104,7 +106,10 @@ pub struct SysCatalogReaderImpl { worker_node_manager: WorkerNodeManagerRef, // Read from meta. meta_client: Arc, + // Read auth context. auth_context: Arc, + // Read config. + config: Arc>, } impl SysCatalogReaderImpl { @@ -114,6 +119,7 @@ impl SysCatalogReaderImpl { worker_node_manager: WorkerNodeManagerRef, meta_client: Arc, auth_context: Arc, + config: Arc>, ) -> Self { Self { catalog_reader, @@ -121,6 +127,7 @@ impl SysCatalogReaderImpl { worker_node_manager, meta_client, auth_context, + config, } } } 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 5acd1dab05a7d..4fc0fb057108f 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 @@ -15,11 +15,28 @@ use risingwave_common::types::Fields; use risingwave_frontend_macro::system_catalog; +use crate::catalog::system_catalog::SysCatalogReaderImpl; + /// The catalog `pg_settings` stores settings. /// Ref: [`https://www.postgresql.org/docs/current/view-pg-settings.html`] -#[system_catalog(view, "pg_catalog.pg_settings")] #[derive(Fields)] struct PgSetting { name: String, setting: String, + short_desc: String, +} + +#[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(); + + all_variables + .iter() + .map(|info| PgSetting { + name: info.name.clone(), + setting: info.setting.clone(), + short_desc: info.description.clone(), + }) + .collect() } diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index c155cfe9aa234..5e1838e765a49 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -52,7 +52,7 @@ use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId}; use crate::scheduler::task_context::FrontendBatchTaskContext; use crate::scheduler::worker_node_manager::WorkerNodeSelector; use crate::scheduler::{ReadSnapshot, SchedulerError, SchedulerResult}; -use crate::session::{AuthContext, FrontendEnv, SessionImpl}; +use crate::session::{FrontendEnv, SessionImpl}; pub type LocalQueryStream = ReceiverStream>; @@ -94,10 +94,6 @@ impl LocalQueryExecution { } } - fn auth_context(&self) -> Arc { - self.session.auth_context() - } - fn shutdown_rx(&self) -> ShutdownToken { self.session.reset_cancel_query_flag() } @@ -106,7 +102,7 @@ impl LocalQueryExecution { pub async fn run_inner(self) { debug!(%self.query.query_id, self.sql, "Starting to run query"); - let context = FrontendBatchTaskContext::new(self.front_env.clone(), self.auth_context()); + let context = FrontendBatchTaskContext::new(self.session.clone()); let task_id = TaskId { query_id: self.query.query_id.id.clone(), diff --git a/src/frontend/src/scheduler/mod.rs b/src/frontend/src/scheduler/mod.rs index bb27231c21fdc..225f15d5b66ca 100644 --- a/src/frontend/src/scheduler/mod.rs +++ b/src/frontend/src/scheduler/mod.rs @@ -66,6 +66,6 @@ impl ExecutionContext { } pub fn to_batch_task_context(&self) -> FrontendBatchTaskContext { - FrontendBatchTaskContext::new(self.session.env().clone(), self.session.auth_context()) + FrontendBatchTaskContext::new(self.session.clone()) } } diff --git a/src/frontend/src/scheduler/task_context.rs b/src/frontend/src/scheduler/task_context.rs index ed1c4d78dc45c..dfb2496dad556 100644 --- a/src/frontend/src/scheduler/task_context.rs +++ b/src/frontend/src/scheduler/task_context.rs @@ -25,18 +25,17 @@ use risingwave_connector::source::monitor::SourceMetrics; use risingwave_rpc_client::ComputeClientPoolRef; use crate::catalog::system_catalog::SysCatalogReaderImpl; -use crate::session::{AuthContext, FrontendEnv}; +use crate::session::SessionImpl; /// Batch task execution context in frontend. #[derive(Clone)] pub struct FrontendBatchTaskContext { - env: FrontendEnv, - auth_context: Arc, + session: Arc, } impl FrontendBatchTaskContext { - pub fn new(env: FrontendEnv, auth_context: Arc) -> Self { - Self { env, auth_context } + pub fn new(session: Arc) -> Self { + Self { session } } } @@ -47,16 +46,17 @@ impl BatchTaskContext for FrontendBatchTaskContext { fn catalog_reader(&self) -> SysCatalogReaderRef { Arc::new(SysCatalogReaderImpl::new( - self.env.catalog_reader().clone(), - self.env.user_info_reader().clone(), - self.env.worker_node_manager_ref(), - self.env.meta_client_ref(), - self.auth_context.clone(), + self.session.env().catalog_reader().clone(), + self.session.env().user_info_reader().clone(), + self.session.env().worker_node_manager_ref(), + self.session.env().meta_client_ref(), + self.session.auth_context(), + self.session.shared_config(), )) } fn is_local_addr(&self, peer_addr: &HostAddr) -> bool { - is_local_address(self.env.server_address(), peer_addr) + is_local_address(self.session.env().server_address(), peer_addr) } fn state_store(&self) -> risingwave_storage::store_impl::StateStoreImpl { @@ -68,11 +68,11 @@ impl BatchTaskContext for FrontendBatchTaskContext { } fn client_pool(&self) -> ComputeClientPoolRef { - self.env.client_pool() + self.session.env().client_pool() } fn get_config(&self) -> &BatchConfig { - self.env.batch_config() + self.session.env().batch_config() } fn dml_manager(&self) -> risingwave_dml::dml_manager::DmlManagerRef { @@ -80,7 +80,7 @@ impl BatchTaskContext for FrontendBatchTaskContext { } fn source_metrics(&self) -> Arc { - self.env.source_metrics() + self.session.env().source_metrics() } fn store_mem_usage(&self, _val: usize) { From a256378b6d164976ade6c1263634581077cb2ebc Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Sun, 18 Feb 2024 18:31:31 +0800 Subject: [PATCH 4/4] refactor(meta): merge periodic and scheduled barrier and non-async handle barrier (#14977) --- src/meta/service/src/scale_service.rs | 4 +- src/meta/src/barrier/mod.rs | 66 ++++------- src/meta/src/barrier/notifier.rs | 8 +- src/meta/src/barrier/recovery.rs | 20 ++-- src/meta/src/barrier/rpc.rs | 74 ++++++++---- src/meta/src/barrier/schedule.rs | 156 ++++++++++++++------------ src/meta/src/rpc/ddl_controller.rs | 2 +- src/meta/src/stream/stream_manager.rs | 6 +- 8 files changed, 176 insertions(+), 160 deletions(-) diff --git a/src/meta/service/src/scale_service.rs b/src/meta/service/src/scale_service.rs index 33899856a57bc..f36a3c3ec0f0e 100644 --- a/src/meta/service/src/scale_service.rs +++ b/src/meta/service/src/scale_service.rs @@ -134,7 +134,7 @@ impl ScaleService for ScaleServiceImpl { &self, request: Request, ) -> Result, Status> { - self.barrier_manager.check_status_running().await?; + self.barrier_manager.check_status_running()?; let RescheduleRequest { reschedules, @@ -228,7 +228,7 @@ impl ScaleService for ScaleServiceImpl { &self, request: Request, ) -> Result, Status> { - self.barrier_manager.check_status_running().await?; + self.barrier_manager.check_status_running()?; let req = request.into_inner(); diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 3bad3cbd15a2d..b13dc19ef61c6 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -19,6 +19,7 @@ use std::mem::take; use std::sync::Arc; use std::time::Duration; +use arc_swap::ArcSwap; use fail::fail_point; use itertools::Itertools; use prometheus::HistogramTimer; @@ -137,7 +138,7 @@ struct Scheduled { #[derive(Clone)] pub struct GlobalBarrierManagerContext { - status: Arc>, + status: Arc>, tracker: Arc>, @@ -407,7 +408,7 @@ impl GlobalBarrierManager { let tracker = CreateMviewProgressTracker::new(); let context = GlobalBarrierManagerContext { - status: Arc::new(Mutex::new(BarrierManagerStatus::Starting)), + status: Arc::new(ArcSwap::new(Arc::new(BarrierManagerStatus::Starting))), metadata_manager, hummock_manager, source_manager, @@ -482,6 +483,7 @@ impl GlobalBarrierManager { let interval = Duration::from_millis( self.env.system_params_reader().await.barrier_interval_ms() as u64, ); + self.scheduled_barriers.set_min_interval(interval); tracing::info!( "Starting barrier manager with: interval={:?}, enable_recovery={}, in_flight_barrier_nums={}", interval, @@ -519,8 +521,7 @@ impl GlobalBarrierManager { // Even if there's no actor to recover, we still go through the recovery process to // inject the first `Initial` barrier. self.context - .set_status(BarrierManagerStatus::Recovering(RecoveryReason::Bootstrap)) - .await; + .set_status(BarrierManagerStatus::Recovering(RecoveryReason::Bootstrap)); let span = tracing::info_span!("bootstrap_recovery", prev_epoch = prev_epoch.value().0); let paused = self.take_pause_on_bootstrap().await.unwrap_or(false); @@ -531,10 +532,8 @@ impl GlobalBarrierManager { .await; } - self.context.set_status(BarrierManagerStatus::Running).await; + self.context.set_status(BarrierManagerStatus::Running); - let mut min_interval = tokio::time::interval(interval); - min_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); let (local_notification_tx, mut local_notification_rx) = tokio::sync::mpsc::unbounded_channel(); self.env @@ -610,11 +609,7 @@ impl GlobalBarrierManager { let notification = notification.unwrap(); // Handle barrier interval and checkpoint frequency changes if let LocalNotification::SystemParamsChange(p) = ¬ification { - let new_interval = Duration::from_millis(p.barrier_interval_ms() as u64); - if new_interval != min_interval.period() { - min_interval = tokio::time::interval(new_interval); - min_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); - } + self.scheduled_barriers.set_min_interval(Duration::from_millis(p.barrier_interval_ms() as u64)); self.scheduled_barriers .set_checkpoint_frequency(p.checkpoint_frequency() as usize) } @@ -626,15 +621,11 @@ impl GlobalBarrierManager { ) .await; } - - // There's barrier scheduled. - _ = self.scheduled_barriers.wait_one(), if self.checkpoint_control.can_inject_barrier(self.in_flight_barrier_nums) => { - min_interval.reset(); // Reset the interval as we have a new barrier. - self.handle_new_barrier().await; - } - // Minimum interval reached. - _ = min_interval.tick(), if self.checkpoint_control.can_inject_barrier(self.in_flight_barrier_nums) => { - self.handle_new_barrier().await; + scheduled = self.scheduled_barriers.next_barrier(), + if self + .checkpoint_control + .can_inject_barrier(self.in_flight_barrier_nums) => { + self.handle_new_barrier(scheduled); } } self.checkpoint_control.update_barrier_nums_metrics(); @@ -642,18 +633,14 @@ impl GlobalBarrierManager { } /// Handle the new barrier from the scheduled queue and inject it. - async fn handle_new_barrier(&mut self) { - assert!(self - .checkpoint_control - .can_inject_barrier(self.in_flight_barrier_nums)); - + fn handle_new_barrier(&mut self, scheduled: Scheduled) { let Scheduled { command, mut notifiers, send_latency_timer, checkpoint, span, - } = self.scheduled_barriers.pop_or_default().await; + } = scheduled; let info = self.state.apply_command(&command); @@ -676,15 +663,12 @@ impl GlobalBarrierManager { command, kind, self.context.clone(), - span.clone(), + span, )); send_latency_timer.observe_duration(); - self.rpc_manager - .inject_barrier(command_ctx.clone()) - .instrument(span) - .await; + self.rpc_manager.inject_barrier(command_ctx.clone()); // Notify about the injection. let prev_paused_reason = self.state.paused_reason(); @@ -696,7 +680,7 @@ impl GlobalBarrierManager { prev_paused_reason, curr_paused_reason, }; - notifiers.iter_mut().for_each(|n| n.notify_injected(info)); + notifiers.iter_mut().for_each(|n| n.notify_started(info)); // Update the paused state after the barrier is injected. self.state.set_paused_reason(curr_paused_reason); @@ -775,8 +759,7 @@ impl GlobalBarrierManager { self.context .set_status(BarrierManagerStatus::Recovering(RecoveryReason::Failover( err.clone(), - ))) - .await; + ))); let latest_snapshot = self.context.hummock_manager.latest_snapshot(); let prev_epoch = TracedEpoch::new(latest_snapshot.committed_epoch.into()); // we can only recovery from the committed epoch let span = tracing::info_span!( @@ -788,7 +771,7 @@ impl GlobalBarrierManager { // No need to clean dirty tables for barrier recovery, // The foreground stream job should cleanup their own tables. self.recovery(prev_epoch, None).instrument(span).await; - self.context.set_status(BarrierManagerStatus::Running).await; + self.context.set_status(BarrierManagerStatus::Running); } else { panic!("failed to execute barrier: {}", err.as_report()); } @@ -940,9 +923,9 @@ impl GlobalBarrierManager { impl GlobalBarrierManagerContext { /// Check the status of barrier manager, return error if it is not `Running`. - pub async fn check_status_running(&self) -> MetaResult<()> { - let status = self.status.lock().await; - match &*status { + pub fn check_status_running(&self) -> MetaResult<()> { + let status = self.status.load(); + match &**status { BarrierManagerStatus::Starting | BarrierManagerStatus::Recovering(RecoveryReason::Bootstrap) => { bail!("The cluster is bootstrapping") @@ -955,9 +938,8 @@ impl GlobalBarrierManagerContext { } /// Set barrier manager status. - async fn set_status(&self, new_status: BarrierManagerStatus) { - let mut status = self.status.lock().await; - *status = new_status; + fn set_status(&self, new_status: BarrierManagerStatus) { + self.status.store(Arc::new(new_status)); } /// Resolve actor information from cluster, fragment manager and `ChangedTableId`. diff --git a/src/meta/src/barrier/notifier.rs b/src/meta/src/barrier/notifier.rs index 1675544d7a410..e142e9be514cf 100644 --- a/src/meta/src/barrier/notifier.rs +++ b/src/meta/src/barrier/notifier.rs @@ -31,8 +31,8 @@ pub struct BarrierInfo { /// Used for notifying the status of a scheduled command/barrier. #[derive(Debug, Default)] pub(crate) struct Notifier { - /// Get notified when scheduled barrier is injected to compute nodes. - pub injected: Option>, + /// Get notified when scheduled barrier has started to be handled. + pub started: Option>, /// Get notified when scheduled barrier is collected or failed. pub collected: Option>>, @@ -43,8 +43,8 @@ pub(crate) struct Notifier { impl Notifier { /// Notify when we have injected a barrier to compute nodes. - pub fn notify_injected(&mut self, info: BarrierInfo) { - if let Some(tx) = self.injected.take() { + pub fn notify_started(&mut self, info: BarrierInfo) { + if let Some(tx) = self.started.take() { tx.send(info).ok(); } } diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 858fa937044dd..3f7f3911e7302 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -301,10 +301,7 @@ impl GlobalBarrierManagerContext { impl GlobalBarrierManager { /// Pre buffered drop and cancel command, return true if any. async fn pre_apply_drop_cancel(&self) -> MetaResult { - let (dropped_actors, cancelled) = self - .scheduled_barriers - .pre_apply_drop_cancel_scheduled() - .await; + let (dropped_actors, cancelled) = self.scheduled_barriers.pre_apply_drop_cancel_scheduled(); let applied = !dropped_actors.is_empty() || !cancelled.is_empty(); if !cancelled.is_empty() { match &self.context.metadata_manager { @@ -335,8 +332,7 @@ impl GlobalBarrierManager { pub async fn recovery(&mut self, prev_epoch: TracedEpoch, paused_reason: Option) { // Mark blocked and abort buffered schedules, they might be dirty already. self.scheduled_barriers - .abort_and_mark_blocked("cluster is under recovering") - .await; + .abort_and_mark_blocked("cluster is under recovering"); tracing::info!("recovery start!"); self.context @@ -465,9 +461,13 @@ impl GlobalBarrierManager { command_ctx.wait_epoch_commit(mce).await?; } }; - let await_barrier_complete = - self.context.inject_barrier(command_ctx.clone()).await; - let res = match await_barrier_complete.await.result { + + let res = match self + .context + .inject_barrier(command_ctx.clone(), None, None) + .await + .result + { Ok(response) => { if let Err(err) = command_ctx.post_collect().await { warn!(error = %err.as_report(), "post_collect failed"); @@ -499,7 +499,7 @@ impl GlobalBarrierManager { .expect("Retry until recovery success."); recovery_timer.observe_duration(); - self.scheduled_barriers.mark_ready().await; + self.scheduled_barriers.mark_ready(); tracing::info!( epoch = state.in_flight_prev_epoch().value().0, diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 670ee7cf10929..877f935f25207 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -35,6 +35,7 @@ use risingwave_rpc_client::error::RpcError; use risingwave_rpc_client::StreamClient; use rw_futures_util::pending_on_none; use tokio::sync::oneshot; +use tracing::Instrument; use uuid::Uuid; use super::command::CommandContext; @@ -47,6 +48,8 @@ pub(super) struct BarrierRpcManager { /// Futures that await on the completion of barrier. injected_in_progress_barrier: FuturesUnordered, + + prev_injecting_barrier: Option>, } impl BarrierRpcManager { @@ -54,15 +57,24 @@ impl BarrierRpcManager { Self { context, injected_in_progress_barrier: FuturesUnordered::new(), + prev_injecting_barrier: None, } } pub(super) fn clear(&mut self) { self.injected_in_progress_barrier = FuturesUnordered::new(); + self.prev_injecting_barrier = None; } - pub(super) async fn inject_barrier(&mut self, command_context: Arc) { - let await_complete_future = self.context.inject_barrier(command_context).await; + pub(super) fn inject_barrier(&mut self, command_context: Arc) { + // this is to notify that the barrier has been injected so that the next + // barrier can be injected to avoid out of order barrier injection. + // TODO: can be removed when bidi-stream control in implemented. + let (inject_tx, inject_rx) = oneshot::channel(); + let prev_inject_rx = self.prev_injecting_barrier.replace(inject_rx); + let await_complete_future = + self.context + .inject_barrier(command_context, Some(inject_tx), prev_inject_rx); self.injected_in_progress_barrier .push(await_complete_future); } @@ -76,35 +88,49 @@ pub(super) type BarrierCompletionFuture = impl Future, + inject_tx: Option>, + prev_inject_rx: Option>, ) -> BarrierCompletionFuture { let (tx, rx) = oneshot::channel(); let prev_epoch = command_context.prev_epoch.value().0; - let result = self - .stream_rpc_manager - .inject_barrier(command_context.clone()) - .await; - match result { - Ok(node_need_collect) => { - // todo: the collect handler should be abort when recovery. - tokio::spawn({ - let stream_rpc_manager = self.stream_rpc_manager.clone(); - async move { - stream_rpc_manager - .collect_barrier(node_need_collect, command_context, tx) - .await - } - }); + let stream_rpc_manager = self.stream_rpc_manager.clone(); + // todo: the collect handler should be abort when recovery. + let _join_handle = tokio::spawn(async move { + let span = command_context.span.clone(); + if let Some(prev_inject_rx) = prev_inject_rx { + if prev_inject_rx.await.is_err() { + let _ = tx.send(BarrierCompletion { + prev_epoch, + result: Err(anyhow!("prev barrier failed to be injected").into()), + }); + return; + } } - Err(e) => { - let _ = tx.send(BarrierCompletion { - prev_epoch, - result: Err(e), - }); + let result = stream_rpc_manager + .inject_barrier(command_context.clone()) + .instrument(span.clone()) + .await; + match result { + Ok(node_need_collect) => { + if let Some(inject_tx) = inject_tx { + let _ = inject_tx.send(()); + } + stream_rpc_manager + .collect_barrier(node_need_collect, command_context, tx) + .instrument(span.clone()) + .await; + } + Err(e) => { + let _ = tx.send(BarrierCompletion { + prev_epoch, + result: Err(e), + }); + } } - } + }); rx.map(move |result| match result { Ok(completion) => completion, Err(_e) => BarrierCompletion { diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 56152c18baa70..c481b756b1828 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -14,16 +14,18 @@ use std::collections::{HashSet, VecDeque}; use std::iter::once; -use std::sync::atomic::{AtomicBool, AtomicUsize, Ordering}; use std::sync::Arc; -use std::time::Instant; +use std::time::{Duration, Instant}; use anyhow::{anyhow, Context}; use assert_matches::assert_matches; +use parking_lot::Mutex; use risingwave_common::catalog::TableId; use risingwave_pb::hummock::HummockSnapshot; use risingwave_pb::meta::PausedReason; -use tokio::sync::{oneshot, watch, RwLock}; +use tokio::select; +use tokio::sync::{oneshot, watch}; +use tokio::time::Interval; use super::notifier::{BarrierInfo, Notifier}; use super::{Command, Scheduled}; @@ -37,19 +39,11 @@ use crate::{MetaError, MetaResult}; /// We manually implement one here instead of using channels since we may need to update the front /// of the queue to add some notifiers for instant flushes. struct Inner { - queue: RwLock, + queue: Mutex, /// When `queue` is not empty anymore, all subscribers of this watcher will be notified. changed_tx: watch::Sender<()>, - /// The numbers of barrier (checkpoint = false) since the last barrier (checkpoint = true) - num_uncheckpointed_barrier: AtomicUsize, - - /// Force checkpoint in next barrier. - force_checkpoint: AtomicBool, - - checkpoint_frequency: AtomicUsize, - /// Used for recording send latency of each barrier. metrics: Arc, } @@ -62,7 +56,7 @@ enum QueueStatus { Blocked(String), } -struct ScheduledQueue { +pub(super) struct ScheduledQueue { queue: VecDeque, status: QueueStatus, } @@ -154,11 +148,8 @@ impl BarrierScheduler { checkpoint_frequency, ); let inner = Arc::new(Inner { - queue: RwLock::new(ScheduledQueue::new()), + queue: Mutex::new(ScheduledQueue::new()), changed_tx: watch::channel(()).0, - num_uncheckpointed_barrier: AtomicUsize::new(0), - checkpoint_frequency: AtomicUsize::new(checkpoint_frequency), - force_checkpoint: AtomicBool::new(false), metrics, }); @@ -167,13 +158,19 @@ impl BarrierScheduler { inner: inner.clone(), hummock_manager, }, - ScheduledBarriers { inner }, + ScheduledBarriers { + num_uncheckpointed_barrier: 0, + force_checkpoint: false, + checkpoint_frequency, + inner, + min_interval: None, + }, ) } /// Push a scheduled barrier into the queue. - async fn push(&self, scheduleds: impl IntoIterator) -> MetaResult<()> { - let mut queue = self.inner.queue.write().await; + fn push(&self, scheduleds: impl IntoIterator) -> MetaResult<()> { + let mut queue = self.inner.queue.lock(); for scheduled in scheduleds { queue.push_back(scheduled)?; if queue.len() == 1 { @@ -184,8 +181,8 @@ impl BarrierScheduler { } /// Try to cancel scheduled cmd for create streaming job, return true if cancelled. - pub async fn try_cancel_scheduled_create(&self, table_id: TableId) -> bool { - let queue = &mut self.inner.queue.write().await; + pub fn try_cancel_scheduled_create(&self, table_id: TableId) -> bool { + let queue = &mut self.inner.queue.lock(); if let Some(idx) = queue.queue.iter().position(|scheduled| { if let Command::CreateStreamingJob { table_fragments, .. @@ -207,12 +204,12 @@ impl BarrierScheduler { /// Attach `new_notifiers` to the very first scheduled barrier. If there's no one scheduled, a /// default barrier will be created. If `new_checkpoint` is true, the barrier will become a /// checkpoint. - async fn attach_notifiers( + fn attach_notifiers( &self, new_notifiers: Vec, new_checkpoint: bool, ) -> MetaResult<()> { - let mut queue = self.inner.queue.write().await; + let mut queue = self.inner.queue.lock(); match queue.queue.front_mut() { Some(Scheduled { notifiers, @@ -243,7 +240,7 @@ impl BarrierScheduler { collected: Some(tx), ..Default::default() }; - self.attach_notifiers(vec![notifier], checkpoint).await?; + self.attach_notifiers(vec![notifier], checkpoint)?; rx.await.unwrap() } @@ -258,23 +255,23 @@ impl BarrierScheduler { let mut scheduleds = Vec::with_capacity(commands.len()); for command in commands { - let (injected_tx, injected_rx) = oneshot::channel(); + let (started_tx, started_rx) = oneshot::channel(); let (collect_tx, collect_rx) = oneshot::channel(); let (finish_tx, finish_rx) = oneshot::channel(); - contexts.push((injected_rx, collect_rx, finish_rx)); + contexts.push((started_rx, collect_rx, finish_rx)); scheduleds.push(self.inner.new_scheduled( command.need_checkpoint(), command, once(Notifier { - injected: Some(injected_tx), + started: Some(started_tx), collected: Some(collect_tx), finished: Some(finish_tx), }), )); } - self.push(scheduleds).await?; + self.push(scheduleds)?; let mut infos = Vec::with_capacity(contexts.len()); @@ -340,21 +337,42 @@ impl BarrierScheduler { /// The receiver side of the barrier scheduling queue. /// Held by the [`super::GlobalBarrierManager`] to execute these commands. pub struct ScheduledBarriers { + min_interval: Option, + + /// Force checkpoint in next barrier. + force_checkpoint: bool, + + /// The numbers of barrier (checkpoint = false) since the last barrier (checkpoint = true) + num_uncheckpointed_barrier: usize, + checkpoint_frequency: usize, inner: Arc, } impl ScheduledBarriers { - /// Pop a scheduled barrier from the queue, or a default checkpoint barrier if not exists. - pub(super) async fn pop_or_default(&self) -> Scheduled { - let mut queue = self.inner.queue.write().await; + pub(super) fn set_min_interval(&mut self, min_interval: Duration) { + let set_new_interval = match &self.min_interval { + None => true, + Some(prev_min_interval) => min_interval != prev_min_interval.period(), + }; + if set_new_interval { + let mut min_interval = tokio::time::interval(min_interval); + min_interval.set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + self.min_interval = Some(min_interval); + } + } + + pub(super) async fn next_barrier(&mut self) -> Scheduled { let checkpoint = self.try_get_checkpoint(); - let scheduled = match queue.queue.pop_front() { - Some(mut scheduled) => { + let scheduled = select! { + biased; + mut scheduled = self.inner.next_scheduled() => { + if let Some(min_interval) = &mut self.min_interval { + min_interval.reset(); + } scheduled.checkpoint = scheduled.checkpoint || checkpoint; scheduled - } - None => { - // If no command scheduled, create a periodic barrier by default. + }, + _ = self.min_interval.as_mut().expect("should have set min interval").tick() => { self.inner .new_scheduled(checkpoint, Command::barrier(), std::iter::empty()) } @@ -362,23 +380,28 @@ impl ScheduledBarriers { self.update_num_uncheckpointed_barrier(scheduled.checkpoint); scheduled } +} - /// Wait for at least one scheduled barrier in the queue. - pub(super) async fn wait_one(&self) { - let queue = self.inner.queue.read().await; - if queue.len() > 0 { - return; +impl Inner { + async fn next_scheduled(&self) -> Scheduled { + loop { + let mut rx = self.changed_tx.subscribe(); + { + let mut queue = self.queue.lock(); + if let Some(scheduled) = queue.queue.pop_front() { + break scheduled; + } + } + rx.changed().await.unwrap(); } - let mut rx = self.inner.changed_tx.subscribe(); - drop(queue); - - rx.changed().await.unwrap(); } +} +impl ScheduledBarriers { /// Mark command scheduler as blocked and abort all queued scheduled command and notify with /// specific reason. - pub(super) async fn abort_and_mark_blocked(&self, reason: impl Into + Copy) { - let mut queue = self.inner.queue.write().await; + pub(super) fn abort_and_mark_blocked(&self, reason: impl Into + Copy) { + let mut queue = self.inner.queue.lock(); queue.mark_blocked(reason.into()); while let Some(Scheduled { notifiers, .. }) = queue.queue.pop_front() { notifiers @@ -388,15 +411,15 @@ impl ScheduledBarriers { } /// Mark command scheduler as ready to accept new command. - pub(super) async fn mark_ready(&self) { - let mut queue = self.inner.queue.write().await; + pub(super) fn mark_ready(&self) { + let mut queue = self.inner.queue.lock(); queue.mark_ready(); } /// Try to pre apply drop and cancel scheduled command and return them if any. /// It should only be called in recovery. - pub(super) async fn pre_apply_drop_cancel_scheduled(&self) -> (Vec, HashSet) { - let mut queue = self.inner.queue.write().await; + pub(super) fn pre_apply_drop_cancel_scheduled(&self) -> (Vec, HashSet) { + let mut queue = self.inner.queue.lock(); assert_matches!(queue.status, QueueStatus::Blocked(_)); let (mut drop_table_ids, mut cancel_table_ids) = (vec![], HashSet::new()); @@ -426,37 +449,26 @@ impl ScheduledBarriers { /// Whether the barrier(checkpoint = true) should be injected. fn try_get_checkpoint(&self) -> bool { - self.inner - .num_uncheckpointed_barrier - .load(Ordering::Relaxed) - + 1 - >= self.inner.checkpoint_frequency.load(Ordering::Relaxed) - || self.inner.force_checkpoint.load(Ordering::Relaxed) + self.num_uncheckpointed_barrier + 1 >= self.checkpoint_frequency || self.force_checkpoint } /// Make the `checkpoint` of the next barrier must be true - pub fn force_checkpoint_in_next_barrier(&self) { - self.inner.force_checkpoint.store(true, Ordering::Relaxed) + pub fn force_checkpoint_in_next_barrier(&mut self) { + self.force_checkpoint = true; } /// Update the `checkpoint_frequency` - pub fn set_checkpoint_frequency(&self, frequency: usize) { - self.inner - .checkpoint_frequency - .store(frequency, Ordering::Relaxed); + pub fn set_checkpoint_frequency(&mut self, frequency: usize) { + self.checkpoint_frequency = frequency; } /// Update the `num_uncheckpointed_barrier` - fn update_num_uncheckpointed_barrier(&self, checkpoint: bool) { + fn update_num_uncheckpointed_barrier(&mut self, checkpoint: bool) { if checkpoint { - self.inner - .num_uncheckpointed_barrier - .store(0, Ordering::Relaxed); - self.inner.force_checkpoint.store(false, Ordering::Relaxed); + self.num_uncheckpointed_barrier = 0; + self.force_checkpoint = false; } else { - self.inner - .num_uncheckpointed_barrier - .fetch_add(1, Ordering::Relaxed); + self.num_uncheckpointed_barrier += 1; } } } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 7de106523599e..25b6ed25464a3 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -269,7 +269,7 @@ impl DdlController { /// would be a huge hassle and pain if we don't spawn here. pub async fn run_command(&self, command: DdlCommand) -> MetaResult { if !command.allow_in_recovery() { - self.barrier_manager.check_status_running().await?; + self.barrier_manager.check_status_running()?; } let ctrl = self.clone(); let fut = async move { diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index d7388f50da09c..e0f4f8cb82b53 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -284,11 +284,7 @@ impl GlobalStreamManager { .await { // try to cancel buffered creating command. - if self - .barrier_scheduler - .try_cancel_scheduled_create(table_id) - .await - { + if self.barrier_scheduler.try_cancel_scheduled_create(table_id) { tracing::debug!( "cancelling streaming job {table_id} in buffer queue." );