diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index d5b372c368fc1..db435e399bd59 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -42,6 +42,7 @@ use crate::{ hash_build_probe_order::HashBuildProbeOrder, optimizer::PhysicalOptimizerRule, }, }; +pub use datafusion_physical_expr::execution_props::ExecutionProps; use parking_lot::RwLock; use std::string::String; use std::sync::Arc; @@ -1128,68 +1129,6 @@ impl SessionConfig { } } -/// Holds per-execution properties and data (such as starting timestamps, etc). -/// An instance of this struct is created each time a [`LogicalPlan`] is prepared for -/// execution (optimized). If the same plan is optimized multiple times, a new -/// `ExecutionProps` is created each time. -/// -/// It is important that this structure be cheap to create as it is -/// done so during predicate pruning and expression simplification -#[derive(Clone)] -pub struct ExecutionProps { - pub(crate) query_execution_start_time: DateTime, - /// providers for scalar variables - pub var_providers: Option>>, -} - -impl Default for ExecutionProps { - fn default() -> Self { - Self::new() - } -} - -impl ExecutionProps { - /// Creates a new execution props - pub fn new() -> Self { - ExecutionProps { - query_execution_start_time: chrono::Utc::now(), - var_providers: None, - } - } - - /// Marks the execution of query started timestamp - pub fn start_execution(&mut self) -> &Self { - self.query_execution_start_time = chrono::Utc::now(); - &*self - } - - /// Registers a variable provider, returning the existing - /// provider, if any - pub fn add_var_provider( - &mut self, - var_type: VarType, - provider: Arc, - ) -> Option> { - let mut var_providers = self.var_providers.take().unwrap_or_default(); - - let old_provider = var_providers.insert(var_type, provider); - - self.var_providers = Some(var_providers); - - old_provider - } - - /// Returns the provider for the var_type, if any - pub fn get_var_provider( - &self, - var_type: VarType, - ) -> Option> { - self.var_providers - .as_ref() - .and_then(|var_providers| var_providers.get(&var_type).map(Arc::clone)) - } -} - /// Execution context for registering data sources and executing queries #[derive(Clone)] pub struct SessionState { @@ -1652,7 +1591,6 @@ impl FunctionRegistry for TaskContext { mod tests { use super::*; use crate::execution::context::QueryPlanner; - use crate::physical_plan::functions::make_scalar_function; use crate::test; use crate::test_util::parquet_test_data; use crate::variable::VarType; @@ -1666,6 +1604,7 @@ mod tests { use arrow::record_batch::RecordBatch; use async_trait::async_trait; use datafusion_expr::Volatility; + use datafusion_physical_expr::functions::make_scalar_function; use std::fs::File; use std::sync::Weak; use std::thread::{self, JoinHandle}; diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index e8852fdc142af..4b47222855d13 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -238,7 +238,7 @@ pub use datafusion_row as row; #[cfg(feature = "jit")] pub use datafusion_jit as jit; -pub mod from_slice; +pub use physical_expr::from_slice; #[cfg(test)] pub mod test; diff --git a/datafusion/core/src/optimizer/simplify_expressions.rs b/datafusion/core/src/optimizer/simplify_expressions.rs index b85377df33fdc..162da48941c45 100644 --- a/datafusion/core/src/optimizer/simplify_expressions.rs +++ b/datafusion/core/src/optimizer/simplify_expressions.rs @@ -20,7 +20,6 @@ use crate::execution::context::ExecutionProps; use crate::logical_plan::{ExprSimplifiable, SimplifyInfo}; use crate::optimizer::optimizer::{OptimizerConfig, OptimizerRule}; -use crate::physical_plan::planner::create_physical_expr; use arrow::array::new_null_array; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -33,6 +32,7 @@ use datafusion_expr::{ utils::from_plan, Expr, ExprSchemable, Operator, Volatility, }; +use datafusion_physical_expr::create_physical_expr; /// Provides simplification information based on schema and properties pub(crate) struct SimplifyContext<'a, 'b> { diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index aca53ead80345..b79e16090186f 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -31,15 +31,7 @@ use std::convert::TryFrom; use std::{collections::HashSet, sync::Arc}; -use arrow::{ - array::{new_null_array, ArrayRef, BooleanArray}, - datatypes::{DataType, Field, Schema, SchemaRef}, - record_batch::RecordBatch, -}; -use datafusion_expr::utils::expr_to_columns; - use crate::execution::context::ExecutionProps; -use crate::physical_plan::planner::create_physical_expr; use crate::prelude::lit; use crate::{ error::{DataFusionError, Result}, @@ -47,6 +39,13 @@ use crate::{ optimizer::utils, physical_plan::{ColumnarValue, PhysicalExpr}, }; +use arrow::{ + array::{new_null_array, ArrayRef, BooleanArray}, + datatypes::{DataType, Field, Schema, SchemaRef}, + record_batch::RecordBatch, +}; +use datafusion_expr::utils::expr_to_columns; +use datafusion_physical_expr::create_physical_expr; /// Interface to pass statistics information to [`PruningPredicate`] /// diff --git a/datafusion/core/src/physical_plan/functions.rs b/datafusion/core/src/physical_plan/functions.rs deleted file mode 100644 index 98dc9a16dce88..0000000000000 --- a/datafusion/core/src/physical_plan/functions.rs +++ /dev/null @@ -1,3172 +0,0 @@ -// 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. - -//! Declaration of built-in (scalar) functions. -//! This module contains built-in functions' enumeration and metadata. -//! -//! Generally, a function has: -//! * a signature -//! * a return type, that is a function of the incoming argument's types -//! * the computation, that must accept each valid signature -//! -//! * Signature: see `Signature` -//! * Return type: a function `(arg_types) -> return_type`. E.g. for sqrt, ([f32]) -> f32, ([f64]) -> f64. -//! -//! This module also has a set of coercion rules to improve user experience: if an argument i32 is passed -//! to a function that supports f64, it is coerced to f64. - -use super::{type_coercion::coerce, ColumnarValue, PhysicalExpr}; -use crate::execution::context::ExecutionProps; -use crate::physical_plan::expressions::{ - cast_column, nullif_func, DEFAULT_DATAFUSION_CAST_OPTIONS, -}; -use crate::{ - error::{DataFusionError, Result}, - logical_expr::{function, BuiltinScalarFunction, ScalarFunctionImplementation}, - scalar::ScalarValue, -}; -use arrow::{ - array::ArrayRef, - compute::kernels::length::{bit_length, length}, - datatypes::TimeUnit, - datatypes::{DataType, Int32Type, Int64Type, Schema}, -}; -use datafusion_physical_expr::array_expressions; -use datafusion_physical_expr::conditional_expressions; -use datafusion_physical_expr::datetime_expressions; -use datafusion_physical_expr::math_expressions; -use datafusion_physical_expr::string_expressions; -use datafusion_physical_expr::struct_expressions; -use std::sync::Arc; - -/// Create a physical (function) expression. -/// This function errors when `args`' can't be coerced to a valid argument type of the function. -pub fn create_physical_expr( - fun: &BuiltinScalarFunction, - input_phy_exprs: &[Arc], - input_schema: &Schema, - execution_props: &ExecutionProps, -) -> Result> { - let coerced_phy_exprs = - coerce(input_phy_exprs, input_schema, &function::signature(fun))?; - - let coerced_expr_types = coerced_phy_exprs - .iter() - .map(|e| e.data_type(input_schema)) - .collect::>>()?; - - let data_type = function::return_type(fun, &coerced_expr_types)?; - - let fun_expr: ScalarFunctionImplementation = match fun { - // These functions need args and input schema to pick an implementation - // Unlike the string functions, which actually figure out the function to use with each array, - // here we return either a cast fn or string timestamp translation based on the expression data type - // so we don't have to pay a per-array/batch cost. - BuiltinScalarFunction::ToTimestamp => { - Arc::new(match coerced_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Nanosecond, None), - &DEFAULT_DATAFUSION_CAST_OPTIONS, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp, - other => { - return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp", - other, - ))) - } - }) - } - BuiltinScalarFunction::ToTimestampMillis => { - Arc::new(match coerced_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Millisecond, None), - &DEFAULT_DATAFUSION_CAST_OPTIONS, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp_millis, - other => { - return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp_millis", - other, - ))) - } - }) - } - BuiltinScalarFunction::ToTimestampMicros => { - Arc::new(match coerced_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Microsecond, None), - &DEFAULT_DATAFUSION_CAST_OPTIONS, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp_micros, - other => { - return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp_micros", - other, - ))) - } - }) - } - BuiltinScalarFunction::ToTimestampSeconds => Arc::new({ - match coerced_phy_exprs[0].data_type(input_schema) { - Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { - |col_values: &[ColumnarValue]| { - cast_column( - &col_values[0], - &DataType::Timestamp(TimeUnit::Second, None), - &DEFAULT_DATAFUSION_CAST_OPTIONS, - ) - } - } - Ok(DataType::Utf8) => datetime_expressions::to_timestamp_seconds, - other => { - return Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_timestamp_seconds", - other, - ))) - } - } - }), - // These don't need args and input schema - _ => create_physical_fun(fun, execution_props)?, - }; - - Ok(Arc::new(ScalarFunctionExpr::new( - &format!("{}", fun), - fun_expr, - coerced_phy_exprs, - &data_type, - ))) -} - -pub use datafusion_physical_expr::ScalarFunctionExpr; - -#[cfg(feature = "crypto_expressions")] -macro_rules! invoke_if_crypto_expressions_feature_flag { - ($FUNC:ident, $NAME:expr) => {{ - use datafusion_physical_expr::crypto_expressions; - crypto_expressions::$FUNC - }}; -} - -#[cfg(not(feature = "crypto_expressions"))] -macro_rules! invoke_if_crypto_expressions_feature_flag { - ($FUNC:ident, $NAME:expr) => { - |_: &[ColumnarValue]| -> Result { - Err(DataFusionError::Internal(format!( - "function {} requires compilation with feature flag: crypto_expressions.", - $NAME - ))) - } - }; -} - -#[cfg(feature = "regex_expressions")] -macro_rules! invoke_if_regex_expressions_feature_flag { - ($FUNC:ident, $T:tt, $NAME:expr) => {{ - use datafusion_physical_expr::regex_expressions; - regex_expressions::$FUNC::<$T> - }}; -} - -#[cfg(not(feature = "regex_expressions"))] -macro_rules! invoke_if_regex_expressions_feature_flag { - ($FUNC:ident, $T:tt, $NAME:expr) => { - |_: &[ArrayRef]| -> Result { - Err(DataFusionError::Internal(format!( - "function {} requires compilation with feature flag: regex_expressions.", - $NAME - ))) - } - }; -} - -#[cfg(feature = "unicode_expressions")] -macro_rules! invoke_if_unicode_expressions_feature_flag { - ($FUNC:ident, $T:tt, $NAME:expr) => {{ - use datafusion_physical_expr::unicode_expressions; - unicode_expressions::$FUNC::<$T> - }}; -} - -#[cfg(not(feature = "unicode_expressions"))] -macro_rules! invoke_if_unicode_expressions_feature_flag { - ($FUNC:ident, $T:tt, $NAME:expr) => { - |_: &[ArrayRef]| -> Result { - Err(DataFusionError::Internal(format!( - "function {} requires compilation with feature flag: unicode_expressions.", - $NAME - ))) - } - }; -} - -/// decorates a function to handle [`ScalarValue`]s by converting them to arrays before calling the function -/// and vice-versa after evaluation. -pub fn make_scalar_function(inner: F) -> ScalarFunctionImplementation -where - F: Fn(&[ArrayRef]) -> Result + Sync + Send + 'static, -{ - Arc::new(move |args: &[ColumnarValue]| { - // first, identify if any of the arguments is an Array. If yes, store its `len`, - // as any scalar will need to be converted to an array of len `len`. - let len = args - .iter() - .fold(Option::::None, |acc, arg| match arg { - ColumnarValue::Scalar(_) => acc, - ColumnarValue::Array(a) => Some(a.len()), - }); - - // to array - let args = if let Some(len) = len { - args.iter() - .map(|arg| arg.clone().into_array(len)) - .collect::>() - } else { - args.iter() - .map(|arg| arg.clone().into_array(1)) - .collect::>() - }; - - let result = (inner)(&args); - - // maybe back to scalar - if len.is_some() { - result.map(ColumnarValue::Array) - } else { - ScalarValue::try_from_array(&result?, 0).map(ColumnarValue::Scalar) - } - }) -} - -/// Create a physical scalar function. -pub fn create_physical_fun( - fun: &BuiltinScalarFunction, - execution_props: &ExecutionProps, -) -> Result { - Ok(match fun { - // math functions - BuiltinScalarFunction::Abs => Arc::new(math_expressions::abs), - BuiltinScalarFunction::Acos => Arc::new(math_expressions::acos), - BuiltinScalarFunction::Asin => Arc::new(math_expressions::asin), - BuiltinScalarFunction::Atan => Arc::new(math_expressions::atan), - BuiltinScalarFunction::Ceil => Arc::new(math_expressions::ceil), - BuiltinScalarFunction::Cos => Arc::new(math_expressions::cos), - BuiltinScalarFunction::Exp => Arc::new(math_expressions::exp), - BuiltinScalarFunction::Floor => Arc::new(math_expressions::floor), - BuiltinScalarFunction::Log => Arc::new(math_expressions::log10), - BuiltinScalarFunction::Ln => Arc::new(math_expressions::ln), - BuiltinScalarFunction::Log10 => Arc::new(math_expressions::log10), - BuiltinScalarFunction::Log2 => Arc::new(math_expressions::log2), - BuiltinScalarFunction::Random => Arc::new(math_expressions::random), - BuiltinScalarFunction::Round => Arc::new(math_expressions::round), - BuiltinScalarFunction::Signum => Arc::new(math_expressions::signum), - BuiltinScalarFunction::Sin => Arc::new(math_expressions::sin), - BuiltinScalarFunction::Sqrt => Arc::new(math_expressions::sqrt), - BuiltinScalarFunction::Tan => Arc::new(math_expressions::tan), - BuiltinScalarFunction::Trunc => Arc::new(math_expressions::trunc), - BuiltinScalarFunction::Power => { - Arc::new(|args| make_scalar_function(math_expressions::power)(args)) - } - - // string functions - BuiltinScalarFunction::Array => Arc::new(array_expressions::array), - BuiltinScalarFunction::Struct => Arc::new(struct_expressions::struct_expr), - BuiltinScalarFunction::Ascii => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::ascii::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::ascii::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function ascii", - other, - ))), - }), - BuiltinScalarFunction::BitLength => Arc::new(|args| match &args[0] { - ColumnarValue::Array(v) => Ok(ColumnarValue::Array(bit_length(v.as_ref())?)), - ColumnarValue::Scalar(v) => match v { - ScalarValue::Utf8(v) => Ok(ColumnarValue::Scalar(ScalarValue::Int32( - v.as_ref().map(|x| (x.len() * 8) as i32), - ))), - ScalarValue::LargeUtf8(v) => Ok(ColumnarValue::Scalar( - ScalarValue::Int64(v.as_ref().map(|x| (x.len() * 8) as i64)), - )), - _ => unreachable!(), - }, - }), - BuiltinScalarFunction::Btrim => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::btrim::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::btrim::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function btrim", - other, - ))), - }), - BuiltinScalarFunction::CharacterLength => { - Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_unicode_expressions_feature_flag!( - character_length, - Int32Type, - "character_length" - ); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_unicode_expressions_feature_flag!( - character_length, - Int64Type, - "character_length" - ); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function character_length", - other, - ))), - }) - } - BuiltinScalarFunction::Chr => { - Arc::new(|args| make_scalar_function(string_expressions::chr)(args)) - } - BuiltinScalarFunction::Coalesce => Arc::new(conditional_expressions::coalesce), - BuiltinScalarFunction::Concat => Arc::new(string_expressions::concat), - BuiltinScalarFunction::ConcatWithSeparator => { - Arc::new(|args| make_scalar_function(string_expressions::concat_ws)(args)) - } - BuiltinScalarFunction::DatePart => Arc::new(datetime_expressions::date_part), - BuiltinScalarFunction::DateTrunc => Arc::new(datetime_expressions::date_trunc), - BuiltinScalarFunction::Now => { - // bind value for now at plan time - Arc::new(datetime_expressions::make_now( - execution_props.query_execution_start_time, - )) - } - BuiltinScalarFunction::InitCap => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::initcap::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::initcap::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function initcap", - other, - ))), - }), - BuiltinScalarFunction::Left => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_unicode_expressions_feature_flag!(left, i32, "left"); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_unicode_expressions_feature_flag!(left, i64, "left"); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function left", - other, - ))), - }), - BuiltinScalarFunction::Lower => Arc::new(string_expressions::lower), - BuiltinScalarFunction::Lpad => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_unicode_expressions_feature_flag!(lpad, i32, "lpad"); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_unicode_expressions_feature_flag!(lpad, i64, "lpad"); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function lpad", - other, - ))), - }), - BuiltinScalarFunction::Ltrim => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::ltrim::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::ltrim::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function ltrim", - other, - ))), - }), - BuiltinScalarFunction::MD5 => { - Arc::new(invoke_if_crypto_expressions_feature_flag!(md5, "md5")) - } - BuiltinScalarFunction::Digest => { - Arc::new(invoke_if_crypto_expressions_feature_flag!(digest, "digest")) - } - BuiltinScalarFunction::NullIf => Arc::new(nullif_func), - BuiltinScalarFunction::OctetLength => Arc::new(|args| match &args[0] { - ColumnarValue::Array(v) => Ok(ColumnarValue::Array(length(v.as_ref())?)), - ColumnarValue::Scalar(v) => match v { - ScalarValue::Utf8(v) => Ok(ColumnarValue::Scalar(ScalarValue::Int32( - v.as_ref().map(|x| x.len() as i32), - ))), - ScalarValue::LargeUtf8(v) => Ok(ColumnarValue::Scalar( - ScalarValue::Int64(v.as_ref().map(|x| x.len() as i64)), - )), - _ => unreachable!(), - }, - }), - BuiltinScalarFunction::RegexpMatch => { - Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_regex_expressions_feature_flag!( - regexp_match, - i32, - "regexp_match" - ); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_regex_expressions_feature_flag!( - regexp_match, - i64, - "regexp_match" - ); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function regexp_match", - other - ))), - }) - } - BuiltinScalarFunction::RegexpReplace => { - Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_regex_expressions_feature_flag!( - regexp_replace, - i32, - "regexp_replace" - ); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_regex_expressions_feature_flag!( - regexp_replace, - i64, - "regexp_replace" - ); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function regexp_replace", - other, - ))), - }) - } - BuiltinScalarFunction::Repeat => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::repeat::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::repeat::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function repeat", - other, - ))), - }), - BuiltinScalarFunction::Replace => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::replace::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::replace::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function replace", - other, - ))), - }), - BuiltinScalarFunction::Reverse => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = - invoke_if_unicode_expressions_feature_flag!(reverse, i32, "reverse"); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = - invoke_if_unicode_expressions_feature_flag!(reverse, i64, "reverse"); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function reverse", - other, - ))), - }), - BuiltinScalarFunction::Right => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = - invoke_if_unicode_expressions_feature_flag!(right, i32, "right"); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = - invoke_if_unicode_expressions_feature_flag!(right, i64, "right"); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function right", - other, - ))), - }), - BuiltinScalarFunction::Rpad => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_unicode_expressions_feature_flag!(rpad, i32, "rpad"); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_unicode_expressions_feature_flag!(rpad, i64, "rpad"); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function rpad", - other, - ))), - }), - BuiltinScalarFunction::Rtrim => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::rtrim::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::rtrim::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function rtrim", - other, - ))), - }), - BuiltinScalarFunction::SHA224 => { - Arc::new(invoke_if_crypto_expressions_feature_flag!(sha224, "sha224")) - } - BuiltinScalarFunction::SHA256 => { - Arc::new(invoke_if_crypto_expressions_feature_flag!(sha256, "sha256")) - } - BuiltinScalarFunction::SHA384 => { - Arc::new(invoke_if_crypto_expressions_feature_flag!(sha384, "sha384")) - } - BuiltinScalarFunction::SHA512 => { - Arc::new(invoke_if_crypto_expressions_feature_flag!(sha512, "sha512")) - } - BuiltinScalarFunction::SplitPart => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::split_part::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::split_part::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function split_part", - other, - ))), - }), - BuiltinScalarFunction::StartsWith => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::starts_with::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::starts_with::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function starts_with", - other, - ))), - }), - BuiltinScalarFunction::Strpos => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_unicode_expressions_feature_flag!( - strpos, Int32Type, "strpos" - ); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_unicode_expressions_feature_flag!( - strpos, Int64Type, "strpos" - ); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function strpos", - other, - ))), - }), - BuiltinScalarFunction::Substr => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = - invoke_if_unicode_expressions_feature_flag!(substr, i32, "substr"); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = - invoke_if_unicode_expressions_feature_flag!(substr, i64, "substr"); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function substr", - other, - ))), - }), - BuiltinScalarFunction::ToHex => Arc::new(|args| match args[0].data_type() { - DataType::Int32 => { - make_scalar_function(string_expressions::to_hex::)(args) - } - DataType::Int64 => { - make_scalar_function(string_expressions::to_hex::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function to_hex", - other, - ))), - }), - BuiltinScalarFunction::Translate => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - let func = invoke_if_unicode_expressions_feature_flag!( - translate, - i32, - "translate" - ); - make_scalar_function(func)(args) - } - DataType::LargeUtf8 => { - let func = invoke_if_unicode_expressions_feature_flag!( - translate, - i64, - "translate" - ); - make_scalar_function(func)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function translate", - other, - ))), - }), - BuiltinScalarFunction::Trim => Arc::new(|args| match args[0].data_type() { - DataType::Utf8 => { - make_scalar_function(string_expressions::btrim::)(args) - } - DataType::LargeUtf8 => { - make_scalar_function(string_expressions::btrim::)(args) - } - other => Err(DataFusionError::Internal(format!( - "Unsupported data type {:?} for function trim", - other, - ))), - }), - BuiltinScalarFunction::Upper => Arc::new(string_expressions::upper), - _ => { - return Err(DataFusionError::Internal(format!( - "create_physical_fun: Unsupported scalar function {:?}", - fun - ))) - } - }) -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::from_slice::FromSlice; - use crate::{ - error::Result, - physical_plan::expressions::{col, lit}, - scalar::ScalarValue, - }; - use arrow::{ - array::{ - Array, ArrayRef, BinaryArray, BooleanArray, FixedSizeListArray, Float32Array, - Float64Array, Int32Array, StringArray, UInt32Array, UInt64Array, - }, - datatypes::Field, - record_batch::RecordBatch, - }; - - /// $FUNC function to test - /// $ARGS arguments (vec) to pass to function - /// $EXPECTED a Result> where Result allows testing errors and Option allows testing Null - /// $EXPECTED_TYPE is the expected value type - /// $DATA_TYPE is the function to test result type - /// $ARRAY_TYPE is the column type after function applied - macro_rules! test_function { - ($FUNC:ident, $ARGS:expr, $EXPECTED:expr, $EXPECTED_TYPE:ty, $DATA_TYPE: ident, $ARRAY_TYPE:ident) => { - // used to provide type annotation - let expected: Result> = $EXPECTED; - let execution_props = ExecutionProps::new(); - - // any type works here: we evaluate against a literal of `value` - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - let columns: Vec = vec![Arc::new(Int32Array::from_slice(&[1]))]; - - let expr = - create_physical_expr(&BuiltinScalarFunction::$FUNC, $ARGS, &schema, &execution_props)?; - - // type is correct - assert_eq!(expr.data_type(&schema)?, DataType::$DATA_TYPE); - - let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; - - match expected { - Ok(expected) => { - let result = expr.evaluate(&batch)?; - let result = result.into_array(batch.num_rows()); - let result = result.as_any().downcast_ref::<$ARRAY_TYPE>().unwrap(); - - // value is correct - match expected { - Some(v) => assert_eq!(result.value(0), v), - None => assert!(result.is_null(0)), - }; - } - Err(expected_error) => { - // evaluate is expected error - cannot use .expect_err() due to Debug not being implemented - match expr.evaluate(&batch) { - Ok(_) => assert!(false, "expected error"), - Err(error) => { - assert_eq!(error.to_string(), expected_error.to_string()); - } - } - } - }; - }; - } - - #[test] - fn test_functions() -> Result<()> { - test_function!( - Ascii, - &[lit(ScalarValue::Utf8(Some("x".to_string())))], - Ok(Some(120)), - i32, - Int32, - Int32Array - ); - test_function!( - Ascii, - &[lit(ScalarValue::Utf8(Some("ésoj".to_string())))], - Ok(Some(233)), - i32, - Int32, - Int32Array - ); - test_function!( - Ascii, - &[lit(ScalarValue::Utf8(Some("💯".to_string())))], - Ok(Some(128175)), - i32, - Int32, - Int32Array - ); - test_function!( - Ascii, - &[lit(ScalarValue::Utf8(Some("💯a".to_string())))], - Ok(Some(128175)), - i32, - Int32, - Int32Array - ); - test_function!( - Ascii, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(0)), - i32, - Int32, - Int32Array - ); - test_function!( - Ascii, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - i32, - Int32, - Int32Array - ); - test_function!( - BitLength, - &[lit(ScalarValue::Utf8(Some("chars".to_string())))], - Ok(Some(40)), - i32, - Int32, - Int32Array - ); - test_function!( - BitLength, - &[lit(ScalarValue::Utf8(Some("josé".to_string())))], - Ok(Some(40)), - i32, - Int32, - Int32Array - ); - test_function!( - BitLength, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(0)), - i32, - Int32, - Int32Array - ); - test_function!( - Btrim, - &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Btrim, - &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Btrim, - &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Btrim, - &[lit(ScalarValue::Utf8(Some("\n trim \n".to_string())))], - Ok(Some("\n trim \n")), - &str, - Utf8, - StringArray - ); - test_function!( - Btrim, - &[ - lit(ScalarValue::Utf8(Some("xyxtrimyyx".to_string()))), - lit(ScalarValue::Utf8(Some("xyz".to_string()))), - ], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Btrim, - &[ - lit(ScalarValue::Utf8(Some("\nxyxtrimyyx\n".to_string()))), - lit(ScalarValue::Utf8(Some("xyz\n".to_string()))), - ], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Btrim, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("xyz".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - test_function!( - Btrim, - &[ - lit(ScalarValue::Utf8(Some("xyxtrimyyx".to_string()))), - lit(ScalarValue::Utf8(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - CharacterLength, - &[lit(ScalarValue::Utf8(Some("chars".to_string())))], - Ok(Some(5)), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - CharacterLength, - &[lit(ScalarValue::Utf8(Some("josé".to_string())))], - Ok(Some(4)), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - CharacterLength, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(0)), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - CharacterLength, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - i32, - Int32, - Int32Array - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - CharacterLength, - &[lit(ScalarValue::Utf8(Some("josé".to_string())))], - Err(DataFusionError::Internal( - "function character_length requires compilation with feature flag: unicode_expressions.".to_string() - )), - i32, - Int32, - Int32Array - ); - test_function!( - Chr, - &[lit(ScalarValue::Int64(Some(128175)))], - Ok(Some("💯")), - &str, - Utf8, - StringArray - ); - test_function!( - Chr, - &[lit(ScalarValue::Int64(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - test_function!( - Chr, - &[lit(ScalarValue::Int64(Some(120)))], - Ok(Some("x")), - &str, - Utf8, - StringArray - ); - test_function!( - Chr, - &[lit(ScalarValue::Int64(Some(128175)))], - Ok(Some("💯")), - &str, - Utf8, - StringArray - ); - test_function!( - Chr, - &[lit(ScalarValue::Int64(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - test_function!( - Chr, - &[lit(ScalarValue::Int64(Some(0)))], - Err(DataFusionError::Execution( - "null character not permitted.".to_string(), - )), - &str, - Utf8, - StringArray - ); - test_function!( - Chr, - &[lit(ScalarValue::Int64(Some(i64::MAX)))], - Err(DataFusionError::Execution( - "requested character too large for encoding.".to_string(), - )), - &str, - Utf8, - StringArray - ); - test_function!( - Concat, - &[ - lit(ScalarValue::Utf8(Some("aa".to_string()))), - lit(ScalarValue::Utf8(Some("bb".to_string()))), - lit(ScalarValue::Utf8(Some("cc".to_string()))), - ], - Ok(Some("aabbcc")), - &str, - Utf8, - StringArray - ); - test_function!( - Concat, - &[ - lit(ScalarValue::Utf8(Some("aa".to_string()))), - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("cc".to_string()))), - ], - Ok(Some("aacc")), - &str, - Utf8, - StringArray - ); - test_function!( - Concat, - &[lit(ScalarValue::Utf8(None))], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - test_function!( - ConcatWithSeparator, - &[ - lit(ScalarValue::Utf8(Some("|".to_string()))), - lit(ScalarValue::Utf8(Some("aa".to_string()))), - lit(ScalarValue::Utf8(Some("bb".to_string()))), - lit(ScalarValue::Utf8(Some("cc".to_string()))), - ], - Ok(Some("aa|bb|cc")), - &str, - Utf8, - StringArray - ); - test_function!( - ConcatWithSeparator, - &[ - lit(ScalarValue::Utf8(Some("|".to_string()))), - lit(ScalarValue::Utf8(None)), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - test_function!( - ConcatWithSeparator, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("aa".to_string()))), - lit(ScalarValue::Utf8(Some("bb".to_string()))), - lit(ScalarValue::Utf8(Some("cc".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - test_function!( - ConcatWithSeparator, - &[ - lit(ScalarValue::Utf8(Some("|".to_string()))), - lit(ScalarValue::Utf8(Some("aa".to_string()))), - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("cc".to_string()))), - ], - Ok(Some("aa|cc")), - &str, - Utf8, - StringArray - ); - test_function!( - Exp, - &[lit(ScalarValue::Int32(Some(1)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::UInt32(Some(1)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::UInt64(Some(1)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::Float64(Some(1.0)))], - Ok(Some((1.0_f64).exp())), - f64, - Float64, - Float64Array - ); - test_function!( - Exp, - &[lit(ScalarValue::Float32(Some(1.0)))], - Ok(Some((1.0_f32).exp())), - f32, - Float32, - Float32Array - ); - test_function!( - InitCap, - &[lit(ScalarValue::Utf8(Some("hi THOMAS".to_string())))], - Ok(Some("Hi Thomas")), - &str, - Utf8, - StringArray - ); - test_function!( - InitCap, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - test_function!( - InitCap, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - test_function!( - InitCap, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int8(Some(2))), - ], - Ok(Some("ab")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(200))), - ], - Ok(Some("abcde")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(-2))), - ], - Ok(Some("abc")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(-200))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(0))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Int64(Some(2))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some("joséé")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Int64(Some(-3))), - ], - Ok(Some("joséé")), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Left, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int8(Some(2))), - ], - Err(DataFusionError::Internal( - "function left requires compilation with feature flag: unicode_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some(" josé")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some(" hi")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(0))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(Some("xyxhi")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(21))), - lit(ScalarValue::Utf8(Some("abcdef".to_string()))), - ], - Ok(Some("abcdefabcdefabcdefahi")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some(" ".to_string()))), - ], - Ok(Some(" hi")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some("".to_string()))), - ], - Ok(Some("hi")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(None)), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(10))), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(Some("xyxyxyjosé")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(10))), - lit(ScalarValue::Utf8(Some("éñ".to_string()))), - ], - Ok(Some("éñéñéñjosé")), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Lpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Err(DataFusionError::Internal( - "function lpad requires compilation with feature flag: unicode_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - test_function!( - Ltrim, - &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Ltrim, - &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], - Ok(Some("trim ")), - &str, - Utf8, - StringArray - ); - test_function!( - Ltrim, - &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], - Ok(Some("trim ")), - &str, - Utf8, - StringArray - ); - test_function!( - Ltrim, - &[lit(ScalarValue::Utf8(Some("trim".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Ltrim, - &[lit(ScalarValue::Utf8(Some("\n trim ".to_string())))], - Ok(Some("\n trim ")), - &str, - Utf8, - StringArray - ); - test_function!( - Ltrim, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - MD5, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Ok(Some("34b7da764b21d298ef307d04d8152dc5")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - MD5, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some("d41d8cd98f00b204e9800998ecf8427e")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - MD5, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "crypto_expressions"))] - test_function!( - MD5, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Err(DataFusionError::Internal( - "function md5 requires compilation with feature flag: crypto_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - test_function!( - OctetLength, - &[lit(ScalarValue::Utf8(Some("chars".to_string())))], - Ok(Some(5)), - i32, - Int32, - Int32Array - ); - test_function!( - OctetLength, - &[lit(ScalarValue::Utf8(Some("josé".to_string())))], - Ok(Some(5)), - i32, - Int32, - Int32Array - ); - test_function!( - OctetLength, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(0)), - i32, - Int32, - Int32Array - ); - test_function!( - OctetLength, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("Thomas".to_string()))), - lit(ScalarValue::Utf8(Some(".[mN]a.".to_string()))), - lit(ScalarValue::Utf8(Some("M".to_string()))), - ], - Ok(Some("ThM")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), - lit(ScalarValue::Utf8(Some("b..".to_string()))), - lit(ScalarValue::Utf8(Some("X".to_string()))), - ], - Ok(Some("fooXbaz")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), - lit(ScalarValue::Utf8(Some("b..".to_string()))), - lit(ScalarValue::Utf8(Some("X".to_string()))), - lit(ScalarValue::Utf8(Some("g".to_string()))), - ], - Ok(Some("fooXX")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), - lit(ScalarValue::Utf8(Some("b(..)".to_string()))), - lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), - lit(ScalarValue::Utf8(Some("g".to_string()))), - ], - Ok(Some("fooXarYXazY")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("b(..)".to_string()))), - lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), - lit(ScalarValue::Utf8(Some("g".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), - lit(ScalarValue::Utf8(Some("g".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), - lit(ScalarValue::Utf8(Some("b(..)".to_string()))), - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("g".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), - lit(ScalarValue::Utf8(Some("b(..)".to_string()))), - lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), - lit(ScalarValue::Utf8(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("ABCabcABC".to_string()))), - lit(ScalarValue::Utf8(Some("(abc)".to_string()))), - lit(ScalarValue::Utf8(Some("X".to_string()))), - lit(ScalarValue::Utf8(Some("gi".to_string()))), - ], - Ok(Some("XXX")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "regex_expressions")] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("ABCabcABC".to_string()))), - lit(ScalarValue::Utf8(Some("(abc)".to_string()))), - lit(ScalarValue::Utf8(Some("X".to_string()))), - lit(ScalarValue::Utf8(Some("i".to_string()))), - ], - Ok(Some("XabcABC")), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "regex_expressions"))] - test_function!( - RegexpReplace, - &[ - lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), - lit(ScalarValue::Utf8(Some("b..".to_string()))), - lit(ScalarValue::Utf8(Some("X".to_string()))), - ], - Err(DataFusionError::Internal( - "function regexp_replace requires compilation with feature flag: regex_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - test_function!( - Repeat, - &[ - lit(ScalarValue::Utf8(Some("Pg".to_string()))), - lit(ScalarValue::Int64(Some(4))), - ], - Ok(Some("PgPgPgPg")), - &str, - Utf8, - StringArray - ); - test_function!( - Repeat, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Int64(Some(4))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - test_function!( - Repeat, - &[ - lit(ScalarValue::Utf8(Some("Pg".to_string()))), - lit(ScalarValue::Int64(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Reverse, - &[lit(ScalarValue::Utf8(Some("abcde".to_string())))], - Ok(Some("edcba")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Reverse, - &[lit(ScalarValue::Utf8(Some("loẅks".to_string())))], - Ok(Some("skẅol")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Reverse, - &[lit(ScalarValue::Utf8(Some("loẅks".to_string())))], - Ok(Some("skẅol")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Reverse, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Reverse, - &[lit(ScalarValue::Utf8(Some("abcde".to_string())))], - Err(DataFusionError::Internal( - "function reverse requires compilation with feature flag: unicode_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int8(Some(2))), - ], - Ok(Some("de")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(200))), - ], - Ok(Some("abcde")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(-2))), - ], - Ok(Some("cde")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(-200))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(Some(0))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Int64(Some(2))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int64(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some("éésoj")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Int64(Some(-3))), - ], - Ok(Some("éésoj")), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Right, - &[ - lit(ScalarValue::Utf8(Some("abcde".to_string()))), - lit(ScalarValue::Int8(Some(2))), - ], - Err(DataFusionError::Internal( - "function right requires compilation with feature flag: unicode_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some("josé ")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some("hi ")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(0))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(Some("hixyx")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(21))), - lit(ScalarValue::Utf8(Some("abcdef".to_string()))), - ], - Ok(Some("hiabcdefabcdefabcdefa")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some(" ".to_string()))), - ], - Ok(Some("hi ")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some("".to_string()))), - ], - Ok(Some("hi")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(None)), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("hi".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Utf8(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(10))), - lit(ScalarValue::Utf8(Some("xy".to_string()))), - ], - Ok(Some("joséxyxyxy")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(10))), - lit(ScalarValue::Utf8(Some("éñ".to_string()))), - ], - Ok(Some("josééñéñéñ")), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Rpad, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Err(DataFusionError::Internal( - "function rpad requires compilation with feature flag: unicode_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - test_function!( - Rtrim, - &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Rtrim, - &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], - Ok(Some(" trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Rtrim, - &[lit(ScalarValue::Utf8(Some(" trim \n".to_string())))], - Ok(Some(" trim \n")), - &str, - Utf8, - StringArray - ); - test_function!( - Rtrim, - &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], - Ok(Some(" trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Rtrim, - &[lit(ScalarValue::Utf8(Some("trim".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Rtrim, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA224, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Ok(Some(&[ - 11u8, 246u8, 203u8, 98u8, 100u8, 156u8, 66u8, 169u8, 174u8, 56u8, 118u8, - 171u8, 111u8, 109u8, 146u8, 173u8, 54u8, 203u8, 84u8, 20u8, 228u8, 149u8, - 248u8, 135u8, 50u8, 146u8, 190u8, 77u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA224, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(&[ - 209u8, 74u8, 2u8, 140u8, 42u8, 58u8, 43u8, 201u8, 71u8, 97u8, 2u8, 187u8, - 40u8, 130u8, 52u8, 196u8, 21u8, 162u8, 176u8, 31u8, 130u8, 142u8, 166u8, - 42u8, 197u8, 179u8, 228u8, 47u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA224, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &[u8], - Binary, - BinaryArray - ); - #[cfg(not(feature = "crypto_expressions"))] - test_function!( - SHA224, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Err(DataFusionError::Internal( - "function sha224 requires compilation with feature flag: crypto_expressions.".to_string() - )), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA256, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Ok(Some(&[ - 225u8, 96u8, 143u8, 117u8, 197u8, 215u8, 129u8, 63u8, 61u8, 64u8, 49u8, - 203u8, 48u8, 191u8, 183u8, 134u8, 80u8, 125u8, 152u8, 19u8, 117u8, 56u8, - 255u8, 142u8, 18u8, 138u8, 111u8, 247u8, 78u8, 132u8, 230u8, 67u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA256, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(&[ - 227u8, 176u8, 196u8, 66u8, 152u8, 252u8, 28u8, 20u8, 154u8, 251u8, 244u8, - 200u8, 153u8, 111u8, 185u8, 36u8, 39u8, 174u8, 65u8, 228u8, 100u8, 155u8, - 147u8, 76u8, 164u8, 149u8, 153u8, 27u8, 120u8, 82u8, 184u8, 85u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA256, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &[u8], - Binary, - BinaryArray - ); - #[cfg(not(feature = "crypto_expressions"))] - test_function!( - SHA256, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Err(DataFusionError::Internal( - "function sha256 requires compilation with feature flag: crypto_expressions.".to_string() - )), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA384, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Ok(Some(&[ - 9u8, 111u8, 91u8, 104u8, 170u8, 119u8, 132u8, 142u8, 79u8, 223u8, 92u8, - 28u8, 11u8, 53u8, 13u8, 226u8, 219u8, 250u8, 214u8, 15u8, 253u8, 124u8, - 37u8, 217u8, 234u8, 7u8, 198u8, 193u8, 155u8, 138u8, 77u8, 85u8, 169u8, - 24u8, 126u8, 177u8, 23u8, 197u8, 87u8, 136u8, 63u8, 88u8, 193u8, 109u8, - 250u8, 195u8, 227u8, 67u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA384, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(&[ - 56u8, 176u8, 96u8, 167u8, 81u8, 172u8, 150u8, 56u8, 76u8, 217u8, 50u8, - 126u8, 177u8, 177u8, 227u8, 106u8, 33u8, 253u8, 183u8, 17u8, 20u8, 190u8, - 7u8, 67u8, 76u8, 12u8, 199u8, 191u8, 99u8, 246u8, 225u8, 218u8, 39u8, - 78u8, 222u8, 191u8, 231u8, 111u8, 101u8, 251u8, 213u8, 26u8, 210u8, - 241u8, 72u8, 152u8, 185u8, 91u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA384, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &[u8], - Binary, - BinaryArray - ); - #[cfg(not(feature = "crypto_expressions"))] - test_function!( - SHA384, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Err(DataFusionError::Internal( - "function sha384 requires compilation with feature flag: crypto_expressions.".to_string() - )), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA512, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Ok(Some(&[ - 110u8, 27u8, 155u8, 63u8, 232u8, 64u8, 104u8, 14u8, 55u8, 5u8, 31u8, - 122u8, 213u8, 233u8, 89u8, 214u8, 243u8, 154u8, 208u8, 248u8, 136u8, - 93u8, 133u8, 81u8, 102u8, 245u8, 92u8, 101u8, 148u8, 105u8, 211u8, 200u8, - 183u8, 129u8, 24u8, 196u8, 74u8, 42u8, 73u8, 199u8, 45u8, 219u8, 72u8, - 28u8, 214u8, 216u8, 115u8, 16u8, 52u8, 225u8, 28u8, 192u8, 48u8, 7u8, - 11u8, 168u8, 67u8, 169u8, 11u8, 52u8, 149u8, 203u8, 141u8, 62u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA512, - &[lit(ScalarValue::Utf8(Some("".to_string())))], - Ok(Some(&[ - 207u8, 131u8, 225u8, 53u8, 126u8, 239u8, 184u8, 189u8, 241u8, 84u8, 40u8, - 80u8, 214u8, 109u8, 128u8, 7u8, 214u8, 32u8, 228u8, 5u8, 11u8, 87u8, - 21u8, 220u8, 131u8, 244u8, 169u8, 33u8, 211u8, 108u8, 233u8, 206u8, 71u8, - 208u8, 209u8, 60u8, 93u8, 133u8, 242u8, 176u8, 255u8, 131u8, 24u8, 210u8, - 135u8, 126u8, 236u8, 47u8, 99u8, 185u8, 49u8, 189u8, 71u8, 65u8, 122u8, - 129u8, 165u8, 56u8, 50u8, 122u8, 249u8, 39u8, 218u8, 62u8 - ])), - &[u8], - Binary, - BinaryArray - ); - #[cfg(feature = "crypto_expressions")] - test_function!( - SHA512, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &[u8], - Binary, - BinaryArray - ); - #[cfg(not(feature = "crypto_expressions"))] - test_function!( - SHA512, - &[lit(ScalarValue::Utf8(Some("tom".to_string())))], - Err(DataFusionError::Internal( - "function sha512 requires compilation with feature flag: crypto_expressions.".to_string() - )), - &[u8], - Binary, - BinaryArray - ); - test_function!( - SplitPart, - &[ - lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))), - lit(ScalarValue::Utf8(Some("~@~".to_string()))), - lit(ScalarValue::Int64(Some(2))), - ], - Ok(Some("def")), - &str, - Utf8, - StringArray - ); - test_function!( - SplitPart, - &[ - lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))), - lit(ScalarValue::Utf8(Some("~@~".to_string()))), - lit(ScalarValue::Int64(Some(20))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - test_function!( - SplitPart, - &[ - lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))), - lit(ScalarValue::Utf8(Some("~@~".to_string()))), - lit(ScalarValue::Int64(Some(-1))), - ], - Err(DataFusionError::Execution( - "field position must be greater than zero".to_string(), - )), - &str, - Utf8, - StringArray - ); - test_function!( - StartsWith, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Utf8(Some("alph".to_string()))), - ], - Ok(Some(true)), - bool, - Boolean, - BooleanArray - ); - test_function!( - StartsWith, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Utf8(Some("blph".to_string()))), - ], - Ok(Some(false)), - bool, - Boolean, - BooleanArray - ); - test_function!( - StartsWith, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("alph".to_string()))), - ], - Ok(None), - bool, - Boolean, - BooleanArray - ); - test_function!( - StartsWith, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Utf8(None)), - ], - Ok(None), - bool, - Boolean, - BooleanArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Strpos, - &[ - lit(ScalarValue::Utf8(Some("abc".to_string()))), - lit(ScalarValue::Utf8(Some("c".to_string()))), - ], - Ok(Some(3)), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Strpos, - &[ - lit(ScalarValue::Utf8(Some("josé".to_string()))), - lit(ScalarValue::Utf8(Some("é".to_string()))), - ], - Ok(Some(4)), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Strpos, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Utf8(Some("so".to_string()))), - ], - Ok(Some(6)), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Strpos, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Utf8(Some("abc".to_string()))), - ], - Ok(Some(0)), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Strpos, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("abc".to_string()))), - ], - Ok(None), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Strpos, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Utf8(None)), - ], - Ok(None), - i32, - Int32, - Int32Array - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Strpos, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Utf8(None)), - ], - Err(DataFusionError::Internal( - "function strpos requires compilation with feature flag: unicode_expressions.".to_string() - )), - i32, - Int32, - Int32Array - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(0))), - ], - Ok(Some("alphabet")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some("ésoj")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Int64(Some(-5))), - ], - Ok(Some("joséésoj")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(1))), - ], - Ok(Some("alphabet")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(2))), - ], - Ok(Some("lphabet")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(3))), - ], - Ok(Some("phabet")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(-3))), - ], - Ok(Some("alphabet")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(30))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(3))), - lit(ScalarValue::Int64(Some(2))), - ], - Ok(Some("ph")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(3))), - lit(ScalarValue::Int64(Some(20))), - ], - Ok(Some("phabet")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(0))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some("alph")), - &str, - Utf8, - StringArray - ); - // starting from 5 (10 + -5) - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(-5))), - lit(ScalarValue::Int64(Some(10))), - ], - Ok(Some("alph")), - &str, - Utf8, - StringArray - ); - // starting from -1 (4 + -5) - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(-5))), - lit(ScalarValue::Int64(Some(4))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - // starting from 0 (5 + -5) - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(-5))), - lit(ScalarValue::Int64(Some(5))), - ], - Ok(Some("")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(None)), - lit(ScalarValue::Int64(Some(20))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(3))), - lit(ScalarValue::Int64(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(1))), - lit(ScalarValue::Int64(Some(-1))), - ], - Err(DataFusionError::Execution( - "negative substring length not allowed: substr(, 1, -1)".to_string(), - )), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), - lit(ScalarValue::Int64(Some(5))), - lit(ScalarValue::Int64(Some(2))), - ], - Ok(Some("és")), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Substr, - &[ - lit(ScalarValue::Utf8(Some("alphabet".to_string()))), - lit(ScalarValue::Int64(Some(0))), - ], - Err(DataFusionError::Internal( - "function substr requires compilation with feature flag: unicode_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Translate, - &[ - lit(ScalarValue::Utf8(Some("12345".to_string()))), - lit(ScalarValue::Utf8(Some("143".to_string()))), - lit(ScalarValue::Utf8(Some("ax".to_string()))), - ], - Ok(Some("a2x5")), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Translate, - &[ - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("143".to_string()))), - lit(ScalarValue::Utf8(Some("ax".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Translate, - &[ - lit(ScalarValue::Utf8(Some("12345".to_string()))), - lit(ScalarValue::Utf8(None)), - lit(ScalarValue::Utf8(Some("ax".to_string()))), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Translate, - &[ - lit(ScalarValue::Utf8(Some("12345".to_string()))), - lit(ScalarValue::Utf8(Some("143".to_string()))), - lit(ScalarValue::Utf8(None)), - ], - Ok(None), - &str, - Utf8, - StringArray - ); - #[cfg(feature = "unicode_expressions")] - test_function!( - Translate, - &[ - lit(ScalarValue::Utf8(Some("é2íñ5".to_string()))), - lit(ScalarValue::Utf8(Some("éñí".to_string()))), - lit(ScalarValue::Utf8(Some("óü".to_string()))), - ], - Ok(Some("ó2ü5")), - &str, - Utf8, - StringArray - ); - #[cfg(not(feature = "unicode_expressions"))] - test_function!( - Translate, - &[ - lit(ScalarValue::Utf8(Some("12345".to_string()))), - lit(ScalarValue::Utf8(Some("143".to_string()))), - lit(ScalarValue::Utf8(Some("ax".to_string()))), - ], - Err(DataFusionError::Internal( - "function translate requires compilation with feature flag: unicode_expressions.".to_string() - )), - &str, - Utf8, - StringArray - ); - test_function!( - Trim, - &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Trim, - &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Trim, - &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], - Ok(Some("trim")), - &str, - Utf8, - StringArray - ); - test_function!( - Trim, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - test_function!( - Upper, - &[lit(ScalarValue::Utf8(Some("upper".to_string())))], - Ok(Some("UPPER")), - &str, - Utf8, - StringArray - ); - test_function!( - Upper, - &[lit(ScalarValue::Utf8(Some("UPPER".to_string())))], - Ok(Some("UPPER")), - &str, - Utf8, - StringArray - ); - test_function!( - Upper, - &[lit(ScalarValue::Utf8(None))], - Ok(None), - &str, - Utf8, - StringArray - ); - Ok(()) - } - - #[test] - fn test_empty_arguments_error() -> Result<()> { - let execution_props = ExecutionProps::new(); - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - - // pick some arbitrary functions to test - let funs = [ - BuiltinScalarFunction::Concat, - BuiltinScalarFunction::ToTimestamp, - BuiltinScalarFunction::Abs, - BuiltinScalarFunction::Repeat, - ]; - - for fun in funs.iter() { - let expr = create_physical_expr(fun, &[], &schema, &execution_props); - - match expr { - Ok(..) => { - return Err(DataFusionError::Plan(format!( - "Builtin scalar function {} does not support empty arguments", - fun - ))); - } - Err(DataFusionError::Internal(err)) => { - if err - != format!( - "Builtin scalar function {} does not support empty arguments", - fun - ) - { - return Err(DataFusionError::Internal(format!( - "Builtin scalar function {} didn't got the right error message with empty arguments", fun))); - } - } - Err(..) => { - return Err(DataFusionError::Internal(format!( - "Builtin scalar function {} didn't got the right error with empty arguments", fun))); - } - } - } - Ok(()) - } - - #[test] - fn test_empty_arguments() -> Result<()> { - let execution_props = ExecutionProps::new(); - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - - let funs = [BuiltinScalarFunction::Now, BuiltinScalarFunction::Random]; - - for fun in funs.iter() { - create_physical_expr(fun, &[], &schema, &execution_props)?; - } - Ok(()) - } - - fn generic_test_array( - value1: ArrayRef, - value2: ArrayRef, - expected_type: DataType, - expected: &str, - ) -> Result<()> { - // any type works here: we evaluate against a literal of `value` - let schema = Schema::new(vec![ - Field::new("a", value1.data_type().clone(), false), - Field::new("b", value2.data_type().clone(), false), - ]); - let columns: Vec = vec![value1, value2]; - let execution_props = ExecutionProps::new(); - - let expr = create_physical_expr( - &BuiltinScalarFunction::Array, - &[col("a", &schema)?, col("b", &schema)?], - &schema, - &execution_props, - )?; - - // type is correct - assert_eq!( - expr.data_type(&schema)?, - // type equals to a common coercion - DataType::FixedSizeList(Box::new(Field::new("item", expected_type, true)), 2) - ); - - // evaluate works - let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); - - // downcast works - let result = result - .as_any() - .downcast_ref::() - .unwrap(); - - // value is correct - assert_eq!(format!("{:?}", result.value(0)), expected); - - Ok(()) - } - - #[test] - fn test_array() -> Result<()> { - generic_test_array( - Arc::new(StringArray::from_slice(&["aa"])), - Arc::new(StringArray::from_slice(&["bb"])), - DataType::Utf8, - "StringArray\n[\n \"aa\",\n \"bb\",\n]", - )?; - - // different types, to validate that casting happens - generic_test_array( - Arc::new(UInt32Array::from_slice(&[1u32])), - Arc::new(UInt64Array::from_slice(&[1u64])), - DataType::UInt64, - "PrimitiveArray\n[\n 1,\n 1,\n]", - )?; - - // different types (another order), to validate that casting happens - generic_test_array( - Arc::new(UInt64Array::from_slice(&[1u64])), - Arc::new(UInt32Array::from_slice(&[1u32])), - DataType::UInt64, - "PrimitiveArray\n[\n 1,\n 1,\n]", - ) - } - - #[test] - #[cfg(feature = "regex_expressions")] - fn test_regexp_match() -> Result<()> { - use arrow::array::ListArray; - let schema = Schema::new(vec![Field::new("a", DataType::Utf8, false)]); - let execution_props = ExecutionProps::new(); - - let col_value: ArrayRef = Arc::new(StringArray::from_slice(&["aaa-555"])); - let pattern = lit(ScalarValue::Utf8(Some(r".*-(\d*)".to_string()))); - let columns: Vec = vec![col_value]; - let expr = create_physical_expr( - &BuiltinScalarFunction::RegexpMatch, - &[col("a", &schema)?, pattern], - &schema, - &execution_props, - )?; - - // type is correct - assert_eq!( - expr.data_type(&schema)?, - DataType::List(Box::new(Field::new("item", DataType::Utf8, true))) - ); - - // evaluate works - let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); - - // downcast works - let result = result.as_any().downcast_ref::().unwrap(); - let first_row = result.value(0); - let first_row = first_row.as_any().downcast_ref::().unwrap(); - - // value is correct - let expected = "555".to_string(); - assert_eq!(first_row.value(0), expected); - - Ok(()) - } - - #[test] - #[cfg(feature = "regex_expressions")] - fn test_regexp_match_all_literals() -> Result<()> { - use arrow::array::ListArray; - let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); - let execution_props = ExecutionProps::new(); - - let col_value = lit(ScalarValue::Utf8(Some("aaa-555".to_string()))); - let pattern = lit(ScalarValue::Utf8(Some(r".*-(\d*)".to_string()))); - let columns: Vec = vec![Arc::new(Int32Array::from_slice(&[1]))]; - let expr = create_physical_expr( - &BuiltinScalarFunction::RegexpMatch, - &[col_value, pattern], - &schema, - &execution_props, - )?; - - // type is correct - assert_eq!( - expr.data_type(&schema)?, - DataType::List(Box::new(Field::new("item", DataType::Utf8, true))) - ); - - // evaluate works - let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; - let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); - - // downcast works - let result = result.as_any().downcast_ref::().unwrap(); - let first_row = result.value(0); - let first_row = first_row.as_any().downcast_ref::().unwrap(); - - // value is correct - let expected = "555".to_string(); - assert_eq!(first_row.value(0), expected); - - Ok(()) - } -} diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index 2a89ea0df92d6..74e5ab4f9bb7b 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -552,12 +552,8 @@ pub mod cross_join; pub mod display; pub mod empty; pub mod explain; -use crate::execution::context::TaskContext; -pub use datafusion_physical_expr::expressions; - pub mod file_format; pub mod filter; -pub mod functions; pub mod hash_join; pub mod hash_utils; pub mod join_utils; @@ -570,9 +566,10 @@ pub mod repartition; pub mod sort_merge_join; pub mod sorts; pub mod stream; -pub mod type_coercion; pub mod udaf; -pub mod udf; pub mod union; pub mod values; pub mod windows; + +use crate::execution::context::TaskContext; +pub use datafusion_physical_expr::{expressions, functions, type_coercion, udf}; diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index ad957409c8264..39603d8f1f8e1 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -19,8 +19,8 @@ use super::analyze::AnalyzeExec; use super::{ - aggregates, empty::EmptyExec, expressions::binary, functions, - hash_join::PartitionMode, udaf, union::UnionExec, values::ValuesExec, windows, + aggregates, empty::EmptyExec, hash_join::PartitionMode, udaf, union::UnionExec, + values::ValuesExec, windows, }; use crate::execution::context::{ExecutionProps, SessionState}; use crate::logical_expr::utils::generate_sort_key; @@ -29,41 +29,35 @@ use crate::logical_plan::plan::{ SubqueryAlias, TableScan, Window, }; use crate::logical_plan::{ - unalias, unnormalize_cols, CrossJoin, DFSchema, Expr, LogicalPlan, Operator, + unalias, unnormalize_cols, CrossJoin, DFSchema, Expr, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, ToStringifiedPlan, Union, UserDefinedLogicalNode, }; use crate::logical_plan::{Limit, Values}; +use crate::physical_expr::create_physical_expr; use crate::physical_optimizer::optimizer::PhysicalOptimizerRule; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode}; use crate::physical_plan::cross_join::CrossJoinExec; use crate::physical_plan::explain::ExplainExec; -use crate::physical_plan::expressions; -use crate::physical_plan::expressions::{ - CaseExpr, Column, GetIndexedFieldExpr, Literal, PhysicalSortExpr, -}; +use crate::physical_plan::expressions::{Column, PhysicalSortExpr}; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::hash_join::HashJoinExec; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::udf; use crate::physical_plan::windows::WindowAggExec; use crate::physical_plan::{join_utils, Partitioning}; use crate::physical_plan::{AggregateExpr, ExecutionPlan, PhysicalExpr, WindowExpr}; -use crate::scalar::ScalarValue; -use crate::variable::VarType; use crate::{ error::{DataFusionError, Result}, physical_plan::displayable, }; use arrow::compute::SortOptions; +use arrow::datatypes::DataType; use arrow::datatypes::{Schema, SchemaRef}; -use arrow::{compute::can_cast_types, datatypes::DataType}; use async_trait::async_trait; use datafusion_expr::{expr::GroupingSet, utils::expr_to_columns}; -use datafusion_physical_expr::expressions::DateIntervalExpr; use datafusion_sql::utils::window_expr_common_partition_keys; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; @@ -1009,309 +1003,6 @@ impl DefaultPhysicalPlanner { } } -/// Create a physical expression from a logical expression ([Expr]) -pub fn create_physical_expr( - e: &Expr, - input_dfschema: &DFSchema, - input_schema: &Schema, - execution_props: &ExecutionProps, -) -> Result> { - match e { - Expr::Alias(expr, ..) => Ok(create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?), - Expr::Column(c) => { - let idx = input_dfschema.index_of_column(c)?; - Ok(Arc::new(Column::new(&c.name, idx))) - } - Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), - Expr::ScalarVariable(_, variable_names) => { - if &variable_names[0][0..2] == "@@" { - match execution_props.get_var_provider(VarType::System) { - Some(provider) => { - let scalar_value = provider.get_value(variable_names.clone())?; - Ok(Arc::new(Literal::new(scalar_value))) - } - _ => Err(DataFusionError::Plan( - "No system variable provider found".to_string(), - )), - } - } else { - match execution_props.get_var_provider(VarType::UserDefined) { - Some(provider) => { - let scalar_value = provider.get_value(variable_names.clone())?; - Ok(Arc::new(Literal::new(scalar_value))) - } - _ => Err(DataFusionError::Plan( - "No user defined variable provider found".to_string(), - )), - } - } - } - Expr::BinaryExpr { left, op, right } => { - let lhs = create_physical_expr( - left, - input_dfschema, - input_schema, - execution_props, - )?; - let rhs = create_physical_expr( - right, - input_dfschema, - input_schema, - execution_props, - )?; - match ( - lhs.data_type(input_schema)?, - op, - rhs.data_type(input_schema)?, - ) { - ( - DataType::Date32 | DataType::Date64, - Operator::Plus | Operator::Minus, - DataType::Interval(_), - ) => Ok(Arc::new(DateIntervalExpr::try_new( - lhs, - *op, - rhs, - input_schema, - )?)), - _ => { - // assume that we can coerce both sides into a common type - // and then perform a binary operation - binary(lhs, *op, rhs, input_schema) - } - } - } - Expr::Case { - expr, - when_then_expr, - else_expr, - .. - } => { - let expr: Option> = if let Some(e) = expr { - Some(create_physical_expr( - e.as_ref(), - input_dfschema, - input_schema, - execution_props, - )?) - } else { - None - }; - let when_expr = when_then_expr - .iter() - .map(|(w, _)| { - create_physical_expr( - w.as_ref(), - input_dfschema, - input_schema, - execution_props, - ) - }) - .collect::>>()?; - let then_expr = when_then_expr - .iter() - .map(|(_, t)| { - create_physical_expr( - t.as_ref(), - input_dfschema, - input_schema, - execution_props, - ) - }) - .collect::>>()?; - let when_then_expr: Vec<(Arc, Arc)> = - when_expr - .iter() - .zip(then_expr.iter()) - .map(|(w, t)| (w.clone(), t.clone())) - .collect(); - let else_expr: Option> = if let Some(e) = else_expr { - Some(create_physical_expr( - e.as_ref(), - input_dfschema, - input_schema, - execution_props, - )?) - } else { - None - }; - Ok(Arc::new(CaseExpr::try_new( - expr, - &when_then_expr, - else_expr, - )?)) - } - Expr::Cast { expr, data_type } => expressions::cast( - create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, - input_schema, - data_type.clone(), - ), - Expr::TryCast { expr, data_type } => expressions::try_cast( - create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, - input_schema, - data_type.clone(), - ), - Expr::Not(expr) => expressions::not(create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?), - Expr::Negative(expr) => expressions::negative( - create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, - input_schema, - ), - Expr::IsNull(expr) => expressions::is_null(create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?), - Expr::IsNotNull(expr) => expressions::is_not_null(create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?), - Expr::GetIndexedField { expr, key } => Ok(Arc::new(GetIndexedFieldExpr::new( - create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, - key.clone(), - ))), - - Expr::ScalarFunction { fun, args } => { - let physical_args = args - .iter() - .map(|e| { - create_physical_expr(e, input_dfschema, input_schema, execution_props) - }) - .collect::>>()?; - functions::create_physical_expr( - fun, - &physical_args, - input_schema, - execution_props, - ) - } - Expr::ScalarUDF { fun, args } => { - let mut physical_args = vec![]; - for e in args { - physical_args.push(create_physical_expr( - e, - input_dfschema, - input_schema, - execution_props, - )?); - } - - udf::create_physical_expr(fun.clone().as_ref(), &physical_args, input_schema) - } - Expr::Between { - expr, - negated, - low, - high, - } => { - let value_expr = create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?; - let low_expr = - create_physical_expr(low, input_dfschema, input_schema, execution_props)?; - let high_expr = create_physical_expr( - high, - input_dfschema, - input_schema, - execution_props, - )?; - - // rewrite the between into the two binary operators - let binary_expr = binary( - binary(value_expr.clone(), Operator::GtEq, low_expr, input_schema)?, - Operator::And, - binary(value_expr.clone(), Operator::LtEq, high_expr, input_schema)?, - input_schema, - ); - - if *negated { - expressions::not(binary_expr?) - } else { - binary_expr - } - } - Expr::InList { - expr, - list, - negated, - } => match expr.as_ref() { - Expr::Literal(ScalarValue::Utf8(None)) => { - Ok(expressions::lit(ScalarValue::Boolean(None))) - } - _ => { - let value_expr = create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?; - let value_expr_data_type = value_expr.data_type(input_schema)?; - - let list_exprs = list - .iter() - .map(|expr| match expr { - Expr::Literal(ScalarValue::Utf8(None)) => create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - ), - _ => { - let list_expr = create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?; - let list_expr_data_type = - list_expr.data_type(input_schema)?; - - if list_expr_data_type == value_expr_data_type { - Ok(list_expr) - } else if can_cast_types( - &list_expr_data_type, - &value_expr_data_type, - ) { - expressions::cast( - list_expr, - input_schema, - value_expr.data_type(input_schema)?, - ) - } else { - Err(DataFusionError::Plan(format!( - "Unsupported CAST from {:?} to {:?}", - list_expr_data_type, value_expr_data_type - ))) - } - } - }) - .collect::>>()?; - - expressions::in_list(value_expr, list_exprs, negated) - } - }, - other => Err(DataFusionError::NotImplemented(format!( - "Physical plan does not support logical expression {:?}", - other - ))), - } -} - /// Create a window expression with a name from a logical expression pub fn create_window_expr_with_name( e: &Expr, diff --git a/datafusion/core/src/variable/mod.rs b/datafusion/core/src/variable/mod.rs index 6ff326f26a02d..6efa8eb86211b 100644 --- a/datafusion/core/src/variable/mod.rs +++ b/datafusion/core/src/variable/mod.rs @@ -17,24 +17,4 @@ //! Variable provider -use crate::error::Result; -use crate::scalar::ScalarValue; -use arrow::datatypes::DataType; - -/// Variable type, system/user defined -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum VarType { - /// System variable, like @@version - System, - /// User defined variable, like @name - UserDefined, -} - -/// A var provider for @variable -pub trait VarProvider { - /// Get variable value - fn get_value(&self, var_names: Vec) -> Result; - - /// Return the type of the given variable - fn get_type(&self, var_names: &[String]) -> Option; -} +pub use datafusion_physical_expr::var_provider::{VarProvider, VarType}; diff --git a/datafusion/physical-expr/src/execution_props.rs b/datafusion/physical-expr/src/execution_props.rs new file mode 100644 index 0000000000000..2e68206668eaa --- /dev/null +++ b/datafusion/physical-expr/src/execution_props.rs @@ -0,0 +1,83 @@ +// 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 crate::var_provider::{VarProvider, VarType}; +use chrono::{DateTime, Utc}; +use std::collections::HashMap; +use std::sync::Arc; + +/// Holds per-execution properties and data (such as starting timestamps, etc). +/// An instance of this struct is created each time a [`LogicalPlan`] is prepared for +/// execution (optimized). If the same plan is optimized multiple times, a new +/// `ExecutionProps` is created each time. +/// +/// It is important that this structure be cheap to create as it is +/// done so during predicate pruning and expression simplification +#[derive(Clone)] +pub struct ExecutionProps { + pub query_execution_start_time: DateTime, + /// providers for scalar variables + pub var_providers: Option>>, +} + +impl Default for ExecutionProps { + fn default() -> Self { + Self::new() + } +} + +impl ExecutionProps { + /// Creates a new execution props + pub fn new() -> Self { + ExecutionProps { + query_execution_start_time: chrono::Utc::now(), + var_providers: None, + } + } + + /// Marks the execution of query started timestamp + pub fn start_execution(&mut self) -> &Self { + self.query_execution_start_time = chrono::Utc::now(); + &*self + } + + /// Registers a variable provider, returning the existing + /// provider, if any + pub fn add_var_provider( + &mut self, + var_type: VarType, + provider: Arc, + ) -> Option> { + let mut var_providers = self.var_providers.take().unwrap_or_default(); + + let old_provider = var_providers.insert(var_type, provider); + + self.var_providers = Some(var_providers); + + old_provider + } + + /// Returns the provider for the var_type, if any + pub fn get_var_provider( + &self, + var_type: VarType, + ) -> Option> { + self.var_providers + .as_ref() + .and_then(|var_providers| var_providers.get(&var_type).map(Arc::clone)) + } +} diff --git a/datafusion/core/src/from_slice.rs b/datafusion/physical-expr/src/from_slice.rs similarity index 100% rename from datafusion/core/src/from_slice.rs rename to datafusion/physical-expr/src/from_slice.rs diff --git a/datafusion/physical-expr/src/functions.rs b/datafusion/physical-expr/src/functions.rs index 1350d49510d58..7c55a60a854fb 100644 --- a/datafusion/physical-expr/src/functions.rs +++ b/datafusion/physical-expr/src/functions.rs @@ -29,120 +29,3135 @@ //! This module also has a set of coercion rules to improve user experience: if an argument i32 is passed //! to a function that supports f64, it is coerced to f64. -use crate::PhysicalExpr; -use arrow::datatypes::{DataType, Schema}; -use arrow::record_batch::RecordBatch; -use datafusion_common::Result; -use datafusion_expr::BuiltinScalarFunction; -use datafusion_expr::ColumnarValue; -pub use datafusion_expr::NullColumnarValue; -use datafusion_expr::ScalarFunctionImplementation; -use std::any::Any; -use std::fmt::Debug; -use std::fmt::{self, Formatter}; +use crate::execution_props::ExecutionProps; +use crate::{ + array_expressions, conditional_expressions, datetime_expressions, + expressions::{cast_column, nullif_func, DEFAULT_DATAFUSION_CAST_OPTIONS}, + math_expressions, string_expressions, struct_expressions, + type_coercion::coerce, + PhysicalExpr, ScalarFunctionExpr, +}; +use arrow::{ + array::ArrayRef, + compute::kernels::length::{bit_length, length}, + datatypes::TimeUnit, + datatypes::{DataType, Int32Type, Int64Type, Schema}, +}; +use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_expr::{ + function, BuiltinScalarFunction, ColumnarValue, ScalarFunctionImplementation, +}; use std::sync::Arc; -/// Physical expression of a scalar function -pub struct ScalarFunctionExpr { - fun: ScalarFunctionImplementation, - name: String, - args: Vec>, - return_type: DataType, +/// Create a physical (function) expression. +/// This function errors when `args`' can't be coerced to a valid argument type of the function. +pub fn create_physical_expr( + fun: &BuiltinScalarFunction, + input_phy_exprs: &[Arc], + input_schema: &Schema, + execution_props: &ExecutionProps, +) -> Result> { + let coerced_phy_exprs = + coerce(input_phy_exprs, input_schema, &function::signature(fun))?; + + let coerced_expr_types = coerced_phy_exprs + .iter() + .map(|e| e.data_type(input_schema)) + .collect::>>()?; + + let data_type = function::return_type(fun, &coerced_expr_types)?; + + let fun_expr: ScalarFunctionImplementation = match fun { + // These functions need args and input schema to pick an implementation + // Unlike the string functions, which actually figure out the function to use with each array, + // here we return either a cast fn or string timestamp translation based on the expression data type + // so we don't have to pay a per-array/batch cost. + BuiltinScalarFunction::ToTimestamp => { + Arc::new(match coerced_phy_exprs[0].data_type(input_schema) { + Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { + |col_values: &[ColumnarValue]| { + cast_column( + &col_values[0], + &DataType::Timestamp(TimeUnit::Nanosecond, None), + &DEFAULT_DATAFUSION_CAST_OPTIONS, + ) + } + } + Ok(DataType::Utf8) => datetime_expressions::to_timestamp, + other => { + return Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function to_timestamp", + other, + ))) + } + }) + } + BuiltinScalarFunction::ToTimestampMillis => { + Arc::new(match coerced_phy_exprs[0].data_type(input_schema) { + Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { + |col_values: &[ColumnarValue]| { + cast_column( + &col_values[0], + &DataType::Timestamp(TimeUnit::Millisecond, None), + &DEFAULT_DATAFUSION_CAST_OPTIONS, + ) + } + } + Ok(DataType::Utf8) => datetime_expressions::to_timestamp_millis, + other => { + return Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function to_timestamp_millis", + other, + ))) + } + }) + } + BuiltinScalarFunction::ToTimestampMicros => { + Arc::new(match coerced_phy_exprs[0].data_type(input_schema) { + Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { + |col_values: &[ColumnarValue]| { + cast_column( + &col_values[0], + &DataType::Timestamp(TimeUnit::Microsecond, None), + &DEFAULT_DATAFUSION_CAST_OPTIONS, + ) + } + } + Ok(DataType::Utf8) => datetime_expressions::to_timestamp_micros, + other => { + return Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function to_timestamp_micros", + other, + ))) + } + }) + } + BuiltinScalarFunction::ToTimestampSeconds => Arc::new({ + match coerced_phy_exprs[0].data_type(input_schema) { + Ok(DataType::Int64) | Ok(DataType::Timestamp(_, None)) => { + |col_values: &[ColumnarValue]| { + cast_column( + &col_values[0], + &DataType::Timestamp(TimeUnit::Second, None), + &DEFAULT_DATAFUSION_CAST_OPTIONS, + ) + } + } + Ok(DataType::Utf8) => datetime_expressions::to_timestamp_seconds, + other => { + return Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function to_timestamp_seconds", + other, + ))) + } + } + }), + // These don't need args and input schema + _ => create_physical_fun(fun, execution_props)?, + }; + + Ok(Arc::new(ScalarFunctionExpr::new( + &format!("{}", fun), + fun_expr, + coerced_phy_exprs, + &data_type, + ))) +} + +#[cfg(feature = "crypto_expressions")] +macro_rules! invoke_if_crypto_expressions_feature_flag { + ($FUNC:ident, $NAME:expr) => {{ + use crate::crypto_expressions; + crypto_expressions::$FUNC + }}; +} + +#[cfg(not(feature = "crypto_expressions"))] +macro_rules! invoke_if_crypto_expressions_feature_flag { + ($FUNC:ident, $NAME:expr) => { + |_: &[ColumnarValue]| -> Result { + Err(DataFusionError::Internal(format!( + "function {} requires compilation with feature flag: crypto_expressions.", + $NAME + ))) + } + }; +} + +#[cfg(feature = "regex_expressions")] +macro_rules! invoke_if_regex_expressions_feature_flag { + ($FUNC:ident, $T:tt, $NAME:expr) => {{ + use crate::regex_expressions; + regex_expressions::$FUNC::<$T> + }}; } -impl Debug for ScalarFunctionExpr { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - f.debug_struct("ScalarFunctionExpr") - .field("fun", &"") - .field("name", &self.name) - .field("args", &self.args) - .field("return_type", &self.return_type) - .finish() +#[cfg(not(feature = "regex_expressions"))] +macro_rules! invoke_if_regex_expressions_feature_flag { + ($FUNC:ident, $T:tt, $NAME:expr) => { + |_: &[ArrayRef]| -> Result { + Err(DataFusionError::Internal(format!( + "function {} requires compilation with feature flag: regex_expressions.", + $NAME + ))) + } + }; +} + +#[cfg(feature = "unicode_expressions")] +macro_rules! invoke_if_unicode_expressions_feature_flag { + ($FUNC:ident, $T:tt, $NAME:expr) => {{ + use crate::unicode_expressions; + unicode_expressions::$FUNC::<$T> + }}; +} + +#[cfg(not(feature = "unicode_expressions"))] +macro_rules! invoke_if_unicode_expressions_feature_flag { + ($FUNC:ident, $T:tt, $NAME:expr) => { + |_: &[ArrayRef]| -> Result { + Err(DataFusionError::Internal(format!( + "function {} requires compilation with feature flag: unicode_expressions.", + $NAME + ))) } + }; } -impl ScalarFunctionExpr { - /// Create a new Scalar function - pub fn new( - name: &str, - fun: ScalarFunctionImplementation, - args: Vec>, - return_type: &DataType, - ) -> Self { - Self { - fun, - name: name.to_owned(), - args, - return_type: return_type.clone(), +/// decorates a function to handle [`ScalarValue`]s by converting them to arrays before calling the function +/// and vice-versa after evaluation. +pub fn make_scalar_function(inner: F) -> ScalarFunctionImplementation +where + F: Fn(&[ArrayRef]) -> Result + Sync + Send + 'static, +{ + Arc::new(move |args: &[ColumnarValue]| { + // first, identify if any of the arguments is an Array. If yes, store its `len`, + // as any scalar will need to be converted to an array of len `len`. + let len = args + .iter() + .fold(Option::::None, |acc, arg| match arg { + ColumnarValue::Scalar(_) => acc, + ColumnarValue::Array(a) => Some(a.len()), + }); + + // to array + let args = if let Some(len) = len { + args.iter() + .map(|arg| arg.clone().into_array(len)) + .collect::>() + } else { + args.iter() + .map(|arg| arg.clone().into_array(1)) + .collect::>() + }; + + let result = (inner)(&args); + + // maybe back to scalar + if len.is_some() { + result.map(ColumnarValue::Array) + } else { + ScalarValue::try_from_array(&result?, 0).map(ColumnarValue::Scalar) + } + }) +} + +/// Create a physical scalar function. +pub fn create_physical_fun( + fun: &BuiltinScalarFunction, + execution_props: &ExecutionProps, +) -> Result { + Ok(match fun { + // math functions + BuiltinScalarFunction::Abs => Arc::new(math_expressions::abs), + BuiltinScalarFunction::Acos => Arc::new(math_expressions::acos), + BuiltinScalarFunction::Asin => Arc::new(math_expressions::asin), + BuiltinScalarFunction::Atan => Arc::new(math_expressions::atan), + BuiltinScalarFunction::Ceil => Arc::new(math_expressions::ceil), + BuiltinScalarFunction::Cos => Arc::new(math_expressions::cos), + BuiltinScalarFunction::Exp => Arc::new(math_expressions::exp), + BuiltinScalarFunction::Floor => Arc::new(math_expressions::floor), + BuiltinScalarFunction::Log => Arc::new(math_expressions::log10), + BuiltinScalarFunction::Ln => Arc::new(math_expressions::ln), + BuiltinScalarFunction::Log10 => Arc::new(math_expressions::log10), + BuiltinScalarFunction::Log2 => Arc::new(math_expressions::log2), + BuiltinScalarFunction::Random => Arc::new(math_expressions::random), + BuiltinScalarFunction::Round => Arc::new(math_expressions::round), + BuiltinScalarFunction::Signum => Arc::new(math_expressions::signum), + BuiltinScalarFunction::Sin => Arc::new(math_expressions::sin), + BuiltinScalarFunction::Sqrt => Arc::new(math_expressions::sqrt), + BuiltinScalarFunction::Tan => Arc::new(math_expressions::tan), + BuiltinScalarFunction::Trunc => Arc::new(math_expressions::trunc), + BuiltinScalarFunction::Power => { + Arc::new(|args| make_scalar_function(math_expressions::power)(args)) + } + + // string functions + BuiltinScalarFunction::Array => Arc::new(array_expressions::array), + BuiltinScalarFunction::Struct => Arc::new(struct_expressions::struct_expr), + BuiltinScalarFunction::Ascii => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::ascii::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::ascii::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function ascii", + other, + ))), + }), + BuiltinScalarFunction::BitLength => Arc::new(|args| match &args[0] { + ColumnarValue::Array(v) => Ok(ColumnarValue::Array(bit_length(v.as_ref())?)), + ColumnarValue::Scalar(v) => match v { + ScalarValue::Utf8(v) => Ok(ColumnarValue::Scalar(ScalarValue::Int32( + v.as_ref().map(|x| (x.len() * 8) as i32), + ))), + ScalarValue::LargeUtf8(v) => Ok(ColumnarValue::Scalar( + ScalarValue::Int64(v.as_ref().map(|x| (x.len() * 8) as i64)), + )), + _ => unreachable!(), + }, + }), + BuiltinScalarFunction::Btrim => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::btrim::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::btrim::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function btrim", + other, + ))), + }), + BuiltinScalarFunction::CharacterLength => { + Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + character_length, + Int32Type, + "character_length" + ); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + character_length, + Int64Type, + "character_length" + ); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function character_length", + other, + ))), + }) + } + BuiltinScalarFunction::Chr => { + Arc::new(|args| make_scalar_function(string_expressions::chr)(args)) + } + BuiltinScalarFunction::Coalesce => Arc::new(conditional_expressions::coalesce), + BuiltinScalarFunction::Concat => Arc::new(string_expressions::concat), + BuiltinScalarFunction::ConcatWithSeparator => { + Arc::new(|args| make_scalar_function(string_expressions::concat_ws)(args)) + } + BuiltinScalarFunction::DatePart => Arc::new(datetime_expressions::date_part), + BuiltinScalarFunction::DateTrunc => Arc::new(datetime_expressions::date_trunc), + BuiltinScalarFunction::Now => { + // bind value for now at plan time + Arc::new(datetime_expressions::make_now( + execution_props.query_execution_start_time, + )) + } + BuiltinScalarFunction::InitCap => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::initcap::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::initcap::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function initcap", + other, + ))), + }), + BuiltinScalarFunction::Left => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!(left, i32, "left"); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!(left, i64, "left"); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function left", + other, + ))), + }), + BuiltinScalarFunction::Lower => Arc::new(string_expressions::lower), + BuiltinScalarFunction::Lpad => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!(lpad, i32, "lpad"); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!(lpad, i64, "lpad"); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function lpad", + other, + ))), + }), + BuiltinScalarFunction::Ltrim => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::ltrim::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::ltrim::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function ltrim", + other, + ))), + }), + BuiltinScalarFunction::MD5 => { + Arc::new(invoke_if_crypto_expressions_feature_flag!(md5, "md5")) } + BuiltinScalarFunction::Digest => { + Arc::new(invoke_if_crypto_expressions_feature_flag!(digest, "digest")) + } + BuiltinScalarFunction::NullIf => Arc::new(nullif_func), + BuiltinScalarFunction::OctetLength => Arc::new(|args| match &args[0] { + ColumnarValue::Array(v) => Ok(ColumnarValue::Array(length(v.as_ref())?)), + ColumnarValue::Scalar(v) => match v { + ScalarValue::Utf8(v) => Ok(ColumnarValue::Scalar(ScalarValue::Int32( + v.as_ref().map(|x| x.len() as i32), + ))), + ScalarValue::LargeUtf8(v) => Ok(ColumnarValue::Scalar( + ScalarValue::Int64(v.as_ref().map(|x| x.len() as i64)), + )), + _ => unreachable!(), + }, + }), + BuiltinScalarFunction::RegexpMatch => { + Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_regex_expressions_feature_flag!( + regexp_match, + i32, + "regexp_match" + ); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_regex_expressions_feature_flag!( + regexp_match, + i64, + "regexp_match" + ); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function regexp_match", + other + ))), + }) + } + BuiltinScalarFunction::RegexpReplace => { + Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_regex_expressions_feature_flag!( + regexp_replace, + i32, + "regexp_replace" + ); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_regex_expressions_feature_flag!( + regexp_replace, + i64, + "regexp_replace" + ); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function regexp_replace", + other, + ))), + }) + } + BuiltinScalarFunction::Repeat => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::repeat::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::repeat::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function repeat", + other, + ))), + }), + BuiltinScalarFunction::Replace => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::replace::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::replace::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function replace", + other, + ))), + }), + BuiltinScalarFunction::Reverse => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = + invoke_if_unicode_expressions_feature_flag!(reverse, i32, "reverse"); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = + invoke_if_unicode_expressions_feature_flag!(reverse, i64, "reverse"); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function reverse", + other, + ))), + }), + BuiltinScalarFunction::Right => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = + invoke_if_unicode_expressions_feature_flag!(right, i32, "right"); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = + invoke_if_unicode_expressions_feature_flag!(right, i64, "right"); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function right", + other, + ))), + }), + BuiltinScalarFunction::Rpad => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!(rpad, i32, "rpad"); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!(rpad, i64, "rpad"); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function rpad", + other, + ))), + }), + BuiltinScalarFunction::Rtrim => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::rtrim::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::rtrim::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function rtrim", + other, + ))), + }), + BuiltinScalarFunction::SHA224 => { + Arc::new(invoke_if_crypto_expressions_feature_flag!(sha224, "sha224")) + } + BuiltinScalarFunction::SHA256 => { + Arc::new(invoke_if_crypto_expressions_feature_flag!(sha256, "sha256")) + } + BuiltinScalarFunction::SHA384 => { + Arc::new(invoke_if_crypto_expressions_feature_flag!(sha384, "sha384")) + } + BuiltinScalarFunction::SHA512 => { + Arc::new(invoke_if_crypto_expressions_feature_flag!(sha512, "sha512")) + } + BuiltinScalarFunction::SplitPart => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::split_part::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::split_part::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function split_part", + other, + ))), + }), + BuiltinScalarFunction::StartsWith => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::starts_with::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::starts_with::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function starts_with", + other, + ))), + }), + BuiltinScalarFunction::Strpos => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + strpos, Int32Type, "strpos" + ); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + strpos, Int64Type, "strpos" + ); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function strpos", + other, + ))), + }), + BuiltinScalarFunction::Substr => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = + invoke_if_unicode_expressions_feature_flag!(substr, i32, "substr"); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = + invoke_if_unicode_expressions_feature_flag!(substr, i64, "substr"); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function substr", + other, + ))), + }), + BuiltinScalarFunction::ToHex => Arc::new(|args| match args[0].data_type() { + DataType::Int32 => { + make_scalar_function(string_expressions::to_hex::)(args) + } + DataType::Int64 => { + make_scalar_function(string_expressions::to_hex::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function to_hex", + other, + ))), + }), + BuiltinScalarFunction::Translate => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + translate, + i32, + "translate" + ); + make_scalar_function(func)(args) + } + DataType::LargeUtf8 => { + let func = invoke_if_unicode_expressions_feature_flag!( + translate, + i64, + "translate" + ); + make_scalar_function(func)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function translate", + other, + ))), + }), + BuiltinScalarFunction::Trim => Arc::new(|args| match args[0].data_type() { + DataType::Utf8 => { + make_scalar_function(string_expressions::btrim::)(args) + } + DataType::LargeUtf8 => { + make_scalar_function(string_expressions::btrim::)(args) + } + other => Err(DataFusionError::Internal(format!( + "Unsupported data type {:?} for function trim", + other, + ))), + }), + BuiltinScalarFunction::Upper => Arc::new(string_expressions::upper), + _ => { + return Err(DataFusionError::Internal(format!( + "create_physical_fun: Unsupported scalar function {:?}", + fun + ))) + } + }) +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::expressions::{col, lit}; + use crate::from_slice::FromSlice; + use arrow::{ + array::{ + Array, ArrayRef, BinaryArray, BooleanArray, FixedSizeListArray, Float32Array, + Float64Array, Int32Array, StringArray, UInt32Array, UInt64Array, + }, + datatypes::Field, + record_batch::RecordBatch, + }; + use datafusion_common::{Result, ScalarValue}; + + /// $FUNC function to test + /// $ARGS arguments (vec) to pass to function + /// $EXPECTED a Result> where Result allows testing errors and Option allows testing Null + /// $EXPECTED_TYPE is the expected value type + /// $DATA_TYPE is the function to test result type + /// $ARRAY_TYPE is the column type after function applied + macro_rules! test_function { + ($FUNC:ident, $ARGS:expr, $EXPECTED:expr, $EXPECTED_TYPE:ty, $DATA_TYPE: ident, $ARRAY_TYPE:ident) => { + // used to provide type annotation + let expected: Result> = $EXPECTED; + let execution_props = ExecutionProps::new(); + + // any type works here: we evaluate against a literal of `value` + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let columns: Vec = vec![Arc::new(Int32Array::from_slice(&[1]))]; + + let expr = + create_physical_expr(&BuiltinScalarFunction::$FUNC, $ARGS, &schema, &execution_props)?; + + // type is correct + assert_eq!(expr.data_type(&schema)?, DataType::$DATA_TYPE); + + let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; + + match expected { + Ok(expected) => { + let result = expr.evaluate(&batch)?; + let result = result.into_array(batch.num_rows()); + let result = result.as_any().downcast_ref::<$ARRAY_TYPE>().unwrap(); + + // value is correct + match expected { + Some(v) => assert_eq!(result.value(0), v), + None => assert!(result.is_null(0)), + }; + } + Err(expected_error) => { + // evaluate is expected error - cannot use .expect_err() due to Debug not being implemented + match expr.evaluate(&batch) { + Ok(_) => assert!(false, "expected error"), + Err(error) => { + assert_eq!(error.to_string(), expected_error.to_string()); + } + } + } + }; + }; } - /// Get the scalar function implementation - pub fn fun(&self) -> &ScalarFunctionImplementation { - &self.fun + #[test] + fn test_functions() -> Result<()> { + test_function!( + Ascii, + &[lit(ScalarValue::Utf8(Some("x".to_string())))], + Ok(Some(120)), + i32, + Int32, + Int32Array + ); + test_function!( + Ascii, + &[lit(ScalarValue::Utf8(Some("ésoj".to_string())))], + Ok(Some(233)), + i32, + Int32, + Int32Array + ); + test_function!( + Ascii, + &[lit(ScalarValue::Utf8(Some("💯".to_string())))], + Ok(Some(128175)), + i32, + Int32, + Int32Array + ); + test_function!( + Ascii, + &[lit(ScalarValue::Utf8(Some("💯a".to_string())))], + Ok(Some(128175)), + i32, + Int32, + Int32Array + ); + test_function!( + Ascii, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(0)), + i32, + Int32, + Int32Array + ); + test_function!( + Ascii, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + i32, + Int32, + Int32Array + ); + test_function!( + BitLength, + &[lit(ScalarValue::Utf8(Some("chars".to_string())))], + Ok(Some(40)), + i32, + Int32, + Int32Array + ); + test_function!( + BitLength, + &[lit(ScalarValue::Utf8(Some("josé".to_string())))], + Ok(Some(40)), + i32, + Int32, + Int32Array + ); + test_function!( + BitLength, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(0)), + i32, + Int32, + Int32Array + ); + test_function!( + Btrim, + &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Btrim, + &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Btrim, + &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Btrim, + &[lit(ScalarValue::Utf8(Some("\n trim \n".to_string())))], + Ok(Some("\n trim \n")), + &str, + Utf8, + StringArray + ); + test_function!( + Btrim, + &[ + lit(ScalarValue::Utf8(Some("xyxtrimyyx".to_string()))), + lit(ScalarValue::Utf8(Some("xyz".to_string()))), + ], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Btrim, + &[ + lit(ScalarValue::Utf8(Some("\nxyxtrimyyx\n".to_string()))), + lit(ScalarValue::Utf8(Some("xyz\n".to_string()))), + ], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Btrim, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("xyz".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + test_function!( + Btrim, + &[ + lit(ScalarValue::Utf8(Some("xyxtrimyyx".to_string()))), + lit(ScalarValue::Utf8(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + CharacterLength, + &[lit(ScalarValue::Utf8(Some("chars".to_string())))], + Ok(Some(5)), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + CharacterLength, + &[lit(ScalarValue::Utf8(Some("josé".to_string())))], + Ok(Some(4)), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + CharacterLength, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(0)), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + CharacterLength, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + i32, + Int32, + Int32Array + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + CharacterLength, + &[lit(ScalarValue::Utf8(Some("josé".to_string())))], + Err(DataFusionError::Internal( + "function character_length requires compilation with feature flag: unicode_expressions.".to_string() + )), + i32, + Int32, + Int32Array + ); + test_function!( + Chr, + &[lit(ScalarValue::Int64(Some(128175)))], + Ok(Some("💯")), + &str, + Utf8, + StringArray + ); + test_function!( + Chr, + &[lit(ScalarValue::Int64(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + test_function!( + Chr, + &[lit(ScalarValue::Int64(Some(120)))], + Ok(Some("x")), + &str, + Utf8, + StringArray + ); + test_function!( + Chr, + &[lit(ScalarValue::Int64(Some(128175)))], + Ok(Some("💯")), + &str, + Utf8, + StringArray + ); + test_function!( + Chr, + &[lit(ScalarValue::Int64(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + test_function!( + Chr, + &[lit(ScalarValue::Int64(Some(0)))], + Err(DataFusionError::Execution( + "null character not permitted.".to_string(), + )), + &str, + Utf8, + StringArray + ); + test_function!( + Chr, + &[lit(ScalarValue::Int64(Some(i64::MAX)))], + Err(DataFusionError::Execution( + "requested character too large for encoding.".to_string(), + )), + &str, + Utf8, + StringArray + ); + test_function!( + Concat, + &[ + lit(ScalarValue::Utf8(Some("aa".to_string()))), + lit(ScalarValue::Utf8(Some("bb".to_string()))), + lit(ScalarValue::Utf8(Some("cc".to_string()))), + ], + Ok(Some("aabbcc")), + &str, + Utf8, + StringArray + ); + test_function!( + Concat, + &[ + lit(ScalarValue::Utf8(Some("aa".to_string()))), + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("cc".to_string()))), + ], + Ok(Some("aacc")), + &str, + Utf8, + StringArray + ); + test_function!( + Concat, + &[lit(ScalarValue::Utf8(None))], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + test_function!( + ConcatWithSeparator, + &[ + lit(ScalarValue::Utf8(Some("|".to_string()))), + lit(ScalarValue::Utf8(Some("aa".to_string()))), + lit(ScalarValue::Utf8(Some("bb".to_string()))), + lit(ScalarValue::Utf8(Some("cc".to_string()))), + ], + Ok(Some("aa|bb|cc")), + &str, + Utf8, + StringArray + ); + test_function!( + ConcatWithSeparator, + &[ + lit(ScalarValue::Utf8(Some("|".to_string()))), + lit(ScalarValue::Utf8(None)), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + test_function!( + ConcatWithSeparator, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("aa".to_string()))), + lit(ScalarValue::Utf8(Some("bb".to_string()))), + lit(ScalarValue::Utf8(Some("cc".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + test_function!( + ConcatWithSeparator, + &[ + lit(ScalarValue::Utf8(Some("|".to_string()))), + lit(ScalarValue::Utf8(Some("aa".to_string()))), + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("cc".to_string()))), + ], + Ok(Some("aa|cc")), + &str, + Utf8, + StringArray + ); + test_function!( + Exp, + &[lit(ScalarValue::Int32(Some(1)))], + Ok(Some((1.0_f64).exp())), + f64, + Float64, + Float64Array + ); + test_function!( + Exp, + &[lit(ScalarValue::UInt32(Some(1)))], + Ok(Some((1.0_f64).exp())), + f64, + Float64, + Float64Array + ); + test_function!( + Exp, + &[lit(ScalarValue::UInt64(Some(1)))], + Ok(Some((1.0_f64).exp())), + f64, + Float64, + Float64Array + ); + test_function!( + Exp, + &[lit(ScalarValue::Float64(Some(1.0)))], + Ok(Some((1.0_f64).exp())), + f64, + Float64, + Float64Array + ); + test_function!( + Exp, + &[lit(ScalarValue::Float32(Some(1.0)))], + Ok(Some((1.0_f32).exp())), + f32, + Float32, + Float32Array + ); + test_function!( + InitCap, + &[lit(ScalarValue::Utf8(Some("hi THOMAS".to_string())))], + Ok(Some("Hi Thomas")), + &str, + Utf8, + StringArray + ); + test_function!( + InitCap, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + test_function!( + InitCap, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + test_function!( + InitCap, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int8(Some(2))), + ], + Ok(Some("ab")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(200))), + ], + Ok(Some("abcde")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(-2))), + ], + Ok(Some("abc")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(-200))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(0))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Int64(Some(2))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some("joséé")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Int64(Some(-3))), + ], + Ok(Some("joséé")), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Left, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int8(Some(2))), + ], + Err(DataFusionError::Internal( + "function left requires compilation with feature flag: unicode_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some(" josé")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some(" hi")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(0))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(Some("xyxhi")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(21))), + lit(ScalarValue::Utf8(Some("abcdef".to_string()))), + ], + Ok(Some("abcdefabcdefabcdefahi")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some(" ".to_string()))), + ], + Ok(Some(" hi")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some("".to_string()))), + ], + Ok(Some("hi")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(None)), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(10))), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(Some("xyxyxyjosé")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(10))), + lit(ScalarValue::Utf8(Some("éñ".to_string()))), + ], + Ok(Some("éñéñéñjosé")), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Lpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Err(DataFusionError::Internal( + "function lpad requires compilation with feature flag: unicode_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + test_function!( + Ltrim, + &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Ltrim, + &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], + Ok(Some("trim ")), + &str, + Utf8, + StringArray + ); + test_function!( + Ltrim, + &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], + Ok(Some("trim ")), + &str, + Utf8, + StringArray + ); + test_function!( + Ltrim, + &[lit(ScalarValue::Utf8(Some("trim".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Ltrim, + &[lit(ScalarValue::Utf8(Some("\n trim ".to_string())))], + Ok(Some("\n trim ")), + &str, + Utf8, + StringArray + ); + test_function!( + Ltrim, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + MD5, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Ok(Some("34b7da764b21d298ef307d04d8152dc5")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + MD5, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some("d41d8cd98f00b204e9800998ecf8427e")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + MD5, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "crypto_expressions"))] + test_function!( + MD5, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Err(DataFusionError::Internal( + "function md5 requires compilation with feature flag: crypto_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + test_function!( + OctetLength, + &[lit(ScalarValue::Utf8(Some("chars".to_string())))], + Ok(Some(5)), + i32, + Int32, + Int32Array + ); + test_function!( + OctetLength, + &[lit(ScalarValue::Utf8(Some("josé".to_string())))], + Ok(Some(5)), + i32, + Int32, + Int32Array + ); + test_function!( + OctetLength, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(0)), + i32, + Int32, + Int32Array + ); + test_function!( + OctetLength, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("Thomas".to_string()))), + lit(ScalarValue::Utf8(Some(".[mN]a.".to_string()))), + lit(ScalarValue::Utf8(Some("M".to_string()))), + ], + Ok(Some("ThM")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), + lit(ScalarValue::Utf8(Some("b..".to_string()))), + lit(ScalarValue::Utf8(Some("X".to_string()))), + ], + Ok(Some("fooXbaz")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), + lit(ScalarValue::Utf8(Some("b..".to_string()))), + lit(ScalarValue::Utf8(Some("X".to_string()))), + lit(ScalarValue::Utf8(Some("g".to_string()))), + ], + Ok(Some("fooXX")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), + lit(ScalarValue::Utf8(Some("b(..)".to_string()))), + lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), + lit(ScalarValue::Utf8(Some("g".to_string()))), + ], + Ok(Some("fooXarYXazY")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("b(..)".to_string()))), + lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), + lit(ScalarValue::Utf8(Some("g".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), + lit(ScalarValue::Utf8(Some("g".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), + lit(ScalarValue::Utf8(Some("b(..)".to_string()))), + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("g".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), + lit(ScalarValue::Utf8(Some("b(..)".to_string()))), + lit(ScalarValue::Utf8(Some("X\\1Y".to_string()))), + lit(ScalarValue::Utf8(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("ABCabcABC".to_string()))), + lit(ScalarValue::Utf8(Some("(abc)".to_string()))), + lit(ScalarValue::Utf8(Some("X".to_string()))), + lit(ScalarValue::Utf8(Some("gi".to_string()))), + ], + Ok(Some("XXX")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "regex_expressions")] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("ABCabcABC".to_string()))), + lit(ScalarValue::Utf8(Some("(abc)".to_string()))), + lit(ScalarValue::Utf8(Some("X".to_string()))), + lit(ScalarValue::Utf8(Some("i".to_string()))), + ], + Ok(Some("XabcABC")), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "regex_expressions"))] + test_function!( + RegexpReplace, + &[ + lit(ScalarValue::Utf8(Some("foobarbaz".to_string()))), + lit(ScalarValue::Utf8(Some("b..".to_string()))), + lit(ScalarValue::Utf8(Some("X".to_string()))), + ], + Err(DataFusionError::Internal( + "function regexp_replace requires compilation with feature flag: regex_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + test_function!( + Repeat, + &[ + lit(ScalarValue::Utf8(Some("Pg".to_string()))), + lit(ScalarValue::Int64(Some(4))), + ], + Ok(Some("PgPgPgPg")), + &str, + Utf8, + StringArray + ); + test_function!( + Repeat, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Int64(Some(4))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + test_function!( + Repeat, + &[ + lit(ScalarValue::Utf8(Some("Pg".to_string()))), + lit(ScalarValue::Int64(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Reverse, + &[lit(ScalarValue::Utf8(Some("abcde".to_string())))], + Ok(Some("edcba")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Reverse, + &[lit(ScalarValue::Utf8(Some("loẅks".to_string())))], + Ok(Some("skẅol")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Reverse, + &[lit(ScalarValue::Utf8(Some("loẅks".to_string())))], + Ok(Some("skẅol")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Reverse, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Reverse, + &[lit(ScalarValue::Utf8(Some("abcde".to_string())))], + Err(DataFusionError::Internal( + "function reverse requires compilation with feature flag: unicode_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int8(Some(2))), + ], + Ok(Some("de")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(200))), + ], + Ok(Some("abcde")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(-2))), + ], + Ok(Some("cde")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(-200))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(Some(0))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Int64(Some(2))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int64(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some("éésoj")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Int64(Some(-3))), + ], + Ok(Some("éésoj")), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Right, + &[ + lit(ScalarValue::Utf8(Some("abcde".to_string()))), + lit(ScalarValue::Int8(Some(2))), + ], + Err(DataFusionError::Internal( + "function right requires compilation with feature flag: unicode_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some("josé ")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some("hi ")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(0))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(Some("hixyx")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(21))), + lit(ScalarValue::Utf8(Some("abcdef".to_string()))), + ], + Ok(Some("hiabcdefabcdefabcdefa")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some(" ".to_string()))), + ], + Ok(Some("hi ")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some("".to_string()))), + ], + Ok(Some("hi")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(None)), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("hi".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Utf8(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(10))), + lit(ScalarValue::Utf8(Some("xy".to_string()))), + ], + Ok(Some("joséxyxyxy")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(10))), + lit(ScalarValue::Utf8(Some("éñ".to_string()))), + ], + Ok(Some("josééñéñéñ")), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Rpad, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Err(DataFusionError::Internal( + "function rpad requires compilation with feature flag: unicode_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + test_function!( + Rtrim, + &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Rtrim, + &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], + Ok(Some(" trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Rtrim, + &[lit(ScalarValue::Utf8(Some(" trim \n".to_string())))], + Ok(Some(" trim \n")), + &str, + Utf8, + StringArray + ); + test_function!( + Rtrim, + &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], + Ok(Some(" trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Rtrim, + &[lit(ScalarValue::Utf8(Some("trim".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Rtrim, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA224, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Ok(Some(&[ + 11u8, 246u8, 203u8, 98u8, 100u8, 156u8, 66u8, 169u8, 174u8, 56u8, 118u8, + 171u8, 111u8, 109u8, 146u8, 173u8, 54u8, 203u8, 84u8, 20u8, 228u8, 149u8, + 248u8, 135u8, 50u8, 146u8, 190u8, 77u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA224, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(&[ + 209u8, 74u8, 2u8, 140u8, 42u8, 58u8, 43u8, 201u8, 71u8, 97u8, 2u8, 187u8, + 40u8, 130u8, 52u8, 196u8, 21u8, 162u8, 176u8, 31u8, 130u8, 142u8, 166u8, + 42u8, 197u8, 179u8, 228u8, 47u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA224, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &[u8], + Binary, + BinaryArray + ); + #[cfg(not(feature = "crypto_expressions"))] + test_function!( + SHA224, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Err(DataFusionError::Internal( + "function sha224 requires compilation with feature flag: crypto_expressions.".to_string() + )), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA256, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Ok(Some(&[ + 225u8, 96u8, 143u8, 117u8, 197u8, 215u8, 129u8, 63u8, 61u8, 64u8, 49u8, + 203u8, 48u8, 191u8, 183u8, 134u8, 80u8, 125u8, 152u8, 19u8, 117u8, 56u8, + 255u8, 142u8, 18u8, 138u8, 111u8, 247u8, 78u8, 132u8, 230u8, 67u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA256, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(&[ + 227u8, 176u8, 196u8, 66u8, 152u8, 252u8, 28u8, 20u8, 154u8, 251u8, 244u8, + 200u8, 153u8, 111u8, 185u8, 36u8, 39u8, 174u8, 65u8, 228u8, 100u8, 155u8, + 147u8, 76u8, 164u8, 149u8, 153u8, 27u8, 120u8, 82u8, 184u8, 85u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA256, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &[u8], + Binary, + BinaryArray + ); + #[cfg(not(feature = "crypto_expressions"))] + test_function!( + SHA256, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Err(DataFusionError::Internal( + "function sha256 requires compilation with feature flag: crypto_expressions.".to_string() + )), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA384, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Ok(Some(&[ + 9u8, 111u8, 91u8, 104u8, 170u8, 119u8, 132u8, 142u8, 79u8, 223u8, 92u8, + 28u8, 11u8, 53u8, 13u8, 226u8, 219u8, 250u8, 214u8, 15u8, 253u8, 124u8, + 37u8, 217u8, 234u8, 7u8, 198u8, 193u8, 155u8, 138u8, 77u8, 85u8, 169u8, + 24u8, 126u8, 177u8, 23u8, 197u8, 87u8, 136u8, 63u8, 88u8, 193u8, 109u8, + 250u8, 195u8, 227u8, 67u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA384, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(&[ + 56u8, 176u8, 96u8, 167u8, 81u8, 172u8, 150u8, 56u8, 76u8, 217u8, 50u8, + 126u8, 177u8, 177u8, 227u8, 106u8, 33u8, 253u8, 183u8, 17u8, 20u8, 190u8, + 7u8, 67u8, 76u8, 12u8, 199u8, 191u8, 99u8, 246u8, 225u8, 218u8, 39u8, + 78u8, 222u8, 191u8, 231u8, 111u8, 101u8, 251u8, 213u8, 26u8, 210u8, + 241u8, 72u8, 152u8, 185u8, 91u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA384, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &[u8], + Binary, + BinaryArray + ); + #[cfg(not(feature = "crypto_expressions"))] + test_function!( + SHA384, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Err(DataFusionError::Internal( + "function sha384 requires compilation with feature flag: crypto_expressions.".to_string() + )), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA512, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Ok(Some(&[ + 110u8, 27u8, 155u8, 63u8, 232u8, 64u8, 104u8, 14u8, 55u8, 5u8, 31u8, + 122u8, 213u8, 233u8, 89u8, 214u8, 243u8, 154u8, 208u8, 248u8, 136u8, + 93u8, 133u8, 81u8, 102u8, 245u8, 92u8, 101u8, 148u8, 105u8, 211u8, 200u8, + 183u8, 129u8, 24u8, 196u8, 74u8, 42u8, 73u8, 199u8, 45u8, 219u8, 72u8, + 28u8, 214u8, 216u8, 115u8, 16u8, 52u8, 225u8, 28u8, 192u8, 48u8, 7u8, + 11u8, 168u8, 67u8, 169u8, 11u8, 52u8, 149u8, 203u8, 141u8, 62u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA512, + &[lit(ScalarValue::Utf8(Some("".to_string())))], + Ok(Some(&[ + 207u8, 131u8, 225u8, 53u8, 126u8, 239u8, 184u8, 189u8, 241u8, 84u8, 40u8, + 80u8, 214u8, 109u8, 128u8, 7u8, 214u8, 32u8, 228u8, 5u8, 11u8, 87u8, + 21u8, 220u8, 131u8, 244u8, 169u8, 33u8, 211u8, 108u8, 233u8, 206u8, 71u8, + 208u8, 209u8, 60u8, 93u8, 133u8, 242u8, 176u8, 255u8, 131u8, 24u8, 210u8, + 135u8, 126u8, 236u8, 47u8, 99u8, 185u8, 49u8, 189u8, 71u8, 65u8, 122u8, + 129u8, 165u8, 56u8, 50u8, 122u8, 249u8, 39u8, 218u8, 62u8 + ])), + &[u8], + Binary, + BinaryArray + ); + #[cfg(feature = "crypto_expressions")] + test_function!( + SHA512, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &[u8], + Binary, + BinaryArray + ); + #[cfg(not(feature = "crypto_expressions"))] + test_function!( + SHA512, + &[lit(ScalarValue::Utf8(Some("tom".to_string())))], + Err(DataFusionError::Internal( + "function sha512 requires compilation with feature flag: crypto_expressions.".to_string() + )), + &[u8], + Binary, + BinaryArray + ); + test_function!( + SplitPart, + &[ + lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))), + lit(ScalarValue::Utf8(Some("~@~".to_string()))), + lit(ScalarValue::Int64(Some(2))), + ], + Ok(Some("def")), + &str, + Utf8, + StringArray + ); + test_function!( + SplitPart, + &[ + lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))), + lit(ScalarValue::Utf8(Some("~@~".to_string()))), + lit(ScalarValue::Int64(Some(20))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + test_function!( + SplitPart, + &[ + lit(ScalarValue::Utf8(Some("abc~@~def~@~ghi".to_string()))), + lit(ScalarValue::Utf8(Some("~@~".to_string()))), + lit(ScalarValue::Int64(Some(-1))), + ], + Err(DataFusionError::Execution( + "field position must be greater than zero".to_string(), + )), + &str, + Utf8, + StringArray + ); + test_function!( + StartsWith, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Utf8(Some("alph".to_string()))), + ], + Ok(Some(true)), + bool, + Boolean, + BooleanArray + ); + test_function!( + StartsWith, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Utf8(Some("blph".to_string()))), + ], + Ok(Some(false)), + bool, + Boolean, + BooleanArray + ); + test_function!( + StartsWith, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("alph".to_string()))), + ], + Ok(None), + bool, + Boolean, + BooleanArray + ); + test_function!( + StartsWith, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Utf8(None)), + ], + Ok(None), + bool, + Boolean, + BooleanArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Strpos, + &[ + lit(ScalarValue::Utf8(Some("abc".to_string()))), + lit(ScalarValue::Utf8(Some("c".to_string()))), + ], + Ok(Some(3)), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Strpos, + &[ + lit(ScalarValue::Utf8(Some("josé".to_string()))), + lit(ScalarValue::Utf8(Some("é".to_string()))), + ], + Ok(Some(4)), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Strpos, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Utf8(Some("so".to_string()))), + ], + Ok(Some(6)), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Strpos, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Utf8(Some("abc".to_string()))), + ], + Ok(Some(0)), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Strpos, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("abc".to_string()))), + ], + Ok(None), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Strpos, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Utf8(None)), + ], + Ok(None), + i32, + Int32, + Int32Array + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Strpos, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Utf8(None)), + ], + Err(DataFusionError::Internal( + "function strpos requires compilation with feature flag: unicode_expressions.".to_string() + )), + i32, + Int32, + Int32Array + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(0))), + ], + Ok(Some("alphabet")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some("ésoj")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Int64(Some(-5))), + ], + Ok(Some("joséésoj")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(1))), + ], + Ok(Some("alphabet")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(2))), + ], + Ok(Some("lphabet")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(3))), + ], + Ok(Some("phabet")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(-3))), + ], + Ok(Some("alphabet")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(30))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(3))), + lit(ScalarValue::Int64(Some(2))), + ], + Ok(Some("ph")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(3))), + lit(ScalarValue::Int64(Some(20))), + ], + Ok(Some("phabet")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(0))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some("alph")), + &str, + Utf8, + StringArray + ); + // starting from 5 (10 + -5) + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(-5))), + lit(ScalarValue::Int64(Some(10))), + ], + Ok(Some("alph")), + &str, + Utf8, + StringArray + ); + // starting from -1 (4 + -5) + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(-5))), + lit(ScalarValue::Int64(Some(4))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + // starting from 0 (5 + -5) + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(-5))), + lit(ScalarValue::Int64(Some(5))), + ], + Ok(Some("")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(None)), + lit(ScalarValue::Int64(Some(20))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(3))), + lit(ScalarValue::Int64(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(1))), + lit(ScalarValue::Int64(Some(-1))), + ], + Err(DataFusionError::Execution( + "negative substring length not allowed: substr(, 1, -1)".to_string(), + )), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("joséésoj".to_string()))), + lit(ScalarValue::Int64(Some(5))), + lit(ScalarValue::Int64(Some(2))), + ], + Ok(Some("és")), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Substr, + &[ + lit(ScalarValue::Utf8(Some("alphabet".to_string()))), + lit(ScalarValue::Int64(Some(0))), + ], + Err(DataFusionError::Internal( + "function substr requires compilation with feature flag: unicode_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Translate, + &[ + lit(ScalarValue::Utf8(Some("12345".to_string()))), + lit(ScalarValue::Utf8(Some("143".to_string()))), + lit(ScalarValue::Utf8(Some("ax".to_string()))), + ], + Ok(Some("a2x5")), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Translate, + &[ + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("143".to_string()))), + lit(ScalarValue::Utf8(Some("ax".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Translate, + &[ + lit(ScalarValue::Utf8(Some("12345".to_string()))), + lit(ScalarValue::Utf8(None)), + lit(ScalarValue::Utf8(Some("ax".to_string()))), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Translate, + &[ + lit(ScalarValue::Utf8(Some("12345".to_string()))), + lit(ScalarValue::Utf8(Some("143".to_string()))), + lit(ScalarValue::Utf8(None)), + ], + Ok(None), + &str, + Utf8, + StringArray + ); + #[cfg(feature = "unicode_expressions")] + test_function!( + Translate, + &[ + lit(ScalarValue::Utf8(Some("é2íñ5".to_string()))), + lit(ScalarValue::Utf8(Some("éñí".to_string()))), + lit(ScalarValue::Utf8(Some("óü".to_string()))), + ], + Ok(Some("ó2ü5")), + &str, + Utf8, + StringArray + ); + #[cfg(not(feature = "unicode_expressions"))] + test_function!( + Translate, + &[ + lit(ScalarValue::Utf8(Some("12345".to_string()))), + lit(ScalarValue::Utf8(Some("143".to_string()))), + lit(ScalarValue::Utf8(Some("ax".to_string()))), + ], + Err(DataFusionError::Internal( + "function translate requires compilation with feature flag: unicode_expressions.".to_string() + )), + &str, + Utf8, + StringArray + ); + test_function!( + Trim, + &[lit(ScalarValue::Utf8(Some(" trim ".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Trim, + &[lit(ScalarValue::Utf8(Some("trim ".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Trim, + &[lit(ScalarValue::Utf8(Some(" trim".to_string())))], + Ok(Some("trim")), + &str, + Utf8, + StringArray + ); + test_function!( + Trim, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + test_function!( + Upper, + &[lit(ScalarValue::Utf8(Some("upper".to_string())))], + Ok(Some("UPPER")), + &str, + Utf8, + StringArray + ); + test_function!( + Upper, + &[lit(ScalarValue::Utf8(Some("UPPER".to_string())))], + Ok(Some("UPPER")), + &str, + Utf8, + StringArray + ); + test_function!( + Upper, + &[lit(ScalarValue::Utf8(None))], + Ok(None), + &str, + Utf8, + StringArray + ); + Ok(()) } - /// The name for this expression - pub fn name(&self) -> &str { - &self.name + #[test] + fn test_empty_arguments_error() -> Result<()> { + let execution_props = ExecutionProps::new(); + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + + // pick some arbitrary functions to test + let funs = [ + BuiltinScalarFunction::Concat, + BuiltinScalarFunction::ToTimestamp, + BuiltinScalarFunction::Abs, + BuiltinScalarFunction::Repeat, + ]; + + for fun in funs.iter() { + let expr = create_physical_expr(fun, &[], &schema, &execution_props); + + match expr { + Ok(..) => { + return Err(DataFusionError::Plan(format!( + "Builtin scalar function {} does not support empty arguments", + fun + ))); + } + Err(DataFusionError::Internal(err)) => { + if err + != format!( + "Builtin scalar function {} does not support empty arguments", + fun + ) + { + return Err(DataFusionError::Internal(format!( + "Builtin scalar function {} didn't got the right error message with empty arguments", fun))); + } + } + Err(..) => { + return Err(DataFusionError::Internal(format!( + "Builtin scalar function {} didn't got the right error with empty arguments", fun))); + } + } + } + Ok(()) } - /// Input arguments - pub fn args(&self) -> &[Arc] { - &self.args + #[test] + fn test_empty_arguments() -> Result<()> { + let execution_props = ExecutionProps::new(); + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + + let funs = [BuiltinScalarFunction::Now, BuiltinScalarFunction::Random]; + + for fun in funs.iter() { + create_physical_expr(fun, &[], &schema, &execution_props)?; + } + Ok(()) } - /// Data type produced by this expression - pub fn return_type(&self) -> &DataType { - &self.return_type + fn generic_test_array( + value1: ArrayRef, + value2: ArrayRef, + expected_type: DataType, + expected: &str, + ) -> Result<()> { + // any type works here: we evaluate against a literal of `value` + let schema = Schema::new(vec![ + Field::new("a", value1.data_type().clone(), false), + Field::new("b", value2.data_type().clone(), false), + ]); + let columns: Vec = vec![value1, value2]; + let execution_props = ExecutionProps::new(); + + let expr = create_physical_expr( + &BuiltinScalarFunction::Array, + &[col("a", &schema)?, col("b", &schema)?], + &schema, + &execution_props, + )?; + + // type is correct + assert_eq!( + expr.data_type(&schema)?, + // type equals to a common coercion + DataType::FixedSizeList(Box::new(Field::new("item", expected_type, true)), 2) + ); + + // evaluate works + let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; + let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + + // downcast works + let result = result + .as_any() + .downcast_ref::() + .unwrap(); + + // value is correct + assert_eq!(format!("{:?}", result.value(0)), expected); + + Ok(()) } -} -impl fmt::Display for ScalarFunctionExpr { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!( - f, - "{}({})", - self.name, - self.args - .iter() - .map(|e| format!("{}", e)) - .collect::>() - .join(", ") + #[test] + fn test_array() -> Result<()> { + generic_test_array( + Arc::new(StringArray::from_slice(&["aa"])), + Arc::new(StringArray::from_slice(&["bb"])), + DataType::Utf8, + "StringArray\n[\n \"aa\",\n \"bb\",\n]", + )?; + + // different types, to validate that casting happens + generic_test_array( + Arc::new(UInt32Array::from_slice(&[1u32])), + Arc::new(UInt64Array::from_slice(&[1u64])), + DataType::UInt64, + "PrimitiveArray\n[\n 1,\n 1,\n]", + )?; + + // different types (another order), to validate that casting happens + generic_test_array( + Arc::new(UInt64Array::from_slice(&[1u64])), + Arc::new(UInt32Array::from_slice(&[1u32])), + DataType::UInt64, + "PrimitiveArray\n[\n 1,\n 1,\n]", ) } -} -impl PhysicalExpr for ScalarFunctionExpr { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } + #[test] + #[cfg(feature = "regex_expressions")] + fn test_regexp_match() -> Result<()> { + use arrow::array::ListArray; + let schema = Schema::new(vec![Field::new("a", DataType::Utf8, false)]); + let execution_props = ExecutionProps::new(); - fn data_type(&self, _input_schema: &Schema) -> Result { - Ok(self.return_type.clone()) - } + let col_value: ArrayRef = Arc::new(StringArray::from_slice(&["aaa-555"])); + let pattern = lit(ScalarValue::Utf8(Some(r".*-(\d*)".to_string()))); + let columns: Vec = vec![col_value]; + let expr = create_physical_expr( + &BuiltinScalarFunction::RegexpMatch, + &[col("a", &schema)?, pattern], + &schema, + &execution_props, + )?; + + // type is correct + assert_eq!( + expr.data_type(&schema)?, + DataType::List(Box::new(Field::new("item", DataType::Utf8, true))) + ); + + // evaluate works + let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; + let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + + // downcast works + let result = result.as_any().downcast_ref::().unwrap(); + let first_row = result.value(0); + let first_row = first_row.as_any().downcast_ref::().unwrap(); - fn nullable(&self, _input_schema: &Schema) -> Result { - Ok(true) + // value is correct + let expected = "555".to_string(); + assert_eq!(first_row.value(0), expected); + + Ok(()) } - fn evaluate(&self, batch: &RecordBatch) -> Result { - // evaluate the arguments, if there are no arguments we'll instead pass in a null array - // indicating the batch size (as a convention) - let inputs = match (self.args.len(), self.name.parse::()) { - (0, Ok(scalar_fun)) if scalar_fun.supports_zero_argument() => { - vec![NullColumnarValue::from(batch)] - } - _ => self - .args - .iter() - .map(|e| e.evaluate(batch)) - .collect::>>()?, - }; + #[test] + #[cfg(feature = "regex_expressions")] + fn test_regexp_match_all_literals() -> Result<()> { + use arrow::array::ListArray; + let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]); + let execution_props = ExecutionProps::new(); + + let col_value = lit(ScalarValue::Utf8(Some("aaa-555".to_string()))); + let pattern = lit(ScalarValue::Utf8(Some(r".*-(\d*)".to_string()))); + let columns: Vec = vec![Arc::new(Int32Array::from_slice(&[1]))]; + let expr = create_physical_expr( + &BuiltinScalarFunction::RegexpMatch, + &[col_value, pattern], + &schema, + &execution_props, + )?; + + // type is correct + assert_eq!( + expr.data_type(&schema)?, + DataType::List(Box::new(Field::new("item", DataType::Utf8, true))) + ); + + // evaluate works + let batch = RecordBatch::try_new(Arc::new(schema.clone()), columns)?; + let result = expr.evaluate(&batch)?.into_array(batch.num_rows()); + + // downcast works + let result = result.as_any().downcast_ref::().unwrap(); + let first_row = result.value(0); + let first_row = first_row.as_any().downcast_ref::().unwrap(); + + // value is correct + let expected = "555".to_string(); + assert_eq!(first_row.value(0), expected); - // evaluate the function - let fun = self.fun.as_ref(); - (fun)(&inputs) + Ok(()) } } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 34231934ddce5..6461a1fb1d13f 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -21,20 +21,28 @@ pub mod conditional_expressions; #[cfg(feature = "crypto_expressions")] pub mod crypto_expressions; pub mod datetime_expressions; +pub mod execution_props; pub mod expressions; -mod functions; +pub mod from_slice; +pub mod functions; pub mod math_expressions; mod physical_expr; +pub mod planner; #[cfg(feature = "regex_expressions")] pub mod regex_expressions; +mod scalar_function; mod sort_expr; pub mod string_expressions; pub mod struct_expressions; +pub mod type_coercion; +pub mod udf; #[cfg(feature = "unicode_expressions")] pub mod unicode_expressions; +pub mod var_provider; pub mod window; pub use aggregate::AggregateExpr; -pub use functions::ScalarFunctionExpr; pub use physical_expr::PhysicalExpr; +pub use planner::create_physical_expr; +pub use scalar_function::ScalarFunctionExpr; pub use sort_expr::PhysicalSortExpr; diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs new file mode 100644 index 0000000000000..92580fce01091 --- /dev/null +++ b/datafusion/physical-expr/src/planner.rs @@ -0,0 +1,336 @@ +// 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 crate::{ + execution_props::ExecutionProps, + expressions::{ + self, binary, CaseExpr, Column, DateIntervalExpr, GetIndexedFieldExpr, Literal, + }, + functions, udf, + var_provider::VarType, + PhysicalExpr, +}; +use arrow::{ + compute::can_cast_types, + datatypes::{DataType, Schema}, +}; +use datafusion_common::{DFSchema, DataFusionError, Result, ScalarValue}; +use datafusion_expr::{Expr, Operator}; +use std::sync::Arc; + +/// Create a physical expression from a logical expression ([Expr]) +pub fn create_physical_expr( + e: &Expr, + input_dfschema: &DFSchema, + input_schema: &Schema, + execution_props: &ExecutionProps, +) -> Result> { + match e { + Expr::Alias(expr, ..) => Ok(create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?), + Expr::Column(c) => { + let idx = input_dfschema.index_of_column(c)?; + Ok(Arc::new(Column::new(&c.name, idx))) + } + Expr::Literal(value) => Ok(Arc::new(Literal::new(value.clone()))), + Expr::ScalarVariable(_, variable_names) => { + if &variable_names[0][0..2] == "@@" { + match execution_props.get_var_provider(VarType::System) { + Some(provider) => { + let scalar_value = provider.get_value(variable_names.clone())?; + Ok(Arc::new(Literal::new(scalar_value))) + } + _ => Err(DataFusionError::Plan( + "No system variable provider found".to_string(), + )), + } + } else { + match execution_props.get_var_provider(VarType::UserDefined) { + Some(provider) => { + let scalar_value = provider.get_value(variable_names.clone())?; + Ok(Arc::new(Literal::new(scalar_value))) + } + _ => Err(DataFusionError::Plan( + "No user defined variable provider found".to_string(), + )), + } + } + } + Expr::BinaryExpr { left, op, right } => { + let lhs = create_physical_expr( + left, + input_dfschema, + input_schema, + execution_props, + )?; + let rhs = create_physical_expr( + right, + input_dfschema, + input_schema, + execution_props, + )?; + match ( + lhs.data_type(input_schema)?, + op, + rhs.data_type(input_schema)?, + ) { + ( + DataType::Date32 | DataType::Date64, + Operator::Plus | Operator::Minus, + DataType::Interval(_), + ) => Ok(Arc::new(DateIntervalExpr::try_new( + lhs, + *op, + rhs, + input_schema, + )?)), + _ => { + // assume that we can coerce both sides into a common type + // and then perform a binary operation + binary(lhs, *op, rhs, input_schema) + } + } + } + Expr::Case { + expr, + when_then_expr, + else_expr, + .. + } => { + let expr: Option> = if let Some(e) = expr { + Some(create_physical_expr( + e.as_ref(), + input_dfschema, + input_schema, + execution_props, + )?) + } else { + None + }; + let when_expr = when_then_expr + .iter() + .map(|(w, _)| { + create_physical_expr( + w.as_ref(), + input_dfschema, + input_schema, + execution_props, + ) + }) + .collect::>>()?; + let then_expr = when_then_expr + .iter() + .map(|(_, t)| { + create_physical_expr( + t.as_ref(), + input_dfschema, + input_schema, + execution_props, + ) + }) + .collect::>>()?; + let when_then_expr: Vec<(Arc, Arc)> = + when_expr + .iter() + .zip(then_expr.iter()) + .map(|(w, t)| (w.clone(), t.clone())) + .collect(); + let else_expr: Option> = if let Some(e) = else_expr { + Some(create_physical_expr( + e.as_ref(), + input_dfschema, + input_schema, + execution_props, + )?) + } else { + None + }; + Ok(Arc::new(CaseExpr::try_new( + expr, + &when_then_expr, + else_expr, + )?)) + } + Expr::Cast { expr, data_type } => expressions::cast( + create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, + input_schema, + data_type.clone(), + ), + Expr::TryCast { expr, data_type } => expressions::try_cast( + create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, + input_schema, + data_type.clone(), + ), + Expr::Not(expr) => expressions::not(create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?), + Expr::Negative(expr) => expressions::negative( + create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, + input_schema, + ), + Expr::IsNull(expr) => expressions::is_null(create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?), + Expr::IsNotNull(expr) => expressions::is_not_null(create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?), + Expr::GetIndexedField { expr, key } => Ok(Arc::new(GetIndexedFieldExpr::new( + create_physical_expr(expr, input_dfschema, input_schema, execution_props)?, + key.clone(), + ))), + + Expr::ScalarFunction { fun, args } => { + let physical_args = args + .iter() + .map(|e| { + create_physical_expr(e, input_dfschema, input_schema, execution_props) + }) + .collect::>>()?; + functions::create_physical_expr( + fun, + &physical_args, + input_schema, + execution_props, + ) + } + Expr::ScalarUDF { fun, args } => { + let mut physical_args = vec![]; + for e in args { + physical_args.push(create_physical_expr( + e, + input_dfschema, + input_schema, + execution_props, + )?); + } + + udf::create_physical_expr(fun.clone().as_ref(), &physical_args, input_schema) + } + Expr::Between { + expr, + negated, + low, + high, + } => { + let value_expr = create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?; + let low_expr = + create_physical_expr(low, input_dfschema, input_schema, execution_props)?; + let high_expr = create_physical_expr( + high, + input_dfschema, + input_schema, + execution_props, + )?; + + // rewrite the between into the two binary operators + let binary_expr = binary( + binary(value_expr.clone(), Operator::GtEq, low_expr, input_schema)?, + Operator::And, + binary(value_expr.clone(), Operator::LtEq, high_expr, input_schema)?, + input_schema, + ); + + if *negated { + expressions::not(binary_expr?) + } else { + binary_expr + } + } + Expr::InList { + expr, + list, + negated, + } => match expr.as_ref() { + Expr::Literal(ScalarValue::Utf8(None)) => { + Ok(expressions::lit(ScalarValue::Boolean(None))) + } + _ => { + let value_expr = create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?; + let value_expr_data_type = value_expr.data_type(input_schema)?; + + let list_exprs = list + .iter() + .map(|expr| match expr { + Expr::Literal(ScalarValue::Utf8(None)) => create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + ), + _ => { + let list_expr = create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?; + let list_expr_data_type = + list_expr.data_type(input_schema)?; + + if list_expr_data_type == value_expr_data_type { + Ok(list_expr) + } else if can_cast_types( + &list_expr_data_type, + &value_expr_data_type, + ) { + expressions::cast( + list_expr, + input_schema, + value_expr.data_type(input_schema)?, + ) + } else { + Err(DataFusionError::Plan(format!( + "Unsupported CAST from {:?} to {:?}", + list_expr_data_type, value_expr_data_type + ))) + } + } + }) + .collect::>>()?; + + expressions::in_list(value_expr, list_exprs, negated) + } + }, + other => Err(DataFusionError::NotImplemented(format!( + "Physical plan does not support logical expression {:?}", + other + ))), + } +} diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs new file mode 100644 index 0000000000000..1350d49510d58 --- /dev/null +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -0,0 +1,148 @@ +// 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. + +//! Declaration of built-in (scalar) functions. +//! This module contains built-in functions' enumeration and metadata. +//! +//! Generally, a function has: +//! * a signature +//! * a return type, that is a function of the incoming argument's types +//! * the computation, that must accept each valid signature +//! +//! * Signature: see `Signature` +//! * Return type: a function `(arg_types) -> return_type`. E.g. for sqrt, ([f32]) -> f32, ([f64]) -> f64. +//! +//! This module also has a set of coercion rules to improve user experience: if an argument i32 is passed +//! to a function that supports f64, it is coerced to f64. + +use crate::PhysicalExpr; +use arrow::datatypes::{DataType, Schema}; +use arrow::record_batch::RecordBatch; +use datafusion_common::Result; +use datafusion_expr::BuiltinScalarFunction; +use datafusion_expr::ColumnarValue; +pub use datafusion_expr::NullColumnarValue; +use datafusion_expr::ScalarFunctionImplementation; +use std::any::Any; +use std::fmt::Debug; +use std::fmt::{self, Formatter}; +use std::sync::Arc; + +/// Physical expression of a scalar function +pub struct ScalarFunctionExpr { + fun: ScalarFunctionImplementation, + name: String, + args: Vec>, + return_type: DataType, +} + +impl Debug for ScalarFunctionExpr { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + f.debug_struct("ScalarFunctionExpr") + .field("fun", &"") + .field("name", &self.name) + .field("args", &self.args) + .field("return_type", &self.return_type) + .finish() + } +} + +impl ScalarFunctionExpr { + /// Create a new Scalar function + pub fn new( + name: &str, + fun: ScalarFunctionImplementation, + args: Vec>, + return_type: &DataType, + ) -> Self { + Self { + fun, + name: name.to_owned(), + args, + return_type: return_type.clone(), + } + } + + /// Get the scalar function implementation + pub fn fun(&self) -> &ScalarFunctionImplementation { + &self.fun + } + + /// The name for this expression + pub fn name(&self) -> &str { + &self.name + } + + /// Input arguments + pub fn args(&self) -> &[Arc] { + &self.args + } + + /// Data type produced by this expression + pub fn return_type(&self) -> &DataType { + &self.return_type + } +} + +impl fmt::Display for ScalarFunctionExpr { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "{}({})", + self.name, + self.args + .iter() + .map(|e| format!("{}", e)) + .collect::>() + .join(", ") + ) + } +} + +impl PhysicalExpr for ScalarFunctionExpr { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, _input_schema: &Schema) -> Result { + Ok(self.return_type.clone()) + } + + fn nullable(&self, _input_schema: &Schema) -> Result { + Ok(true) + } + + fn evaluate(&self, batch: &RecordBatch) -> Result { + // evaluate the arguments, if there are no arguments we'll instead pass in a null array + // indicating the batch size (as a convention) + let inputs = match (self.args.len(), self.name.parse::()) { + (0, Ok(scalar_fun)) if scalar_fun.supports_zero_argument() => { + vec![NullColumnarValue::from(batch)] + } + _ => self + .args + .iter() + .map(|e| e.evaluate(batch)) + .collect::>>()?, + }; + + // evaluate the function + let fun = self.fun.as_ref(); + (fun)(&inputs) + } +} diff --git a/datafusion/core/src/physical_plan/type_coercion.rs b/datafusion/physical-expr/src/type_coercion.rs similarity index 98% rename from datafusion/core/src/physical_plan/type_coercion.rs rename to datafusion/physical-expr/src/type_coercion.rs index 9f9103c5951ff..fb5f59ef376de 100644 --- a/datafusion/core/src/physical_plan/type_coercion.rs +++ b/datafusion/physical-expr/src/type_coercion.rs @@ -29,14 +29,12 @@ //! i64. However, i64 -> i32 is never performed as there are i64 //! values which can not be represented by i32 values. -use std::{sync::Arc, vec}; - -use arrow::datatypes::Schema; - use super::PhysicalExpr; -use crate::error::Result; -use crate::physical_plan::expressions::try_cast; +use crate::expressions::try_cast; +use arrow::datatypes::Schema; +use datafusion_common::Result; use datafusion_expr::{type_coercion::data_types, Signature}; +use std::{sync::Arc, vec}; /// Returns `expressions` coerced to types compatible with /// `signature`, if possible. @@ -68,10 +66,10 @@ pub fn coerce( #[cfg(test)] mod tests { use super::*; + use crate::expressions::col; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::DataFusionError; use datafusion_expr::Volatility; - use datafusion_physical_expr::expressions::col; #[test] fn test_coerce() -> Result<()> { diff --git a/datafusion/core/src/physical_plan/udf.rs b/datafusion/physical-expr/src/udf.rs similarity index 93% rename from datafusion/core/src/physical_plan/udf.rs rename to datafusion/physical-expr/src/udf.rs index 58e66da48a7d2..74bcb4921eb3e 100644 --- a/datafusion/core/src/physical_plan/udf.rs +++ b/datafusion/physical-expr/src/udf.rs @@ -18,13 +18,10 @@ //! UDF support use super::type_coercion::coerce; -use crate::error::Result; -use crate::physical_plan::functions::ScalarFunctionExpr; -use crate::physical_plan::PhysicalExpr; +use crate::{PhysicalExpr, ScalarFunctionExpr}; use arrow::datatypes::Schema; - +use datafusion_common::Result; pub use datafusion_expr::ScalarUDF; - use std::sync::Arc; /// Create a physical expression of the UDF. diff --git a/datafusion/physical-expr/src/var_provider.rs b/datafusion/physical-expr/src/var_provider.rs new file mode 100644 index 0000000000000..db68200d4b06e --- /dev/null +++ b/datafusion/physical-expr/src/var_provider.rs @@ -0,0 +1,39 @@ +// 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. + +//! Variable provider + +use arrow::datatypes::DataType; +use datafusion_common::{Result, ScalarValue}; + +/// Variable type, system/user defined +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub enum VarType { + /// System variable, like @@version + System, + /// User defined variable, like @name + UserDefined, +} + +/// A var provider for @variable +pub trait VarProvider { + /// Get variable value + fn get_value(&self, var_names: Vec) -> Result; + + /// Return the type of the given variable + fn get_type(&self, var_names: &[String]) -> Option; +}