risingwave_frontend/catalog/system_catalog/pg_catalog/
pg_class.rs

1// Copyright 2022 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use risingwave_common::types::Fields;
16use risingwave_frontend_macro::system_catalog;
17use risingwave_pb::id::{ObjectId, SchemaId, UserId};
18
19use crate::catalog::OwnedByUserCatalog;
20use crate::catalog::system_catalog::SysCatalogReaderImpl;
21use crate::error::Result;
22
23/// The catalog `pg_class` catalogs tables and most everything else that has columns or is otherwise
24/// similar to a table. Ref: `https://www.postgresql.org/docs/current/catalog-pg-class.html`
25#[derive(Fields)]
26struct PgClass {
27    #[primary_key]
28    oid: ObjectId,
29    relname: String,
30    relnamespace: SchemaId,
31    relowner: UserId,
32    // p = permanent table, u = unlogged table, t = temporary table
33    relpersistence: String,
34    // r = ordinary table, i = index, S = sequence, t = TOAST table, v = view, m = materialized view,
35    // c = composite type, f = foreign table, p = partitioned table, I = partitioned index
36    relkind: String,
37    relpages: i16,
38    relam: i32,
39    reltablespace: i32,
40    reloptions: Vec<String>,
41    relispartition: bool,
42    // PG uses pg_node_tree type but RW doesn't support it
43    relpartbound: Option<String>,
44}
45
46#[system_catalog(table, "pg_catalog.pg_class")]
47fn read_pg_class_info(reader: &SysCatalogReaderImpl) -> Result<Vec<PgClass>> {
48    let catalog_reader = reader.catalog_reader.read_guard();
49    let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?;
50
51    Ok(schemas
52        .flat_map(|schema| {
53            schema
54                .iter_user_table()
55                .map(|table| PgClass {
56                    oid: table.id.as_object_id(),
57                    relname: table.name.clone(),
58                    relnamespace: table.schema_id,
59                    relowner: table.owner,
60                    relpersistence: "p".to_owned(),
61                    relkind: "r".to_owned(),
62                    relpages: 0,
63                    relam: 0,
64                    reltablespace: 0,
65                    reloptions: vec![],
66                    relispartition: false,
67                    relpartbound: None,
68                })
69                .chain(schema.iter_all_mvs().map(|mview| PgClass {
70                    oid: mview.id.as_object_id(),
71                    relname: mview.name.clone(),
72                    relnamespace: mview.schema_id,
73                    relowner: mview.owner,
74                    relpersistence: "p".to_owned(),
75                    relkind: "m".to_owned(),
76                    relpages: 0,
77                    relam: 0,
78                    reltablespace: 0,
79                    reloptions: vec![],
80                    relispartition: false,
81                    relpartbound: None,
82                }))
83                .chain(schema.iter_system_tables().map(|table| PgClass {
84                    oid: table.id.as_object_id(),
85                    relname: table.name.clone(),
86                    relnamespace: schema.id(),
87                    relowner: table.owner,
88                    relpersistence: "p".to_owned(),
89                    relkind: "r".to_owned(),
90                    relpages: 0,
91                    relam: 0,
92                    reltablespace: 0,
93                    reloptions: vec![],
94                    relispartition: false,
95                    relpartbound: None,
96                }))
97                .chain(schema.iter_index().map(|index| PgClass {
98                    oid: index.id.as_object_id(),
99                    relname: index.name.clone(),
100                    relnamespace: schema.id(),
101                    relowner: index.owner(),
102                    relpersistence: "p".to_owned(),
103                    relkind: "i".to_owned(),
104                    relpages: 0,
105                    relam: 0,
106                    reltablespace: 0,
107                    reloptions: vec![],
108                    relispartition: false,
109                    relpartbound: None,
110                }))
111                .chain(schema.iter_view().map(|view| PgClass {
112                    oid: view.id.as_object_id(),
113                    relname: view.name.clone(),
114                    relnamespace: schema.id(),
115                    relowner: view.owner,
116                    relpersistence: "p".to_owned(),
117                    relkind: "v".to_owned(),
118                    relpages: 0,
119                    relam: 0,
120                    reltablespace: 0,
121                    reloptions: vec![],
122                    relispartition: false,
123                    relpartbound: None,
124                }))
125                .chain(schema.iter_source().map(|source| PgClass {
126                    oid: source.id.as_object_id(),
127                    relname: source.name.clone(),
128                    relnamespace: schema.id(),
129                    relowner: source.owner,
130                    relpersistence: "p".to_owned(),
131                    relkind: "s".to_owned(), // s for the source in rw.
132                    relpages: 0,
133                    relam: 0,
134                    reltablespace: 0,
135                    reloptions: vec![],
136                    relispartition: false,
137                    relpartbound: None,
138                }))
139                .chain(schema.iter_sink().map(|sink| PgClass {
140                    oid: sink.id.as_object_id(),
141                    relname: sink.name.clone(),
142                    relnamespace: schema.id(),
143                    relowner: sink.owner,
144                    relpersistence: "p".to_owned(),
145                    relkind: "k".to_owned(), // k for the sink in rw.
146                    relpages: 0,
147                    relam: 0,
148                    reltablespace: 0,
149                    reloptions: vec![],
150                    relispartition: false,
151                    relpartbound: None,
152                }))
153                .chain(schema.iter_subscription().map(|subscription| PgClass {
154                    oid: subscription.id.as_object_id(),
155                    relname: subscription.name.clone(),
156                    relnamespace: schema.id(),
157                    relowner: subscription.owner,
158                    relpersistence: "p".to_owned(),
159                    relkind: "u".to_owned(), // u for the subscription in rw.
160                    relpages: 0,
161                    relam: 0,
162                    reltablespace: 0,
163                    reloptions: vec![],
164                    relispartition: false,
165                    relpartbound: None,
166                }))
167                .chain(schema.iter_connections().map(|connection| PgClass {
168                    oid: connection.id.as_object_id(),
169                    relname: connection.name.clone(),
170                    relnamespace: schema.id(),
171                    relowner: connection.owner,
172                    relpersistence: "p".to_owned(),
173                    relkind: "c".to_owned(), // c for the connection in rw.
174                    relpages: 0,
175                    relam: 0,
176                    reltablespace: 0,
177                    reloptions: vec![],
178                    relispartition: false,
179                    relpartbound: None,
180                }))
181        })
182        .collect())
183}