datafusion_expr/
async_udf.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use crate::ptr_eq::{arc_ptr_eq, arc_ptr_hash};
19use crate::{ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl};
20use arrow::datatypes::{DataType, FieldRef};
21use async_trait::async_trait;
22use datafusion_common::error::Result;
23use datafusion_common::internal_err;
24use datafusion_expr_common::columnar_value::ColumnarValue;
25use datafusion_expr_common::signature::Signature;
26use std::any::Any;
27use std::fmt::{Debug, Display};
28use std::hash::{Hash, Hasher};
29use std::sync::Arc;
30
31/// A scalar UDF that can invoke using async methods
32///
33/// Note this is less efficient than the ScalarUDFImpl, but it can be used
34/// to register remote functions in the context.
35///
36/// The name is chosen to mirror ScalarUDFImpl
37#[async_trait]
38pub trait AsyncScalarUDFImpl: ScalarUDFImpl {
39    /// The ideal batch size for this function.
40    ///
41    /// This is used to determine what size of data to be evaluated at once.
42    /// If None, the whole batch will be evaluated at once.
43    fn ideal_batch_size(&self) -> Option<usize> {
44        None
45    }
46
47    /// Invoke the function asynchronously with the async arguments
48    async fn invoke_async_with_args(
49        &self,
50        args: ScalarFunctionArgs,
51    ) -> Result<ColumnarValue>;
52}
53
54/// A scalar UDF that must be invoked using async methods
55///
56/// Note this is not meant to be used directly, but is meant to be an implementation detail
57/// for AsyncUDFImpl.
58#[derive(Debug)]
59pub struct AsyncScalarUDF {
60    inner: Arc<dyn AsyncScalarUDFImpl>,
61}
62
63impl PartialEq for AsyncScalarUDF {
64    fn eq(&self, other: &Self) -> bool {
65        let Self { inner } = self;
66        // TODO when MSRV >= 1.86.0, switch to `inner.equals(other.inner.as_ref())` leveraging trait upcasting.
67        arc_ptr_eq(inner, &other.inner)
68    }
69}
70impl Eq for AsyncScalarUDF {}
71
72impl Hash for AsyncScalarUDF {
73    fn hash<H: Hasher>(&self, state: &mut H) {
74        let Self { inner } = self;
75        arc_ptr_hash(inner, state);
76    }
77}
78
79impl AsyncScalarUDF {
80    pub fn new(inner: Arc<dyn AsyncScalarUDFImpl>) -> Self {
81        Self { inner }
82    }
83
84    /// The ideal batch size for this function
85    pub fn ideal_batch_size(&self) -> Option<usize> {
86        self.inner.ideal_batch_size()
87    }
88
89    /// Turn this AsyncUDF into a ScalarUDF, suitable for
90    /// registering in the context
91    pub fn into_scalar_udf(self) -> ScalarUDF {
92        ScalarUDF::new_from_impl(self)
93    }
94
95    /// Invoke the function asynchronously with the async arguments
96    pub async fn invoke_async_with_args(
97        &self,
98        args: ScalarFunctionArgs,
99    ) -> Result<ColumnarValue> {
100        self.inner.invoke_async_with_args(args).await
101    }
102}
103
104impl ScalarUDFImpl for AsyncScalarUDF {
105    fn as_any(&self) -> &dyn Any {
106        self
107    }
108
109    fn name(&self) -> &str {
110        self.inner.name()
111    }
112
113    fn signature(&self) -> &Signature {
114        self.inner.signature()
115    }
116
117    fn return_type(&self, arg_types: &[DataType]) -> Result<DataType> {
118        self.inner.return_type(arg_types)
119    }
120
121    fn return_field_from_args(&self, args: ReturnFieldArgs) -> Result<FieldRef> {
122        self.inner.return_field_from_args(args)
123    }
124
125    fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result<ColumnarValue> {
126        internal_err!("async functions should not be called directly")
127    }
128}
129
130impl Display for AsyncScalarUDF {
131    fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result {
132        write!(f, "AsyncScalarUDF: {}", self.inner.name())
133    }
134}