risingwave_frontend/handler/
drop_subscription.rs1use pgwire::pg_response::{PgResponse, StatementType};
16use risingwave_sqlparser::ast::ObjectName;
17
18use super::{HandlerArgs, RwPgResponse};
19use crate::Binder;
20use crate::catalog::root_catalog::SchemaPath;
21use crate::error::Result;
22
23pub async fn handle_drop_subscription(
24 handler_args: HandlerArgs,
25 subscription_name: ObjectName,
26 if_exists: bool,
27 cascade: bool,
28) -> Result<RwPgResponse> {
29 let session = handler_args.session;
30 let db_name = &session.database();
31 let (schema_name, subscription_name) =
32 Binder::resolve_schema_qualified_name(db_name, subscription_name)?;
33 let search_path = session.config().search_path();
34 let user_name = &session.user_name();
35 let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);
36
37 let subscription = {
38 let catalog_reader = session.env().catalog_reader().read_guard();
39 let (subscription, schema_name) =
40 match catalog_reader.get_subscription_by_name(db_name, schema_path, &subscription_name)
41 {
42 Ok((subscription, schema)) => (subscription.clone(), schema),
43 Err(e) => {
44 return if if_exists {
45 Ok(RwPgResponse::builder(StatementType::DROP_SUBSCRIPTION)
46 .notice(format!(
47 "subscription \"{}\" does not exist, skipping",
48 subscription_name
49 ))
50 .into())
51 } else {
52 Err(e.into())
53 };
54 }
55 };
56
57 session.check_privilege_for_drop_alter(schema_name, &*subscription)?;
58
59 subscription
60 };
61
62 let subscription_id = subscription.id;
63
64 let catalog_writer = session.catalog_writer()?;
65 catalog_writer
66 .drop_subscription(subscription_id.subscription_id, cascade)
67 .await?;
68
69 Ok(PgResponse::empty_result(StatementType::DROP_SUBSCRIPTION))
70}