risingwave_frontend/handler/
alter_connection_props.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 super::RwPgResponse;
16use crate::catalog::root_catalog::SchemaPath;
17use crate::error::{ErrorCode, Result};
18use crate::handler::{HandlerArgs, ObjectName, SqlOption, StatementType};
19use crate::utils::resolve_connection_ref_and_secret_ref;
20use crate::{Binder, WithOptions};
21
22pub async fn handle_alter_connection_connector_props(
23    handler_args: HandlerArgs,
24    connection_name: ObjectName,
25    alter_props: Vec<SqlOption>,
26) -> Result<RwPgResponse> {
27    let session = handler_args.session;
28    let db_name = &session.database();
29    let (schema_name, real_connection_name) =
30        Binder::resolve_schema_qualified_name(db_name, &connection_name)?;
31    let search_path = session.config().search_path();
32    let user_name = &session.user_name();
33    let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);
34
35    let connection_id = {
36        let reader = session.env().catalog_reader().read_guard();
37        let (connection, schema_name) =
38            reader.get_connection_by_name(db_name, schema_path, &real_connection_name)?;
39
40        session.check_privilege_for_drop_alter(schema_name, &**connection)?;
41
42        tracing::debug!(
43            "handle_alter_connection_connector_props triggered: connection_name: {}, connection_id: {}",
44            real_connection_name,
45            connection.id
46        );
47
48        connection.id
49    };
50
51    let meta_client = session.env().meta_client();
52    let (resolved_with_options, _, connector_conn_ref) = resolve_connection_ref_and_secret_ref(
53        WithOptions::try_from(alter_props.as_ref() as &[SqlOption])?,
54        &session,
55        None,
56    )?;
57    let (changed_props, changed_secret_refs) = resolved_with_options.into_parts();
58
59    if connector_conn_ref.is_some() {
60        return Err(ErrorCode::InvalidInputSyntax(
61            "ALTER CONNECTION CONNECTOR does not support nested CONNECTION references".to_owned(),
62        )
63        .into());
64    }
65
66    meta_client
67        .alter_connection_connector_props(
68            connection_id.as_raw_id(),
69            changed_props,
70            changed_secret_refs,
71        )
72        .await?;
73
74    Ok(RwPgResponse::empty_result(StatementType::ALTER_CONNECTION))
75}