risingwave_frontend/handler/
drop_database.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::error::{ErrorCode, Result};
21use crate::handler::HandlerArgs;
22
23pub async fn handle_drop_database(
24    handler_args: HandlerArgs,
25    database_name: ObjectName,
26    if_exists: bool,
27) -> Result<RwPgResponse> {
28    let session = handler_args.session;
29    let catalog_reader = session.env().catalog_reader();
30    let database_name = Binder::resolve_database_name(database_name)?;
31    if session.database() == database_name {
32        return Err(ErrorCode::PermissionDenied(
33            "cannot drop the currently open database".to_owned(),
34        )
35        .into());
36    }
37    let database = {
38        let reader = catalog_reader.read_guard();
39        match reader.get_database_by_name(&database_name) {
40            Ok(db) => db.clone(),
41            Err(err) => {
42                // Unable to find this database. If `if_exists` is true,
43                // we can just return success.
44                return if if_exists {
45                    Ok(PgResponse::builder(StatementType::DROP_DATABASE)
46                        .notice(format!(
47                            "database \"{}\" does not exist, skipping",
48                            database_name
49                        ))
50                        .into())
51                } else {
52                    Err(err.into())
53                };
54            }
55        }
56    };
57
58    session.check_privilege_for_drop_alter_db_schema(&database)?;
59
60    let catalog_writer = session.catalog_writer()?;
61    catalog_writer.drop_database(database.id()).await?;
62    Ok(PgResponse::empty_result(StatementType::DROP_DATABASE))
63}
64
65#[cfg(test)]
66mod tests {
67    use crate::test_utils::LocalFrontend;
68
69    #[tokio::test]
70    async fn test_drop_database() {
71        let frontend = LocalFrontend::new(Default::default()).await;
72        let session = frontend.session_ref();
73        let catalog_reader = session.env().catalog_reader();
74
75        frontend.run_sql("CREATE DATABASE database").await.unwrap();
76
77        frontend.run_sql("CREATE SCHEMA schema").await.unwrap();
78
79        frontend.run_sql("DROP SCHEMA public").await.unwrap();
80
81        frontend.run_sql("CREATE USER user WITH NOSUPERUSER NOCREATEDB PASSWORD 'md5827ccb0eea8a706c4c34a16891f84e7b'").await.unwrap();
82        let user_id = {
83            let user_reader = session.env().user_info_reader();
84            user_reader
85                .read_guard()
86                .get_user_by_name("user")
87                .unwrap()
88                .id
89        };
90        let res = frontend
91            .run_user_sql(
92                "DROP DATABASE database",
93                "dev".to_owned(),
94                "user".to_owned(),
95                user_id,
96            )
97            .await;
98        assert!(res.is_err());
99
100        frontend.run_sql("DROP DATABASE database").await.unwrap();
101
102        let database = catalog_reader
103            .read_guard()
104            .get_database_by_name("database")
105            .ok()
106            .cloned();
107        assert!(database.is_none());
108    }
109}