From e863fb0a5e77be631dab63fe6bfb44921981d0e4 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 2 May 2025 14:51:13 -0600 Subject: [PATCH 01/14] bump df to rev 6bda479 --- native/Cargo.lock | 86 +++++++------------ native/Cargo.toml | 2 +- native/core/Cargo.toml | 2 +- native/core/src/execution/operators/copy.rs | 2 +- native/core/src/execution/operators/filter.rs | 2 +- native/core/src/execution/planner.rs | 34 ++------ .../src/execution/shuffle/shuffle_writer.rs | 2 +- native/core/src/parquet/schema_adapter.rs | 8 ++ native/spark-expr/src/hash_funcs/sha2.rs | 6 +- 9 files changed, 54 insertions(+), 90 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index ce7dc7f7f9..e725716b86 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -302,6 +302,8 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7450c76ab7c5a6805be3440dc2e2096010da58f7cab301fdc996a4ee3ee74e49" dependencies = [ "bitflags 2.9.0", + "serde", + "serde_json", ] [[package]] @@ -864,8 +866,7 @@ dependencies = [ [[package]] name = "datafusion" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffe060b978f74ab446be722adb8a274e052e005bf6dfd171caadc3abaad10080" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "arrow-ipc", @@ -915,8 +916,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61fe34f401bd03724a1f96d12108144f8cd495a3cdda2bf5e091822fb80b7e66" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -941,8 +941,7 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4411b8e3bce5e0fc7521e44f201def2e2d5d1b5f176fb56e8cdc9942c890f00" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -1048,8 +1047,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0734015d81c8375eb5d4869b7f7ecccc2ee8d6cb81948ef737cd0e7b743bd69c" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "ahash", "arrow", @@ -1071,8 +1069,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5167bb1d2ccbb87c6bc36c295274d7a0519b14afcfdaf401d53cbcaa4ef4968b" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "futures", "log", @@ -1082,8 +1079,7 @@ dependencies = [ [[package]] name = "datafusion-datasource" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04e602dcdf2f50c2abf297cc2203c73531e6f48b29516af7695d338cf2a778b1" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -1112,8 +1108,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3bb2253952dc32296ed5b84077cb2e0257fea4be6373e1c376426e17ead4ef6" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -1137,8 +1132,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b8c7f47a5d2fe03bfa521ec9bafdb8a5c82de8377f60967c3663f00c8790352" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -1162,8 +1156,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "27d15868ea39ed2dc266728b554f6304acd473de2142281ecfa1294bb7415923" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -1193,14 +1186,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a91f8c2c5788ef32f48ff56c68e5b545527b744822a284373ac79bba1ba47292" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" [[package]] name = "datafusion-execution" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06f004d100f49a3658c9da6fb0c3a9b760062d96cd4ad82ccc3b7b69a9fb2f84" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "dashmap", @@ -1218,8 +1209,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a4e4ce3802609be38eeb607ee72f6fe86c3091460de9dbfae9e18db423b3964" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "chrono", @@ -1238,8 +1228,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "422ac9cf3b22bbbae8cdf8ceb33039107fde1b5492693168f13bd566b1bcc839" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "datafusion-common", @@ -1251,8 +1240,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ddf0a0a2db5d2918349c978d42d80926c6aa2459cd8a3c533a84ec4bb63479e" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "arrow-buffer", @@ -1280,8 +1268,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "408a05dafdc70d05a38a29005b8b15e21b0238734dab1e98483fcb58038c5aba" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "ahash", "arrow", @@ -1301,8 +1288,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "756d21da2dd6c9bef97af1504970ff56cbf35d03fbd4ffd62827f02f4d2279d4" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "ahash", "arrow", @@ -1314,8 +1300,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d8d50f6334b378930d992d801a10ac5b3e93b846b39e4a05085742572844537" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "arrow-ord", @@ -1335,8 +1320,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc9a97220736c8fff1446e936be90d57216c06f28969f9ffd3b72ac93c958c8a" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -1351,8 +1335,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cefc2d77646e1aadd1d6a9c40088937aedec04e68c5f0465939912e1291f8193" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1368,8 +1351,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd4aff082c42fa6da99ce0698c85addd5252928c908eb087ca3cfa64ff16b313" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1378,8 +1360,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df6f88d7ee27daf8b108ba910f9015176b36fbc72902b1ca5c2a5f1d1717e1a1" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "datafusion-expr", "quote", @@ -1389,8 +1370,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "084d9f979c4b155346d3c34b18f4256e6904ded508e9554d90fed416415c3515" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "chrono", @@ -1407,8 +1387,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64c536062b0076f4e30084065d805f389f9fe38af0ca75bcbac86bc5e9fbab65" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "ahash", "arrow", @@ -1429,8 +1408,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8a92b53b3193fac1916a1c5b8e3f4347c526f6822e56b71faa5fb372327a863" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "ahash", "arrow", @@ -1443,8 +1421,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fa0a5ac94c7cf3da97bedabd69d6bbca12aef84b9b37e6e9e8c25286511b5e2" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "datafusion-common", @@ -1461,8 +1438,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "690c615db468c2e5fe5085b232d8b1c088299a6c63d87fd960a354a71f7acb55" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "ahash", "arrow", @@ -1491,8 +1467,7 @@ dependencies = [ [[package]] name = "datafusion-session" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad229a134c7406c057ece00c8743c0c34b97f4e72f78b475fe17b66c5e14fa4f" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "async-trait", @@ -1515,8 +1490,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "47.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64f6ab28b72b664c21a27b22a2ff815fd390ed224c26e89a93b5a8154a4e8607" +source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" dependencies = [ "arrow", "bigdecimal", diff --git a/native/Cargo.toml b/native/Cargo.toml index c8cb6f0c8c..b0fdbb0ed7 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -38,7 +38,7 @@ arrow = { version = "55.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "55.0.0", default-features = false, features = ["experimental"] } -datafusion = { version = "47.0.0", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion = { git = "https://github.com/apache/datafusion", rev = "6bda479", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index e50226935e..ddf5ff35fa 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -77,7 +77,7 @@ jni = { version = "0.21", features = ["invocation"] } lazy_static = "1.4" assertables = "9" hex = "0.4.3" -datafusion-functions-nested = { version = "47.0.0" } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "6bda479" } [features] default = [] diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index 8ae62b38c3..84d39414ad 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -148,7 +148,7 @@ impl ExecutionPlan for CopyExec { } fn statistics(&self) -> DataFusionResult { - self.input.statistics() + self.input.partition_statistics(None) } fn properties(&self) -> &PlanProperties { diff --git a/native/core/src/execution/operators/filter.rs b/native/core/src/execution/operators/filter.rs index 5f0c5d08b2..573287a088 100644 --- a/native/core/src/execution/operators/filter.rs +++ b/native/core/src/execution/operators/filter.rs @@ -166,7 +166,7 @@ impl FilterExec { predicate: &Arc, default_selectivity: u8, ) -> Result { - let input_stats = input.statistics()?; + let input_stats = input.partition_statistics(None)?; let schema = input.schema(); if !check_support(predicate, &schema) { let selectivity = default_selectivity as f64 / 100.0; diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index fe285d6a60..6266391b0e 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -79,8 +79,7 @@ use datafusion::common::{ use datafusion::datasource::listing::PartitionedFile; use datafusion::logical_expr::type_coercion::other::get_coerce_type_for_case_expression; use datafusion::logical_expr::{ - AggregateUDF, ReturnTypeArgs, WindowFrame, WindowFrameBound, WindowFrameUnits, - WindowFunctionDefinition, + AggregateUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, }; use datafusion::physical_expr::expressions::{Literal, StatsType}; use datafusion::physical_expr::window::WindowExpr; @@ -884,7 +883,7 @@ impl PhysicalPlanner { func_name, fun_expr, vec![left, right], - data_type, + Field::new("foo", data_type, false), ))) } _ => Ok(Arc::new(BinaryExpr::new(left, op, right))), @@ -2189,30 +2188,11 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - // TODO this should try and find scalar - let arguments = args + let arg_types = args .iter() - .map(|e| { - e.as_ref() - .as_any() - .downcast_ref::() - .map(|lit| lit.value()) - }) - .collect::>(); - - let nullables = arguments.iter().map(|_| true).collect::>(); - - let args = ReturnTypeArgs { - arg_types: &coerced_types, - scalar_arguments: &arguments, - nullables: &nullables, - }; - - let data_type = func - .inner() - .return_type_from_args(args)? - .return_type() - .clone(); + .map(|expr| expr.data_type(input_schema.as_ref())) + .collect::, _>>()?; + let data_type = func.inner().return_type(&arg_types)?.clone(); (data_type, coerced_types) } @@ -2244,7 +2224,7 @@ impl PhysicalPlanner { fun_name, fun_expr, args.to_vec(), - data_type, + Field::new("foo", data_type, false), )); Ok(scalar_expr) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 69390cda55..e3c0066e47 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -141,7 +141,7 @@ impl ExecutionPlan for ShuffleWriterExec { } fn statistics(&self) -> Result { - self.input.statistics() + self.input.partition_statistics(None) } fn properties(&self) -> &PlanProperties { diff --git a/native/core/src/parquet/schema_adapter.rs b/native/core/src/parquet/schema_adapter.rs index 238bda7fbf..456cc62e9b 100644 --- a/native/core/src/parquet/schema_adapter.rs +++ b/native/core/src/parquet/schema_adapter.rs @@ -20,6 +20,7 @@ use crate::parquet::parquet_support::{spark_parquet_convert, SparkParquetOptions}; use arrow::array::{new_null_array, RecordBatch, RecordBatchOptions}; use arrow::datatypes::{Schema, SchemaRef}; +use datafusion::common::ColumnStatistics; use datafusion::datasource::schema_adapter::{SchemaAdapter, SchemaAdapterFactory, SchemaMapper}; use datafusion::physical_plan::ColumnarValue; use std::sync::Arc; @@ -226,6 +227,13 @@ impl SchemaMapper for SchemaMapping { let record_batch = RecordBatch::try_new_with_options(schema, cols, &options)?; Ok(record_batch) } + + fn map_column_statistics( + &self, + _file_col_statistics: &[ColumnStatistics], + ) -> datafusion::common::Result> { + Ok(vec![]) + } } #[cfg(test)] diff --git a/native/spark-expr/src/hash_funcs/sha2.rs b/native/spark-expr/src/hash_funcs/sha2.rs index 5ed698d5a8..6fb8b930d4 100644 --- a/native/spark-expr/src/hash_funcs/sha2.rs +++ b/native/spark-expr/src/hash_funcs/sha2.rs @@ -17,7 +17,7 @@ use crate::math_funcs::hex::hex_strings; use arrow::array::{Array, StringArray}; -use arrow::datatypes::DataType; +use arrow::datatypes::{DataType, Field}; use datafusion::common::cast::as_binary_array; use datafusion::common::{exec_err, DataFusionError, ScalarValue}; use datafusion::functions::crypto::{sha224, sha256, sha384, sha512}; @@ -55,10 +55,12 @@ fn wrap_digest_result_as_hex_string( ColumnarValue::Array(array) => array.len(), ColumnarValue::Scalar(_) => 1, }; + let return_field = Field::new("foo", DataType::UInt32, false); let value = digest.invoke_with_args(ScalarFunctionArgs { args: args.into(), + arg_fields: vec![], number_rows: row_count, - return_type: &DataType::Utf8, + return_field: &return_field, })?; match value { ColumnarValue::Array(array) => { From ae483bbd1d720da9ebdd89d7b20b8e3155d732ee Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 2 May 2025 14:52:47 -0600 Subject: [PATCH 02/14] fix --- native/spark-expr/src/hash_funcs/sha2.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/spark-expr/src/hash_funcs/sha2.rs b/native/spark-expr/src/hash_funcs/sha2.rs index 6fb8b930d4..e58fe7775c 100644 --- a/native/spark-expr/src/hash_funcs/sha2.rs +++ b/native/spark-expr/src/hash_funcs/sha2.rs @@ -55,7 +55,7 @@ fn wrap_digest_result_as_hex_string( ColumnarValue::Array(array) => array.len(), ColumnarValue::Scalar(_) => 1, }; - let return_field = Field::new("foo", DataType::UInt32, false); + let return_field = Field::new("foo", DataType::Utf8, false); let value = digest.invoke_with_args(ScalarFunctionArgs { args: args.into(), arg_fields: vec![], From ee13ee0a3425c4b649dacfa34a9b723c9690ad9c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 2 May 2025 16:16:43 -0600 Subject: [PATCH 03/14] fix --- native/core/src/execution/planner.rs | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 6266391b0e..15e4f9ccee 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -883,7 +883,7 @@ impl PhysicalPlanner { func_name, fun_expr, vec![left, right], - Field::new("foo", data_type, false), + Field::new("foo", data_type, true), ))) } _ => Ok(Arc::new(BinaryExpr::new(left, op, right))), @@ -2188,11 +2188,7 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - let arg_types = args - .iter() - .map(|expr| expr.data_type(input_schema.as_ref())) - .collect::, _>>()?; - let data_type = func.inner().return_type(&arg_types)?.clone(); + let data_type = func.inner().return_type(&coerced_types)?.clone(); (data_type, coerced_types) } From 6907d6911434c76432d09ca1e18b22004f66cd8c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Fri, 2 May 2025 16:19:03 -0600 Subject: [PATCH 04/14] fix --- native/core/src/execution/planner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 15e4f9ccee..685e3e727c 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2220,7 +2220,7 @@ impl PhysicalPlanner { fun_name, fun_expr, args.to_vec(), - Field::new("foo", data_type, false), + Field::new("foo", data_type, true), )); Ok(scalar_expr) From b3673e8b5d1100b75c82969132bc9da7623e27fb Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sat, 3 May 2025 09:39:04 -0600 Subject: [PATCH 05/14] fix --- native/core/src/execution/planner.rs | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 685e3e727c..1b10f975c2 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -79,7 +79,8 @@ use datafusion::common::{ use datafusion::datasource::listing::PartitionedFile; use datafusion::logical_expr::type_coercion::other::get_coerce_type_for_case_expression; use datafusion::logical_expr::{ - AggregateUDF, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunctionDefinition, + AggregateUDF, ReturnFieldArgs, WindowFrame, WindowFrameBound, WindowFrameUnits, + WindowFunctionDefinition, }; use datafusion::physical_expr::expressions::{Literal, StatsType}; use datafusion::physical_expr::window::WindowExpr; @@ -2188,7 +2189,23 @@ impl PhysicalPlanner { .coerce_types(&input_expr_types) .unwrap_or_else(|_| input_expr_types.clone()); - let data_type = func.inner().return_type(&coerced_types)?.clone(); + let arg_fields = coerced_types + .iter() + .enumerate() + .map(|(i, dt)| Field::new(format!("arg{i}"), dt.clone(), true)) + .collect::>(); + + let args = ReturnFieldArgs { + arg_fields: &arg_fields, + scalar_arguments: &[], + }; + + let data_type = func + .inner() + .return_field_from_args(args)? + .clone() + .data_type() + .clone(); (data_type, coerced_types) } From 58fe4eebfa841a0dcfbb22cd7a1ad7e79cb15124 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Sun, 4 May 2025 12:25:47 -0600 Subject: [PATCH 06/14] fix --- native/core/src/execution/planner.rs | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 1b10f975c2..9b88aefa9e 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2195,9 +2195,20 @@ impl PhysicalPlanner { .map(|(i, dt)| Field::new(format!("arg{i}"), dt.clone(), true)) .collect::>(); + // TODO this should try and find scalar + let arguments = args + .iter() + .map(|e| { + e.as_ref() + .as_any() + .downcast_ref::() + .map(|lit| lit.value()) + }) + .collect::>(); + let args = ReturnFieldArgs { arg_fields: &arg_fields, - scalar_arguments: &[], + scalar_arguments: &arguments, }; let data_type = func From fbc494876f701bdb8f8d1bb7805048a705edcd33 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 8 May 2025 08:33:33 -0600 Subject: [PATCH 07/14] bump to latest rev --- native/Cargo.lock | 253 ++++++++---------- native/Cargo.toml | 2 +- native/core/Cargo.toml | 2 +- native/core/src/parquet/parquet_exec.rs | 4 +- .../spark-expr/src/conversion_funcs/cast.rs | 1 + 5 files changed, 122 insertions(+), 140 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index e725716b86..6c91f6bfc6 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -19,16 +19,16 @@ checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" [[package]] name = "ahash" -version = "0.8.11" +version = "0.8.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" dependencies = [ "cfg-if", "const-random", - "getrandom 0.2.16", + "getrandom 0.3.2", "once_cell", "version_check", - "zerocopy 0.7.35", + "zerocopy", ] [[package]] @@ -169,7 +169,7 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.15.2", + "hashbrown 0.15.3", "num", ] @@ -351,7 +351,7 @@ checksum = "e539d3fca749fcee5236ab05e93a52867dd549cc157c8cb7f99595f3cedffdb5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -377,9 +377,9 @@ checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "backtrace" -version = "0.3.74" +version = "0.3.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d82cb332cdfaed17ae235a638438ac4d4839913cc2af585c3c6746e8f8bee1a" +checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" dependencies = [ "addr2line", "cfg-if", @@ -503,9 +503,9 @@ checksum = "1628fb46dfa0b37568d12e5edd512553eccf6a22a78e8bde00bb4aed84d5bdbf" [[package]] name = "bytemuck" -version = "1.22.0" +version = "1.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6b1fc10dbac614ebc03540c9dbd60e83887fda27794998c6528f1782047d540" +checksum = "9134a6ef01ce4b366b50689c94f82c14bc72bc5d0386829828a2e2752ef7958c" [[package]] name = "byteorder" @@ -527,9 +527,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.19" +version = "1.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e3a13707ac958681c13b39b458c073d0d9bc8a22cb1b2f4c8e55eb72c13f362" +checksum = "8691782945451c1c383942c4874dbe63814f61cb57ef773cda2972682b7bb3c0" dependencies = [ "jobserver", "libc", @@ -565,9 +565,9 @@ checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" [[package]] name = "chrono" -version = "0.4.40" +version = "0.4.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a7964611d71df112cb1730f2ee67324fcf4d0fc6606acbbe9bfe06df124637c" +checksum = "c469d952047f47f91b68d1cba3f10d63c11d73e4636f24f08daf0278abf01c4d" dependencies = [ "android-tzdata", "iana-time-zone", @@ -866,7 +866,7 @@ dependencies = [ [[package]] name = "datafusion" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "arrow-ipc", @@ -916,7 +916,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -941,7 +941,7 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -1047,7 +1047,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "ahash", "arrow", @@ -1069,7 +1069,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "futures", "log", @@ -1079,7 +1079,7 @@ dependencies = [ [[package]] name = "datafusion-datasource" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -1108,7 +1108,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -1132,7 +1132,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -1156,7 +1156,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -1186,12 +1186,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" [[package]] name = "datafusion-execution" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "dashmap", @@ -1209,7 +1209,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "chrono", @@ -1228,7 +1228,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "datafusion-common", @@ -1240,7 +1240,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "arrow-buffer", @@ -1268,7 +1268,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "ahash", "arrow", @@ -1288,7 +1288,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "ahash", "arrow", @@ -1300,7 +1300,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "arrow-ord", @@ -1320,7 +1320,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -1335,7 +1335,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1351,7 +1351,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1360,17 +1360,17 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] name = "datafusion-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "chrono", @@ -1387,7 +1387,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "ahash", "arrow", @@ -1408,7 +1408,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "ahash", "arrow", @@ -1421,7 +1421,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "datafusion-common", @@ -1438,7 +1438,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "ahash", "arrow", @@ -1467,7 +1467,7 @@ dependencies = [ [[package]] name = "datafusion-session" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "async-trait", @@ -1490,7 +1490,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=6bda479#6bda4796a3c8142b87d0fad072bdbe25f9d12934" +source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" dependencies = [ "arrow", "bigdecimal", @@ -1547,7 +1547,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -1573,7 +1573,7 @@ checksum = "44f23cf4b44bfce11a86ace86f8a73ffdec849c9fd00a386a53d278bd9e81fb3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -1736,7 +1736,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -1820,9 +1820,9 @@ checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "h2" -version = "0.4.9" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75249d144030531f8dee69fe9cea04d3edf809a017ae445e2abdff6629e86633" +checksum = "a9421a676d1b147b16b82c9225157dc629087ef8ec4d5e2960f9437a90dac0a5" dependencies = [ "atomic-waker", "bytes", @@ -1860,9 +1860,9 @@ dependencies = [ [[package]] name = "hashbrown" -version = "0.15.2" +version = "0.15.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" +checksum = "84b26c544d002229e640969970a2e74021aadf6e2f96372b9c58eff97de08eb3" [[package]] name = "heck" @@ -1872,9 +1872,9 @@ checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" [[package]] name = "hermit-abi" -version = "0.5.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbd780fe5cc30f81464441920d82ac8740e2e46b29a6fad543ddd075229ce37e" +checksum = "f154ce46856750ed433c8649605bf7ed2de3bc35fd9d2a9f30cddd873c80cb08" [[package]] name = "hex" @@ -2134,7 +2134,7 @@ checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -2165,7 +2165,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cea70ddb795996207ad57735b50c5982d8844f38ba9ee5f1aedcfb708a2aa11e" dependencies = [ "equivalent", - "hashbrown 0.15.2", + "hashbrown 0.15.3", ] [[package]] @@ -2390,9 +2390,9 @@ dependencies = [ [[package]] name = "libm" -version = "0.2.13" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9627da5196e5d8ed0b0495e61e518847578da83483c37288316d9b2e03a7f72" +checksum = "f9fbbcab51052fe104eb5e5d351cf728d30a5be1fe14d9be8a3b097481fb97de" [[package]] name = "libmimalloc-sys" @@ -2700,7 +2700,7 @@ dependencies = [ "md-5", "parking_lot", "percent-encoding", - "quick-xml 0.37.4", + "quick-xml 0.37.5", "rand 0.8.5", "reqwest", "ring", @@ -2786,7 +2786,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.15.2", + "hashbrown 0.15.3", "lz4_flex", "num", "num-bigint", @@ -2950,7 +2950,7 @@ version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" dependencies = [ - "zerocopy 0.8.24", + "zerocopy", ] [[package]] @@ -2960,7 +2960,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "664ec5419c51e34154eec046ebcba56312d5a2fc3b09a06da188e1ad21afadf6" dependencies = [ "proc-macro2", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -2998,7 +2998,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.100", + "syn 2.0.101", "tempfile", ] @@ -3012,7 +3012,7 @@ dependencies = [ "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -3026,9 +3026,9 @@ dependencies = [ [[package]] name = "psm" -version = "0.1.25" +version = "0.1.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f58e5423e24c18cc840e1c98370b3993c6649cd1678b4d24318bcf0a083cbe88" +checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" dependencies = [ "cc", ] @@ -3044,9 +3044,9 @@ dependencies = [ [[package]] name = "quick-xml" -version = "0.37.4" +version = "0.37.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4ce8c88de324ff838700f36fb6ab86c96df0e3c4ab6ef3a9b2044465cce1369" +checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" dependencies = [ "memchr", "serde", @@ -3094,9 +3094,9 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.11" +version = "0.5.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "541d0f57c6ec747a90738a52741d3221f7960e8ac2f0ff4b1a63680e033b4ab5" +checksum = "ee4e529991f949c5e25755532370b8af5d114acae52326361d68d47af64aa842" dependencies = [ "cfg_aliases", "libc", @@ -3217,14 +3217,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] name = "redox_syscall" -version = "0.5.11" +version = "0.5.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2f103c6d277498fbceb16e84d317e2a400f160f46904d5f5410848c829511a3" +checksum = "928fca9cf2aa042393a8325b9ead81d2f0df4cb12e1e24cef072922ccd99c5af" dependencies = [ "bitflags 2.9.0", ] @@ -3369,9 +3369,9 @@ dependencies = [ [[package]] name = "rustix" -version = "1.0.5" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d97817398dd4bb2e6da002002db259209759911da105da92bec29ccb12cf58bf" +checksum = "c71e83d6afe7ff64890ec6b71d6a69bb8a610ab78ce364b3352876bb4c801266" dependencies = [ "bitflags 2.9.0", "errno", @@ -3382,9 +3382,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.26" +version = "0.23.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df51b5869f3a441595eac5e8ff14d486ff285f7b8c0df8770e49c3b56351f0f0" +checksum = "730944ca083c1c233a75c09f199e973ca499344a2b7ba9e755c457e86fb4a321" dependencies = [ "once_cell", "ring", @@ -3417,18 +3417,19 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.11.0" +version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "917ce264624a4b4db1c364dcc35bfca9ded014d0a958cd47ad3e960e988ea51c" +checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" dependencies = [ "web-time", + "zeroize", ] [[package]] name = "rustls-webpki" -version = "0.103.1" +version = "0.103.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fef8b8769aaccf73098557a87cd1816b4f9c7c16811c9c77142aa695c16f2c03" +checksum = "7149975849f1abb3832b246010ef62ccc80d3a76169517ada7188252b9cfb437" dependencies = [ "ring", "rustls-pki-types", @@ -3533,7 +3534,7 @@ checksum = "5b0276cf7f2c73365f7157c8123c21cd9a50fbbd844757af28ca1f5925fc2a00" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -3575,9 +3576,9 @@ dependencies = [ [[package]] name = "sha2" -version = "0.10.8" +version = "0.10.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" dependencies = [ "cfg-if", "cpufeatures", @@ -3658,7 +3659,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -3669,9 +3670,9 @@ checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" [[package]] name = "stacker" -version = "0.1.20" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "601f9201feb9b09c00266478bf459952b9ef9a6b94edb2f21eba14ab681a60a9" +checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" dependencies = [ "cc", "cfg-if", @@ -3700,9 +3701,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "symbolic-common" -version = "12.15.4" +version = "12.15.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23eae23242dffa2e8e66c0e20f4ca1e28391f64e361db1e921a209c9bc70ec3a" +checksum = "6a1150bdda9314f6cfeeea801c23f5593c6e6a6c72e64f67e48d723a12b8efdb" dependencies = [ "debugid", "memmap2", @@ -3712,9 +3713,9 @@ dependencies = [ [[package]] name = "symbolic-demangle" -version = "12.15.4" +version = "12.15.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "153faacda0d58dc1eb3e8bbd5dab998041e95bd7f4ab2caeeadc89410617f144" +checksum = "9f66537def48fbc704a92e4fdaab7833bc7cb2255faca8182592fb5fa617eb82" dependencies = [ "cpp_demangle", "rustc-demangle", @@ -3734,9 +3735,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.100" +version = "2.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b09a44accad81e1ba1cd74a32461ba89dee89095ba17b32f5d03683b1b1fc2a0" +checksum = "8ce2b7fc941b3a24138a0a7cf8e858bfc6a992e7978a068a5c760deb0ed43caf" dependencies = [ "proc-macro2", "quote", @@ -3754,13 +3755,13 @@ dependencies = [ [[package]] name = "synstructure" -version = "0.13.1" +version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -3772,7 +3773,7 @@ dependencies = [ "fastrand", "getrandom 0.3.2", "once_cell", - "rustix 1.0.5", + "rustix 1.0.7", "windows-sys 0.59.0", ] @@ -3802,7 +3803,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -3813,7 +3814,7 @@ checksum = "7f7cf42b4507d8ea322120659672cf1b9dbb93f8f2d4ecfd6e51350ff5b17a1d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -3903,9 +3904,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.44.2" +version = "1.45.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6b88822cbe49de4185e3a4cbf8321dd487cf5fe0c5c65695fef6346371e9c48" +checksum = "2513ca694ef9ede0fb23fe71a4ee4107cb102b9dc1930f6d0fd77aae068ae165" dependencies = [ "backtrace", "bytes", @@ -3926,7 +3927,7 @@ checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -3941,9 +3942,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.14" +version = "0.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b9590b93e6fcc1739458317cccd391ad3955e2bde8913edf6f95f9e65a8f034" +checksum = "66a539a9ad6d5d281510d5bd368c973d636c02dbf8a67300bfb6b950696ad7df" dependencies = [ "bytes", "futures-core", @@ -3998,7 +3999,7 @@ checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -4185,7 +4186,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", "wasm-bindgen-shared", ] @@ -4220,7 +4221,7 @@ checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4331,7 +4332,7 @@ checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -4342,7 +4343,7 @@ checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -4642,48 +4643,28 @@ checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", "synstructure", ] [[package]] name = "zerocopy" -version = "0.7.35" +version = "0.8.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" +checksum = "a1702d9583232ddb9174e01bb7c15a2ab8fb1bc6f227aa1233858c351a3ba0cb" dependencies = [ - "zerocopy-derive 0.7.35", -] - -[[package]] -name = "zerocopy" -version = "0.8.24" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2586fea28e186957ef732a5f8b3be2da217d65c5969d4b1e17f973ebbe876879" -dependencies = [ - "zerocopy-derive 0.8.24", -] - -[[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.100", + "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.8.24" +version = "0.8.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a996a8f63c5c4448cd959ac1bab0aaa3306ccfd060472f85943ee0750f0169be" +checksum = "28a6e20d751156648aa063f3800b706ee209a32c0b4d9f24be3d980b01be55ef" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] @@ -4703,7 +4684,7 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", "synstructure", ] @@ -4732,7 +4713,7 @@ checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.100", + "syn 2.0.101", ] [[package]] diff --git a/native/Cargo.toml b/native/Cargo.toml index b0fdbb0ed7..889b59c654 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -38,7 +38,7 @@ arrow = { version = "55.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "55.0.0", default-features = false, features = ["experimental"] } -datafusion = { git = "https://github.com/apache/datafusion", rev = "6bda479", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion = { git = "https://github.com/apache/datafusion", rev = "d01082e", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index ddf5ff35fa..906a7c6be0 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -77,7 +77,7 @@ jni = { version = "0.21", features = ["invocation"] } lazy_static = "1.4" assertables = "9" hex = "0.4.3" -datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "6bda479" } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "d01082e" } [features] default = [] diff --git a/native/core/src/parquet/parquet_exec.rs b/native/core/src/parquet/parquet_exec.rs index fa7213fdfc..2fde38cdc9 100644 --- a/native/core/src/parquet/parquet_exec.rs +++ b/native/core/src/parquet/parquet_exec.rs @@ -78,8 +78,8 @@ pub(crate) fn init_datasource_exec( )) }); - if let (Some(filter), Some(data_schema)) = (cnf_data_filters, &data_schema) { - parquet_source = parquet_source.with_predicate(Arc::clone(data_schema), filter); + if let Some(filter) = cnf_data_filters { + parquet_source = parquet_source.with_predicate(filter); } } diff --git a/native/spark-expr/src/conversion_funcs/cast.rs b/native/spark-expr/src/conversion_funcs/cast.rs index 59647f2f0e..f33c7f3fe7 100644 --- a/native/spark-expr/src/conversion_funcs/cast.rs +++ b/native/spark-expr/src/conversion_funcs/cast.rs @@ -2104,6 +2104,7 @@ fn date_parser(date_str: &str, eval_mode: EvalMode) -> SparkResult> date_segments[2] as u32, ) { Some(date) => { + #[allow(deprecated)] // TODO fix this let duration_since_epoch = date .signed_duration_since(NaiveDateTime::UNIX_EPOCH.date()) .num_days(); From 61054f7747000dfd067066def8515c3a0443b9f3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 12 May 2025 08:36:01 -0600 Subject: [PATCH 08/14] bump df rev --- native/Cargo.lock | 56 +++++++++++++++++++++--------------------- native/Cargo.toml | 2 +- native/core/Cargo.toml | 2 +- 3 files changed, 30 insertions(+), 30 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 316c8d3097..4e157989cd 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -866,7 +866,7 @@ dependencies = [ [[package]] name = "datafusion" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "arrow-ipc", @@ -916,7 +916,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -941,7 +941,7 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -1049,7 +1049,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "ahash", "arrow", @@ -1071,7 +1071,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "futures", "log", @@ -1081,7 +1081,7 @@ dependencies = [ [[package]] name = "datafusion-datasource" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -1110,7 +1110,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -1134,7 +1134,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -1158,7 +1158,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -1188,12 +1188,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" [[package]] name = "datafusion-execution" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "dashmap", @@ -1211,7 +1211,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "chrono", @@ -1230,7 +1230,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "datafusion-common", @@ -1242,7 +1242,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "arrow-buffer", @@ -1270,7 +1270,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "ahash", "arrow", @@ -1290,7 +1290,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "ahash", "arrow", @@ -1302,7 +1302,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "arrow-ord", @@ -1322,7 +1322,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -1337,7 +1337,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1353,7 +1353,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1362,7 +1362,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "datafusion-expr", "quote", @@ -1372,7 +1372,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "chrono", @@ -1389,7 +1389,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "ahash", "arrow", @@ -1410,7 +1410,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "ahash", "arrow", @@ -1423,7 +1423,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "datafusion-common", @@ -1440,7 +1440,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "ahash", "arrow", @@ -1469,7 +1469,7 @@ dependencies = [ [[package]] name = "datafusion-session" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "async-trait", @@ -1492,7 +1492,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=d01082e#d01082e9f1a196d5a46a5096b415d69afe093b1e" +source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" dependencies = [ "arrow", "bigdecimal", diff --git a/native/Cargo.toml b/native/Cargo.toml index 889b59c654..b157f0b4e5 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -38,7 +38,7 @@ arrow = { version = "55.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "55.0.0", default-features = false, features = ["experimental"] } -datafusion = { git = "https://github.com/apache/datafusion", rev = "d01082e", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion = { git = "https://github.com/apache/datafusion", rev = "f7c7d92", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 2e43d9bb28..529926cce2 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -81,7 +81,7 @@ jni = { version = "0.21", features = ["invocation"] } lazy_static = "1.4" assertables = "9" hex = "0.4.3" -datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "d01082e" } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "f7c7d92" } [features] default = [] From 8dc5c418c6846bc5c7b20d7f62d9c7ef9a0bc9f3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 13 May 2025 07:18:39 -0600 Subject: [PATCH 09/14] bump df rev --- native/Cargo.lock | 491 +++++++++++++++++++++++++++++------------ native/Cargo.toml | 2 +- native/core/Cargo.toml | 2 +- 3 files changed, 349 insertions(+), 146 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 4e157989cd..5fe7f23753 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -866,7 +866,7 @@ dependencies = [ [[package]] name = "datafusion" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "arrow-ipc", @@ -876,24 +876,24 @@ dependencies = [ "chrono", "datafusion-catalog", "datafusion-catalog-listing", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", "datafusion-datasource", "datafusion-datasource-csv", "datafusion-datasource-json", "datafusion-datasource-parquet", - "datafusion-execution", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-functions", - "datafusion-functions-aggregate", - "datafusion-functions-nested", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-nested 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-functions-table", "datafusion-functions-window", - "datafusion-macros", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-optimizer", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-session", @@ -916,17 +916,17 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "datafusion-session", "datafusion-sql", @@ -941,17 +941,17 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -974,7 +974,7 @@ dependencies = [ "datafusion-comet-objectstore-hdfs", "datafusion-comet-proto", "datafusion-comet-spark-expr", - "datafusion-functions-nested", + "datafusion-functions-nested 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "futures", "hex", "itertools 0.14.0", @@ -1049,7 +1049,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "ahash", "arrow", @@ -1068,10 +1068,30 @@ dependencies = [ "web-time", ] +[[package]] +name = "datafusion-common" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "ahash", + "arrow", + "arrow-ipc", + "base64", + "half", + "hashbrown 0.14.5", + "indexmap", + "libc", + "log", + "paste", + "sqlparser", + "tokio", + "web-time", +] + [[package]] name = "datafusion-common-runtime" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "futures", "log", @@ -1081,18 +1101,18 @@ dependencies = [ [[package]] name = "datafusion-datasource" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "bytes", "chrono", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1110,19 +1130,19 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1134,19 +1154,19 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1158,20 +1178,20 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "bytes", "datafusion-catalog", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-session", @@ -1188,17 +1208,22 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" + +[[package]] +name = "datafusion-doc" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" [[package]] name = "datafusion-execution" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "dashmap", - "datafusion-common", - "datafusion-expr", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "futures", "log", "object_store", @@ -1208,19 +1233,56 @@ dependencies = [ "url", ] +[[package]] +name = "datafusion-execution" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "arrow", + "dashmap", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "futures", + "log", + "object_store", + "parking_lot", + "rand 0.8.5", + "tempfile", + "url", +] + +[[package]] +name = "datafusion-expr" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "arrow", + "chrono", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "indexmap", + "paste", + "serde_json", + "sqlparser", +] + [[package]] name = "datafusion-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "chrono", - "datafusion-common", - "datafusion-doc", - "datafusion-expr-common", - "datafusion-functions-aggregate-common", - "datafusion-functions-window-common", - "datafusion-physical-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "indexmap", "paste", "serde_json", @@ -1230,10 +1292,22 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "arrow", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "indexmap", + "itertools 0.14.0", + "paste", +] + +[[package]] +name = "datafusion-expr-common" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "indexmap", "itertools 0.14.0", "paste", @@ -1242,7 +1316,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "arrow-buffer", @@ -1250,12 +1324,12 @@ dependencies = [ "blake2", "blake3", "chrono", - "datafusion-common", - "datafusion-doc", - "datafusion-execution", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-macros", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "hex", "itertools 0.14.0", "log", @@ -1267,21 +1341,65 @@ dependencies = [ "uuid", ] +[[package]] +name = "datafusion-functions" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "arrow", + "arrow-buffer", + "base64", + "chrono", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "hex", + "itertools 0.14.0", + "log", + "rand 0.8.5", + "regex", + "unicode-segmentation", + "uuid", +] + [[package]] name = "datafusion-functions-aggregate" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-doc", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-aggregate-common", - "datafusion-macros", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "half", + "log", + "paste", +] + +[[package]] +name = "datafusion-functions-aggregate" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "half", "log", "paste", @@ -1290,30 +1408,62 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-expr-common", - "datafusion-physical-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", +] + +[[package]] +name = "datafusion-functions-aggregate-common" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", +] + +[[package]] +name = "datafusion-functions-nested" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "arrow", + "arrow-ord", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "itertools 0.14.0", + "log", + "paste", ] [[package]] name = "datafusion-functions-nested" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "arrow-ord", - "datafusion-common", - "datafusion-doc", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions", - "datafusion-functions-aggregate", - "datafusion-macros", - "datafusion-physical-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-functions 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "itertools 0.14.0", "log", "paste", @@ -1322,13 +1472,13 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common", - "datafusion-expr", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "parking_lot", "paste", @@ -1337,15 +1487,15 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" -dependencies = [ - "datafusion-common", - "datafusion-doc", - "datafusion-expr", - "datafusion-functions-window-common", - "datafusion-macros", - "datafusion-physical-expr", - "datafusion-physical-expr-common", +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "log", "paste", ] @@ -1353,18 +1503,37 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ - "datafusion-common", - "datafusion-physical-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", +] + +[[package]] +name = "datafusion-functions-window-common" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", +] + +[[package]] +name = "datafusion-macros" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "quote", + "syn 2.0.101", ] [[package]] name = "datafusion-macros" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ - "datafusion-expr", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "quote", "syn 2.0.101", ] @@ -1372,13 +1541,13 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "chrono", - "datafusion-common", - "datafusion-expr", - "datafusion-physical-expr", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "indexmap", "itertools 0.14.0", "log", @@ -1389,15 +1558,36 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "half", + "hashbrown 0.14.5", + "indexmap", + "itertools 0.14.0", + "log", + "paste", + "petgraph", +] + +[[package]] +name = "datafusion-physical-expr" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-functions-aggregate-common", - "datafusion-physical-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "half", "hashbrown 0.14.5", "indexmap", @@ -1410,12 +1600,25 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +dependencies = [ + "ahash", + "arrow", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "hashbrown 0.14.5", + "itertools 0.14.0", +] + +[[package]] +name = "datafusion-physical-expr-common" +version = "47.0.0" +source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "ahash", "arrow", - "datafusion-common", - "datafusion-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", "hashbrown 0.14.5", "itertools 0.14.0", ] @@ -1423,15 +1626,15 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", - "datafusion-common", - "datafusion-execution", - "datafusion-expr", - "datafusion-expr-common", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "itertools 0.14.0", "log", @@ -1440,7 +1643,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "ahash", "arrow", @@ -1448,13 +1651,13 @@ dependencies = [ "arrow-schema", "async-trait", "chrono", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-functions-window-common", - "datafusion-physical-expr", - "datafusion-physical-expr-common", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "futures", "half", "hashbrown 0.14.5", @@ -1469,16 +1672,16 @@ dependencies = [ [[package]] name = "datafusion-session" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-common-runtime", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr", + "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "datafusion-physical-plan", "datafusion-sql", "futures", @@ -1492,12 +1695,12 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=f7c7d92#f7c7d92199518a486cbcd3b1a6fe2f436d798a02" +source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", "bigdecimal", - "datafusion-common", - "datafusion-expr", + "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", "indexmap", "log", "regex", diff --git a/native/Cargo.toml b/native/Cargo.toml index b157f0b4e5..a55883e3cf 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -38,7 +38,7 @@ arrow = { version = "55.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "55.0.0", default-features = false, features = ["experimental"] } -datafusion = { git = "https://github.com/apache/datafusion", rev = "f7c7d92", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion = { git = "https://github.com/apache/datafusion", rev = "7a3d020", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index 529926cce2..d6465567e3 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -81,7 +81,7 @@ jni = { version = "0.21", features = ["invocation"] } lazy_static = "1.4" assertables = "9" hex = "0.4.3" -datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "f7c7d92" } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "7a3d020af2a3736db11ba7477efc1a5dae96c2d3" } [features] default = [] From a661700f3b7bd45e65d4bd0981d53539b30435c0 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 13 May 2025 07:25:11 -0600 Subject: [PATCH 10/14] fix --- native/Cargo.lock | 433 +++++++++++------------------------------ native/core/Cargo.toml | 2 +- 2 files changed, 116 insertions(+), 319 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 5fe7f23753..1614908851 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -876,24 +876,24 @@ dependencies = [ "chrono", "datafusion-catalog", "datafusion-catalog-listing", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-datasource-csv", "datafusion-datasource-json", "datafusion-datasource-parquet", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-nested 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-functions-nested", "datafusion-functions-table", "datafusion-functions-window", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-macros", "datafusion-optimizer", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-session", @@ -921,12 +921,12 @@ dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-plan", "datafusion-session", "datafusion-sql", @@ -946,12 +946,12 @@ dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-datasource", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -974,7 +974,7 @@ dependencies = [ "datafusion-comet-objectstore-hdfs", "datafusion-comet-proto", "datafusion-comet-spark-expr", - "datafusion-functions-nested 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-functions-nested", "futures", "hex", "itertools 0.14.0", @@ -1068,26 +1068,6 @@ dependencies = [ "web-time", ] -[[package]] -name = "datafusion-common" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "ahash", - "arrow", - "arrow-ipc", - "base64", - "half", - "hashbrown 0.14.5", - "indexmap", - "libc", - "log", - "paste", - "sqlparser", - "tokio", - "web-time", -] - [[package]] name = "datafusion-common-runtime" version = "47.0.0" @@ -1107,12 +1087,12 @@ dependencies = [ "async-trait", "bytes", "chrono", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1136,13 +1116,13 @@ dependencies = [ "async-trait", "bytes", "datafusion-catalog", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1160,13 +1140,13 @@ dependencies = [ "async-trait", "bytes", "datafusion-catalog", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", @@ -1184,14 +1164,14 @@ dependencies = [ "async-trait", "bytes", "datafusion-catalog", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-session", @@ -1210,11 +1190,6 @@ name = "datafusion-doc" version = "47.0.0" source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -[[package]] -name = "datafusion-doc" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" - [[package]] name = "datafusion-execution" version = "47.0.0" @@ -1222,26 +1197,8 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "arrow", "dashmap", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "futures", - "log", - "object_store", - "parking_lot", - "rand 0.8.5", - "tempfile", - "url", -] - -[[package]] -name = "datafusion-execution" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "arrow", - "dashmap", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-expr", "futures", "log", "object_store", @@ -1258,31 +1215,12 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "arrow", "chrono", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "indexmap", - "paste", - "serde_json", - "sqlparser", -] - -[[package]] -name = "datafusion-expr" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "arrow", - "chrono", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-doc", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", + "datafusion-physical-expr-common", "indexmap", "paste", "serde_json", @@ -1295,19 +1233,7 @@ version = "47.0.0" source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "indexmap", - "itertools 0.14.0", - "paste", -] - -[[package]] -name = "datafusion-expr-common" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", "indexmap", "itertools 0.14.0", "paste", @@ -1324,12 +1250,12 @@ dependencies = [ "blake2", "blake3", "chrono", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", "hex", "itertools 0.14.0", "log", @@ -1341,30 +1267,6 @@ dependencies = [ "uuid", ] -[[package]] -name = "datafusion-functions" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "arrow", - "arrow-buffer", - "base64", - "chrono", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "hex", - "itertools 0.14.0", - "log", - "rand 0.8.5", - "regex", - "unicode-segmentation", - "uuid", -] - [[package]] name = "datafusion-functions-aggregate" version = "47.0.0" @@ -1372,34 +1274,14 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "ahash", "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "half", - "log", - "paste", -] - -[[package]] -name = "datafusion-functions-aggregate" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "half", "log", "paste", @@ -1412,21 +1294,9 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "ahash", "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", -] - -[[package]] -name = "datafusion-functions-aggregate-common" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-expr-common", + "datafusion-physical-expr-common", ] [[package]] @@ -1436,34 +1306,14 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "arrow", "arrow-ord", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "itertools 0.14.0", - "log", - "paste", -] - -[[package]] -name = "datafusion-functions-nested" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "arrow", - "arrow-ord", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-functions 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-functions-aggregate 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-functions-aggregate", + "datafusion-macros", + "datafusion-physical-expr-common", "itertools 0.14.0", "log", "paste", @@ -1477,8 +1327,8 @@ dependencies = [ "arrow", "async-trait", "datafusion-catalog", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", + "datafusion-expr", "datafusion-physical-plan", "parking_lot", "paste", @@ -1489,13 +1339,13 @@ name = "datafusion-functions-window" version = "47.0.0" source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-doc 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-macros 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", + "datafusion-doc", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "log", "paste", ] @@ -1505,17 +1355,8 @@ name = "datafusion-functions-window-common" version = "47.0.0" source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", -] - -[[package]] -name = "datafusion-functions-window-common" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-physical-expr-common", ] [[package]] @@ -1523,17 +1364,7 @@ name = "datafusion-macros" version = "47.0.0" source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "quote", - "syn 2.0.101", -] - -[[package]] -name = "datafusion-macros" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-expr", "quote", "syn 2.0.101", ] @@ -1545,9 +1376,9 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "arrow", "chrono", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", "indexmap", "itertools 0.14.0", "log", @@ -1562,32 +1393,11 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "ahash", "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "half", - "hashbrown 0.14.5", - "indexmap", - "itertools 0.14.0", - "log", - "paste", - "petgraph", -] - -[[package]] -name = "datafusion-physical-expr" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-functions-aggregate-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", "indexmap", @@ -1604,21 +1414,8 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "ahash", "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "hashbrown 0.14.5", - "itertools 0.14.0", -] - -[[package]] -name = "datafusion-physical-expr-common" -version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020af2a3736db11ba7477efc1a5dae96c2d3)", + "datafusion-common", + "datafusion-expr-common", "hashbrown 0.14.5", "itertools 0.14.0", ] @@ -1629,12 +1426,12 @@ version = "47.0.0" source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" dependencies = [ "arrow", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "itertools 0.14.0", "log", @@ -1651,13 +1448,13 @@ dependencies = [ "arrow-schema", "async-trait", "chrono", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-functions-window-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "futures", "half", "hashbrown 0.14.5", @@ -1677,11 +1474,11 @@ dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", "datafusion-common-runtime", - "datafusion-execution 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-physical-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-plan", "datafusion-sql", "futures", @@ -1699,8 +1496,8 @@ source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db dependencies = [ "arrow", "bigdecimal", - "datafusion-common 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", - "datafusion-expr 47.0.0 (git+https://github.com/apache/datafusion?rev=7a3d020)", + "datafusion-common", + "datafusion-expr", "indexmap", "log", "regex", diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index d6465567e3..d32da5e60d 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -81,7 +81,7 @@ jni = { version = "0.21", features = ["invocation"] } lazy_static = "1.4" assertables = "9" hex = "0.4.3" -datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "7a3d020af2a3736db11ba7477efc1a5dae96c2d3" } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "7a3d020" } [features] default = [] From 501030ef9d2f8846d24a175cc70da0203e3987cb Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 May 2025 08:31:35 -0600 Subject: [PATCH 11/14] bump DF version --- native/Cargo.lock | 151 ++++++++++++++++++++++++----------------- native/Cargo.toml | 4 +- native/core/Cargo.toml | 2 +- 3 files changed, 90 insertions(+), 67 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 1614908851..824be7f324 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -123,9 +123,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3095aaf545942ff5abd46654534f15b03a90fba78299d661e045e5d587222f0d" +checksum = "b1bb018b6960c87fd9d025009820406f74e83281185a8bdcb44880d2aa5c9a87" dependencies = [ "arrow-arith", "arrow-array", @@ -144,9 +144,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00752064ff47cee746e816ddb8450520c3a52cbad1e256f6fa861a35f86c45e7" +checksum = "44de76b51473aa888ecd6ad93ceb262fb8d40d1f1154a4df2f069b3590aa7575" dependencies = [ "arrow-array", "arrow-buffer", @@ -158,9 +158,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cebfe926794fbc1f49ddd0cdaf898956ca9f6e79541efce62dabccfd81380472" +checksum = "29ed77e22744475a9a53d00026cf8e166fe73cf42d89c4c4ae63607ee1cfcc3f" dependencies = [ "ahash", "arrow-buffer", @@ -175,9 +175,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0303c7ec4cf1a2c60310fc4d6bbc3350cd051a17bf9e9c0a8e47b4db79277824" +checksum = "b0391c96eb58bf7389171d1e103112d3fc3e5625ca6b372d606f2688f1ea4cce" dependencies = [ "bytes", "half", @@ -186,9 +186,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "335f769c5a218ea823d3760a743feba1ef7857cba114c01399a891c2fff34285" +checksum = "f39e1d774ece9292697fcbe06b5584401b26bd34be1bec25c33edae65c2420ff" dependencies = [ "arrow-array", "arrow-buffer", @@ -207,9 +207,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "510db7dfbb4d5761826516cc611d97b3a68835d0ece95b034a052601109c0b1b" +checksum = "9055c972a07bf12c2a827debfd34f88d3b93da1941d36e1d9fee85eebe38a12a" dependencies = [ "arrow-array", "arrow-cast", @@ -223,9 +223,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8affacf3351a24039ea24adab06f316ded523b6f8c3dbe28fbac5f18743451b" +checksum = "cf75ac27a08c7f48b88e5c923f267e980f27070147ab74615ad85b5c5f90473d" dependencies = [ "arrow-buffer", "arrow-schema", @@ -235,9 +235,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69880a9e6934d9cba2b8630dd08a3463a91db8693b16b499d54026b6137af284" +checksum = "a222f0d93772bd058d1268f4c28ea421a603d66f7979479048c429292fac7b2e" dependencies = [ "arrow-array", "arrow-buffer", @@ -249,9 +249,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8dafd17a05449e31e0114d740530e0ada7379d7cb9c338fd65b09a8130960b0" +checksum = "9085342bbca0f75e8cb70513c0807cc7351f1fbf5cb98192a67d5e3044acb033" dependencies = [ "arrow-array", "arrow-buffer", @@ -271,9 +271,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "895644523af4e17502d42c3cb6b27cb820f0cb77954c22d75c23a85247c849e1" +checksum = "ab2f1065a5cad7b9efa9e22ce5747ce826aa3855766755d4904535123ef431e7" dependencies = [ "arrow-array", "arrow-buffer", @@ -284,9 +284,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9be8a2a4e5e7d9c822b2b8095ecd77010576d824f654d347817640acfc97d229" +checksum = "3703a0e3e92d23c3f756df73d2dc9476873f873a76ae63ef9d3de17fda83b2d8" dependencies = [ "arrow-array", "arrow-buffer", @@ -297,9 +297,9 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7450c76ab7c5a6805be3440dc2e2096010da58f7cab301fdc996a4ee3ee74e49" +checksum = "73a47aa0c771b5381de2b7f16998d351a6f4eb839f1e13d48353e17e873d969b" dependencies = [ "bitflags 2.9.0", "serde", @@ -308,9 +308,9 @@ dependencies = [ [[package]] name = "arrow-select" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa5f5a93c75f46ef48e4001535e7b6c922eeb0aa20b73cf58d09e13d057490d8" +checksum = "24b7b85575702b23b85272b01bc1c25a01c9b9852305e5d0078c79ba25d995d4" dependencies = [ "ahash", "arrow-array", @@ -322,9 +322,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e7005d858d84b56428ba2a98a107fe88c0132c61793cf6b8232a1f9bfc0452b" +checksum = "9260fddf1cdf2799ace2b4c2fc0356a9789fa7551e0953e35435536fecefebbd" dependencies = [ "arrow-array", "arrow-buffer", @@ -476,9 +476,9 @@ dependencies = [ [[package]] name = "brotli" -version = "7.0.0" +version = "8.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc97b8f16f944bba54f0433f07e30be199b6dc2bd25937444bbad560bcea29bd" +checksum = "9991eea70ea4f293524138648e41ee89b0b2b12ddef3b255effa43c8056e0e0d" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -487,9 +487,9 @@ dependencies = [ [[package]] name = "brotli-decompressor" -version = "4.0.3" +version = "5.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a334ef7c9e23abf0ce748e8cd309037da93e606ad52eb372e4ce327a0dcfbdfd" +checksum = "874bb8112abecc98cbd6d81ea4fa7e94fb9449648c93cc89aa40c81c24d7de03" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -866,7 +866,7 @@ dependencies = [ [[package]] name = "datafusion" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "arrow-ipc", @@ -916,7 +916,7 @@ dependencies = [ [[package]] name = "datafusion-catalog" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -941,7 +941,7 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -1049,7 +1049,7 @@ dependencies = [ [[package]] name = "datafusion-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "ahash", "arrow", @@ -1071,7 +1071,7 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "futures", "log", @@ -1081,7 +1081,7 @@ dependencies = [ [[package]] name = "datafusion-datasource" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -1110,7 +1110,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -1134,7 +1134,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -1158,7 +1158,7 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -1188,12 +1188,12 @@ dependencies = [ [[package]] name = "datafusion-doc" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" [[package]] name = "datafusion-execution" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "dashmap", @@ -1211,7 +1211,7 @@ dependencies = [ [[package]] name = "datafusion-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "chrono", @@ -1230,7 +1230,7 @@ dependencies = [ [[package]] name = "datafusion-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "datafusion-common", @@ -1242,7 +1242,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "arrow-buffer", @@ -1270,7 +1270,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "ahash", "arrow", @@ -1290,7 +1290,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "ahash", "arrow", @@ -1302,7 +1302,7 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "arrow-ord", @@ -1322,7 +1322,7 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -1337,7 +1337,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "datafusion-common", "datafusion-doc", @@ -1353,7 +1353,7 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1362,7 +1362,7 @@ dependencies = [ [[package]] name = "datafusion-macros" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "datafusion-expr", "quote", @@ -1372,7 +1372,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "chrono", @@ -1389,7 +1389,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "ahash", "arrow", @@ -1404,13 +1404,13 @@ dependencies = [ "itertools 0.14.0", "log", "paste", - "petgraph", + "petgraph 0.8.1", ] [[package]] name = "datafusion-physical-expr-common" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "ahash", "arrow", @@ -1423,7 +1423,7 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "datafusion-common", @@ -1440,7 +1440,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "ahash", "arrow", @@ -1469,7 +1469,7 @@ dependencies = [ [[package]] name = "datafusion-session" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "async-trait", @@ -1492,7 +1492,7 @@ dependencies = [ [[package]] name = "datafusion-sql" version = "47.0.0" -source = "git+https://github.com/apache/datafusion?rev=7a3d020#7a3d020af2a3736db11ba7477efc1a5dae96c2d3" +source = "git+https://github.com/apache/datafusion?rev=efd9587#efd9587b2f271dd54c31e19416f04810f42675ca" dependencies = [ "arrow", "bigdecimal", @@ -1645,6 +1645,12 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" +[[package]] +name = "foldhash" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" + [[package]] name = "form_urlencoded" version = "1.2.1" @@ -1865,6 +1871,11 @@ name = "hashbrown" version = "0.15.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "84b26c544d002229e640969970a2e74021aadf6e2f96372b9c58eff97de08eb3" +dependencies = [ + "allocator-api2", + "equivalent", + "foldhash", +] [[package]] name = "heck" @@ -2769,9 +2780,9 @@ dependencies = [ [[package]] name = "parquet" -version = "55.0.0" +version = "55.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd31a8290ac5b19f09ad77ee7a1e6a541f1be7674ad410547d5f1eef6eef4a9c" +checksum = "be7b2d778f6b841d37083ebdf32e33a524acde1266b5884a8ca29bf00dfa1231" dependencies = [ "ahash", "arrow-array", @@ -2840,6 +2851,18 @@ dependencies = [ "indexmap", ] +[[package]] +name = "petgraph" +version = "0.8.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7a98c6720655620a521dcc722d0ad66cd8afd5d86e34a89ef691c50b7b24de06" +dependencies = [ + "fixedbitset", + "hashbrown 0.15.3", + "indexmap", + "serde", +] + [[package]] name = "phf" version = "0.11.3" @@ -3020,7 +3043,7 @@ dependencies = [ "log", "multimap", "once_cell", - "petgraph", + "petgraph 0.7.1", "prettyplease", "prost", "prost-types", diff --git a/native/Cargo.toml b/native/Cargo.toml index a55883e3cf..bf4d776ee0 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -34,11 +34,11 @@ edition = "2021" rust-version = "1.85" [workspace.dependencies] -arrow = { version = "55.0.0", features = ["prettyprint", "ffi", "chrono-tz"] } +arrow = { version = "55.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } parquet = { version = "55.0.0", default-features = false, features = ["experimental"] } -datafusion = { git = "https://github.com/apache/datafusion", rev = "7a3d020", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } +datafusion = { git = "https://github.com/apache/datafusion", rev = "efd9587", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } chrono = { version = "0.4", default-features = false, features = ["clock"] } diff --git a/native/core/Cargo.toml b/native/core/Cargo.toml index d32da5e60d..a598421d3b 100644 --- a/native/core/Cargo.toml +++ b/native/core/Cargo.toml @@ -81,7 +81,7 @@ jni = { version = "0.21", features = ["invocation"] } lazy_static = "1.4" assertables = "9" hex = "0.4.3" -datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "7a3d020" } +datafusion-functions-nested = { git = "https://github.com/apache/datafusion", rev = "efd9587" } [features] default = [] From 56a20d17df66be643ec36d48e6345975a7c6eb06 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 May 2025 13:45:24 -0600 Subject: [PATCH 12/14] address feedback --- native/Cargo.toml | 2 +- native/core/src/execution/planner.rs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/native/Cargo.toml b/native/Cargo.toml index bf4d776ee0..8a674b1216 100644 --- a/native/Cargo.toml +++ b/native/Cargo.toml @@ -37,7 +37,7 @@ rust-version = "1.85" arrow = { version = "55.1.0", features = ["prettyprint", "ffi", "chrono-tz"] } async-trait = { version = "0.1" } bytes = { version = "1.10.0" } -parquet = { version = "55.0.0", default-features = false, features = ["experimental"] } +parquet = { version = "55.1.0", default-features = false, features = ["experimental"] } datafusion = { git = "https://github.com/apache/datafusion", rev = "efd9587", default-features = false, features = ["unicode_expressions", "crypto_expressions", "nested_expressions", "parquet"] } datafusion-comet-spark-expr = { path = "spark-expr" } datafusion-comet-proto = { path = "proto" } diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5265d9cbbe..4bda4463f6 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -884,7 +884,7 @@ impl PhysicalPlanner { func_name, fun_expr, vec![left, right], - Field::new("foo", data_type, true), + Field::new(func_name, data_type, true), ))) } _ => Ok(Arc::new(BinaryExpr::new(left, op, right))), @@ -2247,7 +2247,7 @@ impl PhysicalPlanner { fun_name, fun_expr, args.to_vec(), - Field::new("foo", data_type, true), + Field::new(fun_name, data_type, true), )); Ok(scalar_expr) From aeede319f456eae1851177b985119418650dc2e9 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 May 2025 13:49:38 -0600 Subject: [PATCH 13/14] address feedback --- native/core/src/execution/operators/copy.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/operators/copy.rs b/native/core/src/execution/operators/copy.rs index 84d39414ad..bafcd911a7 100644 --- a/native/core/src/execution/operators/copy.rs +++ b/native/core/src/execution/operators/copy.rs @@ -147,8 +147,8 @@ impl ExecutionPlan for CopyExec { ))) } - fn statistics(&self) -> DataFusionResult { - self.input.partition_statistics(None) + fn partition_statistics(&self, partition: Option) -> DataFusionResult { + self.input.partition_statistics(partition) } fn properties(&self) -> &PlanProperties { From 2b2b0853c106623d02cbbb5df889e814c7c14bfa Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 20 May 2025 13:52:52 -0600 Subject: [PATCH 14/14] fix deprecated usage --- native/spark-expr/src/conversion_funcs/cast.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/native/spark-expr/src/conversion_funcs/cast.rs b/native/spark-expr/src/conversion_funcs/cast.rs index 367e8886ef..ba8433a366 100644 --- a/native/spark-expr/src/conversion_funcs/cast.rs +++ b/native/spark-expr/src/conversion_funcs/cast.rs @@ -38,7 +38,7 @@ use arrow::{ record_batch::RecordBatch, util::display::FormatOptions, }; -use chrono::{NaiveDate, NaiveDateTime, TimeZone, Timelike}; +use chrono::{DateTime, NaiveDate, TimeZone, Timelike}; use datafusion::common::{ cast::as_generic_string_array, internal_err, Result as DataFusionResult, ScalarValue, }; @@ -2107,9 +2107,8 @@ fn date_parser(date_str: &str, eval_mode: EvalMode) -> SparkResult> date_segments[2] as u32, ) { Some(date) => { - #[allow(deprecated)] // TODO fix this let duration_since_epoch = date - .signed_duration_since(NaiveDateTime::UNIX_EPOCH.date()) + .signed_duration_since(DateTime::UNIX_EPOCH.naive_utc().date()) .num_days(); Ok(Some(duration_since_epoch.to_i32().unwrap())) }