Skip to content

Commit

Permalink
Fix ScalaUDF with zero arguments
Browse files Browse the repository at this point in the history
  • Loading branch information
viirya committed Jan 28, 2024
1 parent ff7dfc3 commit dd09c03
Show file tree
Hide file tree
Showing 8 changed files with 159 additions and 27 deletions.
18 changes: 17 additions & 1 deletion datafusion/core/src/physical_optimizer/projection_pushdown.rs
Original file line number Diff line number Diff line change
Expand Up @@ -1200,7 +1200,7 @@ mod tests {
use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics};
use datafusion_execution::object_store::ObjectStoreUrl;
use datafusion_execution::{SendableRecordBatchStream, TaskContext};
use datafusion_expr::{ColumnarValue, Operator};
use datafusion_expr::{ColumnarValue, Operator, Signature, Volatility};
use datafusion_physical_expr::expressions::{
BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr,
};
Expand Down Expand Up @@ -1245,6 +1245,10 @@ mod tests {
],
DataType::Int32,
None,
Signature::exact(
vec![DataType::Float32, DataType::Float32],
Volatility::Immutable,
),
)),
Arc::new(CaseExpr::try_new(
Some(Arc::new(Column::new("d", 2))),
Expand Down Expand Up @@ -1311,6 +1315,10 @@ mod tests {
],
DataType::Int32,
None,
Signature::exact(
vec![DataType::Float32, DataType::Float32],
Volatility::Immutable,
),
)),
Arc::new(CaseExpr::try_new(
Some(Arc::new(Column::new("d", 3))),
Expand Down Expand Up @@ -1380,6 +1388,10 @@ mod tests {
],
DataType::Int32,
None,
Signature::exact(
vec![DataType::Float32, DataType::Float32],
Volatility::Immutable,
),
)),
Arc::new(CaseExpr::try_new(
Some(Arc::new(Column::new("d", 2))),
Expand Down Expand Up @@ -1446,6 +1458,10 @@ mod tests {
],
DataType::Int32,
None,
Signature::exact(
vec![DataType::Float32, DataType::Float32],
Volatility::Immutable,
),
)),
Arc::new(CaseExpr::try_new(
Some(Arc::new(Column::new("d_new", 3))),
Expand Down
117 changes: 115 additions & 2 deletions datafusion/core/tests/user_defined/user_defined_scalar_functions.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,15 +16,22 @@
// under the License.

use arrow::compute::kernels::numeric::add;
use arrow_array::{ArrayRef, Float64Array, Int32Array, RecordBatch};
use arrow_array::{
ArrayRef, Float64Array, Int32Array, Int64Array, RecordBatch, UInt64Array, UInt8Array,
};
use arrow_schema::DataType::Float64;
use arrow_schema::{DataType, Field, Schema};
use datafusion::prelude::*;
use datafusion::{execution::registry::FunctionRegistry, test_util};
use datafusion_common::cast::as_float64_array;
use datafusion_common::{assert_batches_eq, cast::as_int32_array, Result, ScalarValue};
use datafusion_expr::TypeSignature::{Any, Variadic};
use datafusion_expr::{
create_udaf, create_udf, Accumulator, ColumnarValue, LogicalPlanBuilder, Volatility,
create_udaf, create_udf, Accumulator, ColumnarValue, LogicalPlanBuilder, ScalarUDF,
ScalarUDFImpl, Signature, Volatility,
};
use rand::{thread_rng, Rng};
use std::iter;
use std::sync::Arc;

/// test that casting happens on udfs.
Expand Down Expand Up @@ -392,6 +399,112 @@ async fn test_user_defined_functions_with_alias() -> Result<()> {
Ok(())
}

#[derive(Debug)]
pub struct RandomUDF {
signature: Signature,
}

impl RandomUDF {
pub fn new() -> Self {
Self {
signature: Signature::one_of(
vec![Any(0), Variadic(vec![Float64])],
Volatility::Volatile,
),
}
}
}

impl ScalarUDFImpl for RandomUDF {
fn as_any(&self) -> &dyn std::any::Any {
self
}

fn name(&self) -> &str {
"random_udf"
}

fn signature(&self) -> &Signature {
&self.signature
}

fn return_type(&self, _arg_types: &[DataType]) -> Result<DataType> {
Ok(Float64)
}

fn invoke(&self, args: &[ColumnarValue]) -> Result<ColumnarValue> {
let len: usize = match &args[0] {
ColumnarValue::Array(array) => array.len(),
_ => {
return Err(datafusion::error::DataFusionError::Internal(
"Invalid argument type".to_string(),
))
}
};
let mut rng = thread_rng();
let values = iter::repeat_with(|| rng.gen_range(0.1..1.0)).take(len);
let array = Float64Array::from_iter_values(values);
Ok(ColumnarValue::Array(Arc::new(array)))
}
}

#[tokio::test]
async fn test_user_defined_functions_zero_argument() -> Result<()> {
let ctx = SessionContext::new();

let schema = Arc::new(Schema::new(vec![
Field::new("index", DataType::UInt8, false),
Field::new("uint", DataType::UInt64, true),
Field::new("int", DataType::Int64, true),
Field::new("float", DataType::Float64, true),
]));

let batch = RecordBatch::try_new(
schema,
vec![
Arc::new(UInt8Array::from_iter_values([1, 2, 3])),
Arc::new(UInt64Array::from(vec![Some(2), Some(3), None])),
Arc::new(Int64Array::from(vec![Some(-2), Some(3), None])),
Arc::new(Float64Array::from(vec![Some(1.0), Some(3.3), None])),
],
)?;

ctx.register_batch("data_table", batch)?;

let random_normal_udf = ScalarUDF::from(RandomUDF::new());
ctx.register_udf(random_normal_udf);

let result = plan_and_collect(
&ctx,
"SELECT random_udf() AS random_udf, random() AS native_random FROM data_table",
)
.await?;

assert_eq!(result.len(), 1);
let batch = &result[0];
let random_udf = batch
.column(0)
.as_any()
.downcast_ref::<Float64Array>()
.unwrap();
let native_random = batch
.column(1)
.as_any()
.downcast_ref::<Float64Array>()
.unwrap();

assert_eq!(random_udf.len(), native_random.len());

let mut previous = 1.0;
for i in 0..random_udf.len() {
assert!(random_udf.value(i) >= 0.0 && random_udf.value(i) < 1.0);
assert!(random_udf.value(i) != previous);
previous = random_udf.value(i);
}

Ok(())
}

fn create_udf_context() -> SessionContext {
let ctx = SessionContext::new();
// register a custom UDF
Expand Down
1 change: 1 addition & 0 deletions datafusion/physical-expr/src/functions.rs
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ pub fn create_physical_expr(
input_phy_exprs.to_vec(),
data_type,
monotonicity,
fun.signature().clone(),
)))
}

Expand Down
18 changes: 6 additions & 12 deletions datafusion/physical-expr/src/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -260,7 +260,7 @@ pub fn create_physical_expr(
}

Expr::ScalarFunction(ScalarFunction { func_def, args }) => {
let mut physical_args = args
let physical_args = args
.iter()
.map(|e| create_physical_expr(e, input_dfschema, execution_props))
.collect::<Result<Vec<_>>>()?;
Expand All @@ -273,17 +273,11 @@ pub fn create_physical_expr(
execution_props,
)
}
ScalarFunctionDefinition::UDF(fun) => {
// udfs with zero params expect null array as input
if args.is_empty() {
physical_args.push(Arc::new(Literal::new(ScalarValue::Null)));
}
udf::create_physical_expr(
fun.clone().as_ref(),
&physical_args,
input_schema,
)
}
ScalarFunctionDefinition::UDF(fun) => udf::create_physical_expr(
fun.clone().as_ref(),
&physical_args,
input_schema,
),
ScalarFunctionDefinition::Name(_) => {
internal_err!("Function `Expr` with name should be resolved.")
}
Expand Down
10 changes: 9 additions & 1 deletion datafusion/physical-expr/src/scalar_function.rs
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ use arrow::record_batch::RecordBatch;
use datafusion_common::Result;
use datafusion_expr::{
expr_vec_fmt, BuiltinScalarFunction, ColumnarValue, FuncMonotonicity,
ScalarFunctionImplementation,
ScalarFunctionImplementation, Signature,
};

/// Physical expression of a scalar function
Expand All @@ -58,6 +58,8 @@ pub struct ScalarFunctionExpr {
// and it specifies the effect of an increase or decrease in
// the corresponding `arg` to the function value.
monotonicity: Option<FuncMonotonicity>,
// Signature of the function
signature: Signature,
}

impl Debug for ScalarFunctionExpr {
Expand All @@ -79,13 +81,15 @@ impl ScalarFunctionExpr {
args: Vec<Arc<dyn PhysicalExpr>>,
return_type: DataType,
monotonicity: Option<FuncMonotonicity>,
signature: Signature,
) -> Self {
Self {
fun,
name: name.to_owned(),
args,
return_type,
monotonicity,
signature,
}
}

Expand Down Expand Up @@ -149,6 +153,9 @@ impl PhysicalExpr for ScalarFunctionExpr {
{
vec![ColumnarValue::create_null_array(batch.num_rows())]
}
(0, _) if self.signature.type_signature.supports_zero_argument() => {
vec![ColumnarValue::create_null_array(batch.num_rows())]
}
_ => self
.args
.iter()
Expand All @@ -175,6 +182,7 @@ impl PhysicalExpr for ScalarFunctionExpr {
children,
self.return_type().clone(),
self.monotonicity.clone(),
self.signature.clone(),
)))
}

Expand Down
1 change: 1 addition & 0 deletions datafusion/physical-expr/src/udf.rs
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ pub fn create_physical_expr(
input_phy_exprs.to_vec(),
fun.return_type(&input_exprs_types)?,
fun.monotonicity()?,
fun.signature().clone(),
)))
}

Expand Down
19 changes: 8 additions & 11 deletions datafusion/proto/src/physical_plan/from_proto.rs
Original file line number Diff line number Diff line change
Expand Up @@ -340,21 +340,17 @@ pub fn parse_physical_expr(
// TODO Do not create new the ExecutionProps
let execution_props = ExecutionProps::new();

let fun_expr = functions::create_physical_fun(
functions::create_physical_expr(
&(&scalar_function).into(),
&args,
input_schema,
&execution_props,
)?;

Arc::new(ScalarFunctionExpr::new(
&e.name,
fun_expr,
args,
convert_required!(e.return_type)?,
None,
))
)?
}
ExprType::ScalarUdf(e) => {
let scalar_fun = registry.udf(e.name.as_str())?.fun().clone();
let udf = registry.udf(e.name.as_str())?;
let signature = udf.signature();
let scalar_fun = udf.fun().clone();

let args = e
.args
Expand All @@ -368,6 +364,7 @@ pub fn parse_physical_expr(
args,
convert_required!(e.return_type)?,
None,
signature.clone(),
))
}
ExprType::LikeExpr(like_expr) => Arc::new(LikeExpr::new(
Expand Down
2 changes: 2 additions & 0 deletions datafusion/proto/tests/cases/roundtrip_physical_plan.rs
Original file line number Diff line number Diff line change
Expand Up @@ -552,6 +552,7 @@ fn roundtrip_builtin_scalar_function() -> Result<()> {
vec![col("a", &schema)?],
DataType::Int64,
None,
Signature::exact(vec![DataType::Int64], Volatility::Immutable),
);

let project =
Expand Down Expand Up @@ -589,6 +590,7 @@ fn roundtrip_scalar_udf() -> Result<()> {
vec![col("a", &schema)?],
DataType::Int64,
None,
Signature::exact(vec![DataType::Int64], Volatility::Immutable),
);

let project =
Expand Down

0 comments on commit dd09c03

Please sign in to comment.