From 36ac4542cb59c6e4176368bbcc0015fadc2dceb9 Mon Sep 17 00:00:00 2001 From: Yuhao Su Date: Wed, 17 Jul 2024 12:49:34 -0500 Subject: [PATCH] init --- .../catalog/system_catalog/rw_catalog/mod.rs | 1 + .../system_catalog/rw_catalog/rw_secrets.rs | 45 +++++++++++++++++++ 2 files changed, 46 insertions(+) create mode 100644 src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs index 4e0e165a6d524..890dc92b68a04 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs @@ -43,6 +43,7 @@ mod rw_parallel_units; mod rw_relation_info; mod rw_relations; mod rw_schemas; +mod rw_secrets; mod rw_sinks; mod rw_sources; mod rw_streaming_parallelism; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs new file mode 100644 index 0000000000000..09c9a98798e06 --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_secrets.rs @@ -0,0 +1,45 @@ +// 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 risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; + +use crate::catalog::system_catalog::SysCatalogReaderImpl; +use crate::error::Result; + +#[derive(Fields)] +struct RwSecret { + #[primary_key] + id: i32, + name: String, + owner: i32, + acl: String, +} + +#[system_catalog(table, "rw_catalog.rw_secrets")] +fn read_rw_view_info(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + + Ok(schemas + .flat_map(|schema| { + schema.iter_secret().map(|secret| RwSecret { + id: secret.id.secret_id() as i32, + name: secret.name.clone(), + owner: secret.owner as i32, + acl: "".into(), + }) + }) + .collect()) +}