Skip to content

Commit 784f0bb

Browse files
committed
modified into_scalar_udf method
1 parent be6412a commit 784f0bb

File tree

3 files changed

+7
-9
lines changed

3 files changed

+7
-9
lines changed

datafusion-examples/examples/async_udf.rs

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@ use datafusion::common::internal_err;
2424
use datafusion::common::types::{logical_int64, logical_string};
2525
use datafusion::common::utils::take_function_args;
2626
use datafusion::config::ConfigOptions;
27-
use datafusion::execution::{FunctionRegistry, SessionStateBuilder};
2827
use datafusion::logical_expr::async_udf::{
2928
AsyncScalarFunctionArgs, AsyncScalarUDF, AsyncScalarUDFImpl,
3029
};
@@ -40,15 +39,14 @@ use std::sync::Arc;
4039

4140
#[tokio::main]
4241
async fn main() -> Result<()> {
43-
let mut state = SessionStateBuilder::new().build();
42+
let ctx: SessionContext = SessionContext::new();
4443

4544
let async_upper = AsyncUpper::new();
4645
let udf = AsyncScalarUDF::new(Arc::new(async_upper));
47-
state.register_udf(udf.into_scalar_udf())?;
46+
ctx.register_udf(udf.into_scalar_udf());
4847
let async_equal = AsyncEqual::new();
4948
let udf = AsyncScalarUDF::new(Arc::new(async_equal));
50-
state.register_udf(udf.into_scalar_udf())?;
51-
let ctx = SessionContext::new_with_state(state);
49+
ctx.register_udf(udf.into_scalar_udf());
5250
ctx.register_batch("animal", animal()?)?;
5351

5452
// use Async UDF in the projection

datafusion/expr/src/async_udf.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -100,8 +100,8 @@ impl AsyncScalarUDF {
100100

101101
/// Turn this AsyncUDF into a ScalarUDF, suitable for
102102
/// registering in the context
103-
pub fn into_scalar_udf(self) -> Arc<ScalarUDF> {
104-
Arc::new(ScalarUDF::new_from_impl(self))
103+
pub fn into_scalar_udf(self) -> ScalarUDF {
104+
ScalarUDF::new_from_impl(self)
105105
}
106106

107107
/// Invoke the function asynchronously with the async arguments

datafusion/physical-plan/src/async_func.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -116,11 +116,11 @@ impl DisplayAs for AsyncFuncExec {
116116
let exprs = expr.join(", ");
117117
match t {
118118
DisplayFormatType::Default | DisplayFormatType::Verbose => {
119-
write!(f, "AsyncFuncExec: async_expr=[{}]", exprs)
119+
write!(f, "AsyncFuncExec: async_expr=[{exprs}]")
120120
}
121121
DisplayFormatType::TreeRender => {
122122
writeln!(f, "format=async_expr")?;
123-
writeln!(f, "async_expr={}", exprs)?;
123+
writeln!(f, "async_expr={exprs}")?;
124124
Ok(())
125125
}
126126
}

0 commit comments

Comments
 (0)