From dfd68aa827f18af11f6d8f56290d9c2986234556 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 7 Dec 2024 10:05:18 -0700 Subject: [PATCH 01/40] 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 a7f8359d87..67d041a399 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 85c46a6d02..4ac85479f2 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 daa0837c82..80dba4c038 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 871786bb15..72d7729d93 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 ed03ab667f..e922171bd2 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 48b80384b0..75841d1812 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 33c4924cbd..352f176002 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 c6a7a4143d..68c8ae7299 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 3d74b277cc..9077c0e485 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 b439e29e6a..71cd035d2e 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 532bf74375..7d637f6840 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 a265fdc29e..163e1560b6 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 c227b3a025..bde6e703d6 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 d885ff90b6..ab142aee69 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 db4ad1956a..18a2314fb1 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/40] 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 75841d1812..6792eb753d 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 352f176002..59904cdb78 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 6bf35e7115..e5f36c6f95 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 bde6e703d6..9a73ad08d6 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 1ba495e215..3cf604da0b 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/40] 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 80dba4c038..4b9753ec51 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 7d637f6840..65517431d2 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 c6209406fd..b3d5baf3b7 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/40] 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 6792eb753d..2487e64bce 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 59904cdb78..747288ca86 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 9a73ad08d6..03c8163811 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/40] 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 2487e64bce..a56f38f8ba 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 747288ca86..5259e7d36d 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 a2b9ebe5b5..36234935e1 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 03c8163811..94bd93d7c4 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/40] 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 a56f38f8ba..2bb14df36e 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 5259e7d36d..a83dba5d63 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 06717aabeb..71ff0e9dcc 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 94bd93d7c4..15f446ef35 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/40] 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 b3d5baf3b7..43194fdda2 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/40] 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 67d041a399..76a0d182ba 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 4ac85479f2..b2daab271d 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/40] 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 7820497d46..befada923e 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 163e1560b6..47d6a9a6f1 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 36234935e1..716e38bb25 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 f62d0220c9..5b886893fe 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 e5f36c6f95..ef8480f355 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 9166e39766..56d8e231c3 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 193a90fb55..f1bafa0833 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 7dc17b5688..ff823894f8 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 c5331ad7bd..4282f87669 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 c7626285a2..cfbfc8ae23 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 3cf604da0b..330fe9f424 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 cda8246d90..3f0fe8fc9f 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 ab142aee69..35ddd2185a 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 91953dd600..6e2b38e84c 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 1f68eb8606..81421f3cf2 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 18a2314fb1..778a094e44 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 2f4d8091c2..93caa3808f 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/40] 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 de922d8312..3e52e4f18b 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 e922171bd2..e67283dc96 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 8dfe717422..c58f4b6a8f 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 200b4ec5a3..6d972c86a5 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 3eeb29c16e..3be7df9d2c 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 a6babd0f7e..f0395103c1 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 a83dba5d63..ce0283109e 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 befada923e..751676b73f 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 47d6a9a6f1..1fe2fa2fb7 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 716e38bb25..a78915faf3 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 5b886893fe..1d616332e0 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 ef8480f355..b656e4b1d8 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 56d8e231c3..4231be1474 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 f1bafa0833..8091a2f2b1 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 ff823894f8..ca655b8698 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 4282f87669..2ebe23adeb 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 cfbfc8ae23..5679812d71 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 330fe9f424..c38145bc17 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 3f0fe8fc9f..190011b547 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 35ddd2185a..ceda6755a1 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 6e2b38e84c..0423e378ca 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 81421f3cf2..64e318459d 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 778a094e44..690106517d 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 93caa3808f..fa16fac12d 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/40] 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 a78915faf3..677bdb596f 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 1d616332e0..471242f9ef 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 8091a2f2b1..a0b01bb612 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 ca655b8698..a313f88da3 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 2ebe23adeb..0b9a5df8ab 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 c38145bc17..898a2c0c47 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 190011b547..3c450ab856 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 0423e378ca..9cddae3503 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 64e318459d..1dca149c86 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/40] 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 3e52e4f18b..b98f0ca0fc 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 e67283dc96..ef4facdadd 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 c58f4b6a8f..7cd66c1527 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 6d972c86a5..3aef4cd69d 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 3be7df9d2c..a3eaa8c5c5 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 f0395103c1..b0f239d89f 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 68b97d6608..923df014fc 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 35fa23b460..f42a148c65 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 8afe134cd3..8dab0088fc 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 677bdb596f..818064554f 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 471242f9ef..8cfeada0ef 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 a0b01bb612..d6eeffb4b8 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 a313f88da3..7445784d41 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 0b9a5df8ab..5ead1489bf 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 3c450ab856..48ef437856 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 9cddae3503..4c5afa6b48 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 1dca149c86..9c636763a6 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/40] 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 b98f0ca0fc..bf8fd7f352 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 ef4facdadd..81d5c2c0a1 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 3aef4cd69d..4095d8d0a1 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 a3eaa8c5c5..68de583f8b 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 b0f239d89f..6ba290a71e 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/40] 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 818064554f..11fa25ef65 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 8cfeada0ef..5eb09ab8e0 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 d6eeffb4b8..cf641e14a6 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 7445784d41..cffd8b2204 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 5ead1489bf..7088b6e79f 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 48ef437856..c9c5bffdaa 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 4c5afa6b48..ba3081b2c6 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 9c636763a6..91b46c6f04 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/40] 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 bf8fd7f352..c6f2a1ca98 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 81d5c2c0a1..46d35691f5 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 7cd66c1527..f35f91ea4b 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 4095d8d0a1..f9433840e6 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 11fa25ef65..e1bc4944ef 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/40] 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 c6f2a1ca98..17c28fe1e7 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 46d35691f5..528bbd5d96 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 f35f91ea4b..264da71bc0 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 f9433840e6..95f49d98fa 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 68de583f8b..d933a6096a 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 6ba290a71e..14f68c9cd6 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 ce0283109e..3159952ca9 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 751676b73f..816440ac9a 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 1fe2fa2fb7..05fc28e583 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 e1bc4944ef..d7c31836ff 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 5eb09ab8e0..17c603c28c 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 b656e4b1d8..e4ddab95de 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 4231be1474..fa3563cdea 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 cf641e14a6..01c754ad6d 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 cffd8b2204..fc31b11a0b 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 7088b6e79f..078ce4b5a4 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 5679812d71..7f367a8bb9 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 898a2c0c47..bfb56dcc20 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 c9c5bffdaa..7cc49e4281 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 ceda6755a1..f3f34d9bfa 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 ba3081b2c6..fb549f9ce8 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 690106517d..18a2314fb1 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 fa16fac12d..f3648a679f 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/40] 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 43194fdda2..051ac5eb62 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/40] 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 76a0d182ba..f063909ff3 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 b2daab271d..ded7d89fa2 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 3159952ca9..251faf6457 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/40] 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 251faf6457..85c65f1078 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/40] 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 85c65f1078..a16039d25b 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/40] 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 a16039d25b..5154e0c0a9 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/40] 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 a07ecc35ef..0000000000 --- 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 5154e0c0a9..251faf6457 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/40] 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 251faf6457..e07f4ea057 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/40] 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 e07f4ea057..251faf6457 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/40] 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 f063909ff3..8eeb472465 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 ded7d89fa2..0852960501 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/40] 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 251faf6457..fc077248d5 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/40] 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 0852960501..b2daab271d 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/40] 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 8eeb472465..dad9aa961d 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 b2daab271d..9353a2b850 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/40] 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 89ca7d87d7..6f9e478508 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 9353a2b850..861bb03cc0 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/40] 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 6f9e478508..7fa030398e 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 861bb03cc0..93fe5992f1 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 d0bc2fd9dd..07fa41e67f 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 bfb56dcc20..1ec5ffb69a 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, From d1f646a210b2f72114865cd249f72bb99683b61e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 19 Dec 2024 14:50:00 -0700 Subject: [PATCH 31/40] bump DF to rev 9f530dd --- native/Cargo.toml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/native/Cargo.toml b/native/Cargo.toml index 1b1544c5c3..f2310b3f94 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,15 +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 = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "1fc7769" } -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 = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd" } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", 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 2d08de53254f46084900a2fbd0bce4a537590b50 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 19 Dec 2024 14:54:58 -0700 Subject: [PATCH 32/40] add Cargo.lock --- native/Cargo.lock | 3767 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 3767 insertions(+) create mode 100644 native/Cargo.lock diff --git a/native/Cargo.lock b/native/Cargo.lock new file mode 100644 index 0000000000..aad1733ad7 --- /dev/null +++ b/native/Cargo.lock @@ -0,0 +1,3767 @@ +# This file is automatically @generated by Cargo. +# It is not intended for manual editing. +version = 3 + +[[package]] +name = "addr2line" +version = "0.24.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" +dependencies = [ + "gimli", +] + +[[package]] +name = "adler2" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" + +[[package]] +name = "ahash" +version = "0.8.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +dependencies = [ + "cfg-if", + "const-random", + "getrandom", + "once_cell", + "version_check", + "zerocopy", +] + +[[package]] +name = "aho-corasick" +version = "1.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e60d3430d3a69478ad0993f19238d2df97c507009a52b3c10addcd7f6bcb916" +dependencies = [ + "memchr", +] + +[[package]] +name = "alloc-no-stdlib" +version = "2.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc7bb162ec39d46ab1ca8c77bf72e890535becd1751bb45f64c597edb4c8c6b3" + +[[package]] +name = "alloc-stdlib" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94fb8275041c72129eb51b7d0322c29b8387a0386127718b096429201a5d6ece" +dependencies = [ + "alloc-no-stdlib", +] + +[[package]] +name = "allocator-api2" +version = "0.2.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" + +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + +[[package]] +name = "android_system_properties" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "819e7219dbd41043ac279b19830f2efc897156490d7fd6ea916720117ee66311" +dependencies = [ + "libc", +] + +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + +[[package]] +name = "anstyle" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" + +[[package]] +name = "anyhow" +version = "1.0.94" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1fd03a028ef38ba2276dce7e33fcd6369c158a1bca17946c4b1b701891c1ff7" + +[[package]] +name = "arc-swap" +version = "1.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" + +[[package]] +name = "arrayref" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "76a2e8124351fda1ef8aaaa3bbd7ebbcb486bbcd4225aca0aa0d84bb2db8fecb" + +[[package]] +name = "arrayvec" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" + +[[package]] +name = "arrow" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c91839b07e474b3995035fd8ac33ee54f9c9ccbbb1ea33d9909c71bffdf1259d" +dependencies = [ + "arrow-arith", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-csv", + "arrow-data", + "arrow-ipc", + "arrow-json", + "arrow-ord", + "arrow-row", + "arrow-schema", + "arrow-select", + "arrow-string", +] + +[[package]] +name = "arrow-arith" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "855c57c4efd26722b044dcd3e348252560e3e0333087fb9f6479dc0bf744054f" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "half", + "num", +] + +[[package]] +name = "arrow-array" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd03279cea46569acf9295f6224fbc370c5df184b4d2ecfe97ccb131d5615a7f" +dependencies = [ + "ahash", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "chrono-tz 0.10.0", + "half", + "hashbrown 0.15.2", + "num", +] + +[[package]] +name = "arrow-buffer" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e4a9b9b1d6d7117f6138e13bc4dd5daa7f94e671b70e8c9c4dc37b4f5ecfc16" +dependencies = [ + "bytes", + "half", + "num", +] + +[[package]] +name = "arrow-cast" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc70e39916e60c5b7af7a8e2719e3ae589326039e1e863675a008bee5ffe90fd" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "atoi", + "base64", + "chrono", + "comfy-table", + "half", + "lexical-core", + "num", + "ryu", +] + +[[package]] +name = "arrow-csv" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "789b2af43c1049b03a8d088ff6b2257cdcea1756cd76b174b1f2600356771b97" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "csv", + "csv-core", + "lazy_static", + "lexical-core", + "regex", +] + +[[package]] +name = "arrow-data" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e4e75edf21ffd53744a9b8e3ed11101f610e7ceb1a29860432824f1834a1f623" +dependencies = [ + "arrow-buffer", + "arrow-schema", + "half", + "num", +] + +[[package]] +name = "arrow-ipc" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d186a909dece9160bf8312f5124d797884f608ef5435a36d9d608e0b2a9bcbf8" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "flatbuffers", + "lz4_flex", +] + +[[package]] +name = "arrow-json" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b66ff2fedc1222942d0bd2fd391cb14a85baa3857be95c9373179bd616753b85" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-schema", + "chrono", + "half", + "indexmap", + "lexical-core", + "num", + "serde", + "serde_json", +] + +[[package]] +name = "arrow-ord" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ece7b5bc1180e6d82d1a60e1688c199829e8842e38497563c3ab6ea813e527fd" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "half", + "num", +] + +[[package]] +name = "arrow-row" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "745c114c8f0e8ce211c83389270de6fbe96a9088a7b32c2a041258a443fe83ff" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "half", +] + +[[package]] +name = "arrow-schema" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b95513080e728e4cec37f1ff5af4f12c9688d47795d17cda80b6ec2cf74d4678" +dependencies = [ + "bitflags 2.6.0", +] + +[[package]] +name = "arrow-select" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e415279094ea70323c032c6e739c48ad8d80e78a09bef7117b8718ad5bf3722" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "num", +] + +[[package]] +name = "arrow-string" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "11d956cae7002eb8d83a27dbd34daaea1cf5b75852f0b84deb4d93a276e92bbf" +dependencies = [ + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "arrow-select", + "memchr", + "num", + "regex", + "regex-syntax", +] + +[[package]] +name = "assertables" +version = "7.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c24e9d990669fbd16806bff449e4ac644fd9b1fca014760087732fe4102f131" + +[[package]] +name = "async-trait" +version = "0.1.83" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "atoi" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f28d99ec8bfea296261ca1af174f24225171fea9664ba9003cbebee704810528" +dependencies = [ + "num-traits", +] + +[[package]] +name = "autocfg" +version = "1.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" + +[[package]] +name = "backtrace" +version = "0.3.74" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8d82cb332cdfaed17ae235a638438ac4d4839913cc2af585c3c6746e8f8bee1a" +dependencies = [ + "addr2line", + "cfg-if", + "libc", + "miniz_oxide", + "object", + "rustc-demangle", + "windows-targets 0.52.6", +] + +[[package]] +name = "base64" +version = "0.22.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" + +[[package]] +name = "bigdecimal" +version = "0.4.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f31f3af01c5c65a07985c804d3366560e6fa7883d640a122819b14ec327482c" +dependencies = [ + "autocfg", + "libm", + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "bitflags" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bitflags" +version = "2.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b048fb63fd8b5923fc5aa7b340d8e156aec7ec02f0c78fa8a6ddc2613f6f71de" + +[[package]] +name = "blake2" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46502ad458c9a52b69d4d4d32775c788b7a1b85e8bc9d482d92250fc0e3f8efe" +dependencies = [ + "digest", +] + +[[package]] +name = "blake3" +version = "1.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8ee0c1824c4dea5b5f81736aff91bae041d2c07ee1192bec91054e10e3e601e" +dependencies = [ + "arrayref", + "arrayvec", + "cc", + "cfg-if", + "constant_time_eq", +] + +[[package]] +name = "block-buffer" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3078c7629b62d3f0439517fa394996acacc5cbc91c5a20d8c658e77abd503a71" +dependencies = [ + "generic-array", +] + +[[package]] +name = "brotli" +version = "3.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d640d25bc63c50fb1f0b545ffd80207d2e10a4c965530809b40ba3386825c391" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor 2.5.1", +] + +[[package]] +name = "brotli" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", + "brotli-decompressor 4.0.1", +] + +[[package]] +name = "brotli-decompressor" +version = "2.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4e2e4afe60d7dd600fdd3de8d0f08c2b7ec039712e3b6137ff98b7004e82de4f" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "brotli-decompressor" +version = "4.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a45bd2e4095a8b518033b128020dd4a55aab1c0a381ba4404a472630f4bc362" +dependencies = [ + "alloc-no-stdlib", + "alloc-stdlib", +] + +[[package]] +name = "bumpalo" +version = "3.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" + +[[package]] +name = "bytemuck" +version = "1.20.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b37c88a63ffd85d15b406896cc343916d7cf57838a847b3a6f2ca5d39a5695a" + +[[package]] +name = "byteorder" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" + +[[package]] +name = "bytes" +version = "1.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "325918d6fe32f23b19878fe4b34794ae41fc19ddbe53b10571a4874d44ffd39b" + +[[package]] +name = "cast" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" + +[[package]] +name = "cc" +version = "1.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c31a0499c1dc64f458ad13872de75c0eb7e3fdb0e67964610c914b034fc5956e" +dependencies = [ + "jobserver", + "libc", + "shlex", +] + +[[package]] +name = "cesu8" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d43a04d8753f35258c91f8ec639f792891f748a1edbd759cf1dcea3382ad83c" + +[[package]] +name = "cfg-if" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" + +[[package]] +name = "chrono" +version = "0.4.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e36cc9d416881d2e24f9a963be5fb1cd90966419ac844274161d10488b3e825" +dependencies = [ + "android-tzdata", + "iana-time-zone", + "num-traits", + "windows-targets 0.52.6", +] + +[[package]] +name = "chrono-tz" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d59ae0466b83e838b81a54256c39d5d7c20b9d7daa10510a242d9b75abd5936e" +dependencies = [ + "chrono", + "chrono-tz-build 0.2.1", + "phf", +] + +[[package]] +name = "chrono-tz" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd6dd8046d00723a59a2f8c5f295c515b9bb9a331ee4f8f3d4dd49e428acd3b6" +dependencies = [ + "chrono", + "chrono-tz-build 0.4.0", + "phf", +] + +[[package]] +name = "chrono-tz-build" +version = "0.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "433e39f13c9a060046954e0592a8d0a4bcb1040125cbf91cb8ee58964cfb350f" +dependencies = [ + "parse-zoneinfo", + "phf", + "phf_codegen", +] + +[[package]] +name = "chrono-tz-build" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e94fea34d77a245229e7746bd2beb786cd2a896f306ff491fb8cecb3074b10a7" +dependencies = [ + "parse-zoneinfo", + "phf_codegen", +] + +[[package]] +name = "ciborium" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" + +[[package]] +name = "ciborium-ll" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" +dependencies = [ + "ciborium-io", + "half", +] + +[[package]] +name = "clap" +version = "4.5.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3135e7ec2ef7b10c6ed8950f0f792ed96ee093fa088608f1c76e569722700c84" +dependencies = [ + "clap_builder", +] + +[[package]] +name = "clap_builder" +version = "4.5.23" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30582fc632330df2bd26877bde0c1f4470d57c582bbc070376afcd04d8cb4838" +dependencies = [ + "anstyle", + "clap_lex", +] + +[[package]] +name = "clap_lex" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" + +[[package]] +name = "combine" +version = "4.6.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba5a308b75df32fe02788e748662718f03fde005016435c444eea572398219fd" +dependencies = [ + "bytes", + "memchr", +] + +[[package]] +name = "comfy-table" +version = "7.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24f165e7b643266ea80cb858aed492ad9280e3e05ce24d4a99d7d7b889b6a4d9" +dependencies = [ + "strum", + "strum_macros", + "unicode-width", +] + +[[package]] +name = "const-random" +version = "0.1.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87e00182fe74b066627d63b85fd550ac2998d4b0bd86bfed477a0ae4c7c71359" +dependencies = [ + "const-random-macro", +] + +[[package]] +name = "const-random-macro" +version = "0.1.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" +dependencies = [ + "getrandom", + "once_cell", + "tiny-keccak", +] + +[[package]] +name = "constant_time_eq" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" + +[[package]] +name = "core-foundation-sys" +version = "0.8.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" + +[[package]] +name = "cpp_demangle" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96e58d342ad113c2b878f16d5d034c03be492ae460cdbc02b7f0f2284d310c7d" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "cpufeatures" +version = "0.2.16" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "16b80225097f2e5ae4e7179dd2266824648f3e2f49d9134d584b76389d31c4c3" +dependencies = [ + "libc", +] + +[[package]] +name = "crc32fast" +version = "1.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" +dependencies = [ + "cfg-if", +] + +[[package]] +name = "criterion" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +dependencies = [ + "anes", + "cast", + "ciborium", + "clap", + "criterion-plot", + "futures", + "is-terminal", + "itertools 0.10.5", + "num-traits", + "once_cell", + "oorandom", + "plotters", + "rayon", + "regex", + "serde", + "serde_derive", + "serde_json", + "tinytemplate", + "tokio", + "walkdir", +] + +[[package]] +name = "criterion-plot" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +dependencies = [ + "cast", + "itertools 0.10.5", +] + +[[package]] +name = "crossbeam-deque" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" +dependencies = [ + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-utils" +version = "0.8.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" + +[[package]] +name = "crunchy" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a81dae078cea95a014a339291cec439d2f232ebe854a9d672b796c6afafa9b7" + +[[package]] +name = "crypto-common" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bfb12502f3fc46cca1bb51ac28df9d618d813cdc3d2f25b9fe775a34af26bb3" +dependencies = [ + "generic-array", + "typenum", +] + +[[package]] +name = "csv" +version = "1.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" +dependencies = [ + "csv-core", + "itoa", + "ryu", + "serde", +] + +[[package]] +name = "csv-core" +version = "0.1.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5efa2b3d7902f4b634a20cae3c9c4e6209dc4779feb6863329607560143efa70" +dependencies = [ + "memchr", +] + +[[package]] +name = "dashmap" +version = "6.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5041cc499144891f3790297212f32a74fb938e5136a14943f338ef9e0ae276cf" +dependencies = [ + "cfg-if", + "crossbeam-utils", + "hashbrown 0.14.5", + "lock_api", + "once_cell", + "parking_lot_core", +] + +[[package]] +name = "datafusion" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "arrow-array", + "arrow-ipc", + "arrow-schema", + "async-trait", + "bytes", + "chrono", + "dashmap", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-table", + "datafusion-functions-window", + "datafusion-optimizer", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-optimizer", + "datafusion-physical-plan", + "datafusion-sql", + "futures", + "glob", + "itertools 0.13.0", + "log", + "object_store", + "parking_lot", + "parquet", + "rand", + "regex", + "sqlparser", + "tempfile", + "tokio", + "url", + "uuid", +] + +[[package]] +name = "datafusion-catalog" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow-schema", + "async-trait", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-plan", + "parking_lot", +] + +[[package]] +name = "datafusion-comet" +version = "0.5.0" +dependencies = [ + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "assertables", + "async-trait", + "brotli 3.5.0", + "bytes", + "crc32fast", + "criterion", + "datafusion", + "datafusion-comet-proto", + "datafusion-comet-spark-expr", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-nested", + "datafusion-physical-expr", + "flate2", + "futures", + "hex", + "itertools 0.11.0", + "jni", + "lazy_static", + "log", + "log4rs", + "lz4", + "mimalloc", + "num", + "once_cell", + "parquet", + "paste", + "pprof", + "prost 0.12.6", + "rand", + "regex", + "serde", + "simd-adler32", + "snap", + "tempfile", + "thiserror", + "tokio", + "zstd 0.11.2+zstd.1.5.2", +] + +[[package]] +name = "datafusion-comet-proto" +version = "0.5.0" +dependencies = [ + "prost 0.12.6", + "prost-build", +] + +[[package]] +name = "datafusion-comet-spark-expr" +version = "0.5.0" +dependencies = [ + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-data", + "arrow-schema", + "chrono", + "chrono-tz 0.8.6", + "criterion", + "datafusion", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", + "futures", + "num", + "parquet", + "rand", + "regex", + "thiserror", + "tokio", + "twox-hash 2.1.0", +] + +[[package]] +name = "datafusion-common" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "half", + "hashbrown 0.14.5", + "indexmap", + "libc", + "object_store", + "parquet", + "paste", + "recursive", + "sqlparser", + "tokio", + "web-time", +] + +[[package]] +name = "datafusion-common-runtime" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "log", + "tokio", +] + +[[package]] +name = "datafusion-doc" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" + +[[package]] +name = "datafusion-execution" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "dashmap", + "datafusion-common", + "datafusion-expr", + "futures", + "log", + "object_store", + "parking_lot", + "rand", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "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", +] + +[[package]] +name = "datafusion-expr-common" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "datafusion-common", + "itertools 0.13.0", +] + +[[package]] +name = "datafusion-functions" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "arrow-buffer", + "base64", + "blake2", + "blake3", + "chrono", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", + "hashbrown 0.14.5", + "hex", + "itertools 0.13.0", + "log", + "md-5", + "rand", + "regex", + "sha2", + "unicode-segmentation", + "uuid", +] + +[[package]] +name = "datafusion-functions-aggregate" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +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", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-aggregate-common" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr-common", + "datafusion-physical-expr-common", +] + +[[package]] +name = "datafusion-functions-nested" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-ord", + "arrow-schema", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-physical-expr-common", + "itertools 0.13.0", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-table" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "async-trait", + "datafusion-catalog", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-plan", + "parking_lot", + "paste", +] + +[[package]] +name = "datafusion-functions-window" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "datafusion-common", + "datafusion-doc", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-window-common" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "datafusion-common", + "datafusion-physical-expr-common", +] + +[[package]] +name = "datafusion-macros" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "quote", + "syn 2.0.90", +] + +[[package]] +name = "datafusion-optimizer" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "chrono", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", + "indexmap", + "itertools 0.13.0", + "log", + "recursive", + "regex", + "regex-syntax", +] + +[[package]] +name = "datafusion-physical-expr" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-schema", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", + "half", + "hashbrown 0.14.5", + "indexmap", + "itertools 0.13.0", + "log", + "paste", + "petgraph", +] + +[[package]] +name = "datafusion-physical-expr-common" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "ahash", + "arrow", + "datafusion-common", + "datafusion-expr-common", + "hashbrown 0.14.5", + "itertools 0.13.0", +] + +[[package]] +name = "datafusion-physical-optimizer" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "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 = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "ahash", + "arrow", + "arrow-array", + "arrow-buffer", + "arrow-ord", + "arrow-schema", + "async-trait", + "chrono", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "futures", + "half", + "hashbrown 0.14.5", + "indexmap", + "itertools 0.13.0", + "log", + "parking_lot", + "pin-project-lite", + "tokio", +] + +[[package]] +name = "datafusion-sql" +version = "43.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +dependencies = [ + "arrow", + "arrow-array", + "arrow-schema", + "bigdecimal", + "datafusion-common", + "datafusion-expr", + "indexmap", + "log", + "recursive", + "regex", + "sqlparser", +] + +[[package]] +name = "debugid" +version = "0.8.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef552e6f588e446098f6ba40d89ac146c8c7b64aade83c051ee00bb5d2bc18d" +dependencies = [ + "uuid", +] + +[[package]] +name = "derivative" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "destructure_traitobject" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c877555693c14d2f84191cfd3ad8582790fc52b5e2274b40b59cf5f5cea25c7" + +[[package]] +name = "digest" +version = "0.10.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" +dependencies = [ + "block-buffer", + "crypto-common", + "subtle", +] + +[[package]] +name = "displaydoc" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "either" +version = "1.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "60b1af1c220855b6ceac025d3f6ecdd2b7c4894bfe9cd9bda4fbb4bc7c0d4cf0" + +[[package]] +name = "equivalent" +version = "1.0.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5443807d6dff69373d433ab9ef5378ad8df50ca6298caf15de6e52e24aaf54d5" + +[[package]] +name = "errno" +version = "0.3.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" +dependencies = [ + "libc", + "windows-sys 0.59.0", +] + +[[package]] +name = "fastrand" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" + +[[package]] +name = "findshlibs" +version = "0.10.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "40b9e59cd0f7e0806cca4be089683ecb6434e602038df21fe6bf6711b2f07f64" +dependencies = [ + "cc", + "lazy_static", + "libc", + "winapi", +] + +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + +[[package]] +name = "flatbuffers" +version = "24.3.25" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" +dependencies = [ + "bitflags 1.3.2", + "rustc_version", +] + +[[package]] +name = "flate2" +version = "1.0.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c936bfdafb507ebbf50b8074c54fa31c5be9a1e7e5f467dd659697041407d07c" +dependencies = [ + "crc32fast", + "miniz_oxide", +] + +[[package]] +name = "fnv" +version = "1.0.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" + +[[package]] +name = "form_urlencoded" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" +dependencies = [ + "percent-encoding", +] + +[[package]] +name = "futures" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" +dependencies = [ + "futures-channel", + "futures-core", + "futures-executor", + "futures-io", + "futures-sink", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-channel" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" +dependencies = [ + "futures-core", + "futures-sink", +] + +[[package]] +name = "futures-core" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" + +[[package]] +name = "futures-executor" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" +dependencies = [ + "futures-core", + "futures-task", + "futures-util", +] + +[[package]] +name = "futures-io" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" + +[[package]] +name = "futures-macro" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "futures-sink" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" + +[[package]] +name = "futures-task" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" + +[[package]] +name = "futures-util" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" +dependencies = [ + "futures-channel", + "futures-core", + "futures-io", + "futures-macro", + "futures-sink", + "futures-task", + "memchr", + "pin-project-lite", + "pin-utils", + "slab", +] + +[[package]] +name = "generic-array" +version = "0.14.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +dependencies = [ + "typenum", + "version_check", +] + +[[package]] +name = "getrandom" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" +dependencies = [ + "cfg-if", + "libc", + "wasi", +] + +[[package]] +name = "gimli" +version = "0.31.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" + +[[package]] +name = "glob" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" + +[[package]] +name = "half" +version = "2.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6dd08c532ae367adf81c312a4580bc67f1d0fe8bc9c460520283f4c0ff277888" +dependencies = [ + "cfg-if", + "crunchy", + "num-traits", +] + +[[package]] +name = "hashbrown" +version = "0.14.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" +dependencies = [ + "ahash", + "allocator-api2", +] + +[[package]] +name = "hashbrown" +version = "0.15.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" + +[[package]] +name = "heck" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6d621efb26863f0e9924c6ac577e8275e5e6b77455db64ffa6c65c904e9e132c" +dependencies = [ + "unicode-segmentation", +] + +[[package]] +name = "heck" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" + +[[package]] +name = "hermit-abi" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fbf6a919d6cf397374f7dfeeea91d974c7c0a7221d0d0f4f20d859d329e53fcc" + +[[package]] +name = "hex" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" + +[[package]] +name = "home" +version = "0.5.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589533453244b0995c858700322199b2becb13b627df2851f64a2775d024abcf" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "humantime" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" + +[[package]] +name = "iana-time-zone" +version = "0.1.61" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "235e081f3925a06703c2d0117ea8b91f042756fd6e7a6e5d901e8ca1a996b220" +dependencies = [ + "android_system_properties", + "core-foundation-sys", + "iana-time-zone-haiku", + "js-sys", + "wasm-bindgen", + "windows-core", +] + +[[package]] +name = "iana-time-zone-haiku" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" +dependencies = [ + "cc", +] + +[[package]] +name = "icu_collections" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "db2fa452206ebee18c4b5c2274dbf1de17008e874b4dc4f0aea9d01ca79e4526" +dependencies = [ + "displaydoc", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_locid" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13acbb8371917fc971be86fc8057c41a64b521c184808a698c02acc242dbf637" +dependencies = [ + "displaydoc", + "litemap", + "tinystr", + "writeable", + "zerovec", +] + +[[package]] +name = "icu_locid_transform" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01d11ac35de8e40fdeda00d9e1e9d92525f3f9d887cdd7aa81d727596788b54e" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_locid_transform_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_locid_transform_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdc8ff3388f852bede6b579ad4e978ab004f139284d7b28715f773507b946f6e" + +[[package]] +name = "icu_normalizer" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19ce3e0da2ec68599d193c93d088142efd7f9c5d6fc9b803774855747dc6a84f" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_normalizer_data", + "icu_properties", + "icu_provider", + "smallvec", + "utf16_iter", + "utf8_iter", + "write16", + "zerovec", +] + +[[package]] +name = "icu_normalizer_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8cafbf7aa791e9b22bec55a167906f9e1215fd475cd22adfcf660e03e989516" + +[[package]] +name = "icu_properties" +version = "1.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93d6020766cfc6302c15dbbc9c8778c37e62c14427cb7f6e601d849e092aeef5" +dependencies = [ + "displaydoc", + "icu_collections", + "icu_locid_transform", + "icu_properties_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_properties_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "67a8effbc3dd3e4ba1afa8ad918d5684b8868b3b26500753effea8d2eed19569" + +[[package]] +name = "icu_provider" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ed421c8a8ef78d3e2dbc98a973be2f3770cb42b606e3ab18d6237c4dfde68d9" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_provider_macros", + "stable_deref_trait", + "tinystr", + "writeable", + "yoke", + "zerofrom", + "zerovec", +] + +[[package]] +name = "icu_provider_macros" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "idna" +version = "1.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "686f825264d630750a544639377bae737628043f20d38bbc029e8f29ea968a7e" +dependencies = [ + "idna_adapter", + "smallvec", + "utf8_iter", +] + +[[package]] +name = "idna_adapter" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "daca1df1c957320b2cf139ac61e7bd64fed304c5040df000a745aa1de3b4ef71" +dependencies = [ + "icu_normalizer", + "icu_properties", +] + +[[package]] +name = "indexmap" +version = "2.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62f822373a4fe84d4bb149bf54e584a7f4abec90e072ed49cda0edea5b95471f" +dependencies = [ + "equivalent", + "hashbrown 0.15.2", +] + +[[package]] +name = "inferno" +version = "0.11.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "232929e1d75fe899576a3d5c7416ad0d88dbfbb3c3d6aa00873a7408a50ddb88" +dependencies = [ + "ahash", + "indexmap", + "is-terminal", + "itoa", + "log", + "num-format", + "once_cell", + "quick-xml", + "rgb", + "str_stack", +] + +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + +[[package]] +name = "is-terminal" +version = "0.4.13" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "261f68e344040fbd0edea105bef17c66edf46f984ddb1115b775ce31be948f4b" +dependencies = [ + "hermit-abi", + "libc", + "windows-sys 0.52.0", +] + +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b1c173a5686ce8bfa551b3563d0c2170bf24ca44da99c7ca4bfdab5418c3fe57" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" +dependencies = [ + "either", +] + +[[package]] +name = "itertools" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "413ee7dfc52ee1a4949ceeb7dbc8a33f2d6c088194d9f922fb8318faf1f01186" +dependencies = [ + "either", +] + +[[package]] +name = "itoa" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d75a2a4b1b190afb6f5425f10f6a8f959d2ea0b9c2b1d79553551850539e4674" + +[[package]] +name = "java-locator" +version = "0.1.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f25f28894af6a5dd349ed5ec46e178654e75f62edb6717ac74007102a57deb5" +dependencies = [ + "glob", +] + +[[package]] +name = "jni" +version = "0.21.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1a87aa2bb7d2af34197c04845522473242e1aa17c12f4935d5856491a7fb8c97" +dependencies = [ + "cesu8", + "cfg-if", + "combine", + "java-locator", + "jni-sys", + "libloading", + "log", + "thiserror", + "walkdir", + "windows-sys 0.45.0", +] + +[[package]] +name = "jni-sys" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8eaf4bc02d17cbdd7ff4c7438cafcdf7fb9a4613313ad11b4f8fefe7d3fa0130" + +[[package]] +name = "jobserver" +version = "0.1.32" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" +dependencies = [ + "libc", +] + +[[package]] +name = "js-sys" +version = "0.3.76" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6717b6b5b077764fb5966237269cb3c64edddde4b14ce42647430a78ced9e7b7" +dependencies = [ + "once_cell", + "wasm-bindgen", +] + +[[package]] +name = "lazy_static" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" + +[[package]] +name = "lexical-core" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b765c31809609075565a70b4b71402281283aeda7ecaf4818ac14a7b2ade8958" +dependencies = [ + "lexical-parse-float", + "lexical-parse-integer", + "lexical-util", + "lexical-write-float", + "lexical-write-integer", +] + +[[package]] +name = "lexical-parse-float" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "de6f9cb01fb0b08060209a057c048fcbab8717b4c1ecd2eac66ebfe39a65b0f2" +dependencies = [ + "lexical-parse-integer", + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-parse-integer" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72207aae22fc0a121ba7b6d479e42cbfea549af1479c3f3a4f12c70dd66df12e" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "lexical-util" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5a82e24bf537fd24c177ffbbdc6ebcc8d54732c35b50a3f28cc3f4e4c949a0b3" +dependencies = [ + "static_assertions", +] + +[[package]] +name = "lexical-write-float" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c5afc668a27f460fb45a81a757b6bf2f43c2d7e30cb5a2dcd3abf294c78d62bd" +dependencies = [ + "lexical-util", + "lexical-write-integer", + "static_assertions", +] + +[[package]] +name = "lexical-write-integer" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "629ddff1a914a836fb245616a7888b62903aae58fa771e1d83943035efa0f978" +dependencies = [ + "lexical-util", + "static_assertions", +] + +[[package]] +name = "libc" +version = "0.2.169" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5aba8db14291edd000dfcc4d620c7ebfb122c613afb886ca8803fa4e128a20a" + +[[package]] +name = "libloading" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b67380fd3b2fbe7527a606e18729d21c6f3951633d0500574c4dc22d2d638b9f" +dependencies = [ + "cfg-if", + "winapi", +] + +[[package]] +name = "libm" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8355be11b20d696c8f18f6cc018c4e372165b1fa8126cef092399c9951984ffa" + +[[package]] +name = "libmimalloc-sys" +version = "0.1.39" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "23aa6811d3bd4deb8a84dde645f943476d13b248d818edcf8ce0b2f37f036b44" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "linux-raw-sys" +version = "0.4.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78b3ae25bc7c8c38cec158d1f2757ee79e9b3740fbc7ccf0e59e4b08d793fa89" + +[[package]] +name = "litemap" +version = "0.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ee93343901ab17bd981295f2cf0026d4ad018c7c31ba84549a4ddbb47a45104" + +[[package]] +name = "lock_api" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" +dependencies = [ + "autocfg", + "scopeguard", +] + +[[package]] +name = "log" +version = "0.4.22" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" +dependencies = [ + "serde", +] + +[[package]] +name = "log-mdc" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a94d21414c1f4a51209ad204c1776a3d0765002c76c6abcb602a6f09f1e881c7" + +[[package]] +name = "log4rs" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0816135ae15bd0391cf284eab37e6e3ee0a6ee63d2ceeb659862bd8d0a984ca6" +dependencies = [ + "anyhow", + "arc-swap", + "chrono", + "derivative", + "fnv", + "humantime", + "libc", + "log", + "log-mdc", + "once_cell", + "parking_lot", + "rand", + "serde", + "serde-value", + "serde_json", + "serde_yaml", + "thiserror", + "thread-id", + "typemap-ors", + "winapi", +] + +[[package]] +name = "lz4" +version = "1.28.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4d1febb2b4a79ddd1980eede06a8f7902197960aa0383ffcfdd62fe723036725" +dependencies = [ + "lz4-sys", +] + +[[package]] +name = "lz4-sys" +version = "1.11.1+lz4-1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6bd8c0d6c6ed0cd30b3652886bb8711dc4bb01d637a68105a3d5158039b418e6" +dependencies = [ + "cc", + "libc", +] + +[[package]] +name = "lz4_flex" +version = "0.11.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" +dependencies = [ + "twox-hash 1.6.3", +] + +[[package]] +name = "md-5" +version = "0.10.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d89e7ee0cfbedfc4da3340218492196241d89eefb6dab27de5df917a6d2e78cf" +dependencies = [ + "cfg-if", + "digest", +] + +[[package]] +name = "memchr" +version = "2.7.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" + +[[package]] +name = "memmap2" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd3f7eed9d3848f8b98834af67102b720745c4ec028fcd0aa0239277e7de374f" +dependencies = [ + "libc", +] + +[[package]] +name = "mimalloc" +version = "0.1.43" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68914350ae34959d83f732418d51e2427a794055d0b9529f48259ac07af65633" +dependencies = [ + "libmimalloc-sys", +] + +[[package]] +name = "miniz_oxide" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ffbe83022cedc1d264172192511ae958937694cd57ce297164951b8b3568394" +dependencies = [ + "adler2", +] + +[[package]] +name = "multimap" +version = "0.8.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5ce46fe64a9d73be07dcbe690a38ce1b293be448fd8ce1e6c1b8062c9f72c6a" + +[[package]] +name = "nix" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "598beaf3cc6fdd9a5dfb1630c2800c7acd31df7aaf0f565796fba2b53ca1af1b" +dependencies = [ + "bitflags 1.3.2", + "cfg-if", + "libc", +] + +[[package]] +name = "num" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + +[[package]] +name = "num-bigint" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" +dependencies = [ + "num-integer", + "num-traits", +] + +[[package]] +name = "num-complex" +version = "0.4.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "73f88a1307638156682bada9d7604135552957b7818057dcef22705b4d509495" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-format" +version = "0.4.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a652d9771a63711fd3c3deb670acfbe5c30a4072e664d7a3bf5a9e1056ac72c3" +dependencies = [ + "arrayvec", + "itoa", +] + +[[package]] +name = "num-integer" +version = "0.1.46" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7969661fd2958a5cb096e56c8e1ad0444ac2bbcd0061bd28660485a44879858f" +dependencies = [ + "num-traits", +] + +[[package]] +name = "num-iter" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" +dependencies = [ + "num-bigint", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-traits" +version = "0.2.19" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "071dfc062690e90b734c0b2273ce72ad0ffa95f0c74596bc250dcfd960262841" +dependencies = [ + "autocfg", + "libm", +] + +[[package]] +name = "object" +version = "0.36.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" +dependencies = [ + "memchr", +] + +[[package]] +name = "object_store" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6eb4c22c6154a1e759d7099f9ffad7cc5ef8245f9efbab4a41b92623079c82f3" +dependencies = [ + "async-trait", + "bytes", + "chrono", + "futures", + "humantime", + "itertools 0.13.0", + "parking_lot", + "percent-encoding", + "snafu", + "tokio", + "tracing", + "url", + "walkdir", +] + +[[package]] +name = "once_cell" +version = "1.20.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1261fe7e33c73b354eab43b1273a57c8f967d0391e80353e51f764ac02cf6775" + +[[package]] +name = "oorandom" +version = "11.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b410bbe7e14ab526a0e86877eb47c6996a2bd7746f027ba551028c925390e4e9" + +[[package]] +name = "ordered-float" +version = "2.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" +dependencies = [ + "num-traits", +] + +[[package]] +name = "parking_lot" +version = "0.12.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" +dependencies = [ + "lock_api", + "parking_lot_core", +] + +[[package]] +name = "parking_lot_core" +version = "0.9.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" +dependencies = [ + "cfg-if", + "libc", + "redox_syscall", + "smallvec", + "windows-targets 0.52.6", +] + +[[package]] +name = "parquet" +version = "53.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b449890367085eb65d7d3321540abc3d7babbd179ce31df0016e90719114191" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-schema", + "arrow-select", + "base64", + "brotli 7.0.0", + "bytes", + "chrono", + "flate2", + "futures", + "half", + "hashbrown 0.15.2", + "lz4_flex", + "num", + "num-bigint", + "object_store", + "paste", + "seq-macro", + "snap", + "thrift", + "tokio", + "twox-hash 1.6.3", + "zstd 0.13.2", + "zstd-sys", +] + +[[package]] +name = "parse-zoneinfo" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1f2a05b18d44e2957b88f96ba460715e295bc1d7510468a2f3d3b44535d26c24" +dependencies = [ + "regex", +] + +[[package]] +name = "paste" +version = "1.0.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57c0d7b74b563b49d38dae00a0c37d4d6de9b432382b2892f0574ddcae73fd0a" + +[[package]] +name = "percent-encoding" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" + +[[package]] +name = "petgraph" +version = "0.6.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" +dependencies = [ + "fixedbitset", + "indexmap", +] + +[[package]] +name = "phf" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ade2d8b8f33c7333b51bcf0428d37e217e9f32192ae4772156f65063b8ce03dc" +dependencies = [ + "phf_shared", +] + +[[package]] +name = "phf_codegen" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e8d39688d359e6b34654d328e262234662d16cc0f60ec8dcbe5e718709342a5a" +dependencies = [ + "phf_generator", + "phf_shared", +] + +[[package]] +name = "phf_generator" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "48e4cc64c2ad9ebe670cb8fd69dd50ae301650392e81c05f9bfcb2d5bdbc24b0" +dependencies = [ + "phf_shared", + "rand", +] + +[[package]] +name = "phf_shared" +version = "0.11.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "90fcb95eef784c2ac79119d1dd819e162b5da872ce6f3c3abe1e8ca1c082f72b" +dependencies = [ + "siphasher", +] + +[[package]] +name = "pin-project-lite" +version = "0.2.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "915a1e146535de9163f3987b8944ed8cf49a18bb0056bcebcdcece385cece4ff" + +[[package]] +name = "pin-utils" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b870d8c151b6f2fb93e84a13146138f05d02ed11c7e7c54f8826aaaf7c9f184" + +[[package]] +name = "pkg-config" +version = "0.3.31" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "953ec861398dccce10c670dfeaf3ec4911ca479e9c02154b3a215178c5f566f2" + +[[package]] +name = "plotters" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5aeb6f403d7a4911efb1e33402027fc44f29b5bf6def3effcc22d7bb75f2b747" +dependencies = [ + "num-traits", + "plotters-backend", + "plotters-svg", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "plotters-backend" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df42e13c12958a16b3f7f4386b9ab1f3e7933914ecea48da7139435263a4172a" + +[[package]] +name = "plotters-svg" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51bae2ac328883f7acdfea3d66a7c35751187f870bc81f94563733a154d7a670" +dependencies = [ + "plotters-backend", +] + +[[package]] +name = "pprof" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ef5c97c51bd34c7e742402e216abdeb44d415fbe6ae41d56b114723e953711cb" +dependencies = [ + "backtrace", + "cfg-if", + "findshlibs", + "inferno", + "libc", + "log", + "nix", + "once_cell", + "parking_lot", + "smallvec", + "symbolic-demangle", + "tempfile", + "thiserror", +] + +[[package]] +name = "ppv-lite86" +version = "0.2.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +dependencies = [ + "zerocopy", +] + +[[package]] +name = "proc-macro2" +version = "1.0.92" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37d3544b3f2748c54e147655edb5025752e2303145b5aefb3c3ea2c78b973bb0" +dependencies = [ + "unicode-ident", +] + +[[package]] +name = "prost" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "444879275cb4fd84958b1a1d5420d15e6fcf7c235fe47f053c9c2a80aceb6001" +dependencies = [ + "bytes", + "prost-derive 0.9.0", +] + +[[package]] +name = "prost" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "deb1435c188b76130da55f17a466d252ff7b1418b2ad3e037d127b94e3411f29" +dependencies = [ + "bytes", + "prost-derive 0.12.6", +] + +[[package]] +name = "prost-build" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "62941722fb675d463659e49c4f3fe1fe792ff24fe5bbaa9c08cd3b98a1c354f5" +dependencies = [ + "bytes", + "heck 0.3.3", + "itertools 0.10.5", + "lazy_static", + "log", + "multimap", + "petgraph", + "prost 0.9.0", + "prost-types", + "regex", + "tempfile", + "which", +] + +[[package]] +name = "prost-derive" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9cc1a3263e07e0bf68e96268f37665207b49560d98739662cdfaae215c720fe" +dependencies = [ + "anyhow", + "itertools 0.10.5", + "proc-macro2", + "quote", + "syn 1.0.109", +] + +[[package]] +name = "prost-derive" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" +dependencies = [ + "anyhow", + "itertools 0.12.1", + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "prost-types" +version = "0.9.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "534b7a0e836e3c482d2693070f982e39e7611da9695d4d1f5a4b186b51faef0a" +dependencies = [ + "bytes", + "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" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f50b1c63b38611e7d4d7f68b82d3ad0cc71a2ad2e7f61fc10f1328d917c93cd" +dependencies = [ + "memchr", +] + +[[package]] +name = "quote" +version = "1.0.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" +dependencies = [ + "proc-macro2", +] + +[[package]] +name = "rand" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "34af8d1a0e25924bc5b7c43c079c942339d8f0a8b57c39049bef581b46327404" +dependencies = [ + "libc", + "rand_chacha", + "rand_core", +] + +[[package]] +name = "rand_chacha" +version = "0.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e6c10a63a0fa32252be49d21e7709d4d4baf8d231c2dbce1eaa8141b9b127d88" +dependencies = [ + "ppv-lite86", + "rand_core", +] + +[[package]] +name = "rand_core" +version = "0.6.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" +dependencies = [ + "getrandom", +] + +[[package]] +name = "rayon" +version = "1.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b418a60154510ca1a002a752ca9714984e21e4241e804d32555251faf8b78ffa" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1465873a3dfdaa8ae7cb14b4383657caab0b3e8a0aa9ae8e04b044854c8dfce2" +dependencies = [ + "crossbeam-deque", + "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.90", +] + +[[package]] +name = "redox_syscall" +version = "0.5.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" +dependencies = [ + "bitflags 2.6.0", +] + +[[package]] +name = "regex" +version = "1.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" +dependencies = [ + "aho-corasick", + "memchr", + "regex-automata", + "regex-syntax", +] + +[[package]] +name = "regex-automata" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" +dependencies = [ + "aho-corasick", + "memchr", + "regex-syntax", +] + +[[package]] +name = "regex-syntax" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2b15c43186be67a4fd63bee50d0303afffcef381492ebe2c5d87f324e1b8815c" + +[[package]] +name = "rgb" +version = "0.8.50" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57397d16646700483b67d2dd6511d79318f9d057fdbd21a4066aeac8b41d310a" +dependencies = [ + "bytemuck", +] + +[[package]] +name = "rustc-demangle" +version = "0.1.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" + +[[package]] +name = "rustc_version" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfcb3a22ef46e85b45de6ee7e79d063319ebb6594faafcf1c225ea92ab6e9b92" +dependencies = [ + "semver", +] + +[[package]] +name = "rustix" +version = "0.38.42" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f93dc38ecbab2eb790ff964bb77fa94faf256fd3e73285fd7ba0903b76bedb85" +dependencies = [ + "bitflags 2.6.0", + "errno", + "libc", + "linux-raw-sys", + "windows-sys 0.59.0", +] + +[[package]] +name = "rustversion" +version = "1.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" + +[[package]] +name = "ryu" +version = "1.0.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3cb5ba0dc43242ce17de99c180e96db90b235b8a9fdc9543c96d2209116bd9f" + +[[package]] +name = "same-file" +version = "1.0.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "93fc1dc3aaa9bfed95e02e6eadabb4baf7e3078b0bd1b4d7b6b0b68378900502" +dependencies = [ + "winapi-util", +] + +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + +[[package]] +name = "semver" +version = "1.0.24" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3cb6eb87a131f756572d7fb904f6e7b68633f09cca868c5df1c4b8d1a694bbba" + +[[package]] +name = "seq-macro" +version = "0.3.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" + +[[package]] +name = "serde" +version = "1.0.216" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b9781016e935a97e8beecf0c933758c97a5520d32930e460142b4cd80c6338e" +dependencies = [ + "serde_derive", +] + +[[package]] +name = "serde-value" +version = "0.7.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f3a1a3341211875ef120e117ea7fd5228530ae7e7036a779fdc9117be6b3282c" +dependencies = [ + "ordered-float", + "serde", +] + +[[package]] +name = "serde_derive" +version = "1.0.216" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "46f859dbbf73865c6627ed570e78961cd3ac92407a2d117204c49232485da55e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "serde_json" +version = "1.0.133" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7fceb2473b9166b2294ef05efcb65a3db80803f0b03ef86a5fc88a2b85ee377" +dependencies = [ + "itoa", + "memchr", + "ryu", + "serde", +] + +[[package]] +name = "serde_yaml" +version = "0.9.34+deprecated" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" +dependencies = [ + "indexmap", + "itoa", + "ryu", + "serde", + "unsafe-libyaml", +] + +[[package]] +name = "sha2" +version = "0.10.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +dependencies = [ + "cfg-if", + "cpufeatures", + "digest", +] + +[[package]] +name = "shlex" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" + +[[package]] +name = "simd-adler32" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" + +[[package]] +name = "siphasher" +version = "0.3.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38b58827f4464d87d377d175e90bf58eb00fd8716ff0a62f80356b5e61555d0d" + +[[package]] +name = "slab" +version = "0.4.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] + +[[package]] +name = "smallvec" +version = "1.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c5e1a9a646d36c3599cd173a41282daf47c44583ad367b8e6837255952e5c67" + +[[package]] +name = "snafu" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "223891c85e2a29c3fe8fb900c1fae5e69c2e42415e3177752e8718475efa5019" +dependencies = [ + "snafu-derive", +] + +[[package]] +name = "snafu-derive" +version = "0.8.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03c3c6b7927ffe7ecaa769ee0e3994da3b8cafc8f444578982c83ecb161af917" +dependencies = [ + "heck 0.5.0", + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "snap" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" + +[[package]] +name = "sqlparser" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9a875d8cd437cc8a97e9aeaeea352ec9a19aea99c23e9effb17757291de80b08" +dependencies = [ + "log", + "sqlparser_derive", +] + +[[package]] +name = "sqlparser_derive" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "stable_deref_trait" +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" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" + +[[package]] +name = "str_stack" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9091b6114800a5f2141aee1d1b9d6ca3592ac062dc5decb3764ec5895a47b4eb" + +[[package]] +name = "strum" +version = "0.26.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" + +[[package]] +name = "strum_macros" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" +dependencies = [ + "heck 0.5.0", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.90", +] + +[[package]] +name = "subtle" +version = "2.6.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" + +[[package]] +name = "symbolic-common" +version = "12.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cd33e73f154e36ec223c18013f7064a2c120f1162fc086ac9933542def186b00" +dependencies = [ + "debugid", + "memmap2", + "stable_deref_trait", + "uuid", +] + +[[package]] +name = "symbolic-demangle" +version = "12.12.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89e51191290147f071777e37fe111800bb82a9059f9c95b19d2dd41bfeddf477" +dependencies = [ + "cpp_demangle", + "rustc-demangle", + "symbolic-common", +] + +[[package]] +name = "syn" +version = "1.0.109" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "72b64191b275b66ffe2469e8af2c1cfe3bafa67b529ead792a6d0160888b4237" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "syn" +version = "2.0.90" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "919d3b74a5dd0ccd15aeb8f93e7006bd9e14c295087c9896a110f490752bcf31" +dependencies = [ + "proc-macro2", + "quote", + "unicode-ident", +] + +[[package]] +name = "synstructure" +version = "0.13.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "tempfile" +version = "3.14.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "28cce251fcbc87fac86a866eeb0d6c2d536fc16d06f184bb61aeae11aa4cee0c" +dependencies = [ + "cfg-if", + "fastrand", + "once_cell", + "rustix", + "windows-sys 0.59.0", +] + +[[package]] +name = "thiserror" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" +dependencies = [ + "thiserror-impl", +] + +[[package]] +name = "thiserror-impl" +version = "1.0.69" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "thread-id" +version = "4.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cfe8f25bbdd100db7e1d34acf7fd2dc59c4bf8f7483f505eaa7d4f12f76cc0ea" +dependencies = [ + "libc", + "winapi", +] + +[[package]] +name = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding", + "ordered-float", +] + +[[package]] +name = "tiny-keccak" +version = "2.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c9d3793400a45f954c52e73d068316d76b6f4e36977e3fcebb13a2721e80237" +dependencies = [ + "crunchy", +] + +[[package]] +name = "tinystr" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9117f5d4db391c1cf6927e7bea3db74b9a1c1add8f7eda9ffd5364f40f57b82f" +dependencies = [ + "displaydoc", + "zerovec", +] + +[[package]] +name = "tinytemplate" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +dependencies = [ + "serde", + "serde_json", +] + +[[package]] +name = "tokio" +version = "1.42.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5cec9b21b0450273377fc97bd4c33a8acffc8c996c987a7c5b319a0083707551" +dependencies = [ + "backtrace", + "bytes", + "pin-project-lite", + "tokio-macros", +] + +[[package]] +name = "tokio-macros" +version = "2.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "tracing" +version = "0.1.41" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "784e0ac535deb450455cbfa28a6f0df145ea1bb7ae51b821cf5e7927fdcfbdd0" +dependencies = [ + "pin-project-lite", + "tracing-attributes", + "tracing-core", +] + +[[package]] +name = "tracing-attributes" +version = "0.1.28" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "tracing-core" +version = "0.1.33" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e672c95779cf947c5311f83787af4fa8fffd12fb27e4993211a84bdfd9610f9c" +dependencies = [ + "once_cell", +] + +[[package]] +name = "twox-hash" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "static_assertions", +] + +[[package]] +name = "twox-hash" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" +dependencies = [ + "rand", +] + +[[package]] +name = "typemap-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a68c24b707f02dd18f1e4ccceb9d49f2058c2fb86384ef9972592904d7a28867" +dependencies = [ + "unsafe-any-ors", +] + +[[package]] +name = "typenum" +version = "1.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" + +[[package]] +name = "unicode-ident" +version = "1.0.14" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "adb9e6ca4f869e1180728b7950e35922a7fc6397f7b641499e8f3ef06e50dc83" + +[[package]] +name = "unicode-segmentation" +version = "1.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" + +[[package]] +name = "unicode-width" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" + +[[package]] +name = "unsafe-any-ors" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0a303d30665362d9680d7d91d78b23f5f899504d4f08b3c4cf08d055d87c0ad" +dependencies = [ + "destructure_traitobject", +] + +[[package]] +name = "unsafe-libyaml" +version = "0.2.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "673aac59facbab8a9007c7f6108d11f63b603f7cabff99fabf650fea5c32b861" + +[[package]] +name = "url" +version = "2.5.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" +dependencies = [ + "form_urlencoded", + "idna", + "percent-encoding", +] + +[[package]] +name = "utf16_iter" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8232dd3cdaed5356e0f716d285e4b40b932ac434100fe9b7e0e8e935b9e6246" + +[[package]] +name = "utf8_iter" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b6c140620e7ffbb22c2dee59cafe6084a59b5ffc27a8859a5f0d494b5d52b6be" + +[[package]] +name = "uuid" +version = "1.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8c5f0a0af699448548ad1a2fbf920fb4bee257eae39953ba95cb84891a0446a" +dependencies = [ + "getrandom", +] + +[[package]] +name = "version_check" +version = "0.9.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b928f33d975fc6ad9f86c8f283853ad26bdd5b10b7f1542aa2fa15e2289105a" + +[[package]] +name = "walkdir" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "29790946404f91d9c5d06f9874efddea1dc06c5efe94541a7d6863108e3a5e4b" +dependencies = [ + "same-file", + "winapi-util", +] + +[[package]] +name = "wasi" +version = "0.11.0+wasi-snapshot-preview1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" + +[[package]] +name = "wasm-bindgen" +version = "0.2.99" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a474f6281d1d70c17ae7aa6a613c87fce69a127e2624002df63dcb39d6cf6396" +dependencies = [ + "cfg-if", + "once_cell", + "wasm-bindgen-macro", +] + +[[package]] +name = "wasm-bindgen-backend" +version = "0.2.99" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f89bb38646b4f81674e8f5c3fb81b562be1fd936d84320f3264486418519c79" +dependencies = [ + "bumpalo", + "log", + "proc-macro2", + "quote", + "syn 2.0.90", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-macro" +version = "0.2.99" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2cc6181fd9a7492eef6fef1f33961e3695e4579b9872a6f7c83aee556666d4fe" +dependencies = [ + "quote", + "wasm-bindgen-macro-support", +] + +[[package]] +name = "wasm-bindgen-macro-support" +version = "0.2.99" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", + "wasm-bindgen-backend", + "wasm-bindgen-shared", +] + +[[package]] +name = "wasm-bindgen-shared" +version = "0.2.99" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "943aab3fdaaa029a6e0271b35ea10b72b943135afe9bffca82384098ad0e06a6" + +[[package]] +name = "web-sys" +version = "0.3.76" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "04dd7223427d52553d3702c004d3b2fe07c148165faa56313cb00211e31c12bc" +dependencies = [ + "js-sys", + "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" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" +dependencies = [ + "either", + "home", + "once_cell", + "rustix", +] + +[[package]] +name = "winapi" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5c839a674fcd7a98952e593242ea400abe93992746761e38641405d28b00f419" +dependencies = [ + "winapi-i686-pc-windows-gnu", + "winapi-x86_64-pc-windows-gnu", +] + +[[package]] +name = "winapi-i686-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" + +[[package]] +name = "winapi-util" +version = "0.1.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" +dependencies = [ + "windows-sys 0.59.0", +] + +[[package]] +name = "winapi-x86_64-pc-windows-gnu" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" + +[[package]] +name = "windows-core" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.45.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "75283be5efb2831d37ea142365f009c02ec203cd29a3ebecbc093d52315b66d0" +dependencies = [ + "windows-targets 0.42.2", +] + +[[package]] +name = "windows-sys" +version = "0.52.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "282be5f36a8ce781fad8c8ae18fa3f9beff57ec1b52cb3de0789201425d9a33d" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-sys" +version = "0.59.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e38bc4d79ed67fd075bcc251a1c39b32a1776bbe92e5bef1f0bf1f8c531853b" +dependencies = [ + "windows-targets 0.52.6", +] + +[[package]] +name = "windows-targets" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e5180c00cd44c9b1c88adb3693291f1cd93605ded80c250a75d472756b4d071" +dependencies = [ + "windows_aarch64_gnullvm 0.42.2", + "windows_aarch64_msvc 0.42.2", + "windows_i686_gnu 0.42.2", + "windows_i686_msvc 0.42.2", + "windows_x86_64_gnu 0.42.2", + "windows_x86_64_gnullvm 0.42.2", + "windows_x86_64_msvc 0.42.2", +] + +[[package]] +name = "windows-targets" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9b724f72796e036ab90c1021d4780d4d3d648aca59e491e6b98e725b84e99973" +dependencies = [ + "windows_aarch64_gnullvm 0.52.6", + "windows_aarch64_msvc 0.52.6", + "windows_i686_gnu 0.52.6", + "windows_i686_gnullvm", + "windows_i686_msvc 0.52.6", + "windows_x86_64_gnu 0.52.6", + "windows_x86_64_gnullvm 0.52.6", + "windows_x86_64_msvc 0.52.6", +] + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "597a5118570b68bc08d8d59125332c54f1ba9d9adeedeef5b99b02ba2b0698f8" + +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e08e8864a60f06ef0d0ff4ba04124db8b0fb3be5776a5cd47641e942e58c4d43" + +[[package]] +name = "windows_aarch64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" + +[[package]] +name = "windows_i686_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c61d927d8da41da96a81f029489353e68739737d3beca43145c8afec9a31a84f" + +[[package]] +name = "windows_i686_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" + +[[package]] +name = "windows_i686_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" + +[[package]] +name = "windows_i686_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "44d840b6ec649f480a41c8d80f9c65108b92d89345dd94027bfe06ac444d1060" + +[[package]] +name = "windows_i686_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8de912b8b8feb55c064867cf047dda097f92d51efad5b491dfb98f6bbb70cb36" + +[[package]] +name = "windows_x86_64_gnu" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "26d41b46a36d453748aedef1486d5c7a85db22e56aff34643984ea85514e94a3" + +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.42.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9aec5da331524158c6d1a4ac0ab1541149c0b9505fde06423b02f5ef0106b9f0" + +[[package]] +name = "windows_x86_64_msvc" +version = "0.52.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" + +[[package]] +name = "write16" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" + +[[package]] +name = "writeable" +version = "0.5.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" + +[[package]] +name = "yoke" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "120e6aef9aa629e3d4f52dc8cc43a015c7724194c97dfaf45180d2daf2b77f40" +dependencies = [ + "serde", + "stable_deref_trait", + "yoke-derive", + "zerofrom", +] + +[[package]] +name = "yoke-derive" +version = "0.7.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", + "synstructure", +] + +[[package]] +name = "zerocopy" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" +dependencies = [ + "byteorder", + "zerocopy-derive", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "zerofrom" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cff3ee08c995dee1859d998dea82f7374f2826091dd9cd47def953cae446cd2e" +dependencies = [ + "zerofrom-derive", +] + +[[package]] +name = "zerofrom-derive" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", + "synstructure", +] + +[[package]] +name = "zerovec" +version = "0.10.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "aa2b893d79df23bfb12d5461018d408ea19dfafe76c2c7ef6d4eba614f8ff079" +dependencies = [ + "yoke", + "zerofrom", + "zerovec-derive", +] + +[[package]] +name = "zerovec-derive" +version = "0.10.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.90", +] + +[[package]] +name = "zstd" +version = "0.11.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "20cc960326ece64f010d2d2107537f26dc589a6573a316bd5b1dba685fa5fde4" +dependencies = [ + "zstd-safe 5.0.2+zstd.1.5.2", +] + +[[package]] +name = "zstd" +version = "0.13.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" +dependencies = [ + "zstd-safe 7.2.1", +] + +[[package]] +name = "zstd-safe" +version = "5.0.2+zstd.1.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d2a5585e04f9eea4b2a3d1eca508c4dee9592a89ef6f450c11719da0726f4db" +dependencies = [ + "libc", + "zstd-sys", +] + +[[package]] +name = "zstd-safe" +version = "7.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" +dependencies = [ + "zstd-sys", +] + +[[package]] +name = "zstd-sys" +version = "2.0.13+zstd.1.5.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" +dependencies = [ + "cc", + "pkg-config", +] From 2cbf3ff14e0dd83b02d58c12f235c63f34b2b0dc Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 22 Dec 2024 07:27:16 -0700 Subject: [PATCH 33/40] bump DF version --- native/Cargo.lock | 54 +++++++++---------- native/Cargo.toml | 18 +++---- native/core/src/execution/operators/copy.rs | 4 +- native/core/src/execution/operators/expand.rs | 6 ++- native/core/src/execution/operators/filter.rs | 3 +- native/core/src/execution/operators/scan.rs | 4 +- .../src/execution/shuffle/shuffle_writer.rs | 6 ++- native/spark-expr/src/cast.rs | 1 - 8 files changed, 52 insertions(+), 44 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index aad1733ad7..75009ca683 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -840,7 +840,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "arrow-array", @@ -884,7 +884,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow-schema", "async-trait", @@ -983,7 +983,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "ahash", "arrow", @@ -994,6 +994,7 @@ dependencies = [ "hashbrown 0.14.5", "indexmap", "libc", + "log", "object_store", "parquet", "paste", @@ -1006,7 +1007,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "log", "tokio", @@ -1015,12 +1016,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "dashmap", @@ -1038,7 +1039,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "chrono", @@ -1058,7 +1059,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "datafusion-common", @@ -1068,7 +1069,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "arrow-buffer", @@ -1097,7 +1098,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "ahash", "arrow", @@ -1118,7 +1119,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "ahash", "arrow", @@ -1130,7 +1131,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "arrow-array", @@ -1151,7 +1152,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "async-trait", @@ -1166,7 +1167,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1182,7 +1183,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1191,7 +1192,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "quote", "syn 2.0.90", @@ -1200,7 +1201,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "chrono", @@ -1210,7 +1211,6 @@ dependencies = [ "indexmap", "itertools 0.13.0", "log", - "recursive", "regex", "regex-syntax", ] @@ -1218,7 +1218,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "ahash", "arrow", @@ -1242,7 +1242,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "ahash", "arrow", @@ -1255,7 +1255,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "datafusion-common", @@ -1271,7 +1271,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "ahash", "arrow", @@ -1302,7 +1302,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=9f530dd#9f530dd1987d863dd4de99198df5b2c3e242bf6e" +source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" dependencies = [ "arrow", "arrow-array", @@ -2956,9 +2956,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "sqlparser" -version = "0.52.0" +version = "0.53.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a875d8cd437cc8a97e9aeaeea352ec9a19aea99c23e9effb17757291de80b08" +checksum = "05a528114c392209b3264855ad491fcce534b94a38771b0a0b97a79379275ce8" dependencies = [ "log", "sqlparser_derive", @@ -2966,9 +2966,9 @@ dependencies = [ [[package]] name = "sqlparser_derive" -version = "0.2.2" +version = "0.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" +checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", diff --git a/native/Cargo.toml b/native/Cargo.toml index f2310b3f94..516a8953ba 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,15 +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 = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd" } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } -datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "9f530dd", default-features = false } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "242f45f" } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", 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/operators/copy.rs b/native/core/src/execution/operators/copy.rs index 8eeda8a5ad..cec00eb28c 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -30,6 +30,7 @@ use arrow_array::{ use arrow_data::transform::MutableArrayData; use arrow_schema::{ArrowError, DataType, Field, FieldRef, Schema, SchemaRef}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use datafusion::{execution::TaskContext, physical_expr::*, physical_plan::*}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result as DataFusionResult}; @@ -78,7 +79,8 @@ impl CopyExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&schema)), Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Self { diff --git a/native/core/src/execution/operators/expand.rs b/native/core/src/execution/operators/expand.rs index fb43a6e49f..f75822d408 100644 --- a/native/core/src/execution/operators/expand.rs +++ b/native/core/src/execution/operators/expand.rs @@ -17,10 +17,11 @@ use arrow_array::{RecordBatch, RecordBatchOptions}; use arrow_schema::SchemaRef; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::{ execution::TaskContext, physical_plan::{ - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, }, }; @@ -54,7 +55,8 @@ impl ExpandExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&schema)), Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Self { diff --git a/native/core/src/execution/operators/filter.rs b/native/core/src/execution/operators/filter.rs index d9a54712dc..eab30a3560 100644 --- a/native/core/src/execution/operators/filter.rs +++ b/native/core/src/execution/operators/filter.rs @@ -210,7 +210,8 @@ impl FilterExec { Ok(PlanProperties::new( eq_properties, input.output_partitioning().clone(), // Output Partitioning - input.execution_mode(), // Execution Mode + input.pipeline_behavior(), + input.boundedness(), )) } } diff --git a/native/core/src/execution/operators/scan.rs b/native/core/src/execution/operators/scan.rs index a297f87c1f..888cd2fdb5 100644 --- a/native/core/src/execution/operators/scan.rs +++ b/native/core/src/execution/operators/scan.rs @@ -28,6 +28,7 @@ use arrow_data::ffi::FFI_ArrowArray; use arrow_data::ArrayData; use arrow_schema::ffi::FFI_ArrowSchema; use arrow_schema::{DataType, Field, Schema, SchemaRef}; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::metrics::{ BaselineMetrics, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }; @@ -122,7 +123,8 @@ impl ScanExec { // The partitioning is not important because we are not using DataFusion's // query planner or optimizer Partitioning::UnknownPartitioning(1), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Ok(Self { diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index fcc8c51f60..256fd80941 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -25,6 +25,7 @@ use arrow::{datatypes::*, ipc::writer::StreamWriter}; use async_trait::async_trait; use bytes::Buf; use crc32fast::Hasher; +use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::{ arrow::{ array::*, @@ -44,7 +45,7 @@ use datafusion::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, + DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }, }; @@ -187,7 +188,8 @@ impl ShuffleWriterExec { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), partitioning.clone(), - ExecutionMode::Bounded, + EmissionType::Final, + Boundedness::Bounded, ); Ok(ShuffleWriterExec { diff --git a/native/spark-expr/src/cast.rs b/native/spark-expr/src/cast.rs index f0e847f75a..6e0e0915cd 100644 --- a/native/spark-expr/src/cast.rs +++ b/native/spark-expr/src/cast.rs @@ -58,7 +58,6 @@ use std::{ sync::Arc, }; - static TIMESTAMP_FORMAT: Option<&str> = Some("%Y-%m-%d %H:%M:%S%.f"); const MICROS_PER_SECOND: i64 = 1000000; From 199f0d02e6393b2c4b3541f0df2bb11e0a12c3c8 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 22 Dec 2024 07:28:07 -0700 Subject: [PATCH 34/40] no default features --- native/Cargo.toml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/Cargo.toml b/native/Cargo.toml index 516a8953ba..da639f79af 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -40,8 +40,8 @@ arrow-data = { version = "53.3.0" } arrow-schema = { version = "53.3.0" } parquet = { version = "53.3.0", default-features = false, features = ["experimental"] } datafusion = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "242f45f" } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", features = ["crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false, features = ["crypto_expressions"] } datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } From 2ed53c8228edabcd62debc4240113c3c7997aefd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 22 Dec 2024 07:28:40 -0700 Subject: [PATCH 35/40] Revert "remove miri checks" This reverts commit 4638fe3aa5501966cd5d8b53acf26c698b10b3c9. --- .github/workflows/miri.yml | 49 ++++++++++++++++++++++++++++ native/core/src/execution/planner.rs | 2 ++ 2 files changed, 51 insertions(+) create mode 100644 .github/workflows/miri.yml diff --git a/.github/workflows/miri.yml b/.github/workflows/miri.yml new file mode 100644 index 0000000000..a07ecc35ef --- /dev/null +++ b/.github/workflows/miri.yml @@ -0,0 +1,49 @@ +# 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/planner.rs b/native/core/src/execution/planner.rs index fa798803de..4ccbea87a9 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2500,6 +2500,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); @@ -2513,6 +2514,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 a75226c0073228ab766a2b8c50873779b3bba715 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 24 Dec 2024 16:14:20 -0500 Subject: [PATCH 36/40] Update pin to DataFusion e99e02b9b9093ceb0c13a2dd32a2a89beba47930 --- native/Cargo.lock | 92 ++++++++++++----------------------------------- native/Cargo.toml | 18 +++++----- 2 files changed, 32 insertions(+), 78 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 75009ca683..042737fc18 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1,6 +1,6 @@ # This file is automatically @generated by Cargo. # It is not intended for manual editing. -version = 3 +version = 4 [[package]] name = "addr2line" @@ -840,7 +840,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "arrow-array", @@ -884,7 +884,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow-schema", "async-trait", @@ -983,7 +983,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "ahash", "arrow", @@ -998,7 +998,6 @@ dependencies = [ "object_store", "parquet", "paste", - "recursive", "sqlparser", "tokio", "web-time", @@ -1007,7 +1006,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "log", "tokio", @@ -1016,12 +1015,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "dashmap", @@ -1039,7 +1038,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "chrono", @@ -1051,7 +1050,6 @@ dependencies = [ "datafusion-physical-expr-common", "indexmap", "paste", - "recursive", "serde_json", "sqlparser", ] @@ -1059,7 +1057,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "datafusion-common", @@ -1069,7 +1067,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "arrow-buffer", @@ -1098,7 +1096,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "ahash", "arrow", @@ -1119,7 +1117,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "ahash", "arrow", @@ -1131,7 +1129,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "arrow-array", @@ -1152,7 +1150,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "async-trait", @@ -1167,7 +1165,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1183,7 +1181,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1192,7 +1190,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "quote", "syn 2.0.90", @@ -1201,7 +1199,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "chrono", @@ -1218,7 +1216,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "ahash", "arrow", @@ -1242,7 +1240,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "ahash", "arrow", @@ -1255,7 +1253,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "datafusion-common", @@ -1265,13 +1263,12 @@ dependencies = [ "datafusion-physical-plan", "itertools 0.13.0", "log", - "recursive", ] [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "ahash", "arrow", @@ -1302,7 +1299,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=242f45f#242f45f4f19c9f25f8f084e8b2c534d9f14fa2d7" +source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" dependencies = [ "arrow", "arrow-array", @@ -1312,7 +1309,6 @@ dependencies = [ "datafusion-expr", "indexmap", "log", - "recursive", "regex", "sqlparser", ] @@ -2617,15 +2613,6 @@ 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" @@ -2694,26 +2681,6 @@ 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.90", -] - [[package]] name = "redox_syscall" version = "0.5.8" @@ -2981,19 +2948,6 @@ 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" diff --git a/native/Cargo.toml b/native/Cargo.toml index da639f79af..db173ab555 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,15 +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 = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false, features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } -datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "242f45f", default-features = false } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false, features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", 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 9d45dfa6c17f204e1c2fa1dd7901030d3048e0f5 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 24 Dec 2024 17:23:18 -0500 Subject: [PATCH 37/40] update pin --- native/Cargo.lock | 44 ++++++++++++++++++++++---------------------- native/Cargo.toml | 18 +++++++++--------- 2 files changed, 31 insertions(+), 31 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 042737fc18..0814bec811 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -840,7 +840,7 @@ dependencies = [ [[package]] name = "datafusion" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "arrow-array", @@ -884,7 +884,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow-schema", "async-trait", @@ -983,7 +983,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "ahash", "arrow", @@ -1006,7 +1006,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "log", "tokio", @@ -1015,12 +1015,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" [[package]] name = "datafusion-execution" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "dashmap", @@ -1038,7 +1038,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "chrono", @@ -1057,7 +1057,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "datafusion-common", @@ -1067,7 +1067,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "arrow-buffer", @@ -1096,7 +1096,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "ahash", "arrow", @@ -1117,7 +1117,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "ahash", "arrow", @@ -1129,7 +1129,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "arrow-array", @@ -1150,7 +1150,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "async-trait", @@ -1165,7 +1165,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1181,7 +1181,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1190,7 +1190,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "quote", "syn 2.0.90", @@ -1199,7 +1199,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "chrono", @@ -1216,7 +1216,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "ahash", "arrow", @@ -1240,7 +1240,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "ahash", "arrow", @@ -1253,7 +1253,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "datafusion-common", @@ -1268,7 +1268,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "ahash", "arrow", @@ -1299,7 +1299,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=e99e02b9b9093ceb0c13a2dd32a2a89beba47930#e99e02b9b9093ceb0c13a2dd32a2a89beba47930" +source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" dependencies = [ "arrow", "arrow-array", diff --git a/native/Cargo.toml b/native/Cargo.toml index db173ab555..e1b2afccaa 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,15 +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 = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false, features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } -datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "e99e02b9b9093ceb0c13a2dd32a2a89beba47930", default-features = false } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false, features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", 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 201dd0e9c383547b2dc5c09a6a1be2cc44025b75 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Dec 2024 09:55:28 -0700 Subject: [PATCH 38/40] Update Cargo.toml Bump to 44.0.0-rc2 --- native/Cargo.toml | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/native/Cargo.toml b/native/Cargo.toml index e1b2afccaa..cf4921ebed 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -39,15 +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 = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } -datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } -datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false, features = ["crypto_expressions"] } -datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } -datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } -datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } -datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } -datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } -datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "3864b113c3e3fe85e18462d6374f8244c4f77b27", default-features = false } +datafusion = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false, features = ["unicode_expressions", "crypto_expressions"] } +datafusion-common = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-functions = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false, features = ["crypto_expressions"] } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-expr = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-expr-common = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-execution = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-physical-plan = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", default-features = false } +datafusion-physical-expr = { git = "https://github.com/apache/datafusion.git", rev = "44.0.0-rc2", 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 57acf7547d214c54695764863d4b2abbad9d684d Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Dec 2024 11:56:30 -0700 Subject: [PATCH 39/40] update cargo lock --- native/Cargo.lock | 176 +++++++++++++++++++++++----------------------- 1 file changed, 88 insertions(+), 88 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 219526dae3..bbc0ff97a9 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -90,9 +90,9 @@ checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anyhow" -version = "1.0.94" +version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1fd03a028ef38ba2276dce7e33fcd6369c158a1bca17946c4b1b701891c1ff7" +checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" [[package]] name = "arc-swap" @@ -346,7 +346,7 @@ checksum = "721cae7de5c34fbb2acd27e21e6d2cf7b886dce0c27388d46c4e6c47ea4318dd" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -470,9 +470,9 @@ checksum = "79296716171880943b8470b5f8d03aa55eb2e645a4874bdbb28adb49162e012c" [[package]] name = "bytemuck" -version = "1.20.0" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b37c88a63ffd85d15b406896cc343916d7cf57838a847b3a6f2ca5d39a5695a" +checksum = "ef657dfab802224e671f5818e9a4935f9b1957ed18e58292690cc39e7a4092a3" [[package]] name = "byteorder" @@ -494,9 +494,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.5" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c31a0499c1dc64f458ad13872de75c0eb7e3fdb0e67964610c914b034fc5956e" +checksum = "8d6dbb628b8f8555f86d0323c2eb39e3ec81901f4b83e091db8a6a76d316a333" dependencies = [ "jobserver", "libc", @@ -818,8 +818,8 @@ dependencies = [ [[package]] name = "datafusion" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-array", @@ -862,8 +862,8 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow-schema", "async-trait", @@ -957,8 +957,8 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -980,8 +980,8 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "log", "tokio", @@ -989,13 +989,13 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" [[package]] name = "datafusion-execution" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "dashmap", @@ -1012,8 +1012,8 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "chrono", @@ -1031,8 +1031,8 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "datafusion-common", @@ -1041,8 +1041,8 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-buffer", @@ -1070,8 +1070,8 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -1091,8 +1091,8 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -1103,8 +1103,8 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-array", @@ -1124,8 +1124,8 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "async-trait", @@ -1139,8 +1139,8 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1155,8 +1155,8 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1164,17 +1164,17 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] name = "datafusion-optimizer" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "chrono", @@ -1190,8 +1190,8 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -1214,8 +1214,8 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -1227,8 +1227,8 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "datafusion-common", @@ -1242,8 +1242,8 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "ahash", "arrow", @@ -1273,8 +1273,8 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "43.0.0" -source = "git+https://github.com/apache/datafusion.git?rev=3864b113c3e3fe85e18462d6374f8244c4f77b27#3864b113c3e3fe85e18462d6374f8244c4f77b27" +version = "44.0.0" +source = "git+https://github.com/apache/datafusion.git?rev=44.0.0-rc2#3cc3fca31e6edc2d953e663bfd7f856bcb70d8c4" dependencies = [ "arrow", "arrow-array", @@ -1333,7 +1333,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -1384,9 +1384,9 @@ checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" [[package]] name = "flatbuffers" -version = "24.3.25" +version = "24.12.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8add37afff2d4ffa83bc748a70b4b1370984f6980768554182424ef71447c35f" +checksum = "4f1baf0dbf96932ec9a3038d57900329c015b0bfb7b63d904f3bc27e2b02a096" dependencies = [ "bitflags 1.3.2", "rustc_version", @@ -1473,7 +1473,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -1746,7 +1746,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -2231,18 +2231,18 @@ dependencies = [ [[package]] name = "object" -version = "0.36.5" +version = "0.36.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aedf0a2d09c573ed1d8d85b30c119153926a2b36dce0ab28322c09a117a4683e" +checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eb4c22c6154a1e759d7099f9ffad7cc5ef8245f9efbab4a41b92623079c82f3" +checksum = "3cfccb68961a56facde1163f9319e0d15743352344e7808a11795fb99698dcaf" dependencies = [ "async-trait", "bytes", @@ -2556,7 +2556,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -2580,9 +2580,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.37" +version = "1.0.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5b9d34b8991d19d98081b46eacdd8eb58c6f2b201139f7c5f643cc155a633af" +checksum = "0e4dccaaaf89514f546c693ddc140f729f958c247918a13380cccc6078391acc" dependencies = [ "proc-macro2", ] @@ -2714,9 +2714,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e819f2bc632f285be6d7cd36e25940d45b2391dd6d9b939e79de557f7014248" +checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "ryu" @@ -2753,9 +2753,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.216" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b9781016e935a97e8beecf0c933758c97a5520d32930e460142b4cd80c6338e" +checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" dependencies = [ "serde_derive", ] @@ -2772,20 +2772,20 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.216" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46f859dbbf73865c6627ed570e78961cd3ac92407a2d117204c49232485da55e" +checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] name = "serde_json" -version = "1.0.133" +version = "1.0.134" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7fceb2473b9166b2294ef05efcb65a3db80803f0b03ef86a5fc88a2b85ee377" +checksum = "d00f4175c42ee48b15416f6193a959ba3a0d67fc699a0db9ad12df9f83991c7d" dependencies = [ "itoa", "memchr", @@ -2868,7 +2868,7 @@ dependencies = [ "heck 0.5.0", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -2895,7 +2895,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -2932,7 +2932,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -2977,9 +2977,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.90" +version = "2.0.92" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "919d3b74a5dd0ccd15aeb8f93e7006bd9e14c295087c9896a110f490752bcf31" +checksum = "70ae51629bf965c5c098cc9e87908a3df5301051a9e087d6f9bef5c9771ed126" dependencies = [ "proc-macro2", "quote", @@ -2994,7 +2994,7 @@ checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -3027,7 +3027,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -3100,7 +3100,7 @@ checksum = "693d596312e88961bc67d7f1f97af8a70227d9f90c31bba5806eec004978d752" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -3122,7 +3122,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -3276,7 +3276,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", "wasm-bindgen-shared", ] @@ -3298,7 +3298,7 @@ checksum = "30d7a95b763d3c45903ed6c81f156801839e5ee968bb07e534c44df0fcd330c2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -3561,7 +3561,7 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", "synstructure", ] @@ -3583,7 +3583,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] @@ -3603,7 +3603,7 @@ checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", "synstructure", ] @@ -3626,7 +3626,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.90", + "syn 2.0.92", ] [[package]] From 466400c7acc2ae55ac3293cfad6a0d7072105bbd Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 28 Dec 2024 12:14:49 -0700 Subject: [PATCH 40/40] revert miri change --- native/core/src/execution/planner.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index e1f8960ada..5a35c62e33 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2513,7 +2513,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); @@ -2527,7 +2526,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 {