risingwave_frontend/handler/
alter_source_column.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
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
// 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 itertools::Itertools;
use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::bail_not_implemented;
use risingwave_common::catalog::max_column_id;
use risingwave_connector::source::{extract_source_struct, SourceEncode, SourceStruct};
use risingwave_sqlparser::ast::{
    AlterSourceOperation, ColumnDef, CreateSourceStatement, ObjectName, Statement,
};
use risingwave_sqlparser::parser::Parser;

use super::create_table::bind_sql_columns;
use super::{HandlerArgs, RwPgResponse};
use crate::catalog::root_catalog::SchemaPath;
use crate::error::{ErrorCode, Result, RwError};
use crate::Binder;

// Note for future drop column:
// 1. Dependencies of generated columns

/// Handle `ALTER TABLE [ADD] COLUMN` statements.
pub async fn handle_alter_source_column(
    handler_args: HandlerArgs,
    source_name: ObjectName,
    operation: AlterSourceOperation,
) -> Result<RwPgResponse> {
    // Get original definition
    let session = handler_args.session;
    let db_name = session.database();
    let (schema_name, real_source_name) =
        Binder::resolve_schema_qualified_name(db_name, source_name.clone())?;
    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 (db_id, schema_id, mut catalog) = {
        let reader = session.env().catalog_reader().read_guard();
        let (source, schema_name) =
            reader.get_source_by_name(db_name, schema_path, &real_source_name)?;
        let db = reader.get_database_by_name(db_name)?;
        let schema = db.get_schema_by_name(schema_name).unwrap();

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

        (db.id(), schema.id(), (**source).clone())
    };

    if catalog.associated_table_id.is_some() {
        return Err(ErrorCode::NotSupported(
            "alter table with connector with ALTER SOURCE statement".to_string(),
            "try to use ALTER TABLE instead".to_string(),
        )
        .into());
    };
    if catalog.info.is_shared() {
        bail_not_implemented!(issue = 16003, "alter shared source");
    }

    // Currently only allow source without schema registry
    let SourceStruct { encode, .. } = extract_source_struct(&catalog.info)?;
    match encode {
        SourceEncode::Avro | SourceEncode::Protobuf => {
            return Err(ErrorCode::NotSupported(
                "alter source with schema registry".to_string(),
                "try `ALTER SOURCE .. FORMAT .. ENCODE .. (...)` instead".to_string(),
            )
            .into());
        }
        SourceEncode::Json if catalog.info.use_schema_registry => {
            return Err(ErrorCode::NotSupported(
                "alter source with schema registry".to_string(),
                "try `ALTER SOURCE .. FORMAT .. ENCODE .. (...)` instead".to_string(),
            )
            .into());
        }
        SourceEncode::Invalid | SourceEncode::Native | SourceEncode::None => {
            return Err(RwError::from(ErrorCode::NotSupported(
                format!("alter source with encode {:?}", encode),
                "Only source with encode JSON | BYTES | CSV | PARQUET can be altered".into(),
            )));
        }
        SourceEncode::Json | SourceEncode::Csv | SourceEncode::Bytes | SourceEncode::Parquet => {}
    }

    let columns = &mut catalog.columns;
    match operation {
        AlterSourceOperation::AddColumn { column_def } => {
            let new_column_name = column_def.name.real_value();
            if columns
                .iter()
                .any(|c| c.column_desc.name == new_column_name)
            {
                Err(ErrorCode::InvalidInputSyntax(format!(
                    "column \"{new_column_name}\" of source \"{source_name}\" already exists"
                )))?
            }
            catalog.definition =
                alter_definition_add_column(&catalog.definition, column_def.clone())?;
            let mut bound_column = bind_sql_columns(&[column_def])?.remove(0);
            bound_column.column_desc.column_id = max_column_id(columns).next();
            columns.push(bound_column);
        }
        _ => unreachable!(),
    }

    // update version
    catalog.version += 1;

    let catalog_writer = session.catalog_writer()?;
    catalog_writer
        .alter_source(catalog.to_prost(schema_id, db_id))
        .await?;

    Ok(PgResponse::empty_result(StatementType::ALTER_SOURCE))
}

/// `alter_definition_add_column` adds a new column to the definition of the relation.
#[inline(always)]
pub fn alter_definition_add_column(definition: &str, column: ColumnDef) -> Result<String> {
    let ast = Parser::parse_sql(definition).expect("failed to parse relation definition");
    let mut stmt = ast
        .into_iter()
        .exactly_one()
        .expect("should contains only one statement");

    match &mut stmt {
        Statement::CreateSource {
            stmt: CreateSourceStatement { columns, .. },
        } => {
            columns.push(column);
        }
        _ => unreachable!(),
    }

    Ok(stmt.to_string())
}

#[cfg(test)]
pub mod tests {
    use std::collections::HashMap;

    use risingwave_common::catalog::{DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME};
    use risingwave_common::types::DataType;

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

    #[tokio::test]
    async fn test_alter_source_column_handler() {
        let frontend = LocalFrontend::new(Default::default()).await;
        let session = frontend.session_ref();
        let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);

        let sql = r#"create source s_shared (v1 int) with (
            connector = 'kafka',
            topic = 'abc',
            properties.bootstrap.server = 'localhost:29092',
        ) FORMAT PLAIN ENCODE JSON;"#;

        frontend
            .run_sql_with_session(session.clone(), sql)
            .await
            .unwrap();

        frontend
            .run_sql_with_session(session.clone(), "SET streaming_use_shared_source TO false;")
            .await
            .unwrap();
        let sql = r#"create source s (v1 int) with (
            connector = 'kafka',
            topic = 'abc',
            properties.bootstrap.server = 'localhost:29092',
          ) FORMAT PLAIN ENCODE JSON;"#;

        frontend
            .run_sql_with_session(session.clone(), sql)
            .await
            .unwrap();

        let get_source = || {
            let catalog_reader = session.env().catalog_reader().read_guard();
            catalog_reader
                .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "s")
                .unwrap()
                .0
                .clone()
        };

        let source = get_source();
        let columns: HashMap<_, _> = source
            .columns
            .iter()
            .map(|col| (col.name(), (col.data_type().clone(), col.column_id())))
            .collect();

        let sql = "alter source s_shared add column v2 varchar;";
        assert_eq!("Feature is not yet implemented: alter shared source\nTracking issue: https://github.com/risingwavelabs/risingwave/issues/16003", &frontend.run_sql(sql).await.unwrap_err().to_string());
        let sql = "alter source s add column v2 varchar;";
        frontend.run_sql(sql).await.unwrap();

        let altered_source = get_source();

        let altered_columns: HashMap<_, _> = altered_source
            .columns
            .iter()
            .map(|col| (col.name(), (col.data_type().clone(), col.column_id())))
            .collect();

        // Check the new column.
        assert_eq!(columns.len() + 1, altered_columns.len());
        assert_eq!(altered_columns["v2"].0, DataType::Varchar);

        // Check the old columns and IDs are not changed.
        assert_eq!(columns["v1"], altered_columns["v1"]);

        // Check version
        assert_eq!(source.version + 1, altered_source.version);

        // Check definition
        let altered_sql = r#"CREATE SOURCE s (v1 INT, v2 CHARACTER VARYING) WITH (connector = 'kafka', topic = 'abc', properties.bootstrap.server = 'localhost:29092') FORMAT PLAIN ENCODE JSON"#;
        assert_eq!(altered_sql, altered_source.definition);
    }
}