risingwave_frontend/catalog/system_catalog/pg_catalog/
pg_constraint.rs1use risingwave_common::types::Fields;
16use risingwave_frontend_macro::system_catalog;
17
18use crate::TableCatalog;
19use crate::catalog::schema_catalog::SchemaCatalog;
20use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemTableCatalog};
21use crate::error::Result;
22
23#[derive(Fields)]
27struct PgConstraint {
28 #[primary_key]
29 oid: i32,
30 conname: String,
31 connamespace: i32,
32 contype: String,
33 condeferrable: bool,
34 convalidated: bool,
35 conrelid: i32,
36 contypid: i32,
37 conindid: i32,
38 conparentid: i32,
39 confrelid: i32,
40 confupdtype: String,
41 confdeltype: String,
42 confmatchtype: String,
43 conislocal: bool,
44 coninhcount: i32,
45 connoinherit: bool,
46 conkey: Option<Vec<i16>>,
47 confkey: Option<Vec<i16>>,
48 conpfeqop: Option<Vec<i32>>,
49 conppeqop: Option<Vec<i32>>,
50 conffeqop: Option<Vec<i32>>,
51 confdelsetcols: Option<Vec<i16>>,
52 conexclop: Option<Vec<i32>>,
53 conbin: Option<String>,
54}
55
56impl PgConstraint {
57 fn from_system_table(schema: &SchemaCatalog, table: &SystemTableCatalog) -> PgConstraint {
58 let conkey: Vec<_> = table.pk.iter().map(|i| (*i + 1) as i16).collect();
60 PgConstraint {
61 oid: table.id.table_id() as i32, conname: format!("{}_pkey", &table.name),
63 connamespace: schema.id() as i32,
64 contype: "p".to_owned(), condeferrable: false,
66 convalidated: true,
67 conrelid: table.id.table_id() as i32,
68 contypid: 0,
69 conindid: table.id.table_id() as i32,
71 conparentid: 0,
72 confrelid: 0,
73 confupdtype: " ".to_owned(),
74 confdeltype: " ".to_owned(),
75 confmatchtype: " ".to_owned(),
76 conislocal: true,
77 coninhcount: 0,
78 connoinherit: true,
79 conkey: Some(conkey),
80 confkey: None,
81 conpfeqop: None,
82 conppeqop: None,
83 conffeqop: None,
84 confdelsetcols: None,
85 conexclop: None,
86 conbin: None,
87 }
88 }
89
90 fn from_table(schema: &SchemaCatalog, table: &TableCatalog) -> PgConstraint {
91 let conkey: Vec<_> = table
93 .pk
94 .iter()
95 .map(|i| (i.column_index + 1) as i16)
96 .collect();
97 PgConstraint {
98 oid: table.id.table_id() as i32, conname: format!("{}_pkey", &table.name),
100 connamespace: schema.id() as i32,
101 contype: "p".to_owned(), condeferrable: false,
103 convalidated: true,
104 conrelid: table.id.table_id() as i32,
105 contypid: 0,
106 conindid: table.id.table_id() as i32,
108 conparentid: 0,
109 confrelid: 0,
110 confupdtype: " ".to_owned(),
111 confdeltype: " ".to_owned(),
112 confmatchtype: " ".to_owned(),
113 conislocal: true,
114 coninhcount: 0,
115 connoinherit: true,
116 conkey: Some(conkey),
117 confkey: None,
118 conpfeqop: None,
119 conppeqop: None,
120 conffeqop: None,
121 confdelsetcols: None,
122 conexclop: None,
123 conbin: None,
124 }
125 }
126}
127
128#[system_catalog(table, "pg_catalog.pg_constraint")]
129fn read_pg_constraint(reader: &SysCatalogReaderImpl) -> Result<Vec<PgConstraint>> {
130 let catalog_reader = reader.catalog_reader.read_guard();
131 let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?;
132
133 Ok(schemas.flat_map(read_pg_constraint_in_schema).collect())
134}
135
136fn read_pg_constraint_in_schema(schema: &SchemaCatalog) -> Vec<PgConstraint> {
137 let system_table_rows = schema
139 .iter_system_tables()
140 .map(|table| PgConstraint::from_system_table(schema, table.as_ref()));
141
142 let table_rows = schema
143 .iter_table_mv_indices()
144 .map(|table| PgConstraint::from_table(schema, table.as_ref()));
145
146 system_table_rows.chain(table_rows).collect()
147}