risingwave_frontend/handler/
drop_secret.rs
1use std::sync::Arc;
16
17use pgwire::pg_response::StatementType;
18use risingwave_common::license::Feature;
19use risingwave_sqlparser::ast::ObjectName;
20
21use crate::Binder;
22use crate::catalog::root_catalog::SchemaPath;
23use crate::catalog::secret_catalog::SecretCatalog;
24use crate::catalog::{DatabaseId, SchemaId};
25use crate::error::Result;
26use crate::handler::{HandlerArgs, RwPgResponse};
27use crate::session::SessionImpl;
28
29pub async fn handle_drop_secret(
30 handler_args: HandlerArgs,
31 secret_name: ObjectName,
32 if_exists: bool,
33) -> Result<RwPgResponse> {
34 Feature::SecretManagement
35 .check_available()
36 .map_err(|e| anyhow::anyhow!(e))?;
37
38 let session = handler_args.session;
39
40 if let Some((secret_catalog, _, _)) =
41 fetch_secret_catalog_with_db_schema_id(&session, &secret_name, if_exists)?
42 {
43 let catalog_writer = session.catalog_writer()?;
44 catalog_writer.drop_secret(secret_catalog.id).await?;
45
46 Ok(RwPgResponse::empty_result(StatementType::DROP_SECRET))
47 } else {
48 Ok(RwPgResponse::builder(StatementType::DROP_SECRET)
49 .notice(format!(
50 "secret \"{}\" does not exist, skipping",
51 secret_name
52 ))
53 .into())
54 }
55}
56
57pub fn fetch_secret_catalog_with_db_schema_id(
59 session: &SessionImpl,
60 secret_name: &ObjectName,
61 if_exists: bool,
62) -> Result<Option<(Arc<SecretCatalog>, DatabaseId, SchemaId)>> {
63 let db_name = &session.database();
64 let (schema_name, secret_name) =
65 Binder::resolve_schema_qualified_name(db_name, secret_name.clone())?;
66 let search_path = session.config().search_path();
67 let user_name = &session.user_name();
68
69 let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);
70
71 let reader = session.env().catalog_reader().read_guard();
72 match reader.get_secret_by_name(db_name, schema_path, &secret_name) {
73 Ok((catalog, schema_name)) => {
74 session.check_privilege_for_drop_alter(schema_name, &**catalog)?;
75
76 let db = reader.get_database_by_name(db_name)?;
77 let schema = db.get_schema_by_name(schema_name).unwrap();
78
79 Ok(Some((Arc::clone(catalog), db.id(), schema.id())))
80 }
81 Err(e) => {
82 if if_exists {
83 Ok(None)
84 } else {
85 Err(e.into())
86 }
87 }
88 }
89}