-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Fix async_udf batch size behaviour #18819
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from all commits
5a0120c
b77c21b
2a026b9
a9f56d3
817f1dc
a4c3cf6
118121b
d5bdbde
2dd8b25
bed6933
f3efa4a
8f5d4e3
7c977f6
475e9f6
59d407c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,139 @@ | ||
| // Licensed to the Apache Software Foundation (ASF) under one | ||
| // or more contributor license agreements. See the NOTICE file | ||
| // distributed with this work for additional information | ||
| // regarding copyright ownership. The ASF licenses this file | ||
| // to you under the Apache License, Version 2.0 (the | ||
| // "License"); you may not use this file except in compliance | ||
| // with the License. You may obtain a copy of the License at | ||
| // | ||
| // http://www.apache.org/licenses/LICENSE-2.0 | ||
| // | ||
| // Unless required by applicable law or agreed to in writing, | ||
| // software distributed under the License is distributed on an | ||
| // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| // KIND, either express or implied. See the License for the | ||
| // specific language governing permissions and limitations | ||
| // under the License. | ||
|
|
||
| use std::sync::Arc; | ||
|
|
||
| use arrow::array::{Int32Array, RecordBatch, StringArray}; | ||
| use arrow::datatypes::{DataType, Field, Schema}; | ||
| use async_trait::async_trait; | ||
| use datafusion::prelude::*; | ||
| use datafusion_common::{assert_batches_eq, Result}; | ||
| use datafusion_expr::async_udf::{AsyncScalarUDF, AsyncScalarUDFImpl}; | ||
| use datafusion_expr::{ | ||
| ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, | ||
| }; | ||
|
|
||
| // This test checks the case where batch_size doesn't evenly divide | ||
| // the number of rows. | ||
| #[tokio::test] | ||
| async fn test_async_udf_with_non_modular_batch_size() -> Result<()> { | ||
| let num_rows = 3; | ||
| let batch_size = 2; | ||
|
|
||
| let schema = Arc::new(Schema::new(vec![ | ||
| Field::new("id", DataType::Int32, false), | ||
| Field::new("prompt", DataType::Utf8, false), | ||
| ])); | ||
|
|
||
| let batch = RecordBatch::try_new( | ||
| schema.clone(), | ||
| vec![ | ||
| Arc::new(Int32Array::from((0..num_rows).collect::<Vec<i32>>())), | ||
| Arc::new(StringArray::from( | ||
| (0..num_rows) | ||
| .map(|i| format!("prompt{i}")) | ||
| .collect::<Vec<_>>(), | ||
| )), | ||
| ], | ||
| )?; | ||
|
|
||
| let ctx = SessionContext::new(); | ||
| ctx.register_batch("test_table", batch)?; | ||
|
|
||
| ctx.register_udf( | ||
| AsyncScalarUDF::new(Arc::new(TestAsyncUDFImpl::new(batch_size))) | ||
| .into_scalar_udf(), | ||
| ); | ||
|
|
||
| let df = ctx | ||
| .sql("SELECT id, test_async_udf(prompt) as result FROM test_table") | ||
| .await?; | ||
|
|
||
| let result = df.collect().await?; | ||
|
|
||
| assert_batches_eq!( | ||
| &[ | ||
| "+----+---------+", | ||
| "| id | result |", | ||
| "+----+---------+", | ||
| "| 0 | prompt0 |", | ||
| "| 1 | prompt1 |", | ||
| "| 2 | prompt2 |", | ||
| "+----+---------+" | ||
| ], | ||
| &result | ||
| ); | ||
|
|
||
| Ok(()) | ||
| } | ||
|
|
||
| #[derive(Debug, PartialEq, Eq, Hash, Clone)] | ||
| struct TestAsyncUDFImpl { | ||
| batch_size: usize, | ||
| signature: Signature, | ||
| } | ||
|
|
||
| impl TestAsyncUDFImpl { | ||
| fn new(batch_size: usize) -> Self { | ||
| Self { | ||
| batch_size, | ||
| signature: Signature::exact(vec![DataType::Utf8], Volatility::Volatile), | ||
| } | ||
| } | ||
| } | ||
|
|
||
| impl ScalarUDFImpl for TestAsyncUDFImpl { | ||
| fn as_any(&self) -> &dyn std::any::Any { | ||
| self | ||
| } | ||
|
|
||
| fn name(&self) -> &str { | ||
| "test_async_udf" | ||
| } | ||
|
|
||
| fn signature(&self) -> &Signature { | ||
| &self.signature | ||
| } | ||
|
|
||
| fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> { | ||
| Ok(DataType::Utf8) | ||
| } | ||
|
|
||
| fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result<ColumnarValue> { | ||
| panic!("Call invoke_async_with_args instead") | ||
| } | ||
| } | ||
|
|
||
| #[async_trait] | ||
| impl AsyncScalarUDFImpl for TestAsyncUDFImpl { | ||
| fn ideal_batch_size(&self) -> Option<usize> { | ||
| Some(self.batch_size) | ||
| } | ||
| async fn invoke_async_with_args( | ||
| &self, | ||
| args: ScalarFunctionArgs, | ||
| ) -> Result<ColumnarValue> { | ||
| let arg1 = &args.args[0]; | ||
| let results = call_external_service(arg1.clone()).await?; | ||
| Ok(results) | ||
| } | ||
| } | ||
|
|
||
| /// Simulates calling an async external service | ||
| async fn call_external_service(arg1: ColumnarValue) -> Result<ColumnarValue> { | ||
| Ok(arg1) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -192,10 +192,16 @@ impl AsyncFuncExpr { | |
| ); | ||
| } | ||
|
|
||
| let datas = ColumnarValue::values_to_arrays(&result_batches)? | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, I agree with this assesment Your solution works and looks good to me. I played around with it locally and I think using the concat kernel might be a little faster, for your consideration: Here is what works well for me locally diff --git a/datafusion/physical-expr/src/async_scalar_function.rs b/datafusion/physical-expr/src/async_scalar_function.rs
index 1a794f411b..afb01f7b5e 100644
--- a/datafusion/physical-expr/src/async_scalar_function.rs
+++ b/datafusion/physical-expr/src/async_scalar_function.rs
@@ -16,7 +16,8 @@
// under the License.
use crate::ScalarFunctionExpr;
-use arrow::array::{make_array, MutableArrayData, RecordBatch};
+use arrow::array::RecordBatch;
+use arrow::compute::concat;
use arrow::datatypes::{DataType, Field, FieldRef, Schema};
use datafusion_common::config::ConfigOptions;
use datafusion_common::Result;
@@ -192,23 +193,22 @@ impl AsyncFuncExpr {
);
}
+ // Create the all the arrays into a single array
let datas = result_batches
- .iter()
+ .into_iter()
.map(|cv| match cv {
- ColumnarValue::Array(arr) => Ok(arr.to_data()),
- ColumnarValue::Scalar(scalar) => {
- Ok(scalar.to_array_of_size(1)?.to_data())
- }
+ ColumnarValue::Array(arr) => Ok(arr),
+ ColumnarValue::Scalar(scalar) => Ok(scalar.to_array_of_size(1)?),
})
.collect::<Result<Vec<_>>>()?;
- let total_len = datas.iter().map(|d| d.len()).sum();
- let mut mutable = MutableArrayData::new(datas.iter().collect(), false, total_len);
- datas.iter().enumerate().for_each(|(i, data)| {
- mutable.extend(i, 0, data.len());
- });
- let array_ref = make_array(mutable.freeze());
- Ok(ColumnarValue::Array(array_ref))
+ // Get references to the arrays as dyn Array to call concat
+ let dyn_arrays = datas
+ .iter()
+ .map(|arr| arr as &dyn arrow::array::Array)
+ .collect::<Vec<_>>();
+ let result_array = concat(&dyn_arrays)?;
+ Ok(ColumnarValue::Array(result_array))
}
} |
||
| let datas = result_batches | ||
| .iter() | ||
| .map(|b| b.to_data()) | ||
| .collect::<Vec<_>>(); | ||
| .map(|cv| match cv { | ||
| ColumnarValue::Array(arr) => Ok(arr.to_data()), | ||
| ColumnarValue::Scalar(scalar) => { | ||
| Ok(scalar.to_array_of_size(1)?.to_data()) | ||
| } | ||
| }) | ||
| .collect::<Result<Vec<_>>>()?; | ||
|
|
||
| let total_len = datas.iter().map(|d| d.len()).sum(); | ||
| let mut mutable = MutableArrayData::new(datas.iter().collect(), false, total_len); | ||
| datas.iter().enumerate().for_each(|(i, data)| { | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I verified this test fails without the code fix in this PR