risingwave_frontend/catalog/
view_catalog.rs1use risingwave_common::catalog::{Field, SYS_CATALOG_START_ID};
16use risingwave_common::util::epoch::Epoch;
17use risingwave_pb::catalog::PbView;
18use risingwave_sqlparser::ast::Statement;
19use risingwave_sqlparser::parser::Parser;
20
21use super::{DatabaseId, OwnedByUserCatalog, SchemaId, ViewId};
22use crate::WithOptions;
23use crate::user::UserId;
24
25#[derive(Clone, Debug)]
26pub struct ViewCatalog {
27 pub id: ViewId,
28 pub name: String,
29 pub schema_id: SchemaId,
30 pub database_id: DatabaseId,
31
32 pub owner: UserId,
33 pub properties: WithOptions,
34 pub sql: String,
35 pub columns: Vec<Field>,
36 pub created_at_epoch: Option<Epoch>,
37 pub created_at_cluster_version: Option<String>,
38}
39
40impl From<&PbView> for ViewCatalog {
41 fn from(view: &PbView) -> Self {
42 ViewCatalog {
43 id: view.id,
44 name: view.name.clone(),
45 schema_id: view.schema_id,
46 database_id: view.database_id,
47 owner: view.owner,
48 properties: WithOptions::new_with_options(view.properties.clone()),
49 sql: view.sql.clone(),
50 columns: view.columns.iter().map(|f| f.into()).collect(),
51 created_at_epoch: view.created_at_epoch.map(Epoch::from),
52 created_at_cluster_version: view.created_at_cluster_version.clone(),
53 }
54 }
55}
56
57impl ViewCatalog {
58 pub fn name(&self) -> &str {
59 &self.name
60 }
61
62 pub fn with_id(mut self, id: ViewId) -> Self {
63 self.id = id;
64 self
65 }
66
67 pub fn create_sql(&self, schema: String) -> String {
69 if schema == "public" {
70 format!("CREATE VIEW {} AS {}", self.name, self.sql)
71 } else {
72 format!("CREATE VIEW {}.{} AS {}", schema, self.name, self.sql)
73 }
74 }
75
76 pub fn sql_ast(&self) -> crate::error::Result<Statement> {
80 Ok(Parser::parse_exactly_one(&self.sql)?)
81 }
82
83 pub fn is_system_view(&self) -> bool {
85 self.id.as_raw_id() >= SYS_CATALOG_START_ID as u32
86 }
87}
88
89impl OwnedByUserCatalog for ViewCatalog {
90 fn owner(&self) -> UserId {
91 self.owner
92 }
93}