risingwave_frontend/catalog/
view_catalog.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use risingwave_common::catalog::{Field, SYS_CATALOG_START_ID};
16use risingwave_pb::catalog::PbView;
17
18use super::{DatabaseId, OwnedByUserCatalog, SchemaId, ViewId};
19use crate::WithOptions;
20use crate::user::UserId;
21
22#[derive(Clone, Debug)]
23pub struct ViewCatalog {
24    pub id: ViewId,
25    pub name: String,
26    pub schema_id: SchemaId,
27    pub database_id: DatabaseId,
28
29    pub owner: UserId,
30    pub properties: WithOptions,
31    pub sql: String,
32    pub columns: Vec<Field>,
33}
34
35impl From<&PbView> for ViewCatalog {
36    fn from(view: &PbView) -> Self {
37        ViewCatalog {
38            id: view.id,
39            name: view.name.clone(),
40            schema_id: view.schema_id,
41            database_id: view.database_id,
42            owner: view.owner,
43            properties: WithOptions::new_with_options(view.properties.clone()),
44            sql: view.sql.clone(),
45            columns: view.columns.iter().map(|f| f.into()).collect(),
46        }
47    }
48}
49
50impl ViewCatalog {
51    pub fn name(&self) -> &str {
52        &self.name
53    }
54
55    pub fn with_id(mut self, id: ViewId) -> Self {
56        self.id = id;
57        self
58    }
59
60    /// Returns the SQL statement that can be used to create this view.
61    pub fn create_sql(&self, schema: String) -> String {
62        if schema == "public" {
63            format!("CREATE VIEW {} AS {}", self.name, self.sql)
64        } else {
65            format!("CREATE VIEW {}.{} AS {}", schema, self.name, self.sql)
66        }
67    }
68
69    /// Returns true if this view is a system view.
70    pub fn is_system_view(&self) -> bool {
71        self.id >= SYS_CATALOG_START_ID as u32
72    }
73}
74
75impl OwnedByUserCatalog for ViewCatalog {
76    fn owner(&self) -> UserId {
77        self.owner
78    }
79}