diff --git a/datafusion-examples/examples/advanced_udf.rs b/datafusion-examples/examples/advanced_udf.rs index 290d1c53334b7..56ae599efa11b 100644 --- a/datafusion-examples/examples/advanced_udf.rs +++ b/datafusion-examples/examples/advanced_udf.rs @@ -39,7 +39,7 @@ use datafusion::prelude::*; /// the power of the second argument `a^b`. /// /// To do so, we must implement the `ScalarUDFImpl` trait. -#[derive(Debug, Clone)] +#[derive(Debug, PartialEq, Eq, Hash)] struct PowUdf { signature: Signature, aliases: Vec, diff --git a/datafusion-examples/examples/async_udf.rs b/datafusion-examples/examples/async_udf.rs index 59d89cb744d1c..b52ec68ea4422 100644 --- a/datafusion-examples/examples/async_udf.rs +++ b/datafusion-examples/examples/async_udf.rs @@ -133,7 +133,7 @@ fn animal() -> Result { /// /// Since this is a simplified example, it does not call an LLM service, but /// could be extended to do so in a real-world scenario. -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct AskLLM { signature: Signature, } diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index 425b0861fff7f..adc88f4305d2c 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -28,7 +28,7 @@ use datafusion::logical_expr::{ ColumnarValue, CreateFunction, Expr, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; -use std::hash::{DefaultHasher, Hash, Hasher}; +use std::hash::Hash; use std::result::Result as RResult; use std::sync::Arc; @@ -107,7 +107,7 @@ impl FunctionFactory for CustomFunctionFactory { } /// this function represents the newly created execution engine. -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct ScalarFunctionWrapper { /// The text of the function body, `$1 + f1($2)` in our example name: String, @@ -154,38 +154,6 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { fn output_ordering(&self, _input: &[ExprProperties]) -> Result { Ok(SortProperties::Unordered) } - - fn equals(&self, other: &dyn ScalarUDFImpl) -> bool { - let Some(other) = other.as_any().downcast_ref::() else { - return false; - }; - let Self { - name, - expr, - signature, - return_type, - } = self; - name == &other.name - && expr == &other.expr - && signature == &other.signature - && return_type == &other.return_type - } - - fn hash_value(&self) -> u64 { - let Self { - name, - expr, - signature, - return_type, - } = self; - let mut hasher = DefaultHasher::new(); - std::any::type_name::().hash(&mut hasher); - name.hash(&mut hasher); - expr.hash(&mut hasher); - signature.hash(&mut hasher); - return_type.hash(&mut hasher); - hasher.finish() - } } impl ScalarFunctionWrapper { diff --git a/datafusion-examples/examples/json_shredding.rs b/datafusion-examples/examples/json_shredding.rs index 866e4a8a152c3..261bf47915c24 100644 --- a/datafusion-examples/examples/json_shredding.rs +++ b/datafusion-examples/examples/json_shredding.rs @@ -282,17 +282,15 @@ impl TableProvider for ExampleTableProvider { } /// Scalar UDF that uses serde_json to access json fields -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct JsonGetStr { signature: Signature, - aliases: [String; 1], } impl Default for JsonGetStr { fn default() -> Self { Self { signature: Signature::variadic_any(Volatility::Immutable), - aliases: ["json_get_str".to_string()], } } } @@ -303,7 +301,7 @@ impl ScalarUDFImpl for JsonGetStr { } fn name(&self) -> &str { - self.aliases[0].as_str() + "json_get_str" } fn signature(&self) -> &Signature { @@ -355,10 +353,6 @@ impl ScalarUDFImpl for JsonGetStr { .collect::(); Ok(ColumnarValue::Array(Arc::new(values))) } - - fn aliases(&self) -> &[String] { - &self.aliases - } } /// Factory for creating ShreddedJsonRewriter instances diff --git a/datafusion-examples/examples/optimizer_rule.rs b/datafusion-examples/examples/optimizer_rule.rs index 176b1a69808c1..9c137b67432c5 100644 --- a/datafusion-examples/examples/optimizer_rule.rs +++ b/datafusion-examples/examples/optimizer_rule.rs @@ -175,7 +175,7 @@ fn is_lit_or_col(expr: &Expr) -> bool { } /// A simple user defined filter function -#[derive(Debug, Clone)] +#[derive(Debug, PartialEq, Eq, Hash, Clone)] struct MyEq { signature: Signature, } diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index ef80925d99912..3e8ab82f9a631 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -512,7 +512,7 @@ fn get_sort_columns( .collect::>>() } -#[derive(Debug, Clone)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct TestScalarUDF { pub(crate) signature: Signature, } diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index 5d396b6b0247d..149c50557c3a3 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -63,7 +63,7 @@ use insta::assert_snapshot; use itertools::Itertools; /// Mocked UDF -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct DummyUDF { signature: Signature, } diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index bf7f58d51b0b9..c9b6622554ea2 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -43,9 +43,9 @@ use datafusion_common::{ use datafusion_expr::expr::FieldMetadata; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::{ - lit_with_metadata, udf_equals_hash, Accumulator, ColumnarValue, CreateFunction, - CreateFunctionBody, LogicalPlanBuilder, OperateFunctionArg, ReturnFieldArgs, - ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, Volatility, + lit_with_metadata, Accumulator, ColumnarValue, CreateFunction, CreateFunctionBody, + LogicalPlanBuilder, OperateFunctionArg, ReturnFieldArgs, ScalarFunctionArgs, + ScalarUDF, ScalarUDFImpl, Signature, Volatility, }; use datafusion_functions_nested::range::range_udf; use parking_lot::Mutex; @@ -218,8 +218,6 @@ impl ScalarUDFImpl for Simple0ArgsScalarUDF { fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some(100)))) } - - udf_equals_hash!(ScalarUDFImpl); } #[tokio::test] @@ -560,8 +558,6 @@ impl ScalarUDFImpl for AddIndexToStringVolatileScalarUDF { }; Ok(ColumnarValue::Array(Arc::new(StringArray::from(answer)))) } - - udf_equals_hash!(ScalarUDFImpl); } #[tokio::test] @@ -665,7 +661,7 @@ async fn volatile_scalar_udf_with_params() -> Result<()> { Ok(()) } -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct CastToI64UDF { signature: Signature, } @@ -787,7 +783,7 @@ async fn deregister_udf() -> Result<()> { Ok(()) } -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct TakeUDF { signature: Signature, } @@ -979,8 +975,6 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { Ok(ExprSimplifyResult::Simplified(replacement)) } - - udf_equals_hash!(ScalarUDFImpl); } impl ScalarFunctionWrapper { @@ -1282,8 +1276,6 @@ impl ScalarUDFImpl for MyRegexUdf { _ => exec_err!("regex_udf only accepts a Utf8 arguments"), } } - - udf_equals_hash!(ScalarUDFImpl); } #[tokio::test] @@ -1471,8 +1463,6 @@ impl ScalarUDFImpl for MetadataBasedUdf { } } } - - udf_equals_hash!(ScalarUDFImpl); } #[tokio::test] @@ -1611,7 +1601,7 @@ async fn test_metadata_based_udf_with_literal() -> Result<()> { /// sides. For the input, we will handle the data differently if there is /// the canonical extension type Bool8. For the output we will add a /// user defined extension type. -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct ExtensionBasedUdf { name: String, signature: Signature, @@ -1790,7 +1780,7 @@ async fn test_extension_based_udf() -> Result<()> { #[tokio::test] async fn test_config_options_work_for_scalar_func() -> Result<()> { - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct TestScalarUDF { signature: Signature, } diff --git a/datafusion/expr/src/async_udf.rs b/datafusion/expr/src/async_udf.rs index ad07d0690e56e..a67738ac7b790 100644 --- a/datafusion/expr/src/async_udf.rs +++ b/datafusion/expr/src/async_udf.rs @@ -16,9 +16,7 @@ // under the License. use crate::ptr_eq::{arc_ptr_eq, arc_ptr_hash}; -use crate::{ - udf_equals_hash, ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, -}; +use crate::{ReturnFieldArgs, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl}; use arrow::datatypes::{DataType, FieldRef}; use async_trait::async_trait; use datafusion_common::error::Result; @@ -127,8 +125,6 @@ impl ScalarUDFImpl for AsyncScalarUDF { fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { internal_err!("async functions should not be called directly") } - - udf_equals_hash!(ScalarUDFImpl); } impl Display for AsyncScalarUDF { diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index bb376b5916962..dbae1058890a5 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -3648,7 +3648,7 @@ mod test { #[test] fn test_is_volatile_scalar_func() { // UDF - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct TestScalarUDF { signature: Signature, } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 0508882de0d2e..48a16ab595146 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -29,8 +29,8 @@ use crate::ptr_eq::PtrEq; use crate::select_expr::SelectExpr; use crate::{ conditional_expressions::CaseBuilder, expr::Sort, logical_plan::Subquery, - udf_equals_hash, AggregateUDF, Expr, LogicalPlan, Operator, PartitionEvaluator, - ScalarFunctionArgs, ScalarFunctionImplementation, ScalarUDF, Signature, Volatility, + AggregateUDF, Expr, LogicalPlan, Operator, PartitionEvaluator, ScalarFunctionArgs, + ScalarFunctionImplementation, ScalarUDF, Signature, Volatility, }; use crate::{ AggregateUDFImpl, ColumnarValue, ScalarUDFImpl, WindowFrame, WindowUDF, WindowUDFImpl, @@ -477,8 +477,6 @@ impl ScalarUDFImpl for SimpleScalarUDF { fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { (self.fun)(&args.args) } - - udf_equals_hash!(ScalarUDFImpl); } /// Creates a new UDAF with a specific signature, state type and return type. diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 54973415c272b..e1d36da9c4db7 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -1230,6 +1230,7 @@ mod test { }; use std::any::Any; use std::cmp::Ordering; + use std::hash::{DefaultHasher, Hash, Hasher}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] struct AMeanUdf { @@ -1319,6 +1320,7 @@ mod test { let eq = a1 == a2; assert!(eq); assert_eq!(a1, a2); + assert_eq!(hash(a1), hash(a2)); } #[test] @@ -1333,4 +1335,10 @@ mod test { assert!(a1 < b1); assert!(!(a1 == b1)); } + + fn hash(value: T) -> u64 { + let hasher = &mut DefaultHasher::new(); + value.hash(hasher); + hasher.finish() + } } diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index c9e903f277ce4..a2e023c2b2fae 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -22,15 +22,16 @@ use crate::expr::schema_name_from_exprs_comma_separated_without_space; use crate::simplify::{ExprSimplifyResult, SimplifyInfo}; use crate::sort_properties::{ExprProperties, SortProperties}; use crate::udf_eq::UdfEq; -use crate::{udf_equals_hash, ColumnarValue, Documentation, Expr, Signature}; +use crate::{ColumnarValue, Documentation, Expr, Signature}; use arrow::datatypes::{DataType, Field, FieldRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::{not_impl_err, ExprSchema, Result, ScalarValue}; +use datafusion_expr_common::dyn_eq::{DynEq, DynHash}; use datafusion_expr_common::interval_arithmetic::Interval; use std::any::Any; use std::cmp::Ordering; use std::fmt::Debug; -use std::hash::{DefaultHasher, Hash, Hasher}; +use std::hash::{Hash, Hasher}; use std::sync::Arc; /// Logical representation of a Scalar User Defined Function. @@ -62,7 +63,7 @@ pub struct ScalarUDF { impl PartialEq for ScalarUDF { fn eq(&self, other: &Self) -> bool { - self.inner.equals(other.inner.as_ref()) + self.inner.dyn_eq(other.inner.as_any()) } } @@ -81,7 +82,7 @@ impl Eq for ScalarUDF {} impl Hash for ScalarUDF { fn hash(&self, state: &mut H) { - self.inner.hash_value().hash(state) + self.inner.dyn_hash(state) } } @@ -367,7 +368,7 @@ pub struct ReturnFieldArgs<'a> { /// # use datafusion_expr::{ScalarUDFImpl, ScalarUDF}; /// # use datafusion_expr::scalar_doc_sections::DOC_SECTION_MATH; /// /// This struct for a simple UDF that adds one to an int32 -/// #[derive(Debug)] +/// #[derive(Debug, PartialEq, Eq, Hash)] /// struct AddOne { /// signature: Signature, /// } @@ -416,7 +417,7 @@ pub struct ReturnFieldArgs<'a> { /// // Call the function `add_one(col)` /// let expr = add_one.call(vec![col("a")]); /// ``` -pub trait ScalarUDFImpl: Debug + Send + Sync { +pub trait ScalarUDFImpl: Debug + DynEq + DynHash + Send + Sync { /// Returns this object as an [`Any`] trait object fn as_any(&self) -> &dyn Any; @@ -697,41 +698,6 @@ pub trait ScalarUDFImpl: Debug + Send + Sync { not_impl_err!("Function {} does not implement coerce_types", self.name()) } - /// Return true if this scalar UDF is equal to the other. - /// - /// Allows customizing the equality of scalar UDFs. - /// *Must* be implemented explicitly if the UDF type has internal state. - /// Must be consistent with [`Self::hash_value`] and follow the same rules as [`Eq`]: - /// - /// - reflexive: `a.equals(a)`; - /// - symmetric: `a.equals(b)` implies `b.equals(a)`; - /// - transitive: `a.equals(b)` and `b.equals(c)` implies `a.equals(c)`. - /// - /// By default, compares type, [`Self::name`], [`Self::aliases`] and [`Self::signature`]. - fn equals(&self, other: &dyn ScalarUDFImpl) -> bool { - self.as_any().type_id() == other.as_any().type_id() - && self.name() == other.name() - && self.aliases() == other.aliases() - && self.signature() == other.signature() - } - - /// Returns a hash value for this scalar UDF. - /// - /// Allows customizing the hash code of scalar UDFs. - /// *Must* be implemented explicitly whenever [`Self::equals`] is implemented. - /// - /// Similarly to [`Hash`] and [`Eq`], if [`Self::equals`] returns true for two UDFs, - /// their `hash_value`s must be the same. - /// - /// By default, it only hashes the type. The other fields are not hashed, as usually the - /// name, signature, and aliases are implied by the UDF type. Recall that UDFs with state - /// (and thus possibly changing fields) must override [`Self::equals`] and [`Self::hash_value`]. - fn hash_value(&self) -> u64 { - let hasher = &mut DefaultHasher::new(); - self.as_any().type_id().hash(hasher); - hasher.finish() - } - /// Returns the documentation for this Scalar UDF. /// /// Documentation can be accessed programmatically as well as generating @@ -842,8 +808,6 @@ impl ScalarUDFImpl for AliasedScalarUDFImpl { self.inner.coerce_types(arg_types) } - udf_equals_hash!(ScalarUDFImpl); - fn documentation(&self) -> Option<&Documentation> { self.inner.documentation() } @@ -964,3 +928,54 @@ The following regular expression functions are supported:"#, description: Some("Functions to work with the union data type, also know as tagged unions, variant types, enums or sum types. Note: Not related to the SQL UNION operator"), }; } + +#[cfg(test)] +mod tests { + use super::*; + use std::hash::DefaultHasher; + + #[derive(Debug, PartialEq, Eq, Hash)] + struct TestScalarUDFImpl { + field: &'static str, + } + impl ScalarUDFImpl for TestScalarUDFImpl { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "TestScalarUDFImpl" + } + + fn signature(&self) -> &Signature { + unimplemented!() + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + unimplemented!() + } + + fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { + unimplemented!() + } + } + + #[test] + fn test_partial_eq() { + let a1 = ScalarUDF::from(TestScalarUDFImpl { field: "a" }); + let a2 = ScalarUDF::from(TestScalarUDFImpl { field: "a" }); + let b = ScalarUDF::from(TestScalarUDFImpl { field: "b" }); + let eq = a1 == a2; + assert!(eq); + assert_eq!(a1, a2); + assert_eq!(hash(&a1), hash(&a2)); + assert_ne!(a1, b); + assert_ne!(a2, b); + } + + fn hash(value: &T) -> u64 { + let hasher = &mut DefaultHasher::new(); + value.hash(hasher); + hasher.finish() + } +} diff --git a/datafusion/expr/src/udf_eq.rs b/datafusion/expr/src/udf_eq.rs index f8dcce4f09637..6664495267129 100644 --- a/datafusion/expr/src/udf_eq.rs +++ b/datafusion/expr/src/udf_eq.rs @@ -81,21 +81,17 @@ trait UdfPointer: Deref { fn hash_value(&self) -> u64; } -macro_rules! impl_for_udf_eq { - ($udf:ty) => { - impl UdfPointer for Arc<$udf> { - fn equals(&self, other: &$udf) -> bool { - self.as_ref().equals(other) - } - - fn hash_value(&self) -> u64 { - self.as_ref().hash_value() - } - } - }; -} +impl UdfPointer for Arc { + fn equals(&self, other: &(dyn ScalarUDFImpl + '_)) -> bool { + self.as_ref().dyn_eq(other.as_any()) + } -impl_for_udf_eq!(dyn ScalarUDFImpl + '_); + fn hash_value(&self) -> u64 { + let hasher = &mut DefaultHasher::new(); + self.as_ref().dyn_hash(hasher); + hasher.finish() + } +} impl UdfPointer for Arc { fn equals(&self, other: &(dyn AggregateUDFImpl + '_)) -> bool { @@ -131,7 +127,7 @@ mod tests { use std::any::Any; use std::hash::DefaultHasher; - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct TestScalarUDF { signature: Signature, name: &'static str, diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index 7e80711a7139b..29bf379f60fbd 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -565,6 +565,7 @@ mod test { use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; use std::any::Any; use std::cmp::Ordering; + use std::hash::{DefaultHasher, Hash, Hasher}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] struct AWindowUDF { @@ -651,6 +652,7 @@ mod test { let eq = a1 == a2; assert!(eq); assert_eq!(a1, a2); + assert_eq!(hash(a1), hash(a2)); } #[test] @@ -663,4 +665,10 @@ mod test { assert!(a1 < b1); assert!(!(a1 == b1)); } + + fn hash(value: T) -> u64 { + let hasher = &mut DefaultHasher::new(); + value.hash(hasher); + hasher.finish() + } } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 80ad0f87846a7..7a612b6fe6eb3 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1262,87 +1262,6 @@ pub fn collect_subquery_cols( }) } -/// Generates implementation of `equals` and `hash_value` methods for a trait, delegating -/// to [`PartialEq`] and [`Hash`] implementations on Self. -/// Meant to be used with traits representing user-defined functions (UDFs). -/// -/// Example showing generation of [`ScalarUDFImpl::equals`] and [`ScalarUDFImpl::hash_value`] -/// implementations. -/// -/// ``` -/// # use arrow::datatypes::DataType; -/// # use datafusion_expr::{udf_equals_hash, ScalarFunctionArgs, ScalarUDFImpl}; -/// # use datafusion_expr_common::columnar_value::ColumnarValue; -/// # use datafusion_expr_common::signature::Signature; -/// # use std::any::Any; -/// -/// // Implementing Eq & Hash is a prerequisite for using this macro, -/// // but the implementation can be derived. -/// #[derive(Debug, PartialEq, Eq, Hash)] -/// struct VarcharToTimestampTz { -/// safe: bool, -/// } -/// -/// impl ScalarUDFImpl for VarcharToTimestampTz { -/// /* other methods omitted for brevity */ -/// # fn as_any(&self) -> &dyn Any { -/// # self -/// # } -/// # -/// # fn name(&self) -> &str { -/// # "varchar_to_timestamp_tz" -/// # } -/// # -/// # fn signature(&self) -> &Signature { -/// # todo!() -/// # } -/// # -/// # fn return_type( -/// # &self, -/// # _arg_types: &[DataType], -/// # ) -> datafusion_common::Result { -/// # todo!() -/// # } -/// # -/// # fn invoke_with_args( -/// # &self, -/// # args: ScalarFunctionArgs, -/// # ) -> datafusion_common::Result { -/// # todo!() -/// # } -/// # -/// udf_equals_hash!(ScalarUDFImpl); -/// } -/// ``` -/// -/// [`ScalarUDFImpl::equals`]: crate::ScalarUDFImpl::equals -/// [`ScalarUDFImpl::hash_value`]: crate::ScalarUDFImpl::hash_value -#[macro_export] -macro_rules! udf_equals_hash { - ($udf_type:tt) => { - fn equals(&self, other: &dyn $udf_type) -> bool { - use ::core::any::Any; - use ::core::cmp::{Eq, PartialEq}; - let Some(other) = ::downcast_ref::(other.as_any()) - else { - return false; - }; - fn assert_self_impls_eq() {} - assert_self_impls_eq::(); - PartialEq::eq(self, other) - } - - fn hash_value(&self) -> u64 { - use ::std::any::type_name; - use ::std::hash::{DefaultHasher, Hash, Hasher}; - let hasher = &mut DefaultHasher::new(); - type_name::().hash(hasher); - Hash::hash(self, hasher); - Hasher::finish(hasher) - } - }; -} - #[cfg(test)] mod tests { use super::*; @@ -1351,13 +1270,9 @@ mod tests { expr::WindowFunction, expr_vec_fmt, grouping_set, lit, rollup, test::function_stub::{max_udaf, min_udaf, sum_udaf}, - Cast, ExprFunctionExt, ScalarFunctionArgs, ScalarUDFImpl, - WindowFunctionDefinition, + Cast, ExprFunctionExt, WindowFunctionDefinition, }; use arrow::datatypes::{UnionFields, UnionMode}; - use datafusion_expr_common::columnar_value::ColumnarValue; - use datafusion_expr_common::signature::Volatility; - use std::any::Any; #[test] fn test_group_window_expr_by_sort_keys_empty_case() -> Result<()> { @@ -1777,91 +1692,4 @@ mod tests { DataType::List(Arc::new(Field::new("my_union", union_type, true))); assert!(!can_hash(&list_union_type)); } - - #[test] - fn test_udf_equals_hash() { - #[derive(Debug, PartialEq, Hash)] - struct StatefulFunctionWithEqHash { - signature: Signature, - state: bool, - } - impl ScalarUDFImpl for StatefulFunctionWithEqHash { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { - "StatefulFunctionWithEqHash" - } - fn signature(&self) -> &Signature { - &self.signature - } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - todo!() - } - fn invoke_with_args( - &self, - _args: ScalarFunctionArgs, - ) -> Result { - todo!() - } - } - - #[derive(Debug, PartialEq, Eq, Hash)] - struct StatefulFunctionWithEqHashWithUdfEqualsHash { - signature: Signature, - state: bool, - } - impl ScalarUDFImpl for StatefulFunctionWithEqHashWithUdfEqualsHash { - fn as_any(&self) -> &dyn Any { - self - } - fn name(&self) -> &str { - "StatefulFunctionWithEqHashWithUdfEqualsHash" - } - fn signature(&self) -> &Signature { - &self.signature - } - fn return_type(&self, _arg_types: &[DataType]) -> Result { - todo!() - } - fn invoke_with_args( - &self, - _args: ScalarFunctionArgs, - ) -> Result { - todo!() - } - udf_equals_hash!(ScalarUDFImpl); - } - - let signature = Signature::exact(vec![DataType::Utf8], Volatility::Immutable); - - // Sadly, without `udf_equals_hash!` macro, the equals and hash_value ignore state fields, - // even though the struct implements `PartialEq` and `Hash`. - let a: Box = Box::new(StatefulFunctionWithEqHash { - signature: signature.clone(), - state: true, - }); - let b: Box = Box::new(StatefulFunctionWithEqHash { - signature: signature.clone(), - state: false, - }); - assert!(a.equals(b.as_ref())); - assert_eq!(a.hash_value(), b.hash_value()); - - // With udf_equals_hash! macro, the equals and hash_value compare the state. - // even though the struct implements `PartialEq` and `Hash`. - let a: Box = - Box::new(StatefulFunctionWithEqHashWithUdfEqualsHash { - signature: signature.clone(), - state: true, - }); - let b: Box = - Box::new(StatefulFunctionWithEqHashWithUdfEqualsHash { - signature: signature.clone(), - state: false, - }); - assert!(!a.equals(b.as_ref())); - // This could be true, but it's very unlikely that boolean true and false hash the same - assert_ne!(a.hash_value(), b.hash_value()); - } } diff --git a/datafusion/ffi/src/udf/mod.rs b/datafusion/ffi/src/udf/mod.rs index 8f877d44f87f3..390b03fe621bb 100644 --- a/datafusion/ffi/src/udf/mod.rs +++ b/datafusion/ffi/src/udf/mod.rs @@ -33,7 +33,7 @@ use arrow::{ }; use arrow_schema::FieldRef; use datafusion::config::ConfigOptions; -use datafusion::logical_expr::{udf_equals_hash, ReturnFieldArgs}; +use datafusion::logical_expr::ReturnFieldArgs; use datafusion::{ error::DataFusionError, logical_expr::type_coercion::functions::data_types_with_scalar_udf, @@ -444,8 +444,6 @@ impl ScalarUDFImpl for ForeignScalarUDF { Ok(rvec_wrapped_to_vec_datatype(&result_types)?) } } - - udf_equals_hash!(ScalarUDFImpl); } #[cfg(test)] diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index 57ff8490b9d4e..af9000fd751c2 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -81,7 +81,7 @@ make_udf_expr_and_func!(ArrayHasAny, description = "Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayHas { signature: Signature, aliases: Vec, @@ -482,7 +482,7 @@ fn array_has_any_inner(args: &[ArrayRef]) -> Result { description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayHasAll { signature: Signature, aliases: Vec, @@ -556,7 +556,7 @@ impl ScalarUDFImpl for ArrayHasAll { description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayHasAny { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/cardinality.rs b/datafusion/functions-nested/src/cardinality.rs index 98bda81ef25f9..1f0f508c6dc83 100644 --- a/datafusion/functions-nested/src/cardinality.rs +++ b/datafusion/functions-nested/src/cardinality.rs @@ -80,7 +80,7 @@ impl Cardinality { description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct Cardinality { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/concat.rs b/datafusion/functions-nested/src/concat.rs index e8b7fc27b4812..43a00fefcbd98 100644 --- a/datafusion/functions-nested/src/concat.rs +++ b/datafusion/functions-nested/src/concat.rs @@ -70,7 +70,7 @@ make_udf_expr_and_func!( ), argument(name = "element", description = "Element to append to the array.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayAppend { signature: Signature, aliases: Vec, @@ -159,7 +159,7 @@ make_udf_expr_and_func!( ), argument(name = "element", description = "Element to prepend to the array.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayPrepend { signature: Signature, aliases: Vec, @@ -250,7 +250,7 @@ make_udf_expr_and_func!( description = "Subsequent array column or literal array to concatenate." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayConcat { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/dimension.rs b/datafusion/functions-nested/src/dimension.rs index d1e6b1be4cfa6..b0fc5bee5494d 100644 --- a/datafusion/functions-nested/src/dimension.rs +++ b/datafusion/functions-nested/src/dimension.rs @@ -64,7 +64,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayDims { signature: Signature, aliases: Vec, @@ -143,7 +143,7 @@ make_udf_expr_and_func!( ), argument(name = "element", description = "Array element.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayNdims { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/distance.rs b/datafusion/functions-nested/src/distance.rs index 3392e194b1760..1ccd22cdf508f 100644 --- a/datafusion/functions-nested/src/distance.rs +++ b/datafusion/functions-nested/src/distance.rs @@ -71,7 +71,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayDistance { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/empty.rs b/datafusion/functions-nested/src/empty.rs index 67c795886bded..27a90ab0442bc 100644 --- a/datafusion/functions-nested/src/empty.rs +++ b/datafusion/functions-nested/src/empty.rs @@ -58,7 +58,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayEmpty { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/except.rs b/datafusion/functions-nested/src/except.rs index 2385f6d12d43e..a7ce36ae33b52 100644 --- a/datafusion/functions-nested/src/except.rs +++ b/datafusion/functions-nested/src/except.rs @@ -66,7 +66,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayExcept { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/extract.rs b/datafusion/functions-nested/src/extract.rs index 95bf5a7341d95..7fb38ace69a79 100644 --- a/datafusion/functions-nested/src/extract.rs +++ b/datafusion/functions-nested/src/extract.rs @@ -103,7 +103,7 @@ make_udf_expr_and_func!( description = "Index to extract the element from the array." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayElement { signature: Signature, aliases: Vec, @@ -320,7 +320,7 @@ pub fn array_slice(array: Expr, begin: Expr, end: Expr, stride: Option) -> description = "Stride of the array slice. The default is 1." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArraySlice { signature: Signature, aliases: Vec, @@ -663,7 +663,7 @@ where description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayPopFront { signature: Signature, aliases: Vec, @@ -770,7 +770,7 @@ where description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayPopBack { signature: Signature, aliases: Vec, @@ -878,7 +878,7 @@ where description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayAnyValue { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/flatten.rs b/datafusion/functions-nested/src/flatten.rs index c6fa2831f4f0a..413b28aa8097e 100644 --- a/datafusion/functions-nested/src/flatten.rs +++ b/datafusion/functions-nested/src/flatten.rs @@ -60,7 +60,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct Flatten { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/length.rs b/datafusion/functions-nested/src/length.rs index 0da12684158e4..d829fac27b1d4 100644 --- a/datafusion/functions-nested/src/length.rs +++ b/datafusion/functions-nested/src/length.rs @@ -64,7 +64,7 @@ make_udf_expr_and_func!( ), argument(name = "dimension", description = "Array dimension.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayLength { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/make_array.rs b/datafusion/functions-nested/src/make_array.rs index babb039191577..e4dd9b3662cca 100644 --- a/datafusion/functions-nested/src/make_array.rs +++ b/datafusion/functions-nested/src/make_array.rs @@ -64,7 +64,7 @@ make_udf_expr_and_func!( description = "Expression to include in the output array. Can be a constant, column, or function, and any combination of arithmetic or string operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct MakeArray { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index 828f2e244112b..03cfdb52c6de7 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -221,7 +221,7 @@ SELECT MAKE_MAP(['key1', 'key2'], ['value1', null]); For `make_map`: The list of values to be mapped to the corresponding keys." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct MapFunc { signature: Signature, } diff --git a/datafusion/functions-nested/src/map_entries.rs b/datafusion/functions-nested/src/map_entries.rs index b3323d0b5c39d..7d9d103206dbc 100644 --- a/datafusion/functions-nested/src/map_entries.rs +++ b/datafusion/functions-nested/src/map_entries.rs @@ -56,7 +56,7 @@ SELECT map_entries(map([100, 5], [42, 43])); description = "Map expression. Can be a constant, column, or function, and any combination of map operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct MapEntriesFunc { signature: Signature, } diff --git a/datafusion/functions-nested/src/map_extract.rs b/datafusion/functions-nested/src/map_extract.rs index 55ab8447c54f1..4aab5d7a60d18 100644 --- a/datafusion/functions-nested/src/map_extract.rs +++ b/datafusion/functions-nested/src/map_extract.rs @@ -68,7 +68,7 @@ SELECT map_extract(MAP {'x': 10, 'y': NULL, 'z': 30}, 'y'); description = "Key to extract from the map. Can be a constant, column, or function, any combination of arithmetic or string operators, or a named expression of the previously listed." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct MapExtract { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/map_keys.rs b/datafusion/functions-nested/src/map_keys.rs index 0f15c06d86d15..080b02090a886 100644 --- a/datafusion/functions-nested/src/map_keys.rs +++ b/datafusion/functions-nested/src/map_keys.rs @@ -56,7 +56,7 @@ SELECT map_keys(map([100, 5], [42, 43])); description = "Map expression. Can be a constant, column, or function, and any combination of map operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct MapKeysFunc { signature: Signature, } diff --git a/datafusion/functions-nested/src/map_values.rs b/datafusion/functions-nested/src/map_values.rs index 8247fdd4a74ce..6ae8a278063da 100644 --- a/datafusion/functions-nested/src/map_values.rs +++ b/datafusion/functions-nested/src/map_values.rs @@ -57,7 +57,7 @@ SELECT map_values(map([100, 5], [42, 43])); description = "Map expression. Can be a constant, column, or function, and any combination of map operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(crate) struct MapValuesFunc { signature: Signature, } diff --git a/datafusion/functions-nested/src/min_max.rs b/datafusion/functions-nested/src/min_max.rs index 0eb416471a86a..117cfbeaa2b2c 100644 --- a/datafusion/functions-nested/src/min_max.rs +++ b/datafusion/functions-nested/src/min_max.rs @@ -58,7 +58,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayMax { signature: Signature, aliases: Vec, @@ -153,7 +153,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct ArrayMin { signature: Signature, } diff --git a/datafusion/functions-nested/src/position.rs b/datafusion/functions-nested/src/position.rs index 76ee7133d3338..be7ce05b6934a 100644 --- a/datafusion/functions-nested/src/position.rs +++ b/datafusion/functions-nested/src/position.rs @@ -81,7 +81,7 @@ make_udf_expr_and_func!( description = "Index at which to start searching (1-indexed)." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayPosition { signature: Signature, aliases: Vec, @@ -245,7 +245,7 @@ make_udf_expr_and_func!( description = "Element to search for position in the array." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayPositions { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/range.rs b/datafusion/functions-nested/src/range.rs index 637a78d158ab2..619b0e84c19a7 100644 --- a/datafusion/functions-nested/src/range.rs +++ b/datafusion/functions-nested/src/range.rs @@ -88,7 +88,7 @@ make_udf_expr_and_func!( description = "Increase by step (cannot be 0). Steps less than a day are supported only for timestamp ranges." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct Range { signature: Signature, aliases: Vec, @@ -218,7 +218,7 @@ make_udf_expr_and_func!( description = "Increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct GenSeries { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/remove.rs b/datafusion/functions-nested/src/remove.rs index 7f5baa18e7693..e500d7196783e 100644 --- a/datafusion/functions-nested/src/remove.rs +++ b/datafusion/functions-nested/src/remove.rs @@ -63,7 +63,7 @@ make_udf_expr_and_func!( description = "Element to be removed from the array." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayRemove { signature: Signature, aliases: Vec, @@ -147,7 +147,7 @@ make_udf_expr_and_func!( ), argument(name = "max", description = "Number of first occurrences to remove.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayRemoveN { signature: Signature, aliases: Vec, @@ -224,7 +224,7 @@ make_udf_expr_and_func!( description = "Element to be removed from the array." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayRemoveAll { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/repeat.rs b/datafusion/functions-nested/src/repeat.rs index 26d67ad3113ff..ed66b9e396762 100644 --- a/datafusion/functions-nested/src/repeat.rs +++ b/datafusion/functions-nested/src/repeat.rs @@ -74,7 +74,7 @@ make_udf_expr_and_func!( description = "Value of how many times to repeat the element." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayRepeat { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/replace.rs b/datafusion/functions-nested/src/replace.rs index 3dbe672c5b028..d791c6d1f10bf 100644 --- a/datafusion/functions-nested/src/replace.rs +++ b/datafusion/functions-nested/src/replace.rs @@ -78,7 +78,7 @@ make_udf_expr_and_func!(ArrayReplaceAll, argument(name = "from", description = "Initial element."), argument(name = "to", description = "Final element.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayReplace { signature: Signature, aliases: Vec, @@ -164,7 +164,7 @@ impl ScalarUDFImpl for ArrayReplace { argument(name = "to", description = "Final element."), argument(name = "max", description = "Number of first occurrences to replace.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayReplaceN { signature: Signature, aliases: Vec, @@ -244,7 +244,7 @@ impl ScalarUDFImpl for ArrayReplaceN { argument(name = "from", description = "Initial element."), argument(name = "to", description = "Final element.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayReplaceAll { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/resize.rs b/datafusion/functions-nested/src/resize.rs index 145d7e80043b8..db27d649a467e 100644 --- a/datafusion/functions-nested/src/resize.rs +++ b/datafusion/functions-nested/src/resize.rs @@ -70,7 +70,7 @@ make_udf_expr_and_func!( description = "Defines new elements' value or empty if value is not set." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayResize { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/reverse.rs b/datafusion/functions-nested/src/reverse.rs index eb36047e09f64..5b134fe5b26a3 100644 --- a/datafusion/functions-nested/src/reverse.rs +++ b/datafusion/functions-nested/src/reverse.rs @@ -61,7 +61,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayReverse { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/set_ops.rs b/datafusion/functions-nested/src/set_ops.rs index 4f9457aa59c62..555767f8f070b 100644 --- a/datafusion/functions-nested/src/set_ops.rs +++ b/datafusion/functions-nested/src/set_ops.rs @@ -94,7 +94,7 @@ make_udf_expr_and_func!( description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayUnion { signature: Signature, aliases: Vec, @@ -185,7 +185,7 @@ impl ScalarUDFImpl for ArrayUnion { description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayIntersect { signature: Signature, aliases: Vec, @@ -260,7 +260,7 @@ impl ScalarUDFImpl for ArrayIntersect { description = "Array expression. Can be a constant, column, or function, and any combination of array operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct ArrayDistinct { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/sort.rs b/datafusion/functions-nested/src/sort.rs index 7b2f41c0541c6..3284ee58dd4e2 100644 --- a/datafusion/functions-nested/src/sort.rs +++ b/datafusion/functions-nested/src/sort.rs @@ -74,7 +74,7 @@ make_udf_expr_and_func!( description = "Whether to sort nulls first(`NULLS FIRST` or `NULLS LAST`)." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArraySort { signature: Signature, aliases: Vec, diff --git a/datafusion/functions-nested/src/string.rs b/datafusion/functions-nested/src/string.rs index d60d1a6e4de02..f7bcdb547f279 100644 --- a/datafusion/functions-nested/src/string.rs +++ b/datafusion/functions-nested/src/string.rs @@ -146,7 +146,7 @@ make_udf_expr_and_func!( description = "Optional. String to replace null values in the array. If not provided, nulls will be handled by default behavior." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrayToString { signature: Signature, aliases: Vec, @@ -242,7 +242,7 @@ make_udf_expr_and_func!( description = "Substring values to be replaced with `NULL`." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub(super) struct StringToArray { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/core/arrow_cast.rs b/datafusion/functions/src/core/arrow_cast.rs index e9dee09e74bf0..94a41ba4bb251 100644 --- a/datafusion/functions/src/core/arrow_cast.rs +++ b/datafusion/functions/src/core/arrow_cast.rs @@ -80,7 +80,7 @@ use datafusion_macros::user_doc; description = "[Arrow data type](https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html) name to cast to, as a string. The format is the same as that returned by [`arrow_typeof`]" ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrowCastFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/arrowtypeof.rs b/datafusion/functions/src/core/arrowtypeof.rs index 2509ed246ac7c..f178890f93704 100644 --- a/datafusion/functions/src/core/arrowtypeof.rs +++ b/datafusion/functions/src/core/arrowtypeof.rs @@ -40,7 +40,7 @@ use std::any::Any; description = "Expression to evaluate. The expression can be a constant, column, or function, and any combination of operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ArrowTypeOfFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/coalesce.rs b/datafusion/functions/src/core/coalesce.rs index 12a4bef247393..b0f3483513edb 100644 --- a/datafusion/functions/src/core/coalesce.rs +++ b/datafusion/functions/src/core/coalesce.rs @@ -46,7 +46,7 @@ use std::any::Any; description = "Expression to use if previous expressions are _null_. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct CoalesceFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/getfield.rs b/datafusion/functions/src/core/getfield.rs index 2f39132871bb5..98b818916995c 100644 --- a/datafusion/functions/src/core/getfield.rs +++ b/datafusion/functions/src/core/getfield.rs @@ -75,7 +75,7 @@ use std::sync::Arc; description = "The field name in the map or struct to retrieve data for. Must evaluate to a string." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct GetFieldFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/greatest.rs b/datafusion/functions/src/core/greatest.rs index 2d7ad2be3986f..6afc5b25512f4 100644 --- a/datafusion/functions/src/core/greatest.rs +++ b/datafusion/functions/src/core/greatest.rs @@ -53,7 +53,7 @@ const SORT_OPTIONS: SortOptions = SortOptions { description = "Expressions to compare and return the greatest value.. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct GreatestFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/least.rs b/datafusion/functions/src/core/least.rs index 662dac3e699fb..31cdf54441117 100644 --- a/datafusion/functions/src/core/least.rs +++ b/datafusion/functions/src/core/least.rs @@ -53,7 +53,7 @@ const SORT_OPTIONS: SortOptions = SortOptions { description = "Expressions to compare and return the smallest value. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LeastFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index 115f4a8aba225..c04074b2a8c87 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -58,7 +58,7 @@ a struct type of fields `field_a` and `field_b`: description = "Expression to include in the output struct. Can be a constant, column, or function, and any combination of arithmetic or string operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct NamedStructFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/nullif.rs b/datafusion/functions/src/core/nullif.rs index ee29714da16b6..be2dd0d2ca160 100644 --- a/datafusion/functions/src/core/nullif.rs +++ b/datafusion/functions/src/core/nullif.rs @@ -53,7 +53,7 @@ This can be used to perform the inverse operation of [`coalesce`](#coalesce).", description = "Expression to compare to expression1. Can be a constant, column, or function, and any combination of operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct NullIfFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/nvl.rs b/datafusion/functions/src/core/nvl.rs index 82d367072a256..c8b34c4b17800 100644 --- a/datafusion/functions/src/core/nvl.rs +++ b/datafusion/functions/src/core/nvl.rs @@ -55,7 +55,7 @@ use std::sync::Arc; description = "Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct NVLFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/core/nvl2.rs b/datafusion/functions/src/core/nvl2.rs index d20b01e29fba8..0f55bddcc91a8 100644 --- a/datafusion/functions/src/core/nvl2.rs +++ b/datafusion/functions/src/core/nvl2.rs @@ -59,7 +59,7 @@ use std::sync::Arc; description = "Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct NVL2Func { signature: Signature, } diff --git a/datafusion/functions/src/core/overlay.rs b/datafusion/functions/src/core/overlay.rs index 0ea5359e9621d..165bc571afe09 100644 --- a/datafusion/functions/src/core/overlay.rs +++ b/datafusion/functions/src/core/overlay.rs @@ -53,7 +53,7 @@ use datafusion_macros::user_doc; description = "The count of characters to be replaced from start position of str. If not specified, will use substr length instead." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct OverlayFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/struct.rs b/datafusion/functions/src/core/struct.rs index f068fc18a8b04..32c7af80e397f 100644 --- a/datafusion/functions/src/core/struct.rs +++ b/datafusion/functions/src/core/struct.rs @@ -64,7 +64,7 @@ select struct(a as field_a, b) from t; description = "Expression to include in the output struct. Can be a constant, column, or function, any combination of arithmetic or string operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct StructFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/core/union_extract.rs b/datafusion/functions/src/core/union_extract.rs index a3d1ec82ffbb0..a71e2e87388d5 100644 --- a/datafusion/functions/src/core/union_extract.rs +++ b/datafusion/functions/src/core/union_extract.rs @@ -49,7 +49,7 @@ use datafusion_macros::user_doc; description = "String expression to operate on. Must be a constant." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct UnionExtractFun { signature: Signature, } diff --git a/datafusion/functions/src/core/union_tag.rs b/datafusion/functions/src/core/union_tag.rs index 5d589d2167cbc..3e6370fa0110c 100644 --- a/datafusion/functions/src/core/union_tag.rs +++ b/datafusion/functions/src/core/union_tag.rs @@ -43,7 +43,7 @@ use std::sync::Arc; ```"#, standard_argument(name = "union", prefix = "Union") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct UnionTagFunc { signature: Signature, } diff --git a/datafusion/functions/src/core/version.rs b/datafusion/functions/src/core/version.rs index d68dbfc546ea5..ef3c5aafa4801 100644 --- a/datafusion/functions/src/core/version.rs +++ b/datafusion/functions/src/core/version.rs @@ -39,7 +39,7 @@ use std::any::Any; +--------------------------------------------+ ```"# )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct VersionFunc { signature: Signature, } diff --git a/datafusion/functions/src/crypto/digest.rs b/datafusion/functions/src/crypto/digest.rs index 2840006169be4..a4999f72f8d56 100644 --- a/datafusion/functions/src/crypto/digest.rs +++ b/datafusion/functions/src/crypto/digest.rs @@ -56,7 +56,7 @@ use std::any::Any; - blake3" ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct DigestFunc { signature: Signature, } diff --git a/datafusion/functions/src/crypto/md5.rs b/datafusion/functions/src/crypto/md5.rs index e209ed06e28bd..88859fdee34a7 100644 --- a/datafusion/functions/src/crypto/md5.rs +++ b/datafusion/functions/src/crypto/md5.rs @@ -45,7 +45,7 @@ use std::any::Any; ```"#, standard_argument(name = "expression", prefix = "String") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct Md5Func { signature: Signature, } diff --git a/datafusion/functions/src/crypto/sha224.rs b/datafusion/functions/src/crypto/sha224.rs index a64a3ef803197..69b79cce72c4e 100644 --- a/datafusion/functions/src/crypto/sha224.rs +++ b/datafusion/functions/src/crypto/sha224.rs @@ -44,7 +44,7 @@ use std::any::Any; ```"#, standard_argument(name = "expression", prefix = "String") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SHA224Func { signature: Signature, } diff --git a/datafusion/functions/src/crypto/sha256.rs b/datafusion/functions/src/crypto/sha256.rs index 94f3ea3b49fa6..9a948ba50c9e1 100644 --- a/datafusion/functions/src/crypto/sha256.rs +++ b/datafusion/functions/src/crypto/sha256.rs @@ -44,7 +44,7 @@ use std::any::Any; ```"#, standard_argument(name = "expression", prefix = "String") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SHA256Func { signature: Signature, } diff --git a/datafusion/functions/src/crypto/sha384.rs b/datafusion/functions/src/crypto/sha384.rs index 023730469c7bd..9e363cf883d29 100644 --- a/datafusion/functions/src/crypto/sha384.rs +++ b/datafusion/functions/src/crypto/sha384.rs @@ -44,7 +44,7 @@ use std::any::Any; ```"#, standard_argument(name = "expression", prefix = "String") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SHA384Func { signature: Signature, } diff --git a/datafusion/functions/src/crypto/sha512.rs b/datafusion/functions/src/crypto/sha512.rs index f48737e5751f0..a185698ca46ff 100644 --- a/datafusion/functions/src/crypto/sha512.rs +++ b/datafusion/functions/src/crypto/sha512.rs @@ -44,7 +44,7 @@ use std::any::Any; ```"#, standard_argument(name = "expression", prefix = "String") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SHA512Func { signature: Signature, } diff --git a/datafusion/functions/src/datetime/current_date.rs b/datafusion/functions/src/datetime/current_date.rs index 2bda1f262abe0..d7239e93dee0c 100644 --- a/datafusion/functions/src/datetime/current_date.rs +++ b/datafusion/functions/src/datetime/current_date.rs @@ -37,7 +37,7 @@ The `current_date()` return value is determined at query time and will return th "#, syntax_example = "current_date()" )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct CurrentDateFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/datetime/current_time.rs b/datafusion/functions/src/datetime/current_time.rs index 9b9d3997e9d75..79d5bfc1783c1 100644 --- a/datafusion/functions/src/datetime/current_time.rs +++ b/datafusion/functions/src/datetime/current_time.rs @@ -36,7 +36,7 @@ The `current_time()` return value is determined at query time and will return th "#, syntax_example = "current_time()" )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct CurrentTimeFunc { signature: Signature, } diff --git a/datafusion/functions/src/datetime/date_bin.rs b/datafusion/functions/src/datetime/date_bin.rs index d71bf31f95b9a..74e286de0f584 100644 --- a/datafusion/functions/src/datetime/date_bin.rs +++ b/datafusion/functions/src/datetime/date_bin.rs @@ -95,7 +95,7 @@ FROM VALUES ('2023-01-01T18:18:18Z'), ('2023-01-03T19:00:03Z') t(time); "# ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct DateBinFunc { signature: Signature, } diff --git a/datafusion/functions/src/datetime/date_part.rs b/datafusion/functions/src/datetime/date_part.rs index 6b844b7cf39cc..aa23a5028dd81 100644 --- a/datafusion/functions/src/datetime/date_part.rs +++ b/datafusion/functions/src/datetime/date_part.rs @@ -79,7 +79,7 @@ use datafusion_macros::user_doc; description = "Time expression to operate on. Can be a constant, column, or function." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct DatePartFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/datetime/date_trunc.rs b/datafusion/functions/src/datetime/date_trunc.rs index 05e8ab74936fe..4165014f22bc8 100644 --- a/datafusion/functions/src/datetime/date_trunc.rs +++ b/datafusion/functions/src/datetime/date_trunc.rs @@ -69,7 +69,7 @@ use chrono::{ description = "Time expression to operate on. Can be a constant, column, or function." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct DateTruncFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/datetime/from_unixtime.rs b/datafusion/functions/src/datetime/from_unixtime.rs index 16eea0be8be61..ed238f0074af6 100644 --- a/datafusion/functions/src/datetime/from_unixtime.rs +++ b/datafusion/functions/src/datetime/from_unixtime.rs @@ -46,7 +46,7 @@ use datafusion_macros::user_doc; description = "Optional timezone to use when converting the integer to a timestamp. If not provided, the default timezone is UTC." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct FromUnixtimeFunc { signature: Signature, } diff --git a/datafusion/functions/src/datetime/make_date.rs b/datafusion/functions/src/datetime/make_date.rs index 677b54cd15f00..053e3db46b6ec 100644 --- a/datafusion/functions/src/datetime/make_date.rs +++ b/datafusion/functions/src/datetime/make_date.rs @@ -66,7 +66,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo description = "Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct MakeDateFunc { signature: Signature, } diff --git a/datafusion/functions/src/datetime/now.rs b/datafusion/functions/src/datetime/now.rs index ffb3aed5a9606..65dadb42a89e1 100644 --- a/datafusion/functions/src/datetime/now.rs +++ b/datafusion/functions/src/datetime/now.rs @@ -37,7 +37,7 @@ The `now()` return value is determined at query time and will return the same ti "#, syntax_example = "now()" )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct NowFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index 2f7e5fa56eb13..ece407e7ceb25 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -63,7 +63,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo description = "Day to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToCharFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/datetime/to_date.rs b/datafusion/functions/src/datetime/to_date.rs index d1b2720867d2f..fd907c03083b6 100644 --- a/datafusion/functions/src/datetime/to_date.rs +++ b/datafusion/functions/src/datetime/to_date.rs @@ -63,7 +63,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo an error will be returned." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToDateFunc { signature: Signature, } diff --git a/datafusion/functions/src/datetime/to_local_time.rs b/datafusion/functions/src/datetime/to_local_time.rs index 1f1c300f2521d..d0a457ba8fc73 100644 --- a/datafusion/functions/src/datetime/to_local_time.rs +++ b/datafusion/functions/src/datetime/to_local_time.rs @@ -96,7 +96,7 @@ FROM ( description = "Time expression to operate on. Can be a constant, column, or function." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToLocalTimeFunc { signature: Signature, } diff --git a/datafusion/functions/src/datetime/to_timestamp.rs b/datafusion/functions/src/datetime/to_timestamp.rs index 9bd94c8ca8d94..265cd485a2ddb 100644 --- a/datafusion/functions/src/datetime/to_timestamp.rs +++ b/datafusion/functions/src/datetime/to_timestamp.rs @@ -66,7 +66,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToTimestampFunc { signature: Signature, } @@ -100,7 +100,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToTimestampSecondsFunc { signature: Signature, } @@ -134,7 +134,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToTimestampMillisFunc { signature: Signature, } @@ -168,7 +168,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToTimestampMicrosFunc { signature: Signature, } @@ -202,7 +202,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToTimestampNanosFunc { signature: Signature, } diff --git a/datafusion/functions/src/datetime/to_unixtime.rs b/datafusion/functions/src/datetime/to_unixtime.rs index 653ec10851695..13c73815f6cda 100644 --- a/datafusion/functions/src/datetime/to_unixtime.rs +++ b/datafusion/functions/src/datetime/to_unixtime.rs @@ -54,7 +54,7 @@ use std::any::Any; description = "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order they appear with the first successful one being returned. If none of the formats successfully parse the expression an error will be returned." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToUnixtimeFunc { signature: Signature, } diff --git a/datafusion/functions/src/encoding/inner.rs b/datafusion/functions/src/encoding/inner.rs index 9a7b49105743e..721088c58e06a 100644 --- a/datafusion/functions/src/encoding/inner.rs +++ b/datafusion/functions/src/encoding/inner.rs @@ -54,7 +54,7 @@ use std::any::Any; ), related_udf(name = "decode") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct EncodeFunc { signature: Signature, } @@ -147,7 +147,7 @@ impl ScalarUDFImpl for EncodeFunc { argument(name = "format", description = "Same arguments as [encode](#encode)"), related_udf(name = "encode") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct DecodeFunc { signature: Signature, } diff --git a/datafusion/functions/src/macros.rs b/datafusion/functions/src/macros.rs index 03fad2f25e370..1972535325a92 100644 --- a/datafusion/functions/src/macros.rs +++ b/datafusion/functions/src/macros.rs @@ -171,7 +171,7 @@ macro_rules! make_math_unary_udf { Signature, Volatility, }; - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] pub struct $UDF { signature: Signature, } @@ -285,7 +285,7 @@ macro_rules! make_math_binary_udf { Signature, Volatility, }; - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] pub struct $UDF { signature: Signature, } diff --git a/datafusion/functions/src/math/abs.rs b/datafusion/functions/src/math/abs.rs index 0c686a59016ac..45c32e660e35d 100644 --- a/datafusion/functions/src/math/abs.rs +++ b/datafusion/functions/src/math/abs.rs @@ -112,7 +112,7 @@ fn create_abs_function(input_data_type: &DataType) -> Result syntax_example = "abs(numeric_expression)", standard_argument(name = "numeric_expression", prefix = "Numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct AbsFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/cot.rs b/datafusion/functions/src/math/cot.rs index 4e56212ddbee8..8006be2effea8 100644 --- a/datafusion/functions/src/math/cot.rs +++ b/datafusion/functions/src/math/cot.rs @@ -34,7 +34,7 @@ use datafusion_macros::user_doc; syntax_example = r#"cot(numeric_expression)"#, standard_argument(name = "numeric_expression", prefix = "Numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct CotFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/factorial.rs b/datafusion/functions/src/math/factorial.rs index c2ac21b78f212..bd95f5b47b409 100644 --- a/datafusion/functions/src/math/factorial.rs +++ b/datafusion/functions/src/math/factorial.rs @@ -41,7 +41,7 @@ use datafusion_macros::user_doc; syntax_example = "factorial(numeric_expression)", standard_argument(name = "numeric_expression", prefix = "Numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct FactorialFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/gcd.rs b/datafusion/functions/src/math/gcd.rs index 7fe253b4afbc0..714718c5e87a0 100644 --- a/datafusion/functions/src/math/gcd.rs +++ b/datafusion/functions/src/math/gcd.rs @@ -37,7 +37,7 @@ use datafusion_macros::user_doc; standard_argument(name = "expression_x", prefix = "First numeric"), standard_argument(name = "expression_y", prefix = "Second numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct GcdFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/iszero.rs b/datafusion/functions/src/math/iszero.rs index bc12dfb7898e8..ec1200f443eb7 100644 --- a/datafusion/functions/src/math/iszero.rs +++ b/datafusion/functions/src/math/iszero.rs @@ -38,7 +38,7 @@ use crate::utils::make_scalar_function; syntax_example = "iszero(numeric_expression)", standard_argument(name = "numeric_expression", prefix = "Numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct IsZeroFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/lcm.rs b/datafusion/functions/src/math/lcm.rs index fc6bf9461f283..ca7dbb5856ca1 100644 --- a/datafusion/functions/src/math/lcm.rs +++ b/datafusion/functions/src/math/lcm.rs @@ -42,7 +42,7 @@ use crate::utils::make_scalar_function; standard_argument(name = "expression_x", prefix = "First numeric"), standard_argument(name = "expression_y", prefix = "Second numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LcmFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index efa9a4c89b310..6342501a86528 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -45,7 +45,7 @@ log(numeric_expression)"#, standard_argument(name = "base", prefix = "Base numeric"), standard_argument(name = "numeric_expression", prefix = "Numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LogFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/nans.rs b/datafusion/functions/src/math/nans.rs index 34a5c2a1c16bb..ef9b2eff20d44 100644 --- a/datafusion/functions/src/math/nans.rs +++ b/datafusion/functions/src/math/nans.rs @@ -33,7 +33,7 @@ use std::sync::Arc; syntax_example = "isnan(numeric_expression)", standard_argument(name = "numeric_expression", prefix = "Numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct IsNanFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/nanvl.rs b/datafusion/functions/src/math/nanvl.rs index 9effb82896ee0..3d05a03e5aa64 100644 --- a/datafusion/functions/src/math/nanvl.rs +++ b/datafusion/functions/src/math/nanvl.rs @@ -45,7 +45,7 @@ Returns the second argument otherwise."#, description = "Numeric expression to return if the first expression is _NaN_. Can be a constant, column, or function, and any combination of arithmetic operators." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct NanvlFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/pi.rs b/datafusion/functions/src/math/pi.rs index 5339a9b14a283..71a8e21a52f26 100644 --- a/datafusion/functions/src/math/pi.rs +++ b/datafusion/functions/src/math/pi.rs @@ -32,7 +32,7 @@ use datafusion_macros::user_doc; description = "Returns an approximate value of π.", syntax_example = "pi()" )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct PiFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/power.rs b/datafusion/functions/src/math/power.rs index 87f27f5a793c4..73325de4cf8b5 100644 --- a/datafusion/functions/src/math/power.rs +++ b/datafusion/functions/src/math/power.rs @@ -42,7 +42,7 @@ use datafusion_macros::user_doc; standard_argument(name = "base", prefix = "Numeric"), standard_argument(name = "exponent", prefix = "Exponent numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct PowerFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/math/random.rs b/datafusion/functions/src/math/random.rs index 92b6ed1895edd..f9c4b198d1dd5 100644 --- a/datafusion/functions/src/math/random.rs +++ b/datafusion/functions/src/math/random.rs @@ -34,7 +34,7 @@ use datafusion_macros::user_doc; The random seed is unique to each row."#, syntax_example = "random()" )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RandomFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/round.rs b/datafusion/functions/src/math/round.rs index fc87b7e63a62f..e13d6b8f9a527 100644 --- a/datafusion/functions/src/math/round.rs +++ b/datafusion/functions/src/math/round.rs @@ -43,7 +43,7 @@ use datafusion_macros::user_doc; description = "Optional. The number of decimal places to round to. Defaults to 0." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RoundFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/signum.rs b/datafusion/functions/src/math/signum.rs index 71d32413afc83..73931f303b867 100644 --- a/datafusion/functions/src/math/signum.rs +++ b/datafusion/functions/src/math/signum.rs @@ -40,7 +40,7 @@ Zero and positive numbers return `1`."#, syntax_example = "signum(numeric_expression)", standard_argument(name = "numeric_expression", prefix = "Numeric") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SignumFunc { signature: Signature, } diff --git a/datafusion/functions/src/math/trunc.rs b/datafusion/functions/src/math/trunc.rs index 2ac291204a0bc..c8e2de502b26c 100644 --- a/datafusion/functions/src/math/trunc.rs +++ b/datafusion/functions/src/math/trunc.rs @@ -47,7 +47,7 @@ use datafusion_macros::user_doc; integer, replaces digits to the left of the decimal point with `0`."# ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct TruncFunc { signature: Signature, } diff --git a/datafusion/functions/src/regex/regexpcount.rs b/datafusion/functions/src/regex/regexpcount.rs index a069455281bdb..ea16dbad3a24d 100644 --- a/datafusion/functions/src/regex/regexpcount.rs +++ b/datafusion/functions/src/regex/regexpcount.rs @@ -61,7 +61,7 @@ use std::sync::Arc; - **U**: swap the meaning of x* and x*?"# ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RegexpCountFunc { signature: Signature, } diff --git a/datafusion/functions/src/regex/regexpinstr.rs b/datafusion/functions/src/regex/regexpinstr.rs index 577a8f5bc33d7..851c182a90dd0 100644 --- a/datafusion/functions/src/regex/regexpinstr.rs +++ b/datafusion/functions/src/regex/regexpinstr.rs @@ -72,7 +72,7 @@ use crate::regex::compile_and_cache_regex; description = "Optional Specifies which capture group (subexpression) to return the position for. Defaults to 0, which returns the position of the entire match." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RegexpInstrFunc { signature: Signature, } diff --git a/datafusion/functions/src/regex/regexplike.rs b/datafusion/functions/src/regex/regexplike.rs index 2080bb9fe818f..0554844d11c14 100644 --- a/datafusion/functions/src/regex/regexplike.rs +++ b/datafusion/functions/src/regex/regexplike.rs @@ -67,7 +67,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo - **U**: swap the meaning of x* and x*?"# ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RegexpLikeFunc { signature: Signature, } diff --git a/datafusion/functions/src/regex/regexpmatch.rs b/datafusion/functions/src/regex/regexpmatch.rs index 1119e66398d1d..ba52822a02f8c 100644 --- a/datafusion/functions/src/regex/regexpmatch.rs +++ b/datafusion/functions/src/regex/regexpmatch.rs @@ -66,7 +66,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo - **U**: swap the meaning of x* and x*?"# ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RegexpMatchFunc { signature: Signature, } diff --git a/datafusion/functions/src/regex/regexpreplace.rs b/datafusion/functions/src/regex/regexpreplace.rs index 3a83564ff11fe..39858119c89c9 100644 --- a/datafusion/functions/src/regex/regexpreplace.rs +++ b/datafusion/functions/src/regex/regexpreplace.rs @@ -82,7 +82,7 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo - **U**: swap the meaning of x* and x*?"# ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RegexpReplaceFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/ascii.rs b/datafusion/functions/src/string/ascii.rs index 63c987906b0f7..40ed71180f3f6 100644 --- a/datafusion/functions/src/string/ascii.rs +++ b/datafusion/functions/src/string/ascii.rs @@ -49,7 +49,7 @@ use std::sync::Arc; standard_argument(name = "str", prefix = "String"), related_udf(name = "chr") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct AsciiFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/bit_length.rs b/datafusion/functions/src/string/bit_length.rs index f8740aa4178b4..1578331e57f89 100644 --- a/datafusion/functions/src/string/bit_length.rs +++ b/datafusion/functions/src/string/bit_length.rs @@ -45,7 +45,7 @@ use datafusion_macros::user_doc; related_udf(name = "length"), related_udf(name = "octet_length") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct BitLengthFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/btrim.rs b/datafusion/functions/src/string/btrim.rs index 2f1711c9962ad..a7fbdb3c69213 100644 --- a/datafusion/functions/src/string/btrim.rs +++ b/datafusion/functions/src/string/btrim.rs @@ -65,7 +65,7 @@ fn btrim(args: &[ArrayRef]) -> Result { related_udf(name = "ltrim"), related_udf(name = "rtrim") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct BTrimFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/string/chr.rs b/datafusion/functions/src/string/chr.rs index 718f96e4f145b..21c6c92ee192c 100644 --- a/datafusion/functions/src/string/chr.rs +++ b/datafusion/functions/src/string/chr.rs @@ -90,7 +90,7 @@ pub fn chr(args: &[ArrayRef]) -> Result { standard_argument(name = "expression", prefix = "String"), related_udf(name = "ascii") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ChrFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/concat.rs b/datafusion/functions/src/string/concat.rs index 06ec82f1b3ed0..e322c757ef041 100644 --- a/datafusion/functions/src/string/concat.rs +++ b/datafusion/functions/src/string/concat.rs @@ -52,7 +52,7 @@ use datafusion_macros::user_doc; ), related_udf(name = "concat_ws") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ConcatFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/concat_ws.rs b/datafusion/functions/src/string/concat_ws.rs index c1ecac7ae99d9..cdd30ac8755ab 100644 --- a/datafusion/functions/src/string/concat_ws.rs +++ b/datafusion/functions/src/string/concat_ws.rs @@ -59,7 +59,7 @@ use datafusion_macros::user_doc; ), related_udf(name = "concat") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ConcatWsFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/contains.rs b/datafusion/functions/src/string/contains.rs index b2aefb8ee374a..6372131659e80 100644 --- a/datafusion/functions/src/string/contains.rs +++ b/datafusion/functions/src/string/contains.rs @@ -46,7 +46,7 @@ use std::sync::Arc; standard_argument(name = "str", prefix = "String"), argument(name = "search_str", description = "The string to search for in str.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ContainsFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/ends_with.rs b/datafusion/functions/src/string/ends_with.rs index eafc310236ee3..6090d9c84d4cd 100644 --- a/datafusion/functions/src/string/ends_with.rs +++ b/datafusion/functions/src/string/ends_with.rs @@ -52,7 +52,7 @@ use datafusion_macros::user_doc; standard_argument(name = "str", prefix = "String"), argument(name = "substr", description = "Substring to test for.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct EndsWithFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/levenshtein.rs b/datafusion/functions/src/string/levenshtein.rs index a1a486c7d3cf4..2f7894df903d6 100644 --- a/datafusion/functions/src/string/levenshtein.rs +++ b/datafusion/functions/src/string/levenshtein.rs @@ -57,7 +57,7 @@ use datafusion_macros::user_doc; description = "String expression to compute Levenshtein distance with str1." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LevenshteinFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/lower.rs b/datafusion/functions/src/string/lower.rs index 139275892933a..ee56a6a549857 100644 --- a/datafusion/functions/src/string/lower.rs +++ b/datafusion/functions/src/string/lower.rs @@ -44,7 +44,7 @@ use datafusion_macros::user_doc; related_udf(name = "initcap"), related_udf(name = "upper") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LowerFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/ltrim.rs b/datafusion/functions/src/string/ltrim.rs index 65849202efc66..dc6d30d38188c 100644 --- a/datafusion/functions/src/string/ltrim.rs +++ b/datafusion/functions/src/string/ltrim.rs @@ -71,7 +71,7 @@ fn ltrim(args: &[ArrayRef]) -> Result { related_udf(name = "btrim"), related_udf(name = "rtrim") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LtrimFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/octet_length.rs b/datafusion/functions/src/string/octet_length.rs index 17ea2726b071e..aa8257ef8fc53 100644 --- a/datafusion/functions/src/string/octet_length.rs +++ b/datafusion/functions/src/string/octet_length.rs @@ -45,7 +45,7 @@ use datafusion_macros::user_doc; related_udf(name = "bit_length"), related_udf(name = "length") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct OctetLengthFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/repeat.rs b/datafusion/functions/src/string/repeat.rs index 2d36cb8356a00..3f6128b6516b9 100644 --- a/datafusion/functions/src/string/repeat.rs +++ b/datafusion/functions/src/string/repeat.rs @@ -51,7 +51,7 @@ use datafusion_macros::user_doc; description = "Number of times to repeat the input string." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RepeatFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/replace.rs b/datafusion/functions/src/string/replace.rs index de70215c49c77..8a2020c5a01b2 100644 --- a/datafusion/functions/src/string/replace.rs +++ b/datafusion/functions/src/string/replace.rs @@ -52,7 +52,7 @@ use datafusion_macros::user_doc; ), standard_argument(name = "replacement", prefix = "Replacement substring") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ReplaceFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/rtrim.rs b/datafusion/functions/src/string/rtrim.rs index bb33274978daf..be0595f65542a 100644 --- a/datafusion/functions/src/string/rtrim.rs +++ b/datafusion/functions/src/string/rtrim.rs @@ -71,7 +71,7 @@ fn rtrim(args: &[ArrayRef]) -> Result { related_udf(name = "btrim"), related_udf(name = "ltrim") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RtrimFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/split_part.rs b/datafusion/functions/src/string/split_part.rs index 724d9c278cca5..8462dd5149cbf 100644 --- a/datafusion/functions/src/string/split_part.rs +++ b/datafusion/functions/src/string/split_part.rs @@ -47,7 +47,7 @@ use std::sync::Arc; argument(name = "delimiter", description = "String or character to split on."), argument(name = "pos", description = "Position of the part to return.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SplitPartFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/starts_with.rs b/datafusion/functions/src/string/starts_with.rs index ecab1af132e03..c4159cba86f34 100644 --- a/datafusion/functions/src/string/starts_with.rs +++ b/datafusion/functions/src/string/starts_with.rs @@ -74,7 +74,7 @@ fn starts_with(args: &[ArrayRef]) -> Result { standard_argument(name = "str", prefix = "String"), argument(name = "substr", description = "Substring to test for.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct StartsWithFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/to_hex.rs b/datafusion/functions/src/string/to_hex.rs index a739a4dfb20ec..26be0066c2df3 100644 --- a/datafusion/functions/src/string/to_hex.rs +++ b/datafusion/functions/src/string/to_hex.rs @@ -87,7 +87,7 @@ where ```"#, standard_argument(name = "int", prefix = "Integer") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ToHexFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/upper.rs b/datafusion/functions/src/string/upper.rs index 99d505c5aafda..8bb2ec1d511cd 100644 --- a/datafusion/functions/src/string/upper.rs +++ b/datafusion/functions/src/string/upper.rs @@ -43,7 +43,7 @@ use std::any::Any; related_udf(name = "initcap"), related_udf(name = "lower") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct UpperFunc { signature: Signature, } diff --git a/datafusion/functions/src/string/uuid.rs b/datafusion/functions/src/string/uuid.rs index 29415a9b20805..a5ad6db5354f3 100644 --- a/datafusion/functions/src/string/uuid.rs +++ b/datafusion/functions/src/string/uuid.rs @@ -42,7 +42,7 @@ use datafusion_macros::user_doc; +--------------------------------------+ ```"# )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct UuidFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/character_length.rs b/datafusion/functions/src/unicode/character_length.rs index c14a92ffdd750..85fe0956a951b 100644 --- a/datafusion/functions/src/unicode/character_length.rs +++ b/datafusion/functions/src/unicode/character_length.rs @@ -45,7 +45,7 @@ use std::sync::Arc; related_udf(name = "bit_length"), related_udf(name = "octet_length") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct CharacterLengthFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/unicode/find_in_set.rs b/datafusion/functions/src/unicode/find_in_set.rs index 3429a8293c154..fa68e539600b0 100644 --- a/datafusion/functions/src/unicode/find_in_set.rs +++ b/datafusion/functions/src/unicode/find_in_set.rs @@ -53,7 +53,7 @@ use datafusion_macros::user_doc; description = "A string list is a string composed of substrings separated by , characters." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct FindInSetFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/initcap.rs b/datafusion/functions/src/unicode/initcap.rs index b70c00e757ee7..62862fbe78980 100644 --- a/datafusion/functions/src/unicode/initcap.rs +++ b/datafusion/functions/src/unicode/initcap.rs @@ -50,7 +50,7 @@ use datafusion_macros::user_doc; related_udf(name = "lower"), related_udf(name = "upper") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct InitcapFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/left.rs b/datafusion/functions/src/unicode/left.rs index f99f0de67ebb2..fceb2a131a2b0 100644 --- a/datafusion/functions/src/unicode/left.rs +++ b/datafusion/functions/src/unicode/left.rs @@ -53,7 +53,7 @@ use datafusion_macros::user_doc; argument(name = "n", description = "Number of characters to return."), related_udf(name = "right") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LeftFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/lpad.rs b/datafusion/functions/src/unicode/lpad.rs index 3c149cc68a538..621dbd4970f26 100644 --- a/datafusion/functions/src/unicode/lpad.rs +++ b/datafusion/functions/src/unicode/lpad.rs @@ -56,7 +56,7 @@ use datafusion_macros::user_doc; ), related_udf(name = "rpad") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct LPadFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/reverse.rs b/datafusion/functions/src/unicode/reverse.rs index 311e9e81a8be9..500e762ec250b 100644 --- a/datafusion/functions/src/unicode/reverse.rs +++ b/datafusion/functions/src/unicode/reverse.rs @@ -44,7 +44,7 @@ use DataType::{LargeUtf8, Utf8, Utf8View}; ```"#, standard_argument(name = "str", prefix = "String") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ReverseFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/right.rs b/datafusion/functions/src/unicode/right.rs index 1ceaf69983311..c492f606e9c5b 100644 --- a/datafusion/functions/src/unicode/right.rs +++ b/datafusion/functions/src/unicode/right.rs @@ -53,7 +53,7 @@ use datafusion_macros::user_doc; argument(name = "n", description = "Number of characters to return."), related_udf(name = "left") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RightFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/rpad.rs b/datafusion/functions/src/unicode/rpad.rs index c68c4d329c74d..6ec78b07980b8 100644 --- a/datafusion/functions/src/unicode/rpad.rs +++ b/datafusion/functions/src/unicode/rpad.rs @@ -55,7 +55,7 @@ use DataType::{LargeUtf8, Utf8, Utf8View}; ), related_udf(name = "lpad") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct RPadFunc { signature: Signature, } diff --git a/datafusion/functions/src/unicode/strpos.rs b/datafusion/functions/src/unicode/strpos.rs index 1c81b46ec78ea..4f238b2644bdf 100644 --- a/datafusion/functions/src/unicode/strpos.rs +++ b/datafusion/functions/src/unicode/strpos.rs @@ -49,7 +49,7 @@ use datafusion_macros::user_doc; standard_argument(name = "str", prefix = "String"), argument(name = "substr", description = "Substring expression to search for.") )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct StrposFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index 4dcbea4807f44..0b35f664532d4 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -56,7 +56,7 @@ use datafusion_macros::user_doc; description = "Number of characters to extract. If not specified, returns the rest of the string after the start position." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SubstrFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/unicode/substrindex.rs b/datafusion/functions/src/unicode/substrindex.rs index 9a18b5d23c5ee..a7ee7388f9013 100644 --- a/datafusion/functions/src/unicode/substrindex.rs +++ b/datafusion/functions/src/unicode/substrindex.rs @@ -62,7 +62,7 @@ If count is negative, everything to the right of the final delimiter (counting f description = "The number of times to search for the delimiter. Can be either a positive or negative number." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SubstrIndexFunc { signature: Signature, aliases: Vec, diff --git a/datafusion/functions/src/unicode/translate.rs b/datafusion/functions/src/unicode/translate.rs index 8b4894643a7a3..911b8d311996e 100644 --- a/datafusion/functions/src/unicode/translate.rs +++ b/datafusion/functions/src/unicode/translate.rs @@ -52,7 +52,7 @@ use datafusion_macros::user_doc; description = "Translation characters. Translation characters replace only characters at the same position in the **chars** string." ) )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct TranslateFunc { signature: Signature, } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 3ab8b9d26ae4a..da19e21e271c8 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -1511,7 +1511,7 @@ mod test { ) } - #[derive(Debug, Clone)] + #[derive(Debug, PartialEq, Eq, Hash)] struct TestScalarUDF { signature: Signature, } diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 88d51e1adea34..d78819c7c315f 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -1646,7 +1646,7 @@ mod test { Ok(()) } - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] pub struct TestUdf { signature: Signature, } @@ -1773,7 +1773,7 @@ mod test { ScalarUDF::new_from_impl(RandomStub::new()) } - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct RandomStub { signature: Signature, } diff --git a/datafusion/optimizer/src/eliminate_group_by_constant.rs b/datafusion/optimizer/src/eliminate_group_by_constant.rs index 9c47ce024f91a..4e16fc0aa159c 100644 --- a/datafusion/optimizer/src/eliminate_group_by_constant.rs +++ b/datafusion/optimizer/src/eliminate_group_by_constant.rs @@ -147,7 +147,7 @@ mod tests { }}; } - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct ScalarUDFMock { signature: Signature, } diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 35ec7d074d5f4..0112d0f7a8ddd 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -3856,7 +3856,7 @@ mod tests { ) } - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct TestScalarUDF { signature: Signature, } diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index a87591b8d96bf..83b50509cd584 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -4384,7 +4384,7 @@ mod tests { unimplemented!("not needed for tests") } } - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct VolatileUdf { signature: Signature, } diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 5cfbf13a25cf0..ea07fc714991f 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -282,7 +282,7 @@ pub(crate) mod tests { use petgraph::visit::Bfs; - #[derive(Debug, Clone)] + #[derive(Debug, PartialEq, Eq, Hash)] pub struct TestScalarUDF { pub(crate) signature: Signature, } diff --git a/datafusion/proto/tests/cases/mod.rs b/datafusion/proto/tests/cases/mod.rs index eb517226cfb5f..1c52a872df13f 100644 --- a/datafusion/proto/tests/cases/mod.rs +++ b/datafusion/proto/tests/cases/mod.rs @@ -20,8 +20,8 @@ use datafusion::logical_expr::ColumnarValue; use datafusion_common::plan_err; use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::{ - udf_equals_hash, Accumulator, AggregateUDFImpl, PartitionEvaluator, - ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, WindowUDFImpl, + Accumulator, AggregateUDFImpl, PartitionEvaluator, ScalarFunctionArgs, ScalarUDFImpl, + Signature, Volatility, WindowUDFImpl, }; use datafusion_functions_window_common::field::WindowUDFFieldArgs; use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; @@ -81,8 +81,6 @@ impl ScalarUDFImpl for MyRegexUdf { fn aliases(&self) -> &[String] { &self.aliases } - - udf_equals_hash!(ScalarUDFImpl); } #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/spark/src/function/array/spark_array.rs b/datafusion/spark/src/function/array/spark_array.rs index f2b6e0d510b0d..9ca6a8a3d1c79 100644 --- a/datafusion/spark/src/function/array/spark_array.rs +++ b/datafusion/spark/src/function/array/spark_array.rs @@ -35,7 +35,7 @@ use crate::function::functions_nested_utils::make_scalar_function; const ARRAY_FIELD_DEFAULT_NAME: &str = "element"; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkArray { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/bitwise/bit_count.rs b/datafusion/spark/src/function/bitwise/bit_count.rs index 5605e80c377b4..73566b9dbdf43 100644 --- a/datafusion/spark/src/function/bitwise/bit_count.rs +++ b/datafusion/spark/src/function/bitwise/bit_count.rs @@ -30,7 +30,7 @@ use datafusion_expr::{ }; use datafusion_functions::utils::make_scalar_function; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkBitCount { signature: Signature, } diff --git a/datafusion/spark/src/function/bitwise/bit_get.rs b/datafusion/spark/src/function/bitwise/bit_get.rs index cd68c538f5e14..89803b6f503f2 100644 --- a/datafusion/spark/src/function/bitwise/bit_get.rs +++ b/datafusion/spark/src/function/bitwise/bit_get.rs @@ -38,7 +38,7 @@ use crate::function::error_utils::{ invalid_arg_count_exec_err, unsupported_data_type_exec_err, }; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkBitGet { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/datetime/date_add.rs b/datafusion/spark/src/function/datetime/date_add.rs index 817faed5a665f..58633e15f9628 100644 --- a/datafusion/spark/src/function/datetime/date_add.rs +++ b/datafusion/spark/src/function/datetime/date_add.rs @@ -31,7 +31,7 @@ use datafusion_expr::{ }; use datafusion_functions::utils::make_scalar_function; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkDateAdd { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/datetime/date_sub.rs b/datafusion/spark/src/function/datetime/date_sub.rs index d350e94d803c7..aa10c05b8a998 100644 --- a/datafusion/spark/src/function/datetime/date_sub.rs +++ b/datafusion/spark/src/function/datetime/date_sub.rs @@ -31,7 +31,7 @@ use datafusion_expr::{ }; use datafusion_functions::utils::make_scalar_function; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkDateSub { signature: Signature, } diff --git a/datafusion/spark/src/function/datetime/last_day.rs b/datafusion/spark/src/function/datetime/last_day.rs index 5a748816f40d9..c01a6403649c5 100644 --- a/datafusion/spark/src/function/datetime/last_day.rs +++ b/datafusion/spark/src/function/datetime/last_day.rs @@ -26,7 +26,7 @@ use datafusion_expr::{ ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Signature, Volatility, }; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkLastDay { signature: Signature, } diff --git a/datafusion/spark/src/function/datetime/next_day.rs b/datafusion/spark/src/function/datetime/next_day.rs index 6d7f62ea3aa44..32739f3e2c591 100644 --- a/datafusion/spark/src/function/datetime/next_day.rs +++ b/datafusion/spark/src/function/datetime/next_day.rs @@ -27,7 +27,7 @@ use datafusion_expr::{ }; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkNextDay { signature: Signature, } diff --git a/datafusion/spark/src/function/hash/crc32.rs b/datafusion/spark/src/function/hash/crc32.rs index ad23380459f57..76e31d12c6487 100644 --- a/datafusion/spark/src/function/hash/crc32.rs +++ b/datafusion/spark/src/function/hash/crc32.rs @@ -31,7 +31,7 @@ use datafusion_expr::{ use datafusion_functions::utils::make_scalar_function; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkCrc32 { signature: Signature, } diff --git a/datafusion/spark/src/function/hash/sha1.rs b/datafusion/spark/src/function/hash/sha1.rs index 8c635d8dc9be6..25cbdd4453505 100644 --- a/datafusion/spark/src/function/hash/sha1.rs +++ b/datafusion/spark/src/function/hash/sha1.rs @@ -32,7 +32,7 @@ use datafusion_functions::utils::make_scalar_function; use sha1::{Digest, Sha1}; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkSha1 { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/hash/sha2.rs b/datafusion/spark/src/function/hash/sha2.rs index a8bb8c21a2a4f..b006607d3eeda 100644 --- a/datafusion/spark/src/function/hash/sha2.rs +++ b/datafusion/spark/src/function/hash/sha2.rs @@ -31,7 +31,7 @@ use std::any::Any; use std::sync::Arc; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkSha2 { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/math/expm1.rs b/datafusion/spark/src/function/math/expm1.rs index 3a3a0c3835d37..42eccf3a2431a 100644 --- a/datafusion/spark/src/function/math/expm1.rs +++ b/datafusion/spark/src/function/math/expm1.rs @@ -28,7 +28,7 @@ use std::any::Any; use std::sync::Arc; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkExpm1 { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/math/factorial.rs b/datafusion/spark/src/function/math/factorial.rs index 10f7f06964691..3addc9700ece9 100644 --- a/datafusion/spark/src/function/math/factorial.rs +++ b/datafusion/spark/src/function/math/factorial.rs @@ -27,7 +27,7 @@ use datafusion_expr::Signature; use datafusion_expr::{ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Volatility}; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkFactorial { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/math/hex.rs b/datafusion/spark/src/function/math/hex.rs index e5a72ea1df288..0cdf4f3a9a77e 100644 --- a/datafusion/spark/src/function/math/hex.rs +++ b/datafusion/spark/src/function/math/hex.rs @@ -37,7 +37,7 @@ use datafusion_expr::{ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl, Volatili use std::fmt::Write; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkHex { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/math/modulus.rs b/datafusion/spark/src/function/math/modulus.rs index 70390f0a39c8b..146cd27898409 100644 --- a/datafusion/spark/src/function/math/modulus.rs +++ b/datafusion/spark/src/function/math/modulus.rs @@ -58,7 +58,7 @@ pub fn spark_pmod(args: &[ColumnarValue]) -> Result { } /// SparkMod implements the Spark-compatible modulo function -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkMod { signature: Signature, } @@ -108,7 +108,7 @@ impl ScalarUDFImpl for SparkMod { } /// SparkMod implements the Spark-compatible modulo function -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkPmod { signature: Signature, } diff --git a/datafusion/spark/src/function/math/rint.rs b/datafusion/spark/src/function/math/rint.rs index 6c962d95054cd..9b61529c5bc44 100644 --- a/datafusion/spark/src/function/math/rint.rs +++ b/datafusion/spark/src/function/math/rint.rs @@ -31,7 +31,7 @@ use datafusion_expr::{ }; use datafusion_functions::utils::make_scalar_function; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkRint { signature: Signature, } diff --git a/datafusion/spark/src/function/string/ascii.rs b/datafusion/spark/src/function/string/ascii.rs index c05aa214ccc0c..bf66a19738a11 100644 --- a/datafusion/spark/src/function/string/ascii.rs +++ b/datafusion/spark/src/function/string/ascii.rs @@ -26,7 +26,7 @@ use std::any::Any; use std::sync::Arc; /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkAscii { signature: Signature, aliases: Vec, diff --git a/datafusion/spark/src/function/string/char.rs b/datafusion/spark/src/function/string/char.rs index d25f393c99708..a1813373c65ff 100644 --- a/datafusion/spark/src/function/string/char.rs +++ b/datafusion/spark/src/function/string/char.rs @@ -29,7 +29,7 @@ use datafusion_expr::{ /// Spark-compatible `char` expression /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct CharFunc { signature: Signature, } diff --git a/datafusion/spark/src/function/string/ilike.rs b/datafusion/spark/src/function/string/ilike.rs index f28e931c5018e..a160749523f1e 100644 --- a/datafusion/spark/src/function/string/ilike.rs +++ b/datafusion/spark/src/function/string/ilike.rs @@ -27,7 +27,7 @@ use std::sync::Arc; /// ILIKE function for case-insensitive pattern matching /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkILike { signature: Signature, } diff --git a/datafusion/spark/src/function/string/like.rs b/datafusion/spark/src/function/string/like.rs index eb2e605b3b1e4..df8eaef7cecbc 100644 --- a/datafusion/spark/src/function/string/like.rs +++ b/datafusion/spark/src/function/string/like.rs @@ -27,7 +27,7 @@ use std::sync::Arc; /// LIKE function for case-sensitive pattern matching /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkLike { signature: Signature, } diff --git a/datafusion/spark/src/function/string/luhn_check.rs b/datafusion/spark/src/function/string/luhn_check.rs index 07a4a4a41dabf..79b2a854f72a0 100644 --- a/datafusion/spark/src/function/string/luhn_check.rs +++ b/datafusion/spark/src/function/string/luhn_check.rs @@ -29,7 +29,7 @@ use datafusion_expr::{ /// Spark-compatible `luhn_check` expression /// -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct SparkLuhnCheck { signature: Signature, } diff --git a/datafusion/spark/src/function/url/parse_url.rs b/datafusion/spark/src/function/url/parse_url.rs index 6892e6a3024a2..5c3b1d7d3edaf 100644 --- a/datafusion/spark/src/function/url/parse_url.rs +++ b/datafusion/spark/src/function/url/parse_url.rs @@ -33,7 +33,7 @@ use datafusion_expr::{ use datafusion_functions::utils::make_scalar_function; use url::Url; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct ParseUrl { signature: Signature, } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 56a1fb6b68543..ddfb9e0039c93 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -1783,7 +1783,7 @@ mod tests { use super::*; /// Mocked UDF - #[derive(Debug)] + #[derive(Debug, PartialEq, Eq, Hash)] struct DummyUDF { signature: Signature, } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index 751254ff201a1..6ad82428b6488 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -25,9 +25,9 @@ use arrow::datatypes::{TimeUnit::Nanosecond, *}; use common::MockContextProvider; use datafusion_common::{assert_contains, DataFusionError, Result}; use datafusion_expr::{ - col, logical_plan::LogicalPlan, test::function_stub::sum_udaf, udf_equals_hash, - ColumnarValue, CreateIndex, DdlStatement, ScalarFunctionArgs, ScalarUDF, - ScalarUDFImpl, Signature, Volatility, + col, logical_plan::LogicalPlan, test::function_stub::sum_udaf, ColumnarValue, + CreateIndex, DdlStatement, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, + Volatility, }; use datafusion_functions::{string, unicode}; use datafusion_sql::{ @@ -3349,8 +3349,6 @@ impl ScalarUDFImpl for DummyUDF { fn invoke_with_args(&self, _args: ScalarFunctionArgs) -> Result { panic!("dummy - not implemented") } - - udf_equals_hash!(ScalarUDFImpl); } fn parse_decimals_parser_options() -> ParserOptions { diff --git a/docs/source/library-user-guide/functions/adding-udfs.md b/docs/source/library-user-guide/functions/adding-udfs.md index e702fb4c4ebaa..2335105882a10 100644 --- a/docs/source/library-user-guide/functions/adding-udfs.md +++ b/docs/source/library-user-guide/functions/adding-udfs.md @@ -83,7 +83,7 @@ use datafusion_doc::Documentation; description = "Add one udf", syntax_example = "add_one(1)" )] -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] struct AddOne { signature: Signature, } @@ -146,7 +146,7 @@ We now need to register the function with DataFusion so that it can be used in t # description = "Add one udf", # syntax_example = "add_one(1)" # )] -# #[derive(Debug)] +# #[derive(Debug, PartialEq, Eq, Hash)] # struct AddOne { # signature: Signature, # } @@ -384,7 +384,7 @@ To add a Scalar Async UDF, you need to: # use std::any::Any; # use std::sync::Arc; -#[derive(Debug)] +#[derive(Debug, PartialEq, Eq, Hash)] pub struct AsyncUpper { signature: Signature, } @@ -488,7 +488,7 @@ We can now transfer the async UDF into the normal scalar using `into_scalar_udf` # use std::any::Any; # use std::sync::Arc; # -# #[derive(Debug)] +# #[derive(Debug, PartialEq, Eq, Hash)] # pub struct AsyncUpper { # signature: Signature, # } diff --git a/docs/source/library-user-guide/upgrading.md b/docs/source/library-user-guide/upgrading.md index bd6272d4fc788..57ce6da117bba 100644 --- a/docs/source/library-user-guide/upgrading.md +++ b/docs/source/library-user-guide/upgrading.md @@ -24,13 +24,21 @@ **Note:** DataFusion `50.0.0` has not been released yet. The information provided in this section pertains to features and changes that have already been merged to the main branch and are awaiting release in this version. You can see the current [status of the `50.0.0 `release here](https://github.com/apache/datafusion/issues/16799) -### `AggregateUDFImpl` and `WindowUDFImpl` traits now require `PartialEq`, `Eq`, and `Hash` traits +### `ScalarUDFImpl`, `AggregateUDFImpl` and `WindowUDFImpl` traits now require `PartialEq`, `Eq`, and `Hash` traits -To address error-proneness of `AggregateUDFImpl::equals` and `WindowUDFImpl::equals` methods and -to make it easy to implement function equality correctly, the `equals` and `hash_value` methods have -been removed from `AggregateUDFImpl` and `WindowUDFImpl` traits. They are replaced the requirement to -implement the `PartialEq`, `Eq`, and `Hash` traits on any type implementing `AggregateUDFImpl` -or `WindowUDFImpl`. Please see [issue #16677] for more details. +To address error-proneness of `ScalarUDFImpl::equals`, `AggregateUDFImpl::equals`and +`WindowUDFImpl::equals` methods and to make it easy to implement function equality correctly, +the `equals` and `hash_value` methods have been removed from `ScalarUDFImpl`, `AggregateUDFImpl` +and `WindowUDFImpl` traits. They are replaced the requirement to implement the `PartialEq`, `Eq`, +and `Hash` traits on any type implementing `ScalarUDFImpl`, `AggregateUDFImpl` or `WindowUDFImpl`. +Please see [issue #16677] for more details. + +Most of the scalar functions are stateless and have a `signature` field. These can be migrated +using regular expressions + +- search for `\#\[derive\(Debug\)\](\n *(pub )?struct \w+ \{\n *signature\: Signature\,\n *\})`, +- replace with `#[derive(Debug, PartialEq, Eq, Hash)]$1`, +- review all the changes and make sure only function structs were changed. [issue #16677]: https://github.com/apache/datafusion/issues/16677