risingwave_frontend/handler/
create_view.rs1use either::Either;
18use itertools::Itertools;
19use pgwire::pg_response::{PgResponse, StatementType};
20use risingwave_common::util::iter_util::ZipEqFast;
21use risingwave_pb::catalog::PbView;
22use risingwave_sqlparser::ast::{Ident, ObjectName, Query, Statement};
23
24use super::RwPgResponse;
25use crate::binder::Binder;
26use crate::error::Result;
27use crate::handler::HandlerArgs;
28use crate::optimizer::OptimizerContext;
29
30pub async fn handle_create_view(
31 handler_args: HandlerArgs,
32 if_not_exists: bool,
33 name: ObjectName,
34 columns: Vec<Ident>,
35 query: Query,
36) -> Result<RwPgResponse> {
37 let session = handler_args.session.clone();
38 let db_name = &session.database();
39 let (schema_name, view_name) = Binder::resolve_schema_qualified_name(db_name, name.clone())?;
40
41 let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?;
42
43 let properties = handler_args.with_options.clone();
44
45 if let Either::Right(resp) = session.check_relation_name_duplicated(
46 name.clone(),
47 StatementType::CREATE_VIEW,
48 if_not_exists,
49 )? {
50 return Ok(resp);
51 }
52
53 let (dependent_relations, schema) = {
55 let context = OptimizerContext::from_handler_args(handler_args);
56 let super::query::BatchQueryPlanResult {
57 schema,
58 dependent_relations,
59 ..
60 } = super::query::gen_batch_plan_by_statement(
61 &session,
62 context.into(),
63 Statement::Query(Box::new(query.clone())),
64 )?;
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,
101 schema_id,
102 database_id,
103 name: view_name,
104 properties,
105 owner: session.user_id(),
106 dependent_relations: dependent_relations
107 .into_iter()
108 .map(|t| t.table_id)
109 .collect_vec(),
110 sql: format!("{}", query),
111 columns: columns.into_iter().map(|f| f.to_prost()).collect(),
112 };
113
114 let catalog_writer = session.catalog_writer()?;
115 catalog_writer.create_view(view).await?;
116
117 Ok(PgResponse::empty_result(StatementType::CREATE_VIEW))
118}