risingwave_frontend/catalog/system_catalog/pg_catalog/pg_class.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168
// Copyright 2025 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 risingwave_common::types::Fields;
use risingwave_frontend_macro::system_catalog;
use crate::catalog::system_catalog::SysCatalogReaderImpl;
use crate::catalog::OwnedByUserCatalog;
use crate::error::Result;
/// 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`]
#[derive(Fields)]
struct PgClass {
#[primary_key]
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,
relpages: i16,
relam: i32,
reltablespace: i32,
reloptions: Vec<String>,
relispartition: bool,
// PG uses pg_node_tree type but RW doesn't support it
relpartbound: Option<String>,
}
#[system_catalog(table, "pg_catalog.pg_class")]
fn read_pg_class_info(reader: &SysCatalogReaderImpl) -> Result<Vec<PgClass>> {
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_user_table()
.map(|table| PgClass {
oid: table.id.table_id as i32,
relname: table.name.clone(),
relnamespace: table.schema_id as i32,
relowner: table.owner as i32,
relpersistence: "p".to_owned(),
relkind: "r".to_owned(),
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
})
.chain(schema.iter_all_mvs().map(|mview| PgClass {
oid: mview.id.table_id as i32,
relname: mview.name.clone(),
relnamespace: mview.schema_id as i32,
relowner: mview.owner as i32,
relpersistence: "p".to_owned(),
relkind: "m".to_owned(),
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
}))
.chain(schema.iter_system_tables().map(|table| PgClass {
oid: table.id.table_id as i32,
relname: table.name.clone(),
relnamespace: schema.id() as i32,
relowner: table.owner as i32,
relpersistence: "p".to_owned(),
relkind: "r".to_owned(),
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
}))
.chain(schema.iter_index().map(|index| PgClass {
oid: index.id.index_id as i32,
relname: index.name.clone(),
relnamespace: schema.id() as i32,
relowner: index.owner() as i32,
relpersistence: "p".to_owned(),
relkind: "i".to_owned(),
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
}))
.chain(schema.iter_view().map(|view| PgClass {
oid: view.id as i32,
relname: view.name.clone(),
relnamespace: schema.id() as i32,
relowner: view.owner as i32,
relpersistence: "p".to_owned(),
relkind: "v".to_owned(),
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
}))
.chain(schema.iter_source().map(|source| PgClass {
oid: source.id as i32,
relname: source.name.clone(),
relnamespace: schema.id() as i32,
relowner: source.owner as i32,
relpersistence: "p".to_owned(),
relkind: "s".to_owned(), // s for the source in rw.
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
}))
.chain(schema.iter_sink().map(|sink| PgClass {
oid: sink.id.sink_id as i32,
relname: sink.name.clone(),
relnamespace: schema.id() as i32,
relowner: sink.owner.user_id as i32,
relpersistence: "p".to_owned(),
relkind: "k".to_owned(), // k for the sink in rw.
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
}))
.chain(schema.iter_subscription().map(|subscription| PgClass {
oid: subscription.id.subscription_id as i32,
relname: subscription.name.clone(),
relnamespace: schema.id() as i32,
relowner: subscription.owner.user_id as i32,
relpersistence: "p".to_owned(),
relkind: "u".to_owned(), // u for the subscription in rw.
relpages: 0,
relam: 0,
reltablespace: 0,
reloptions: vec![],
relispartition: false,
relpartbound: None,
}))
})
.collect())
}