risingwave_frontend/expr/function_impl/
cast_regclass.rs1use risingwave_common::id::ObjectId;
16use risingwave_common::session_config::SearchPath;
17use risingwave_expr::{ExprError, capture_context, function};
18use risingwave_sqlparser::parser::{Parser, ParserError};
19use thiserror::Error;
20use thiserror_ext::AsReport;
21
22use super::context::{AUTH_CONTEXT, CATALOG_READER, DB_NAME, SEARCH_PATH};
23use crate::Binder;
24use crate::binder::ResolveQualifiedNameError;
25use crate::catalog::root_catalog::SchemaPath;
26use crate::catalog::{CatalogError, CatalogReader};
27use crate::session::AuthContext;
28
29#[derive(Error, Debug)]
30enum ResolveRegclassError {
31 #[error("parse object name failed: {0}")]
32 Parser(#[from] ParserError),
33 #[error("catalog error: {0}")]
34 Catalog(#[from] CatalogError),
35 #[error("resolve qualified name error: {0}")]
36 ResolveQualifiedName(#[from] ResolveQualifiedNameError),
37}
38
39impl From<ResolveRegclassError> for ExprError {
40 fn from(e: ResolveRegclassError) -> Self {
41 match e {
42 ResolveRegclassError::Parser(e) => ExprError::Parse(e.to_report_string().into()),
43 ResolveRegclassError::Catalog(e) => ExprError::InvalidParam {
44 name: "name",
45 reason: e.to_report_string().into(),
46 },
47 ResolveRegclassError::ResolveQualifiedName(e) => ExprError::InvalidParam {
48 name: "name",
49 reason: e.to_report_string().into(),
50 },
51 }
52 }
53}
54
55#[capture_context(CATALOG_READER, AUTH_CONTEXT, SEARCH_PATH, DB_NAME)]
56fn resolve_regclass_impl(
57 catalog: &CatalogReader,
58 auth_context: &AuthContext,
59 search_path: &SearchPath,
60 db_name: &str,
61 class_name: &str,
62) -> Result<ObjectId, ExprError> {
63 resolve_regclass_inner(catalog, auth_context, search_path, db_name, class_name)
64 .map_err(Into::into)
65}
66
67fn resolve_regclass_inner(
68 catalog: &CatalogReader,
69 auth_context: &AuthContext,
70 search_path: &SearchPath,
71 db_name: &str,
72 class_name: &str,
73) -> Result<ObjectId, ResolveRegclassError> {
74 let obj = Parser::parse_object_name_str(class_name)?;
79
80 let (schema_name, class_name) = Binder::resolve_schema_qualified_name(db_name, &obj)?;
81 let schema_path = SchemaPath::new(schema_name.as_deref(), search_path, &auth_context.user_name);
82 Ok(catalog
83 .read_guard()
84 .get_id_by_class_name(db_name, schema_path, &class_name)?)
85}
86
87#[function("cast_regclass(varchar) -> int4")]
88fn cast_regclass(class_name: &str) -> Result<i32, ExprError> {
89 let oid = resolve_regclass_impl_captured(class_name)?;
90 Ok(oid.as_i32_id())
91}