risingwave_frontend/handler/
drop_mv.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::catalog::StreamJobStatus;
use risingwave_pb::meta::cancel_creating_jobs_request::{CreatingJobIds, PbJobs};
use risingwave_sqlparser::ast::ObjectName;

use super::RwPgResponse;
use crate::binder::Binder;
use crate::catalog::root_catalog::SchemaPath;
use crate::catalog::table_catalog::TableType;
use crate::catalog::CatalogError;
use crate::error::Result;
use crate::handler::HandlerArgs;

pub async fn handle_drop_mv(
    handler_args: HandlerArgs,
    table_name: ObjectName,
    if_exists: bool,
    cascade: bool,
) -> Result<RwPgResponse> {
    let session = handler_args.session;
    let db_name = session.database();
    let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, table_name)?;
    let search_path = session.config().search_path();
    let user_name = &session.auth_context().user_name;

    let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);

    let (table_id, status) = {
        let reader = session.env().catalog_reader().read_guard();
        let (table, schema_name) =
            match reader.get_any_table_by_name(session.database(), schema_path, &table_name) {
                Ok((t, s)) => (t, s),
                Err(e) => {
                    return if if_exists {
                        Ok(RwPgResponse::builder(StatementType::DROP_MATERIALIZED_VIEW)
                            .notice(format!(
                                "materialized view \"{}\" does not exist, skipping",
                                table_name
                            ))
                            .into())
                    } else {
                        match e {
                            CatalogError::NotFound("table", name) => {
                                Err(CatalogError::NotFound("materialized view", name).into())
                            }
                            _ => Err(e.into()),
                        }
                    };
                }
            };

        session.check_privilege_for_drop_alter(schema_name, &**table)?;

        match table.table_type() {
            TableType::MaterializedView => {}
            _ => return Err(table.bad_drop_error()),
        }

        (table.id(), table.stream_job_status)
    };

    match status {
        StreamJobStatus::Created => {
            let catalog_writer = session.catalog_writer()?;
            catalog_writer
                .drop_materialized_view(table_id, cascade)
                .await?;
        }
        StreamJobStatus::Creating => {
            let canceled_jobs = session
                .env()
                .meta_client()
                .cancel_creating_jobs(PbJobs::Ids(CreatingJobIds {
                    job_ids: vec![table_id.table_id],
                }))
                .await?;
            tracing::info!(?canceled_jobs, "cancelled creating jobs");
        }
    }

    Ok(PgResponse::empty_result(
        StatementType::DROP_MATERIALIZED_VIEW,
    ))
}

#[cfg(test)]
mod tests {
    use risingwave_common::catalog::{DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME};

    use crate::catalog::root_catalog::SchemaPath;
    use crate::test_utils::LocalFrontend;

    #[tokio::test]
    async fn test_drop_mv_handler() {
        let sql_create_table = "create table t (v1 smallint);";
        let sql_create_mv = "create materialized view mv as select v1 from t;";
        let sql_drop_mv = "drop materialized view mv;";
        let frontend = LocalFrontend::new(Default::default()).await;
        frontend.run_sql(sql_create_table).await.unwrap();
        frontend.run_sql(sql_create_mv).await.unwrap();
        frontend.run_sql(sql_drop_mv).await.unwrap();

        let session = frontend.session_ref();
        let catalog_reader = session.env().catalog_reader().read_guard();
        let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);

        let table =
            catalog_reader.get_created_table_by_name(DEFAULT_DATABASE_NAME, schema_path, "mv");
        assert!(table.is_err());
    }
}