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