risingwave_frontend/handler/
drop_sink.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_sqlparser::ast::ObjectName;
17
18use super::RwPgResponse;
19use crate::binder::Binder;
20use crate::catalog::root_catalog::SchemaPath;
21use crate::error::Result;
22use crate::handler::HandlerArgs;
23
24pub async fn handle_drop_sink(
25    handler_args: HandlerArgs,
26    sink_name: ObjectName,
27    if_exists: bool,
28    cascade: bool,
29) -> Result<RwPgResponse> {
30    let session = handler_args.session.clone();
31    let db_name = &session.database();
32    let (schema_name, sink_name) = Binder::resolve_schema_qualified_name(db_name, &sink_name)?;
33    let search_path = session.config().search_path();
34    let user_name = &session.user_name();
35    let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);
36
37    let sink = {
38        let catalog_reader = session.env().catalog_reader().read_guard();
39        let (sink, schema_name) =
40            match catalog_reader.get_any_sink_by_name(db_name, schema_path, &sink_name) {
41                Ok((sink, schema)) => (sink.clone(), schema),
42                Err(e) => {
43                    return if if_exists {
44                        Ok(RwPgResponse::builder(StatementType::DROP_SINK)
45                            .notice(format!("sink \"{}\" does not exist, skipping", sink_name))
46                            .into())
47                    } else {
48                        Err(e.into())
49                    };
50                }
51            };
52
53        session.check_privilege_for_drop_alter(schema_name, &*sink)?;
54
55        sink
56    };
57
58    let sink_id = sink.id;
59
60    let catalog_writer = session.catalog_writer()?;
61    catalog_writer.drop_sink(sink_id.sink_id, cascade).await?;
62
63    Ok(PgResponse::empty_result(StatementType::DROP_SINK))
64}
65
66#[cfg(test)]
67mod tests {
68    use risingwave_common::catalog::{DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME};
69
70    use crate::catalog::root_catalog::SchemaPath;
71    use crate::test_utils::LocalFrontend;
72
73    #[tokio::test]
74    async fn test_drop_sink_handler() {
75        let sql_create_table = "create table t (v1 smallint primary key);";
76        let sql_create_mv = "create materialized view mv as select v1 from t;";
77        let sql_create_sink = "create sink snk from mv with( connector = 'kafka')";
78        let sql_drop_sink = "drop sink snk;";
79        let frontend = LocalFrontend::new(Default::default()).await;
80        frontend.run_sql(sql_create_table).await.unwrap();
81        frontend.run_sql(sql_create_mv).await.unwrap();
82        frontend.run_sql(sql_create_sink).await.unwrap();
83        frontend.run_sql(sql_drop_sink).await.unwrap();
84
85        let session = frontend.session_ref();
86        let catalog_reader = session.env().catalog_reader().read_guard();
87        let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
88
89        let sink =
90            catalog_reader.get_created_table_by_name(DEFAULT_DATABASE_NAME, schema_path, "snk");
91        assert!(sink.is_err());
92    }
93}