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::{ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl};
19use arrow::datatypes::{DataType, FieldRef};
20use async_trait::async_trait;
21use datafusion_common::error::Result;
22use datafusion_common::internal_err;
23use datafusion_expr_common::columnar_value::ColumnarValue;
24use datafusion_expr_common::signature::Signature;
25use std::any::Any;
26use std::fmt::{Debug, Display};
27use std::hash::{Hash, Hasher};
28use std::sync::Arc;
29
30/// A scalar UDF that can invoke using async methods
31///
32/// Note this is less efficient than the ScalarUDFImpl, but it can be used
33/// to register remote functions in the context.
34///
35/// The name is chosen to mirror ScalarUDFImpl
36#[async_trait]
37pub trait AsyncScalarUDFImpl: ScalarUDFImpl {
38    /// The ideal batch size for this function.
39    ///
40    /// This is used to determine what size of data to be evaluated at once.
41    /// If None, the whole batch will be evaluated at once.
42    fn ideal_batch_size(&self) -> Option<usize> {
43        None
44    }
45
46    /// Invoke the function asynchronously with the async arguments
47    async fn invoke_async_with_args(
48        &self,
49        args: ScalarFunctionArgs,
50    ) -> Result<ColumnarValue>;
51}
52
53/// A scalar UDF that must be invoked using async methods
54///
55/// Note this is not meant to be used directly, but is meant to be an implementation detail
56/// for AsyncUDFImpl.
57#[derive(Debug)]
58pub struct AsyncScalarUDF {
59    inner: Arc<dyn AsyncScalarUDFImpl>,
60}
61
62impl PartialEq for AsyncScalarUDF {
63    fn eq(&self, other: &Self) -> bool {
64        // Deconstruct to catch any new fields added in future
65        let Self { inner } = self;
66        inner.dyn_eq(other.inner.as_any())
67    }
68}
69impl Eq for AsyncScalarUDF {}
70
71impl Hash for AsyncScalarUDF {
72    fn hash<H: Hasher>(&self, state: &mut H) {
73        // Deconstruct to catch any new fields added in future
74        let Self { inner } = self;
75        inner.dyn_hash(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}
135
136#[cfg(test)]
137mod tests {
138    use std::{
139        hash::{DefaultHasher, Hash, Hasher},
140        sync::Arc,
141    };
142
143    use arrow::datatypes::DataType;
144    use async_trait::async_trait;
145    use datafusion_common::error::Result;
146    use datafusion_expr_common::{columnar_value::ColumnarValue, signature::Signature};
147
148    use crate::{
149        async_udf::{AsyncScalarUDF, AsyncScalarUDFImpl},
150        ScalarFunctionArgs, ScalarUDFImpl,
151    };
152
153    #[derive(Debug, PartialEq, Eq, Hash, Clone)]
154    struct TestAsyncUDFImpl1 {
155        a: i32,
156    }
157
158    impl ScalarUDFImpl for TestAsyncUDFImpl1 {
159        fn as_any(&self) -> &dyn std::any::Any {
160            self
161        }
162
163        fn name(&self) -> &str {
164            todo!()
165        }
166
167        fn signature(&self) -> &Signature {
168            todo!()
169        }
170
171        fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
172            todo!()
173        }
174
175        fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result<ColumnarValue> {
176            todo!()
177        }
178    }
179
180    #[async_trait]
181    impl AsyncScalarUDFImpl for TestAsyncUDFImpl1 {
182        async fn invoke_async_with_args(
183            &self,
184            _args: ScalarFunctionArgs,
185        ) -> Result<ColumnarValue> {
186            todo!()
187        }
188    }
189
190    #[derive(Debug, PartialEq, Eq, Hash, Clone)]
191    struct TestAsyncUDFImpl2 {
192        a: i32,
193    }
194
195    impl ScalarUDFImpl for TestAsyncUDFImpl2 {
196        fn as_any(&self) -> &dyn std::any::Any {
197            self
198        }
199
200        fn name(&self) -> &str {
201            todo!()
202        }
203
204        fn signature(&self) -> &Signature {
205            todo!()
206        }
207
208        fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
209            todo!()
210        }
211
212        fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result<ColumnarValue> {
213            todo!()
214        }
215    }
216
217    #[async_trait]
218    impl AsyncScalarUDFImpl for TestAsyncUDFImpl2 {
219        async fn invoke_async_with_args(
220            &self,
221            _args: ScalarFunctionArgs,
222        ) -> Result<ColumnarValue> {
223            todo!()
224        }
225    }
226
227    fn hash<T: Hash>(value: &T) -> u64 {
228        let hasher = &mut DefaultHasher::new();
229        value.hash(hasher);
230        hasher.finish()
231    }
232
233    #[test]
234    fn test_async_udf_partial_eq_and_hash() {
235        // Inner is same cloned arc -> equal
236        let inner = Arc::new(TestAsyncUDFImpl1 { a: 1 });
237        let a = AsyncScalarUDF::new(Arc::clone(&inner) as Arc<dyn AsyncScalarUDFImpl>);
238        let b = AsyncScalarUDF::new(inner);
239        assert_eq!(a, b);
240        assert_eq!(hash(&a), hash(&b));
241
242        // Inner is distinct arc -> still equal
243        let a = AsyncScalarUDF::new(Arc::new(TestAsyncUDFImpl1 { a: 1 }));
244        let b = AsyncScalarUDF::new(Arc::new(TestAsyncUDFImpl1 { a: 1 }));
245        assert_eq!(a, b);
246        assert_eq!(hash(&a), hash(&b));
247
248        // Negative case: inner is different value -> not equal
249        let a = AsyncScalarUDF::new(Arc::new(TestAsyncUDFImpl1 { a: 1 }));
250        let b = AsyncScalarUDF::new(Arc::new(TestAsyncUDFImpl1 { a: 2 }));
251        assert_ne!(a, b);
252        assert_ne!(hash(&a), hash(&b));
253
254        // Negative case: different functions -> not equal
255        let a = AsyncScalarUDF::new(Arc::new(TestAsyncUDFImpl1 { a: 1 }));
256        let b = AsyncScalarUDF::new(Arc::new(TestAsyncUDFImpl2 { a: 1 }));
257        assert_ne!(a, b);
258        assert_ne!(hash(&a), hash(&b));
259    }
260}