risingwave_frontend/handler/
drop_source.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_source(
25    handler_args: HandlerArgs,
26    name: ObjectName,
27    if_exists: bool,
28    cascade: bool,
29) -> Result<RwPgResponse> {
30    let session = handler_args.session;
31    let db_name = &session.database();
32    let (schema_name, source_name) = Binder::resolve_schema_qualified_name(db_name, name)?;
33    let search_path = session.config().search_path();
34    let user_name = &session.user_name();
35
36    // Check if temporary source exists, if yes, drop it.
37    if let Some(_source) = session.get_temporary_source(&source_name) {
38        session.drop_temporary_source(&source_name);
39        return Ok(PgResponse::empty_result(StatementType::DROP_SOURCE));
40    }
41
42    let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);
43
44    let (source, schema_name) = {
45        let catalog_reader = session.env().catalog_reader().read_guard();
46
47        if let Ok((table, _)) =
48            catalog_reader.get_created_table_by_name(db_name, schema_path, &source_name)
49        {
50            return Err(table.bad_drop_error());
51        }
52
53        match catalog_reader.get_source_by_name(db_name, schema_path, &source_name) {
54            Ok((s, schema)) => (s.clone(), schema),
55            Err(e) => {
56                return if if_exists {
57                    Ok(RwPgResponse::builder(StatementType::DROP_SOURCE)
58                        .notice(format!(
59                            "source \"{}\" does not exist, skipping",
60                            source_name
61                        ))
62                        .into())
63                } else {
64                    Err(e.into())
65                };
66            }
67        }
68    };
69
70    session.check_privilege_for_drop_alter(schema_name, &*source)?;
71
72    let catalog_writer = session.catalog_writer()?;
73    catalog_writer.drop_source(source.id, cascade).await?;
74
75    Ok(PgResponse::empty_result(StatementType::DROP_SOURCE))
76}