15
15
use risingwave_common:: types:: Fields ;
16
16
use risingwave_frontend_macro:: system_catalog;
17
17
18
+ use crate :: catalog:: schema_catalog:: SchemaCatalog ;
19
+ use crate :: catalog:: system_catalog:: { SysCatalogReaderImpl , SystemTableCatalog } ;
20
+ use crate :: error:: Result ;
21
+ use crate :: TableCatalog ;
22
+
18
23
/// The catalog `pg_constraint` records information about table and index inheritance hierarchies.
19
24
/// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-constraint.html`]
20
25
/// This is introduced only for pg compatibility and is not used in our system.
21
- #[ system_catalog( view, "pg_catalog.pg_constraint" ) ]
22
26
#[ derive( Fields ) ]
23
27
struct PgConstraint {
28
+ #[ primary_key]
24
29
oid : i32 ,
25
30
conname : String ,
26
31
connamespace : i32 ,
@@ -38,12 +43,105 @@ struct PgConstraint {
38
43
conislocal : bool ,
39
44
coninhcount : i32 ,
40
45
connoinherit : bool ,
41
- conkey : Vec < i16 > ,
42
- confkey : Vec < i16 > ,
43
- conpfeqop : Vec < i32 > ,
44
- conppeqop : Vec < i32 > ,
45
- conffeqop : Vec < i32 > ,
46
- confdelsetcols : Vec < i16 > ,
47
- conexclop : Vec < i32 > ,
48
- conbin : String ,
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
+
56
+ impl PgConstraint {
57
+ fn from_system_table ( schema : & SchemaCatalog , table : & SystemTableCatalog ) -> PgConstraint {
58
+ // List of the constrained columns. First column starts from 1.
59
+ let conkey: Vec < _ > = table. pk . iter ( ) . map ( |i| ( * i + 1 ) as i16 ) . collect ( ) ;
60
+ PgConstraint {
61
+ oid : table. id . table_id ( ) as i32 , // Use table_id as a mock oid of constraint here.
62
+ conname : format ! ( "{}_pkey" , & table. name) ,
63
+ connamespace : schema. id ( ) as i32 ,
64
+ contype : "p" . to_owned ( ) , // p = primary key constraint
65
+ condeferrable : false ,
66
+ convalidated : true ,
67
+ conrelid : table. id . table_id ( ) as i32 ,
68
+ contypid : 0 ,
69
+ // Use table_id as a mock index oid of constraint here.
70
+ 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
+ // List of the constrained columns. First column starts from 1.
92
+ 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 , // Use table_id as a mock oid of constraint here.
99
+ conname : format ! ( "{}_pkey" , & table. name) ,
100
+ connamespace : schema. id ( ) as i32 ,
101
+ contype : "p" . to_owned ( ) , // p = primary key constraint
102
+ condeferrable : false ,
103
+ convalidated : true ,
104
+ conrelid : table. id . table_id ( ) as i32 ,
105
+ contypid : 0 ,
106
+ // Use table_id as a mock index oid of constraint here.
107
+ 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" ) ]
129
+ fn 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
+
136
+ fn read_pg_constraint_in_schema ( schema : & SchemaCatalog ) -> Vec < PgConstraint > {
137
+ // Note: We only support primary key constraints now.
138
+ 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_valid_table ( )
144
+ . map ( |table| PgConstraint :: from_table ( schema, table. as_ref ( ) ) ) ;
145
+
146
+ system_table_rows. chain ( table_rows) . collect ( )
49
147
}
0 commit comments