Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: support sink/view column schema in commands show/create and system catalog #13626

Merged
merged 4 commits into from
Nov 24, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
72 changes: 48 additions & 24 deletions e2e_test/ddl/show.slt
Original file line number Diff line number Diff line change
Expand Up @@ -164,30 +164,54 @@ drop view v3;
statement ok
drop view v1;

statement ok
CREATE SINK sink3 AS select * from t3 WITH (
connector = 'blackhole'
);

query TTTT
describe sink3;
----
v1 integer false NULL
v2 integer false NULL
v3 integer false NULL
t3._row_id serial true NULL
distribution key t3._row_id NULL NULL
table description sink3 NULL NULL

query TTTT
show columns from sink3;
----
v1 integer false NULL
v2 integer false NULL
v3 integer false NULL
t3._row_id serial true NULL

statement ok
drop sink sink3;

statement ok
drop table t3;

# todo: re-enable it when we support these commands on view
# https://github.com/risingwavelabs/risingwave/issues/11234
#query TT
#describe pg_matviews;
#----
#schemaname varchar
#matviewname varchar
#matviewowner integer
#definition varchar
#matviewid integer
#matviewtimezone varchar
#matviewgraph varchar
#primary key schemaname, matviewname
#
#query TT
#show columns from pg_catalog.pg_matviews;
#----
#schemaname varchar
#matviewname varchar
#matviewowner integer
#definition varchar
#matviewid integer
#matviewtimezone varchar
#matviewgraph varchar
query TTTT
describe pg_matviews;
----
schemaname character varying false NULL
matviewname character varying false NULL
matviewowner integer false NULL
definition character varying false NULL
matviewid integer false NULL
matviewtimezone character varying false NULL
matviewgraph character varying false NULL
table description pg_matviews NULL NULL

query TTTT
show columns from pg_catalog.pg_matviews;
----
schemaname character varying false NULL
matviewname character varying false NULL
matviewowner integer false NULL
definition character varying false NULL
matviewid integer false NULL
matviewtimezone character varying false NULL
matviewgraph character varying false NULL
68 changes: 68 additions & 0 deletions src/frontend/src/binder/for_system.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
// Copyright 2023 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::sync::Arc;

use risingwave_common::error::Result;
use risingwave_connector::sink::catalog::SinkCatalog;
use risingwave_sqlparser::ast::ObjectName;

use crate::binder::BindFor;
use crate::catalog::root_catalog::SchemaPath;
use crate::catalog::view_catalog::ViewCatalog;
use crate::Binder;

pub struct BoundSink {
pub sink_catalog: Arc<SinkCatalog>,
}

pub struct BoundView {
pub view_catalog: Arc<ViewCatalog>,
}

impl Binder {
pub fn bind_sink_by_name(&self, name: ObjectName) -> Result<BoundSink> {
matches!(self.bind_for, BindFor::System);
let (schema_name, sink_name) = Self::resolve_schema_qualified_name(&self.db_name, name)?;

let search_path = SchemaPath::new(
schema_name.as_deref(),
&self.search_path,
&self.auth_context.user_name,
);
let (sink_catalog, _) =
self.catalog
.get_sink_by_name(&self.db_name, search_path, &sink_name)?;
Ok(BoundSink {
sink_catalog: sink_catalog.clone(),
})
}

pub fn bind_view_by_name(&self, name: ObjectName) -> Result<BoundView> {
matches!(self.bind_for, BindFor::System);
let (schema_name, view_name) = Self::resolve_schema_qualified_name(&self.db_name, name)?;

let search_path = SchemaPath::new(
schema_name.as_deref(),
&self.search_path,
&self.auth_context.user_name,
);
let (view_catalog, _) =
self.catalog
.get_view_by_name(&self.db_name, search_path, &view_name)?;
Ok(BoundView {
view_catalog: view_catalog.clone(),
})
}
}
2 changes: 2 additions & 0 deletions src/frontend/src/binder/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ mod bind_param;
mod create;
mod delete;
mod expr;
mod for_system;
mod insert;
mod query;
mod relation;
Expand All @@ -41,6 +42,7 @@ mod values;
pub use bind_context::{BindContext, Clause, LateralBindContext};
pub use delete::BoundDelete;
pub use expr::{bind_data_type, bind_struct_field};
pub use for_system::*;
pub use insert::BoundInsert;
use pgwire::pg_server::{Session, SessionId};
pub use query::BoundQuery;
Expand Down
25 changes: 24 additions & 1 deletion src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,29 @@ impl SysCatalogReaderImpl {
})
});

let sink_rows = schema
.iter_sink()
.flat_map(|sink| {
sink.full_columns()
.iter()
.enumerate()
.map(|(index, column)| {
OwnedRow::new(vec![
Some(ScalarImpl::Int32(sink.id.sink_id as i32)),
Some(ScalarImpl::Utf8(column.name().into())),
Some(ScalarImpl::Int32(index as i32 + 1)),
Some(ScalarImpl::Bool(column.is_hidden)),
Some(ScalarImpl::Bool(sink.downstream_pk.contains(&index))),
Some(ScalarImpl::Bool(sink.distribution_key.contains(&index))),
Some(ScalarImpl::Utf8(column.data_type().to_string().into())),
Some(ScalarImpl::Int32(column.data_type().to_oid())),
Some(ScalarImpl::Int16(column.data_type().type_len())),
Some(ScalarImpl::Utf8(column.data_type().pg_name().into())),
])
})
})
.chain(view_rows);

let rows = schema
.iter_system_tables()
.flat_map(|table| {
Expand All @@ -86,7 +109,7 @@ impl SysCatalogReaderImpl {
])
})
})
.chain(view_rows);
.chain(sink_rows);

schema
.iter_valid_table()
Expand Down
Loading
Loading