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 std::collections::HashSet;
16
17use pgwire::pg_response::{PgResponse, StatementType};
18use risingwave_pb::ddl_service::{ReplaceJobPlan, TableJobType, replace_job_plan};
19use risingwave_sqlparser::ast::ObjectName;
20
21use super::RwPgResponse;
22use crate::binder::Binder;
23use crate::catalog::root_catalog::SchemaPath;
24use crate::error::Result;
25use crate::handler::HandlerArgs;
26use crate::handler::alter_table_column::hijack_merger_for_target_table;
27use crate::handler::create_sink::{fetch_incoming_sinks, reparse_table_for_sink};
28
29pub async fn handle_drop_sink(
30    handler_args: HandlerArgs,
31    sink_name: ObjectName,
32    if_exists: bool,
33    cascade: bool,
34) -> Result<RwPgResponse> {
35    let session = handler_args.session.clone();
36    let db_name = &session.database();
37    let (schema_name, sink_name) = Binder::resolve_schema_qualified_name(db_name, sink_name)?;
38    let search_path = session.config().search_path();
39    let user_name = &session.user_name();
40    let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);
41
42    let sink = {
43        let catalog_reader = session.env().catalog_reader().read_guard();
44        let (sink, schema_name) =
45            match catalog_reader.get_sink_by_name(db_name, schema_path, &sink_name) {
46                Ok((sink, schema)) => (sink.clone(), schema),
47                Err(e) => {
48                    return if if_exists {
49                        Ok(RwPgResponse::builder(StatementType::DROP_SINK)
50                            .notice(format!("sink \"{}\" does not exist, skipping", sink_name))
51                            .into())
52                    } else {
53                        Err(e.into())
54                    };
55                }
56            };
57
58        session.check_privilege_for_drop_alter(schema_name, &*sink)?;
59
60        sink
61    };
62
63    let sink_id = sink.id;
64
65    let mut affected_table_change = None;
66    if let Some(target_table_id) = &sink.target_table {
67        let table_catalog = {
68            let reader = session.env().catalog_reader().read_guard();
69            let table = reader.get_any_table_by_id(target_table_id)?;
70            table.clone()
71        };
72
73        let (mut graph, mut table, source) =
74            reparse_table_for_sink(&session, &table_catalog).await?;
75
76        assert!(!table_catalog.incoming_sinks.is_empty());
77
78        table
79            .incoming_sinks
80            .clone_from(&table_catalog.incoming_sinks);
81
82        let mut incoming_sink_ids: HashSet<_> =
83            table_catalog.incoming_sinks.iter().copied().collect();
84
85        assert!(incoming_sink_ids.remove(&sink_id.sink_id));
86
87        let columns_without_rw_timestamp = table_catalog.columns_without_rw_timestamp();
88        for sink in fetch_incoming_sinks(&session, &incoming_sink_ids)? {
89            hijack_merger_for_target_table(
90                &mut graph,
91                &columns_without_rw_timestamp,
92                &sink,
93                Some(&sink.unique_identity()),
94            )?;
95        }
96
97        affected_table_change = Some(ReplaceJobPlan {
98            replace_job: Some(replace_job_plan::ReplaceJob::ReplaceTable(
99                replace_job_plan::ReplaceTable {
100                    table: Some(table),
101                    source,
102                    job_type: TableJobType::General as _,
103                },
104            )),
105            fragment_graph: Some(graph),
106            table_col_index_mapping: None,
107        });
108    }
109
110    let catalog_writer = session.catalog_writer()?;
111    catalog_writer
112        .drop_sink(sink_id.sink_id, cascade, affected_table_change)
113        .await?;
114
115    Ok(PgResponse::empty_result(StatementType::DROP_SINK))
116}
117
118#[cfg(test)]
119mod tests {
120    use risingwave_common::catalog::{DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME};
121
122    use crate::catalog::root_catalog::SchemaPath;
123    use crate::test_utils::LocalFrontend;
124
125    #[tokio::test]
126    async fn test_drop_sink_handler() {
127        let sql_create_table = "create table t (v1 smallint primary key);";
128        let sql_create_mv = "create materialized view mv as select v1 from t;";
129        let sql_create_sink = "create sink snk from mv with( connector = 'kafka')";
130        let sql_drop_sink = "drop sink snk;";
131        let frontend = LocalFrontend::new(Default::default()).await;
132        frontend.run_sql(sql_create_table).await.unwrap();
133        frontend.run_sql(sql_create_mv).await.unwrap();
134        frontend.run_sql(sql_create_sink).await.unwrap();
135        frontend.run_sql(sql_drop_sink).await.unwrap();
136
137        let session = frontend.session_ref();
138        let catalog_reader = session.env().catalog_reader().read_guard();
139        let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
140
141        let sink =
142            catalog_reader.get_created_table_by_name(DEFAULT_DATABASE_NAME, schema_path, "snk");
143        assert!(sink.is_err());
144    }
145}