risingwave_frontend/handler/
comment.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 pgwire::pg_response::{PgResponse, StatementType};
16use risingwave_pb::catalog::PbComment;
17use risingwave_sqlparser::ast::{CommentObject, ObjectName};
18
19use super::{HandlerArgs, RwPgResponse};
20use crate::Binder;
21use crate::error::{ErrorCode, Result};
22
23pub async fn handle_comment(
24    handler_args: HandlerArgs,
25    object_type: CommentObject,
26    object_name: ObjectName,
27    comment: Option<String>,
28) -> Result<RwPgResponse> {
29    let session = handler_args.session;
30    let comment = comment.filter(|s| !s.is_empty());
31
32    let comment = {
33        let mut binder = Binder::new_for_ddl(&session);
34        // only `Column` and `Table` object are now supported
35        match object_type {
36            CommentObject::Column => {
37                let [tab @ .., col] = object_name.0.as_slice() else {
38                    return Err(ErrorCode::BindError(format!(
39                        "Invalid column: {}",
40                        object_name.real_value()
41                    ))
42                    .into());
43                };
44
45                let (schema, table) = Binder::resolve_schema_qualified_name(
46                    &session.database(),
47                    ObjectName(tab.to_vec()),
48                )?;
49
50                let (database_id, schema_id) =
51                    session.get_database_and_schema_id_for_create(schema.clone())?;
52                let table = binder.bind_table(schema.as_deref(), &table)?;
53                if table.table_catalog.owner != session.user_id() && !session.is_super_user() {
54                    return Err(ErrorCode::PermissionDenied(format!(
55                        "must be owner of relation {}",
56                        table.table_catalog.name
57                    ))
58                    .into());
59                }
60                binder.bind_columns_to_context(col.real_value(), &table.table_catalog.columns)?;
61
62                let column = binder.bind_column(object_name.0.as_slice())?;
63
64                PbComment {
65                    table_id: table.table_id.into(),
66                    schema_id,
67                    database_id,
68                    column_index: column.as_input_ref().map(|input_ref| input_ref.index as _),
69                    description: comment,
70                }
71            }
72            CommentObject::Table => {
73                let (schema, table) =
74                    Binder::resolve_schema_qualified_name(&session.database(), object_name)?;
75                let (database_id, schema_id) =
76                    session.get_database_and_schema_id_for_create(schema.clone())?;
77                let table = binder.bind_table(schema.as_deref(), &table)?;
78                if table.table_catalog.owner != session.user_id() && !session.is_super_user() {
79                    return Err(ErrorCode::PermissionDenied(format!(
80                        "must be owner of relation {}",
81                        table.table_catalog.name
82                    ))
83                    .into());
84                }
85
86                PbComment {
87                    table_id: table.table_id.into(),
88                    schema_id,
89                    database_id,
90                    column_index: None,
91                    description: comment,
92                }
93            }
94        }
95    };
96
97    let catalog_writer = session.catalog_writer()?;
98    catalog_writer.comment_on(comment).await?;
99
100    Ok(PgResponse::empty_result(StatementType::COMMENT))
101}