risingwave_frontend/catalog/system_catalog/pg_catalog/
pg_class.rs1use risingwave_common::types::Fields;
16use risingwave_frontend_macro::system_catalog;
17
18use crate::catalog::OwnedByUserCatalog;
19use crate::catalog::system_catalog::SysCatalogReaderImpl;
20use crate::error::Result;
21
22#[derive(Fields)]
25struct PgClass {
26 #[primary_key]
27 oid: i32,
28 relname: String,
29 relnamespace: i32,
30 relowner: i32,
31 relpersistence: String,
33 relkind: String,
36 relpages: i16,
37 relam: i32,
38 reltablespace: i32,
39 reloptions: Vec<String>,
40 relispartition: bool,
41 relpartbound: Option<String>,
43}
44
45#[system_catalog(table, "pg_catalog.pg_class")]
46fn read_pg_class_info(reader: &SysCatalogReaderImpl) -> Result<Vec<PgClass>> {
47 let catalog_reader = reader.catalog_reader.read_guard();
48 let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?;
49
50 Ok(schemas
51 .flat_map(|schema| {
52 schema
53 .iter_user_table()
54 .map(|table| PgClass {
55 oid: table.id.table_id as i32,
56 relname: table.name.clone(),
57 relnamespace: table.schema_id as i32,
58 relowner: table.owner as i32,
59 relpersistence: "p".to_owned(),
60 relkind: "r".to_owned(),
61 relpages: 0,
62 relam: 0,
63 reltablespace: 0,
64 reloptions: vec![],
65 relispartition: false,
66 relpartbound: None,
67 })
68 .chain(schema.iter_all_mvs().map(|mview| PgClass {
69 oid: mview.id.table_id as i32,
70 relname: mview.name.clone(),
71 relnamespace: mview.schema_id as i32,
72 relowner: mview.owner as i32,
73 relpersistence: "p".to_owned(),
74 relkind: "m".to_owned(),
75 relpages: 0,
76 relam: 0,
77 reltablespace: 0,
78 reloptions: vec![],
79 relispartition: false,
80 relpartbound: None,
81 }))
82 .chain(schema.iter_system_tables().map(|table| PgClass {
83 oid: table.id.table_id as i32,
84 relname: table.name.clone(),
85 relnamespace: schema.id() as i32,
86 relowner: table.owner as i32,
87 relpersistence: "p".to_owned(),
88 relkind: "r".to_owned(),
89 relpages: 0,
90 relam: 0,
91 reltablespace: 0,
92 reloptions: vec![],
93 relispartition: false,
94 relpartbound: None,
95 }))
96 .chain(schema.iter_index().map(|index| PgClass {
97 oid: index.id.index_id as i32,
98 relname: index.name.clone(),
99 relnamespace: schema.id() as i32,
100 relowner: index.owner() as i32,
101 relpersistence: "p".to_owned(),
102 relkind: "i".to_owned(),
103 relpages: 0,
104 relam: 0,
105 reltablespace: 0,
106 reloptions: vec![],
107 relispartition: false,
108 relpartbound: None,
109 }))
110 .chain(schema.iter_view().map(|view| PgClass {
111 oid: view.id as i32,
112 relname: view.name.clone(),
113 relnamespace: schema.id() as i32,
114 relowner: view.owner as i32,
115 relpersistence: "p".to_owned(),
116 relkind: "v".to_owned(),
117 relpages: 0,
118 relam: 0,
119 reltablespace: 0,
120 reloptions: vec![],
121 relispartition: false,
122 relpartbound: None,
123 }))
124 .chain(schema.iter_source().map(|source| PgClass {
125 oid: source.id as i32,
126 relname: source.name.clone(),
127 relnamespace: schema.id() as i32,
128 relowner: source.owner as i32,
129 relpersistence: "p".to_owned(),
130 relkind: "s".to_owned(), relpages: 0,
132 relam: 0,
133 reltablespace: 0,
134 reloptions: vec![],
135 relispartition: false,
136 relpartbound: None,
137 }))
138 .chain(schema.iter_sink().map(|sink| PgClass {
139 oid: sink.id.sink_id as i32,
140 relname: sink.name.clone(),
141 relnamespace: schema.id() as i32,
142 relowner: sink.owner.user_id as i32,
143 relpersistence: "p".to_owned(),
144 relkind: "k".to_owned(), relpages: 0,
146 relam: 0,
147 reltablespace: 0,
148 reloptions: vec![],
149 relispartition: false,
150 relpartbound: None,
151 }))
152 .chain(schema.iter_subscription().map(|subscription| PgClass {
153 oid: subscription.id.subscription_id as i32,
154 relname: subscription.name.clone(),
155 relnamespace: schema.id() as i32,
156 relowner: subscription.owner.user_id as i32,
157 relpersistence: "p".to_owned(),
158 relkind: "u".to_owned(), relpages: 0,
160 relam: 0,
161 reltablespace: 0,
162 reloptions: vec![],
163 relispartition: false,
164 relpartbound: None,
165 }))
166 .chain(schema.iter_connections().map(|connection| PgClass {
167 oid: connection.id as i32,
168 relname: connection.name.clone(),
169 relnamespace: schema.id() as i32,
170 relowner: connection.owner as i32,
171 relpersistence: "p".to_owned(),
172 relkind: "c".to_owned(), relpages: 0,
174 relam: 0,
175 reltablespace: 0,
176 reloptions: vec![],
177 relispartition: false,
178 relpartbound: None,
179 }))
180 })
181 .collect())
182}