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