risingwave_frontend/catalog/
function_catalog.rs

1// Copyright 2023 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};
23use risingwave_pb::id::UserId;
24
25use crate::catalog::OwnedByUserCatalog;
26
27#[derive(Clone, PartialEq, Eq, Hash, Debug)]
28pub struct FunctionCatalog {
29    pub id: FunctionId,
30    pub name: String,
31    pub owner: UserId,
32    pub kind: FunctionKind,
33    pub arg_names: Vec<String>,
34    pub arg_types: Vec<DataType>,
35    pub return_type: DataType,
36    pub language: String,
37    pub runtime: Option<String>,
38    pub name_in_runtime: Option<String>,
39    pub body: Option<String>,
40    pub link: Option<String>,
41    pub compressed_binary: Option<Vec<u8>>,
42    pub always_retry_on_network_error: bool,
43    pub is_async: Option<bool>,
44    pub is_batched: Option<bool>,
45    pub created_at_epoch: Option<Epoch>,
46    pub created_at_cluster_version: Option<String>,
47}
48
49#[derive(Clone, Display, PartialEq, Eq, Hash, Debug, EnumAsInner)]
50#[display(style = "UPPERCASE")]
51pub enum FunctionKind {
52    Scalar,
53    Table,
54    Aggregate,
55}
56
57impl From<&PbKind> for FunctionKind {
58    fn from(prost: &PbKind) -> Self {
59        use risingwave_pb::catalog::function::*;
60        match prost {
61            Kind::Scalar(ScalarFunction {}) => Self::Scalar,
62            Kind::Table(TableFunction {}) => Self::Table,
63            Kind::Aggregate(AggregateFunction {}) => Self::Aggregate,
64        }
65    }
66}
67
68impl From<&PbFunction> for FunctionCatalog {
69    fn from(prost: &PbFunction) -> Self {
70        FunctionCatalog {
71            id: prost.id,
72            name: prost.name.clone(),
73            owner: prost.owner,
74            kind: prost.kind.as_ref().unwrap().into(),
75            arg_names: prost.arg_names.clone(),
76            arg_types: prost.arg_types.iter().map(|arg| arg.into()).collect(),
77            return_type: prost.return_type.as_ref().expect("no return type").into(),
78            language: prost.language.clone(),
79            runtime: prost.runtime.clone(),
80            name_in_runtime: prost.name_in_runtime.clone(),
81            body: prost.body.clone(),
82            link: prost.link.clone(),
83            compressed_binary: prost.compressed_binary.clone(),
84            always_retry_on_network_error: prost.always_retry_on_network_error,
85            is_async: prost.is_async,
86            is_batched: prost.is_batched,
87            created_at_epoch: prost.created_at_epoch.map(Epoch::from),
88            created_at_cluster_version: prost.created_at_cluster_version.clone(),
89        }
90    }
91}
92
93impl From<&FunctionCatalog> for PbUserDefinedFunctionMetadata {
94    fn from(c: &FunctionCatalog) -> Self {
95        PbUserDefinedFunctionMetadata {
96            arg_names: c.arg_names.clone(),
97            arg_types: c.arg_types.iter().map(|t| t.to_protobuf()).collect(),
98            return_type: Some(c.return_type.to_protobuf()),
99            language: c.language.clone(),
100            runtime: c.runtime.clone(),
101            link: c.link.clone(),
102            identifier: c.name_in_runtime.clone(),
103            body: c.body.clone(),
104            compressed_binary: c.compressed_binary.clone(),
105            version: PbUdfExprVersion::LATEST as _,
106            is_async: c.is_async,
107            is_batched: c.is_batched,
108        }
109    }
110}
111
112impl OwnedByUserCatalog for FunctionCatalog {
113    fn owner(&self) -> UserId {
114        self.owner
115    }
116}