risingwave_frontend/handler/
describe.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
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
// 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 std::fmt::Display;

use itertools::Itertools;
use pgwire::pg_field_descriptor::PgFieldDescriptor;
use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::catalog::{ColumnCatalog, ColumnDesc};
use risingwave_common::types::Fields;
use risingwave_sqlparser::ast::{display_comma_separated, ObjectName};

use super::show::ShowColumnRow;
use super::{fields_to_descriptors, RwPgResponse};
use crate::binder::{Binder, Relation};
use crate::catalog::CatalogError;
use crate::error::Result;
use crate::handler::{HandlerArgs, RwPgResponseBuilderExt};

pub fn handle_describe(handler_args: HandlerArgs, object_name: ObjectName) -> Result<RwPgResponse> {
    let session = handler_args.session;
    let mut binder = Binder::new_for_system(&session);
    let not_found_err =
        CatalogError::NotFound("table, source, sink or view", object_name.to_string());

    // Vec<ColumnCatalog>, Vec<ColumnDesc>, Vec<ColumnDesc>, Vec<Arc<IndexCatalog>>, String, Option<String>
    let (columns, pk_columns, dist_columns, indices, relname, description) =
        if let Ok(relation) = binder.bind_relation_by_name(object_name.clone(), None, None) {
            match relation {
                Relation::Source(s) => {
                    let pk_column_catalogs = s
                        .catalog
                        .pk_col_ids
                        .iter()
                        .map(|&column_id| {
                            s.catalog
                                .columns
                                .iter()
                                .filter(|x| x.column_id() == column_id)
                                .map(|x| x.column_desc.clone())
                                .exactly_one()
                                .unwrap()
                        })
                        .collect_vec();
                    (
                        s.catalog.columns,
                        pk_column_catalogs,
                        vec![],
                        vec![],
                        s.catalog.name,
                        None, // Description
                    )
                }
                Relation::BaseTable(t) => {
                    let pk_column_catalogs = t
                        .table_catalog
                        .pk()
                        .iter()
                        .map(|x| t.table_catalog.columns[x.column_index].column_desc.clone())
                        .collect_vec();
                    let dist_columns = t
                        .table_catalog
                        .distribution_key()
                        .iter()
                        .map(|idx| t.table_catalog.columns[*idx].column_desc.clone())
                        .collect_vec();
                    (
                        t.table_catalog.columns.clone(),
                        pk_column_catalogs,
                        dist_columns,
                        t.table_indexes,
                        t.table_catalog.name.clone(),
                        t.table_catalog.description.clone(),
                    )
                }
                Relation::SystemTable(t) => {
                    let pk_column_catalogs = t
                        .sys_table_catalog
                        .pk
                        .iter()
                        .map(|idx| t.sys_table_catalog.columns[*idx].column_desc.clone())
                        .collect_vec();
                    (
                        t.sys_table_catalog.columns.clone(),
                        pk_column_catalogs,
                        vec![],
                        vec![],
                        t.sys_table_catalog.name.clone(),
                        None, // Description
                    )
                }
                Relation::Share(_) => {
                    if let Ok(view) = binder.bind_view_by_name(object_name.clone()) {
                        let columns = view
                            .view_catalog
                            .columns
                            .iter()
                            .enumerate()
                            .map(|(idx, field)| ColumnCatalog {
                                column_desc: ColumnDesc::from_field_with_column_id(field, idx as _),
                                is_hidden: false,
                            })
                            .collect();
                        (
                            columns,
                            vec![],
                            vec![],
                            vec![],
                            view.view_catalog.name.clone(),
                            None,
                        )
                    } else {
                        return Err(not_found_err.into());
                    }
                }
                _ => {
                    return Err(not_found_err.into());
                }
            }
        } else if let Ok(sink) = binder.bind_sink_by_name(object_name.clone()) {
            let columns = sink.sink_catalog.full_columns().to_vec();
            let pk_columns = sink
                .sink_catalog
                .downstream_pk_indices()
                .into_iter()
                .map(|idx| columns[idx].column_desc.clone())
                .collect_vec();
            let dist_columns = sink
                .sink_catalog
                .distribution_key
                .iter()
                .map(|idx| columns[*idx].column_desc.clone())
                .collect_vec();
            (
                columns,
                pk_columns,
                dist_columns,
                vec![],
                sink.sink_catalog.name.clone(),
                None,
            )
        } else {
            return Err(not_found_err.into());
        };

    // Convert all column descs to rows
    let mut rows = columns
        .into_iter()
        .flat_map(ShowColumnRow::from_catalog)
        .collect_vec();

    fn concat<T>(display_elems: impl IntoIterator<Item = T>) -> String
    where
        T: Display,
    {
        format!(
            "{}",
            display_comma_separated(&display_elems.into_iter().collect::<Vec<_>>())
        )
    }

    // Convert primary key to rows
    if !pk_columns.is_empty() {
        rows.push(ShowColumnRow {
            name: "primary key".into(),
            r#type: concat(pk_columns.iter().map(|x| &x.name)),
            is_hidden: None,
            description: None,
        });
    }

    // Convert distribution keys to rows
    if !dist_columns.is_empty() {
        rows.push(ShowColumnRow {
            name: "distribution key".into(),
            r#type: concat(dist_columns.iter().map(|x| &x.name)),
            is_hidden: None,
            description: None,
        });
    }

    // Convert all indexes to rows
    rows.extend(indices.iter().map(|index| {
        let index_display = index.display();

        ShowColumnRow {
            name: index.name.clone(),
            r#type: if index_display.include_columns.is_empty() {
                format!(
                    "index({}) distributed by({})",
                    display_comma_separated(&index_display.index_columns_with_ordering),
                    display_comma_separated(&index_display.distributed_by_columns),
                )
            } else {
                format!(
                    "index({}) include({}) distributed by({})",
                    display_comma_separated(&index_display.index_columns_with_ordering),
                    display_comma_separated(&index_display.include_columns),
                    display_comma_separated(&index_display.distributed_by_columns),
                )
            },
            is_hidden: None,
            // TODO: index description
            description: None,
        }
    }));

    rows.push(ShowColumnRow {
        name: "table description".into(),
        r#type: relname,
        is_hidden: None,
        description: description.map(Into::into),
    });

    // TODO: table name and description as title of response
    // TODO: recover the original user statement
    Ok(PgResponse::builder(StatementType::DESCRIBE)
        .rows(rows)
        .into())
}

pub fn infer_describe() -> Vec<PgFieldDescriptor> {
    fields_to_descriptors(ShowColumnRow::fields())
}

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

    use futures_async_stream::for_await;

    use crate::test_utils::LocalFrontend;

    #[tokio::test]
    async fn test_describe_handler() {
        let frontend = LocalFrontend::new(Default::default()).await;
        frontend
            .run_sql("create table t (v1 int, v2 int, v3 int primary key, v4 int);")
            .await
            .unwrap();

        frontend
            .run_sql("create index idx1 on t (v1 DESC, v2);")
            .await
            .unwrap();

        let sql = "describe t";
        let mut pg_response = frontend.run_sql(sql).await.unwrap();

        let mut columns = HashMap::new();
        #[for_await]
        for row_set in pg_response.values_stream() {
            let row_set = row_set.unwrap();
            for row in row_set {
                columns.insert(
                    std::str::from_utf8(row.index(0).as_ref().unwrap())
                        .unwrap()
                        .to_string(),
                    std::str::from_utf8(row.index(1).as_ref().unwrap())
                        .unwrap()
                        .to_string(),
                );
            }
        }

        let expected_columns: HashMap<String, String> = maplit::hashmap! {
            "v1".into() => "integer".into(),
            "v2".into() => "integer".into(),
            "v3".into() => "integer".into(),
            "v4".into() => "integer".into(),
            "primary key".into() => "v3".into(),
            "distribution key".into() => "v3".into(),
            "_rw_timestamp".into() => "timestamp with time zone".into(),
            "idx1".into() => "index(v1 DESC, v2 ASC, v3 ASC) include(v4) distributed by(v1)".into(),
            "table description".into() => "t".into(),
        };

        assert_eq!(columns, expected_columns);
    }
}