From dfd68aa827f18af11f6d8f56290d9c2986234556 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:05:18 -0700 Subject: [PATCH 01/30] move aggregate expressions to spark-expr crate --- native/Cargo.lock | 2 + native/Cargo.toml | 1 + native/core/Cargo.toml | 3 +- native/core/src/common/bit.rs | 6 +-- .../datafusion/expressions/checkoverflow.rs | 15 +------ .../execution/datafusion/expressions/mod.rs | 3 -- .../core/src/execution/datafusion/planner.rs | 6 +-- native/core/src/lib.rs | 27 ------------- native/core/src/parquet/read/levels.rs | 7 ++-- native/core/src/parquet/read/values.rs | 2 +- native/spark-expr/Cargo.toml | 3 ++ .../expressions => spark-expr/src}/avg.rs | 0 .../src}/avg_decimal.rs | 2 +- native/spark-expr/src/lib.rs | 3 ++ .../src}/sum_decimal.rs | 10 ++--- native/spark-expr/src/utils.rs | 39 ++++++++++++++++++- 16 files changed, 62 insertions(+), 67 deletions(-) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/avg.rs (100%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/avg_decimal.rs (99%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/sum_decimal.rs (98%) diff --git a/native/Cargo.lock b/native/Cargo.lock index a7f8359d8..67d041a39 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -942,10 +942,12 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", + "futures", "num", "rand", "regex", "thiserror", + "tokio", "twox-hash 2.0.1", ] diff --git a/native/Cargo.toml b/native/Cargo.toml index 85c46a6d0..4ac85479f 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -51,6 +51,7 @@ datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } chrono-tz = { version = "0.8" } +futures = "0.3.28" num = "0.4" rand = "0.8" regex = "1.9.6" diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index daa0837c8..80dba4c03 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -42,7 +42,7 @@ arrow-data = { workspace = true } arrow-schema = { workspace = true } parquet = { workspace = true, default-features = false, features = ["experimental"] } half = { version = "2.4.1", default-features = false } -futures = "0.3.28" +futures = { workspace = true } mimalloc = { version = "*", default-features = false, optional = true } tokio = { version = "1", features = ["rt-multi-thread"] } async-trait = "0.1" @@ -88,7 +88,6 @@ hex = "0.4.3" [features] default = [] -nightly = [] [lib] name = "comet" diff --git a/native/core/src/common/bit.rs b/native/core/src/common/bit.rs index 871786bb1..72d7729d9 100644 --- a/native/core/src/common/bit.rs +++ b/native/core/src/common/bit.rs @@ -17,14 +17,12 @@ use std::{cmp::min, mem::size_of}; -use arrow::buffer::Buffer; - use crate::{ errors::CometResult as Result, - likely, parquet::{data_type::AsBytes, util::bit_packing::unpack32}, - unlikely, }; +use arrow::buffer::Buffer; +use datafusion_comet_spark_expr::utils::{likely, unlikely}; #[inline] pub fn from_ne_slice(bs: &[u8]) -> T { diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs index ed03ab667..e922171bd 100644 --- a/native/core/src/execution/datafusion/expressions/checkoverflow.rs +++ b/native/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -27,8 +27,7 @@ use arrow::{ datatypes::{Decimal128Type, DecimalType}, record_batch::RecordBatch, }; -use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; -use arrow_schema::{DataType, Schema, DECIMAL128_MAX_PRECISION}; +use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; @@ -172,15 +171,3 @@ impl PhysicalExpr for CheckOverflow { self.hash(&mut s); } } - -/// Adapted from arrow-rs `validate_decimal_precision` but returns bool -/// instead of Err to avoid the cost of formatting the error strings and is -/// optimized to remove a memcpy that exists in the original function -/// we can remove this code once we upgrade to a version of arrow-rs that -/// includes https://github.com/apache/arrow-rs/pull/6419 -#[inline] -pub fn is_valid_decimal_precision(value: i128, precision: u8) -> bool { - precision <= DECIMAL128_MAX_PRECISION - && value >= MIN_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] - && value <= MAX_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] -} diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 48b80384b..75841d181 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -23,8 +23,6 @@ mod normalize_nan; pub use normalize_nan::NormalizeNaNAndZero; use crate::errors::CometError; -pub mod avg; -pub mod avg_decimal; pub mod bloom_filter_agg; pub mod bloom_filter_might_contain; pub mod comet_scalar_funcs; @@ -34,7 +32,6 @@ pub mod negative; pub mod stddev; pub mod strings; pub mod subquery; -pub mod sum_decimal; pub mod unbound; pub mod variance; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 33c4924cb..352f17600 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -25,8 +25,6 @@ use crate::{ execution::{ datafusion::{ expressions::{ - avg::Avg, - avg_decimal::AvgDecimal, bitwise_not::BitwiseNotExpr, bloom_filter_agg::BloomFilterAgg, bloom_filter_might_contain::BloomFilterMightContain, @@ -37,7 +35,6 @@ use crate::{ stddev::Stddev, strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}, subquery::Subquery, - sum_decimal::SumDecimal, unbound::UnboundColumn, variance::Variance, NormalizeNaNAndZero, @@ -98,6 +95,9 @@ use datafusion_comet_proto::{ }, spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; +use datafusion_comet_spark_expr::avg::Avg; +use datafusion_comet_spark_expr::avg_decimal::AvgDecimal; +use datafusion_comet_spark_expr::sum_decimal::SumDecimal; use datafusion_comet_spark_expr::{ ArrayInsert, Cast, CreateNamedStruct, DateTruncExpr, GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, RLike, SecondExpr, SparkCastOptions, diff --git a/native/core/src/lib.rs b/native/core/src/lib.rs index c6a7a4143..68c8ae729 100644 --- a/native/core/src/lib.rs +++ b/native/core/src/lib.rs @@ -104,30 +104,3 @@ fn default_logger_config() -> CometResult { .build(root) .map_err(|err| CometError::Config(err.to_string())) } - -// These are borrowed from hashbrown crate: -// https://github.com/rust-lang/hashbrown/blob/master/src/raw/mod.rs - -// On stable we can use #[cold] to get a equivalent effect: this attributes -// suggests that the function is unlikely to be called -#[cfg(not(feature = "nightly"))] -#[inline] -#[cold] -fn cold() {} - -#[cfg(not(feature = "nightly"))] -#[inline] -fn likely(b: bool) -> bool { - if !b { - cold(); - } - b -} -#[cfg(not(feature = "nightly"))] -#[inline] -fn unlikely(b: bool) -> bool { - if b { - cold(); - } - b -} diff --git a/native/core/src/parquet/read/levels.rs b/native/core/src/parquet/read/levels.rs index 3d74b277c..9077c0e48 100644 --- a/native/core/src/parquet/read/levels.rs +++ b/native/core/src/parquet/read/levels.rs @@ -17,15 +17,14 @@ use std::mem; -use arrow::buffer::Buffer; -use parquet::schema::types::ColumnDescPtr; - use super::values::Decoder; use crate::{ common::bit::{self, read_u32, BitReader}, parquet::ParquetMutableVector, - unlikely, }; +use arrow::buffer::Buffer; +use datafusion_comet_spark_expr::utils::unlikely; +use parquet::schema::types::ColumnDescPtr; const INITIAL_BUF_LEN: usize = 16; diff --git a/native/core/src/parquet/read/values.rs b/native/core/src/parquet/read/values.rs index b439e29e6..71cd035d2 100644 --- a/native/core/src/parquet/read/values.rs +++ b/native/core/src/parquet/read/values.rs @@ -28,9 +28,9 @@ use crate::write_val_or_null; use crate::{ common::bit::{self, BitReader}, parquet::{data_type::*, ParquetMutableVector}, - unlikely, }; use arrow::datatypes::DataType as ArrowDataType; +use datafusion_comet_spark_expr::utils::unlikely; pub fn get_decoder( value_data: Buffer, diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 532bf7437..7d637f684 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -29,6 +29,7 @@ edition = { workspace = true } [dependencies] arrow = { workspace = true } arrow-array = { workspace = true } +arrow-data = { workspace = true } arrow-schema = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true } @@ -39,12 +40,14 @@ chrono-tz = { workspace = true } num = { workspace = true } regex = { workspace = true } thiserror = { workspace = true } +futures = { workspace = true } twox-hash = "2.0.0" [dev-dependencies] arrow-data = {workspace = true} criterion = "0.5.1" rand = { workspace = true} +tokio = { version = "1", features = ["rt-multi-thread"] } [lib] diff --git a/native/core/src/execution/datafusion/expressions/avg.rs b/native/spark-expr/src/avg.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/avg.rs rename to native/spark-expr/src/avg.rs diff --git a/native/core/src/execution/datafusion/expressions/avg_decimal.rs b/native/spark-expr/src/avg_decimal.rs similarity index 99% rename from native/core/src/execution/datafusion/expressions/avg_decimal.rs rename to native/spark-expr/src/avg_decimal.rs index a265fdc29..163e1560b 100644 --- a/native/core/src/execution/datafusion/expressions/avg_decimal.rs +++ b/native/spark-expr/src/avg_decimal.rs @@ -28,7 +28,7 @@ use datafusion_common::{not_impl_err, Result, ScalarValue}; use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::expressions::checkoverflow::is_valid_decimal_precision; +use crate::utils::is_valid_decimal_precision; use arrow_array::ArrowNativeTypeOp; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use datafusion::logical_expr::Volatility::Immutable; diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index c227b3a02..bde6e703d 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -23,12 +23,15 @@ mod cast; mod error; mod if_expr; +pub mod avg; +pub mod avg_decimal; mod kernels; mod list; mod regexp; pub mod scalar_funcs; pub mod spark_hash; mod structs; +pub mod sum_decimal; mod temporal; pub mod timezone; mod to_json; diff --git a/native/core/src/execution/datafusion/expressions/sum_decimal.rs b/native/spark-expr/src/sum_decimal.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/sum_decimal.rs rename to native/spark-expr/src/sum_decimal.rs index d885ff90b..ab142aee6 100644 --- a/native/core/src/execution/datafusion/expressions/sum_decimal.rs +++ b/native/spark-expr/src/sum_decimal.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::datafusion::expressions::checkoverflow::is_valid_decimal_precision; -use crate::unlikely; +use crate::utils::{is_valid_decimal_precision, unlikely}; use arrow::{ array::BooleanBufferBuilder, buffer::{BooleanBuffer, NullBuffer}, @@ -113,7 +112,6 @@ impl AggregateUDFImpl for SumDecimal { Ok(Box::new(SumDecimalGroupsAccumulator::new( self.result_type.clone(), self.precision, - self.scale, ))) } @@ -286,18 +284,16 @@ struct SumDecimalGroupsAccumulator { sum: Vec, result_type: DataType, precision: u8, - scale: i8, } impl SumDecimalGroupsAccumulator { - fn new(result_type: DataType, precision: u8, scale: i8) -> Self { + fn new(result_type: DataType, precision: u8) -> Self { Self { is_not_null: BooleanBufferBuilder::new(0), is_empty: BooleanBufferBuilder::new(0), sum: Vec::new(), result_type, precision, - scale, } } @@ -488,11 +484,11 @@ mod tests { use arrow::datatypes::*; use arrow_array::builder::{Decimal128Builder, StringBuilder}; use arrow_array::RecordBatch; + use datafusion::execution::TaskContext; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; use datafusion_common::Result; - use datafusion_execution::TaskContext; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::{Column, Literal}; diff --git a/native/spark-expr/src/utils.rs b/native/spark-expr/src/utils.rs index db4ad1956..18a2314fb 100644 --- a/native/spark-expr/src/utils.rs +++ b/native/spark-expr/src/utils.rs @@ -19,7 +19,7 @@ use arrow_array::{ cast::as_primitive_array, types::{Int32Type, TimestampMicrosecondType}, }; -use arrow_schema::{ArrowError, DataType}; +use arrow_schema::{ArrowError, DataType, DECIMAL128_MAX_PRECISION}; use std::sync::Arc; use crate::timezone::Tz; @@ -27,6 +27,7 @@ use arrow::{ array::{as_dictionary_array, Array, ArrayRef, PrimitiveArray}, temporal_conversions::as_datetime, }; +use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use chrono::{DateTime, Offset, TimeZone}; /// Preprocesses input arrays to add timezone information from Spark to Arrow array datatype or @@ -176,3 +177,39 @@ fn pre_timestamp_cast(array: ArrayRef, timezone: String) -> Result Ok(array), } } + +/// Adapted from arrow-rs `validate_decimal_precision` but returns bool +/// instead of Err to avoid the cost of formatting the error strings and is +/// optimized to remove a memcpy that exists in the original function +/// we can remove this code once we upgrade to a version of arrow-rs that +/// includes https://github.com/apache/arrow-rs/pull/6419 +#[inline] +pub fn is_valid_decimal_precision(value: i128, precision: u8) -> bool { + precision <= DECIMAL128_MAX_PRECISION + && value >= MIN_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] + && value <= MAX_DECIMAL_FOR_EACH_PRECISION[precision as usize - 1] +} + +// These are borrowed from hashbrown crate: +// https://github.com/rust-lang/hashbrown/blob/master/src/raw/mod.rs + +// On stable we can use #[cold] to get a equivalent effect: this attributes +// suggests that the function is unlikely to be called +#[inline] +#[cold] +pub fn cold() {} + +#[inline] +pub fn likely(b: bool) -> bool { + if !b { + cold(); + } + b +} +#[inline] +pub fn unlikely(b: bool) -> bool { + if b { + cold(); + } + b +} From f6381239990005ac51b516252859f41aee0618df Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:08:44 -0700 Subject: [PATCH 02/30] move more expressions --- native/core/src/execution/datafusion/expressions/mod.rs | 4 ---- native/core/src/execution/datafusion/planner.rs | 8 ++++---- .../expressions => spark-expr/src}/correlation.rs | 5 ++--- .../expressions => spark-expr/src}/covariance.rs | 0 native/spark-expr/src/lib.rs | 4 ++++ .../datafusion/expressions => spark-expr/src}/stddev.rs | 2 +- .../datafusion/expressions => spark-expr/src}/variance.rs | 0 7 files changed, 11 insertions(+), 12 deletions(-) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/correlation.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/covariance.rs (100%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/stddev.rs (98%) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/variance.rs (100%) diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 75841d181..6792eb753 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -26,14 +26,10 @@ use crate::errors::CometError; pub mod bloom_filter_agg; pub mod bloom_filter_might_contain; pub mod comet_scalar_funcs; -pub mod correlation; -pub mod covariance; pub mod negative; -pub mod stddev; pub mod strings; pub mod subquery; pub mod unbound; -pub mod variance; pub use datafusion_comet_spark_expr::{EvalMode, SparkError}; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 352f17600..59904cdb7 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -29,14 +29,10 @@ use crate::{ bloom_filter_agg::BloomFilterAgg, bloom_filter_might_contain::BloomFilterMightContain, checkoverflow::CheckOverflow, - correlation::Correlation, - covariance::Covariance, negative, - stddev::Stddev, strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}, subquery::Subquery, unbound::UnboundColumn, - variance::Variance, NormalizeNaNAndZero, }, operators::expand::CometExpandExec, @@ -97,7 +93,11 @@ use datafusion_comet_proto::{ }; use datafusion_comet_spark_expr::avg::Avg; use datafusion_comet_spark_expr::avg_decimal::AvgDecimal; +use datafusion_comet_spark_expr::correlation::Correlation; +use datafusion_comet_spark_expr::covariance::Covariance; +use datafusion_comet_spark_expr::stddev::Stddev; use datafusion_comet_spark_expr::sum_decimal::SumDecimal; +use datafusion_comet_spark_expr::variance::Variance; use datafusion_comet_spark_expr::{ ArrayInsert, Cast, CreateNamedStruct, DateTruncExpr, GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, RLike, SecondExpr, SparkCastOptions, diff --git a/native/core/src/execution/datafusion/expressions/correlation.rs b/native/spark-expr/src/correlation.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/correlation.rs rename to native/spark-expr/src/correlation.rs index 6bf35e711..e5f36c6f9 100644 --- a/native/core/src/execution/datafusion/expressions/correlation.rs +++ b/native/spark-expr/src/correlation.rs @@ -19,9 +19,8 @@ use arrow::compute::{and, filter, is_not_null}; use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::expressions::{ - covariance::CovarianceAccumulator, stddev::StddevAccumulator, -}; +use crate::covariance::CovarianceAccumulator; +use crate::stddev::StddevAccumulator; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, diff --git a/native/core/src/execution/datafusion/expressions/covariance.rs b/native/spark-expr/src/covariance.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/covariance.rs rename to native/spark-expr/src/covariance.rs diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index bde6e703d..9a73ad08d 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -25,17 +25,21 @@ mod if_expr; pub mod avg; pub mod avg_decimal; +pub mod correlation; +pub mod covariance; mod kernels; mod list; mod regexp; pub mod scalar_funcs; pub mod spark_hash; +pub mod stddev; mod structs; pub mod sum_decimal; mod temporal; pub mod timezone; mod to_json; pub mod utils; +pub mod variance; pub use cast::{spark_cast, Cast, SparkCastOptions}; pub use error::{SparkError, SparkResult}; diff --git a/native/core/src/execution/datafusion/expressions/stddev.rs b/native/spark-expr/src/stddev.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/stddev.rs rename to native/spark-expr/src/stddev.rs index 1ba495e21..3cf604da0 100644 --- a/native/core/src/execution/datafusion/expressions/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -17,7 +17,7 @@ use std::{any::Any, sync::Arc}; -use crate::execution::datafusion::expressions::variance::VarianceAccumulator; +use crate::variance::VarianceAccumulator; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, diff --git a/native/core/src/execution/datafusion/expressions/variance.rs b/native/spark-expr/src/variance.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/variance.rs rename to native/spark-expr/src/variance.rs From 32575432d98bc29f696d10fbb82dbe3a96902b51 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:28:18 -0700 Subject: [PATCH 03/30] move benchmark --- native/core/Cargo.toml | 4 ---- native/spark-expr/Cargo.toml | 5 +++++ native/{core => spark-expr}/benches/aggregate.rs | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) rename native/{core => spark-expr}/benches/aggregate.rs (97%) diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 80dba4c03..4b9753ec5 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -122,10 +122,6 @@ harness = false name = "filter" harness = false -[[bench]] -name = "aggregate" -harness = false - [[bench]] name = "bloom_filter_agg" harness = false diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 7d637f684..65517431d 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -69,3 +69,8 @@ harness = false [[bench]] name = "decimal_div" harness = false + +[[bench]] +name = "aggregate" +harness = false + diff --git a/native/core/benches/aggregate.rs b/native/spark-expr/benches/aggregate.rs similarity index 97% rename from native/core/benches/aggregate.rs rename to native/spark-expr/benches/aggregate.rs index c6209406f..b3d5baf3b 100644 --- a/native/core/benches/aggregate.rs +++ b/native/spark-expr/benches/aggregate.rs @@ -19,16 +19,16 @@ use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::builder::{Decimal128Builder, StringBuilder}; use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; -use comet::execution::datafusion::expressions::avg_decimal::AvgDecimal; -use comet::execution::datafusion::expressions::sum_decimal::SumDecimal; use criterion::{black_box, criterion_group, criterion_main, Criterion}; +use datafusion::execution::TaskContext; use datafusion::functions_aggregate::average::avg_udaf; use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; -use datafusion_execution::TaskContext; +use datafusion_comet_spark_expr::avg_decimal::AvgDecimal; +use datafusion_comet_spark_expr::sum_decimal::SumDecimal; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Column; From 42ecfbbe4962c3209f7134fb4b962499cde3bae9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:35:55 -0700 Subject: [PATCH 04/30] normalize_nan --- .../execution/datafusion/expressions/mod.rs | 2 -- .../core/src/execution/datafusion/planner.rs | 15 ++++-------- native/spark-expr/src/lib.rs | 24 +++++++++++++------ .../src}/normalize_nan.rs | 0 4 files changed, 21 insertions(+), 20 deletions(-) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/normalize_nan.rs (100%) diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 6792eb753..2487e64bc 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -19,8 +19,6 @@ pub mod bitwise_not; pub mod checkoverflow; -mod normalize_nan; -pub use normalize_nan::NormalizeNaNAndZero; use crate::errors::CometError; pub mod bloom_filter_agg; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 59904cdb7..747288ca8 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -33,7 +33,6 @@ use crate::{ strings::{Contains, EndsWith, Like, StartsWith, StringSpaceExpr, SubstringExpr}, subquery::Subquery, unbound::UnboundColumn, - NormalizeNaNAndZero, }, operators::expand::CometExpandExec, shuffle_writer::ShuffleWriterExec, @@ -91,17 +90,11 @@ use datafusion_comet_proto::{ }, spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; -use datafusion_comet_spark_expr::avg::Avg; -use datafusion_comet_spark_expr::avg_decimal::AvgDecimal; -use datafusion_comet_spark_expr::correlation::Correlation; -use datafusion_comet_spark_expr::covariance::Covariance; -use datafusion_comet_spark_expr::stddev::Stddev; -use datafusion_comet_spark_expr::sum_decimal::SumDecimal; -use datafusion_comet_spark_expr::variance::Variance; use datafusion_comet_spark_expr::{ - ArrayInsert, Cast, CreateNamedStruct, DateTruncExpr, GetArrayStructFields, GetStructField, - HourExpr, IfExpr, ListExtract, MinuteExpr, RLike, SecondExpr, SparkCastOptions, - TimestampTruncExpr, ToJson, + ArrayInsert, Avg, AvgDecimal, Cast, Correlation, Covariance, CreateNamedStruct, DateTruncExpr, + GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, + NormalizeNaNAndZero, RLike, SecondExpr, SparkCastOptions, Stddev, SumDecimal, + TimestampTruncExpr, ToJson, Variance, }; use datafusion_common::scalar::ScalarStructBuilder; use datafusion_common::{ diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 9a73ad08d..03c816381 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -23,23 +23,33 @@ mod cast; mod error; mod if_expr; -pub mod avg; -pub mod avg_decimal; -pub mod correlation; -pub mod covariance; +mod avg; +pub use avg::Avg; +mod avg_decimal; +pub use avg_decimal::AvgDecimal; +mod correlation; +pub use correlation::Correlation; +mod covariance; +pub use covariance::Covariance; mod kernels; mod list; mod regexp; pub mod scalar_funcs; pub mod spark_hash; -pub mod stddev; +mod stddev; +pub use stddev::Stddev; mod structs; -pub mod sum_decimal; +mod sum_decimal; +pub use sum_decimal::SumDecimal; +mod normalize_nan; mod temporal; pub mod timezone; mod to_json; pub mod utils; -pub mod variance; +pub use normalize_nan::NormalizeNaNAndZero; + +mod variance; +pub use variance::Variance; pub use cast::{spark_cast, Cast, SparkCastOptions}; pub use error::{SparkError, SparkResult}; diff --git a/native/core/src/execution/datafusion/expressions/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs similarity index 100% rename from native/core/src/execution/datafusion/expressions/normalize_nan.rs rename to native/spark-expr/src/normalize_nan.rs From 03bff063d94ff3a11b3dfabe3b739113b9e89554 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:37:43 -0700 Subject: [PATCH 05/30] bitwise not --- .../execution/datafusion/expressions/mod.rs | 1 - .../core/src/execution/datafusion/planner.rs | 5 ++--- .../src}/bitwise_not.rs | 18 +----------------- native/spark-expr/src/lib.rs | 2 ++ 4 files changed, 5 insertions(+), 21 deletions(-) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/bitwise_not.rs (88%) diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index 2487e64bc..a56f38f8b 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -17,7 +17,6 @@ //! Native DataFusion expressions -pub mod bitwise_not; pub mod checkoverflow; use crate::errors::CometError; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 747288ca8..5259e7d36 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -25,7 +25,6 @@ use crate::{ execution::{ datafusion::{ expressions::{ - bitwise_not::BitwiseNotExpr, bloom_filter_agg::BloomFilterAgg, bloom_filter_might_contain::BloomFilterMightContain, checkoverflow::CheckOverflow, @@ -91,8 +90,8 @@ use datafusion_comet_proto::{ spark_partitioning::{partitioning::PartitioningStruct, Partitioning as SparkPartitioning}, }; use datafusion_comet_spark_expr::{ - ArrayInsert, Avg, AvgDecimal, Cast, Correlation, Covariance, CreateNamedStruct, DateTruncExpr, - GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, + ArrayInsert, Avg, AvgDecimal, BitwiseNotExpr, Cast, Correlation, Covariance, CreateNamedStruct, + DateTruncExpr, GetArrayStructFields, GetStructField, HourExpr, IfExpr, ListExtract, MinuteExpr, NormalizeNaNAndZero, RLike, SecondExpr, SparkCastOptions, Stddev, SumDecimal, TimestampTruncExpr, ToJson, Variance, }; diff --git a/native/core/src/execution/datafusion/expressions/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs similarity index 88% rename from native/core/src/execution/datafusion/expressions/bitwise_not.rs rename to native/spark-expr/src/bitwise_not.rs index a2b9ebe5b..36234935e 100644 --- a/native/core/src/execution/datafusion/expressions/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -28,7 +28,7 @@ use arrow::{ }; use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; macro_rules! compute_op { @@ -135,22 +135,6 @@ pub fn bitwise_not(arg: Arc) -> Result> Ok(Arc::new(BitwiseNotExpr::new(arg))) } -fn scalar_bitwise_not(scalar: ScalarValue) -> Result { - match scalar { - ScalarValue::Int8(None) - | ScalarValue::Int16(None) - | ScalarValue::Int32(None) - | ScalarValue::Int64(None) => Ok(scalar), - ScalarValue::Int8(Some(v)) => Ok(ScalarValue::Int8(Some(!v))), - ScalarValue::Int16(Some(v)) => Ok(ScalarValue::Int16(Some(!v))), - ScalarValue::Int32(Some(v)) => Ok(ScalarValue::Int32(Some(!v))), - ScalarValue::Int64(Some(v)) => Ok(ScalarValue::Int64(Some(!v))), - value => Err(DataFusionError::Internal(format!( - "Can not run ! on scalar value {value:?}" - ))), - } -} - #[cfg(test)] mod tests { use arrow::datatypes::*; diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 03c816381..94bd93d7c 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -25,6 +25,8 @@ mod if_expr; mod avg; pub use avg::Avg; +mod bitwise_not; +pub use bitwise_not::{bitwise_not, BitwiseNotExpr}; mod avg_decimal; pub use avg_decimal::AvgDecimal; mod correlation; From 7bc7ab8a7121aea58a762afc22eaf044bf9db473 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:41:17 -0700 Subject: [PATCH 06/30] comet scalar funcs --- native/core/src/execution/datafusion/expressions/mod.rs | 1 - native/core/src/execution/datafusion/planner.rs | 2 +- .../expressions => spark-expr/src}/comet_scalar_funcs.rs | 6 +++--- native/spark-expr/src/lib.rs | 3 ++- 4 files changed, 6 insertions(+), 6 deletions(-) rename native/{core/src/execution/datafusion/expressions => spark-expr/src}/comet_scalar_funcs.rs (98%) diff --git a/native/core/src/execution/datafusion/expressions/mod.rs b/native/core/src/execution/datafusion/expressions/mod.rs index a56f38f8b..2bb14df36 100644 --- a/native/core/src/execution/datafusion/expressions/mod.rs +++ b/native/core/src/execution/datafusion/expressions/mod.rs @@ -22,7 +22,6 @@ pub mod checkoverflow; use crate::errors::CometError; pub mod bloom_filter_agg; pub mod bloom_filter_might_contain; -pub mod comet_scalar_funcs; pub mod negative; pub mod strings; pub mod subquery; diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 5259e7d36..a83dba5d6 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -18,7 +18,6 @@ //! Converts Spark physical plan to DataFusion physical plan use super::expressions::EvalMode; -use crate::execution::datafusion::expressions::comet_scalar_funcs::create_comet_physical_fun; use crate::execution::operators::{CopyMode, FilterExec}; use crate::{ errors::ExpressionError, @@ -73,6 +72,7 @@ use datafusion::{ }, prelude::SessionContext, }; +use datafusion_comet_spark_expr::create_comet_physical_fun; use datafusion_functions_nested::concat::ArrayAppend; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; diff --git a/native/core/src/execution/datafusion/expressions/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs similarity index 98% rename from native/core/src/execution/datafusion/expressions/comet_scalar_funcs.rs rename to native/spark-expr/src/comet_scalar_funcs.rs index 06717aabe..71ff0e9dc 100644 --- a/native/core/src/execution/datafusion/expressions/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -15,15 +15,15 @@ // specific language governing permissions and limitations // under the License. -use arrow_schema::DataType; -use datafusion_comet_spark_expr::scalar_funcs::hash_expressions::{ +use crate::scalar_funcs::hash_expressions::{ spark_sha224, spark_sha256, spark_sha384, spark_sha512, }; -use datafusion_comet_spark_expr::scalar_funcs::{ +use crate::scalar_funcs::{ spark_ceil, spark_date_add, spark_date_sub, spark_decimal_div, spark_floor, spark_hex, spark_isnan, spark_make_decimal, spark_murmur3_hash, spark_read_side_padding, spark_round, spark_unhex, spark_unscaled_value, spark_xxhash64, SparkChrFunc, }; +use arrow_schema::DataType; use datafusion_common::{DataFusionError, Result as DataFusionResult}; use datafusion_expr::registry::FunctionRegistry; use datafusion_expr::{ diff --git a/native/spark-expr/src/lib.rs b/native/spark-expr/src/lib.rs index 94bd93d7c..15f446ef3 100644 --- a/native/spark-expr/src/lib.rs +++ b/native/spark-expr/src/lib.rs @@ -52,8 +52,9 @@ pub use normalize_nan::NormalizeNaNAndZero; mod variance; pub use variance::Variance; - +mod comet_scalar_funcs; pub use cast::{spark_cast, Cast, SparkCastOptions}; +pub use comet_scalar_funcs::create_comet_physical_fun; pub use error::{SparkError, SparkResult}; pub use if_expr::IfExpr; pub use list::{ArrayInsert, GetArrayStructFields, ListExtract}; From 83dc6573e0e8f441cfc40cf7121a58fd8ebbb6df Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:43:54 -0700 Subject: [PATCH 07/30] update bench imports --- native/spark-expr/benches/aggregate.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/spark-expr/benches/aggregate.rs b/native/spark-expr/benches/aggregate.rs index b3d5baf3b..43194fdda 100644 --- a/native/spark-expr/benches/aggregate.rs +++ b/native/spark-expr/benches/aggregate.rs @@ -27,8 +27,8 @@ use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::ExecutionPlan; -use datafusion_comet_spark_expr::avg_decimal::AvgDecimal; -use datafusion_comet_spark_expr::sum_decimal::SumDecimal; +use datafusion_comet_spark_expr::AvgDecimal; +use datafusion_comet_spark_expr::SumDecimal; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; use datafusion_physical_expr::expressions::Column; From 601fed27c362e3ba823a1c77673c55009f15fe07 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 07:16:23 -0700 Subject: [PATCH 08/30] save --- native/Cargo.lock | 280 ++++++++++++++++++++++++++-------------------- native/Cargo.toml | 28 ++--- 2 files changed, 174 insertions(+), 134 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 67d041a39..76a0d182b 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -114,9 +114,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4caf25cdc4a985f91df42ed9e9308e1adbcd341a31a72605c697033fcef163e3" +checksum = "c91839b07e474b3995035fd8ac33ee54f9c9ccbbb1ea33d9909c71bffdf1259d" dependencies = [ "arrow-arith", "arrow-array", @@ -135,9 +135,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91f2dfd1a7ec0aca967dfaa616096aec49779adc8eccec005e2f5e4111b1192a" +checksum = "855c57c4efd26722b044dcd3e348252560e3e0333087fb9f6479dc0bf744054f" dependencies = [ "arrow-array", "arrow-buffer", @@ -150,9 +150,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d39387ca628be747394890a6e47f138ceac1aa912eab64f02519fed24b637af8" +checksum = "bd03279cea46569acf9295f6224fbc370c5df184b4d2ecfe97ccb131d5615a7f" dependencies = [ "ahash", "arrow-buffer", @@ -161,15 +161,15 @@ dependencies = [ "chrono", "chrono-tz 0.10.0", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.1", "num", ] [[package]] name = "arrow-buffer" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e51e05228852ffe3eb391ce7178a0f97d2cf80cc6ef91d3c4a6b3cb688049ec" +checksum = "9e4a9b9b1d6d7117f6138e13bc4dd5daa7f94e671b70e8c9c4dc37b4f5ecfc16" dependencies = [ "bytes", "half", @@ -178,9 +178,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d09aea56ec9fa267f3f3f6cdab67d8a9974cbba90b3aa38c8fe9d0bb071bd8c1" +checksum = "bc70e39916e60c5b7af7a8e2719e3ae589326039e1e863675a008bee5ffe90fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -199,9 +199,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c07b5232be87d115fde73e32f2ca7f1b353bff1b44ac422d3c6fc6ae38f11f0d" +checksum = "789b2af43c1049b03a8d088ff6b2257cdcea1756cd76b174b1f2600356771b97" dependencies = [ "arrow-array", "arrow-buffer", @@ -218,9 +218,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b98ae0af50890b494cebd7d6b04b35e896205c1d1df7b29a6272c5d0d0249ef5" +checksum = "e4e75edf21ffd53744a9b8e3ed11101f610e7ceb1a29860432824f1834a1f623" dependencies = [ "arrow-buffer", "arrow-schema", @@ -230,9 +230,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ed91bdeaff5a1c00d28d8f73466bcb64d32bbd7093b5a30156b4b9f4dba3eee" +checksum = "d186a909dece9160bf8312f5124d797884f608ef5435a36d9d608e0b2a9bcbf8" dependencies = [ "arrow-array", "arrow-buffer", @@ -245,9 +245,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0471f51260a5309307e5d409c9dc70aede1cd9cf1d4ff0f0a1e8e1a2dd0e0d3c" +checksum = "b66ff2fedc1222942d0bd2fd391cb14a85baa3857be95c9373179bd616753b85" dependencies = [ "arrow-array", "arrow-buffer", @@ -265,9 +265,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2883d7035e0b600fb4c30ce1e50e66e53d8656aa729f2bfa4b51d359cf3ded52" +checksum = "ece7b5bc1180e6d82d1a60e1688c199829e8842e38497563c3ab6ea813e527fd" dependencies = [ "arrow-array", "arrow-buffer", @@ -280,9 +280,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "552907e8e587a6fde4f8843fd7a27a576a260f65dab6c065741ea79f633fc5be" +checksum = "745c114c8f0e8ce211c83389270de6fbe96a9088a7b32c2a041258a443fe83ff" dependencies = [ "ahash", "arrow-array", @@ -294,18 +294,18 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "539ada65246b949bd99ffa0881a9a15a4a529448af1a07a9838dd78617dafab1" +checksum = "b95513080e728e4cec37f1ff5af4f12c9688d47795d17cda80b6ec2cf74d4678" dependencies = [ "bitflags 2.6.0", ] [[package]] name = "arrow-select" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6259e566b752da6dceab91766ed8b2e67bf6270eb9ad8a6e07a33c1bede2b125" +checksum = "8e415279094ea70323c032c6e739c48ad8d80e78a09bef7117b8718ad5bf3722" dependencies = [ "ahash", "arrow-array", @@ -317,9 +317,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3179ccbd18ebf04277a095ba7321b93fd1f774f18816bd5f6b3ce2f594edb6c" +checksum = "11d956cae7002eb8d83a27dbd34daaea1cf5b75852f0b84deb4d93a276e92bbf" dependencies = [ "arrow-array", "arrow-buffer", @@ -806,10 +806,8 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbba0799cf6913b456ed07a94f0f3b6e12c62a5d88b10809e2284a0f2b915c05" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ - "ahash", "arrow", "arrow-array", "arrow-ipc", @@ -825,6 +823,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", + "datafusion-functions-table", "datafusion-functions-window", "datafusion-optimizer", "datafusion-physical-expr", @@ -834,16 +833,11 @@ dependencies = [ "datafusion-sql", "futures", "glob", - "half", - "hashbrown 0.14.5", - "indexmap", "itertools 0.13.0", "log", - "num_cpus", "object_store", "parking_lot", "paste", - "pin-project-lite", "rand", "sqlparser", "tempfile", @@ -855,8 +849,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7493c5c2d40eec435b13d92e5703554f4efc7059451fcb8d3a79580ff0e45560" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow-schema", "async-trait", @@ -954,8 +947,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24953049ebbd6f8964f91f60aa3514e121b5e81e068e33b60e77815ab369b25c" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "ahash", "arrow", @@ -966,30 +958,33 @@ dependencies = [ "half", "hashbrown 0.14.5", "indexmap", - "instant", "libc", - "num_cpus", "object_store", "paste", + "recursive", "sqlparser", "tokio", + "web-time", ] [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f06df4ef76872e11c924d3c814fd2a8dd09905ed2e2195f71c857d78abd19685" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "log", "tokio", ] +[[package]] +name = "datafusion-doc" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" + [[package]] name = "datafusion-execution" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bbdcb628d690f3ce5fea7de81642b514486d58ff9779a51f180a69a4eadb361" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow", "chrono", @@ -997,7 +992,6 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.5", "log", "object_store", "parking_lot", @@ -1009,8 +1003,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8036495980e3131f706b7d33ab00b4492d73dc714e3cb74d11b50f9602a73246" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "ahash", "arrow", @@ -1018,12 +1011,14 @@ dependencies = [ "arrow-buffer", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-expr-common", "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", "indexmap", "paste", + "recursive", "serde_json", "sqlparser", "strum", @@ -1033,8 +1028,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4da0f3cb4669f9523b403d6b5a0ec85023e0ab3bf0183afd1517475b3e64fdd2" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow", "datafusion-common", @@ -1045,8 +1039,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f52c4012648b34853e40a2c6bcaa8772f837831019b68aca384fb38436dba162" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow", "arrow-buffer", @@ -1055,8 +1048,10 @@ dependencies = [ "blake3", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-execution", "datafusion-expr", + "datafusion-macros", "hashbrown 0.14.5", "hex", "itertools 0.13.0", @@ -1072,20 +1067,20 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5b8bb624597ba28ed7446df4a9bd7c7a7bde7c578b6b527da3f47371d5f6741" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "ahash", "arrow", "arrow-schema", "datafusion-common", + "datafusion-doc", "datafusion-execution", "datafusion-expr", "datafusion-functions-aggregate-common", + "datafusion-macros", "datafusion-physical-expr", "datafusion-physical-expr-common", "half", - "indexmap", "log", "paste", ] @@ -1093,8 +1088,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fb06208fc470bc8cf1ce2d9a1159d42db591f2c7264a8c1776b53ad8f675143" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "ahash", "arrow", @@ -1107,8 +1101,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fca25bbb87323716d05e54114666e942172ccca23c5a507e9c7851db6e965317" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow", "arrow-array", @@ -1127,11 +1120,34 @@ dependencies = [ "rand", ] +[[package]] +name = "datafusion-functions-table" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +dependencies = [ + "ahash", + "arrow", + "arrow-schema", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "half", + "indexmap", + "log", + "parking_lot", + "paste", +] + [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ae23356c634e54c59f7c51acb7a5b9f6240ffb2cf997049a1a24a8a88598dbe" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "datafusion-common", "datafusion-expr", @@ -1145,18 +1161,26 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4b3d6ff7794acea026de36007077a06b18b89e4f9c3fea7f2215f9f7dd9059b" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", ] +[[package]] +name = "datafusion-macros" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.87", +] + [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bec6241eb80c595fa0e1a8a6b69686b5cf3bd5fdacb8319582a0943b0bd788aa" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow", "async-trait", @@ -1164,28 +1188,24 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.5", "indexmap", "itertools 0.13.0", "log", - "paste", + "recursive", + "regex", "regex-syntax", ] [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3370357b8fc75ec38577700644e5d1b0bc78f38babab99c0b8bd26bafb3e4335" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "ahash", "arrow", "arrow-array", "arrow-buffer", - "arrow-ord", "arrow-schema", - "arrow-string", - "chrono", "datafusion-common", "datafusion-expr", "datafusion-expr-common", @@ -1203,38 +1223,36 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8b7734d94bf2fa6f6e570935b0ddddd8421179ce200065be97874e13d46a47b" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "ahash", "arrow", "datafusion-common", "datafusion-expr-common", "hashbrown 0.14.5", - "rand", + "itertools 0.13.0", ] [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eee8c479522df21d7b395640dff88c5ed05361852dce6544d7c98e9dbcebffe" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow", - "arrow-schema", "datafusion-common", "datafusion-execution", "datafusion-expr-common", "datafusion-physical-expr", "datafusion-physical-plan", "itertools 0.13.0", + "log", + "recursive", ] [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17e1fc2e2c239d14e8556f2622b19a726bf6bc6962cc00c71fc52626274bee24" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "ahash", "arrow", @@ -1248,7 +1266,6 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", @@ -1268,8 +1285,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63e3a4ed41dbee20a5d947a59ca035c225d67dc9cbe869c10f66dcdf25e7ce51" +source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" dependencies = [ "arrow", "arrow-array", @@ -1278,9 +1294,9 @@ dependencies = [ "datafusion-expr", "indexmap", "log", + "recursive", "regex", "sqlparser", - "strum", ] [[package]] @@ -1576,12 +1592,6 @@ version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" -[[package]] -name = "hermit-abi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" - [[package]] name = "hermit-abi" version = "0.4.0" @@ -1799,18 +1809,6 @@ dependencies = [ "str_stack", ] -[[package]] -name = "instant" -version = "0.1.13" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0242819d153cba4b4b05a5a8f2a7e9bbf97b6055b2a002b395c96b5ff3c0222" -dependencies = [ - "cfg-if", - "js-sys", - "wasm-bindgen", - "web-sys", -] - [[package]] name = "integer-encoding" version = "3.0.4" @@ -1823,7 +1821,7 @@ version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "261f68e344040fbd0edea105bef17c66edf46f984ddb1115b775ce31be948f4b" dependencies = [ - "hermit-abi 0.4.0", + "hermit-abi", "libc", "windows-sys 0.52.0", ] @@ -2260,16 +2258,6 @@ dependencies = [ "libm", ] -[[package]] -name = "num_cpus" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" -dependencies = [ - "hermit-abi 0.3.9", - "libc", -] - [[package]] name = "object" version = "0.36.5" @@ -2346,15 +2334,15 @@ dependencies = [ [[package]] name = "parquet" -version = "53.2.0" +version = "53.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dea02606ba6f5e856561d8d507dba8bac060aefca2a6c0f1aa1d361fed91ff3e" +checksum = "2b449890367085eb65d7d3321540abc3d7babbd179ce31df0016e90719114191" dependencies = [ "ahash", "bytes", "chrono", "half", - "hashbrown 0.14.5", + "hashbrown 0.15.1", "num", "num-bigint", "paste", @@ -2593,6 +2581,15 @@ dependencies = [ "prost 0.9.0", ] +[[package]] +name = "psm" +version = "0.1.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "200b9ff220857e53e184257720a14553b2f4aa02577d2ed9842d45d4b9654810" +dependencies = [ + "cc", +] + [[package]] name = "quick-xml" version = "0.26.0" @@ -2661,6 +2658,26 @@ dependencies = [ "crossbeam-utils", ] +[[package]] +name = "recursive" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0786a43debb760f491b1bc0269fe5e84155353c67482b9e60d0cfb596054b43e" +dependencies = [ + "recursive-proc-macro-impl", + "stacker", +] + +[[package]] +name = "recursive-proc-macro-impl" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" +dependencies = [ + "quote", + "syn 2.0.87", +] + [[package]] name = "redox_syscall" version = "0.5.7" @@ -2903,9 +2920,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "sqlparser" -version = "0.51.0" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fe11944a61da0da3f592e19a45ebe5ab92dc14a779907ff1f08fbb797bfefc7" +checksum = "9a875d8cd437cc8a97e9aeaeea352ec9a19aea99c23e9effb17757291de80b08" dependencies = [ "log", "sqlparser_derive", @@ -2928,6 +2945,19 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +[[package]] +name = "stacker" +version = "0.1.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "799c883d55abdb5e98af1a7b3f23b9b6de8ecada0ecac058672d7635eb48ca7b" +dependencies = [ + "cc", + "cfg-if", + "libc", + "psm", + "windows-sys 0.59.0", +] + [[package]] name = "static_assertions" version = "1.1.0" @@ -3347,6 +3377,16 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "web-time" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a6580f308b1fad9207618087a65c04e7a10bc77e02c8e84e9b00dd4b12fa0bb" +dependencies = [ + "js-sys", + "wasm-bindgen", +] + [[package]] name = "which" version = "4.4.2" diff --git a/native/Cargo.toml b/native/Cargo.toml index 4ac85479f..b2daab271 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -33,20 +33,20 @@ edition = "2021" rust-version = "1.79" [workspace.dependencies] -arrow = { version = "53.2.0", features = ["prettyprint", "ffi", "chrono-tz"] } -arrow-array = { version = "53.2.0" } -arrow-buffer = { version = "53.2.0" } -arrow-data = { version = "53.2.0" } -arrow-schema = { version = "53.2.0" } -parquet = { version = "53.2.0", default-features = false, features = ["experimental"] } -datafusion-common = { version = "43.0.0" } -datafusion = { version = "43.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { version = "43.0.0", features = ["crypto_expressions"] } -datafusion-functions-nested = { version = "43.0.0", default-features = false } -datafusion-expr = { version = "43.0.0", default-features = false } -datafusion-execution = { version = "43.0.0", default-features = false } -datafusion-physical-plan = { version = "43.0.0", default-features = false } -datafusion-physical-expr = { version = "43.0.0", default-features = false } +arrow = { version = "53.3.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow-array = { version = "53.3.0" } +arrow-buffer = { version = "53.3.0" } +arrow-data = { version = "53.3.0" } +arrow-schema = { version = "53.3.0" } +parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git" } +datafusion = { git = "https://github.com/apache/datafusion.git", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } From 3b194db65efd7749494a63208f03aceb61bea155 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 07:55:14 -0700 Subject: [PATCH 09/30] save --- native/spark-expr/src/avg.rs | 2 +- native/spark-expr/src/avg_decimal.rs | 2 +- native/spark-expr/src/bitwise_not.rs | 16 ++++++---- native/spark-expr/src/cast.rs | 15 ++++----- native/spark-expr/src/correlation.rs | 2 +- native/spark-expr/src/covariance.rs | 2 +- native/spark-expr/src/if_expr.rs | 21 ++++++------ native/spark-expr/src/list.rs | 33 +++---------------- native/spark-expr/src/normalize_nan.rs | 10 ++---- native/spark-expr/src/regexp.rs | 18 +++++++---- native/spark-expr/src/stddev.rs | 5 +-- native/spark-expr/src/structs.rs | 18 ++--------- native/spark-expr/src/sum_decimal.rs | 2 +- native/spark-expr/src/temporal.rs | 44 ++++---------------------- native/spark-expr/src/to_json.rs | 8 +---- native/spark-expr/src/utils.rs | 16 ++++++++++ native/spark-expr/src/variance.rs | 2 +- 17 files changed, 80 insertions(+), 136 deletions(-) diff --git a/native/spark-expr/src/avg.rs b/native/spark-expr/src/avg.rs index 7820497d4..befada923 100644 --- a/native/spark-expr/src/avg.rs +++ b/native/spark-expr/src/avg.rs @@ -31,7 +31,7 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; use std::{any::Any, sync::Arc}; use arrow_array::ArrowNativeTypeOp; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; diff --git a/native/spark-expr/src/avg_decimal.rs b/native/spark-expr/src/avg_decimal.rs index 163e1560b..47d6a9a6f 100644 --- a/native/spark-expr/src/avg_decimal.rs +++ b/native/spark-expr/src/avg_decimal.rs @@ -32,7 +32,7 @@ use crate::utils::is_valid_decimal_precision; use arrow_array::ArrowNativeTypeOp; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use datafusion::logical_expr::Volatility::Immutable; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::avg_return_type; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index 36234935e..716e38bb2 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -26,10 +26,11 @@ use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; +use crate::IfExpr; macro_rules! compute_op { ($OPERAND:expr, $DT:ident) => {{ @@ -43,12 +44,18 @@ macro_rules! compute_op { } /// BitwiseNot expression -#[derive(Debug, Hash)] +#[derive(Debug, Hash, Eq)] pub struct BitwiseNotExpr { /// Input expression arg: Arc, } +impl PartialEq for BitwiseNotExpr { + fn eq(&self, other: &Self) -> bool { + self.arg.eq(&other.arg) + } +} + impl BitwiseNotExpr { /// Create new bitwise not expression pub fn new(arg: Arc) -> Self { @@ -115,11 +122,6 @@ impl PhysicalExpr for BitwiseNotExpr { Ok(Arc::new(BitwiseNotExpr::new(Arc::clone(&children[0])))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.arg.hash(&mut s); - self.hash(&mut s); - } } impl PartialEq for BitwiseNotExpr { diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index f62d0220c..5b886893f 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -50,7 +50,8 @@ use std::{ }; use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; +use crate::utils::down_cast_any_ref; use num::{ cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, ToPrimitive, @@ -134,7 +135,7 @@ impl TimeStampInfo { } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, Eq)] pub struct Cast { pub child: Arc, pub data_type: DataType, @@ -1498,13 +1499,11 @@ impl PhysicalExpr for Cast { _ => internal_err!("Cast should have exactly one child"), } } +} - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.cast_options.hash(&mut s); - self.hash(&mut s); +impl PartialEq for Cast { + fn eq(&self, other: &Self) -> bool { + todo!() } } diff --git a/native/spark-expr/src/correlation.rs b/native/spark-expr/src/correlation.rs index e5f36c6f9..ef8480f35 100644 --- a/native/spark-expr/src/correlation.rs +++ b/native/spark-expr/src/correlation.rs @@ -26,7 +26,7 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; diff --git a/native/spark-expr/src/covariance.rs b/native/spark-expr/src/covariance.rs index 9166e3976..56d8e231c 100644 --- a/native/spark-expr/src/covariance.rs +++ b/native/spark-expr/src/covariance.rs @@ -25,7 +25,7 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{ downcast_value, unwrap_or_internal_err, DataFusionError, Result, ScalarValue, }; diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index 193a90fb5..f1bafa083 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -26,13 +26,13 @@ use arrow::{ record_batch::RecordBatch, }; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::Result; use datafusion_physical_expr::{expressions::CaseExpr, PhysicalExpr}; /// IfExpr is a wrapper around CaseExpr, because `IF(a, b, c)` is semantically equivalent to /// `CASE WHEN a THEN b ELSE c END`. -#[derive(Debug, Hash)] +#[derive(Debug, Hash, Eq)] pub struct IfExpr { if_expr: Arc, true_expr: Arc, @@ -41,6 +41,15 @@ pub struct IfExpr { case_expr: Arc, } +impl PartialEq for IfExpr { + fn eq(&self, other: &Self) -> bool { + self.if_expr.eq(&other.if_expr) + && self.true_expr.eq(&other.true_expr) + && self.false_expr.eq(&other.false_expr) + && self.case_expr.eq(&other.case_expr) + } +} + impl std::fmt::Display for IfExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!( @@ -106,14 +115,6 @@ impl PhysicalExpr for IfExpr { Arc::clone(&children[2]), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.if_expr.hash(&mut s); - self.true_expr.hash(&mut s); - self.false_expr.hash(&mut s); - self.hash(&mut s); - } } impl PartialEq for IfExpr { diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index 7dc17b568..ff823894f 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -26,7 +26,7 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field, FieldRef, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{ cast::{as_int32_array, as_large_list_array, as_list_array}, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, @@ -44,7 +44,7 @@ use std::{ // https://github.com/apache/spark/blob/master/common/utils/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java const MAX_ROUNDED_ARRAY_LENGTH: usize = 2147483632; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct ListExtract { child: Arc, ordinal: Arc, @@ -176,16 +176,6 @@ impl PhysicalExpr for ListExtract { _ => internal_err!("ListExtract should have exactly two children"), } } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.default_value.hash(&mut s); - self.one_based.hash(&mut s); - self.fail_on_error.hash(&mut s); - self.hash(&mut s); - } } fn one_based_index(index: i32, len: usize) -> DataFusionResult> { @@ -288,7 +278,7 @@ impl PartialEq for ListExtract { } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct GetArrayStructFields { child: Arc, ordinal: usize, @@ -380,12 +370,6 @@ impl PhysicalExpr for GetArrayStructFields { } } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.hash(&mut s); - } } fn get_array_struct_fields( @@ -426,7 +410,7 @@ impl PartialEq for GetArrayStructFields { } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct ArrayInsert { src_array_expr: Arc, pos_expr: Arc, @@ -555,15 +539,6 @@ impl PhysicalExpr for ArrayInsert { _ => internal_err!("ArrayInsert should have exactly three childrens"), } } - - fn dyn_hash(&self, _state: &mut dyn Hasher) { - let mut s = _state; - self.src_array_expr.hash(&mut s); - self.pos_expr.hash(&mut s); - self.item_expr.hash(&mut s); - self.legacy_negative_index.hash(&mut s); - self.hash(&mut s); - } } fn array_insert( diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index c5331ad7b..4282f8766 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -29,10 +29,10 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_physical_expr::PhysicalExpr; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct NormalizeNaNAndZero { pub data_type: DataType, pub child: Arc, @@ -90,12 +90,6 @@ impl PhysicalExpr for NormalizeNaNAndZero { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.hash(&mut s); - } } fn eval_typed>(input: T) -> Vec> { diff --git a/native/spark-expr/src/regexp.rs b/native/spark-expr/src/regexp.rs index c7626285a..cfbfc8ae2 100644 --- a/native/spark-expr/src/regexp.rs +++ b/native/spark-expr/src/regexp.rs @@ -21,7 +21,7 @@ use arrow_array::builder::BooleanBuilder; use arrow_array::types::Int32Type; use arrow_array::{Array, BooleanArray, DictionaryArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; @@ -30,6 +30,7 @@ use std::any::Any; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::Arc; +use datafusion::physical_expr_common::physical_expr::DynEq; /// Implementation of RLIKE operator. /// @@ -53,6 +54,16 @@ impl Hash for RLike { } } +impl DynEq for RLike { + fn dyn_eq(&self, other: &dyn Any) -> bool { + if let Some(other) = other.downcast_ref::() { + self.pattern_str == other.pattern_str + } else { + false + } + } +} + impl RLike { pub fn try_new(child: Arc, pattern: &str) -> Result { Ok(Self { @@ -162,9 +173,4 @@ impl PhysicalExpr for RLike { )?)) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - use std::hash::Hash; - let mut s = state; - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/stddev.rs index 3cf604da0..330fe9f42 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -23,8 +23,9 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result, ScalarValue}; +use datafusion_common::types::{LogicalTypeRef, NativeType}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; use datafusion_physical_expr::expressions::StatsType; @@ -57,7 +58,7 @@ impl Stddev { assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), - signature: Signature::coercible(vec![DataType::Float64], Volatility::Immutable), + signature: Signature::coercible(vec![Arc::new(NativeType::Float64)], Volatility::Immutable), expr, stats_type, null_on_divide_by_zero, diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index cda8246d9..3f0fe8fc9 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -19,7 +19,7 @@ use arrow::record_batch::RecordBatch; use arrow_array::{Array, StructArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use std::{ @@ -29,7 +29,7 @@ use std::{ sync::Arc, }; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct CreateNamedStruct { values: Vec>, names: Vec, @@ -96,12 +96,6 @@ impl PhysicalExpr for CreateNamedStruct { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.values.hash(&mut s); - self.names.hash(&mut s); - self.hash(&mut s); - } } impl Display for CreateNamedStruct { @@ -131,7 +125,7 @@ impl PartialEq for CreateNamedStruct { } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct GetStructField { child: Arc, ordinal: usize, @@ -204,12 +198,6 @@ impl PhysicalExpr for GetStructField { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.ordinal.hash(&mut s); - self.hash(&mut s); - } } impl Display for GetStructField { diff --git a/native/spark-expr/src/sum_decimal.rs b/native/spark-expr/src/sum_decimal.rs index ab142aee6..35ddd2185 100644 --- a/native/spark-expr/src/sum_decimal.rs +++ b/native/spark-expr/src/sum_decimal.rs @@ -25,7 +25,7 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, Result as DFResult, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index 91953dd60..6e2b38e84 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -28,7 +28,7 @@ use arrow::{ }; use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; @@ -38,7 +38,7 @@ use crate::kernels::temporal::{ date_trunc_array_fmt_dyn, date_trunc_dyn, timestamp_trunc_array_fmt_dyn, timestamp_trunc_dyn, }; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct HourExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -123,16 +123,9 @@ impl PhysicalExpr for HourExpr { self.timezone.clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct MinuteExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -218,15 +211,9 @@ impl PhysicalExpr for MinuteExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct SecondExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -312,15 +299,9 @@ impl PhysicalExpr for SecondExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct DateTruncExpr { /// An array with DataType::Date32 child: Arc, @@ -399,15 +380,9 @@ impl PhysicalExpr for DateTruncExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.format.hash(&mut s); - self.hash(&mut s); - } } -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct TimestampTruncExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -525,11 +500,4 @@ impl PhysicalExpr for TimestampTruncExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.format.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 1f68eb860..81421f3cf 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -33,7 +33,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; /// to_json function -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct ToJson { /// The input to convert to JSON expr: Arc, @@ -102,12 +102,6 @@ impl PhysicalExpr for ToJson { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.expr.hash(&mut s); - self.timezone.hash(&mut s); - self.hash(&mut s); - } } /// Convert an array into a JSON value string representation diff --git a/native/spark-expr/src/utils.rs b/native/spark-expr/src/utils.rs index 18a2314fb..778a094e4 100644 --- a/native/spark-expr/src/utils.rs +++ b/native/spark-expr/src/utils.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; use arrow_array::{ cast::as_primitive_array, types::{Int32Type, TimestampMicrosecondType}, @@ -29,6 +30,7 @@ use arrow::{ }; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use chrono::{DateTime, Offset, TimeZone}; +use datafusion_physical_expr::PhysicalExpr; /// Preprocesses input arrays to add timezone information from Spark to Arrow array datatype or /// to apply timezone offset. @@ -213,3 +215,17 @@ pub fn unlikely(b: bool) -> bool { } b } + +pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { + if any.is::>() { + any.downcast_ref::>() + .unwrap() + .as_any() + } else if any.is::>() { + any.downcast_ref::>() + .unwrap() + .as_any() + } else { + any + } +} \ No newline at end of file diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index 2f4d8091c..93caa3808 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -22,7 +22,7 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use crate::utils::down_cast_any_ref; use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; From bc19b7442f29c8819f874aeef7a10bffefab1a01 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 09:07:36 -0700 Subject: [PATCH 10/30] save --- .../expressions/bloom_filter_might_contain.rs | 28 +++++-- .../datafusion/expressions/checkoverflow.rs | 30 +++++-- .../datafusion/expressions/negative.rs | 26 +++--- .../datafusion/expressions/strings.rs | 80 ++++++++++++++----- .../datafusion/expressions/subquery.rs | 12 +-- .../datafusion/expressions/unbound.rs | 10 +-- .../core/src/execution/datafusion/planner.rs | 17 ++-- native/spark-expr/src/avg.rs | 2 +- native/spark-expr/src/avg_decimal.rs | 2 +- native/spark-expr/src/bitwise_not.rs | 5 +- native/spark-expr/src/cast.rs | 2 +- native/spark-expr/src/correlation.rs | 2 +- native/spark-expr/src/covariance.rs | 2 +- native/spark-expr/src/if_expr.rs | 2 +- native/spark-expr/src/list.rs | 46 +++++++++-- native/spark-expr/src/normalize_nan.rs | 19 ++++- native/spark-expr/src/regexp.rs | 5 +- native/spark-expr/src/stddev.rs | 9 ++- native/spark-expr/src/structs.rs | 20 ++++- native/spark-expr/src/sum_decimal.rs | 2 +- native/spark-expr/src/temporal.rs | 78 +++++++++++++++--- native/spark-expr/src/to_json.rs | 18 ++++- native/spark-expr/src/utils.rs | 4 +- native/spark-expr/src/variance.rs | 2 +- 24 files changed, 310 insertions(+), 113 deletions(-) diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs index de922d831..3e52e4f18 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs @@ -15,14 +15,16 @@ // specific language governing permissions and limitations // under the License. +use crate::execution::datafusion::expressions::strings::SubstringExpr; use crate::{ execution::datafusion::util::spark_bloom_filter::SparkBloomFilter, parquet::data_type::AsBytes, }; use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion::physical_plan::ColumnarValue; +use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use std::{ @@ -41,6 +43,18 @@ pub struct BloomFilterMightContain { bloom_filter: Option, } +impl DynEq for BloomFilterMightContain { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for BloomFilterMightContain { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl Display for BloomFilterMightContain { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { write!( @@ -144,10 +158,10 @@ impl PhysicalExpr for BloomFilterMightContain { )?)) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.bloom_filter_expr.hash(&mut s); - self.value_expr.hash(&mut s); - self.hash(&mut s); - } + // fn dyn_hash(&self, state: &mut dyn Hasher) { + // let mut s = state; + // self.bloom_filter_expr.hash(&mut s); + // self.value_expr.hash(&mut s); + // self.hash(&mut s); + // } } diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs index e922171bd..e67283dc9 100644 --- a/native/core/src/execution/datafusion/expressions/checkoverflow.rs +++ b/native/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -22,6 +22,7 @@ use std::{ sync::Arc, }; +use crate::execution::datafusion::expressions::strings::SubstringExpr; use arrow::{ array::{as_primitive_array, Array, ArrayRef, Decimal128Array}, datatypes::{Decimal128Type, DecimalType}, @@ -29,7 +30,8 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; @@ -44,6 +46,18 @@ pub struct CheckOverflow { pub fail_on_error: bool, } +impl DynEq for CheckOverflow { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for CheckOverflow { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl CheckOverflow { pub fn new(child: Arc, data_type: DataType, fail_on_error: bool) -> Self { Self { @@ -163,11 +177,11 @@ impl PhysicalExpr for CheckOverflow { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.fail_on_error.hash(&mut s); - self.hash(&mut s); - } + // fn dyn_hash(&self, state: &mut dyn Hasher) { + // let mut s = state; + // self.child.hash(&mut s); + // self.data_type.hash(&mut s); + // self.fail_on_error.hash(&mut s); + // self.hash(&mut s); + // } } diff --git a/native/core/src/execution/datafusion/expressions/negative.rs b/native/core/src/execution/datafusion/expressions/negative.rs index 8dfe71742..c58f4b6a8 100644 --- a/native/core/src/execution/datafusion/expressions/negative.rs +++ b/native/core/src/execution/datafusion/expressions/negative.rs @@ -21,19 +21,16 @@ use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeT use arrow_array::RecordBatch; use arrow_buffer::IntervalDayTime; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, }; +use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_comet_spark_expr::SparkError; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::{any::Any, hash::Hash, sync::Arc}; pub fn create_negate_expr( expr: Arc, @@ -50,6 +47,18 @@ pub struct NegativeExpr { fail_on_error: bool, } +impl DynEq for NegativeExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for NegativeExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + macro_rules! check_overflow { ($array:expr, $array_type:ty, $min_val:expr, $type_name:expr) => {{ let typed_array = $array @@ -205,11 +214,6 @@ impl PhysicalExpr for NegativeExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s); - } - /// Given the child interval of a NegativeExpr, it calculates the NegativeExpr's interval. /// It replaces the upper and lower bounds after multiplying them with -1. /// Ex: `(a, b]` => `[-b, -a)` diff --git a/native/core/src/execution/datafusion/expressions/strings.rs b/native/core/src/execution/datafusion/expressions/strings.rs index 200b4ec5a..6d972c86a 100644 --- a/native/core/src/execution/datafusion/expressions/strings.rs +++ b/native/core/src/execution/datafusion/expressions/strings.rs @@ -17,6 +17,7 @@ #![allow(deprecated)] +use crate::execution::datafusion::expressions::checkoverflow::CheckOverflow; use crate::execution::kernels::strings::{string_space, substring}; use arrow::{ compute::{ @@ -27,7 +28,10 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; +use datafusion_comet_spark_expr::utils::down_cast_any_ref; +use datafusion_comet_spark_expr::ToJson; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; use std::{ @@ -36,7 +40,6 @@ use std::{ hash::{Hash, Hasher}, sync::Arc, }; - macro_rules! make_predicate_function { ($name: ident, $kernel: ident, $str_scalar_kernel: ident) => { #[derive(Debug, Hash)] @@ -57,6 +60,24 @@ macro_rules! make_predicate_function { } } + // impl DynHash for $name { + // fn dyn_hash(&self, state: &mut dyn Hasher) { + // let mut s = state; + // self.left.hash(&mut s); + // self.right.hash(&mut s); + // self.hash(&mut s); + // } + // } + + impl DynEq for $name { + fn dyn_eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| self.left.eq(&x.left) && self.right.eq(&x.right)) + .unwrap_or(false) + } + } + impl PartialEq for $name { fn eq(&self, other: &dyn Any) -> bool { down_cast_any_ref(other) @@ -122,13 +143,6 @@ macro_rules! make_predicate_function { children[1].clone(), ))) } - - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.left.hash(&mut s); - self.right.hash(&mut s); - self.hash(&mut s); - } } }; } @@ -153,6 +167,18 @@ pub struct StringSpaceExpr { pub child: Arc, } +impl DynEq for StringSpaceExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for StringSpaceExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl SubstringExpr { pub fn new(child: Arc, start: i64, len: u64) -> Self { Self { child, start, len } @@ -190,6 +216,18 @@ impl PartialEq for SubstringExpr { } } +impl DynEq for SubstringExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for SubstringExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl PhysicalExpr for SubstringExpr { fn as_any(&self) -> &dyn Any { self @@ -232,13 +270,13 @@ impl PhysicalExpr for SubstringExpr { ))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.start.hash(&mut s); - self.len.hash(&mut s); - self.hash(&mut s); - } + // fn dyn_hash(&self, state: &mut dyn Hasher) { + // let mut s = state; + // self.child.hash(&mut s); + // self.start.hash(&mut s); + // self.len.hash(&mut s); + // self.hash(&mut s); + // } } impl PartialEq for StringSpaceExpr { @@ -293,9 +331,9 @@ impl PhysicalExpr for StringSpaceExpr { Ok(Arc::new(StringSpaceExpr::new(Arc::clone(&children[0])))) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.hash(&mut s); - } + // fn dyn_hash(&self, state: &mut dyn Hasher) { + // let mut s = state; + // self.child.hash(&mut s); + // self.hash(&mut s); + // } } diff --git a/native/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/datafusion/expressions/subquery.rs index 3eeb29c16..3be7df9d2 100644 --- a/native/core/src/execution/datafusion/expressions/subquery.rs +++ b/native/core/src/execution/datafusion/expressions/subquery.rs @@ -22,7 +22,7 @@ use crate::{ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema, TimeUnit}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; +use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use jni::{ @@ -36,7 +36,7 @@ use std::{ sync::Arc, }; -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct Subquery { /// The ID of the execution context that owns this subquery. We use this ID to retrieve the /// subquery result. @@ -210,8 +210,8 @@ impl PhysicalExpr for Subquery { Ok(self) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s) - } + // fn dyn_hash(&self, state: &mut dyn Hasher) { + // let mut s = state; + // self.hash(&mut s) + // } } diff --git a/native/core/src/execution/datafusion/expressions/unbound.rs b/native/core/src/execution/datafusion/expressions/unbound.rs index a6babd0f7..f0395103c 100644 --- a/native/core/src/execution/datafusion/expressions/unbound.rs +++ b/native/core/src/execution/datafusion/expressions/unbound.rs @@ -17,8 +17,8 @@ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::down_cast_any_ref; use datafusion::physical_plan::ColumnarValue; +use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result}; use datafusion_physical_expr::PhysicalExpr; use std::{ @@ -94,10 +94,10 @@ impl PhysicalExpr for UnboundColumn { Ok(self) } - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.hash(&mut s); - } + // fn dyn_hash(&self, state: &mut dyn Hasher) { + // let mut s = state; + // self.hash(&mut s); + // } } impl PartialEq for UnboundColumn { diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index a83dba5d6..ce0283109 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -100,7 +100,6 @@ use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter}, JoinType as DFJoinType, ScalarValue, }; -use datafusion_expr::expr::find_df_window_func; use datafusion_expr::{ AggregateUDF, ScalarUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, @@ -1932,7 +1931,7 @@ impl PhysicalPlanner { window_func_name, &window_args, partition_by, - sort_exprs, + &LexOrdering::new(sort_exprs.to_vec()), window_frame.into(), input_schema.as_ref(), false, // TODO: Ignore nulls @@ -1982,15 +1981,11 @@ impl PhysicalPlanner { /// Find DataFusion's built-in window function by name. fn find_df_window_function(&self, name: &str) -> Option { - if let Some(f) = find_df_window_func(name) { - Some(f) - } else { - let registry = &self.session_ctx.state(); - registry - .udaf(name) - .map(WindowFunctionDefinition::AggregateUDF) - .ok() - } + let registry = &self.session_ctx.state(); + registry + .udaf(name) + .map(WindowFunctionDefinition::AggregateUDF) + .ok() } /// Create a DataFusion physical partitioning from Spark physical partitioning diff --git a/native/spark-expr/src/avg.rs b/native/spark-expr/src/avg.rs index befada923..751676b73 100644 --- a/native/spark-expr/src/avg.rs +++ b/native/spark-expr/src/avg.rs @@ -30,8 +30,8 @@ use datafusion_common::{not_impl_err, Result, ScalarValue}; use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; use std::{any::Any, sync::Arc}; -use arrow_array::ArrowNativeTypeOp; use crate::utils::down_cast_any_ref; +use arrow_array::ArrowNativeTypeOp; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; diff --git a/native/spark-expr/src/avg_decimal.rs b/native/spark-expr/src/avg_decimal.rs index 47d6a9a6f..1fe2fa2fb 100644 --- a/native/spark-expr/src/avg_decimal.rs +++ b/native/spark-expr/src/avg_decimal.rs @@ -28,11 +28,11 @@ use datafusion_common::{not_impl_err, Result, ScalarValue}; use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; use std::{any::Any, sync::Arc}; +use crate::utils::down_cast_any_ref; use crate::utils::is_valid_decimal_precision; use arrow_array::ArrowNativeTypeOp; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use datafusion::logical_expr::Volatility::Immutable; -use crate::utils::down_cast_any_ref; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::avg_return_type; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF}; diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index 716e38bb2..a78915faf 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -21,16 +21,16 @@ use std::{ sync::Arc, }; +use crate::utils::down_cast_any_ref; +use crate::IfExpr; use arrow::{ array::*, datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use crate::utils::down_cast_any_ref; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; -use crate::IfExpr; macro_rules! compute_op { ($OPERAND:expr, $DT:ident) => {{ @@ -121,7 +121,6 @@ impl PhysicalExpr for BitwiseNotExpr { ) -> Result> { Ok(Arc::new(BitwiseNotExpr::new(Arc::clone(&children[0])))) } - } impl PartialEq for BitwiseNotExpr { diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 5b886893f..1d616332e 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -49,9 +49,9 @@ use std::{ sync::Arc, }; +use crate::utils::down_cast_any_ref; use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; use datafusion::physical_expr_common::physical_expr::DynEq; -use crate::utils::down_cast_any_ref; use num::{ cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, ToPrimitive, diff --git a/native/spark-expr/src/correlation.rs b/native/spark-expr/src/correlation.rs index ef8480f35..b656e4b1d 100644 --- a/native/spark-expr/src/correlation.rs +++ b/native/spark-expr/src/correlation.rs @@ -21,12 +21,12 @@ use std::{any::Any, sync::Arc}; use crate::covariance::CovarianceAccumulator; use crate::stddev::StddevAccumulator; +use crate::utils::down_cast_any_ref; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use crate::utils::down_cast_any_ref; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; diff --git a/native/spark-expr/src/covariance.rs b/native/spark-expr/src/covariance.rs index 56d8e231c..4231be147 100644 --- a/native/spark-expr/src/covariance.rs +++ b/native/spark-expr/src/covariance.rs @@ -19,13 +19,13 @@ use std::{any::Any, sync::Arc}; +use crate::utils::down_cast_any_ref; use arrow::{ array::{ArrayRef, Float64Array}, compute::cast, datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use crate::utils::down_cast_any_ref; use datafusion_common::{ downcast_value, unwrap_or_internal_err, DataFusionError, Result, ScalarValue, }; diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index f1bafa083..8091a2f2b 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -21,12 +21,12 @@ use std::{ sync::Arc, }; +use crate::utils::down_cast_any_ref; use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; use datafusion::logical_expr::ColumnarValue; -use crate::utils::down_cast_any_ref; use datafusion_common::Result; use datafusion_physical_expr::{expressions::CaseExpr, PhysicalExpr}; diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index ff823894f..ca655b869 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::down_cast_any_ref; use arrow::{ array::{as_primitive_array, Capacities, MutableArrayData}, buffer::{NullBuffer, OffsetBuffer}, @@ -26,7 +27,7 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field, FieldRef, Schema}; use datafusion::logical_expr::ColumnarValue; -use crate::utils::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_common::{ cast::{as_int32_array, as_large_list_array, as_list_array}, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, @@ -44,7 +45,7 @@ use std::{ // https://github.com/apache/spark/blob/master/common/utils/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java const MAX_ROUNDED_ARRAY_LENGTH: usize = 2147483632; -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct ListExtract { child: Arc, ordinal: Arc, @@ -53,6 +54,18 @@ pub struct ListExtract { fail_on_error: bool, } +impl DynEq for ListExtract { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for ListExtract { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl ListExtract { pub fn new( child: Arc, @@ -278,12 +291,24 @@ impl PartialEq for ListExtract { } } -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct GetArrayStructFields { child: Arc, ordinal: usize, } +impl DynEq for GetArrayStructFields { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for GetArrayStructFields { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl GetArrayStructFields { pub fn new(child: Arc, ordinal: usize) -> Self { Self { child, ordinal } @@ -369,7 +394,6 @@ impl PhysicalExpr for GetArrayStructFields { _ => internal_err!("GetArrayStructFields should have exactly one child"), } } - } fn get_array_struct_fields( @@ -410,7 +434,7 @@ impl PartialEq for GetArrayStructFields { } } -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct ArrayInsert { src_array_expr: Arc, pos_expr: Arc, @@ -418,6 +442,18 @@ pub struct ArrayInsert { legacy_negative_index: bool, } +impl DynEq for ArrayInsert { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for ArrayInsert { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl ArrayInsert { pub fn new( src_array_expr: Arc, diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index 4282f8766..2ebe23ade 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -22,6 +22,8 @@ use std::{ sync::Arc, }; +use crate::utils::down_cast_any_ref; +use crate::GetStructField; use arrow::{ array::{as_primitive_array, ArrayAccessor, ArrayIter, Float32Array, Float64Array}, datatypes::{ArrowNativeType, Float32Type, Float64Type}, @@ -29,15 +31,27 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use crate::utils::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_physical_expr::PhysicalExpr; -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct NormalizeNaNAndZero { pub data_type: DataType, pub child: Arc, } +impl DynEq for NormalizeNaNAndZero { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for NormalizeNaNAndZero { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl NormalizeNaNAndZero { pub fn new(data_type: DataType, child: Arc) -> Self { Self { data_type, child } @@ -89,7 +103,6 @@ impl PhysicalExpr for NormalizeNaNAndZero { Arc::clone(&children[0]), ))) } - } fn eval_typed>(input: T) -> Vec> { diff --git a/native/spark-expr/src/regexp.rs b/native/spark-expr/src/regexp.rs index cfbfc8ae2..5679812d7 100644 --- a/native/spark-expr/src/regexp.rs +++ b/native/spark-expr/src/regexp.rs @@ -15,13 +15,14 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::down_cast_any_ref; use crate::SparkError; use arrow::compute::take; use arrow_array::builder::BooleanBuilder; use arrow_array::types::Int32Type; use arrow_array::{Array, BooleanArray, DictionaryArray, RecordBatch, StringArray}; use arrow_schema::{DataType, Schema}; -use crate::utils::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; @@ -30,7 +31,6 @@ use std::any::Any; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use datafusion::physical_expr_common::physical_expr::DynEq; /// Implementation of RLIKE operator. /// @@ -172,5 +172,4 @@ impl PhysicalExpr for RLike { &self.pattern_str, )?)) } - } diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/stddev.rs index 330fe9f42..c38145bc1 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -17,15 +17,15 @@ use std::{any::Any, sync::Arc}; +use crate::utils::down_cast_any_ref; use crate::variance::VarianceAccumulator; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use crate::utils::down_cast_any_ref; -use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_common::types::{LogicalTypeRef, NativeType}; +use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; use datafusion_physical_expr::expressions::StatsType; @@ -58,7 +58,10 @@ impl Stddev { assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), - signature: Signature::coercible(vec![Arc::new(NativeType::Float64)], Volatility::Immutable), + signature: Signature::coercible( + vec![Arc::new(NativeType::Float64)], + Volatility::Immutable, + ), expr, stats_type, null_on_divide_by_zero, diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index 3f0fe8fc9..190011b54 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -15,11 +15,13 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::down_cast_any_ref; +use crate::ArrayInsert; use arrow::record_batch::RecordBatch; use arrow_array::{Array, StructArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::logical_expr::ColumnarValue; -use crate::utils::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use std::{ @@ -95,7 +97,6 @@ impl PhysicalExpr for CreateNamedStruct { self.names.clone(), ))) } - } impl Display for CreateNamedStruct { @@ -125,12 +126,24 @@ impl PartialEq for CreateNamedStruct { } } -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct GetStructField { child: Arc, ordinal: usize, } +impl DynEq for GetStructField { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for GetStructField { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl GetStructField { pub fn new(child: Arc, ordinal: usize) -> Self { Self { child, ordinal } @@ -197,7 +210,6 @@ impl PhysicalExpr for GetStructField { self.ordinal, ))) } - } impl Display for GetStructField { diff --git a/native/spark-expr/src/sum_decimal.rs b/native/spark-expr/src/sum_decimal.rs index 35ddd2185..ceda6755a 100644 --- a/native/spark-expr/src/sum_decimal.rs +++ b/native/spark-expr/src/sum_decimal.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::down_cast_any_ref; use crate::utils::{is_valid_decimal_precision, unlikely}; use arrow::{ array::BooleanBufferBuilder, @@ -25,7 +26,6 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator}; -use crate::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, Result as DFResult, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index 6e2b38e84..0423e378c 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -22,13 +22,14 @@ use std::{ sync::Arc, }; +use crate::utils::down_cast_any_ref; use arrow::{ compute::{date_part, DatePart}, record_batch::RecordBatch, }; use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; use datafusion::logical_expr::ColumnarValue; -use crate::utils::down_cast_any_ref; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; @@ -37,14 +38,27 @@ use crate::utils::array_with_timezone; use crate::kernels::temporal::{ date_trunc_array_fmt_dyn, date_trunc_dyn, timestamp_trunc_array_fmt_dyn, timestamp_trunc_dyn, }; +use crate::NormalizeNaNAndZero; -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct HourExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl DynEq for HourExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for HourExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl HourExpr { pub fn new(child: Arc, timezone: String) -> Self { HourExpr { child, timezone } @@ -125,13 +139,25 @@ impl PhysicalExpr for HourExpr { } } -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct MinuteExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl DynEq for MinuteExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for MinuteExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl MinuteExpr { pub fn new(child: Arc, timezone: String) -> Self { MinuteExpr { child, timezone } @@ -210,16 +236,27 @@ impl PhysicalExpr for MinuteExpr { self.timezone.clone(), ))) } - } -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct SecondExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } +impl DynEq for SecondExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for SecondExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl SecondExpr { pub fn new(child: Arc, timezone: String) -> Self { SecondExpr { child, timezone } @@ -298,10 +335,9 @@ impl PhysicalExpr for SecondExpr { self.timezone.clone(), ))) } - } -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct DateTruncExpr { /// An array with DataType::Date32 child: Arc, @@ -309,6 +345,18 @@ pub struct DateTruncExpr { format: Arc, } +impl DynEq for DateTruncExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for DateTruncExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl DateTruncExpr { pub fn new(child: Arc, format: Arc) -> Self { DateTruncExpr { child, format } @@ -379,10 +427,9 @@ impl PhysicalExpr for DateTruncExpr { Arc::clone(&self.format), ))) } - } -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct TimestampTruncExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -397,6 +444,18 @@ pub struct TimestampTruncExpr { timezone: String, } +impl DynEq for TimestampTruncExpr { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for TimestampTruncExpr { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl TimestampTruncExpr { pub fn new( child: Arc, @@ -499,5 +558,4 @@ impl PhysicalExpr for TimestampTruncExpr { self.timezone.clone(), ))) } - } diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 81421f3cf..64e318459 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -20,10 +20,11 @@ // being able to specify Spark-compatible cast from all types to string) use crate::cast::SparkCastOptions; -use crate::{spark_cast, EvalMode}; +use crate::{spark_cast, EvalMode, TimestampTruncExpr}; use arrow_array::builder::StringBuilder; use arrow_array::{Array, ArrayRef, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; +use datafusion::physical_expr_common::physical_expr::DynEq; use datafusion_common::Result; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; @@ -33,7 +34,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; /// to_json function -#[derive(Debug, Hash, PartialEq, Eq)] +#[derive(Debug, Hash)] pub struct ToJson { /// The input to convert to JSON expr: Arc, @@ -41,6 +42,18 @@ pub struct ToJson { timezone: String, } +impl DynEq for ToJson { + fn dyn_eq(&self, other: &dyn Any) -> bool { + todo!() + } +} + +impl PartialEq for ToJson { + fn eq(&self, other: &Self) -> bool { + todo!() + } +} + impl ToJson { pub fn new(expr: Arc, timezone: &str) -> Self { Self { @@ -101,7 +114,6 @@ impl PhysicalExpr for ToJson { &self.timezone, ))) } - } /// Convert an array into a JSON value string representation diff --git a/native/spark-expr/src/utils.rs b/native/spark-expr/src/utils.rs index 778a094e4..690106517 100644 --- a/native/spark-expr/src/utils.rs +++ b/native/spark-expr/src/utils.rs @@ -15,12 +15,12 @@ // specific language governing permissions and limitations // under the License. -use std::any::Any; use arrow_array::{ cast::as_primitive_array, types::{Int32Type, TimestampMicrosecondType}, }; use arrow_schema::{ArrowError, DataType, DECIMAL128_MAX_PRECISION}; +use std::any::Any; use std::sync::Arc; use crate::timezone::Tz; @@ -228,4 +228,4 @@ pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { } else { any } -} \ No newline at end of file +} diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index 93caa3808..fa16fac12 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -17,12 +17,12 @@ use std::{any::Any, sync::Arc}; +use crate::utils::down_cast_any_ref; use arrow::{ array::{ArrayRef, Float64Array}, datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use crate::utils::down_cast_any_ref; use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; From 2731c7e77ce05a357d8f0b153bd4fd25b8958d94 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 15:02:03 -0700 Subject: [PATCH 11/30] remove unused imports --- native/spark-expr/src/bitwise_not.rs | 7 +------ native/spark-expr/src/cast.rs | 3 +-- native/spark-expr/src/if_expr.rs | 6 +----- native/spark-expr/src/list.rs | 2 +- native/spark-expr/src/normalize_nan.rs | 3 +-- native/spark-expr/src/stddev.rs | 2 +- native/spark-expr/src/structs.rs | 2 +- native/spark-expr/src/temporal.rs | 2 +- native/spark-expr/src/to_json.rs | 2 +- 9 files changed, 9 insertions(+), 20 deletions(-) diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index a78915faf..677bdb596 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -15,14 +15,9 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::{any::Any, hash::Hash, sync::Arc}; use crate::utils::down_cast_any_ref; -use crate::IfExpr; use arrow::{ array::*, datatypes::{DataType, Schema}, diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 1d616332e..471242f9e 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -51,7 +51,6 @@ use std::{ use crate::utils::down_cast_any_ref; use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::physical_expr_common::physical_expr::DynEq; use num::{ cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, ToPrimitive, @@ -1502,7 +1501,7 @@ impl PhysicalExpr for Cast { } impl PartialEq for Cast { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index 8091a2f2b..a0b01bb61 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -15,11 +15,7 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::{any::Any, hash::Hash, sync::Arc}; use crate::utils::down_cast_any_ref; use arrow::{ diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index ca655b869..a313f88da 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -36,7 +36,7 @@ use datafusion_physical_expr::PhysicalExpr; use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index 2ebe23ade..0b9a5df8a 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -18,12 +18,11 @@ use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; use crate::utils::down_cast_any_ref; -use crate::GetStructField; use arrow::{ array::{as_primitive_array, ArrayAccessor, ArrayIter, Float32Array, Float64Array}, datatypes::{ArrowNativeType, Float32Type, Float64Type}, diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/stddev.rs index c38145bc1..898a2c0c4 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -24,7 +24,7 @@ use arrow::{ datatypes::{DataType, Field}, }; use datafusion::logical_expr::Accumulator; -use datafusion_common::types::{LogicalTypeRef, NativeType}; +use datafusion_common::types::NativeType; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index 190011b54..3c450ab85 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -27,7 +27,7 @@ use datafusion_physical_expr::PhysicalExpr; use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index 0423e378c..9cddae350 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -18,7 +18,7 @@ use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 64e318459..1dca149c8 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -20,7 +20,7 @@ // being able to specify Spark-compatible cast from all types to string) use crate::cast::SparkCastOptions; -use crate::{spark_cast, EvalMode, TimestampTruncExpr}; +use crate::{spark_cast, EvalMode}; use arrow_array::builder::StringBuilder; use arrow_array::{Array, ArrayRef, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; From 911a0b3bb96d5753e30af5dcfa207a221676da12 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 15:20:39 -0700 Subject: [PATCH 12/30] clippy --- .../expressions/bloom_filter_might_contain.rs | 19 +++---- .../datafusion/expressions/checkoverflow.rs | 24 ++++---- .../datafusion/expressions/negative.rs | 13 +++-- .../datafusion/expressions/strings.rs | 25 ++++----- .../datafusion/expressions/subquery.rs | 2 +- .../datafusion/expressions/unbound.rs | 6 +- .../datafusion/util/spark_bit_array.rs | 2 +- .../datafusion/util/spark_bloom_filter.rs | 2 +- native/core/src/execution/jni_api.rs | 9 ++- native/spark-expr/src/bitwise_not.rs | 13 ++++- native/spark-expr/src/cast.rs | 31 ++++++---- native/spark-expr/src/if_expr.rs | 13 ++++- native/spark-expr/src/list.rs | 28 +++++----- native/spark-expr/src/normalize_nan.rs | 29 ++++++---- native/spark-expr/src/structs.rs | 12 ++-- native/spark-expr/src/temporal.rs | 56 +++++++++---------- native/spark-expr/src/to_json.rs | 12 ++-- 17 files changed, 156 insertions(+), 140 deletions(-) diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs index 3e52e4f18..b98f0ca0f 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs @@ -15,42 +15,37 @@ // specific language governing permissions and limitations // under the License. -use crate::execution::datafusion::expressions::strings::SubstringExpr; use crate::{ execution::datafusion::util::spark_bloom_filter::SparkBloomFilter, parquet::data_type::AsBytes, }; use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion::physical_plan::ColumnarValue; use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; -use std::{ - any::Any, - fmt::Display, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::hash::Hasher; +use std::{any::Any, fmt::Display, sync::Arc}; /// A physical expression that checks if a value might be in a bloom filter. It corresponds to the /// Spark's `BloomFilterMightContain` expression. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct BloomFilterMightContain { pub bloom_filter_expr: Arc, pub value_expr: Arc, bloom_filter: Option, } -impl DynEq for BloomFilterMightContain { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for BloomFilterMightContain { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for BloomFilterMightContain { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs index e67283dc9..ef4facdad 100644 --- a/native/core/src/execution/datafusion/expressions/checkoverflow.rs +++ b/native/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -15,14 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Display, Formatter}, - hash::{Hash, Hasher}, - sync::Arc, -}; - -use crate::execution::datafusion::expressions::strings::SubstringExpr; use arrow::{ array::{as_primitive_array, Array, ArrayRef, Decimal128Array}, datatypes::{Decimal128Type, DecimalType}, @@ -30,30 +22,36 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hasher; +use std::{ + any::Any, + fmt::{Display, Formatter}, + sync::Arc, +}; /// This is from Spark `CheckOverflow` expression. Spark `CheckOverflow` expression rounds decimals /// to given scale and check if the decimals can fit in given precision. As `cast` kernel rounds /// decimals already, Comet `CheckOverflow` expression only checks if the decimals can fit in the /// precision. -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct CheckOverflow { pub child: Arc, pub data_type: DataType, pub fail_on_error: bool, } -impl DynEq for CheckOverflow { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for CheckOverflow { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for CheckOverflow { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/core/src/execution/datafusion/expressions/negative.rs b/native/core/src/execution/datafusion/expressions/negative.rs index c58f4b6a8..7cd66c152 100644 --- a/native/core/src/execution/datafusion/expressions/negative.rs +++ b/native/core/src/execution/datafusion/expressions/negative.rs @@ -21,7 +21,7 @@ use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeT use arrow_array::RecordBatch; use arrow_buffer::IntervalDayTime; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, @@ -30,7 +30,8 @@ use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_comet_spark_expr::SparkError; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; -use std::{any::Any, hash::Hash, sync::Arc}; +use std::hash::Hasher; +use std::{any::Any, sync::Arc}; pub fn create_negate_expr( expr: Arc, @@ -40,21 +41,21 @@ pub fn create_negate_expr( } /// Negative expression -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct NegativeExpr { /// Input expression arg: Arc, fail_on_error: bool, } -impl DynEq for NegativeExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for NegativeExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for NegativeExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/core/src/execution/datafusion/expressions/strings.rs b/native/core/src/execution/datafusion/expressions/strings.rs index 6d972c86a..3aef4cd69 100644 --- a/native/core/src/execution/datafusion/expressions/strings.rs +++ b/native/core/src/execution/datafusion/expressions/strings.rs @@ -17,7 +17,6 @@ #![allow(deprecated)] -use crate::execution::datafusion::expressions::checkoverflow::CheckOverflow; use crate::execution::kernels::strings::{string_space, substring}; use arrow::{ compute::{ @@ -28,18 +27,18 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynEq; -use datafusion::physical_expr_common::physical_expr::DynHash; +use datafusion::physical_expr_common::physical_expr::{DynEq, DynHash}; use datafusion_comet_spark_expr::utils::down_cast_any_ref; -use datafusion_comet_spark_expr::ToJson; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hasher; use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; + macro_rules! make_predicate_function { ($name: ident, $kernel: ident, $str_scalar_kernel: ident) => { #[derive(Debug, Hash)] @@ -155,26 +154,26 @@ make_predicate_function!(EndsWith, ends_with_dyn, ends_with_utf8_scalar_dyn); make_predicate_function!(Contains, contains_dyn, contains_utf8_scalar_dyn); -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct SubstringExpr { pub child: Arc, pub start: i64, pub len: u64, } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct StringSpaceExpr { pub child: Arc, } -impl DynEq for StringSpaceExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for StringSpaceExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for StringSpaceExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } @@ -216,14 +215,14 @@ impl PartialEq for SubstringExpr { } } -impl DynEq for SubstringExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for SubstringExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for SubstringExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/datafusion/expressions/subquery.rs index 3be7df9d2..a3eaa8c5c 100644 --- a/native/core/src/execution/datafusion/expressions/subquery.rs +++ b/native/core/src/execution/datafusion/expressions/subquery.rs @@ -32,7 +32,7 @@ use jni::{ use std::{ any::Any, fmt::{Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, sync::Arc, }; diff --git a/native/core/src/execution/datafusion/expressions/unbound.rs b/native/core/src/execution/datafusion/expressions/unbound.rs index f0395103c..b0f239d89 100644 --- a/native/core/src/execution/datafusion/expressions/unbound.rs +++ b/native/core/src/execution/datafusion/expressions/unbound.rs @@ -21,11 +21,7 @@ use datafusion::physical_plan::ColumnarValue; use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result}; use datafusion_physical_expr::PhysicalExpr; -use std::{ - any::Any, - hash::{Hash, Hasher}, - sync::Arc, -}; +use std::{any::Any, hash::Hash, sync::Arc}; /// This is similar to `UnKnownColumn` in DataFusion, but it has data type. /// This is only used when the column is not bound to a schema, for example, the diff --git a/native/core/src/execution/datafusion/util/spark_bit_array.rs b/native/core/src/execution/datafusion/util/spark_bit_array.rs index 68b97d660..923df014f 100644 --- a/native/core/src/execution/datafusion/util/spark_bit_array.rs +++ b/native/core/src/execution/datafusion/util/spark_bit_array.rs @@ -22,7 +22,7 @@ use std::iter::zip; /// A simple bit array implementation that simulates the behavior of Spark's BitArray which is /// used in the BloomFilter implementation. Some methods are not implemented as they are not /// required for the current use case. -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct SparkBitArray { data: Vec, bit_count: usize, diff --git a/native/core/src/execution/datafusion/util/spark_bloom_filter.rs b/native/core/src/execution/datafusion/util/spark_bloom_filter.rs index 35fa23b46..f42a148c6 100644 --- a/native/core/src/execution/datafusion/util/spark_bloom_filter.rs +++ b/native/core/src/execution/datafusion/util/spark_bloom_filter.rs @@ -27,7 +27,7 @@ const SPARK_BLOOM_FILTER_VERSION_1: i32 = 1; /// A Bloom filter implementation that simulates the behavior of Spark's BloomFilter. /// It's not a complete implementation of Spark's BloomFilter, but just add the minimum /// methods to support mightContainsLong in the native side. -#[derive(Debug, Hash)] +#[derive(Debug, Hash, PartialEq, Eq)] pub struct SparkBloomFilter { bits: SparkBitArray, num_hash_functions: u32, diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 8afe134cd..8dab0088f 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -20,10 +20,7 @@ use arrow::datatypes::DataType as ArrowDataType; use arrow_array::RecordBatch; use datafusion::{ - execution::{ - disk_manager::DiskManagerConfig, - runtime_env::{RuntimeConfig, RuntimeEnv}, - }, + execution::{disk_manager::DiskManagerConfig, runtime_env::RuntimeEnv}, physical_plan::{display::DisplayableExecutionPlan, SendableRecordBatchStream}, prelude::{SessionConfig, SessionContext}, }; @@ -52,6 +49,7 @@ use crate::{ }; use datafusion_comet_proto::spark_operator::Operator; use datafusion_common::ScalarValue; +use datafusion_execution::runtime_env::RuntimeEnvBuilder; use futures::stream::StreamExt; use jni::{ objects::GlobalRef, @@ -176,7 +174,7 @@ fn prepare_datafusion_session_context( batch_size: usize, comet_task_memory_manager: Arc, ) -> CometResult { - let mut rt_config = RuntimeConfig::new().with_disk_manager(DiskManagerConfig::NewOs); + let mut rt_config = RuntimeEnvBuilder::new().with_disk_manager(DiskManagerConfig::NewOs); // Set Comet memory pool for native let memory_pool = CometMemoryPool::new(comet_task_memory_manager); @@ -198,6 +196,7 @@ fn prepare_datafusion_session_context( &ScalarValue::Float64(Some(1.1)), ); + #[allow(deprecated)] let runtime = RuntimeEnv::try_new(rt_config)?; let mut session_ctx = SessionContext::new_with_config_rt(session_config, Arc::new(runtime)); diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index 677bdb596..818064554 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -15,17 +15,18 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, hash::Hash, sync::Arc}; - use crate::utils::down_cast_any_ref; use arrow::{ array::*, datatypes::{DataType, Schema}, record_batch::RecordBatch, }; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hasher; +use std::{any::Any, sync::Arc}; macro_rules! compute_op { ($OPERAND:expr, $DT:ident) => {{ @@ -39,12 +40,18 @@ macro_rules! compute_op { } /// BitwiseNot expression -#[derive(Debug, Hash, Eq)] +#[derive(Debug, Eq)] pub struct BitwiseNotExpr { /// Input expression arg: Arc, } +impl DynHash for BitwiseNotExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { + todo!() + } +} + impl PartialEq for BitwiseNotExpr { fn eq(&self, other: &Self) -> bool { self.arg.eq(&other.arg) diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 471242f9e..8cfeada0e 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::down_cast_any_ref; use arrow::{ array::{ cast::AsArray, @@ -35,28 +36,28 @@ use arrow::{ use arrow_array::builder::StringBuilder; use arrow_array::{DictionaryArray, StringArray, StructArray}; use arrow_schema::{DataType, Field, Schema}; +use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{ cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, }; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; +use num::{ + cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, + ToPrimitive, +}; +use regex::Regex; +use std::hash::Hasher; use std::str::FromStr; use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::{Hash, Hasher}, + hash::Hash, num::Wrapping, sync::Arc, }; -use crate::utils::down_cast_any_ref; -use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use num::{ - cast::AsPrimitive, integer::div_floor, traits::CheckedNeg, CheckedSub, Integer, Num, - ToPrimitive, -}; -use regex::Regex; - use crate::timezone; use crate::utils::array_with_timezone; @@ -134,7 +135,7 @@ impl TimeStampInfo { } } -#[derive(Debug, Hash, Eq)] +#[derive(Debug, Eq)] pub struct Cast { pub child: Arc, pub data_type: DataType, @@ -1450,6 +1451,16 @@ impl Display for Cast { } } +impl DynHash for Cast { + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.type_id().hash(&mut s); + self.child.hash(&mut s); + self.data_type.hash(&mut s); + self.cast_options.hash(&mut s); + } +} + impl PartialEq for Cast { fn eq(&self, other: &dyn Any) -> bool { down_cast_any_ref(other) diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index a0b01bb61..d6eeffb4b 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -15,20 +15,21 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, hash::Hash, sync::Arc}; - use crate::utils::down_cast_any_ref; use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; use datafusion::logical_expr::ColumnarValue; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::Result; use datafusion_physical_expr::{expressions::CaseExpr, PhysicalExpr}; +use std::hash::Hasher; +use std::{any::Any, sync::Arc}; /// IfExpr is a wrapper around CaseExpr, because `IF(a, b, c)` is semantically equivalent to /// `CASE WHEN a THEN b ELSE c END`. -#[derive(Debug, Hash, Eq)] +#[derive(Debug, Eq)] pub struct IfExpr { if_expr: Arc, true_expr: Arc, @@ -37,6 +38,12 @@ pub struct IfExpr { case_expr: Arc, } +impl DynHash for IfExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { + todo!() + } +} + impl PartialEq for IfExpr { fn eq(&self, other: &Self) -> bool { self.if_expr.eq(&other.if_expr) diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index a313f88da..7445784d4 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -27,16 +27,16 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field, FieldRef, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{ cast::{as_int32_array, as_large_list_array, as_list_array}, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, }; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hasher; use std::{ any::Any, fmt::{Debug, Display, Formatter}, - hash::Hash, sync::Arc, }; @@ -45,7 +45,7 @@ use std::{ // https://github.com/apache/spark/blob/master/common/utils/src/main/java/org/apache/spark/unsafe/array/ByteArrayUtils.java const MAX_ROUNDED_ARRAY_LENGTH: usize = 2147483632; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ListExtract { child: Arc, ordinal: Arc, @@ -54,14 +54,14 @@ pub struct ListExtract { fail_on_error: bool, } -impl DynEq for ListExtract { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for ListExtract { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for ListExtract { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } @@ -291,20 +291,20 @@ impl PartialEq for ListExtract { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct GetArrayStructFields { child: Arc, ordinal: usize, } -impl DynEq for GetArrayStructFields { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for GetArrayStructFields { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for GetArrayStructFields { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } @@ -434,7 +434,7 @@ impl PartialEq for GetArrayStructFields { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ArrayInsert { src_array_expr: Arc, pos_expr: Arc, @@ -442,14 +442,14 @@ pub struct ArrayInsert { legacy_negative_index: bool, } -impl DynEq for ArrayInsert { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for ArrayInsert { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for ArrayInsert { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index 0b9a5df8a..5ead1489b 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -15,13 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Display, Formatter}, - hash::Hash, - sync::Arc, -}; - use crate::utils::down_cast_any_ref; use arrow::{ array::{as_primitive_array, ArrayAccessor, ArrayIter, Float32Array, Float64Array}, @@ -30,23 +23,35 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hasher; +use std::{ + any::Any, + fmt::{Display, Formatter}, + sync::Arc, +}; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct NormalizeNaNAndZero { pub data_type: DataType, pub child: Arc, } -impl DynEq for NormalizeNaNAndZero { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for NormalizeNaNAndZero { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } +// impl DynEq for NormalizeNaNAndZero { +// fn dyn_eq(&self, _other: &dyn Any) -> bool { +// todo!() +// } +// } + impl PartialEq for NormalizeNaNAndZero { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index 3c450ab85..48ef43785 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -16,14 +16,14 @@ // under the License. use crate::utils::down_cast_any_ref; -use crate::ArrayInsert; use arrow::record_batch::RecordBatch; use arrow_array::{Array, StructArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hasher; use std::{ any::Any, fmt::{Display, Formatter}, @@ -126,20 +126,20 @@ impl PartialEq for CreateNamedStruct { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct GetStructField { child: Arc, ordinal: usize, } -impl DynEq for GetStructField { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for GetStructField { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for GetStructField { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index 9cddae350..4c5afa6b4 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -15,13 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::{ - any::Any, - fmt::{Debug, Display, Formatter}, - hash::Hash, - sync::Arc, -}; - use crate::utils::down_cast_any_ref; use arrow::{ compute::{date_part, DatePart}, @@ -29,32 +22,37 @@ use arrow::{ }; use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; +use std::hash::Hasher; +use std::{ + any::Any, + fmt::{Debug, Display, Formatter}, + sync::Arc, +}; use crate::utils::array_with_timezone; use crate::kernels::temporal::{ date_trunc_array_fmt_dyn, date_trunc_dyn, timestamp_trunc_array_fmt_dyn, timestamp_trunc_dyn, }; -use crate::NormalizeNaNAndZero; -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct HourExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } -impl DynEq for HourExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for HourExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for HourExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } @@ -139,21 +137,21 @@ impl PhysicalExpr for HourExpr { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct MinuteExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } -impl DynEq for MinuteExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for MinuteExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for MinuteExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } @@ -238,21 +236,21 @@ impl PhysicalExpr for MinuteExpr { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct SecondExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, timezone: String, } -impl DynEq for SecondExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for SecondExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for SecondExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } @@ -337,7 +335,7 @@ impl PhysicalExpr for SecondExpr { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct DateTruncExpr { /// An array with DataType::Date32 child: Arc, @@ -345,14 +343,14 @@ pub struct DateTruncExpr { format: Arc, } -impl DynEq for DateTruncExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for DateTruncExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for DateTruncExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } @@ -429,7 +427,7 @@ impl PhysicalExpr for DateTruncExpr { } } -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct TimestampTruncExpr { /// An array with DataType::Timestamp(TimeUnit::Microsecond, None) child: Arc, @@ -444,14 +442,14 @@ pub struct TimestampTruncExpr { timezone: String, } -impl DynEq for TimestampTruncExpr { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for TimestampTruncExpr { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for TimestampTruncExpr { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 1dca149c8..9c636763a 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -24,17 +24,17 @@ use crate::{spark_cast, EvalMode}; use arrow_array::builder::StringBuilder; use arrow_array::{Array, ArrayRef, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::DynEq; +use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::Result; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; use std::any::Any; use std::fmt::{Debug, Display, Formatter}; -use std::hash::{Hash, Hasher}; +use std::hash::Hasher; use std::sync::Arc; /// to_json function -#[derive(Debug, Hash)] +#[derive(Debug, Eq)] pub struct ToJson { /// The input to convert to JSON expr: Arc, @@ -42,14 +42,14 @@ pub struct ToJson { timezone: String, } -impl DynEq for ToJson { - fn dyn_eq(&self, other: &dyn Any) -> bool { +impl DynHash for ToJson { + fn dyn_hash(&self, _state: &mut dyn Hasher) { todo!() } } impl PartialEq for ToJson { - fn eq(&self, other: &Self) -> bool { + fn eq(&self, _other: &Self) -> bool { todo!() } } From a45c32990d5d200fbbc29270ef8af52b4f9c9d5c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 15:27:36 -0700 Subject: [PATCH 13/30] implement more hashers --- .../expressions/bloom_filter_might_contain.rs | 14 +++---- .../datafusion/expressions/checkoverflow.rs | 41 ++++++++----------- .../datafusion/expressions/strings.rs | 40 +++++++----------- .../datafusion/expressions/subquery.rs | 5 --- .../datafusion/expressions/unbound.rs | 5 --- 5 files changed, 38 insertions(+), 67 deletions(-) diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs index b98f0ca0f..bf8fd7f35 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs @@ -39,8 +39,11 @@ pub struct BloomFilterMightContain { } impl DynHash for BloomFilterMightContain { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.bloom_filter_expr.dyn_hash(&mut s); + self.value_expr.dyn_hash(&mut s); + self.bloom_filter.dyn_hash(&mut s); } } @@ -152,11 +155,4 @@ impl PhysicalExpr for BloomFilterMightContain { Arc::clone(&children[1]), )?)) } - - // fn dyn_hash(&self, state: &mut dyn Hasher) { - // let mut s = state; - // self.bloom_filter_expr.hash(&mut s); - // self.value_expr.hash(&mut s); - // self.hash(&mut s); - // } } diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs index ef4facdad..81d5c2c0a 100644 --- a/native/core/src/execution/datafusion/expressions/checkoverflow.rs +++ b/native/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -45,8 +45,11 @@ pub struct CheckOverflow { } impl DynHash for CheckOverflow { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.dyn_hash(&mut s); + self.data_type.dyn_hash(&mut s); + self.fail_on_error.dyn_hash(&mut s); } } @@ -56,6 +59,19 @@ impl PartialEq for CheckOverflow { } } +impl PartialEq for CheckOverflow { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| { + self.child.eq(&x.child) + && self.data_type.eq(&x.data_type) + && self.fail_on_error.eq(&x.fail_on_error) + }) + .unwrap_or(false) + } +} + impl CheckOverflow { pub fn new(child: Arc, data_type: DataType, fail_on_error: bool) -> Self { Self { @@ -76,19 +92,6 @@ impl Display for CheckOverflow { } } -impl PartialEq for CheckOverflow { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.data_type.eq(&x.data_type) - && self.fail_on_error.eq(&x.fail_on_error) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for CheckOverflow { fn as_any(&self) -> &dyn Any { self @@ -174,12 +177,4 @@ impl PhysicalExpr for CheckOverflow { self.fail_on_error, ))) } - - // fn dyn_hash(&self, state: &mut dyn Hasher) { - // let mut s = state; - // self.child.hash(&mut s); - // self.data_type.hash(&mut s); - // self.fail_on_error.hash(&mut s); - // self.hash(&mut s); - // } } diff --git a/native/core/src/execution/datafusion/expressions/strings.rs b/native/core/src/execution/datafusion/expressions/strings.rs index 3aef4cd69..4095d8d0a 100644 --- a/native/core/src/execution/datafusion/expressions/strings.rs +++ b/native/core/src/execution/datafusion/expressions/strings.rs @@ -178,6 +178,15 @@ impl PartialEq for StringSpaceExpr { } } +impl PartialEq for StringSpaceExpr { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| self.child.eq(&x.child)) + .unwrap_or(false) + } +} + impl SubstringExpr { pub fn new(child: Arc, start: i64, len: u64) -> Self { Self { child, start, len } @@ -216,8 +225,12 @@ impl PartialEq for SubstringExpr { } impl DynHash for SubstringExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.child.hash(&mut s); + self.start.hash(&mut s); + self.len.hash(&mut s); + // self.hash(&mut s); } } @@ -268,23 +281,6 @@ impl PhysicalExpr for SubstringExpr { self.len, ))) } - - // fn dyn_hash(&self, state: &mut dyn Hasher) { - // let mut s = state; - // self.child.hash(&mut s); - // self.start.hash(&mut s); - // self.len.hash(&mut s); - // self.hash(&mut s); - // } -} - -impl PartialEq for StringSpaceExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child)) - .unwrap_or(false) - } } impl PhysicalExpr for StringSpaceExpr { @@ -329,10 +325,4 @@ impl PhysicalExpr for StringSpaceExpr { ) -> datafusion_common::Result> { Ok(Arc::new(StringSpaceExpr::new(Arc::clone(&children[0])))) } - - // fn dyn_hash(&self, state: &mut dyn Hasher) { - // let mut s = state; - // self.child.hash(&mut s); - // self.hash(&mut s); - // } } diff --git a/native/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/datafusion/expressions/subquery.rs index a3eaa8c5c..68de583f8 100644 --- a/native/core/src/execution/datafusion/expressions/subquery.rs +++ b/native/core/src/execution/datafusion/expressions/subquery.rs @@ -209,9 +209,4 @@ impl PhysicalExpr for Subquery { ) -> datafusion_common::Result> { Ok(self) } - - // fn dyn_hash(&self, state: &mut dyn Hasher) { - // let mut s = state; - // self.hash(&mut s) - // } } diff --git a/native/core/src/execution/datafusion/expressions/unbound.rs b/native/core/src/execution/datafusion/expressions/unbound.rs index b0f239d89..6ba290a71 100644 --- a/native/core/src/execution/datafusion/expressions/unbound.rs +++ b/native/core/src/execution/datafusion/expressions/unbound.rs @@ -89,11 +89,6 @@ impl PhysicalExpr for UnboundColumn { ) -> Result> { Ok(self) } - - // fn dyn_hash(&self, state: &mut dyn Hasher) { - // let mut s = state; - // self.hash(&mut s); - // } } impl PartialEq for UnboundColumn { From 227061db66356cc6f8043dbe02f2566bc6885613 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 17:18:53 -0700 Subject: [PATCH 14/30] implement Hash and PartialEq --- native/spark-expr/src/bitwise_not.rs | 5 +- native/spark-expr/src/cast.rs | 60 +++++++++++----------- native/spark-expr/src/if_expr.rs | 13 ++--- native/spark-expr/src/list.rs | 51 +++++++++++-------- native/spark-expr/src/normalize_nan.rs | 22 +++----- native/spark-expr/src/structs.rs | 14 +++--- native/spark-expr/src/temporal.rs | 69 +++++++++++++------------- native/spark-expr/src/to_json.rs | 15 +++--- 8 files changed, 126 insertions(+), 123 deletions(-) diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index 818064554..11fa25ef6 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -47,8 +47,9 @@ pub struct BitwiseNotExpr { } impl DynHash for BitwiseNotExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.arg.dyn_hash(&mut s); } } diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 8cfeada0e..5eb09ab8e 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -37,7 +37,6 @@ use arrow_array::builder::StringBuilder; use arrow_array::{DictionaryArray, StringArray, StructArray}; use arrow_schema::{DataType, Field, Schema}; use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{ cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, }; @@ -48,7 +47,6 @@ use num::{ ToPrimitive, }; use regex::Regex; -use std::hash::Hasher; use std::str::FromStr; use std::{ any::Any, @@ -142,6 +140,35 @@ pub struct Cast { pub cast_options: SparkCastOptions, } +impl PartialEq for Cast { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.data_type.eq(&other.data_type) + && self.cast_options.eq(&other.cast_options) + } +} + +impl Hash for Cast { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + self.cast_options.hash(state); + } +} + +impl PartialEq for Cast { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| { + self.child.eq(&x.child) + && self.cast_options.eq(&x.cast_options) + && self.data_type.eq(&x.data_type) + }) + .unwrap_or(false) + } +} + macro_rules! cast_utf8_to_int { ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident) => {{ let len = $array.len(); @@ -1451,29 +1478,6 @@ impl Display for Cast { } } -impl DynHash for Cast { - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.type_id().hash(&mut s); - self.child.hash(&mut s); - self.data_type.hash(&mut s); - self.cast_options.hash(&mut s); - } -} - -impl PartialEq for Cast { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.cast_options.eq(&x.cast_options) - && self.data_type.eq(&x.data_type) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for Cast { fn as_any(&self) -> &dyn Any { self @@ -1511,12 +1515,6 @@ impl PhysicalExpr for Cast { } } -impl PartialEq for Cast { - fn eq(&self, _other: &Self) -> bool { - todo!() - } -} - fn timestamp_parser( value: &str, eval_mode: EvalMode, diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index d6eeffb4b..cf641e14a 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -21,10 +21,9 @@ use arrow::{ record_batch::RecordBatch, }; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::Result; use datafusion_physical_expr::{expressions::CaseExpr, PhysicalExpr}; -use std::hash::Hasher; +use std::hash::Hash; use std::{any::Any, sync::Arc}; /// IfExpr is a wrapper around CaseExpr, because `IF(a, b, c)` is semantically equivalent to @@ -38,12 +37,14 @@ pub struct IfExpr { case_expr: Arc, } -impl DynHash for IfExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for IfExpr { + fn hash(&self, state: &mut H) { + self.if_expr.hash(state); + self.true_expr.hash(state); + self.false_expr.hash(state); + self.case_expr.hash(state); } } - impl PartialEq for IfExpr { fn eq(&self, other: &Self) -> bool { self.if_expr.eq(&other.if_expr) diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index 7445784d4..cffd8b220 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -27,13 +27,12 @@ use arrow_array::{ }; use arrow_schema::{DataType, Field, FieldRef, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{ cast::{as_int32_array, as_large_list_array, as_list_array}, internal_err, DataFusionError, Result as DataFusionResult, ScalarValue, }; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; +use std::hash::Hash; use std::{ any::Any, fmt::{Debug, Display, Formatter}, @@ -54,15 +53,22 @@ pub struct ListExtract { fail_on_error: bool, } -impl DynHash for ListExtract { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for ListExtract { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); + self.default_value.hash(state); + self.one_based.hash(state); + self.fail_on_error.hash(state); } } - impl PartialEq for ListExtract { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.ordinal.eq(&other.ordinal) + && self.default_value.eq(&other.default_value) + && self.one_based.eq(&other.one_based) + && self.fail_on_error.eq(&other.fail_on_error) } } @@ -297,15 +303,15 @@ pub struct GetArrayStructFields { ordinal: usize, } -impl DynHash for GetArrayStructFields { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for GetArrayStructFields { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); } } - impl PartialEq for GetArrayStructFields { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) } } @@ -442,15 +448,20 @@ pub struct ArrayInsert { legacy_negative_index: bool, } -impl DynHash for ArrayInsert { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for ArrayInsert { + fn hash(&self, state: &mut H) { + self.src_array_expr.hash(state); + self.pos_expr.hash(state); + self.item_expr.hash(state); + self.legacy_negative_index.hash(state); } } - impl PartialEq for ArrayInsert { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.src_array_expr.eq(&other.src_array_expr) + && self.pos_expr.eq(&other.pos_expr) + && self.item_expr.eq(&other.item_expr) + && self.legacy_negative_index.eq(&other.legacy_negative_index) } } diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index 5ead1489b..7088b6e79 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -23,9 +23,8 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; +use std::hash::Hash; use std::{ any::Any, fmt::{Display, Formatter}, @@ -38,21 +37,16 @@ pub struct NormalizeNaNAndZero { pub child: Arc, } -impl DynHash for NormalizeNaNAndZero { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl PartialEq for NormalizeNaNAndZero { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.data_type.eq(&other.data_type) } } -// impl DynEq for NormalizeNaNAndZero { -// fn dyn_eq(&self, _other: &dyn Any) -> bool { -// todo!() -// } -// } - -impl PartialEq for NormalizeNaNAndZero { - fn eq(&self, _other: &Self) -> bool { - todo!() +impl Hash for NormalizeNaNAndZero { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); } } diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index 48ef43785..c9c5bffda 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -20,10 +20,8 @@ use arrow::record_batch::RecordBatch; use arrow_array::{Array, StructArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{DataFusionError, Result as DataFusionResult, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; use std::{ any::Any, fmt::{Display, Formatter}, @@ -132,15 +130,15 @@ pub struct GetStructField { ordinal: usize, } -impl DynHash for GetStructField { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for GetStructField { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.ordinal.hash(state); } } - impl PartialEq for GetStructField { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.ordinal.eq(&other.ordinal) } } diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index 4c5afa6b4..ba3081b2c 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use crate::utils::array_with_timezone; use crate::utils::down_cast_any_ref; use arrow::{ compute::{date_part, DatePart}, @@ -22,18 +23,15 @@ use arrow::{ }; use arrow_schema::{DataType, Schema, TimeUnit::Microsecond}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; +use std::hash::Hash; use std::{ any::Any, fmt::{Debug, Display, Formatter}, sync::Arc, }; -use crate::utils::array_with_timezone; - use crate::kernels::temporal::{ date_trunc_array_fmt_dyn, date_trunc_dyn, timestamp_trunc_array_fmt_dyn, timestamp_trunc_dyn, }; @@ -45,15 +43,15 @@ pub struct HourExpr { timezone: String, } -impl DynHash for HourExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for HourExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); } } - impl PartialEq for HourExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) } } @@ -144,15 +142,15 @@ pub struct MinuteExpr { timezone: String, } -impl DynHash for MinuteExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for MinuteExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); } } - impl PartialEq for MinuteExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) } } @@ -243,15 +241,15 @@ pub struct SecondExpr { timezone: String, } -impl DynHash for SecondExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for SecondExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.timezone.hash(state); } } - impl PartialEq for SecondExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.timezone.eq(&other.timezone) } } @@ -343,15 +341,15 @@ pub struct DateTruncExpr { format: Arc, } -impl DynHash for DateTruncExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for DateTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); } } - impl PartialEq for DateTruncExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.format.eq(&other.format) } } @@ -442,15 +440,18 @@ pub struct TimestampTruncExpr { timezone: String, } -impl DynHash for TimestampTruncExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for TimestampTruncExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.format.hash(state); + self.timezone.hash(state); } } - impl PartialEq for TimestampTruncExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.format.eq(&other.format) + && self.timezone.eq(&other.timezone) } } diff --git a/native/spark-expr/src/to_json.rs b/native/spark-expr/src/to_json.rs index 9c636763a..91b46c6f0 100644 --- a/native/spark-expr/src/to_json.rs +++ b/native/spark-expr/src/to_json.rs @@ -24,13 +24,12 @@ use crate::{spark_cast, EvalMode}; use arrow_array::builder::StringBuilder; use arrow_array::{Array, ArrayRef, RecordBatch, StringArray, StructArray}; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_common::Result; use datafusion_expr::ColumnarValue; use datafusion_physical_expr::PhysicalExpr; use std::any::Any; use std::fmt::{Debug, Display, Formatter}; -use std::hash::Hasher; +use std::hash::Hash; use std::sync::Arc; /// to_json function @@ -42,15 +41,15 @@ pub struct ToJson { timezone: String, } -impl DynHash for ToJson { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for ToJson { + fn hash(&self, state: &mut H) { + self.expr.hash(state); + self.timezone.hash(state); } } - impl PartialEq for ToJson { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.expr.eq(&other.expr) && self.timezone.eq(&other.timezone) } } From bf0072d6d520af4afa6b8ad98350cc55547f2d4c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 17:26:29 -0700 Subject: [PATCH 15/30] implement Hash and PartialEq --- .../expressions/bloom_filter_might_contain.rs | 20 +++--- .../datafusion/expressions/checkoverflow.rs | 20 +++--- .../datafusion/expressions/negative.rs | 14 ++-- .../datafusion/expressions/strings.rs | 71 +++++++------------ native/spark-expr/src/bitwise_not.rs | 10 ++- 5 files changed, 57 insertions(+), 78 deletions(-) diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs index bf8fd7f35..c6f2a1ca9 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs @@ -21,12 +21,11 @@ use crate::{ use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion::physical_plan::ColumnarValue; use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; +use std::hash::Hash; use std::{any::Any, fmt::Display, sync::Arc}; /// A physical expression that checks if a value might be in a bloom filter. It corresponds to the @@ -38,18 +37,19 @@ pub struct BloomFilterMightContain { bloom_filter: Option, } -impl DynHash for BloomFilterMightContain { - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.bloom_filter_expr.dyn_hash(&mut s); - self.value_expr.dyn_hash(&mut s); - self.bloom_filter.dyn_hash(&mut s); +impl Hash for BloomFilterMightContain { + fn hash(&self, state: &mut H) { + self.bloom_filter_expr.hash(state); + self.value_expr.hash(state); + self.bloom_filter.hash(state); } } impl PartialEq for BloomFilterMightContain { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.bloom_filter_expr.eq(&other.bloom_filter_expr) + && self.value_expr.eq(&other.value_expr) + && self.bloom_filter.eq(&other.bloom_filter) } } diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs index 81d5c2c0a..46d35691f 100644 --- a/native/core/src/execution/datafusion/expressions/checkoverflow.rs +++ b/native/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -22,11 +22,10 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; +use std::hash::Hash; use std::{ any::Any, fmt::{Display, Formatter}, @@ -44,18 +43,19 @@ pub struct CheckOverflow { pub fail_on_error: bool, } -impl DynHash for CheckOverflow { - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.dyn_hash(&mut s); - self.data_type.dyn_hash(&mut s); - self.fail_on_error.dyn_hash(&mut s); +impl Hash for CheckOverflow { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.data_type.hash(state); + self.fail_on_error.hash(state); } } impl PartialEq for CheckOverflow { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) + && self.data_type.eq(&other.data_type) + && self.fail_on_error.eq(&other.fail_on_error) } } diff --git a/native/core/src/execution/datafusion/expressions/negative.rs b/native/core/src/execution/datafusion/expressions/negative.rs index 7cd66c152..f35f91ea4 100644 --- a/native/core/src/execution/datafusion/expressions/negative.rs +++ b/native/core/src/execution/datafusion/expressions/negative.rs @@ -21,7 +21,6 @@ use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeT use arrow_array::RecordBatch; use arrow_buffer::IntervalDayTime; use arrow_schema::{DataType, Schema}; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, @@ -30,7 +29,7 @@ use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_comet_spark_expr::SparkError; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; -use std::hash::Hasher; +use std::hash::Hash; use std::{any::Any, sync::Arc}; pub fn create_negate_expr( @@ -48,15 +47,16 @@ pub struct NegativeExpr { fail_on_error: bool, } -impl DynHash for NegativeExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for NegativeExpr { + fn hash(&self, state: &mut H) { + self.arg.hash(state); + self.fail_on_error.hash(state); } } impl PartialEq for NegativeExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.arg.eq(&other.arg) && self.fail_on_error.eq(&other.fail_on_error) } } diff --git a/native/core/src/execution/datafusion/expressions/strings.rs b/native/core/src/execution/datafusion/expressions/strings.rs index 4095d8d0a..f9433840e 100644 --- a/native/core/src/execution/datafusion/expressions/strings.rs +++ b/native/core/src/execution/datafusion/expressions/strings.rs @@ -27,11 +27,9 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion::physical_expr_common::physical_expr::{DynEq, DynHash}; use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; use std::{ any::Any, fmt::{Display, Formatter}, @@ -41,7 +39,7 @@ use std::{ macro_rules! make_predicate_function { ($name: ident, $kernel: ident, $str_scalar_kernel: ident) => { - #[derive(Debug, Hash)] + #[derive(Debug, Eq)] pub struct $name { left: Arc, right: Arc, @@ -59,30 +57,16 @@ macro_rules! make_predicate_function { } } - // impl DynHash for $name { - // fn dyn_hash(&self, state: &mut dyn Hasher) { - // let mut s = state; - // self.left.hash(&mut s); - // self.right.hash(&mut s); - // self.hash(&mut s); - // } - // } - - impl DynEq for $name { - fn dyn_eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.left.eq(&x.left) && self.right.eq(&x.right)) - .unwrap_or(false) + impl Hash for $name { + fn hash(&self, state: &mut H) { + self.left.hash(state); + self.right.hash(state); } } - impl PartialEq for $name { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.left.eq(&x.left) && self.right.eq(&x.right)) - .unwrap_or(false) + impl PartialEq for $name { + fn eq(&self, other: &Self) -> bool { + self.left.eq(&other.left) && self.right.eq(&other.right) } } @@ -161,20 +145,33 @@ pub struct SubstringExpr { pub len: u64, } +impl Hash for SubstringExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); + self.start.hash(state); + self.len.hash(state); + } +} + +impl PartialEq for SubstringExpr { + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) && self.start.eq(&other.start) && self.len.eq(&other.len) + } +} #[derive(Debug, Eq)] pub struct StringSpaceExpr { pub child: Arc, } -impl DynHash for StringSpaceExpr { - fn dyn_hash(&self, _state: &mut dyn Hasher) { - todo!() +impl Hash for StringSpaceExpr { + fn hash(&self, state: &mut H) { + self.child.hash(state); } } impl PartialEq for StringSpaceExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() + fn eq(&self, other: &Self) -> bool { + self.child.eq(&other.child) } } @@ -224,22 +221,6 @@ impl PartialEq for SubstringExpr { } } -impl DynHash for SubstringExpr { - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.child.hash(&mut s); - self.start.hash(&mut s); - self.len.hash(&mut s); - // self.hash(&mut s); - } -} - -impl PartialEq for SubstringExpr { - fn eq(&self, _other: &Self) -> bool { - todo!() - } -} - impl PhysicalExpr for SubstringExpr { fn as_any(&self) -> &dyn Any { self diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index 11fa25ef6..e1bc4944e 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -21,11 +21,10 @@ use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; -use datafusion::physical_expr_common::physical_expr::DynHash; use datafusion::{error::DataFusionError, logical_expr::ColumnarValue}; use datafusion_common::Result; use datafusion_physical_expr::PhysicalExpr; -use std::hash::Hasher; +use std::hash::Hash; use std::{any::Any, sync::Arc}; macro_rules! compute_op { @@ -46,10 +45,9 @@ pub struct BitwiseNotExpr { arg: Arc, } -impl DynHash for BitwiseNotExpr { - fn dyn_hash(&self, state: &mut dyn Hasher) { - let mut s = state; - self.arg.dyn_hash(&mut s); +impl Hash for BitwiseNotExpr { + fn hash(&self, state: &mut H) { + self.arg.hash(state); } } From 97bbedc16038d60f96b51cef242549be62ccbaf9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 17:36:52 -0700 Subject: [PATCH 16/30] implement Hash and PartialEq --- .../expressions/bloom_filter_might_contain.rs | 13 ----- .../datafusion/expressions/checkoverflow.rs | 14 ------ .../datafusion/expressions/negative.rs | 10 ---- .../datafusion/expressions/strings.rs | 19 ------- .../datafusion/expressions/subquery.rs | 14 ------ .../datafusion/expressions/unbound.rs | 12 +---- .../core/src/execution/datafusion/planner.rs | 28 ++--------- native/spark-expr/src/avg.rs | 22 ++------ native/spark-expr/src/avg_decimal.rs | 20 +------- native/spark-expr/src/bitwise_not.rs | 10 ---- native/spark-expr/src/cast.rs | 14 ------ native/spark-expr/src/correlation.rs | 29 +---------- native/spark-expr/src/covariance.rs | 26 +--------- native/spark-expr/src/if_expr.rs | 14 ------ native/spark-expr/src/list.rs | 45 ----------------- native/spark-expr/src/normalize_nan.rs | 10 ---- native/spark-expr/src/regexp.rs | 10 ---- native/spark-expr/src/stddev.rs | 20 +------- native/spark-expr/src/structs.rs | 27 ---------- native/spark-expr/src/sum_decimal.rs | 28 ++--------- native/spark-expr/src/temporal.rs | 50 ------------------- native/spark-expr/src/utils.rs | 16 ------ native/spark-expr/src/variance.rs | 19 +------ 23 files changed, 21 insertions(+), 449 deletions(-) diff --git a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs index c6f2a1ca9..17c28fe1e 100644 --- a/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs +++ b/native/core/src/execution/datafusion/expressions/bloom_filter_might_contain.rs @@ -22,7 +22,6 @@ use arrow::record_batch::RecordBatch; use arrow_array::cast::as_primitive_array; use arrow_schema::{DataType, Schema}; use datafusion::physical_plan::ColumnarValue; -use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use std::hash::Hash; @@ -63,18 +62,6 @@ impl Display for BloomFilterMightContain { } } -impl PartialEq for BloomFilterMightContain { - fn eq(&self, _other: &dyn Any) -> bool { - down_cast_any_ref(_other) - .downcast_ref::() - .map(|other| { - self.bloom_filter_expr.eq(&other.bloom_filter_expr) - && self.value_expr.eq(&other.value_expr) - }) - .unwrap_or(false) - } -} - fn evaluate_bloom_filter( bloom_filter_expr: &Arc, ) -> Result> { diff --git a/native/core/src/execution/datafusion/expressions/checkoverflow.rs b/native/core/src/execution/datafusion/expressions/checkoverflow.rs index 46d35691f..528bbd5d9 100644 --- a/native/core/src/execution/datafusion/expressions/checkoverflow.rs +++ b/native/core/src/execution/datafusion/expressions/checkoverflow.rs @@ -22,7 +22,6 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use std::hash::Hash; @@ -59,19 +58,6 @@ impl PartialEq for CheckOverflow { } } -impl PartialEq for CheckOverflow { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.data_type.eq(&x.data_type) - && self.fail_on_error.eq(&x.fail_on_error) - }) - .unwrap_or(false) - } -} - impl CheckOverflow { pub fn new(child: Arc, data_type: DataType, fail_on_error: bool) -> Self { Self { diff --git a/native/core/src/execution/datafusion/expressions/negative.rs b/native/core/src/execution/datafusion/expressions/negative.rs index f35f91ea4..264da71bc 100644 --- a/native/core/src/execution/datafusion/expressions/negative.rs +++ b/native/core/src/execution/datafusion/expressions/negative.rs @@ -25,7 +25,6 @@ use datafusion::{ logical_expr::{interval_arithmetic::Interval, ColumnarValue}, physical_expr::PhysicalExpr, }; -use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_comet_spark_expr::SparkError; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::sort_properties::ExprProperties; @@ -261,12 +260,3 @@ impl PhysicalExpr for NegativeExpr { Ok(properties) } } - -impl PartialEq for NegativeExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.arg.eq(&x.arg)) - .unwrap_or(false) - } -} diff --git a/native/core/src/execution/datafusion/expressions/strings.rs b/native/core/src/execution/datafusion/expressions/strings.rs index f9433840e..95f49d98f 100644 --- a/native/core/src/execution/datafusion/expressions/strings.rs +++ b/native/core/src/execution/datafusion/expressions/strings.rs @@ -27,7 +27,6 @@ use arrow::{ }; use arrow_schema::{DataType, Schema}; use datafusion::logical_expr::ColumnarValue; -use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{DataFusionError, ScalarValue::Utf8}; use datafusion_physical_expr::PhysicalExpr; use std::{ @@ -175,15 +174,6 @@ impl PartialEq for StringSpaceExpr { } } -impl PartialEq for StringSpaceExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child)) - .unwrap_or(false) - } -} - impl SubstringExpr { pub fn new(child: Arc, start: i64, len: u64) -> Self { Self { child, start, len } @@ -212,15 +202,6 @@ impl Display for StringSpaceExpr { } } -impl PartialEq for SubstringExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.start.eq(&x.start) && self.len.eq(&x.len)) - .unwrap_or(false) - } -} - impl PhysicalExpr for SubstringExpr { fn as_any(&self) -> &dyn Any { self diff --git a/native/core/src/execution/datafusion/expressions/subquery.rs b/native/core/src/execution/datafusion/expressions/subquery.rs index 68de583f8..d933a6096 100644 --- a/native/core/src/execution/datafusion/expressions/subquery.rs +++ b/native/core/src/execution/datafusion/expressions/subquery.rs @@ -22,7 +22,6 @@ use crate::{ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema, TimeUnit}; use datafusion::logical_expr::ColumnarValue; -use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, ScalarValue}; use datafusion_physical_expr::PhysicalExpr; use jni::{ @@ -63,19 +62,6 @@ impl Display for Subquery { } } -impl PartialEq for Subquery { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.id.eq(&x.id) - && self.data_type.eq(&x.data_type) - && self.exec_context_id.eq(&x.exec_context_id) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for Subquery { fn as_any(&self) -> &dyn Any { self diff --git a/native/core/src/execution/datafusion/expressions/unbound.rs b/native/core/src/execution/datafusion/expressions/unbound.rs index 6ba290a71..14f68c9cd 100644 --- a/native/core/src/execution/datafusion/expressions/unbound.rs +++ b/native/core/src/execution/datafusion/expressions/unbound.rs @@ -18,10 +18,9 @@ use arrow_array::RecordBatch; use arrow_schema::{DataType, Schema}; use datafusion::physical_plan::ColumnarValue; -use datafusion_comet_spark_expr::utils::down_cast_any_ref; use datafusion_common::{internal_err, Result}; use datafusion_physical_expr::PhysicalExpr; -use std::{any::Any, hash::Hash, sync::Arc}; +use std::{hash::Hash, sync::Arc}; /// This is similar to `UnKnownColumn` in DataFusion, but it has data type. /// This is only used when the column is not bound to a schema, for example, the @@ -90,12 +89,3 @@ impl PhysicalExpr for UnboundColumn { Ok(self) } } - -impl PartialEq for UnboundColumn { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self == x) - .unwrap_or(false) - } -} diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index ce0283109..3159952ca 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -1509,10 +1509,7 @@ impl PhysicalPlanner { let builder = match datatype { DataType::Decimal128(_, _) => { - let func = AggregateUDF::new_from_impl(SumDecimal::try_new( - Arc::clone(&child), - datatype, - )?); + let func = AggregateUDF::new_from_impl(SumDecimal::try_new(datatype)?); AggregateExprBuilder::new(Arc::new(func), vec![child]) } _ => { @@ -1537,11 +1534,8 @@ impl PhysicalPlanner { let input_datatype = to_arrow_datatype(expr.sum_datatype.as_ref().unwrap()); let builder = match datatype { DataType::Decimal128(_, _) => { - let func = AggregateUDF::new_from_impl(AvgDecimal::new( - Arc::clone(&child), - datatype, - input_datatype, - )); + let func = + AggregateUDF::new_from_impl(AvgDecimal::new(datatype, input_datatype)); AggregateExprBuilder::new(Arc::new(func), vec![child]) } _ => { @@ -1550,11 +1544,7 @@ impl PhysicalPlanner { // failure since it should have already been checked at Spark side. let child: Arc = Arc::new(CastExpr::new(Arc::clone(&child), datatype.clone(), None)); - let func = AggregateUDF::new_from_impl(Avg::new( - Arc::clone(&child), - "avg", - datatype, - )); + let func = AggregateUDF::new_from_impl(Avg::new("avg", datatype)); AggregateExprBuilder::new(Arc::new(func), vec![child]) } }; @@ -1632,8 +1622,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Covariance::new( - Arc::clone(&child1), - Arc::clone(&child2), "covariance", datatype, StatsType::Sample, @@ -1649,8 +1637,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Covariance::new( - Arc::clone(&child1), - Arc::clone(&child2), "covariance_pop", datatype, StatsType::Population, @@ -1676,7 +1662,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Variance::new( - Arc::clone(&child), "variance", datatype, StatsType::Sample, @@ -1687,7 +1672,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Variance::new( - Arc::clone(&child), "variance_pop", datatype, StatsType::Population, @@ -1708,7 +1692,6 @@ impl PhysicalPlanner { match expr.stats_type { 0 => { let func = AggregateUDF::new_from_impl(Stddev::new( - Arc::clone(&child), "stddev", datatype, StatsType::Sample, @@ -1719,7 +1702,6 @@ impl PhysicalPlanner { } 1 => { let func = AggregateUDF::new_from_impl(Stddev::new( - Arc::clone(&child), "stddev_pop", datatype, StatsType::Population, @@ -1741,8 +1723,6 @@ impl PhysicalPlanner { self.create_expr(expr.child2.as_ref().unwrap(), Arc::clone(&schema))?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); let func = AggregateUDF::new_from_impl(Correlation::new( - Arc::clone(&child1), - Arc::clone(&child2), "correlation", datatype, expr.null_on_divide_by_zero, diff --git a/native/spark-expr/src/avg.rs b/native/spark-expr/src/avg.rs index 751676b73..816440ac9 100644 --- a/native/spark-expr/src/avg.rs +++ b/native/spark-expr/src/avg.rs @@ -27,10 +27,9 @@ use datafusion::logical_expr::{ type_coercion::aggregates::avg_return_type, Accumulator, EmitTo, GroupsAccumulator, Signature, }; use datafusion_common::{not_impl_err, Result, ScalarValue}; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; +use datafusion_physical_expr::expressions::format_state_name; use std::{any::Any, sync::Arc}; -use crate::utils::down_cast_any_ref; use arrow_array::ArrowNativeTypeOp; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; @@ -42,20 +41,19 @@ use DataType::*; pub struct Avg { name: String, signature: Signature, - expr: Arc, + // expr: Arc, input_data_type: DataType, result_data_type: DataType, } impl Avg { /// Create a new AVG aggregate function - pub fn new(expr: Arc, name: impl Into, data_type: DataType) -> Self { + pub fn new(name: impl Into, data_type: DataType) -> Self { let result_data_type = avg_return_type("avg", &data_type).unwrap(); Self { name: name.into(), signature: Signature::user_defined(Immutable), - expr, input_data_type: data_type, result_data_type, } @@ -139,20 +137,6 @@ impl AggregateUDFImpl for Avg { } } -impl PartialEq for Avg { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.input_data_type == x.input_data_type - && self.result_data_type == x.result_data_type - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the average #[derive(Debug, Default)] pub struct AvgAccumulator { diff --git a/native/spark-expr/src/avg_decimal.rs b/native/spark-expr/src/avg_decimal.rs index 1fe2fa2fb..05fc28e58 100644 --- a/native/spark-expr/src/avg_decimal.rs +++ b/native/spark-expr/src/avg_decimal.rs @@ -25,10 +25,9 @@ use arrow_array::{ use arrow_schema::{DataType, Field}; use datafusion::logical_expr::{Accumulator, EmitTo, GroupsAccumulator, Signature}; use datafusion_common::{not_impl_err, Result, ScalarValue}; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; +use datafusion_physical_expr::expressions::format_state_name; use std::{any::Any, sync::Arc}; -use crate::utils::down_cast_any_ref; use crate::utils::is_valid_decimal_precision; use arrow_array::ArrowNativeTypeOp; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; @@ -43,17 +42,15 @@ use DataType::*; #[derive(Debug, Clone)] pub struct AvgDecimal { signature: Signature, - expr: Arc, sum_data_type: DataType, result_data_type: DataType, } impl AvgDecimal { /// Create a new AVG aggregate function - pub fn new(expr: Arc, result_type: DataType, sum_type: DataType) -> Self { + pub fn new(result_type: DataType, sum_type: DataType) -> Self { Self { signature: Signature::user_defined(Immutable), - expr, result_data_type: result_type, sum_data_type: sum_type, } @@ -156,19 +153,6 @@ impl AggregateUDFImpl for AvgDecimal { } } -impl PartialEq for AvgDecimal { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.sum_data_type == x.sum_data_type - && self.result_data_type == x.result_data_type - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the average for decimals #[derive(Debug)] struct AvgDecimalAccumulator { diff --git a/native/spark-expr/src/bitwise_not.rs b/native/spark-expr/src/bitwise_not.rs index e1bc4944e..d7c31836f 100644 --- a/native/spark-expr/src/bitwise_not.rs +++ b/native/spark-expr/src/bitwise_not.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use arrow::{ array::*, datatypes::{DataType, Schema}, @@ -124,15 +123,6 @@ impl PhysicalExpr for BitwiseNotExpr { } } -impl PartialEq for BitwiseNotExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.arg.eq(&x.arg)) - .unwrap_or(false) - } -} - pub fn bitwise_not(arg: Arc) -> Result> { Ok(Arc::new(BitwiseNotExpr::new(arg))) } diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index 5eb09ab8e..17c603c28 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use arrow::{ array::{ cast::AsArray, @@ -156,19 +155,6 @@ impl Hash for Cast { } } -impl PartialEq for Cast { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.cast_options.eq(&x.cast_options) - && self.data_type.eq(&x.data_type) - }) - .unwrap_or(false) - } -} - macro_rules! cast_utf8_to_int { ($array:expr, $eval_mode:expr, $array_type:ty, $cast_method:ident) => {{ let len = $array.len(); diff --git a/native/spark-expr/src/correlation.rs b/native/spark-expr/src/correlation.rs index b656e4b1d..e4ddab95d 100644 --- a/native/spark-expr/src/correlation.rs +++ b/native/spark-expr/src/correlation.rs @@ -21,7 +21,6 @@ use std::{any::Any, sync::Arc}; use crate::covariance::CovarianceAccumulator; use crate::stddev::StddevAccumulator; -use crate::utils::down_cast_any_ref; use arrow::{ array::ArrayRef, datatypes::{DataType, Field}, @@ -31,8 +30,8 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// CORR aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -43,26 +42,16 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Correlation { name: String, signature: Signature, - expr1: Arc, - expr2: Arc, null_on_divide_by_zero: bool, } impl Correlation { - pub fn new( - expr1: Arc, - expr2: Arc, - name: impl Into, - data_type: DataType, - null_on_divide_by_zero: bool, - ) -> Self { + pub fn new(name: impl Into, data_type: DataType, null_on_divide_by_zero: bool) -> Self { // the result of correlation just support FLOAT64 data type. assert!(matches!(data_type, DataType::Float64)); Self { name: name.into(), signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), - expr1, - expr2, null_on_divide_by_zero, } } @@ -131,20 +120,6 @@ impl AggregateUDFImpl for Correlation { } } -impl PartialEq for Correlation { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr1.eq(&x.expr1) - && self.expr2.eq(&x.expr2) - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - }) - .unwrap_or(false) - } -} - /// An accumulator to compute correlation #[derive(Debug)] pub struct CorrelationAccumulator { diff --git a/native/spark-expr/src/covariance.rs b/native/spark-expr/src/covariance.rs index 4231be147..fa3563cde 100644 --- a/native/spark-expr/src/covariance.rs +++ b/native/spark-expr/src/covariance.rs @@ -17,9 +17,8 @@ * under the License. */ -use std::{any::Any, sync::Arc}; +use std::any::Any; -use crate::utils::down_cast_any_ref; use arrow::{ array::{ArrayRef, Float64Array}, compute::cast, @@ -32,8 +31,8 @@ use datafusion_common::{ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// COVAR_SAMP and COVAR_POP aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -43,8 +42,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Covariance { name: String, signature: Signature, - expr1: Arc, - expr2: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -52,8 +49,6 @@ pub struct Covariance { impl Covariance { /// Create a new COVAR aggregate function pub fn new( - expr1: Arc, - expr2: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -64,8 +59,6 @@ impl Covariance { Self { name: name.into(), signature: Signature::uniform(2, NUMERICS.to_vec(), Volatility::Immutable), - expr1, - expr2, stats_type, null_on_divide_by_zero, } @@ -126,21 +119,6 @@ impl AggregateUDFImpl for Covariance { } } -impl PartialEq for Covariance { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr1.eq(&x.expr1) - && self.expr2.eq(&x.expr2) - && self.stats_type == x.stats_type - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - }) - .unwrap_or(false) - } -} - /// An accumulator to compute covariance #[derive(Debug)] pub struct CovarianceAccumulator { diff --git a/native/spark-expr/src/if_expr.rs b/native/spark-expr/src/if_expr.rs index cf641e14a..01c754ad6 100644 --- a/native/spark-expr/src/if_expr.rs +++ b/native/spark-expr/src/if_expr.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, @@ -121,19 +120,6 @@ impl PhysicalExpr for IfExpr { } } -impl PartialEq for IfExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.if_expr.eq(&x.if_expr) - && self.true_expr.eq(&x.true_expr) - && self.false_expr.eq(&x.false_expr) - }) - .unwrap_or(false) - } -} - #[cfg(test)] mod tests { use arrow::{array::StringArray, datatypes::*}; diff --git a/native/spark-expr/src/list.rs b/native/spark-expr/src/list.rs index cffd8b220..fc31b11a0 100644 --- a/native/spark-expr/src/list.rs +++ b/native/spark-expr/src/list.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use arrow::{ array::{as_primitive_array, Capacities, MutableArrayData}, buffer::{NullBuffer, OffsetBuffer}, @@ -276,27 +275,6 @@ impl Display for ListExtract { } } -impl PartialEq for ListExtract { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.ordinal.eq(&x.ordinal) - && (self.default_value.is_none() == x.default_value.is_none()) - && self - .default_value - .as_ref() - .zip(x.default_value.as_ref()) - .map(|(s, x)| s.eq(x)) - .unwrap_or(true) - && self.one_based.eq(&x.one_based) - && self.fail_on_error.eq(&x.fail_on_error) - }) - .unwrap_or(false) - } -} - #[derive(Debug, Eq)] pub struct GetArrayStructFields { child: Arc, @@ -431,15 +409,6 @@ impl Display for GetArrayStructFields { } } -impl PartialEq for GetArrayStructFields { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.ordinal.eq(&x.ordinal)) - .unwrap_or(false) - } -} - #[derive(Debug, Eq)] pub struct ArrayInsert { src_array_expr: Arc, @@ -716,20 +685,6 @@ impl Display for ArrayInsert { } } -impl PartialEq for ArrayInsert { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.src_array_expr.eq(&x.src_array_expr) - && self.pos_expr.eq(&x.pos_expr) - && self.item_expr.eq(&x.item_expr) - && self.legacy_negative_index.eq(&x.legacy_negative_index) - }) - .unwrap_or(false) - } -} - #[cfg(test)] mod test { use crate::list::{array_insert, list_extract, zero_based_index}; diff --git a/native/spark-expr/src/normalize_nan.rs b/native/spark-expr/src/normalize_nan.rs index 7088b6e79..078ce4b5a 100644 --- a/native/spark-expr/src/normalize_nan.rs +++ b/native/spark-expr/src/normalize_nan.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use arrow::{ array::{as_primitive_array, ArrayAccessor, ArrayIter, Float32Array, Float64Array}, datatypes::{ArrowNativeType, Float32Type, Float64Type}, @@ -125,15 +124,6 @@ impl Display for NormalizeNaNAndZero { } } -impl PartialEq for NormalizeNaNAndZero { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.data_type.eq(&x.data_type)) - .unwrap_or(false) - } -} - trait FloatDouble: ArrowNativeType { fn is_nan(&self) -> bool; fn nan(&self) -> Self; diff --git a/native/spark-expr/src/regexp.rs b/native/spark-expr/src/regexp.rs index 5679812d7..7f367a8bb 100644 --- a/native/spark-expr/src/regexp.rs +++ b/native/spark-expr/src/regexp.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use crate::SparkError; use arrow::compute::take; use arrow_array::builder::BooleanBuilder; @@ -104,15 +103,6 @@ impl Display for RLike { } } -impl PartialEq for RLike { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.pattern_str.eq(&x.pattern_str)) - .unwrap_or(false) - } -} - impl PhysicalExpr for RLike { fn as_any(&self) -> &dyn Any { self diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/stddev.rs index 898a2c0c4..bfb56dcc2 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -17,7 +17,6 @@ use std::{any::Any, sync::Arc}; -use crate::utils::down_cast_any_ref; use crate::variance::VarianceAccumulator; use arrow::{ array::ArrayRef, @@ -28,8 +27,8 @@ use datafusion_common::types::NativeType; use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::{AggregateUDFImpl, Signature, Volatility}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// STDDEV and STDDEV_SAMP (standard deviation) aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -40,7 +39,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Stddev { name: String, signature: Signature, - expr: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -48,7 +46,6 @@ pub struct Stddev { impl Stddev { /// Create a new STDDEV aggregate function pub fn new( - expr: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -62,7 +59,6 @@ impl Stddev { vec![Arc::new(NativeType::Float64)], Volatility::Immutable, ), - expr, stats_type, null_on_divide_by_zero, } @@ -125,20 +121,6 @@ impl AggregateUDFImpl for Stddev { } } -impl PartialEq for Stddev { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.expr.eq(&x.expr) - && self.null_on_divide_by_zero == x.null_on_divide_by_zero - && self.stats_type == x.stats_type - }) - .unwrap_or(false) - } -} - /// An accumulator to compute the standard deviation #[derive(Debug)] pub struct StddevAccumulator { diff --git a/native/spark-expr/src/structs.rs b/native/spark-expr/src/structs.rs index c9c5bffda..7cc49e428 100644 --- a/native/spark-expr/src/structs.rs +++ b/native/spark-expr/src/structs.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use arrow::record_batch::RecordBatch; use arrow_array::{Array, StructArray}; use arrow_schema::{DataType, Field, Schema}; @@ -107,23 +106,6 @@ impl Display for CreateNamedStruct { } } -impl PartialEq for CreateNamedStruct { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.values - .iter() - .zip(x.values.iter()) - .all(|(a, b)| a.eq(b)) - && self.values.len() == x.values.len() - && self.names.iter().zip(x.names.iter()).all(|(a, b)| a.eq(b)) - && self.names.len() == x.names.len() - }) - .unwrap_or(false) - } -} - #[derive(Debug, Eq)] pub struct GetStructField { child: Arc, @@ -220,15 +202,6 @@ impl Display for GetStructField { } } -impl PartialEq for GetStructField { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.ordinal.eq(&x.ordinal)) - .unwrap_or(false) - } -} - #[cfg(test)] mod test { use super::CreateNamedStruct; diff --git a/native/spark-expr/src/sum_decimal.rs b/native/spark-expr/src/sum_decimal.rs index ceda6755a..f3f34d9bf 100644 --- a/native/spark-expr/src/sum_decimal.rs +++ b/native/spark-expr/src/sum_decimal.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use crate::utils::down_cast_any_ref; use crate::utils::{is_valid_decimal_precision, unlikely}; use arrow::{ array::BooleanBufferBuilder, @@ -30,15 +29,12 @@ use datafusion_common::{DataFusionError, Result as DFResult, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, ReversedUDAF, Signature}; -use datafusion_physical_expr::PhysicalExpr; use std::{any::Any, ops::BitAnd, sync::Arc}; #[derive(Debug)] pub struct SumDecimal { /// Aggregate function signature signature: Signature, - /// The expression that provides the input decimal values to be summed - expr: Arc, /// The data type of the SUM result. This will always be a decimal type /// with the same precision and scale as specified in this struct result_type: DataType, @@ -49,7 +45,7 @@ pub struct SumDecimal { } impl SumDecimal { - pub fn try_new(expr: Arc, data_type: DataType) -> DFResult { + pub fn try_new(data_type: DataType) -> DFResult { // The `data_type` is the SUM result type passed from Spark side let (precision, scale) = match data_type { DataType::Decimal128(p, s) => (p, s), @@ -61,7 +57,6 @@ impl SumDecimal { }; Ok(Self { signature: Signature::user_defined(Immutable), - expr, result_type: data_type, precision, scale, @@ -132,20 +127,6 @@ impl AggregateUDFImpl for SumDecimal { } } -impl PartialEq for SumDecimal { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - // note that we do not compare result_type because this - // is guaranteed to match if the precision and scale - // match - self.precision == x.precision && self.scale == x.scale && self.expr.eq(&x.expr) - }) - .unwrap_or(false) - } -} - #[derive(Debug)] struct SumDecimalAccumulator { sum: i128, @@ -491,13 +472,13 @@ mod tests { use datafusion_common::Result; use datafusion_expr::AggregateUDF; use datafusion_physical_expr::aggregate::AggregateExprBuilder; - use datafusion_physical_expr::expressions::{Column, Literal}; + use datafusion_physical_expr::expressions::Column; + use datafusion_physical_expr::PhysicalExpr; use futures::StreamExt; #[test] fn invalid_data_type() { - let expr = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))); - assert!(SumDecimal::try_new(expr, DataType::Int32).is_err()); + assert!(SumDecimal::try_new(DataType::Int32).is_err()); } #[tokio::test] @@ -518,7 +499,6 @@ mod tests { Arc::new(MemoryExec::try_new(partitions, Arc::clone(&schema), None).unwrap()); let aggregate_udf = Arc::new(AggregateUDF::new_from_impl(SumDecimal::try_new( - Arc::clone(&c1), data_type.clone(), )?)); diff --git a/native/spark-expr/src/temporal.rs b/native/spark-expr/src/temporal.rs index ba3081b2c..fb549f9ce 100644 --- a/native/spark-expr/src/temporal.rs +++ b/native/spark-expr/src/temporal.rs @@ -16,7 +16,6 @@ // under the License. use crate::utils::array_with_timezone; -use crate::utils::down_cast_any_ref; use arrow::{ compute::{date_part, DatePart}, record_batch::RecordBatch, @@ -71,15 +70,6 @@ impl Display for HourExpr { } } -impl PartialEq for HourExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for HourExpr { fn as_any(&self) -> &dyn Any { self @@ -170,15 +160,6 @@ impl Display for MinuteExpr { } } -impl PartialEq for MinuteExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for MinuteExpr { fn as_any(&self) -> &dyn Any { self @@ -269,15 +250,6 @@ impl Display for SecondExpr { } } -impl PartialEq for SecondExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.timezone.eq(&x.timezone)) - .unwrap_or(false) - } -} - impl PhysicalExpr for SecondExpr { fn as_any(&self) -> &dyn Any { self @@ -369,15 +341,6 @@ impl Display for DateTruncExpr { } } -impl PartialEq for DateTruncExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| self.child.eq(&x.child) && self.format.eq(&x.format)) - .unwrap_or(false) - } -} - impl PhysicalExpr for DateTruncExpr { fn as_any(&self) -> &dyn Any { self @@ -479,19 +442,6 @@ impl Display for TimestampTruncExpr { } } -impl PartialEq for TimestampTruncExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.child.eq(&x.child) - && self.format.eq(&x.format) - && self.timezone.eq(&x.timezone) - }) - .unwrap_or(false) - } -} - impl PhysicalExpr for TimestampTruncExpr { fn as_any(&self) -> &dyn Any { self diff --git a/native/spark-expr/src/utils.rs b/native/spark-expr/src/utils.rs index 690106517..18a2314fb 100644 --- a/native/spark-expr/src/utils.rs +++ b/native/spark-expr/src/utils.rs @@ -20,7 +20,6 @@ use arrow_array::{ types::{Int32Type, TimestampMicrosecondType}, }; use arrow_schema::{ArrowError, DataType, DECIMAL128_MAX_PRECISION}; -use std::any::Any; use std::sync::Arc; use crate::timezone::Tz; @@ -30,7 +29,6 @@ use arrow::{ }; use arrow_data::decimal::{MAX_DECIMAL_FOR_EACH_PRECISION, MIN_DECIMAL_FOR_EACH_PRECISION}; use chrono::{DateTime, Offset, TimeZone}; -use datafusion_physical_expr::PhysicalExpr; /// Preprocesses input arrays to add timezone information from Spark to Arrow array datatype or /// to apply timezone offset. @@ -215,17 +213,3 @@ pub fn unlikely(b: bool) -> bool { } b } - -pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { - if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else { - any - } -} diff --git a/native/spark-expr/src/variance.rs b/native/spark-expr/src/variance.rs index fa16fac12..f3648a679 100644 --- a/native/spark-expr/src/variance.rs +++ b/native/spark-expr/src/variance.rs @@ -15,9 +15,8 @@ // specific language governing permissions and limitations // under the License. -use std::{any::Any, sync::Arc}; +use std::any::Any; -use crate::utils::down_cast_any_ref; use arrow::{ array::{ArrayRef, Float64Array}, datatypes::{DataType, Field}, @@ -27,8 +26,8 @@ use datafusion_common::{downcast_value, DataFusionError, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::Volatility::Immutable; use datafusion_expr::{AggregateUDFImpl, Signature}; +use datafusion_physical_expr::expressions::format_state_name; use datafusion_physical_expr::expressions::StatsType; -use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; /// VAR_SAMP and VAR_POP aggregate expression /// The implementation mostly is the same as the DataFusion's implementation. The reason @@ -39,7 +38,6 @@ use datafusion_physical_expr::{expressions::format_state_name, PhysicalExpr}; pub struct Variance { name: String, signature: Signature, - expr: Arc, stats_type: StatsType, null_on_divide_by_zero: bool, } @@ -47,7 +45,6 @@ pub struct Variance { impl Variance { /// Create a new VARIANCE aggregate function pub fn new( - expr: Arc, name: impl Into, data_type: DataType, stats_type: StatsType, @@ -58,7 +55,6 @@ impl Variance { Self { name: name.into(), signature: Signature::numeric(1, Immutable), - expr, stats_type, null_on_divide_by_zero, } @@ -118,17 +114,6 @@ impl AggregateUDFImpl for Variance { } } -impl PartialEq for Variance { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name && self.expr.eq(&x.expr) && self.stats_type == x.stats_type - }) - .unwrap_or(false) - } -} - /// An accumulator to compute variance #[derive(Debug)] pub struct VarianceAccumulator { From 97e2ac3367e823e156083944f0e2717c338b650d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 9 Dec 2024 17:41:24 -0700 Subject: [PATCH 17/30] benches --- native/spark-expr/benches/aggregate.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/native/spark-expr/benches/aggregate.rs b/native/spark-expr/benches/aggregate.rs index 43194fdda..051ac5eb6 100644 --- a/native/spark-expr/benches/aggregate.rs +++ b/native/spark-expr/benches/aggregate.rs @@ -66,7 +66,6 @@ fn criterion_benchmark(c: &mut Criterion) { group.bench_function("avg_decimal_comet", |b| { let comet_avg_decimal = Arc::new(AggregateUDF::new_from_impl(AvgDecimal::new( - Arc::clone(&c1), DataType::Decimal128(38, 10), DataType::Decimal128(38, 10), ))); @@ -96,7 +95,7 @@ fn criterion_benchmark(c: &mut Criterion) { group.bench_function("sum_decimal_comet", |b| { let comet_sum_decimal = Arc::new(AggregateUDF::new_from_impl( - SumDecimal::try_new(Arc::clone(&c1), DataType::Decimal128(38, 10)).unwrap(), + SumDecimal::try_new(DataType::Decimal128(38, 10)).unwrap(), )); b.to_async(&rt).iter(|| { black_box(agg_test( From 6a73f627ba98b4fde459080b7401f7f465d11d2d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 10 Dec 2024 09:03:43 -0700 Subject: [PATCH 18/30] fix ScalarUDFImpl.return_type failure --- native/Cargo.lock | 44 +++++++++---------- native/Cargo.toml | 18 ++++---- .../core/src/execution/datafusion/planner.rs | 2 + 3 files changed, 33 insertions(+), 31 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 76a0d182b..f063909ff 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -806,7 +806,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "arrow-array", @@ -849,7 +849,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow-schema", "async-trait", @@ -947,7 +947,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -970,7 +970,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "log", "tokio", @@ -979,12 +979,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "chrono", @@ -1003,7 +1003,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -1028,7 +1028,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "datafusion-common", @@ -1039,7 +1039,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "arrow-buffer", @@ -1067,7 +1067,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -1088,7 +1088,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -1101,7 +1101,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "arrow-array", @@ -1123,7 +1123,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -1147,7 +1147,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "datafusion-common", "datafusion-expr", @@ -1161,7 +1161,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1170,7 +1170,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "proc-macro2", "quote", @@ -1180,7 +1180,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "async-trait", @@ -1199,7 +1199,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -1223,7 +1223,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -1236,7 +1236,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "datafusion-common", @@ -1252,7 +1252,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "ahash", "arrow", @@ -1285,7 +1285,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#d3cfc45cef28a2d707bdd4b14da33a4d38a209d6" +source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" dependencies = [ "arrow", "arrow-array", diff --git a/native/Cargo.toml b/native/Cargo.toml index b2daab271..ded7d89fa 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -22,7 +22,7 @@ resolver = "2" [workspace.package] version = "0.5.0" homepage = "https://datafusion.apache.org/comet" -repository = "https://github.com/apache/datafusion-comet" +repository = "https://github.com/andygrove/datafusion-comet" authors = ["Apache DataFusion "] description = "Apache DataFusion Comet: High performance accelerator for Apache Spark" readme = "README.md" @@ -39,14 +39,14 @@ arrow-buffer = { version = "53.3.0" } arrow-data = { version = "53.3.0" } arrow-schema = { version = "53.3.0" } parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git" } -datafusion = { git = "https://github.com/apache/datafusion.git", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-common = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade" } +datafusion = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } +datafusion-expr = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } +datafusion-execution = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } +datafusion-physical-plan = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } +datafusion-physical-expr = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 3159952ca..251faf645 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2021,6 +2021,8 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); + // TODO need to call `return_type_from_exprs` instead + #[allow(deprecated)] let data_type = func.inner().return_type(&coerced_types)?; (data_type, coerced_types) From 606403edb9105a5d01b857fd063b3e9be2173e94 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 10 Dec 2024 10:43:17 -0700 Subject: [PATCH 19/30] exclude test from miri --- native/core/src/execution/datafusion/planner.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 251faf645..85c65f107 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2447,6 +2447,7 @@ mod tests { } #[tokio::test()] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `rust_psm_stack_pointer` async fn from_datafusion_error_to_comet() { let err_msg = "exec error"; let err = datafusion_common::DataFusionError::Execution(err_msg.to_string()); From 3ae78665109c52f2b936fcc18dad9bc475623a2e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 10 Dec 2024 12:05:41 -0700 Subject: [PATCH 20/30] ignore correct test --- native/core/src/execution/datafusion/planner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 85c65f107..a16039d25 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2447,7 +2447,6 @@ mod tests { } #[tokio::test()] - #[cfg_attr(miri, ignore)] // miri can't call foreign function `rust_psm_stack_pointer` async fn from_datafusion_error_to_comet() { let err_msg = "exec error"; let err = datafusion_common::DataFusionError::Execution(err_msg.to_string()); @@ -2505,6 +2504,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `rust_psm_stack_pointer` fn spark_plan_metrics_filter() { let op_scan = create_scan(); let op = create_filter(op_scan, 0); From fd5279a39a3d134e24d21134f9a7d8b4fe3778a8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 10 Dec 2024 12:23:49 -0700 Subject: [PATCH 21/30] ignore another test --- native/core/src/execution/datafusion/planner.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index a16039d25..5154e0c0a 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2518,6 +2518,7 @@ mod tests { } #[test] + #[cfg_attr(miri, ignore)] // miri can't call foreign function `rust_psm_stack_pointer` fn spark_plan_metrics_hash_join() { let op_scan = create_scan(); let op_join = Operator { From 4638fe3aa5501966cd5d8b53acf26c698b10b3c9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 10 Dec 2024 18:47:09 -0700 Subject: [PATCH 22/30] remove miri checks --- .github/workflows/miri.yml | 49 ------------------- .../core/src/execution/datafusion/planner.rs | 2 - 2 files changed, 51 deletions(-) delete mode 100644 .github/workflows/miri.yml diff --git a/.github/workflows/miri.yml b/.github/workflows/miri.yml deleted file mode 100644 index a07ecc35e..000000000 --- a/.github/workflows/miri.yml +++ /dev/null @@ -1,49 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -name: Run Miri Safety Checks - -on: - push: - paths-ignore: - - "doc/**" - - "docs/**" - - "**.md" - pull_request: - paths-ignore: - - "doc/**" - - "docs/**" - - "**.md" - # manual trigger - # https://docs.github.com/en/actions/managing-workflow-runs/manually-running-a-workflow - workflow_dispatch: - -jobs: - miri: - name: "Miri" - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v3 - - name: Install Miri - run: | - rustup toolchain install nightly --component miri - rustup override set nightly - cargo miri setup - - name: Test with Miri - run: | - cd native - MIRIFLAGS="-Zmiri-disable-isolation" cargo miri test diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 5154e0c0a..251faf645 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2504,7 +2504,6 @@ mod tests { } #[test] - #[cfg_attr(miri, ignore)] // miri can't call foreign function `rust_psm_stack_pointer` fn spark_plan_metrics_filter() { let op_scan = create_scan(); let op = create_filter(op_scan, 0); @@ -2518,7 +2517,6 @@ mod tests { } #[test] - #[cfg_attr(miri, ignore)] // miri can't call foreign function `rust_psm_stack_pointer` fn spark_plan_metrics_hash_join() { let op_scan = create_scan(); let op_join = Operator { From febc1f1ec1301f9b359fc23ad6a117224fce35b7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 11 Dec 2024 05:38:09 -0700 Subject: [PATCH 23/30] use return_type_from_exprs --- native/core/src/execution/datafusion/planner.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 251faf645..e07f4ea05 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2021,9 +2021,11 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - // TODO need to call `return_type_from_exprs` instead - #[allow(deprecated)] - let data_type = func.inner().return_type(&coerced_types)?; + let data_type = func.inner().return_type_from_exprs( + &args, + &input_schema, + &coerced_types, + )?; (data_type, coerced_types) } From 93187d0e7947c569aceedea11688abb9db61f749 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 11 Dec 2024 06:07:02 -0700 Subject: [PATCH 24/30] Revert "use return_type_from_exprs" This reverts commit febc1f1ec1301f9b359fc23ad6a117224fce35b7. --- native/core/src/execution/datafusion/planner.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index e07f4ea05..251faf645 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2021,11 +2021,9 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - let data_type = func.inner().return_type_from_exprs( - &args, - &input_schema, - &coerced_types, - )?; + // TODO need to call `return_type_from_exprs` instead + #[allow(deprecated)] + let data_type = func.inner().return_type(&coerced_types)?; (data_type, coerced_types) } From 1ed7f3ab054992c6c48123ee462492e2cbbd501b Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 11 Dec 2024 06:08:42 -0700 Subject: [PATCH 25/30] use DF main branch --- native/Cargo.lock | 44 ++++++++++++++++++++++---------------------- native/Cargo.toml | 16 ++++++++-------- 2 files changed, 30 insertions(+), 30 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index f063909ff..8eeb47246 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -806,7 +806,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "arrow-array", @@ -849,7 +849,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow-schema", "async-trait", @@ -947,7 +947,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -970,7 +970,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "log", "tokio", @@ -979,12 +979,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "chrono", @@ -1003,7 +1003,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -1028,7 +1028,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "datafusion-common", @@ -1039,7 +1039,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "arrow-buffer", @@ -1067,7 +1067,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -1088,7 +1088,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -1101,7 +1101,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "arrow-array", @@ -1123,7 +1123,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -1147,7 +1147,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "datafusion-common", "datafusion-expr", @@ -1161,7 +1161,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1170,7 +1170,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "proc-macro2", "quote", @@ -1180,7 +1180,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "async-trait", @@ -1199,7 +1199,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -1223,7 +1223,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -1236,7 +1236,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "datafusion-common", @@ -1252,7 +1252,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "ahash", "arrow", @@ -1285,7 +1285,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/andygrove/datafusion.git?branch=comet-upgrade#e17342cc6801f341eba8ee8262785c21dd0a9c2a" +source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" dependencies = [ "arrow", "arrow-array", diff --git a/native/Cargo.toml b/native/Cargo.toml index ded7d89fa..085296050 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,14 +39,14 @@ arrow-buffer = { version = "53.3.0" } arrow-data = { version = "53.3.0" } arrow-schema = { version = "53.3.0" } parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } -datafusion-common = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade" } -datafusion = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } -datafusion-expr = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } -datafusion-execution = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } -datafusion-physical-plan = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } -datafusion-physical-expr = { git = "https://github.com/andygrove/datafusion.git", branch="comet-upgrade", default-features = false } +datafusion-common = { git = "https://github.com/apache/datafusion.git" } +datafusion = { git = "https://github.com/apache/datafusion.git", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } From b7dcbd31ac631cd74d3496c3d384ca9840fbaef9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 11 Dec 2024 07:07:54 -0700 Subject: [PATCH 26/30] hacky workaround for regression in ScalarUDFImpl.return_type --- native/core/src/execution/datafusion/planner.rs | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/native/core/src/execution/datafusion/planner.rs b/native/core/src/execution/datafusion/planner.rs index 251faf645..fc077248d 100644 --- a/native/core/src/execution/datafusion/planner.rs +++ b/native/core/src/execution/datafusion/planner.rs @@ -2021,9 +2021,15 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - // TODO need to call `return_type_from_exprs` instead - #[allow(deprecated)] - let data_type = func.inner().return_type(&coerced_types)?; + let data_type = match fun_name { + // workaround for https://github.com/apache/datafusion/issues/13716 + "datepart" => DataType::Int32, + _ => { + // TODO need to call `return_type_from_exprs` instead + #[allow(deprecated)] + func.inner().return_type(&coerced_types)? + } + }; (data_type, coerced_types) } From 7bcfd18766118289b66117cd050c257e2490913d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 11 Dec 2024 08:46:05 -0700 Subject: [PATCH 27/30] fix repo url --- native/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/Cargo.toml b/native/Cargo.toml index 085296050..b2daab271 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -22,7 +22,7 @@ resolver = "2" [workspace.package] version = "0.5.0" homepage = "https://datafusion.apache.org/comet" -repository = "https://github.com/andygrove/datafusion-comet" +repository = "https://github.com/apache/datafusion-comet" authors = ["Apache DataFusion "] description = "Apache DataFusion Comet: High performance accelerator for Apache Spark" readme = "README.md" From e14a07af002a94214a7749823f600ab5b0e7db34 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 11 Dec 2024 08:47:28 -0700 Subject: [PATCH 28/30] pin to revision --- native/Cargo.lock | 44 ++++++++++++++++++++++---------------------- native/Cargo.toml | 16 ++++++++-------- 2 files changed, 30 insertions(+), 30 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 8eeb47246..dad9aa961 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -806,7 +806,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "arrow-array", @@ -849,7 +849,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow-schema", "async-trait", @@ -947,7 +947,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -970,7 +970,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "log", "tokio", @@ -979,12 +979,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "chrono", @@ -1003,7 +1003,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -1028,7 +1028,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "datafusion-common", @@ -1039,7 +1039,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "arrow-buffer", @@ -1067,7 +1067,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -1088,7 +1088,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -1101,7 +1101,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "arrow-array", @@ -1123,7 +1123,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -1147,7 +1147,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "datafusion-common", "datafusion-expr", @@ -1161,7 +1161,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1170,7 +1170,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "proc-macro2", "quote", @@ -1180,7 +1180,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "async-trait", @@ -1199,7 +1199,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -1223,7 +1223,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -1236,7 +1236,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "datafusion-common", @@ -1252,7 +1252,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "ahash", "arrow", @@ -1285,7 +1285,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git#a52eb47cf8ed0f74936cf5c2ae179e006f87826f" +source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" dependencies = [ "arrow", "arrow-array", diff --git a/native/Cargo.toml b/native/Cargo.toml index b2daab271..9353a2b85 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,14 +39,14 @@ arrow-buffer = { version = "53.3.0" } arrow-data = { version = "53.3.0" } arrow-schema = { version = "53.3.0" } parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git" } -datafusion = { git = "https://github.com/apache/datafusion.git", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", default-features = false } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "08119e6" } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } From 9653541240b7060bd00697acec5a45947bd296cc Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 13 Dec 2024 09:57:08 -0700 Subject: [PATCH 29/30] bump to latest rev --- native/Cargo.lock | 73 ++++++++++++++++------------------------------- native/Cargo.toml | 16 +++++------ 2 files changed, 32 insertions(+), 57 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 89ca7d87d..6f9e47850 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -806,7 +806,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", "arrow-array", @@ -837,7 +837,6 @@ dependencies = [ "log", "object_store", "parking_lot", - "paste", "rand", "sqlparser", "tempfile", @@ -849,7 +848,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow-schema", "async-trait", @@ -946,14 +945,13 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "ahash", "arrow", "arrow-array", "arrow-buffer", "arrow-schema", - "chrono", "half", "hashbrown 0.14.5", "indexmap", @@ -969,7 +967,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "log", "tokio", @@ -978,15 +976,14 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", - "chrono", "dashmap", "datafusion-common", "datafusion-expr", @@ -1002,12 +999,9 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ - "ahash", "arrow", - "arrow-array", - "arrow-buffer", "chrono", "datafusion-common", "datafusion-doc", @@ -1020,25 +1014,22 @@ dependencies = [ "recursive", "serde_json", "sqlparser", - "strum", - "strum_macros", ] [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", "datafusion-common", "itertools 0.13.0", - "paste", ] [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", "arrow-buffer", @@ -1066,7 +1057,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "ahash", "arrow", @@ -1087,20 +1078,19 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "ahash", "arrow", "datafusion-common", "datafusion-expr-common", "datafusion-physical-expr-common", - "rand", ] [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", "arrow-array", @@ -1116,29 +1106,19 @@ dependencies = [ "itertools 0.13.0", "log", "paste", - "rand", ] [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ - "ahash", "arrow", - "arrow-schema", "async-trait", "datafusion-catalog", "datafusion-common", - "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate-common", - "datafusion-physical-expr", - "datafusion-physical-expr-common", "datafusion-physical-plan", - "half", - "indexmap", - "log", "parking_lot", "paste", ] @@ -1146,11 +1126,13 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "datafusion-common", + "datafusion-doc", "datafusion-expr", "datafusion-functions-window-common", + "datafusion-macros", "datafusion-physical-expr", "datafusion-physical-expr-common", "log", @@ -1160,7 +1142,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1169,9 +1151,8 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ - "proc-macro2", "quote", "syn 2.0.87", ] @@ -1179,10 +1160,9 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", - "async-trait", "chrono", "datafusion-common", "datafusion-expr", @@ -1198,7 +1178,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "ahash", "arrow", @@ -1222,7 +1202,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "ahash", "arrow", @@ -1235,7 +1215,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", "datafusion-common", @@ -1251,7 +1231,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "ahash", "arrow", @@ -1274,17 +1254,15 @@ dependencies = [ "indexmap", "itertools 0.13.0", "log", - "once_cell", "parking_lot", "pin-project-lite", - "rand", "tokio", ] [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=08119e6#08119e634e1740801bf543d8106f92de340ec011" +source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" dependencies = [ "arrow", "arrow-array", @@ -2974,9 +2952,6 @@ name = "strum" version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" -dependencies = [ - "strum_macros", -] [[package]] name = "strum_macros" diff --git a/native/Cargo.toml b/native/Cargo.toml index 9353a2b85..861bb03cc 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,14 +39,14 @@ arrow-buffer = { version = "53.3.0" } arrow-data = { version = "53.3.0" } arrow-schema = { version = "53.3.0" } parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "08119e6" } -datafusion = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "08119e6", default-features = false } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "03e39da" } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } From df5d1837bbd820323500c98a381398562f545f64 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Dec 2024 11:57:08 -0700 Subject: [PATCH 30/30] bump to latest DF rev --- native/Cargo.lock | 46 +++++++++++++++++---------------- native/Cargo.toml | 17 ++++++------ native/spark-expr/Cargo.toml | 1 + native/spark-expr/src/stddev.rs | 6 ++++- 4 files changed, 39 insertions(+), 31 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 6f9e47850..7fa030398 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -806,7 +806,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "arrow-array", @@ -848,7 +848,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow-schema", "async-trait", @@ -932,6 +932,7 @@ dependencies = [ "datafusion", "datafusion-common", "datafusion-expr", + "datafusion-expr-common", "datafusion-physical-expr", "futures", "num", @@ -945,7 +946,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "ahash", "arrow", @@ -967,7 +968,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "log", "tokio", @@ -976,12 +977,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "dashmap", @@ -999,7 +1000,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "chrono", @@ -1019,7 +1020,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "datafusion-common", @@ -1029,7 +1030,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "arrow-buffer", @@ -1041,6 +1042,7 @@ dependencies = [ "datafusion-doc", "datafusion-execution", "datafusion-expr", + "datafusion-expr-common", "datafusion-macros", "hashbrown 0.14.5", "hex", @@ -1057,7 +1059,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "ahash", "arrow", @@ -1078,7 +1080,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "ahash", "arrow", @@ -1090,7 +1092,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "arrow-array", @@ -1111,7 +1113,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "async-trait", @@ -1126,7 +1128,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1142,7 +1144,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1151,7 +1153,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "quote", "syn 2.0.87", @@ -1160,7 +1162,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "chrono", @@ -1178,7 +1180,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "ahash", "arrow", @@ -1202,7 +1204,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "ahash", "arrow", @@ -1215,7 +1217,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "datafusion-common", @@ -1231,7 +1233,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "ahash", "arrow", @@ -1262,7 +1264,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=03e39da#03e39da62e403e064d21b57e9d6c200464c03749" +source = "git+https://github.com/apache/datafusion.git?rev=1fc7769#1fc776981496c0f0648ece8069992c65a7021c83" dependencies = [ "arrow", "arrow-array", diff --git a/native/Cargo.toml b/native/Cargo.toml index 861bb03cc..93fe5992f 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,14 +39,15 @@ arrow-buffer = { version = "53.3.0" } arrow-data = { version = "53.3.0" } arrow-schema = { version = "53.3.0" } parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "03e39da" } -datafusion = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "03e39da", default-features = false } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769" } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false } datafusion-comet-spark-expr = { path = "spark-expr", version = "0.5.0" } datafusion-comet-proto = { path = "proto", version = "0.5.0" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index d0bc2fd9d..07fa41e67 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -36,6 +36,7 @@ chrono = { workspace = true } datafusion = { workspace = true } datafusion-common = { workspace = true } datafusion-expr = { workspace = true } +datafusion-expr-common = { workspace = true } datafusion-physical-expr = { workspace = true } chrono-tz = { workspace = true } num = { workspace = true } diff --git a/native/spark-expr/src/stddev.rs b/native/spark-expr/src/stddev.rs index bfb56dcc2..1ec5ffb69 100644 --- a/native/spark-expr/src/stddev.rs +++ b/native/spark-expr/src/stddev.rs @@ -56,7 +56,11 @@ impl Stddev { Self { name: name.into(), signature: Signature::coercible( - vec![Arc::new(NativeType::Float64)], + vec![ + datafusion_expr_common::signature::TypeSignatureClass::Native(Arc::new( + NativeType::Float64, + )), + ], Volatility::Immutable, ), stats_type,