risingwave_frontend/catalog/
function_catalog.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use enum_as_inner::EnumAsInner;
16use parse_display::Display;
17use risingwave_common::catalog::FunctionId;
18use risingwave_common::types::DataType;
19use risingwave_common::util::epoch::Epoch;
20use risingwave_pb::catalog::PbFunction;
21use risingwave_pb::catalog::function::PbKind;
22use risingwave_pb::expr::{PbUdfExprVersion, PbUserDefinedFunctionMetadata};
23
24use crate::catalog::OwnedByUserCatalog;
25
26#[derive(Clone, PartialEq, Eq, Hash, Debug)]
27pub struct FunctionCatalog {
28    pub id: FunctionId,
29    pub name: String,
30    pub owner: u32,
31    pub kind: FunctionKind,
32    pub arg_names: Vec<String>,
33    pub arg_types: Vec<DataType>,
34    pub return_type: DataType,
35    pub language: String,
36    pub runtime: Option<String>,
37    pub name_in_runtime: Option<String>,
38    pub body: Option<String>,
39    pub link: Option<String>,
40    pub compressed_binary: Option<Vec<u8>>,
41    pub always_retry_on_network_error: bool,
42    pub is_async: Option<bool>,
43    pub is_batched: Option<bool>,
44    pub created_at_epoch: Option<Epoch>,
45    pub created_at_cluster_version: Option<String>,
46}
47
48#[derive(Clone, Display, PartialEq, Eq, Hash, Debug, EnumAsInner)]
49#[display(style = "UPPERCASE")]
50pub enum FunctionKind {
51    Scalar,
52    Table,
53    Aggregate,
54}
55
56impl From<&PbKind> for FunctionKind {
57    fn from(prost: &PbKind) -> Self {
58        use risingwave_pb::catalog::function::*;
59        match prost {
60            Kind::Scalar(ScalarFunction {}) => Self::Scalar,
61            Kind::Table(TableFunction {}) => Self::Table,
62            Kind::Aggregate(AggregateFunction {}) => Self::Aggregate,
63        }
64    }
65}
66
67impl From<&PbFunction> for FunctionCatalog {
68    fn from(prost: &PbFunction) -> Self {
69        FunctionCatalog {
70            id: prost.id.into(),
71            name: prost.name.clone(),
72            owner: prost.owner,
73            kind: prost.kind.as_ref().unwrap().into(),
74            arg_names: prost.arg_names.clone(),
75            arg_types: prost.arg_types.iter().map(|arg| arg.into()).collect(),
76            return_type: prost.return_type.as_ref().expect("no return type").into(),
77            language: prost.language.clone(),
78            runtime: prost.runtime.clone(),
79            name_in_runtime: prost.name_in_runtime.clone(),
80            body: prost.body.clone(),
81            link: prost.link.clone(),
82            compressed_binary: prost.compressed_binary.clone(),
83            always_retry_on_network_error: prost.always_retry_on_network_error,
84            is_async: prost.is_async,
85            is_batched: prost.is_batched,
86            created_at_epoch: prost.created_at_epoch.map(Epoch::from),
87            created_at_cluster_version: prost.created_at_cluster_version.clone(),
88        }
89    }
90}
91
92impl From<&FunctionCatalog> for PbUserDefinedFunctionMetadata {
93    fn from(c: &FunctionCatalog) -> Self {
94        PbUserDefinedFunctionMetadata {
95            arg_names: c.arg_names.clone(),
96            arg_types: c.arg_types.iter().map(|t| t.to_protobuf()).collect(),
97            return_type: Some(c.return_type.to_protobuf()),
98            language: c.language.clone(),
99            runtime: c.runtime.clone(),
100            link: c.link.clone(),
101            identifier: c.name_in_runtime.clone(),
102            body: c.body.clone(),
103            compressed_binary: c.compressed_binary.clone(),
104            version: PbUdfExprVersion::LATEST as _,
105        }
106    }
107}
108
109impl OwnedByUserCatalog for FunctionCatalog {
110    fn owner(&self) -> u32 {
111        self.owner
112    }
113}