risingwave_frontend/catalog/system_catalog/pg_catalog/
pg_constraint.rsuse risingwave_common::types::Fields;
use risingwave_frontend_macro::system_catalog;
use crate::catalog::schema_catalog::SchemaCatalog;
use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemTableCatalog};
use crate::error::Result;
use crate::TableCatalog;
#[derive(Fields)]
struct PgConstraint {
#[primary_key]
oid: i32,
conname: String,
connamespace: i32,
contype: String,
condeferrable: bool,
convalidated: bool,
conrelid: i32,
contypid: i32,
conindid: i32,
conparentid: i32,
confrelid: i32,
confupdtype: String,
confdeltype: String,
confmatchtype: String,
conislocal: bool,
coninhcount: i32,
connoinherit: bool,
conkey: Option<Vec<i16>>,
confkey: Option<Vec<i16>>,
conpfeqop: Option<Vec<i32>>,
conppeqop: Option<Vec<i32>>,
conffeqop: Option<Vec<i32>>,
confdelsetcols: Option<Vec<i16>>,
conexclop: Option<Vec<i32>>,
conbin: Option<String>,
}
impl PgConstraint {
fn from_system_table(schema: &SchemaCatalog, table: &SystemTableCatalog) -> PgConstraint {
let conkey: Vec<_> = table.pk.iter().map(|i| (*i + 1) as i16).collect();
PgConstraint {
oid: table.id.table_id() as i32, conname: format!("{}_pkey", &table.name),
connamespace: schema.id() as i32,
contype: "p".to_owned(), condeferrable: false,
convalidated: true,
conrelid: table.id.table_id() as i32,
contypid: 0,
conindid: table.id.table_id() as i32,
conparentid: 0,
confrelid: 0,
confupdtype: " ".to_owned(),
confdeltype: " ".to_owned(),
confmatchtype: " ".to_owned(),
conislocal: true,
coninhcount: 0,
connoinherit: true,
conkey: Some(conkey),
confkey: None,
conpfeqop: None,
conppeqop: None,
conffeqop: None,
confdelsetcols: None,
conexclop: None,
conbin: None,
}
}
fn from_table(schema: &SchemaCatalog, table: &TableCatalog) -> PgConstraint {
let conkey: Vec<_> = table
.pk
.iter()
.map(|i| (i.column_index + 1) as i16)
.collect();
PgConstraint {
oid: table.id.table_id() as i32, conname: format!("{}_pkey", &table.name),
connamespace: schema.id() as i32,
contype: "p".to_owned(), condeferrable: false,
convalidated: true,
conrelid: table.id.table_id() as i32,
contypid: 0,
conindid: table.id.table_id() as i32,
conparentid: 0,
confrelid: 0,
confupdtype: " ".to_owned(),
confdeltype: " ".to_owned(),
confmatchtype: " ".to_owned(),
conislocal: true,
coninhcount: 0,
connoinherit: true,
conkey: Some(conkey),
confkey: None,
conpfeqop: None,
conppeqop: None,
conffeqop: None,
confdelsetcols: None,
conexclop: None,
conbin: None,
}
}
}
#[system_catalog(table, "pg_catalog.pg_constraint")]
fn read_pg_constraint(reader: &SysCatalogReaderImpl) -> Result<Vec<PgConstraint>> {
let catalog_reader = reader.catalog_reader.read_guard();
let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?;
Ok(schemas.flat_map(read_pg_constraint_in_schema).collect())
}
fn read_pg_constraint_in_schema(schema: &SchemaCatalog) -> Vec<PgConstraint> {
let system_table_rows = schema
.iter_system_tables()
.map(|table| PgConstraint::from_system_table(schema, table.as_ref()));
let table_rows = schema
.iter_valid_table()
.map(|table| PgConstraint::from_table(schema, table.as_ref()));
system_table_rows.chain(table_rows).collect()
}