risingwave_frontend/handler/
create_view.rs1use either::Either;
18use pgwire::pg_response::{PgResponse, StatementType};
19use risingwave_common::util::iter_util::ZipEqFast;
20use risingwave_pb::catalog::PbView;
21use risingwave_sqlparser::ast::{Ident, ObjectName, Query, Statement};
22
23use super::RwPgResponse;
24use crate::binder::Binder;
25use crate::error::Result;
26use crate::handler::HandlerArgs;
27use crate::optimizer::OptimizerContext;
28
29pub async fn handle_create_view(
30 handler_args: HandlerArgs,
31 if_not_exists: bool,
32 name: ObjectName,
33 columns: Vec<Ident>,
34 query: Query,
35) -> Result<RwPgResponse> {
36 let session = handler_args.session.clone();
37 let db_name = &session.database();
38 let (schema_name, view_name) = Binder::resolve_schema_qualified_name(db_name, &name)?;
39
40 let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?;
41
42 let properties = handler_args.with_options.clone();
43
44 if let Either::Right(resp) = session.check_relation_name_duplicated(
45 name.clone(),
46 StatementType::CREATE_VIEW,
47 if_not_exists,
48 )? {
49 return Ok(resp);
50 }
51
52 let (dependent_relations, schema) = {
54 let context = OptimizerContext::from_handler_args(handler_args);
55 let super::query::RwBatchQueryPlanResult {
56 schema,
57 dependent_relations,
58 ..
59 } = super::query::gen_batch_plan_by_statement(
60 &session,
61 context.into(),
62 Statement::Query(Box::new(query.clone())),
63 )?
64 .unwrap_rw()?;
65
66 (dependent_relations, schema)
67 };
68
69 let columns = if columns.is_empty() {
70 schema.fields().to_vec()
71 } else {
72 if columns.len() != schema.fields().len() {
73 return Err(crate::error::ErrorCode::InternalError(
74 "view has different number of columns than the query's columns".to_owned(),
75 )
76 .into());
77 }
78 schema
79 .fields()
80 .iter()
81 .zip_eq_fast(columns)
82 .map(|(f, c)| {
83 let mut field = f.clone();
84 field.name = c.real_value();
85 field
86 })
87 .collect()
88 };
89
90 let (properties, secret_refs, connection_refs) = properties.into_parts();
91 if !secret_refs.is_empty() || !connection_refs.is_empty() {
92 return Err(crate::error::ErrorCode::InvalidParameterValue(
93 "Secret reference and Connection reference are not allowed in create view options"
94 .to_owned(),
95 )
96 .into());
97 }
98
99 let view = PbView {
100 id: 0.into(),
101 schema_id,
102 database_id,
103 name: view_name,
104 properties,
105 owner: session.user_id(),
106 sql: format!("{}", query),
107 columns: columns.into_iter().map(|f| f.to_prost()).collect(),
108 created_at_epoch: None,
109 created_at_cluster_version: None,
110 };
111
112 let catalog_writer = session.catalog_writer()?;
113 catalog_writer
114 .create_view(view, dependent_relations.into_iter().collect())
115 .await?;
116
117 Ok(PgResponse::empty_result(StatementType::CREATE_VIEW))
118}