@@ -8,7 +8,8 @@ use ::http::HeaderName;
88use arrow:: datatypes:: DataType ;
99use datafusion_common:: { DataFusionError , Result as DataFusionResult , config:: ConfigOptions } ;
1010use datafusion_expr:: {
11- ColumnarValue , ScalarFunctionArgs , ScalarUDFImpl , Signature , async_udf:: AsyncScalarUDFImpl ,
11+ ColumnarValue , ScalarFunctionArgs , ScalarUDFImpl , Signature ,
12+ async_udf:: { AsyncScalarUDF , AsyncScalarUDFImpl } ,
1213} ;
1314use tokio:: { runtime:: Handle , sync:: Mutex } ;
1415use wasmtime:: {
@@ -398,6 +399,11 @@ impl WasmScalarUdf {
398399
399400 Ok ( udfs)
400401 }
402+
403+ /// Convert this [WasmScalarUdf] into an [AsyncScalarUDF].
404+ pub fn as_async_udf ( self ) -> AsyncScalarUDF {
405+ AsyncScalarUDF :: new ( Arc :: new ( self ) )
406+ }
401407}
402408
403409impl std:: fmt:: Debug for WasmScalarUdf {
@@ -490,7 +496,7 @@ impl AsyncScalarUDFImpl for WasmScalarUdf {
490496 let columnar_value: ColumnarValue = return_type. try_into ( ) ?;
491497 match columnar_value {
492498 ColumnarValue :: Array ( v) => Ok ( v) ,
493- ColumnarValue :: Scalar ( v) => v. to_array_of_size ( 1 ) ,
499+ ColumnarValue :: Scalar ( v) => v. to_array_of_size ( args . number_rows as usize ) ,
494500 }
495501 }
496502}
0 commit comments