diff --git a/.gitignore b/.gitignore index 05570eacf630..8195760513f7 100644 --- a/.gitignore +++ b/.gitignore @@ -64,6 +64,9 @@ datafusion/sqllogictests/test_files/tpch/data/* # Scratch temp dir for sqllogictests datafusion/sqllogictest/test_files/scratch* +# temp file for core +datafusion/core/*.parquet + # rat filtered_rat.txt rat.txt diff --git a/Cargo.toml b/Cargo.toml index 9cf9d2dfdad9..1ca6cdfdb12d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -53,6 +53,8 @@ members = [ "datafusion-examples/examples/ffi/ffi_module_loader", "test-utils", "benchmarks", + "datafusion/macros", + "datafusion/doc", ] resolver = "2" @@ -90,9 +92,8 @@ arrow-ipc = { version = "53.3.0", default-features = false, features = [ ] } arrow-ord = { version = "53.3.0", default-features = false } arrow-schema = { version = "53.3.0", default-features = false } -arrow-string = { version = "53.3.0", default-features = false } async-trait = "0.1.73" -bigdecimal = "=0.4.1" +bigdecimal = "0.4.6" bytes = "1.4" chrono = { version = "0.4.38", default-features = false } ctor = "0.2.0" @@ -101,6 +102,7 @@ datafusion = { path = "datafusion/core", version = "43.0.0", default-features = datafusion-catalog = { path = "datafusion/catalog", version = "43.0.0" } datafusion-common = { path = "datafusion/common", version = "43.0.0", default-features = false } datafusion-common-runtime = { path = "datafusion/common-runtime", version = "43.0.0" } +datafusion-doc = { path = "datafusion/doc", version = "43.0.0" } datafusion-execution = { path = "datafusion/execution", version = "43.0.0" } datafusion-expr = { path = "datafusion/expr", version = "43.0.0" } datafusion-expr-common = { path = "datafusion/expr-common", version = "43.0.0" } @@ -112,6 +114,7 @@ datafusion-functions-nested = { path = "datafusion/functions-nested", version = datafusion-functions-table = { path = "datafusion/functions-table", version = "43.0.0" } datafusion-functions-window = { path = "datafusion/functions-window", version = "43.0.0" } datafusion-functions-window-common = { path = "datafusion/functions-window-common", version = "43.0.0" } +datafusion-macros = { path = "datafusion/macros", version = "43.0.0" } datafusion-optimizer = { path = "datafusion/optimizer", version = "43.0.0", default-features = false } datafusion-physical-expr = { path = "datafusion/physical-expr", version = "43.0.0", default-features = false } datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "43.0.0", default-features = false } @@ -120,8 +123,6 @@ datafusion-physical-plan = { path = "datafusion/physical-plan", version = "43.0. datafusion-proto = { path = "datafusion/proto", version = "43.0.0" } datafusion-proto-common = { path = "datafusion/proto-common", version = "43.0.0" } datafusion-sql = { path = "datafusion/sql", version = "43.0.0" } -datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "43.0.0" } -datafusion-substrait = { path = "datafusion/substrait", version = "43.0.0" } doc-comment = "0.3" env_logger = "0.11" futures = "0.3" @@ -130,7 +131,6 @@ hashbrown = { version = "0.14.5", features = ["raw"] } indexmap = "2.0.0" itertools = "0.13" log = "^0.4" -num_cpus = "1.13.0" object_store = { version = "0.11.0", default-features = false } parking_lot = "0.12" parquet = { version = "53.3.0", default-features = false, features = [ diff --git a/benchmarks/Cargo.toml b/benchmarks/Cargo.toml index 7f29f7471b6f..ad8debaf2fa3 100644 --- a/benchmarks/Cargo.toml +++ b/benchmarks/Cargo.toml @@ -42,7 +42,6 @@ env_logger = { workspace = true } futures = { workspace = true } log = { workspace = true } mimalloc = { version = "0.1", optional = true, default-features = false } -num_cpus = { workspace = true } parquet = { workspace = true, default-features = true } serde = { version = "1.0.136", features = ["derive"] } serde_json = { workspace = true } diff --git a/benchmarks/src/bin/external_aggr.rs b/benchmarks/src/bin/external_aggr.rs index 6438593a20a0..950c3048c1cc 100644 --- a/benchmarks/src/bin/external_aggr.rs +++ b/benchmarks/src/bin/external_aggr.rs @@ -39,6 +39,7 @@ use datafusion::physical_plan::{collect, displayable}; use datafusion::prelude::*; use datafusion_benchmarks::util::{BenchmarkRun, CommonOpt}; use datafusion_common::instant::Instant; +use datafusion_common::utils::get_available_parallelism; use datafusion_common::{exec_datafusion_err, exec_err, DEFAULT_PARQUET_EXTENSION}; #[derive(Debug, StructOpt)] @@ -325,7 +326,9 @@ impl ExternalAggrConfig { } fn partitions(&self) -> usize { - self.common.partitions.unwrap_or(num_cpus::get()) + self.common + .partitions + .unwrap_or(get_available_parallelism()) } /// Parse memory limit from string to number of bytes diff --git a/benchmarks/src/bin/h2o.rs b/benchmarks/src/bin/h2o.rs index 1ddeb786a591..328db3d85b13 100644 --- a/benchmarks/src/bin/h2o.rs +++ b/benchmarks/src/bin/h2o.rs @@ -27,6 +27,7 @@ use datafusion::datasource::MemTable; use datafusion::prelude::CsvReadOptions; use datafusion::{arrow::util::pretty, error::Result, prelude::SessionContext}; use datafusion_benchmarks::util::BenchmarkRun; +use datafusion_common::utils::get_available_parallelism; use std::path::PathBuf; use std::sync::Arc; use structopt::StructOpt; @@ -91,7 +92,7 @@ async fn group_by(opt: &GroupBy) -> Result<()> { .with_listing_options(ListingOptions::new(Arc::new(CsvFormat::default()))) .with_schema(Arc::new(schema)); let csv = ListingTable::try_new(listing_config)?; - let partition_size = num_cpus::get(); + let partition_size = get_available_parallelism(); let memtable = MemTable::load(Arc::new(csv), Some(partition_size), &ctx.state()).await?; ctx.register_table("x", Arc::new(memtable))?; diff --git a/benchmarks/src/imdb/run.rs b/benchmarks/src/imdb/run.rs index 47c356990881..8d2317c62ef1 100644 --- a/benchmarks/src/imdb/run.rs +++ b/benchmarks/src/imdb/run.rs @@ -35,6 +35,7 @@ use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{collect, displayable}; use datafusion::prelude::*; use datafusion_common::instant::Instant; +use datafusion_common::utils::get_available_parallelism; use datafusion_common::{DEFAULT_CSV_EXTENSION, DEFAULT_PARQUET_EXTENSION}; use log::info; @@ -468,7 +469,9 @@ impl RunOpt { } fn partitions(&self) -> usize { - self.common.partitions.unwrap_or(num_cpus::get()) + self.common + .partitions + .unwrap_or(get_available_parallelism()) } } diff --git a/benchmarks/src/sort.rs b/benchmarks/src/sort.rs index f4b707611cfb..9cf09c57205a 100644 --- a/benchmarks/src/sort.rs +++ b/benchmarks/src/sort.rs @@ -28,7 +28,7 @@ use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion::test_util::parquet::TestParquetFile; use datafusion_common::instant::Instant; - +use datafusion_common::utils::get_available_parallelism; use structopt::StructOpt; /// Test performance of sorting large datasets @@ -147,7 +147,9 @@ impl RunOpt { rundata.start_new_case(title); for i in 0..self.common.iterations { let config = SessionConfig::new().with_target_partitions( - self.common.partitions.unwrap_or(num_cpus::get()), + self.common + .partitions + .unwrap_or(get_available_parallelism()), ); let ctx = SessionContext::new_with_config(config); let (rows, elapsed) = diff --git a/benchmarks/src/sort_tpch.rs b/benchmarks/src/sort_tpch.rs index 4b83b3b8889a..137f4e20b787 100644 --- a/benchmarks/src/sort_tpch.rs +++ b/benchmarks/src/sort_tpch.rs @@ -37,6 +37,7 @@ use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{displayable, execute_stream}; use datafusion::prelude::*; use datafusion_common::instant::Instant; +use datafusion_common::utils::get_available_parallelism; use datafusion_common::DEFAULT_PARQUET_EXTENSION; use crate::util::{BenchmarkRun, CommonOpt}; @@ -315,6 +316,8 @@ impl RunOpt { } fn partitions(&self) -> usize { - self.common.partitions.unwrap_or(num_cpus::get()) + self.common + .partitions + .unwrap_or(get_available_parallelism()) } } diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index 9ff1f72d8606..de3ee3d67db2 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -37,6 +37,7 @@ use datafusion::physical_plan::display::DisplayableExecutionPlan; use datafusion::physical_plan::{collect, displayable}; use datafusion::prelude::*; use datafusion_common::instant::Instant; +use datafusion_common::utils::get_available_parallelism; use datafusion_common::{DEFAULT_CSV_EXTENSION, DEFAULT_PARQUET_EXTENSION}; use log::info; @@ -296,7 +297,9 @@ impl RunOpt { } fn partitions(&self) -> usize { - self.common.partitions.unwrap_or(num_cpus::get()) + self.common + .partitions + .unwrap_or(get_available_parallelism()) } } diff --git a/benchmarks/src/util/options.rs b/benchmarks/src/util/options.rs index b9398e5b522f..b1570a1d1bc1 100644 --- a/benchmarks/src/util/options.rs +++ b/benchmarks/src/util/options.rs @@ -16,6 +16,7 @@ // under the License. use datafusion::prelude::SessionConfig; +use datafusion_common::utils::get_available_parallelism; use structopt::StructOpt; // Common benchmark options (don't use doc comments otherwise this doc @@ -48,7 +49,9 @@ impl CommonOpt { /// Modify the existing config appropriately pub fn update_config(&self, config: SessionConfig) -> SessionConfig { config - .with_target_partitions(self.partitions.unwrap_or(num_cpus::get())) + .with_target_partitions( + self.partitions.unwrap_or(get_available_parallelism()), + ) .with_batch_size(self.batch_size) } } diff --git a/benchmarks/src/util/run.rs b/benchmarks/src/util/run.rs index 5ee6691576b4..13969f4d3949 100644 --- a/benchmarks/src/util/run.rs +++ b/benchmarks/src/util/run.rs @@ -16,6 +16,7 @@ // under the License. use datafusion::{error::Result, DATAFUSION_VERSION}; +use datafusion_common::utils::get_available_parallelism; use serde::{Serialize, Serializer}; use serde_json::Value; use std::{ @@ -68,7 +69,7 @@ impl RunContext { Self { benchmark_version: env!("CARGO_PKG_VERSION").to_owned(), datafusion_version: DATAFUSION_VERSION.to_owned(), - num_cpus: num_cpus::get(), + num_cpus: get_available_parallelism(), start_time: SystemTime::now(), arguments: std::env::args().skip(1).collect::>(), } diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 82c4c258182b..ee095329cf1d 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1200,7 +1200,6 @@ dependencies = [ name = "datafusion" version = "43.0.0" dependencies = [ - "ahash", "apache-avro", "arrow", "arrow-array", @@ -1231,18 +1230,13 @@ dependencies = [ "flate2", "futures", "glob", - "half", - "hashbrown 0.14.5", - "indexmap", "itertools", "log", "num-traits", - "num_cpus", "object_store", "parking_lot", "parquet", "paste", - "pin-project-lite", "rand", "sqlparser", "tempfile", @@ -1313,7 +1307,6 @@ dependencies = [ "hashbrown 0.14.5", "indexmap", "libc", - "num_cpus", "object_store", "parquet", "paste", @@ -1331,6 +1324,10 @@ dependencies = [ "tokio", ] +[[package]] +name = "datafusion-doc" +version = "43.0.0" + [[package]] name = "datafusion-execution" version = "43.0.0" @@ -1341,7 +1338,6 @@ dependencies = [ "datafusion-common", "datafusion-expr", "futures", - "hashbrown 0.14.5", "log", "object_store", "parking_lot", @@ -1360,6 +1356,7 @@ dependencies = [ "arrow-buffer", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-expr-common", "datafusion-functions-aggregate-common", "datafusion-functions-window-common", @@ -1394,8 +1391,10 @@ dependencies = [ "blake3", "chrono", "datafusion-common", + "datafusion-doc", "datafusion-execution", "datafusion-expr", + "datafusion-macros", "hashbrown 0.14.5", "hex", "itertools", @@ -1422,7 +1421,6 @@ dependencies = [ "datafusion-physical-expr", "datafusion-physical-expr-common", "half", - "indexmap", "log", "paste", ] @@ -1504,6 +1502,16 @@ dependencies = [ "datafusion-physical-expr-common", ] +[[package]] +name = "datafusion-macros" +version = "43.0.0" +dependencies = [ + "datafusion-doc", + "proc-macro2", + "quote", + "syn", +] + [[package]] name = "datafusion-optimizer" version = "43.0.0" @@ -1514,11 +1522,9 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "hashbrown 0.14.5", "indexmap", "itertools", "log", - "paste", "recursive", "regex", "regex-syntax", @@ -1532,10 +1538,7 @@ dependencies = [ "arrow", "arrow-array", "arrow-buffer", - "arrow-ord", "arrow-schema", - "arrow-string", - "chrono", "datafusion-common", "datafusion-expr", "datafusion-expr-common", @@ -1560,7 +1563,6 @@ dependencies = [ "datafusion-expr-common", "hashbrown 0.14.5", "itertools", - "rand", ] [[package]] @@ -1568,11 +1570,9 @@ name = "datafusion-physical-optimizer" version = "43.0.0" dependencies = [ "arrow", - "arrow-schema", "datafusion-common", "datafusion-execution", "datafusion-expr-common", - "datafusion-optimizer", "datafusion-physical-expr", "datafusion-physical-plan", "itertools", @@ -1596,7 +1596,6 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", @@ -1627,7 +1626,6 @@ dependencies = [ "recursive", "regex", "sqlparser", - "strum 0.26.3", ] [[package]] @@ -2781,16 +2779,6 @@ dependencies = [ "libm", ] -[[package]] -name = "num_cpus" -version = "1.16.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" -dependencies = [ - "hermit-abi", - "libc", -] - [[package]] name = "object" version = "0.36.5" diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 0305d9bd037c..d8aaad801e5c 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -72,12 +72,8 @@ env_logger = { workspace = true } futures = { workspace = true } log = { workspace = true } mimalloc = { version = "0.1", default-features = false } -num_cpus = { workspace = true } object_store = { workspace = true, features = ["aws", "http"] } prost = { workspace = true } -prost-derive = { workspace = true } -serde = { version = "1.0.136", features = ["derive"] } -serde_json = { workspace = true } tempfile = { workspace = true } test-utils = { path = "../test-utils" } tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] } diff --git a/datafusion-examples/examples/advanced_udf.rs b/datafusion-examples/examples/advanced_udf.rs index 9a3ee9c8ebcd..aee3be6c9285 100644 --- a/datafusion-examples/examples/advanced_udf.rs +++ b/datafusion-examples/examples/advanced_udf.rs @@ -91,7 +91,11 @@ impl ScalarUDFImpl for PowUdf { /// /// However, it also means the implementation is more complex than when /// using `create_udf`. - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { // DataFusion has arranged for the correct inputs to be passed to this // function, but we check again to make sure assert_eq!(args.len(), 2); diff --git a/datafusion-examples/examples/custom_datasource.rs b/datafusion-examples/examples/custom_datasource.rs index 7440e592962b..90e9d2c7a632 100644 --- a/datafusion-examples/examples/custom_datasource.rs +++ b/datafusion-examples/examples/custom_datasource.rs @@ -21,22 +21,22 @@ use std::fmt::{self, Debug, Formatter}; use std::sync::{Arc, Mutex}; use std::time::Duration; +use async_trait::async_trait; use datafusion::arrow::array::{UInt64Builder, UInt8Builder}; use datafusion::arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::arrow::record_batch::RecordBatch; use datafusion::datasource::{provider_as_source, TableProvider, TableType}; use datafusion::error::Result; use datafusion::execution::context::TaskContext; +use datafusion::logical_expr::LogicalPlanBuilder; +use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::memory::MemoryStream; use datafusion::physical_plan::{ project_schema, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, SendableRecordBatchStream, }; use datafusion::prelude::*; -use datafusion_expr::LogicalPlanBuilder; -use datafusion_physical_expr::EquivalenceProperties; -use async_trait::async_trait; use datafusion::catalog::Session; use tokio::time::timeout; diff --git a/datafusion-examples/examples/function_factory.rs b/datafusion-examples/examples/function_factory.rs index b42f25437d77..b2771149aae5 100644 --- a/datafusion-examples/examples/function_factory.rs +++ b/datafusion-examples/examples/function_factory.rs @@ -26,7 +26,9 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{exec_err, internal_err, DataFusionError}; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; -use datafusion_expr::{CreateFunction, Expr, ScalarUDF, ScalarUDFImpl, Signature}; +use datafusion_expr::{ + CreateFunction, Expr, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Signature, +}; /// This example shows how to utilize [FunctionFactory] to implement simple /// SQL-macro like functions using a `CREATE FUNCTION` statement. The same @@ -132,9 +134,9 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { Ok(self.return_type.clone()) } - fn invoke( + fn invoke_with_args( &self, - _args: &[datafusion_expr::ColumnarValue], + _args: ScalarFunctionArgs, ) -> Result { // Since this function is always simplified to another expression, it // should never actually be invoked diff --git a/datafusion-examples/examples/optimizer_rule.rs b/datafusion-examples/examples/optimizer_rule.rs index e0b552620a9a..0f28a1670252 100644 --- a/datafusion-examples/examples/optimizer_rule.rs +++ b/datafusion-examples/examples/optimizer_rule.rs @@ -205,7 +205,11 @@ impl ScalarUDFImpl for MyEq { Ok(DataType::Boolean) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { // this example simply returns "true" which is not what a real // implementation would do. Ok(ColumnarValue::Scalar(ScalarValue::from(true))) diff --git a/datafusion/catalog/LICENSE.txt b/datafusion/catalog/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/catalog/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/catalog/NOTICE.txt b/datafusion/catalog/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/catalog/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/catalog/src/catalog.rs b/datafusion/catalog/src/catalog.rs index 048a7f14ed37..85f2dede2f27 100644 --- a/datafusion/catalog/src/catalog.rs +++ b/datafusion/catalog/src/catalog.rs @@ -101,7 +101,6 @@ use datafusion_common::Result; /// [`UnityCatalogProvider`]: https://github.com/delta-io/delta-rs/blob/951436ecec476ce65b5ed3b58b50fb0846ca7b91/crates/deltalake-core/src/data_catalog/unity/datafusion.rs#L111-L123 /// /// [`TableProvider`]: crate::TableProvider - pub trait CatalogProvider: Debug + Sync + Send { /// Returns the catalog provider as [`Any`] /// so that it can be downcast to a specific implementation. diff --git a/datafusion/common-runtime/LICENSE.txt b/datafusion/common-runtime/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/common-runtime/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/common-runtime/NOTICE.txt b/datafusion/common-runtime/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/common-runtime/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 9f2db95721f5..d76848dfe95e 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -58,7 +58,6 @@ half = { workspace = true } hashbrown = { workspace = true } indexmap = { workspace = true } libc = "0.2.140" -num_cpus = { workspace = true } object_store = { workspace = true, optional = true } parquet = { workspace = true, optional = true, default-features = true } paste = "1.0.15" diff --git a/datafusion/common/LICENSE.txt b/datafusion/common/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/common/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/common/NOTICE.txt b/datafusion/common/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/common/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/common/src/cast.rs b/datafusion/common/src/cast.rs index 0586fcf5e2ae..bb9d80996571 100644 --- a/datafusion/common/src/cast.rs +++ b/datafusion/common/src/cast.rs @@ -36,33 +36,66 @@ use arrow::{ }, datatypes::{ArrowDictionaryKeyType, ArrowPrimitiveType}, }; -use arrow_array::{BinaryViewArray, StringViewArray}; +use arrow_array::{ + BinaryViewArray, Float16Array, Int16Array, Int8Array, LargeBinaryArray, + LargeStringArray, StringViewArray, UInt16Array, +}; // Downcast ArrayRef to Date32Array pub fn as_date32_array(array: &dyn Array) -> Result<&Date32Array> { Ok(downcast_value!(array, Date32Array)) } +// Downcast ArrayRef to Date64Array +pub fn as_date64_array(array: &dyn Array) -> Result<&Date64Array> { + Ok(downcast_value!(array, Date64Array)) +} + // Downcast ArrayRef to StructArray pub fn as_struct_array(array: &dyn Array) -> Result<&StructArray> { Ok(downcast_value!(array, StructArray)) } +// Downcast ArrayRef to Int8Array +pub fn as_int8_array(array: &dyn Array) -> Result<&Int8Array> { + Ok(downcast_value!(array, Int8Array)) +} + // Downcast ArrayRef to UInt8Array pub fn as_uint8_array(array: &dyn Array) -> Result<&UInt8Array> { Ok(downcast_value!(array, UInt8Array)) } +// Downcast ArrayRef to Int16Array +pub fn as_int16_array(array: &dyn Array) -> Result<&Int16Array> { + Ok(downcast_value!(array, Int16Array)) +} + +// Downcast ArrayRef to UInt16Array +pub fn as_uint16_array(array: &dyn Array) -> Result<&UInt16Array> { + Ok(downcast_value!(array, UInt16Array)) +} + // Downcast ArrayRef to Int32Array pub fn as_int32_array(array: &dyn Array) -> Result<&Int32Array> { Ok(downcast_value!(array, Int32Array)) } +// Downcast ArrayRef to UInt32Array +pub fn as_uint32_array(array: &dyn Array) -> Result<&UInt32Array> { + Ok(downcast_value!(array, UInt32Array)) +} + // Downcast ArrayRef to Int64Array pub fn as_int64_array(array: &dyn Array) -> Result<&Int64Array> { Ok(downcast_value!(array, Int64Array)) } +// Downcast ArrayRef to UInt64Array +pub fn as_uint64_array(array: &dyn Array) -> Result<&UInt64Array> { + Ok(downcast_value!(array, UInt64Array)) +} + // Downcast ArrayRef to Decimal128Array pub fn as_decimal128_array(array: &dyn Array) -> Result<&Decimal128Array> { Ok(downcast_value!(array, Decimal128Array)) @@ -73,6 +106,11 @@ pub fn as_decimal256_array(array: &dyn Array) -> Result<&Decimal256Array> { Ok(downcast_value!(array, Decimal256Array)) } +// Downcast ArrayRef to Float16Array +pub fn as_float16_array(array: &dyn Array) -> Result<&Float16Array> { + Ok(downcast_value!(array, Float16Array)) +} + // Downcast ArrayRef to Float32Array pub fn as_float32_array(array: &dyn Array) -> Result<&Float32Array> { Ok(downcast_value!(array, Float32Array)) @@ -93,14 +131,9 @@ pub fn as_string_view_array(array: &dyn Array) -> Result<&StringViewArray> { Ok(downcast_value!(array, StringViewArray)) } -// Downcast ArrayRef to UInt32Array -pub fn as_uint32_array(array: &dyn Array) -> Result<&UInt32Array> { - Ok(downcast_value!(array, UInt32Array)) -} - -// Downcast ArrayRef to UInt64Array -pub fn as_uint64_array(array: &dyn Array) -> Result<&UInt64Array> { - Ok(downcast_value!(array, UInt64Array)) +// Downcast ArrayRef to LargeStringArray +pub fn as_large_string_array(array: &dyn Array) -> Result<&LargeStringArray> { + Ok(downcast_value!(array, LargeStringArray)) } // Downcast ArrayRef to BooleanArray @@ -232,6 +265,11 @@ pub fn as_binary_view_array(array: &dyn Array) -> Result<&BinaryViewArray> { Ok(downcast_value!(array, BinaryViewArray)) } +// Downcast ArrayRef to LargeBinaryArray +pub fn as_large_binary_array(array: &dyn Array) -> Result<&LargeBinaryArray> { + Ok(downcast_value!(array, LargeBinaryArray)) +} + // Downcast ArrayRef to FixedSizeListArray pub fn as_fixed_size_list_array(array: &dyn Array) -> Result<&FixedSizeListArray> { Ok(downcast_value!(array, FixedSizeListArray)) @@ -242,11 +280,6 @@ pub fn as_fixed_size_binary_array(array: &dyn Array) -> Result<&FixedSizeBinaryA Ok(downcast_value!(array, FixedSizeBinaryArray)) } -// Downcast ArrayRef to Date64Array -pub fn as_date64_array(array: &dyn Array) -> Result<&Date64Array> { - Ok(downcast_value!(array, Date64Array)) -} - // Downcast ArrayRef to GenericBinaryArray pub fn as_generic_string_array( array: &dyn Array, diff --git a/datafusion/common/src/column.rs b/datafusion/common/src/column.rs index c47ed2815906..d940bcf3146e 100644 --- a/datafusion/common/src/column.rs +++ b/datafusion/common/src/column.rs @@ -109,21 +109,23 @@ impl Column { /// where `"foo.BAR"` would be parsed to a reference to column named `foo.BAR` pub fn from_qualified_name(flat_name: impl Into) -> Self { let flat_name = flat_name.into(); - Self::from_idents(&mut parse_identifiers_normalized(&flat_name, false)) - .unwrap_or_else(|| Self { + Self::from_idents(&mut parse_identifiers_normalized(&flat_name, false)).unwrap_or( + Self { relation: None, name: flat_name, - }) + }, + ) } /// Deserialize a fully qualified name string into a column preserving column text case pub fn from_qualified_name_ignore_case(flat_name: impl Into) -> Self { let flat_name = flat_name.into(); - Self::from_idents(&mut parse_identifiers_normalized(&flat_name, true)) - .unwrap_or_else(|| Self { + Self::from_idents(&mut parse_identifiers_normalized(&flat_name, true)).unwrap_or( + Self { relation: None, name: flat_name, - }) + }, + ) } /// return the column's name. diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 1ad10d164868..3a07a238a4c9 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -24,6 +24,7 @@ use std::str::FromStr; use crate::error::_config_err; use crate::parsers::CompressionTypeVariant; +use crate::utils::get_available_parallelism; use crate::{DataFusionError, Result}; /// A macro that wraps a configuration struct and automatically derives @@ -250,7 +251,7 @@ config_namespace! { /// concurrency. /// /// Defaults to the number of CPU cores on the system - pub target_partitions: usize, default = num_cpus::get() + pub target_partitions: usize, default = get_available_parallelism() /// The default time zone /// @@ -266,7 +267,7 @@ config_namespace! { /// This is mostly use to plan `UNION` children in parallel. /// /// Defaults to the number of CPU cores on the system - pub planning_concurrency: usize, default = num_cpus::get() + pub planning_concurrency: usize, default = get_available_parallelism() /// When set to true, skips verifying that the schema produced by /// planning the input of `LogicalPlan::Aggregate` exactly matches the diff --git a/datafusion/common/src/dfschema.rs b/datafusion/common/src/dfschema.rs index e893cee089c9..45620c3cacc8 100644 --- a/datafusion/common/src/dfschema.rs +++ b/datafusion/common/src/dfschema.rs @@ -656,9 +656,26 @@ impl DFSchema { (othertype, DataType::Dictionary(_, v1)) => v1.as_ref() == othertype, (DataType::List(f1), DataType::List(f2)) | (DataType::LargeList(f1), DataType::LargeList(f2)) - | (DataType::FixedSizeList(f1, _), DataType::FixedSizeList(f2, _)) - | (DataType::Map(f1, _), DataType::Map(f2, _)) => { - Self::field_is_logically_equal(f1, f2) + | (DataType::FixedSizeList(f1, _), DataType::FixedSizeList(f2, _)) => { + // Don't compare the names of the technical inner field + // Usually "item" but that's not mandated + Self::datatype_is_logically_equal(f1.data_type(), f2.data_type()) + } + (DataType::Map(f1, _), DataType::Map(f2, _)) => { + // Don't compare the names of the technical inner fields + // Usually "entries", "key", "value" but that's not mandated + match (f1.data_type(), f2.data_type()) { + (DataType::Struct(f1_inner), DataType::Struct(f2_inner)) => { + f1_inner.len() == f2_inner.len() + && f1_inner.iter().zip(f2_inner.iter()).all(|(f1, f2)| { + Self::datatype_is_logically_equal( + f1.data_type(), + f2.data_type(), + ) + }) + } + _ => panic!("Map type should have an inner struct field"), + } } (DataType::Struct(fields1), DataType::Struct(fields2)) => { let iter1 = fields1.iter(); @@ -695,9 +712,26 @@ impl DFSchema { } (DataType::List(f1), DataType::List(f2)) | (DataType::LargeList(f1), DataType::LargeList(f2)) - | (DataType::FixedSizeList(f1, _), DataType::FixedSizeList(f2, _)) - | (DataType::Map(f1, _), DataType::Map(f2, _)) => { - Self::field_is_semantically_equal(f1, f2) + | (DataType::FixedSizeList(f1, _), DataType::FixedSizeList(f2, _)) => { + // Don't compare the names of the technical inner field + // Usually "item" but that's not mandated + Self::datatype_is_semantically_equal(f1.data_type(), f2.data_type()) + } + (DataType::Map(f1, _), DataType::Map(f2, _)) => { + // Don't compare the names of the technical inner fields + // Usually "entries", "key", "value" but that's not mandated + match (f1.data_type(), f2.data_type()) { + (DataType::Struct(f1_inner), DataType::Struct(f2_inner)) => { + f1_inner.len() == f2_inner.len() + && f1_inner.iter().zip(f2_inner.iter()).all(|(f1, f2)| { + Self::datatype_is_semantically_equal( + f1.data_type(), + f2.data_type(), + ) + }) + } + _ => panic!("Map type should have an inner struct field"), + } } (DataType::Struct(fields1), DataType::Struct(fields2)) => { let iter1 = fields1.iter(); @@ -1332,6 +1366,286 @@ mod tests { Ok(()) } + #[test] + fn test_datatype_is_logically_equal() { + assert!(DFSchema::datatype_is_logically_equal( + &DataType::Int8, + &DataType::Int8 + )); + + assert!(!DFSchema::datatype_is_logically_equal( + &DataType::Int8, + &DataType::Int16 + )); + + // Test lists + + // Succeeds if both have the same element type, disregards names and nullability + assert!(DFSchema::datatype_is_logically_equal( + &DataType::List(Field::new("item", DataType::Int8, true).into()), + &DataType::List(Field::new("element", DataType::Int8, false).into()) + )); + + // Fails if element type is different + assert!(!DFSchema::datatype_is_logically_equal( + &DataType::List(Field::new("item", DataType::Int8, true).into()), + &DataType::List(Field::new("item", DataType::Int16, true).into()) + )); + + // Test maps + let map_field = DataType::Map( + Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Int8, false), + Field::new("value", DataType::Int8, true), + ])), + true, + ) + .into(), + true, + ); + + // Succeeds if both maps have the same key and value types, disregards names and nullability + assert!(DFSchema::datatype_is_logically_equal( + &map_field, + &DataType::Map( + Field::new( + "pairs", + DataType::Struct(Fields::from(vec![ + Field::new("one", DataType::Int8, false), + Field::new("two", DataType::Int8, false) + ])), + true + ) + .into(), + true + ) + )); + // Fails if value type is different + assert!(!DFSchema::datatype_is_logically_equal( + &map_field, + &DataType::Map( + Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Int8, false), + Field::new("value", DataType::Int16, true) + ])), + true + ) + .into(), + true + ) + )); + + // Fails if key type is different + assert!(!DFSchema::datatype_is_logically_equal( + &map_field, + &DataType::Map( + Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Int16, false), + Field::new("value", DataType::Int8, true) + ])), + true + ) + .into(), + true + ) + )); + + // Test structs + + let struct_field = DataType::Struct(Fields::from(vec![ + Field::new("a", DataType::Int8, true), + Field::new("b", DataType::Int8, true), + ])); + + // Succeeds if both have same names and datatypes, ignores nullability + assert!(DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![ + Field::new("a", DataType::Int8, false), + Field::new("b", DataType::Int8, true), + ])) + )); + + // Fails if field names are different + assert!(!DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![ + Field::new("x", DataType::Int8, true), + Field::new("y", DataType::Int8, true), + ])) + )); + + // Fails if types are different + assert!(!DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![ + Field::new("a", DataType::Int16, true), + Field::new("b", DataType::Int8, true), + ])) + )); + + // Fails if more or less fields + assert!(!DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![Field::new("a", DataType::Int8, true),])) + )); + } + + #[test] + fn test_datatype_is_logically_equivalent_to_dictionary() { + // Dictionary is logically equal to its value type + assert!(DFSchema::datatype_is_logically_equal( + &DataType::Utf8, + &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) + )); + } + + #[test] + fn test_datatype_is_semantically_equal() { + assert!(DFSchema::datatype_is_semantically_equal( + &DataType::Int8, + &DataType::Int8 + )); + + assert!(!DFSchema::datatype_is_semantically_equal( + &DataType::Int8, + &DataType::Int16 + )); + + // Test lists + + // Succeeds if both have the same element type, disregards names and nullability + assert!(DFSchema::datatype_is_semantically_equal( + &DataType::List(Field::new("item", DataType::Int8, true).into()), + &DataType::List(Field::new("element", DataType::Int8, false).into()) + )); + + // Fails if element type is different + assert!(!DFSchema::datatype_is_semantically_equal( + &DataType::List(Field::new("item", DataType::Int8, true).into()), + &DataType::List(Field::new("item", DataType::Int16, true).into()) + )); + + // Test maps + let map_field = DataType::Map( + Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Int8, false), + Field::new("value", DataType::Int8, true), + ])), + true, + ) + .into(), + true, + ); + + // Succeeds if both maps have the same key and value types, disregards names and nullability + assert!(DFSchema::datatype_is_semantically_equal( + &map_field, + &DataType::Map( + Field::new( + "pairs", + DataType::Struct(Fields::from(vec![ + Field::new("one", DataType::Int8, false), + Field::new("two", DataType::Int8, false) + ])), + true + ) + .into(), + true + ) + )); + // Fails if value type is different + assert!(!DFSchema::datatype_is_semantically_equal( + &map_field, + &DataType::Map( + Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Int8, false), + Field::new("value", DataType::Int16, true) + ])), + true + ) + .into(), + true + ) + )); + + // Fails if key type is different + assert!(!DFSchema::datatype_is_semantically_equal( + &map_field, + &DataType::Map( + Field::new( + "entries", + DataType::Struct(Fields::from(vec![ + Field::new("key", DataType::Int16, false), + Field::new("value", DataType::Int8, true) + ])), + true + ) + .into(), + true + ) + )); + + // Test structs + + let struct_field = DataType::Struct(Fields::from(vec![ + Field::new("a", DataType::Int8, true), + Field::new("b", DataType::Int8, true), + ])); + + // Succeeds if both have same names and datatypes, ignores nullability + assert!(DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![ + Field::new("a", DataType::Int8, false), + Field::new("b", DataType::Int8, true), + ])) + )); + + // Fails if field names are different + assert!(!DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![ + Field::new("x", DataType::Int8, true), + Field::new("y", DataType::Int8, true), + ])) + )); + + // Fails if types are different + assert!(!DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![ + Field::new("a", DataType::Int16, true), + Field::new("b", DataType::Int8, true), + ])) + )); + + // Fails if more or less fields + assert!(!DFSchema::datatype_is_logically_equal( + &struct_field, + &DataType::Struct(Fields::from(vec![Field::new("a", DataType::Int8, true),])) + )); + } + + #[test] + fn test_datatype_is_not_semantically_equivalent_to_dictionary() { + // Dictionary is not semantically equal to its value type + assert!(!DFSchema::datatype_is_semantically_equal( + &DataType::Utf8, + &DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) + )); + } + fn test_schema_2() -> Schema { Schema::new(vec![ Field::new("c100", DataType::Boolean, true), diff --git a/datafusion/common/src/hash_utils.rs b/datafusion/common/src/hash_utils.rs index e18d70844d32..0d1d93acf1fc 100644 --- a/datafusion/common/src/hash_utils.rs +++ b/datafusion/common/src/hash_utils.rs @@ -63,7 +63,7 @@ pub trait HashValue { fn hash_one(&self, state: &RandomState) -> u64; } -impl<'a, T: HashValue + ?Sized> HashValue for &'a T { +impl HashValue for &T { fn hash_one(&self, state: &RandomState) -> u64 { T::hash_one(self, state) } @@ -321,8 +321,7 @@ fn hash_fixed_list_array( hashes_buffer: &mut [u64], ) -> Result<()> { let values = Arc::clone(array.values()); - let value_len = array.value_length(); - let offset_size = value_len as usize / array.len(); + let value_length = array.value_length() as usize; let nulls = array.nulls(); let mut values_hashes = vec![0u64; values.len()]; create_hashes(&[values], random_state, &mut values_hashes)?; @@ -330,7 +329,8 @@ fn hash_fixed_list_array( for i in 0..array.len() { if nulls.is_valid(i) { let hash = &mut hashes_buffer[i]; - for values_hash in &values_hashes[i * offset_size..(i + 1) * offset_size] + for values_hash in + &values_hashes[i * value_length..(i + 1) * value_length] { *hash = combine_hashes(*hash, *values_hash); } @@ -339,7 +339,7 @@ fn hash_fixed_list_array( } else { for i in 0..array.len() { let hash = &mut hashes_buffer[i]; - for values_hash in &values_hashes[i * offset_size..(i + 1) * offset_size] { + for values_hash in &values_hashes[i * value_length..(i + 1) * value_length] { *hash = combine_hashes(*hash, *values_hash); } } @@ -454,6 +454,16 @@ mod tests { Ok(()) } + #[test] + fn create_hashes_for_empty_fixed_size_lit() -> Result<()> { + let empty_array = FixedSizeListBuilder::new(StringBuilder::new(), 1).finish(); + let random_state = RandomState::with_seeds(0, 0, 0, 0); + let hashes_buff = &mut vec![0; 0]; + let hashes = create_hashes(&[Arc::new(empty_array)], &random_state, hashes_buff)?; + assert_eq!(hashes, &Vec::::new()); + Ok(()) + } + #[test] fn create_hashes_for_float_arrays() -> Result<()> { let f32_arr = Arc::new(Float32Array::from(vec![0.12, 0.5, 1f32, 444.7])); diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 5595f4f9fa70..edba0b84431f 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -40,7 +40,9 @@ use crate::cast::{ use crate::error::{DataFusionError, Result, _exec_err, _internal_err, _not_impl_err}; use crate::hash_utils::create_hashes; use crate::utils::{ - array_into_fixed_size_list_array, array_into_large_list_array, array_into_list_array, + array_into_fixed_size_list_array_with_field_name, array_into_large_list_array, + array_into_large_list_array_with_field_name, array_into_list_array, + array_into_list_array_with_field_name, }; use arrow::compute::kernels::numeric::*; use arrow::util::display::{array_value_to_string, ArrayFormatter, FormatOptions}; @@ -2663,27 +2665,36 @@ impl ScalarValue { let list_array = array.as_list::(); let nested_array = list_array.value(index); // Produces a single element `ListArray` with the value at `index`. - let arr = - Arc::new(array_into_list_array(nested_array, field.is_nullable())); + let arr = Arc::new(array_into_list_array_with_field_name( + nested_array, + field.is_nullable(), + field.name(), + )); ScalarValue::List(arr) } - DataType::LargeList(_) => { + DataType::LargeList(field) => { let list_array = as_large_list_array(array); let nested_array = list_array.value(index); // Produces a single element `LargeListArray` with the value at `index`. - let arr = Arc::new(array_into_large_list_array(nested_array)); + let arr = Arc::new(array_into_large_list_array_with_field_name( + nested_array, + field.name(), + )); ScalarValue::LargeList(arr) } // TODO: There is no test for FixedSizeList now, add it later - DataType::FixedSizeList(_, _) => { + DataType::FixedSizeList(field, _) => { let list_array = as_fixed_size_list_array(array)?; let nested_array = list_array.value(index); // Produces a single element `ListArray` with the value at `index`. let list_size = nested_array.len(); - let arr = - Arc::new(array_into_fixed_size_list_array(nested_array, list_size)); + let arr = Arc::new(array_into_fixed_size_list_array_with_field_name( + nested_array, + list_size, + field.name(), + )); ScalarValue::FixedSizeList(arr) } @@ -5970,6 +5981,51 @@ mod tests { ScalarValue::from("larger than 12 bytes string"), DataType::Utf8View, ); + check_scalar_cast( + { + let element_field = + Arc::new(Field::new("element", DataType::Int32, true)); + + let mut builder = + ListBuilder::new(Int32Builder::new()).with_field(element_field); + builder.append_value([Some(1)]); + builder.append(true); + + ScalarValue::List(Arc::new(builder.finish())) + }, + DataType::List(Arc::new(Field::new("element", DataType::Int64, true))), + ); + check_scalar_cast( + { + let element_field = + Arc::new(Field::new("element", DataType::Int32, true)); + + let mut builder = FixedSizeListBuilder::new(Int32Builder::new(), 1) + .with_field(element_field); + builder.values().append_value(1); + builder.append(true); + + ScalarValue::FixedSizeList(Arc::new(builder.finish())) + }, + DataType::FixedSizeList( + Arc::new(Field::new("element", DataType::Int64, true)), + 1, + ), + ); + check_scalar_cast( + { + let element_field = + Arc::new(Field::new("element", DataType::Int32, true)); + + let mut builder = + LargeListBuilder::new(Int32Builder::new()).with_field(element_field); + builder.append_value([Some(1)]); + builder.append(true); + + ScalarValue::LargeList(Arc::new(builder.finish())) + }, + DataType::LargeList(Arc::new(Field::new("element", DataType::Int64, true))), + ); } // mimics how casting work on scalar values by `casting` `scalar` to `desired_type` diff --git a/datafusion/common/src/utils/mod.rs b/datafusion/common/src/utils/mod.rs index dacf90af9bbf..f3bba8e254d9 100644 --- a/datafusion/common/src/utils/mod.rs +++ b/datafusion/common/src/utils/mod.rs @@ -39,8 +39,10 @@ use sqlparser::parser::Parser; use std::borrow::{Borrow, Cow}; use std::cmp::{min, Ordering}; use std::collections::HashSet; +use std::num::NonZero; use std::ops::Range; use std::sync::Arc; +use std::thread::available_parallelism; /// Applies an optional projection to a [`SchemaRef`], returning the /// projected schema @@ -319,8 +321,6 @@ pub fn longest_consecutive_prefix>( count } -/// Array Utils - /// Wrap an array into a single element `ListArray`. /// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` /// The field in the list array is nullable. @@ -328,8 +328,6 @@ pub fn array_into_list_array_nullable(arr: ArrayRef) -> ListArray { array_into_list_array(arr, true) } -/// Array Utils - /// Wrap an array into a single element `ListArray`. /// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` pub fn array_into_list_array(arr: ArrayRef, nullable: bool) -> ListArray { @@ -342,6 +340,20 @@ pub fn array_into_list_array(arr: ArrayRef, nullable: bool) -> ListArray { ) } +pub fn array_into_list_array_with_field_name( + arr: ArrayRef, + nullable: bool, + field_name: &str, +) -> ListArray { + let offsets = OffsetBuffer::from_lengths([arr.len()]); + ListArray::new( + Arc::new(Field::new(field_name, arr.data_type().to_owned(), nullable)), + offsets, + arr, + None, + ) +} + /// Wrap an array into a single element `LargeListArray`. /// For example `[1, 2, 3]` would be converted into `[[1, 2, 3]]` pub fn array_into_large_list_array(arr: ArrayRef) -> LargeListArray { @@ -354,6 +366,19 @@ pub fn array_into_large_list_array(arr: ArrayRef) -> LargeListArray { ) } +pub fn array_into_large_list_array_with_field_name( + arr: ArrayRef, + field_name: &str, +) -> LargeListArray { + let offsets = OffsetBuffer::from_lengths([arr.len()]); + LargeListArray::new( + Arc::new(Field::new(field_name, arr.data_type().to_owned(), true)), + offsets, + arr, + None, + ) +} + pub fn array_into_fixed_size_list_array( arr: ArrayRef, list_size: usize, @@ -367,6 +392,20 @@ pub fn array_into_fixed_size_list_array( ) } +pub fn array_into_fixed_size_list_array_with_field_name( + arr: ArrayRef, + list_size: usize, + field_name: &str, +) -> FixedSizeListArray { + let list_size = list_size as i32; + FixedSizeListArray::new( + Arc::new(Field::new(field_name, arr.data_type().to_owned(), true)), + list_size, + arr, + None, + ) +} + /// Wrap arrays into a single element `ListArray`. /// /// Example: @@ -528,7 +567,7 @@ pub mod datafusion_strsim { struct StringWrapper<'a>(&'a str); - impl<'a, 'b> IntoIterator for &'a StringWrapper<'b> { + impl<'b> IntoIterator for &StringWrapper<'b> { type Item = char; type IntoIter = Chars<'b>; @@ -724,6 +763,16 @@ pub fn combine_limit( (combined_skip, combined_fetch) } +/// Returns the estimated number of threads available for parallel execution. +/// +/// This is a wrapper around `std::thread::available_parallelism`, providing a default value +/// of `1` if the system's parallelism cannot be determined. +pub fn get_available_parallelism() -> usize { + available_parallelism() + .unwrap_or(NonZero::new(1).expect("literal value `1` shouldn't be zero")) + .get() +} + #[cfg(test)] mod tests { use crate::ScalarValue::Null; diff --git a/datafusion/common/src/utils/proxy.rs b/datafusion/common/src/utils/proxy.rs index 5d14a1517129..b32164f682fa 100644 --- a/datafusion/common/src/utils/proxy.rs +++ b/datafusion/common/src/utils/proxy.rs @@ -17,7 +17,10 @@ //! [`VecAllocExt`] and [`RawTableAllocExt`] to help tracking of memory allocations -use hashbrown::raw::{Bucket, RawTable}; +use hashbrown::{ + hash_table::HashTable, + raw::{Bucket, RawTable}, +}; use std::mem::size_of; /// Extension trait for [`Vec`] to account for allocations. @@ -173,3 +176,71 @@ impl RawTableAllocExt for RawTable { } } } + +/// Extension trait for hash browns [`HashTable`] to account for allocations. +pub trait HashTableAllocExt { + /// Item type. + type T; + + /// Insert new element into table and increase + /// `accounting` by any newly allocated bytes. + /// + /// Returns the bucket where the element was inserted. + /// Note that allocation counts capacity, not size. + /// + /// # Example: + /// ``` + /// # use datafusion_common::utils::proxy::HashTableAllocExt; + /// # use hashbrown::hash_table::HashTable; + /// let mut table = HashTable::new(); + /// let mut allocated = 0; + /// let hash_fn = |x: &u32| (*x as u64) % 1000; + /// // pretend 0x3117 is the hash value for 1 + /// table.insert_accounted(1, hash_fn, &mut allocated); + /// assert_eq!(allocated, 64); + /// + /// // insert more values + /// for i in 0..100 { table.insert_accounted(i, hash_fn, &mut allocated); } + /// assert_eq!(allocated, 400); + /// ``` + fn insert_accounted( + &mut self, + x: Self::T, + hasher: impl Fn(&Self::T) -> u64, + accounting: &mut usize, + ); +} + +impl HashTableAllocExt for HashTable +where + T: Eq, +{ + type T = T; + + fn insert_accounted( + &mut self, + x: Self::T, + hasher: impl Fn(&Self::T) -> u64, + accounting: &mut usize, + ) { + let hash = hasher(&x); + + // NOTE: `find_entry` does NOT grow! + match self.find_entry(hash, |y| y == &x) { + Ok(_occupied) => {} + Err(_absent) => { + if self.len() == self.capacity() { + // need to request more memory + let bump_elements = self.capacity().max(16); + let bump_size = bump_elements * size_of::(); + *accounting = (*accounting).checked_add(bump_size).expect("overflow"); + + self.reserve(bump_elements, &hasher); + } + + // still need to insert the element since first try failed + self.entry(hash, |y| y == &x, hasher).insert(x); + } + } + } +} diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 45a5a84b798d..6c5a31e3624a 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -77,7 +77,6 @@ unicode_expressions = [ ] [dependencies] -ahash = { workspace = true } apache-avro = { version = "0.16", optional = true } arrow = { workspace = true } arrow-array = { workspace = true } @@ -115,18 +114,13 @@ datafusion-sql = { workspace = true } flate2 = { version = "1.0.24", optional = true } futures = { workspace = true } glob = "0.3.0" -half = { workspace = true } -hashbrown = { workspace = true } -indexmap = { workspace = true } itertools = { workspace = true } log = { workspace = true } num-traits = { version = "0.2", optional = true } -num_cpus = { workspace = true } object_store = { workspace = true } parking_lot = { workspace = true } parquet = { workspace = true, optional = true, default-features = true } paste = "1.0.15" -pin-project-lite = "^0.2.7" rand = { workspace = true } sqlparser = { workspace = true } tempfile = { workspace = true } @@ -140,14 +134,11 @@ zstd = { version = "0.13", optional = true, default-features = false } [dev-dependencies] arrow-buffer = { workspace = true } async-trait = { workspace = true } -bigdecimal = { workspace = true } criterion = { version = "0.5", features = ["async_tokio"] } -csv = "1.1.6" ctor = { workspace = true } datafusion-functions-window-common = { workspace = true } doc-comment = { workspace = true } env_logger = { workspace = true } -half = { workspace = true, default-features = true } paste = "^1.0" postgres-protocol = "0.6.4" postgres-types = { version = "0.2.4", features = ["derive", "with-chrono-0_4"] } diff --git a/datafusion/core/LICENSE.txt b/datafusion/core/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/core/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/core/NOTICE.txt b/datafusion/core/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/core/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index bcf803573cdf..82ee52d7b2e3 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -50,7 +50,8 @@ use arrow::datatypes::{DataType, Field}; use arrow_schema::{Schema, SchemaRef}; use datafusion_common::config::{CsvOptions, JsonOptions}; use datafusion_common::{ - plan_err, Column, DFSchema, DataFusionError, ParamValues, SchemaError, UnnestOptions, + exec_err, not_impl_err, plan_err, Column, DFSchema, DataFusionError, ParamValues, + SchemaError, UnnestOptions, }; use datafusion_expr::dml::InsertOp; use datafusion_expr::{case, is_null, lit, SortExpr}; @@ -869,16 +870,16 @@ impl DataFrame { for result in describe_record_batch.iter() { let array_ref = match result { Ok(df) => { - let batchs = df.clone().collect().await; - match batchs { - Ok(batchs) - if batchs.len() == 1 - && batchs[0] + let batches = df.clone().collect().await; + match batches { + Ok(batches) + if batches.len() == 1 + && batches[0] .column_by_name(field.name()) .is_some() => { let column = - batchs[0].column_by_name(field.name()).unwrap(); + batches[0].column_by_name(field.name()).unwrap(); if column.data_type().is_null() { Arc::new(StringArray::from(vec!["null"])) @@ -901,9 +902,7 @@ impl DataFrame { { Arc::new(StringArray::from(vec!["null"])) } - Err(other_err) => { - panic!("{other_err}") - } + Err(e) => return exec_err!("{}", e), }; array_datas.push(array_ref); } @@ -1564,10 +1563,10 @@ impl DataFrame { writer_options: Option, ) -> Result, DataFusionError> { if options.insert_op != InsertOp::Append { - return Err(DataFusionError::NotImplemented(format!( + return not_impl_err!( "{} is not implemented for DataFrame::write_csv.", options.insert_op - ))); + ); } let format = if let Some(csv_opts) = writer_options { @@ -1625,10 +1624,10 @@ impl DataFrame { writer_options: Option, ) -> Result, DataFusionError> { if options.insert_op != InsertOp::Append { - return Err(DataFusionError::NotImplemented(format!( + return not_impl_err!( "{} is not implemented for DataFrame::write_json.", options.insert_op - ))); + ); } let format = if let Some(json_opts) = writer_options { @@ -1671,7 +1670,7 @@ impl DataFrame { /// # } /// ``` pub fn with_column(self, name: &str, expr: Expr) -> Result { - let window_func_exprs = find_window_exprs(&[expr.clone()]); + let window_func_exprs = find_window_exprs(std::slice::from_ref(&expr)); let (window_fn_str, plan) = if window_func_exprs.is_empty() { (None, self.plan) diff --git a/datafusion/core/src/dataframe/parquet.rs b/datafusion/core/src/dataframe/parquet.rs index f90b35fde6ba..0af68783c41f 100644 --- a/datafusion/core/src/dataframe/parquet.rs +++ b/datafusion/core/src/dataframe/parquet.rs @@ -26,6 +26,7 @@ use super::{ }; use datafusion_common::config::TableParquetOptions; +use datafusion_common::not_impl_err; use datafusion_expr::dml::InsertOp; impl DataFrame { @@ -59,10 +60,10 @@ impl DataFrame { writer_options: Option, ) -> Result, DataFusionError> { if options.insert_op != InsertOp::Append { - return Err(DataFusionError::NotImplemented(format!( + return not_impl_err!( "{} is not implemented for DataFrame::write_parquet.", options.insert_op - ))); + ); } let format = if let Some(parquet_opts) = writer_options { diff --git a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs index 9f089c7c0cea..f3358bce7623 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/arrow_array_reader.rs @@ -60,7 +60,7 @@ pub struct AvroArrowArrayReader<'a, R: Read> { schema_lookup: BTreeMap, } -impl<'a, R: Read> AvroArrowArrayReader<'a, R> { +impl AvroArrowArrayReader<'_, R> { pub fn try_new( reader: R, schema: SchemaRef, diff --git a/datafusion/core/src/datasource/avro_to_arrow/reader.rs b/datafusion/core/src/datasource/avro_to_arrow/reader.rs index e6310cec7475..dbc24da46366 100644 --- a/datafusion/core/src/datasource/avro_to_arrow/reader.rs +++ b/datafusion/core/src/datasource/avro_to_arrow/reader.rs @@ -128,7 +128,7 @@ pub struct Reader<'a, R: Read> { batch_size: usize, } -impl<'a, R: Read> Reader<'a, R> { +impl Reader<'_, R> { /// Create a new Avro Reader from any value that implements the `Read` trait. /// /// If reading a `File`, you can customise the Reader, such as to enable schema @@ -157,7 +157,7 @@ impl<'a, R: Read> Reader<'a, R> { } } -impl<'a, R: Read> Iterator for Reader<'a, R> { +impl Iterator for Reader<'_, R> { type Item = ArrowResult; /// Returns the next batch of results (defined by `self.batch_size`), or `None` if there diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index fd0935c6e031..2697e5621af3 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -476,7 +476,7 @@ mod tests { .infer_schema( &state, &(store.clone() as Arc), - &[object_meta.clone()], + std::slice::from_ref(&object_meta), ) .await?; let actual_fields = inferred_schema @@ -515,7 +515,7 @@ mod tests { .infer_schema( &state, &(store.clone() as Arc), - &[object_meta.clone()], + std::slice::from_ref(&object_meta), ) .await; diff --git a/datafusion/core/src/datasource/file_format/avro.rs b/datafusion/core/src/datasource/file_format/avro.rs index 5190bdbe153a..f854b9506a64 100644 --- a/datafusion/core/src/datasource/file_format/avro.rs +++ b/datafusion/core/src/datasource/file_format/avro.rs @@ -25,8 +25,8 @@ use std::sync::Arc; use arrow::datatypes::Schema; use arrow::datatypes::SchemaRef; use async_trait::async_trait; +use datafusion_common::internal_err; use datafusion_common::parsers::CompressionTypeVariant; -use datafusion_common::DataFusionError; use datafusion_common::GetExt; use datafusion_common::DEFAULT_AVRO_EXTENSION; use datafusion_physical_expr::PhysicalExpr; @@ -105,9 +105,7 @@ impl FileFormat for AvroFormat { let ext = self.get_ext(); match file_compression_type.get_variant() { CompressionTypeVariant::UNCOMPRESSED => Ok(ext), - _ => Err(DataFusionError::Internal( - "Avro FileFormat does not support compression.".into(), - )), + _ => internal_err!("Avro FileFormat does not support compression."), } } diff --git a/datafusion/core/src/datasource/file_format/mod.rs b/datafusion/core/src/datasource/file_format/mod.rs index eb2a85367f80..f47e2107ade6 100644 --- a/datafusion/core/src/datasource/file_format/mod.rs +++ b/datafusion/core/src/datasource/file_format/mod.rs @@ -573,7 +573,9 @@ pub(crate) mod test_util { let store = Arc::new(LocalFileSystem::new()) as _; let meta = local_unpartitioned_file(format!("{store_root}/{file_name}")); - let file_schema = format.infer_schema(state, &store, &[meta.clone()]).await?; + let file_schema = format + .infer_schema(state, &store, std::slice::from_ref(&meta)) + .await?; let statistics = format .infer_stats(state, &store, file_schema.clone(), &meta) diff --git a/datafusion/core/src/datasource/file_format/options.rs b/datafusion/core/src/datasource/file_format/options.rs index e392515cacb1..95576f448c64 100644 --- a/datafusion/core/src/datasource/file_format/options.rs +++ b/datafusion/core/src/datasource/file_format/options.rs @@ -89,7 +89,7 @@ pub struct CsvReadOptions<'a> { pub file_sort_order: Vec>, } -impl<'a> Default for CsvReadOptions<'a> { +impl Default for CsvReadOptions<'_> { fn default() -> Self { Self::new() } @@ -243,7 +243,7 @@ pub struct ParquetReadOptions<'a> { pub file_sort_order: Vec>, } -impl<'a> Default for ParquetReadOptions<'a> { +impl Default for ParquetReadOptions<'_> { fn default() -> Self { Self { file_extension: DEFAULT_PARQUET_EXTENSION, @@ -323,7 +323,7 @@ pub struct ArrowReadOptions<'a> { pub table_partition_cols: Vec<(String, DataType)>, } -impl<'a> Default for ArrowReadOptions<'a> { +impl Default for ArrowReadOptions<'_> { fn default() -> Self { Self { schema: None, @@ -368,7 +368,7 @@ pub struct AvroReadOptions<'a> { pub table_partition_cols: Vec<(String, DataType)>, } -impl<'a> Default for AvroReadOptions<'a> { +impl Default for AvroReadOptions<'_> { fn default() -> Self { Self { schema: None, @@ -420,7 +420,7 @@ pub struct NdJsonReadOptions<'a> { pub file_sort_order: Vec>, } -impl<'a> Default for NdJsonReadOptions<'a> { +impl Default for NdJsonReadOptions<'_> { fn default() -> Self { Self { schema: None, diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index c1314bdb8641..1d08de172273 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -49,7 +49,7 @@ use datafusion_common::file_options::parquet_writer::ParquetWriterOptions; use datafusion_common::parsers::CompressionTypeVariant; use datafusion_common::stats::Precision; use datafusion_common::{ - internal_datafusion_err, not_impl_err, DataFusionError, GetExt, + internal_datafusion_err, internal_err, not_impl_err, DataFusionError, GetExt, DEFAULT_PARQUET_EXTENSION, }; use datafusion_common_runtime::SpawnedTask; @@ -323,9 +323,7 @@ impl FileFormat for ParquetFormat { let ext = self.get_ext(); match file_compression_type.get_variant() { CompressionTypeVariant::UNCOMPRESSED => Ok(ext), - _ => Err(DataFusionError::Internal( - "Parquet FileFormat does not support compression.".into(), - )), + _ => internal_err!("Parquet FileFormat does not support compression."), } } @@ -479,7 +477,7 @@ impl<'a> ObjectStoreFetch<'a> { } } -impl<'a> MetadataFetch for ObjectStoreFetch<'a> { +impl MetadataFetch for ObjectStoreFetch<'_> { fn fetch( &mut self, range: Range, diff --git a/datafusion/core/src/datasource/file_format/write/demux.rs b/datafusion/core/src/datasource/file_format/write/demux.rs index 71cf747c328d..115d1ea710f3 100644 --- a/datafusion/core/src/datasource/file_format/write/demux.rs +++ b/datafusion/core/src/datasource/file_format/write/demux.rs @@ -37,7 +37,7 @@ use datafusion_common::cast::{ as_boolean_array, as_date32_array, as_date64_array, as_int32_array, as_int64_array, as_string_array, as_string_view_array, }; -use datafusion_common::{exec_datafusion_err, DataFusionError}; +use datafusion_common::{exec_datafusion_err, not_impl_err, DataFusionError}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::TaskContext; @@ -438,10 +438,10 @@ fn compute_partition_keys_by_row<'a>( ) } _ => { - return Err(DataFusionError::NotImplemented(format!( + return not_impl_err!( "it is not yet supported to write to hive partitions with datatype {}", dtype - ))) + ) } } diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 04c64156b125..a601aec32f16 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -135,7 +135,7 @@ pub fn split_files( partitioned_files.sort_by(|a, b| a.path().cmp(b.path())); // effectively this is div with rounding up instead of truncating - let chunk_size = (partitioned_files.len() + n - 1) / n; + let chunk_size = partitioned_files.len().div_ceil(n); let mut chunks = Vec::with_capacity(n); let mut current_chunk = Vec::with_capacity(chunk_size); for file in partitioned_files.drain(..) { diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 68d219ef0e5e..def6504189bb 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -284,7 +284,7 @@ mod tests { let meta = local_unpartitioned_file(filename); let file_schema = AvroFormat {} - .infer_schema(&state, &store, &[meta.clone()]) + .infer_schema(&state, &store, std::slice::from_ref(&meta)) .await?; let avro_exec = AvroExec::new( @@ -349,7 +349,7 @@ mod tests { let object_store_url = ObjectStoreUrl::local_filesystem(); let meta = local_unpartitioned_file(filename); let actual_schema = AvroFormat {} - .infer_schema(&state, &object_store, &[meta.clone()]) + .infer_schema(&state, &object_store, std::slice::from_ref(&meta)) .await?; let mut builder = SchemaBuilder::from(actual_schema.fields()); @@ -422,7 +422,7 @@ mod tests { let object_store_url = ObjectStoreUrl::local_filesystem(); let meta = local_unpartitioned_file(filename); let file_schema = AvroFormat {} - .infer_schema(&state, &object_store, &[meta.clone()]) + .infer_schema(&state, &object_store, std::slice::from_ref(&meta)) .await?; let mut partitioned_file = PartitionedFile::from(meta); diff --git a/datafusion/core/src/datasource/physical_plan/file_groups.rs b/datafusion/core/src/datasource/physical_plan/file_groups.rs index 28f975ae193d..f9a19f1d9691 100644 --- a/datafusion/core/src/datasource/physical_plan/file_groups.rs +++ b/datafusion/core/src/datasource/physical_plan/file_groups.rs @@ -217,8 +217,7 @@ impl FileGroupPartitioner { return None; } - let target_partition_size = - (total_size as usize + (target_partitions) - 1) / (target_partitions); + let target_partition_size = (total_size as usize).div_ceil(target_partitions); let current_partition_index: usize = 0; let current_partition_size: usize = 0; diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index fd14be08c7a3..a5f2bd1760b3 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -1165,7 +1165,7 @@ mod tests { }) .collect::>() }) - .map_err(|e| e.to_string().leak() as &'static str); + .map_err(|e| e.strip_backtrace().leak() as &'static str); assert_eq!(results_by_name, case.expected_result, "{}", case.name); } diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index c86f8fbd262f..c07e8ca74543 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -430,7 +430,7 @@ mod tests { .object_meta; let schema = JsonFormat::default() .with_file_compression_type(file_compression_type.to_owned()) - .infer_schema(state, &store, &[meta.clone()]) + .infer_schema(state, &store, std::slice::from_ref(&meta)) .await .unwrap(); diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 2b50458bb581..449b7bb43519 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -139,7 +139,7 @@ impl DisplayAs for FileScanConfig { #[derive(Debug)] struct FileGroupsDisplay<'a>(&'a [Vec]); -impl<'a> DisplayAs for FileGroupsDisplay<'a> { +impl DisplayAs for FileGroupsDisplay<'_> { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { let n_groups = self.0.len(); let groups = if n_groups == 1 { "group" } else { "groups" }; @@ -171,7 +171,7 @@ impl<'a> DisplayAs for FileGroupsDisplay<'a> { #[derive(Debug)] pub(crate) struct FileGroupDisplay<'a>(pub &'a [PartitionedFile]); -impl<'a> DisplayAs for FileGroupDisplay<'a> { +impl DisplayAs for FileGroupDisplay<'_> { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> FmtResult { write!(f, "[")?; match t { diff --git a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs index 9dd0b9e206a9..446d5f531185 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/mod.rs @@ -1702,7 +1702,7 @@ mod tests { let store = Arc::new(LocalFileSystem::new()) as _; let file_schema = ParquetFormat::default() - .infer_schema(&state, &store, &[meta.clone()]) + .infer_schema(&state, &store, std::slice::from_ref(&meta)) .await?; let group_empty = vec![vec![file_range(&meta, 0, 2)]]; @@ -1734,7 +1734,7 @@ mod tests { let meta = local_unpartitioned_file(filename); let schema = ParquetFormat::default() - .infer_schema(&state, &store, &[meta.clone()]) + .infer_schema(&state, &store, std::slice::from_ref(&meta)) .await .unwrap(); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs index 07f50bca1d1d..4d0a8451a0d4 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/page_filter.rs @@ -449,7 +449,7 @@ impl<'a> PagesPruningStatistics<'a> { Some(vec) } } -impl<'a> PruningStatistics for PagesPruningStatistics<'a> { +impl PruningStatistics for PagesPruningStatistics<'_> { fn min_values(&self, _column: &datafusion_common::Column) -> Option { match self.converter.data_page_mins( self.column_index, diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs index a97e7c7d2552..af5ffb9d5743 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_filter.rs @@ -336,7 +336,7 @@ impl<'schema> PushdownChecker<'schema> { } } -impl<'schema> TreeNodeRewriter for PushdownChecker<'schema> { +impl TreeNodeRewriter for PushdownChecker<'_> { type Node = Arc; fn f_down( diff --git a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs index 7406676652f6..516310dc81ae 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/row_group_filter.rs @@ -374,7 +374,7 @@ impl<'a> RowGroupPruningStatistics<'a> { } } -impl<'a> PruningStatistics for RowGroupPruningStatistics<'a> { +impl PruningStatistics for RowGroupPruningStatistics<'_> { fn min_values(&self, column: &Column) -> Option { self.statistics_converter(column) .and_then(|c| Ok(c.row_group_mins(self.metadata_iter())?)) diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs index 488098e7861c..5e0257022e76 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/statistics.rs @@ -34,7 +34,7 @@ use arrow::{ }; use arrow_array::RecordBatch; use arrow_schema::SchemaRef; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -232,9 +232,7 @@ impl MinMaxStatistics { // check that sort columns are non-nullable if field.is_nullable() { - return Err(DataFusionError::Plan( - "cannot sort by nullable column".to_string(), - )); + return plan_err!("cannot sort by nullable column"); } Ok(SortColumn { diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 1469b671d6c7..4cc3200df17d 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -1765,7 +1765,7 @@ impl<'a> BadPlanVisitor<'a> { } } -impl<'n, 'a> TreeNodeVisitor<'n> for BadPlanVisitor<'a> { +impl<'n> TreeNodeVisitor<'n> for BadPlanVisitor<'_> { type Node = LogicalPlan; fn f_down(&mut self, node: &'n Self::Node) -> Result { diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 8f17ffb575aa..4ccad5ffd323 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -1649,7 +1649,7 @@ struct SessionContextProvider<'a> { tables: HashMap>, } -impl<'a> ContextProvider for SessionContextProvider<'a> { +impl ContextProvider for SessionContextProvider<'_> { fn get_expr_planners(&self) -> &[Arc] { &self.state.expr_planners } @@ -1944,7 +1944,7 @@ impl<'a> SessionSimplifyProvider<'a> { } } -impl<'a> SimplifyInfo for SessionSimplifyProvider<'a> { +impl SimplifyInfo for SessionSimplifyProvider<'_> { fn is_boolean_type(&self, expr: &Expr) -> datafusion_common::Result { Ok(expr.get_type(self.df_schema)? == DataType::Boolean) } diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 511aaacf3ef1..9d65c6ded423 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -123,7 +123,7 @@ fn supports_swap(join_type: JoinType) -> bool { /// This function returns the new join type we get after swapping the given /// join's inputs. -fn swap_join_type(join_type: JoinType) -> JoinType { +pub(crate) fn swap_join_type(join_type: JoinType) -> JoinType { match join_type { JoinType::Inner => JoinType::Inner, JoinType::Full => JoinType::Full, @@ -256,7 +256,7 @@ fn swap_nl_join(join: &NestedLoopJoinExec) -> Result> { /// the output should not be impacted. This function creates the expressions /// that will allow to swap back the values from the original left as the first /// columns and those on the right next. -fn swap_reverting_projection( +pub(crate) fn swap_reverting_projection( left_schema: &Schema, right_schema: &Schema, ) -> Vec<(Arc, String)> { @@ -278,7 +278,7 @@ fn swap_reverting_projection( } /// Swaps join sides for filter column indices and produces new JoinFilter -fn swap_filter(filter: &JoinFilter) -> JoinFilter { +pub(crate) fn swap_filter(filter: &JoinFilter) -> JoinFilter { let column_indices = filter .column_indices() .iter() @@ -736,6 +736,7 @@ mod tests_statistical { use arrow::datatypes::{DataType, Field}; use datafusion_common::{stats::Precision, JoinType, ScalarValue}; use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::PhysicalExprRef; @@ -1089,8 +1090,8 @@ mod tests_statistical { Arc::clone(&big), Arc::clone(&small), vec![( - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()), - Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()), + col("big_col", &big.schema()).unwrap(), + col("small_col", &small.schema()).unwrap(), )], None, &JoinType::Inner, @@ -1106,10 +1107,8 @@ mod tests_statistical { Arc::clone(&medium), Arc::new(child_join), vec![( - Arc::new( - Column::new_with_schema("medium_col", &medium.schema()).unwrap(), - ), - Arc::new(Column::new_with_schema("small_col", &child_schema).unwrap()), + col("medium_col", &medium.schema()).unwrap(), + col("small_col", &child_schema).unwrap(), )], None, &JoinType::Left, @@ -1421,8 +1420,8 @@ mod tests_statistical { )); let join_on = vec![( - Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, + col("small_col", &small.schema()).unwrap(), + col("big_col", &big.schema()).unwrap(), )]; check_join_partition_mode( small.clone(), @@ -1433,8 +1432,8 @@ mod tests_statistical { ); let join_on = vec![( - Arc::new(Column::new_with_schema("big_col", &big.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, + col("big_col", &big.schema()).unwrap(), + col("small_col", &small.schema()).unwrap(), )]; check_join_partition_mode( big, @@ -1445,8 +1444,8 @@ mod tests_statistical { ); let join_on = vec![( - Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("empty_col", &empty.schema()).unwrap()) as _, + col("small_col", &small.schema()).unwrap(), + col("empty_col", &empty.schema()).unwrap(), )]; check_join_partition_mode( small.clone(), @@ -1457,8 +1456,8 @@ mod tests_statistical { ); let join_on = vec![( - Arc::new(Column::new_with_schema("empty_col", &empty.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("small_col", &small.schema()).unwrap()) as _, + col("empty_col", &empty.schema()).unwrap(), + col("small_col", &small.schema()).unwrap(), )]; check_join_partition_mode( empty, @@ -1627,6 +1626,7 @@ mod hash_join_tests { use arrow::datatypes::{DataType, Field}; use arrow::record_batch::RecordBatch; + use datafusion_physical_expr::expressions::col; struct TestCase { case: String, @@ -1969,7 +1969,7 @@ mod hash_join_tests { false, )]))), 2, - )) as Arc; + )) as _; let right_exec = Arc::new(UnboundedExec::new( (!right_unbounded).then_some(1), RecordBatch::new_empty(Arc::new(Schema::new(vec![Field::new( @@ -1978,21 +1978,21 @@ mod hash_join_tests { false, )]))), 2, - )) as Arc; + )) as _; let join = Arc::new(HashJoinExec::try_new( Arc::clone(&left_exec), Arc::clone(&right_exec), vec![( - Arc::new(Column::new_with_schema("a", &left_exec.schema())?), - Arc::new(Column::new_with_schema("b", &right_exec.schema())?), + col("a", &left_exec.schema())?, + col("b", &right_exec.schema())?, )], None, &t.initial_join_type, None, t.initial_mode, false, - )?); + )?) as _; let optimized_join_plan = hash_join_swap_subrule(join, &ConfigOptions::new())?; diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 2c2ff6d48aec..3ac40bfb62ea 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -1382,7 +1382,11 @@ mod tests { Ok(DataType::Int32) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { unimplemented!("DummyUDF::invoke") } } diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 88bb0b6fef23..9156301393c0 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -17,6 +17,8 @@ //! Collection of testing utility functions that are leveraged by the query optimizer rules +#![allow(missing_docs)] + use std::any::Any; use std::fmt::Formatter; use std::sync::Arc; diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index 9ac75c8f3efb..d8304c2f0a86 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -17,6 +17,8 @@ //! Common unit test utility methods +#![allow(missing_docs)] + use std::any::Any; use std::fs::File; use std::io::prelude::*; diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs index af454bee7ce8..2aeecd8ff2ea 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs @@ -22,8 +22,8 @@ use datafusion::{ prelude::{SessionConfig, SessionContext}, }; use datafusion_catalog::TableProvider; -use datafusion_common::error::Result; use datafusion_common::ScalarValue; +use datafusion_common::{error::Result, utils::get_available_parallelism}; use datafusion_expr::col; use rand::{thread_rng, Rng}; @@ -73,7 +73,7 @@ impl SessionContextGenerator { ]; let max_batch_size = cmp::max(1, dataset_ref.total_rows_num); - let max_target_partitions = num_cpus::get(); + let max_target_partitions = get_available_parallelism(); Self { dataset: dataset_ref, diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 262f68079f3f..881949047bff 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -581,7 +581,11 @@ impl ScalarUDFImpl for TestScalarUDF { Ok(input[0].sort_properties) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; let arr: ArrayRef = match args[0].data_type() { diff --git a/datafusion/core/tests/parquet/external_access_plan.rs b/datafusion/core/tests/parquet/external_access_plan.rs index 96267eeff5a7..fa23f5c699e2 100644 --- a/datafusion/core/tests/parquet/external_access_plan.rs +++ b/datafusion/core/tests/parquet/external_access_plan.rs @@ -33,7 +33,8 @@ use datafusion_physical_plan::ExecutionPlan; use parquet::arrow::arrow_reader::{RowSelection, RowSelector}; use parquet::arrow::ArrowWriter; use parquet::file::properties::WriterProperties; -use std::sync::{Arc, OnceLock}; +use std::path::Path; +use std::sync::Arc; use tempfile::NamedTempFile; #[tokio::test] @@ -314,12 +315,19 @@ impl TestFull { let TestData { _temp_file: _, - schema, - file_name, - file_size, + ref schema, + ref file_name, + ref file_size, } = get_test_data(); - let mut partitioned_file = PartitionedFile::new(file_name, *file_size); + let new_file_name = if cfg!(target_os = "windows") { + // Windows path separator is different from Unix + file_name.replace("\\", "/") + } else { + file_name.clone() + }; + + let mut partitioned_file = PartitionedFile::new(new_file_name, *file_size); // add the access plan, if any, as an extension if let Some(access_plan) = access_plan { @@ -355,6 +363,8 @@ impl TestFull { pretty_format_batches(&results).unwrap() ); + std::fs::remove_file(file_name).unwrap(); + Ok(MetricsFinder::find_metrics(plan.as_ref()).unwrap()) } } @@ -369,45 +379,41 @@ struct TestData { file_size: u64, } -static TEST_DATA: OnceLock = OnceLock::new(); - /// Return a parquet file with 2 row groups each with 5 rows -fn get_test_data() -> &'static TestData { - TEST_DATA.get_or_init(|| { - let scenario = Scenario::UTF8; - let row_per_group = 5; +fn get_test_data() -> TestData { + let scenario = Scenario::UTF8; + let row_per_group = 5; - let mut temp_file = tempfile::Builder::new() - .prefix("user_access_plan") - .suffix(".parquet") - .tempfile() - .expect("tempfile creation"); + let mut temp_file = tempfile::Builder::new() + .prefix("user_access_plan") + .suffix(".parquet") + .tempfile_in(Path::new("")) + .expect("tempfile creation"); - let props = WriterProperties::builder() - .set_max_row_group_size(row_per_group) - .build(); + let props = WriterProperties::builder() + .set_max_row_group_size(row_per_group) + .build(); - let batches = create_data_batch(scenario); - let schema = batches[0].schema(); + let batches = create_data_batch(scenario); + let schema = batches[0].schema(); - let mut writer = - ArrowWriter::try_new(&mut temp_file, schema.clone(), Some(props)).unwrap(); + let mut writer = + ArrowWriter::try_new(&mut temp_file, schema.clone(), Some(props)).unwrap(); - for batch in batches { - writer.write(&batch).expect("writing batch"); - } - writer.close().unwrap(); + for batch in batches { + writer.write(&batch).expect("writing batch"); + } + writer.close().unwrap(); - let file_name = temp_file.path().to_string_lossy().to_string(); - let file_size = temp_file.path().metadata().unwrap().len(); + let file_name = temp_file.path().to_string_lossy().to_string(); + let file_size = temp_file.path().metadata().unwrap().len(); - TestData { - _temp_file: temp_file, - schema, - file_name, - file_size, - } - }) + TestData { + _temp_file: temp_file, + schema, + file_name, + file_size, + } } /// Return the total value of the specified metric name diff --git a/datafusion/core/tests/parquet/mod.rs b/datafusion/core/tests/parquet/mod.rs index cd298d1c5543..3f68222a2ce3 100644 --- a/datafusion/core/tests/parquet/mod.rs +++ b/datafusion/core/tests/parquet/mod.rs @@ -43,8 +43,6 @@ use std::sync::Arc; use tempfile::NamedTempFile; mod custom_reader; -// Don't run on windows as tempfiles don't seem to work the same -#[cfg(not(target_os = "windows"))] mod external_access_plan; mod file_statistics; #[cfg(not(target_family = "windows"))] diff --git a/datafusion/core/tests/parquet/page_pruning.rs b/datafusion/core/tests/parquet/page_pruning.rs index 6f03ff31696c..65bfd0340125 100644 --- a/datafusion/core/tests/parquet/page_pruning.rs +++ b/datafusion/core/tests/parquet/page_pruning.rs @@ -54,7 +54,7 @@ async fn get_parquet_exec(state: &SessionState, filter: Expr) -> ParquetExec { }; let schema = ParquetFormat::default() - .infer_schema(state, &store, &[meta.clone()]) + .infer_schema(state, &store, std::slice::from_ref(&meta)) .await .unwrap(); diff --git a/datafusion/core/tests/sql/mod.rs b/datafusion/core/tests/sql/mod.rs index 177427b47d21..82f73eadba8c 100644 --- a/datafusion/core/tests/sql/mod.rs +++ b/datafusion/core/tests/sql/mod.rs @@ -32,6 +32,7 @@ use datafusion::prelude::*; use datafusion::test_util; use datafusion::{assert_batches_eq, assert_batches_sorted_eq}; use datafusion::{execution::context::SessionContext, physical_plan::displayable}; +use datafusion_common::utils::get_available_parallelism; use datafusion_common::{assert_contains, assert_not_contains}; use object_store::path::Path; use std::fs::File; @@ -259,7 +260,7 @@ impl ExplainNormalizer { // convert things like partitioning=RoundRobinBatch(16) // to partitioning=RoundRobinBatch(NUM_CORES) - let needle = format!("RoundRobinBatch({})", num_cpus::get()); + let needle = format!("RoundRobinBatch({})", get_available_parallelism()); replacements.push((needle, "RoundRobinBatch(NUM_CORES)".to_string())); Self { replacements } diff --git a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs index cf403e5d640f..a59394f90814 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -520,10 +520,6 @@ impl ScalarUDFImpl for AddIndexToStringVolatileScalarUDF { Ok(self.return_type.clone()) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { - not_impl_err!("index_with_offset function does not accept arguments") - } - fn invoke_batch( &self, args: &[ColumnarValue], @@ -720,7 +716,11 @@ impl ScalarUDFImpl for CastToI64UDF { Ok(ExprSimplifyResult::Simplified(new_expr)) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { unimplemented!("Function should have been simplified prior to evaluation") } } @@ -848,7 +848,11 @@ impl ScalarUDFImpl for TakeUDF { } // The actual implementation - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let take_idx = match &args[2] { ColumnarValue::Scalar(ScalarValue::Int64(Some(v))) if v < &2 => *v as usize, _ => unreachable!(), @@ -956,7 +960,11 @@ impl ScalarUDFImpl for ScalarFunctionWrapper { Ok(self.return_type.clone()) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { internal_err!("This function should not get invoked!") } @@ -1240,7 +1248,11 @@ impl ScalarUDFImpl for MyRegexUdf { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args { [ColumnarValue::Scalar(ScalarValue::Utf8(value))] => { Ok(ColumnarValue::Scalar(ScalarValue::Boolean( diff --git a/datafusion/doc/Cargo.toml b/datafusion/doc/Cargo.toml new file mode 100644 index 000000000000..c188bcb2a535 --- /dev/null +++ b/datafusion/doc/Cargo.toml @@ -0,0 +1,35 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "datafusion-doc" +description = "Documentation module for DataFusion query engine" +keywords = ["datafusion", "query", "sql"] +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_doc" +path = "src/lib.rs" diff --git a/datafusion/expr/src/udf_docs.rs b/datafusion/doc/src/lib.rs similarity index 75% rename from datafusion/expr/src/udf_docs.rs rename to datafusion/doc/src/lib.rs index a124361e42a3..5bc986d07f8e 100644 --- a/datafusion/expr/src/udf_docs.rs +++ b/datafusion/doc/src/lib.rs @@ -15,15 +15,13 @@ // specific language governing permissions and limitations // under the License. -use datafusion_common::exec_err; -use datafusion_common::Result; - -/// Documentation for use by [`ScalarUDFImpl`](crate::ScalarUDFImpl), -/// [`AggregateUDFImpl`](crate::AggregateUDFImpl) and [`WindowUDFImpl`](crate::WindowUDFImpl) functions +#[allow(rustdoc::broken_intra_doc_links)] +/// Documentation for use by [`ScalarUDFImpl`](ScalarUDFImpl), +/// [`AggregateUDFImpl`](AggregateUDFImpl) and [`WindowUDFImpl`](WindowUDFImpl) functions /// that will be used to generate public documentation. /// -/// The name of the udf will be pulled from the [`ScalarUDFImpl::name`](crate::ScalarUDFImpl::name), -/// [`AggregateUDFImpl::name`](crate::AggregateUDFImpl::name) or [`WindowUDFImpl::name`](crate::WindowUDFImpl::name) +/// The name of the udf will be pulled from the [`ScalarUDFImpl::name`](ScalarUDFImpl::name), +/// [`AggregateUDFImpl::name`](AggregateUDFImpl::name) or [`WindowUDFImpl::name`](WindowUDFImpl::name) /// function as appropriate. /// /// All strings in the documentation are required to be @@ -58,8 +56,12 @@ pub struct Documentation { impl Documentation { /// Returns a new [`DocumentationBuilder`] with no options set. - pub fn builder() -> DocumentationBuilder { - DocumentationBuilder::new() + pub fn builder( + doc_section: DocSection, + description: impl Into, + syntax_example: impl Into, + ) -> DocumentationBuilder { + DocumentationBuilder::new(doc_section, description, syntax_example) } } @@ -79,23 +81,23 @@ pub struct DocSection { /// Example: /// /// ```rust -/// # use datafusion_expr::Documentation; -/// # use datafusion_expr::scalar_doc_sections::DOC_SECTION_MATH; -/// # use datafusion_common::Result; -/// # -/// # fn main() -> Result<()> { -/// let documentation = Documentation::builder() -/// .with_doc_section(DOC_SECTION_MATH) -/// .with_description("Add one to an int32") -/// .with_syntax_example("add_one(2)") +/// +/// # fn main() { +/// use datafusion_doc::{DocSection, Documentation}; +/// let doc_section = DocSection { +/// include: true, +/// label: "Display Label", +/// description: None, +/// }; +/// +/// let documentation = Documentation::builder(doc_section, "Add one to an int32".to_owned(), "add_one(2)".to_owned()) /// .with_argument("arg_1", "The int32 number to add one to") -/// .build()?; -/// Ok(()) +/// .build(); /// # } pub struct DocumentationBuilder { - pub doc_section: Option, - pub description: Option, - pub syntax_example: Option, + pub doc_section: DocSection, + pub description: String, + pub syntax_example: String, pub sql_example: Option, pub arguments: Option>, pub alternative_syntax: Option>, @@ -103,11 +105,15 @@ pub struct DocumentationBuilder { } impl DocumentationBuilder { - pub fn new() -> Self { + pub fn new( + doc_section: DocSection, + description: impl Into, + syntax_example: impl Into, + ) -> Self { Self { - doc_section: None, - description: None, - syntax_example: None, + doc_section, + description: description.into(), + syntax_example: syntax_example.into(), sql_example: None, arguments: None, alternative_syntax: None, @@ -116,17 +122,17 @@ impl DocumentationBuilder { } pub fn with_doc_section(mut self, doc_section: DocSection) -> Self { - self.doc_section = Some(doc_section); + self.doc_section = doc_section; self } pub fn with_description(mut self, description: impl Into) -> Self { - self.description = Some(description.into()); + self.description = description.into(); self } pub fn with_syntax_example(mut self, syntax_example: impl Into) -> Self { - self.syntax_example = Some(syntax_example.into()); + self.syntax_example = syntax_example.into(); self } @@ -190,7 +196,10 @@ impl DocumentationBuilder { self } - pub fn build(self) -> Result { + /// Build the documentation from provided components + /// + /// Panics if `doc_section`, `description` or `syntax_example` is not set + pub fn build(self) -> Documentation { let Self { doc_section, description, @@ -201,30 +210,14 @@ impl DocumentationBuilder { related_udfs, } = self; - if doc_section.is_none() { - return exec_err!("Documentation must have a doc section"); - } - if description.is_none() { - return exec_err!("Documentation must have a description"); - } - if syntax_example.is_none() { - return exec_err!("Documentation must have a syntax_example"); - } - - Ok(Documentation { - doc_section: doc_section.unwrap(), - description: description.unwrap(), - syntax_example: syntax_example.unwrap(), + Documentation { + doc_section, + description, + syntax_example, sql_example, arguments, alternative_syntax, related_udfs, - }) - } -} - -impl Default for DocumentationBuilder { - fn default() -> Self { - Self::new() + } } } diff --git a/datafusion/execution/Cargo.toml b/datafusion/execution/Cargo.toml index fb2e7e914fe5..ac1eb729b6ff 100644 --- a/datafusion/execution/Cargo.toml +++ b/datafusion/execution/Cargo.toml @@ -42,7 +42,6 @@ dashmap = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } futures = { workspace = true } -hashbrown = { workspace = true } log = { workspace = true } object_store = { workspace = true } parking_lot = { workspace = true } diff --git a/datafusion/execution/LICENSE.txt b/datafusion/execution/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/execution/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/execution/NOTICE.txt b/datafusion/execution/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/execution/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/execution/src/cache/mod.rs b/datafusion/execution/src/cache/mod.rs index da19bff5658a..4271bebd0b32 100644 --- a/datafusion/execution/src/cache/mod.rs +++ b/datafusion/execution/src/cache/mod.rs @@ -22,7 +22,6 @@ pub mod cache_unit; /// This interface does not get `mut` references and thus has to handle its own /// locking via internal mutability. It can be accessed via multiple concurrent queries /// during planning and execution. - pub trait CacheAccessor: Send + Sync { // Extra info but not part of the cache key or cache value. type Extra: Clone; diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index 5bf30b724d0b..45d467f133bf 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -23,7 +23,9 @@ use std::{cmp::Ordering, sync::Arc}; mod pool; pub mod proxy { - pub use datafusion_common::utils::proxy::{RawTableAllocExt, VecAllocExt}; + pub use datafusion_common::utils::proxy::{ + HashTableAllocExt, RawTableAllocExt, VecAllocExt, + }; } pub use pool::*; diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 4022eb07de0c..d302452f7573 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -66,7 +66,7 @@ impl Debug for RuntimeEnv { } impl RuntimeEnv { - #[deprecated(note = "please use `try_new` instead")] + #[deprecated(since = "43.0.0", note = "please use `try_new` instead")] pub fn new(config: RuntimeConfig) -> Result { Self::try_new(config) } diff --git a/datafusion/expr-common/LICENSE.txt b/datafusion/expr-common/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/expr-common/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/expr-common/NOTICE.txt b/datafusion/expr-common/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/expr-common/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/expr-common/src/signature.rs b/datafusion/expr-common/src/signature.rs index 0fffd84b7047..32cbb6d0aecb 100644 --- a/datafusion/expr-common/src/signature.rs +++ b/datafusion/expr-common/src/signature.rs @@ -18,7 +18,7 @@ //! Signature module contains foundational types that are used to represent signatures, types, //! and return types of functions in DataFusion. -use crate::type_coercion::aggregates::{NUMERICS, STRINGS}; +use crate::type_coercion::aggregates::NUMERICS; use arrow::datatypes::DataType; use datafusion_common::types::{LogicalTypeRef, NativeType}; use itertools::Itertools; @@ -113,6 +113,15 @@ pub enum TypeSignature { /// arguments like `vec![DataType::Int32]` or `vec![DataType::Float32]` /// since i32 and f32 can be casted to f64 Coercible(Vec), + /// The arguments will be coerced to a single type based on the comparison rules. + /// For example, i32 and i64 has coerced type Int64. + /// + /// Note: + /// - If compares with numeric and string, numeric is preferred for numeric string cases. For example, nullif('2', 1) has coerced types Int64. + /// - If the result is Null, it will be coerced to String (Utf8View). + /// + /// See `comparison_coercion_numeric` for more details. + Comparable(usize), /// Fixed number of arguments of arbitrary types, number should be larger than 0 Any(usize), /// Matches exactly one of a list of [`TypeSignature`]s. Coercion is attempted to match @@ -131,13 +140,20 @@ pub enum TypeSignature { Numeric(usize), /// Fixed number of arguments of all the same string types. /// The precedence of type from high to low is Utf8View, LargeUtf8 and Utf8. - /// Null is considerd as `Utf8` by default + /// Null is considered as `Utf8` by default /// Dictionary with string value type is also handled. String(usize), /// Zero argument NullAry, } +impl TypeSignature { + #[inline] + pub fn is_one_of(&self) -> bool { + matches!(self, TypeSignature::OneOf(_)) + } +} + #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum ArrayFunctionSignature { /// Specialized Signature for ArrayAppend and similar functions @@ -210,6 +226,9 @@ impl TypeSignature { TypeSignature::Numeric(num) => { vec![format!("Numeric({num})")] } + TypeSignature::Comparable(num) => { + vec![format!("Comparable({num})")] + } TypeSignature::Coercible(types) => { vec![Self::join_types(types, ", ")] } @@ -284,13 +303,13 @@ impl TypeSignature { .cloned() .map(|numeric_type| vec![numeric_type; *arg_count]) .collect(), - TypeSignature::String(arg_count) => STRINGS - .iter() - .cloned() - .map(|string_type| vec![string_type; *arg_count]) - .collect(), + TypeSignature::String(arg_count) => get_data_types(&NativeType::String) + .into_iter() + .map(|dt| vec![dt; *arg_count]) + .collect::>(), // TODO: Implement for other types TypeSignature::Any(_) + | TypeSignature::Comparable(_) | TypeSignature::NullAry | TypeSignature::VariadicAny | TypeSignature::ArraySignature(_) @@ -412,6 +431,14 @@ impl Signature { } } + /// Used for function that expects comparable data types, it will try to coerced all the types into single final one. + pub fn comparable(arg_count: usize, volatility: Volatility) -> Self { + Self { + type_signature: TypeSignature::Comparable(arg_count), + volatility, + } + } + pub fn nullary(volatility: Volatility) -> Self { Signature { type_signature: TypeSignature::NullAry, diff --git a/datafusion/expr-common/src/type_coercion/aggregates.rs b/datafusion/expr-common/src/type_coercion/aggregates.rs index 384d688cc27e..13d52959aba6 100644 --- a/datafusion/expr-common/src/type_coercion/aggregates.rs +++ b/datafusion/expr-common/src/type_coercion/aggregates.rs @@ -294,19 +294,19 @@ pub fn coerce_avg_type(func_name: &str, arg_types: &[DataType]) -> Result Result { - return match &data_type { + match &data_type { DataType::Decimal128(p, s) => Ok(DataType::Decimal128(*p, *s)), DataType::Decimal256(p, s) => Ok(DataType::Decimal256(*p, *s)), d if d.is_numeric() => Ok(DataType::Float64), - DataType::Dictionary(_, v) => return coerced_type(func_name, v.as_ref()), + DataType::Dictionary(_, v) => coerced_type(func_name, v.as_ref()), _ => { - return plan_err!( + plan_err!( "The function {:?} does not support inputs of type {:?}.", func_name, data_type ) } - }; + } } Ok(vec![coerced_type(func_name, &arg_types[0])?]) } diff --git a/datafusion/expr-common/src/type_coercion/binary.rs b/datafusion/expr-common/src/type_coercion/binary.rs index bff74252df7b..4b2f3b5e46b5 100644 --- a/datafusion/expr-common/src/type_coercion/binary.rs +++ b/datafusion/expr-common/src/type_coercion/binary.rs @@ -28,6 +28,7 @@ use arrow::datatypes::{ DataType, Field, FieldRef, Fields, TimeUnit, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION, DECIMAL256_MAX_SCALE, }; +use datafusion_common::types::NativeType; use datafusion_common::{ exec_datafusion_err, exec_err, internal_err, plan_datafusion_err, plan_err, Result, }; @@ -641,6 +642,21 @@ pub fn comparison_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Option { + if lhs_type == rhs_type { + // same type => equality is possible + return Some(lhs_type.clone()); + } + binary_numeric_coercion(lhs_type, rhs_type) + .or_else(|| string_coercion(lhs_type, rhs_type)) + .or_else(|| null_coercion(lhs_type, rhs_type)) + .or_else(|| string_numeric_coercion_as_numeric(lhs_type, rhs_type)) +} + /// Coerce `lhs_type` and `rhs_type` to a common type for the purposes of a comparison operation /// where one is numeric and one is `Utf8`/`LargeUtf8`. fn string_numeric_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option { @@ -654,6 +670,24 @@ fn string_numeric_coercion(lhs_type: &DataType, rhs_type: &DataType) -> Option Option { + let lhs_logical_type = NativeType::from(lhs_type); + let rhs_logical_type = NativeType::from(rhs_type); + if lhs_logical_type.is_numeric() && rhs_logical_type == NativeType::String { + return Some(lhs_type.to_owned()); + } + if rhs_logical_type.is_numeric() && lhs_logical_type == NativeType::String { + return Some(rhs_type.to_owned()); + } + + None +} + /// Coerce `lhs_type` and `rhs_type` to a common type for the purposes of a comparison operation /// where one is temporal and one is `Utf8View`/`Utf8`/`LargeUtf8`. /// diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 19cd5ed3158b..438662e0642b 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -44,6 +44,7 @@ arrow-array = { workspace = true } arrow-buffer = { workspace = true } chrono = { workspace = true } datafusion-common = { workspace = true } +datafusion-doc = { workspace = true } datafusion-expr-common = { workspace = true } datafusion-functions-aggregate-common = { workspace = true } datafusion-functions-window-common = { workspace = true } diff --git a/datafusion/expr/LICENSE.txt b/datafusion/expr/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/expr/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/expr/NOTICE.txt b/datafusion/expr/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/expr/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 8490c08a70bb..c495b5396f53 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1118,7 +1118,7 @@ impl Expr { } /// Returns a full and complete string representation of this expression. - #[deprecated(note = "use format! instead")] + #[deprecated(since = "42.0.0", note = "use format! instead")] pub fn canonical_name(&self) -> String { format!("{self}") } @@ -1852,7 +1852,7 @@ macro_rules! expr_vec_fmt { } struct SchemaDisplay<'a>(&'a Expr); -impl<'a> Display for SchemaDisplay<'a> { +impl Display for SchemaDisplay<'_> { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { match self.0 { // The same as Display @@ -2389,7 +2389,7 @@ mod test { use crate::expr_fn::col; use crate::{ case, lit, qualified_wildcard, wildcard, wildcard_with_options, ColumnarValue, - ScalarUDF, ScalarUDFImpl, Volatility, + ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl, Volatility, }; use sqlparser::ast; use sqlparser::ast::{Ident, IdentWithAlias}; @@ -2518,7 +2518,10 @@ mod test { Ok(DataType::Utf8) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_with_args( + &self, + _args: ScalarFunctionArgs, + ) -> Result { Ok(ColumnarValue::Scalar(ScalarValue::from("a"))) } } diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index 7fd4e64e0e62..681eb3c0afd5 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -434,10 +434,24 @@ impl SimpleScalarUDF { volatility: Volatility, fun: ScalarFunctionImplementation, ) -> Self { - let name = name.into(); - let signature = Signature::exact(input_types, volatility); - Self { + Self::new_with_signature( name, + Signature::exact(input_types, volatility), + return_type, + fun, + ) + } + + /// Create a new `SimpleScalarUDF` from a name, signature, return type and + /// implementation. Implementing [`ScalarUDFImpl`] allows more flexibility + pub fn new_with_signature( + name: impl Into, + signature: Signature, + return_type: DataType, + fun: ScalarFunctionImplementation, + ) -> Self { + Self { + name: name.into(), signature, return_type, fun, @@ -462,7 +476,11 @@ impl ScalarUDFImpl for SimpleScalarUDF { Ok(self.return_type.clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { (self.fun)(args) } } @@ -515,7 +533,7 @@ impl Debug for SimpleAggregateUDF { } impl SimpleAggregateUDF { - /// Create a new `AggregateUDFImpl` from a name, input types, return type, state type and + /// Create a new `SimpleAggregateUDF` from a name, input types, return type, state type and /// implementation. Implementing [`AggregateUDFImpl`] allows more flexibility pub fn new( name: impl Into, @@ -536,6 +554,8 @@ impl SimpleAggregateUDF { } } + /// Create a new `SimpleAggregateUDF` from a name, signature, return type, state type and + /// implementation. Implementing [`AggregateUDFImpl`] allows more flexibility pub fn new_with_signature( name: impl Into, signature: Signature, diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index 27b2d71b1f42..c5016a18d443 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -34,7 +34,6 @@ mod partition_evaluator; mod table_source; mod udaf; mod udf; -mod udf_docs; mod udwf; pub mod conditional_expressions; @@ -66,6 +65,7 @@ pub mod var_provider; pub mod window_frame; pub mod window_state; +pub use datafusion_doc::{DocSection, Documentation, DocumentationBuilder}; pub use datafusion_expr_common::accumulator::Accumulator; pub use datafusion_expr_common::columnar_value::ColumnarValue; pub use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; @@ -92,8 +92,7 @@ pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; pub use udaf::{ aggregate_doc_sections, AggregateUDF, AggregateUDFImpl, ReversedUDAF, StatisticsArgs, }; -pub use udf::{scalar_doc_sections, ScalarUDF, ScalarUDFImpl}; -pub use udf_docs::{DocSection, Documentation, DocumentationBuilder}; +pub use udf::{scalar_doc_sections, ScalarFunctionArgs, ScalarUDF, ScalarUDFImpl}; pub use udwf::{window_doc_sections, ReversedUDWF, WindowUDF, WindowUDFImpl}; pub use window_frame::{WindowFrame, WindowFrameBound, WindowFrameUnits}; diff --git a/datafusion/expr/src/logical_plan/ddl.rs b/datafusion/expr/src/logical_plan/ddl.rs index 8c64a017988e..3efaf296c29c 100644 --- a/datafusion/expr/src/logical_plan/ddl.rs +++ b/datafusion/expr/src/logical_plan/ddl.rs @@ -125,7 +125,7 @@ impl DdlStatement { /// See [crate::LogicalPlan::display] for an example pub fn display(&self) -> impl Display + '_ { struct Wrapper<'a>(&'a DdlStatement); - impl<'a> Display for Wrapper<'a> { + impl Display for Wrapper<'_> { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self.0 { DdlStatement::CreateExternalTable(CreateExternalTable { diff --git a/datafusion/expr/src/logical_plan/display.rs b/datafusion/expr/src/logical_plan/display.rs index b808defcb959..14758b61e859 100644 --- a/datafusion/expr/src/logical_plan/display.rs +++ b/datafusion/expr/src/logical_plan/display.rs @@ -59,7 +59,7 @@ impl<'a, 'b> IndentVisitor<'a, 'b> { } } -impl<'n, 'a, 'b> TreeNodeVisitor<'n> for IndentVisitor<'a, 'b> { +impl<'n> TreeNodeVisitor<'n> for IndentVisitor<'_, '_> { type Node = LogicalPlan; fn f_down( @@ -113,7 +113,7 @@ impl<'n, 'a, 'b> TreeNodeVisitor<'n> for IndentVisitor<'a, 'b> { pub fn display_schema(schema: &Schema) -> impl fmt::Display + '_ { struct Wrapper<'a>(&'a Schema); - impl<'a> fmt::Display for Wrapper<'a> { + impl fmt::Display for Wrapper<'_> { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "[")?; for (idx, field) in self.0.fields().iter().enumerate() { @@ -181,7 +181,7 @@ impl<'a, 'b> GraphvizVisitor<'a, 'b> { } } -impl<'n, 'a, 'b> TreeNodeVisitor<'n> for GraphvizVisitor<'a, 'b> { +impl<'n> TreeNodeVisitor<'n> for GraphvizVisitor<'_, '_> { type Node = LogicalPlan; fn f_down( @@ -654,7 +654,7 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { } } -impl<'n, 'a, 'b> TreeNodeVisitor<'n> for PgJsonVisitor<'a, 'b> { +impl<'n> TreeNodeVisitor<'n> for PgJsonVisitor<'_, '_> { type Node = LogicalPlan; fn f_down( diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index e9f4f1f80972..e2590a9f255c 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -1536,7 +1536,7 @@ impl LogicalPlan { // Boilerplate structure to wrap LogicalPlan with something // that that can be formatted struct Wrapper<'a>(&'a LogicalPlan); - impl<'a> Display for Wrapper<'a> { + impl Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let with_schema = false; let mut visitor = IndentVisitor::new(f, with_schema); @@ -1579,7 +1579,7 @@ impl LogicalPlan { // Boilerplate structure to wrap LogicalPlan with something // that that can be formatted struct Wrapper<'a>(&'a LogicalPlan); - impl<'a> Display for Wrapper<'a> { + impl Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let with_schema = true; let mut visitor = IndentVisitor::new(f, with_schema); @@ -1599,7 +1599,7 @@ impl LogicalPlan { // Boilerplate structure to wrap LogicalPlan with something // that that can be formatted struct Wrapper<'a>(&'a LogicalPlan); - impl<'a> Display for Wrapper<'a> { + impl Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut visitor = PgJsonVisitor::new(f); visitor.with_schema(true); @@ -1645,7 +1645,7 @@ impl LogicalPlan { // Boilerplate structure to wrap LogicalPlan with something // that that can be formatted struct Wrapper<'a>(&'a LogicalPlan); - impl<'a> Display for Wrapper<'a> { + impl Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut visitor = GraphvizVisitor::new(f); @@ -1696,7 +1696,7 @@ impl LogicalPlan { // Boilerplate structure to wrap LogicalPlan with something // that that can be formatted struct Wrapper<'a>(&'a LogicalPlan); - impl<'a> Display for Wrapper<'a> { + impl Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { match self.0 { LogicalPlan::EmptyRelation(_) => write!(f, "EmptyRelation"), diff --git a/datafusion/expr/src/logical_plan/statement.rs b/datafusion/expr/src/logical_plan/statement.rs index 26df379f5e4a..a8b53e8a1fb1 100644 --- a/datafusion/expr/src/logical_plan/statement.rs +++ b/datafusion/expr/src/logical_plan/statement.rs @@ -85,7 +85,7 @@ impl Statement { /// See [crate::LogicalPlan::display] for an example pub fn display(&self) -> impl Display + '_ { struct Wrapper<'a>(&'a Statement); - impl<'a> Display for Wrapper<'a> { + impl Display for Wrapper<'_> { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self.0 { Statement::TransactionStart(TransactionStart { diff --git a/datafusion/expr/src/simplify.rs b/datafusion/expr/src/simplify.rs index e636fabf10fb..467ce8bf53e2 100644 --- a/datafusion/expr/src/simplify.rs +++ b/datafusion/expr/src/simplify.rs @@ -71,7 +71,7 @@ impl<'a> SimplifyContext<'a> { } } -impl<'a> SimplifyInfo for SimplifyContext<'a> { +impl SimplifyInfo for SimplifyContext<'_> { /// Returns true if this Expr has boolean type fn is_boolean_type(&self, expr: &Expr) -> Result { if let Some(schema) = &self.schema { diff --git a/datafusion/expr/src/table_source.rs b/datafusion/expr/src/table_source.rs index bdb602d48dee..e9a677de50c1 100644 --- a/datafusion/expr/src/table_source.rs +++ b/datafusion/expr/src/table_source.rs @@ -99,7 +99,7 @@ pub trait TableSource: Sync + Send { } /// Tests whether the table provider can make use of any or all filter expressions - /// to optimise data retrieval. + /// to optimise data retrieval. Only non-volatile expressions are passed to this function. fn supports_filters_pushdown( &self, filters: &[&Expr], diff --git a/datafusion/expr/src/type_coercion/functions.rs b/datafusion/expr/src/type_coercion/functions.rs index 6836713d8016..5f52c7ccc20e 100644 --- a/datafusion/expr/src/type_coercion/functions.rs +++ b/datafusion/expr/src/type_coercion/functions.rs @@ -29,7 +29,7 @@ use datafusion_common::{ }; use datafusion_expr_common::{ signature::{ArrayFunctionSignature, FIXED_SIZE_LIST_WILDCARD, TIMEZONE_WILDCARD}, - type_coercion::binary::string_coercion, + type_coercion::binary::{comparison_coercion_numeric, string_coercion}, }; use std::sync::Arc; @@ -182,6 +182,7 @@ fn is_well_supported_signature(type_signature: &TypeSignature) -> bool { | TypeSignature::Coercible(_) | TypeSignature::Any(_) | TypeSignature::NullAry + | TypeSignature::Comparable(_) ) } @@ -194,13 +195,18 @@ fn try_coerce_types( // Well-supported signature that returns exact valid types. if !valid_types.is_empty() && is_well_supported_signature(type_signature) { - // exact valid types - assert_eq!(valid_types.len(), 1); + // There may be many valid types if valid signature is OneOf + // Otherwise, there should be only one valid type + if !type_signature.is_one_of() { + assert_eq!(valid_types.len(), 1); + } + let valid_types = valid_types.swap_remove(0); if let Some(t) = maybe_data_types_without_coercion(&valid_types, current_types) { return Ok(t); } } else { + // TODO: Deprecate this branch after all signatures are well-supported (aka coercion has happened already) // Try and coerce the argument types to match the signature, returning the // coerced types from the first matching signature. for valid_types in valid_types { @@ -515,6 +521,23 @@ fn get_valid_types( vec![vec![valid_type; *number]] } + TypeSignature::Comparable(num) => { + function_length_check(current_types.len(), *num)?; + let mut target_type = current_types[0].to_owned(); + for data_type in current_types.iter().skip(1) { + if let Some(dt) = comparison_coercion_numeric(&target_type, data_type) { + target_type = dt; + } else { + return plan_err!("{target_type} and {data_type} is not comparable"); + } + } + // Convert null to String type. + if target_type.is_null() { + vec![vec![DataType::Utf8View; *num]] + } else { + vec![vec![target_type; *num]] + } + } TypeSignature::Coercible(target_types) => { function_length_check(current_types.len(), target_types.len())?; diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index dbbf88447ba3..56c9822495f8 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -333,13 +333,9 @@ where /// /// fn get_doc() -> &'static Documentation { /// DOCUMENTATION.get_or_init(|| { -/// Documentation::builder() -/// .with_doc_section(DOC_SECTION_AGGREGATE) -/// .with_description("calculates a geometric mean") -/// .with_syntax_example("geo_mean(2.0)") +/// Documentation::builder(DOC_SECTION_AGGREGATE, "calculates a geometric mean", "geo_mean(2.0)") /// .with_argument("arg1", "The Float64 number for the geometric mean") /// .build() -/// .unwrap() /// }) /// } /// diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 1a5d50477b1c..bf9c9f407eff 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -203,10 +203,7 @@ impl ScalarUDF { self.inner.simplify(args, info) } - /// Invoke the function on `args`, returning the appropriate result. - /// - /// See [`ScalarUDFImpl::invoke`] for more details. - #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] + #[deprecated(since = "42.1.0", note = "Use `invoke_with_args` instead")] pub fn invoke(&self, args: &[ColumnarValue]) -> Result { #[allow(deprecated)] self.inner.invoke(args) @@ -216,9 +213,6 @@ impl ScalarUDF { self.inner.is_nullable(args, schema) } - /// Invoke the function with `args` and number of rows, returning the appropriate result. - /// - /// See [`ScalarUDFImpl::invoke_batch`] for more details. pub fn invoke_batch( &self, args: &[ColumnarValue], @@ -227,9 +221,17 @@ impl ScalarUDF { self.inner.invoke_batch(args, number_rows) } + /// Invoke the function on `args`, returning the appropriate result. + /// + /// See [`ScalarUDFImpl::invoke_with_args`] for details. + pub fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + self.inner.invoke_with_args(args) + } + /// Invoke the function without `args` but number of rows, returning the appropriate result. /// - /// See [`ScalarUDFImpl::invoke_no_args`] for more details. + /// Note: This method is deprecated and will be removed in future releases. + /// User defined functions should implement [`Self::invoke_with_args`] instead. #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] pub fn invoke_no_args(&self, number_rows: usize) -> Result { #[allow(deprecated)] @@ -317,14 +319,24 @@ impl ScalarUDF { impl From for ScalarUDF where - F: ScalarUDFImpl + Send + Sync + 'static, + F: ScalarUDFImpl + 'static, { fn from(fun: F) -> Self { Self::new_from_impl(fun) } } -/// Trait for implementing [`ScalarUDF`]. +pub struct ScalarFunctionArgs<'a> { + // The evaluated arguments to the function + pub args: &'a [ColumnarValue], + // The number of rows in record batch being evaluated + pub number_rows: usize, + // The return type of the scalar function returned (from `return_type` or `return_type_from_exprs`) + // when creating the physical expression from the logical expression + pub return_type: &'a DataType, +} + +/// Trait for implementing user defined scalar functions. /// /// This trait exposes the full API for implementing user defined functions and /// can be used to implement any function. @@ -332,18 +344,19 @@ where /// See [`advanced_udf.rs`] for a full example with complete implementation and /// [`ScalarUDF`] for other available options. /// -/// /// [`advanced_udf.rs`]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_udf.rs +/// /// # Basic Example /// ``` /// # use std::any::Any; /// # use std::sync::OnceLock; /// # use arrow::datatypes::DataType; /// # use datafusion_common::{DataFusionError, plan_err, Result}; -/// # use datafusion_expr::{col, ColumnarValue, Documentation, Signature, Volatility}; +/// # use datafusion_expr::{col, ColumnarValue, Documentation, ScalarFunctionArgs, Signature, Volatility}; /// # use datafusion_expr::{ScalarUDFImpl, ScalarUDF}; /// # use datafusion_expr::scalar_doc_sections::DOC_SECTION_MATH; /// +/// /// This struct for a simple UDF that adds one to an int32 /// #[derive(Debug)] /// struct AddOne { /// signature: Signature, @@ -356,18 +369,14 @@ where /// } /// } /// } -/// +/// /// static DOCUMENTATION: OnceLock = OnceLock::new(); /// /// fn get_doc() -> &'static Documentation { /// DOCUMENTATION.get_or_init(|| { -/// Documentation::builder() -/// .with_doc_section(DOC_SECTION_MATH) -/// .with_description("Add one to an int32") -/// .with_syntax_example("add_one(2)") +/// Documentation::builder(DOC_SECTION_MATH, "Add one to an int32", "add_one(2)") /// .with_argument("arg1", "The int32 number to add one to") /// .build() -/// .unwrap() /// }) /// } /// @@ -383,7 +392,9 @@ where /// Ok(DataType::Int32) /// } /// // The actual implementation would add one to the argument -/// fn invoke(&self, args: &[ColumnarValue]) -> Result { unimplemented!() } +/// fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { +/// unimplemented!() +/// } /// fn documentation(&self) -> Option<&Documentation> { /// Some(get_doc()) /// } @@ -479,24 +490,9 @@ pub trait ScalarUDFImpl: Debug + Send + Sync { /// Invoke the function on `args`, returning the appropriate result /// - /// The function will be invoked passed with the slice of [`ColumnarValue`] - /// (either scalar or array). - /// - /// If the function does not take any arguments, please use [invoke_no_args] - /// instead and return [not_impl_err] for this function. - /// - /// - /// # Performance - /// - /// For the best performance, the implementations of `invoke` should handle - /// the common case when one or more of their arguments are constant values - /// (aka [`ColumnarValue::Scalar`]). - /// - /// [`ColumnarValue::values_to_arrays`] can be used to convert the arguments - /// to arrays, which will likely be simpler code, but be slower. - /// - /// [invoke_no_args]: ScalarUDFImpl::invoke_no_args - #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] + /// Note: This method is deprecated and will be removed in future releases. + /// User defined functions should implement [`Self::invoke_with_args`] instead. + #[deprecated(since = "42.1.0", note = "Use `invoke_with_args` instead")] fn invoke(&self, _args: &[ColumnarValue]) -> Result { not_impl_err!( "Function {} does not implement invoke but called", @@ -507,17 +503,12 @@ pub trait ScalarUDFImpl: Debug + Send + Sync { /// Invoke the function with `args` and the number of rows, /// returning the appropriate result. /// - /// The function will be invoked with the slice of [`ColumnarValue`] - /// (either scalar or array). - /// - /// # Performance + /// Note: See notes on [`Self::invoke_with_args`] /// - /// For the best performance, the implementations should handle the common case - /// when one or more of their arguments are constant values (aka - /// [`ColumnarValue::Scalar`]). + /// Note: This method is deprecated and will be removed in future releases. + /// User defined functions should implement [`Self::invoke_with_args`] instead. /// - /// [`ColumnarValue::values_to_arrays`] can be used to convert the arguments - /// to arrays, which will likely be simpler code, but be slower. + /// See for more details. fn invoke_batch( &self, args: &[ColumnarValue], @@ -537,9 +528,26 @@ pub trait ScalarUDFImpl: Debug + Send + Sync { } } + /// Invoke the function returning the appropriate result. + /// + /// # Performance + /// + /// For the best performance, the implementations should handle the common case + /// when one or more of their arguments are constant values (aka + /// [`ColumnarValue::Scalar`]). + /// + /// [`ColumnarValue::values_to_arrays`] can be used to convert the arguments + /// to arrays, which will likely be simpler code, but be slower. + fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { + self.invoke_batch(args.args, args.number_rows) + } + /// Invoke the function without `args`, instead the number of rows are provided, /// returning the appropriate result. - #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] + /// + /// Note: This method is deprecated and will be removed in future releases. + /// User defined functions should implement [`Self::invoke_with_args`] instead. + #[deprecated(since = "42.1.0", note = "Use `invoke_with_args` instead")] fn invoke_no_args(&self, _number_rows: usize) -> Result { not_impl_err!( "Function {} does not implement invoke_no_args but called", @@ -767,6 +775,7 @@ impl ScalarUDFImpl for AliasedScalarUDFImpl { args: &[ColumnarValue], number_rows: usize, ) -> Result { + #[allow(deprecated)] self.inner.invoke_batch(args, number_rows) } diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index 475b864a8a18..4bfc3f07bb14 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -30,10 +30,10 @@ use arrow::datatypes::{DataType, Field}; use crate::expr::WindowFunction; use crate::{ - function::WindowFunctionSimplification, Documentation, Expr, PartitionEvaluator, - Signature, + function::WindowFunctionSimplification, Expr, PartitionEvaluator, Signature, }; use datafusion_common::{not_impl_err, Result}; +use datafusion_doc::Documentation; use datafusion_functions_window_common::expr::ExpressionArgs; use datafusion_functions_window_common::field::WindowUDFFieldArgs; use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; @@ -258,13 +258,9 @@ where /// /// fn get_doc() -> &'static Documentation { /// DOCUMENTATION.get_or_init(|| { -/// Documentation::builder() -/// .with_doc_section(DOC_SECTION_ANALYTICAL) -/// .with_description("smooths the windows") -/// .with_syntax_example("smooth_it(2)") +/// Documentation::builder(DOC_SECTION_ANALYTICAL, "smooths the windows", "smooth_it(2)") /// .with_argument("arg1", "The int32 number to smooth by") /// .build() -/// .unwrap() /// }) /// } /// @@ -557,7 +553,7 @@ impl WindowUDFImpl for AliasedWindowUDFImpl { // Window UDF doc sections for use in public documentation pub mod window_doc_sections { - use crate::DocSection; + use datafusion_doc::DocSection; pub fn doc_sections() -> Vec { vec![ diff --git a/datafusion/ffi/LICENSE.txt b/datafusion/ffi/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/ffi/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/ffi/NOTICE.txt b/datafusion/ffi/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/ffi/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions-aggregate-common/LICENSE.txt b/datafusion/functions-aggregate-common/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/functions-aggregate-common/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/functions-aggregate-common/NOTICE.txt b/datafusion/functions-aggregate-common/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/functions-aggregate-common/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions-aggregate-common/src/merge_arrays.rs b/datafusion/functions-aggregate-common/src/merge_arrays.rs index 544bdc182829..9b9a1240c1a1 100644 --- a/datafusion/functions-aggregate-common/src/merge_arrays.rs +++ b/datafusion/functions-aggregate-common/src/merge_arrays.rs @@ -65,7 +65,7 @@ impl<'a> CustomElement<'a> { // Overwrite ordering implementation such that // - `self.ordering` values are used for comparison, // - When used inside `BinaryHeap` it is a min-heap. -impl<'a> Ord for CustomElement<'a> { +impl Ord for CustomElement<'_> { fn cmp(&self, other: &Self) -> Ordering { // Compares according to custom ordering self.ordering(&self.ordering, &other.ordering) @@ -78,7 +78,7 @@ impl<'a> Ord for CustomElement<'a> { } } -impl<'a> PartialOrd for CustomElement<'a> { +impl PartialOrd for CustomElement<'_> { fn partial_cmp(&self, other: &Self) -> Option { Some(self.cmp(other)) } diff --git a/datafusion/functions-aggregate-common/src/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs index e440abe2de69..083dac615b5d 100644 --- a/datafusion/functions-aggregate-common/src/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -48,6 +48,7 @@ pub fn get_accum_scalar_values_as_arrays( /// Since `Decimal128Arrays` created from `Vec` have /// default precision and scale, this function adjusts the output to /// match `data_type`, if necessary +#[deprecated(since = "44.0.0", note = "use PrimitiveArray::with_datatype")] pub fn adjust_output_array(data_type: &DataType, array: ArrayRef) -> Result { let array = match data_type { DataType::Decimal128(p, s) => Arc::new( diff --git a/datafusion/functions-aggregate/Cargo.toml b/datafusion/functions-aggregate/Cargo.toml index 37e4c7f4a5ad..33a52afbe21a 100644 --- a/datafusion/functions-aggregate/Cargo.toml +++ b/datafusion/functions-aggregate/Cargo.toml @@ -48,7 +48,6 @@ datafusion-functions-aggregate-common = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } half = { workspace = true } -indexmap = { workspace = true } log = { workspace = true } paste = "1.0.14" diff --git a/datafusion/functions-aggregate/LICENSE.txt b/datafusion/functions-aggregate/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/functions-aggregate/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/functions-aggregate/NOTICE.txt b/datafusion/functions-aggregate/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/functions-aggregate/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions-aggregate/src/approx_distinct.rs b/datafusion/functions-aggregate/src/approx_distinct.rs index 1df106feb4d3..e955cea9d1a0 100644 --- a/datafusion/functions-aggregate/src/approx_distinct.rs +++ b/datafusion/functions-aggregate/src/approx_distinct.rs @@ -317,12 +317,7 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_approx_distinct_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_APPROXIMATE) - .with_description( - "Returns the approximate number of distinct input values calculated using the HyperLogLog algorithm.", - ) - .with_syntax_example("approx_distinct(expression)") + Documentation::builder(DOC_SECTION_APPROXIMATE, "Returns the approximate number of distinct input values calculated using the HyperLogLog algorithm.", "approx_distinct(expression)") .with_sql_example(r#"```sql > SELECT approx_distinct(column_name) FROM table_name; +-----------------------------------+ @@ -334,6 +329,5 @@ fn get_approx_distinct_doc() -> &'static Documentation { ) .with_standard_argument("expression", None) .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/approx_median.rs b/datafusion/functions-aggregate/src/approx_median.rs index 96609622a51e..8920c8e5f0c4 100644 --- a/datafusion/functions-aggregate/src/approx_median.rs +++ b/datafusion/functions-aggregate/src/approx_median.rs @@ -130,12 +130,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_approx_median_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_APPROXIMATE) - .with_description( + Documentation::builder( + DOC_SECTION_APPROXIMATE, "Returns the approximate median (50th percentile) of input values. It is an alias of `approx_percentile_cont(x, 0.5)`.", - ) - .with_syntax_example("approx_median(expression)") + + "approx_median(expression)") .with_sql_example(r#"```sql > SELECT approx_median(column_name) FROM table_name; +-----------------------------------+ @@ -147,6 +146,5 @@ fn get_approx_median_doc() -> &'static Documentation { ) .with_standard_argument("expression", None) .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont.rs b/datafusion/functions-aggregate/src/approx_percentile_cont.rs index 53fcfd641ddf..6edae6344ab1 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont.rs @@ -280,12 +280,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_approx_percentile_cont_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_APPROXIMATE) - .with_description( + Documentation::builder( + DOC_SECTION_APPROXIMATE, "Returns the approximate percentile of input values using the t-digest algorithm.", - ) - .with_syntax_example("approx_percentile_cont(expression, percentile, centroids)") + "approx_percentile_cont(expression, percentile, centroids)") .with_sql_example(r#"```sql > SELECT approx_percentile_cont(column_name, 0.75, 100) FROM table_name; +-------------------------------------------------+ @@ -298,7 +296,6 @@ fn get_approx_percentile_cont_doc() -> &'static Documentation { .with_argument("percentile", "Percentile to compute. Must be a float value between 0 and 1 (inclusive).") .with_argument("centroids", "Number of centroids to use in the t-digest algorithm. _Default is 100_. A higher number results in more accurate approximation but requires more memory.") .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs index 5458d0f792b9..7cf8d2dca13f 100644 --- a/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs +++ b/datafusion/functions-aggregate/src/approx_percentile_cont_with_weight.rs @@ -165,12 +165,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_approx_percentile_cont_with_weight_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_APPROXIMATE) - .with_description( + Documentation::builder( + DOC_SECTION_APPROXIMATE, "Returns the weighted approximate percentile of input values using the t-digest algorithm.", - ) - .with_syntax_example("approx_percentile_cont_with_weight(expression, weight, percentile)") + + "approx_percentile_cont_with_weight(expression, weight, percentile)") .with_sql_example(r#"```sql > SELECT approx_percentile_cont_with_weight(column_name, weight_column, 0.90) FROM table_name; +----------------------------------------------------------------------+ @@ -184,7 +183,6 @@ fn get_approx_percentile_cont_with_weight_doc() -> &'static Documentation { .with_argument("weight", "Expression to use as weight. Can be a constant, column, or function, and any combination of arithmetic operators.") .with_argument("percentile", "Percentile to compute. Must be a float value between 0 and 1 (inclusive).") .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 252a07cb11d8..3b9a521ec972 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -154,12 +154,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_agg_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( + Documentation::builder( + DOC_SECTION_GENERAL, "Returns an array created from the expression elements. If ordering is required, elements are inserted in the specified order.", - ) - .with_syntax_example("array_agg(expression [ORDER BY expression])") + + "array_agg(expression [ORDER BY expression])") .with_sql_example(r#"```sql > SELECT array_agg(column_name ORDER BY other_column) FROM table_name; +-----------------------------------------------+ @@ -171,7 +170,6 @@ fn get_array_agg_doc() -> &'static Documentation { ) .with_standard_argument("expression", None) .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 710b7e69ac5c..3fa58f3c2082 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -248,14 +248,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_avg_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( - "Returns the average of numeric values in the specified column.", - ) - .with_syntax_example("avg(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns the average of numeric values in the specified column.", + "avg(expression)", + ) + .with_sql_example( + r#"```sql > SELECT avg(column_name) FROM table_name; +---------------------------+ | avg(column_name) | @@ -263,10 +262,9 @@ fn get_avg_doc() -> &'static Documentation { | 42.75 | +---------------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } diff --git a/datafusion/functions-aggregate/src/bit_and_or_xor.rs b/datafusion/functions-aggregate/src/bit_and_or_xor.rs index 249ff02e7222..aeed78737c1d 100644 --- a/datafusion/functions-aggregate/src/bit_and_or_xor.rs +++ b/datafusion/functions-aggregate/src/bit_and_or_xor.rs @@ -139,13 +139,13 @@ static BIT_AND_DOC: OnceLock = OnceLock::new(); fn get_bit_and_doc() -> &'static Documentation { BIT_AND_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description("Computes the bitwise AND of all non-null input values.") - .with_syntax_example("bit_and(expression)") - .with_standard_argument("expression", Some("Integer")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_GENERAL, + "Computes the bitwise AND of all non-null input values.", + "bit_and(expression)", + ) + .with_standard_argument("expression", Some("Integer")) + .build() }) } @@ -153,13 +153,13 @@ static BIT_OR_DOC: OnceLock = OnceLock::new(); fn get_bit_or_doc() -> &'static Documentation { BIT_OR_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description("Computes the bitwise OR of all non-null input values.") - .with_syntax_example("bit_or(expression)") - .with_standard_argument("expression", Some("Integer")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_GENERAL, + "Computes the bitwise OR of all non-null input values.", + "bit_or(expression)", + ) + .with_standard_argument("expression", Some("Integer")) + .build() }) } @@ -167,15 +167,13 @@ static BIT_XOR_DOC: OnceLock = OnceLock::new(); fn get_bit_xor_doc() -> &'static Documentation { BIT_XOR_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( - "Computes the bitwise exclusive OR of all non-null input values.", - ) - .with_syntax_example("bit_xor(expression)") - .with_standard_argument("expression", Some("Integer")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_GENERAL, + "Computes the bitwise exclusive OR of all non-null input values.", + "bit_xor(expression)", + ) + .with_standard_argument("expression", Some("Integer")) + .build() }) } diff --git a/datafusion/functions-aggregate/src/bool_and_or.rs b/datafusion/functions-aggregate/src/bool_and_or.rs index 87293ccfa21f..df9271d8160a 100644 --- a/datafusion/functions-aggregate/src/bool_and_or.rs +++ b/datafusion/functions-aggregate/src/bool_and_or.rs @@ -186,14 +186,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_bool_and_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( - "Returns true if all non-null input values are true, otherwise false.", - ) - .with_syntax_example("bool_and(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns true if all non-null input values are true, otherwise false.", + "bool_and(expression)", + ) + .with_sql_example( + r#"```sql > SELECT bool_and(column_name) FROM table_name; +----------------------------+ | bool_and(column_name) | @@ -201,10 +200,9 @@ fn get_bool_and_doc() -> &'static Documentation { | true | +----------------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } @@ -335,14 +333,13 @@ impl AggregateUDFImpl for BoolOr { fn get_bool_or_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( - "Returns true if any non-null input value is true, otherwise false.", - ) - .with_syntax_example("bool_or(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns true if any non-null input value is true, otherwise false.", + "bool_or(expression)", + ) + .with_sql_example( + r#"```sql > SELECT bool_or(column_name) FROM table_name; +----------------------------+ | bool_or(column_name) | @@ -350,10 +347,9 @@ fn get_bool_or_doc() -> &'static Documentation { | true | +----------------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index 187a43ecbea3..4711b4240797 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -119,14 +119,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_corr_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( - "Returns the coefficient of correlation between two numeric values.", - ) - .with_syntax_example("corr(expression1, expression2)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STATISTICAL, + "Returns the coefficient of correlation between two numeric values.", + "corr(expression1, expression2)", + ) + .with_sql_example( + r#"```sql > SELECT corr(column1, column2) FROM table_name; +--------------------------------+ | corr(column1, column2) | @@ -134,11 +133,10 @@ fn get_corr_doc() -> &'static Documentation { | 0.85 | +--------------------------------+ ```"#, - ) - .with_standard_argument("expression1", Some("First")) - .with_standard_argument("expression2", Some("Second")) - .build() - .unwrap() + ) + .with_standard_argument("expression1", Some("First")) + .with_standard_argument("expression2", Some("Second")) + .build() }) } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 8fdd702b5b7c..c8f8c8153ce1 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -336,12 +336,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_count_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( + Documentation::builder( + DOC_SECTION_GENERAL, "Returns the number of non-null values in the specified column. To include null values in the total count, use `count(*)`.", - ) - .with_syntax_example("count(expression)") + + "count(expression)") .with_sql_example(r#"```sql > SELECT count(column_name) FROM table_name; +-----------------------+ @@ -359,7 +358,6 @@ fn get_count_doc() -> &'static Documentation { ```"#) .with_standard_argument("expression", None) .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/covariance.rs b/datafusion/functions-aggregate/src/covariance.rs index 063aaa92059d..0c29589e9095 100644 --- a/datafusion/functions-aggregate/src/covariance.rs +++ b/datafusion/functions-aggregate/src/covariance.rs @@ -137,12 +137,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_covar_samp_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description("Returns the sample covariance of a set of number pairs.") - .with_syntax_example("covar_samp(expression1, expression2)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STATISTICAL, + "Returns the sample covariance of a set of number pairs.", + "covar_samp(expression1, expression2)", + ) + .with_sql_example( + r#"```sql > SELECT covar_samp(column1, column2) FROM table_name; +-----------------------------------+ | covar_samp(column1, column2) | @@ -150,11 +151,10 @@ fn get_covar_samp_doc() -> &'static Documentation { | 8.25 | +-----------------------------------+ ```"#, - ) - .with_standard_argument("expression1", Some("First")) - .with_standard_argument("expression2", Some("Second")) - .build() - .unwrap() + ) + .with_standard_argument("expression1", Some("First")) + .with_standard_argument("expression2", Some("Second")) + .build() }) } @@ -233,14 +233,13 @@ impl AggregateUDFImpl for CovariancePopulation { fn get_covar_pop_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( - "Returns the population covariance of a set of number pairs.", - ) - .with_syntax_example("covar_pop(expression1, expression2)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STATISTICAL, + "Returns the population covariance of a set of number pairs.", + "covar_pop(expression1, expression2)", + ) + .with_sql_example( + r#"```sql > SELECT covar_pop(column1, column2) FROM table_name; +-----------------------------------+ | covar_pop(column1, column2) | @@ -248,11 +247,10 @@ fn get_covar_pop_doc() -> &'static Documentation { | 7.63 | +-----------------------------------+ ```"#, - ) - .with_standard_argument("expression1", Some("First")) - .with_standard_argument("expression2", Some("Second")) - .build() - .unwrap() + ) + .with_standard_argument("expression1", Some("First")) + .with_standard_argument("expression2", Some("Second")) + .build() }) } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 493c2cde2be8..b7001f52ac84 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -169,12 +169,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_first_value_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( + Documentation::builder( + DOC_SECTION_GENERAL, "Returns the first element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group.", - ) - .with_syntax_example("first_value(expression [ORDER BY expression])") + + "first_value(expression [ORDER BY expression])") .with_sql_example(r#"```sql > SELECT first_value(column_name ORDER BY other_column) FROM table_name; +-----------------------------------------------+ @@ -186,7 +185,6 @@ fn get_first_value_doc() -> &'static Documentation { ) .with_standard_argument("expression", None) .build() - .unwrap() }) } @@ -491,12 +489,11 @@ impl AggregateUDFImpl for LastValue { fn get_last_value_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( + Documentation::builder( + DOC_SECTION_GENERAL, "Returns the last element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group.", - ) - .with_syntax_example("last_value(expression [ORDER BY expression])") + + "last_value(expression [ORDER BY expression])") .with_sql_example(r#"```sql > SELECT last_value(column_name ORDER BY other_column) FROM table_name; +-----------------------------------------------+ @@ -508,7 +505,6 @@ fn get_last_value_doc() -> &'static Documentation { ) .with_standard_argument("expression", None) .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/grouping.rs b/datafusion/functions-aggregate/src/grouping.rs index 27949aa3df27..4a45890b0e70 100644 --- a/datafusion/functions-aggregate/src/grouping.rs +++ b/datafusion/functions-aggregate/src/grouping.rs @@ -108,12 +108,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_grouping_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( + Documentation::builder( + DOC_SECTION_GENERAL, "Returns 1 if the data is aggregated across the specified column, or 0 if it is not aggregated in the result set.", - ) - .with_syntax_example("grouping(expression)") + + "grouping(expression)") .with_sql_example(r#"```sql > SELECT column_name, GROUPING(column_name) AS group_column FROM table_name @@ -129,6 +128,5 @@ fn get_grouping_doc() -> &'static Documentation { ) .with_argument("expression", "Expression to evaluate whether data is aggregated across the specified column. Can be a constant, column, or function.") .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index a7114bb68bfd..bcffb19b7559 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::cmp::Ordering; use std::fmt::{Debug, Formatter}; use std::mem::{size_of, size_of_val}; use std::sync::{Arc, OnceLock}; @@ -30,7 +31,7 @@ use arrow::{ use arrow::array::Array; use arrow::array::ArrowNativeTypeOp; -use arrow::datatypes::ArrowNativeType; +use arrow::datatypes::{ArrowNativeType, ArrowPrimitiveType}; use datafusion_common::{DataFusionError, HashSet, Result, ScalarValue}; use datafusion_expr::aggregate_doc_sections::DOC_SECTION_GENERAL; @@ -163,12 +164,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_median_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description("Returns the median value in the specified column.") - .with_syntax_example("median(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns the median value in the specified column.", + "median(expression)", + ) + .with_sql_example( + r#"```sql > SELECT median(column_name) FROM table_name; +----------------------+ | median(column_name) | @@ -176,10 +178,9 @@ fn get_median_doc() -> &'static Documentation { | 45.5 | +----------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } @@ -310,6 +311,21 @@ impl Accumulator for DistinctMedianAccumulator { } } +/// Get maximum entry in the slice, +fn slice_max(array: &[T::Native]) -> T::Native +where + T: ArrowPrimitiveType, + T::Native: PartialOrd, // Ensure the type supports PartialOrd for comparison +{ + // Make sure that, array is not empty. + debug_assert!(!array.is_empty()); + // `.unwrap()` is safe here as the array is supposed to be non-empty + *array + .iter() + .max_by(|x, y| x.partial_cmp(y).unwrap_or(Ordering::Less)) + .unwrap() +} + fn calculate_median( mut values: Vec, ) -> Option { @@ -320,8 +336,11 @@ fn calculate_median( None } else if len % 2 == 0 { let (low, high, _) = values.select_nth_unstable_by(len / 2, cmp); - let (_, low, _) = low.select_nth_unstable_by(low.len() - 1, cmp); - let median = low.add_wrapping(*high).div_wrapping(T::Native::usize_as(2)); + // Get the maximum of the low (left side after bi-partitioning) + let left_max = slice_max::(low); + let median = left_max + .add_wrapping(*high) + .div_wrapping(T::Native::usize_as(2)); Some(median) } else { let (_, median, _) = values.select_nth_unstable_by(len / 2, cmp); diff --git a/datafusion/functions-aggregate/src/min_max.rs b/datafusion/functions-aggregate/src/min_max.rs index 618edd343f7d..2077f1567411 100644 --- a/datafusion/functions-aggregate/src/min_max.rs +++ b/datafusion/functions-aggregate/src/min_max.rs @@ -354,12 +354,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_max_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description("Returns the maximum value in the specified column.") - .with_syntax_example("max(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns the maximum value in the specified column.", + "max(expression)", + ) + .with_sql_example( + r#"```sql > SELECT max(column_name) FROM table_name; +----------------------+ | max(column_name) | @@ -367,10 +368,9 @@ fn get_max_doc() -> &'static Documentation { | 150 | +----------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } @@ -1184,12 +1184,13 @@ impl AggregateUDFImpl for Min { fn get_min_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description("Returns the minimum value in the specified column.") - .with_syntax_example("min(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns the minimum value in the specified column.", + "min(expression)", + ) + .with_sql_example( + r#"```sql > SELECT min(column_name) FROM table_name; +----------------------+ | min(column_name) | @@ -1197,10 +1198,9 @@ fn get_min_doc() -> &'static Documentation { | 12 | +----------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index f3e892fa73d8..0c72939633b1 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -173,12 +173,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_nth_value_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Returns the nth value in a group of values.", - ) - .with_syntax_example("nth_value(expression, n ORDER BY expression)") + + "nth_value(expression, n ORDER BY expression)") .with_sql_example(r#"```sql > SELECT dept_id, salary, NTH_VALUE(salary, 2) OVER (PARTITION BY dept_id ORDER BY salary ASC) AS second_salary_by_dept FROM employee; @@ -195,7 +194,6 @@ fn get_nth_value_doc() -> &'static Documentation { .with_argument("expression", "The column or expression to retrieve the nth value from.") .with_argument("n", "The position (nth) of the value to retrieve, based on the ordering.") .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/regr.rs b/datafusion/functions-aggregate/src/regr.rs index 9dd13634ff2d..f302b72f94b5 100644 --- a/datafusion/functions-aggregate/src/regr.rs +++ b/datafusion/functions-aggregate/src/regr.rs @@ -138,130 +138,112 @@ fn get_regr_docs() -> &'static HashMap { let mut hash_map = HashMap::new(); hash_map.insert( RegrType::Slope, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Returns the slope of the linear regression line for non-null pairs in aggregate columns. \ Given input column Y and X: regr_slope(Y, X) returns the slope (k in Y = k*X + b) using minimal RSS fitting.", - ) - .with_syntax_example("regr_slope(expression_y, expression_x)") + + "regr_slope(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::Intercept, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Computes the y-intercept of the linear regression line. For the equation (y = kx + b), \ this function returns b.", - ) - .with_syntax_example("regr_intercept(expression_y, expression_x)") + + "regr_intercept(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::Count, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Counts the number of non-null paired data points.", - ) - .with_syntax_example("regr_count(expression_y, expression_x)") + + "regr_count(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::R2, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Computes the square of the correlation coefficient between the independent and dependent variables.", - ) - .with_syntax_example("regr_r2(expression_y, expression_x)") + + "regr_r2(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::AvgX, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Computes the average of the independent variable (input) expression_x for the non-null paired data points.", - ) - .with_syntax_example("regr_avgx(expression_y, expression_x)") + + "regr_avgx(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::AvgY, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Computes the average of the dependent variable (output) expression_y for the non-null paired data points.", - ) - .with_syntax_example("regr_avgy(expression_y, expression_x)") + + "regr_avgy(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::SXX, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Computes the sum of squares of the independent variable.", - ) - .with_syntax_example("regr_sxx(expression_y, expression_x)") + + "regr_sxx(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::SYY, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Computes the sum of squares of the dependent variable.", - ) - .with_syntax_example("regr_syy(expression_y, expression_x)") + + "regr_syy(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map.insert( RegrType::SXY, - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( + Documentation::builder( + DOC_SECTION_STATISTICAL, "Computes the sum of products of paired data points.", - ) - .with_syntax_example("regr_sxy(expression_y, expression_x)") + + "regr_sxy(expression_y, expression_x)") .with_standard_argument("expression_y", Some("Dependent variable")) .with_standard_argument("expression_x", Some("Independent variable")) .build() - .unwrap() ); hash_map }) diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index d1f43c666138..afc9bf6255c2 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -142,12 +142,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_stddev_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description("Returns the standard deviation of a set of numbers.") - .with_syntax_example("stddev(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STATISTICAL, + "Returns the standard deviation of a set of numbers.", + "stddev(expression)", + ) + .with_sql_example( + r#"```sql > SELECT stddev(column_name) FROM table_name; +----------------------+ | stddev(column_name) | @@ -155,10 +156,9 @@ fn get_stddev_doc() -> &'static Documentation { | 12.34 | +----------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } @@ -264,14 +264,13 @@ impl AggregateUDFImpl for StddevPop { fn get_stddev_pop_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STATISTICAL) - .with_description( - "Returns the population standard deviation of a set of numbers.", - ) - .with_syntax_example("stddev_pop(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STATISTICAL, + "Returns the population standard deviation of a set of numbers.", + "stddev_pop(expression)", + ) + .with_sql_example( + r#"```sql > SELECT stddev_pop(column_name) FROM table_name; +--------------------------+ | stddev_pop(column_name) | @@ -279,10 +278,9 @@ fn get_stddev_pop_doc() -> &'static Documentation { | 10.56 | +--------------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } diff --git a/datafusion/functions-aggregate/src/string_agg.rs b/datafusion/functions-aggregate/src/string_agg.rs index 68267b9f72c7..4fd2d91b46c0 100644 --- a/datafusion/functions-aggregate/src/string_agg.rs +++ b/datafusion/functions-aggregate/src/string_agg.rs @@ -111,12 +111,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_string_agg_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( - "Concatenates the values of string expressions and places separator values between them." - ) - .with_syntax_example("string_agg(expression, delimiter)") + Documentation::builder( + DOC_SECTION_GENERAL, + "Concatenates the values of string expressions and places separator values between them.", + + "string_agg(expression, delimiter)") .with_sql_example(r#"```sql > SELECT string_agg(name, ', ') AS names_list FROM employee; @@ -130,7 +129,6 @@ fn get_string_agg_doc() -> &'static Documentation { .with_argument("expression", "The string expression to concatenate. Can be a column or any valid string expression.") .with_argument("delimiter", "A literal string used as a separator between the concatenated values.") .build() - .unwrap() }) } diff --git a/datafusion/functions-aggregate/src/sum.rs b/datafusion/functions-aggregate/src/sum.rs index 6ad376db4fb9..447b5d8a57c4 100644 --- a/datafusion/functions-aggregate/src/sum.rs +++ b/datafusion/functions-aggregate/src/sum.rs @@ -247,12 +247,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_sum_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description("Returns the sum of all values in the specified column.") - .with_syntax_example("sum(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns the sum of all values in the specified column.", + "sum(expression)", + ) + .with_sql_example( + r#"```sql > SELECT sum(column_name) FROM table_name; +-----------------------+ | sum(column_name) | @@ -260,10 +261,9 @@ fn get_sum_doc() -> &'static Documentation { | 12345 | +-----------------------+ ```"#, - ) - .with_standard_argument("expression", None) - .build() - .unwrap() + ) + .with_standard_argument("expression", None) + .build() }) } diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index 55d4181a96df..9c99a9138dd5 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -145,15 +145,13 @@ static VARIANCE_SAMPLE_DOC: OnceLock = OnceLock::new(); fn get_variance_sample_doc() -> &'static Documentation { VARIANCE_SAMPLE_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( - "Returns the statistical sample variance of a set of numbers.", - ) - .with_syntax_example("var(expression)") - .with_standard_argument("expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns the statistical sample variance of a set of numbers.", + "var(expression)", + ) + .with_standard_argument("expression", Some("Numeric")) + .build() }) } @@ -251,15 +249,13 @@ static VARIANCE_POPULATION_DOC: OnceLock = OnceLock::new(); fn get_variance_population_doc() -> &'static Documentation { VARIANCE_POPULATION_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_GENERAL) - .with_description( - "Returns the statistical population variance of a set of numbers.", - ) - .with_syntax_example("var_pop(expression)") - .with_standard_argument("expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_GENERAL, + "Returns the statistical population variance of a set of numbers.", + "var_pop(expression)", + ) + .with_standard_argument("expression", Some("Numeric")) + .build() }) } diff --git a/datafusion/functions-nested/LICENSE.txt b/datafusion/functions-nested/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/functions-nested/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/functions-nested/NOTICE.txt b/datafusion/functions-nested/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/functions-nested/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions-nested/benches/map.rs b/datafusion/functions-nested/benches/map.rs index 3c4a09c65992..0f1d9ed50636 100644 --- a/datafusion/functions-nested/benches/map.rs +++ b/datafusion/functions-nested/benches/map.rs @@ -96,9 +96,9 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(|| { black_box( - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args map_udf() - .invoke(&[keys.clone(), values.clone()]) + .invoke_batch(&[keys.clone(), values.clone()], 1) .expect("map should work on valid values"), ); }); diff --git a/datafusion/functions-nested/src/array_has.rs b/datafusion/functions-nested/src/array_has.rs index fe1d05199e80..c71314d8263f 100644 --- a/datafusion/functions-nested/src/array_has.rs +++ b/datafusion/functions-nested/src/array_has.rs @@ -98,7 +98,11 @@ impl ScalarUDFImpl for ArrayHas { Ok(DataType::Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match &args[1] { ColumnarValue::Array(array_needle) => { // the needle is already an array, convert the haystack to an array of the same length @@ -142,12 +146,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_has_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns true if the array contains the element.", - ) - .with_syntax_example("array_has(array, element)") + + "array_has(array, element)") .with_sql_example( r#"```sql > select array_has([1, 2, 3], 2); @@ -167,7 +170,6 @@ fn get_array_has_doc() -> &'static Documentation { "Scalar or Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -322,7 +324,11 @@ impl ScalarUDFImpl for ArrayHasAll { Ok(DataType::Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_has_all_inner)(args) } @@ -337,12 +343,11 @@ impl ScalarUDFImpl for ArrayHasAll { fn get_array_has_all_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns true if all elements of sub-array exist in array.", - ) - .with_syntax_example("array_has_all(array, sub-array)") + + "array_has_all(array, sub-array)") .with_sql_example( r#"```sql > select array_has_all([1, 2, 3, 4], [2, 3]); @@ -362,7 +367,6 @@ fn get_array_has_all_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -403,7 +407,11 @@ impl ScalarUDFImpl for ArrayHasAny { Ok(DataType::Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_has_any_inner)(args) } @@ -418,12 +426,11 @@ impl ScalarUDFImpl for ArrayHasAny { fn get_array_has_any_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns true if any elements exist in both arrays.", - ) - .with_syntax_example("array_has_any(array, sub-array)") + + "array_has_any(array, sub-array)") .with_sql_example( r#"```sql > select array_has_any([1, 2, 3], [3, 4]); @@ -443,7 +450,6 @@ fn get_array_has_any_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/cardinality.rs b/datafusion/functions-nested/src/cardinality.rs index b6661e0807f4..45543d1bd68b 100644 --- a/datafusion/functions-nested/src/cardinality.rs +++ b/datafusion/functions-nested/src/cardinality.rs @@ -83,7 +83,11 @@ impl ScalarUDFImpl for Cardinality { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(cardinality_inner)(args) } @@ -100,12 +104,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_cardinality_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the total number of elements in the array.", - ) - .with_syntax_example("cardinality(array)") + + "cardinality(array)") .with_sql_example( r#"```sql > select cardinality([[1, 2, 3, 4], [5, 6, 7, 8]]); @@ -121,7 +124,6 @@ fn get_cardinality_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/concat.rs b/datafusion/functions-nested/src/concat.rs index 4aa6bb5da9b2..ac9b5b2a16ef 100644 --- a/datafusion/functions-nested/src/concat.rs +++ b/datafusion/functions-nested/src/concat.rs @@ -86,7 +86,11 @@ impl ScalarUDFImpl for ArrayAppend { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_append_inner)(args) } @@ -103,12 +107,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_append_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Appends an element to the end of an array.", - ) - .with_syntax_example("array_append(array, element)") + + "array_append(array, element)") .with_sql_example( r#"```sql > select array_append([1, 2, 3], 4); @@ -128,7 +131,6 @@ fn get_array_append_doc() -> &'static Documentation { "Element to append to the array.", ) .build() - .unwrap() }) } @@ -182,7 +184,11 @@ impl ScalarUDFImpl for ArrayPrepend { Ok(arg_types[1].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_prepend_inner)(args) } @@ -199,12 +205,11 @@ static DOCUMENTATION_PREPEND: OnceLock = OnceLock::new(); fn get_array_prepend_doc() -> &'static Documentation { DOCUMENTATION_PREPEND.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Prepends an element to the beginning of an array.", - ) - .with_syntax_example("array_prepend(element, array)") + + "array_prepend(element, array)") .with_sql_example( r#"```sql > select array_prepend(1, [2, 3, 4]); @@ -224,7 +229,6 @@ fn get_array_prepend_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -302,7 +306,11 @@ impl ScalarUDFImpl for ArrayConcat { Ok(expr_type) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_concat_inner)(args) } @@ -317,12 +325,11 @@ impl ScalarUDFImpl for ArrayConcat { fn get_array_concat_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Concatenates arrays.", - ) - .with_syntax_example("array_concat(array[, ..., array_n])") + + "array_concat(array[, ..., array_n])") .with_sql_example( r#"```sql > select array_concat([1, 2], [3, 4], [5, 6]); @@ -342,7 +349,6 @@ fn get_array_concat_doc() -> &'static Documentation { "Subsequent array column or literal array to concatenate.", ) .build() - .unwrap() }) } @@ -432,7 +438,7 @@ fn concat_internal(args: &[ArrayRef]) -> Result { Ok(Arc::new(list_arr)) } -/// Kernal functions +// Kernel functions /// Array_append SQL function pub(crate) fn array_append_inner(args: &[ArrayRef]) -> Result { diff --git a/datafusion/functions-nested/src/dimension.rs b/datafusion/functions-nested/src/dimension.rs index 7df0ed2b40bd..9ca7c87aba55 100644 --- a/datafusion/functions-nested/src/dimension.rs +++ b/datafusion/functions-nested/src/dimension.rs @@ -81,7 +81,11 @@ impl ScalarUDFImpl for ArrayDims { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_dims_inner)(args) } @@ -98,12 +102,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_dims_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns an array of the array's dimensions.", - ) - .with_syntax_example("array_dims(array)") + + "array_dims(array)") .with_sql_example( r#"```sql > select array_dims([[1, 2, 3], [4, 5, 6]]); @@ -119,7 +122,6 @@ fn get_array_dims_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -166,7 +168,11 @@ impl ScalarUDFImpl for ArrayNdims { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_ndims_inner)(args) } @@ -181,12 +187,11 @@ impl ScalarUDFImpl for ArrayNdims { fn get_array_ndims_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the number of dimensions of the array.", - ) - .with_syntax_example("array_ndims(array, element)") + + "array_ndims(array, element)") .with_sql_example( r#"```sql > select array_ndims([[1, 2, 3], [4, 5, 6]]); @@ -206,7 +211,6 @@ fn get_array_ndims_doc() -> &'static Documentation { "Array element.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/distance.rs b/datafusion/functions-nested/src/distance.rs index 4f890e4166e9..381ddeb59a0b 100644 --- a/datafusion/functions-nested/src/distance.rs +++ b/datafusion/functions-nested/src/distance.rs @@ -96,7 +96,11 @@ impl ScalarUDFImpl for ArrayDistance { Ok(result) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_distance_inner)(args) } @@ -113,12 +117,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_distance_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the Euclidean distance between two input arrays of equal length.", - ) - .with_syntax_example("array_distance(array1, array2)") + + "array_distance(array1, array2)") .with_sql_example( r#"```sql > select array_distance([1, 2], [1, 4]); @@ -138,7 +141,6 @@ fn get_array_distance_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/empty.rs b/datafusion/functions-nested/src/empty.rs index 5d310eb23952..5270c84c0338 100644 --- a/datafusion/functions-nested/src/empty.rs +++ b/datafusion/functions-nested/src/empty.rs @@ -73,7 +73,11 @@ impl ScalarUDFImpl for ArrayEmpty { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_empty_inner)(args) } @@ -90,12 +94,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_empty_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns 1 for an empty array or 0 for a non-empty array.", - ) - .with_syntax_example("empty(array)") + + "empty(array)") .with_sql_example( r#"```sql > select empty([1]); @@ -111,7 +114,6 @@ fn get_empty_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/except.rs b/datafusion/functions-nested/src/except.rs index 100fb587d642..83c09ad7fd90 100644 --- a/datafusion/functions-nested/src/except.rs +++ b/datafusion/functions-nested/src/except.rs @@ -73,7 +73,11 @@ impl ScalarUDFImpl for ArrayExcept { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_except_inner)(args) } @@ -90,12 +94,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_except_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns an array of the elements that appear in the first array but not in the second.", - ) - .with_syntax_example("array_except(array1, array2)") + + "array_except(array1, array2)") .with_sql_example( r#"```sql > select array_except([1, 2, 3, 4], [5, 6, 3, 4]); @@ -121,7 +124,6 @@ fn get_array_except_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/extract.rs b/datafusion/functions-nested/src/extract.rs index 275095832edb..13095bc4ba3f 100644 --- a/datafusion/functions-nested/src/extract.rs +++ b/datafusion/functions-nested/src/extract.rs @@ -143,7 +143,11 @@ impl ScalarUDFImpl for ArrayElement { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_element_inner)(args) } @@ -160,12 +164,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_element_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Extracts the element with the index n from the array.", - ) - .with_syntax_example("array_element(array, index)") + + "array_element(array, index)") .with_sql_example( r#"```sql > select array_element([1, 2, 3, 4], 3); @@ -185,7 +188,6 @@ fn get_array_element_doc() -> &'static Documentation { "Index to extract the element from the array.", ) .build() - .unwrap() }) } @@ -347,7 +349,11 @@ impl ScalarUDFImpl for ArraySlice { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_slice_inner)(args) } @@ -362,12 +368,11 @@ impl ScalarUDFImpl for ArraySlice { fn get_array_slice_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns a slice of the array based on 1-indexed start and end positions.", - ) - .with_syntax_example("array_slice(array, begin, end)") + + "array_slice(array, begin, end)") .with_sql_example( r#"```sql > select array_slice([1, 2, 3, 4, 5, 6, 7, 8], 3, 6); @@ -395,7 +400,6 @@ fn get_array_slice_doc() -> &'static Documentation { "Stride of the array slice. The default is 1.", ) .build() - .unwrap() }) } @@ -656,7 +660,11 @@ impl ScalarUDFImpl for ArrayPopFront { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_pop_front_inner)(args) } @@ -671,12 +679,11 @@ impl ScalarUDFImpl for ArrayPopFront { fn get_array_pop_front_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the array without the first element.", - ) - .with_syntax_example("array_pop_front(array)") + + "array_pop_front(array)") .with_sql_example( r#"```sql > select array_pop_front([1, 2, 3]); @@ -692,7 +699,6 @@ fn get_array_pop_front_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -762,7 +768,11 @@ impl ScalarUDFImpl for ArrayPopBack { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_pop_back_inner)(args) } @@ -777,12 +787,11 @@ impl ScalarUDFImpl for ArrayPopBack { fn get_array_pop_back_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the array without the last element.", - ) - .with_syntax_example("array_pop_back(array)") + + "array_pop_back(array)") .with_sql_example( r#"```sql > select array_pop_back([1, 2, 3]); @@ -798,7 +807,6 @@ fn get_array_pop_back_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -877,7 +885,11 @@ impl ScalarUDFImpl for ArrayAnyValue { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_any_value_inner)(args) } fn aliases(&self) -> &[String] { @@ -891,12 +903,11 @@ impl ScalarUDFImpl for ArrayAnyValue { fn get_array_any_value_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the first non-null element in the array.", - ) - .with_syntax_example("array_any_value(array)") + + "array_any_value(array)") .with_sql_example( r#"```sql > select array_any_value([NULL, 1, 2, 3]); @@ -912,7 +923,6 @@ fn get_array_any_value_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/flatten.rs b/datafusion/functions-nested/src/flatten.rs index 4fe631517b09..3650b1f37b1b 100644 --- a/datafusion/functions-nested/src/flatten.rs +++ b/datafusion/functions-nested/src/flatten.rs @@ -91,7 +91,11 @@ impl ScalarUDFImpl for Flatten { Ok(data_type) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(flatten_inner)(args) } @@ -107,12 +111,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_flatten_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Converts an array of arrays to a flat array.\n\n- Applies to any depth of nested arrays\n- Does not change arrays that are already flat\n\nThe flattened array contains all the elements from all source arrays.", - ) - .with_syntax_example("flatten(array)") + + "flatten(array)") .with_sql_example( r#"```sql > select flatten([[1, 2], [3, 4]]); @@ -128,7 +131,6 @@ fn get_flatten_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/length.rs b/datafusion/functions-nested/src/length.rs index 3e039f286421..084b37bf13a4 100644 --- a/datafusion/functions-nested/src/length.rs +++ b/datafusion/functions-nested/src/length.rs @@ -77,7 +77,11 @@ impl ScalarUDFImpl for ArrayLength { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_length_inner)(args) } @@ -94,12 +98,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_length_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the length of the array dimension.", - ) - .with_syntax_example("array_length(array, dimension)") + + "array_length(array, dimension)") .with_sql_example( r#"```sql > select array_length([1, 2, 3, 4, 5], 1); @@ -119,7 +122,6 @@ fn get_array_length_doc() -> &'static Documentation { "Array dimension.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/make_array.rs b/datafusion/functions-nested/src/make_array.rs index c84b6f010968..825824a82d20 100644 --- a/datafusion/functions-nested/src/make_array.rs +++ b/datafusion/functions-nested/src/make_array.rs @@ -147,12 +147,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_make_array_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns an array using the specified input expressions.", - ) - .with_syntax_example("make_array(expression1[, ..., expression_n])") + + "make_array(expression1[, ..., expression_n])") .with_sql_example( r#"```sql > select make_array(1, 2, 3, 4, 5); @@ -168,7 +167,6 @@ fn get_make_array_doc() -> &'static Documentation { "Expression to include in the output array. Can be a constant, column, or function, and any combination of arithmetic or string operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index 73aad10a8e26..d21a19c9fb33 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -214,9 +214,9 @@ impl ScalarUDFImpl for MapFunc { } fn return_type(&self, arg_types: &[DataType]) -> Result { - if arg_types.len() % 2 != 0 { + if arg_types.len() != 2 { return exec_err!( - "map requires an even number of arguments, got {} instead", + "map requires exactly 2 arguments, got {} instead", arg_types.len() ); } @@ -238,7 +238,11 @@ impl ScalarUDFImpl for MapFunc { )) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_map_batch(args) } @@ -251,13 +255,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_map_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MAP) - .with_description( + Documentation::builder( + DOC_SECTION_MAP, "Returns an Arrow map with the specified key-value pairs.\n\n\ - The `make_map` function creates a map from two lists: one for keys and one for values. Each key must be unique and non-null." - ) - .with_syntax_example( + The `make_map` function creates a map from two lists: one for keys and one for values. Each key must be unique and non-null.", + "map(key, value)\nmap(key: value)\nmake_map(['key1', 'key2'], ['value1', 'value2'])" ) .with_sql_example( @@ -301,7 +303,6 @@ SELECT MAKE_MAP(['key1', 'key2'], ['value1', null]); For `make_map`: The list of values to be mapped to the corresponding keys." ) .build() - .unwrap() }) } @@ -372,7 +373,6 @@ fn get_element_type(data_type: &DataType) -> Result<&DataType> { /// | +-------+ | | +-------+ | /// +-----------+ +-----------+ /// ```text - fn make_map_array_internal( keys: ArrayRef, values: ArrayRef, diff --git a/datafusion/functions-nested/src/map_extract.rs b/datafusion/functions-nested/src/map_extract.rs index d2bb6595fe76..d2ab078cb22d 100644 --- a/datafusion/functions-nested/src/map_extract.rs +++ b/datafusion/functions-nested/src/map_extract.rs @@ -85,7 +85,11 @@ impl ScalarUDFImpl for MapExtract { )))) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(map_extract_inner)(args) } @@ -114,12 +118,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_map_extract_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MAP) - .with_description( + Documentation::builder( + DOC_SECTION_MAP, "Returns a list containing the value for the given key or an empty list if the key is not present in the map.", - ) - .with_syntax_example("map_extract(map, key)") + "map_extract(map, key)") .with_sql_example( r#"```sql SELECT map_extract(MAP {'a': 1, 'b': NULL, 'c': 3}, 'a'); @@ -144,7 +146,6 @@ SELECT map_extract(MAP {'x': 10, 'y': NULL, 'z': 30}, 'y'); "Key to extract from the map. Can be a constant, column, or function, any combination of arithmetic or string operators, or a named expression of the previously listed.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/map_keys.rs b/datafusion/functions-nested/src/map_keys.rs index 03e381e372f6..4abdbcad1e82 100644 --- a/datafusion/functions-nested/src/map_keys.rs +++ b/datafusion/functions-nested/src/map_keys.rs @@ -79,7 +79,11 @@ impl ScalarUDFImpl for MapKeysFunc { )))) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(map_keys_inner)(args) } @@ -92,12 +96,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_map_keys_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MAP) - .with_description( - "Returns a list of all keys in the map." - ) - .with_syntax_example("map_keys(map)") + Documentation::builder( + DOC_SECTION_MAP, + "Returns a list of all keys in the map.", + "map_keys(map)") .with_sql_example( r#"```sql SELECT map_keys(MAP {'a': 1, 'b': NULL, 'c': 3}); @@ -114,7 +116,6 @@ SELECT map_keys(map([100, 5], [42, 43])); "Map expression. Can be a constant, column, or function, and any combination of map operators." ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/map_values.rs b/datafusion/functions-nested/src/map_values.rs index dc7d9c9db8ee..f1cc36cade63 100644 --- a/datafusion/functions-nested/src/map_values.rs +++ b/datafusion/functions-nested/src/map_values.rs @@ -79,7 +79,11 @@ impl ScalarUDFImpl for MapValuesFunc { )))) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(map_values_inner)(args) } @@ -92,12 +96,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_map_values_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MAP) - .with_description( - "Returns a list of all values in the map." - ) - .with_syntax_example("map_values(map)") + Documentation::builder( + DOC_SECTION_MAP, + "Returns a list of all values in the map.", + + "map_values(map)") .with_sql_example( r#"```sql SELECT map_values(MAP {'a': 1, 'b': NULL, 'c': 3}); @@ -114,7 +117,6 @@ SELECT map_values(map([100, 5], [42, 43])); "Map expression. Can be a constant, column, or function, and any combination of map operators." ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/planner.rs b/datafusion/functions-nested/src/planner.rs index 1929b8222a1b..5ca51ac20f1e 100644 --- a/datafusion/functions-nested/src/planner.rs +++ b/datafusion/functions-nested/src/planner.rs @@ -185,5 +185,5 @@ impl ExprPlanner for FieldAccessPlanner { } fn is_array_agg(agg_func: &datafusion_expr::expr::AggregateFunction) -> bool { - return agg_func.func.name() == "array_agg"; + agg_func.func.name() == "array_agg" } diff --git a/datafusion/functions-nested/src/position.rs b/datafusion/functions-nested/src/position.rs index adb45141601d..9ed4b4c42d14 100644 --- a/datafusion/functions-nested/src/position.rs +++ b/datafusion/functions-nested/src/position.rs @@ -82,7 +82,11 @@ impl ScalarUDFImpl for ArrayPosition { Ok(UInt64) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_position_inner)(args) } @@ -99,12 +103,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_position_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the position of the first occurrence of the specified element in the array.", - ) - .with_syntax_example("array_position(array, element)\narray_position(array, element, index)") + + "array_position(array, element)\narray_position(array, element, index)") .with_sql_example( r#"```sql > select array_position([1, 2, 2, 3, 1, 4], 2); @@ -134,7 +137,6 @@ fn get_array_position_doc() -> &'static Documentation { "Index at which to start searching.", ) .build() - .unwrap() }) } @@ -253,7 +255,11 @@ impl ScalarUDFImpl for ArrayPositions { Ok(List(Arc::new(Field::new("item", UInt64, true)))) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_positions_inner)(args) } @@ -268,12 +274,11 @@ impl ScalarUDFImpl for ArrayPositions { fn get_array_positions_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Searches for an element in the array, returns all occurrences.", - ) - .with_syntax_example("array_positions(array, element)") + + "array_positions(array, element)") .with_sql_example( r#"```sql > select array_positions([1, 2, 2, 3, 1, 4], 2); @@ -293,7 +298,6 @@ fn get_array_positions_doc() -> &'static Documentation { "Element to search for positions in the array.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/range.rs b/datafusion/functions-nested/src/range.rs index ddc56b1e4ee8..360f0023dc47 100644 --- a/datafusion/functions-nested/src/range.rs +++ b/datafusion/functions-nested/src/range.rs @@ -117,7 +117,11 @@ impl ScalarUDFImpl for Range { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.iter().any(|arg| arg.data_type().is_null()) { return Ok(ColumnarValue::Array(Arc::new(NullArray::new(1)))); } @@ -146,12 +150,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_range_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns an Arrow array between start and stop with step. The range start..end contains all values with start <= x < end. It is empty if start >= end. Step cannot be 0.", - ) - .with_syntax_example("range(start, stop, step)") + + "range(start, stop, step)") .with_sql_example( r#"```sql > select range(2, 10, 3); @@ -182,7 +185,6 @@ fn get_range_doc() -> &'static Documentation { "Increase by step (cannot be 0). Steps less than a day are supported only for timestamp ranges.", ) .build() - .unwrap() }) } @@ -255,7 +257,11 @@ impl ScalarUDFImpl for GenSeries { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.iter().any(|arg| arg.data_type().is_null()) { return Ok(ColumnarValue::Array(Arc::new(NullArray::new(1)))); } @@ -287,12 +293,11 @@ static GENERATE_SERIES_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_generate_series_doc() -> &'static Documentation { GENERATE_SERIES_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Similar to the range function, but it includes the upper bound.", - ) - .with_syntax_example("generate_series(start, stop, step)") + + "generate_series(start, stop, step)") .with_sql_example( r#"```sql > select generate_series(1,3); @@ -316,7 +321,6 @@ fn get_generate_series_doc() -> &'static Documentation { "increase by step (can not be 0). Steps less than a day are supported only for timestamp ranges.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/remove.rs b/datafusion/functions-nested/src/remove.rs index dc1ed4833c67..df0edc99bc62 100644 --- a/datafusion/functions-nested/src/remove.rs +++ b/datafusion/functions-nested/src/remove.rs @@ -74,7 +74,11 @@ impl ScalarUDFImpl for ArrayRemove { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_remove_inner)(args) } @@ -91,12 +95,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_remove_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Removes the first element from the array equal to the given value.", - ) - .with_syntax_example("array_remove(array, element)") + + "array_remove(array, element)") .with_sql_example( r#"```sql > select array_remove([1, 2, 2, 3, 2, 1, 4], 2); @@ -116,7 +119,6 @@ fn get_array_remove_doc() -> &'static Documentation { "Element to be removed from the array.", ) .build() - .unwrap() }) } @@ -160,7 +162,11 @@ impl ScalarUDFImpl for ArrayRemoveN { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_remove_n_inner)(args) } @@ -175,12 +181,11 @@ impl ScalarUDFImpl for ArrayRemoveN { fn get_array_remove_n_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Removes the first `max` elements from the array equal to the given value.", - ) - .with_syntax_example("array_remove_n(array, element, max)") + + "array_remove_n(array, element, max)") .with_sql_example( r#"```sql > select array_remove_n([1, 2, 2, 3, 2, 1, 4], 2, 2); @@ -204,7 +209,6 @@ fn get_array_remove_n_doc() -> &'static Documentation { "Number of first occurrences to remove.", ) .build() - .unwrap() }) } @@ -248,7 +252,11 @@ impl ScalarUDFImpl for ArrayRemoveAll { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_remove_all_inner)(args) } @@ -263,12 +271,11 @@ impl ScalarUDFImpl for ArrayRemoveAll { fn get_array_remove_all_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Removes all elements from the array equal to the given value.", - ) - .with_syntax_example("array_remove_all(array, element)") + + "array_remove_all(array, element)") .with_sql_example( r#"```sql > select array_remove_all([1, 2, 2, 3, 2, 1, 4], 2); @@ -288,7 +295,6 @@ fn get_array_remove_all_doc() -> &'static Documentation { "Element to be removed from the array.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/repeat.rs b/datafusion/functions-nested/src/repeat.rs index 55584c143a54..f67ab83b1d39 100644 --- a/datafusion/functions-nested/src/repeat.rs +++ b/datafusion/functions-nested/src/repeat.rs @@ -79,7 +79,11 @@ impl ScalarUDFImpl for ArrayRepeat { )))) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_repeat_inner)(args) } @@ -96,12 +100,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_repeat_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns an array containing element `count` times.", - ) - .with_syntax_example("array_repeat(element, count)") + + "array_repeat(element, count)") .with_sql_example( r#"```sql > select array_repeat(1, 3); @@ -127,7 +130,6 @@ fn get_array_repeat_doc() -> &'static Documentation { "Value of how many times to repeat the element.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/replace.rs b/datafusion/functions-nested/src/replace.rs index 1d0a1d1f2815..01811b77734d 100644 --- a/datafusion/functions-nested/src/replace.rs +++ b/datafusion/functions-nested/src/replace.rs @@ -90,7 +90,11 @@ impl ScalarUDFImpl for ArrayReplace { Ok(args[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_replace_inner)(args) } @@ -107,12 +111,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_replace_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Replaces the first occurrence of the specified element with another specified element.", - ) - .with_syntax_example("array_replace(array, from, to)") + + "array_replace(array, from, to)") .with_sql_example( r#"```sql > select array_replace([1, 2, 2, 3, 2, 1, 4], 2, 5); @@ -136,7 +139,6 @@ fn get_array_replace_doc() -> &'static Documentation { "Final element.", ) .build() - .unwrap() }) } @@ -172,7 +174,11 @@ impl ScalarUDFImpl for ArrayReplaceN { Ok(args[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_replace_n_inner)(args) } @@ -187,12 +193,11 @@ impl ScalarUDFImpl for ArrayReplaceN { fn get_array_replace_n_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Replaces the first `max` occurrences of the specified element with another specified element.", - ) - .with_syntax_example("array_replace_n(array, from, to, max)") + + "array_replace_n(array, from, to, max)") .with_sql_example( r#"```sql > select array_replace_n([1, 2, 2, 3, 2, 1, 4], 2, 5, 2); @@ -220,7 +225,6 @@ fn get_array_replace_n_doc() -> &'static Documentation { "Number of first occurrences to replace.", ) .build() - .unwrap() }) } @@ -256,7 +260,11 @@ impl ScalarUDFImpl for ArrayReplaceAll { Ok(args[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_replace_all_inner)(args) } @@ -271,12 +279,11 @@ impl ScalarUDFImpl for ArrayReplaceAll { fn get_array_replace_all_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Replaces all occurrences of the specified element with another specified element.", - ) - .with_syntax_example("array_replace_all(array, from, to)") + + "array_replace_all(array, from, to)") .with_sql_example( r#"```sql > select array_replace_all([1, 2, 2, 3, 2, 1, 4], 2, 5); @@ -300,7 +307,6 @@ fn get_array_replace_all_doc() -> &'static Documentation { "Final element.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/resize.rs b/datafusion/functions-nested/src/resize.rs index b0255e7be2a3..c9487dd81843 100644 --- a/datafusion/functions-nested/src/resize.rs +++ b/datafusion/functions-nested/src/resize.rs @@ -80,7 +80,11 @@ impl ScalarUDFImpl for ArrayResize { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_resize_inner)(args) } @@ -97,12 +101,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_resize_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Resizes the list to contain size elements. Initializes new elements with value or empty if value is not set.", - ) - .with_syntax_example("array_resize(array, size, value)") + + "array_resize(array, size, value)") .with_sql_example( r#"```sql > select array_resize([1, 2, 3], 5, 0); @@ -126,7 +129,6 @@ fn get_array_resize_doc() -> &'static Documentation { "Defines new elements' value or empty if value is not set.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/reverse.rs b/datafusion/functions-nested/src/reverse.rs index 1ecf7f848468..aa898268d10b 100644 --- a/datafusion/functions-nested/src/reverse.rs +++ b/datafusion/functions-nested/src/reverse.rs @@ -72,7 +72,11 @@ impl ScalarUDFImpl for ArrayReverse { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_reverse_inner)(args) } @@ -89,12 +93,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_reverse_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns the array with the order of the elements reversed.", - ) - .with_syntax_example("array_reverse(array)") + + "array_reverse(array)") .with_sql_example( r#"```sql > select array_reverse([1, 2, 3, 4]); @@ -110,7 +113,6 @@ fn get_array_reverse_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/set_ops.rs b/datafusion/functions-nested/src/set_ops.rs index ce8d248319fe..889eaed4edcc 100644 --- a/datafusion/functions-nested/src/set_ops.rs +++ b/datafusion/functions-nested/src/set_ops.rs @@ -98,7 +98,11 @@ impl ScalarUDFImpl for ArrayUnion { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_union_inner)(args) } @@ -115,12 +119,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_union_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns an array of elements that are present in both arrays (all elements from both arrays) with out duplicates.", - ) - .with_syntax_example("array_union(array1, array2)") + + "array_union(array1, array2)") .with_sql_example( r#"```sql > select array_union([1, 2, 3, 4], [5, 6, 3, 4]); @@ -146,7 +149,6 @@ fn get_array_union_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -186,7 +188,11 @@ impl ScalarUDFImpl for ArrayIntersect { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_intersect_inner)(args) } @@ -201,12 +207,11 @@ impl ScalarUDFImpl for ArrayIntersect { fn get_array_intersect_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns an array of elements in the intersection of array1 and array2.", - ) - .with_syntax_example("array_intersect(array1, array2)") + + "array_intersect(array1, array2)") .with_sql_example( r#"```sql > select array_intersect([1, 2, 3, 4], [5, 6, 3, 4]); @@ -232,7 +237,6 @@ fn get_array_intersect_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } @@ -282,7 +286,11 @@ impl ScalarUDFImpl for ArrayDistinct { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_distinct_inner)(args) } @@ -297,12 +305,11 @@ impl ScalarUDFImpl for ArrayDistinct { fn get_array_distinct_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Returns distinct values from the array after removing duplicates.", - ) - .with_syntax_example("array_distinct(array)") + + "array_distinct(array)") .with_sql_example( r#"```sql > select array_distinct([1, 3, 2, 3, 1, 2, 4]); @@ -318,7 +325,6 @@ fn get_array_distinct_doc() -> &'static Documentation { "Array expression. Can be a constant, column, or function, and any combination of array operators.", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/sort.rs b/datafusion/functions-nested/src/sort.rs index b29c187f0679..cdd6842acc0e 100644 --- a/datafusion/functions-nested/src/sort.rs +++ b/datafusion/functions-nested/src/sort.rs @@ -86,7 +86,11 @@ impl ScalarUDFImpl for ArraySort { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_sort_inner)(args) } @@ -103,12 +107,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_array_sort_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Sort array.", - ) - .with_syntax_example("array_sort(array, desc, nulls_first)") + + "array_sort(array, desc, nulls_first)") .with_sql_example( r#"```sql > select array_sort([3, 1, 2]); @@ -132,7 +135,6 @@ fn get_array_sort_doc() -> &'static Documentation { "Whether to sort nulls first(`NULLS FIRST` or `NULLS LAST`).", ) .build() - .unwrap() }) } diff --git a/datafusion/functions-nested/src/string.rs b/datafusion/functions-nested/src/string.rs index ce555c36274e..8c6cb73e97c9 100644 --- a/datafusion/functions-nested/src/string.rs +++ b/datafusion/functions-nested/src/string.rs @@ -32,44 +32,26 @@ use std::any::{type_name, Any}; use crate::utils::{downcast_arg, make_scalar_function}; use arrow::compute::cast; +use arrow_array::builder::{ArrayBuilder, LargeStringBuilder, StringViewBuilder}; +use arrow_array::cast::AsArray; +use arrow_array::{GenericStringArray, StringViewArray}; use arrow_schema::DataType::{ - Dictionary, FixedSizeList, LargeList, LargeUtf8, List, Null, Utf8, -}; -use datafusion_common::cast::{ - as_generic_string_array, as_large_list_array, as_list_array, as_string_array, + Dictionary, FixedSizeList, LargeList, LargeUtf8, List, Null, Utf8, Utf8View, }; +use datafusion_common::cast::{as_large_list_array, as_list_array}; use datafusion_common::exec_err; use datafusion_expr::scalar_doc_sections::DOC_SECTION_ARRAY; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_functions::strings::StringArrayType; use std::sync::{Arc, OnceLock}; -macro_rules! to_string { - ($ARG:expr, $ARRAY:expr, $DELIMITER:expr, $NULL_STRING:expr, $WITH_NULL_STRING:expr, $ARRAY_TYPE:ident) => {{ - let arr = downcast_arg!($ARRAY, $ARRAY_TYPE); - for x in arr { - match x { - Some(x) => { - $ARG.push_str(&x.to_string()); - $ARG.push_str($DELIMITER); - } - None => { - if $WITH_NULL_STRING { - $ARG.push_str($NULL_STRING); - $ARG.push_str($DELIMITER); - } - } - } - } - Ok($ARG) - }}; -} - macro_rules! call_array_function { ($DATATYPE:expr, false) => { match $DATATYPE { DataType::Utf8 => array_function!(StringArray), + DataType::Utf8View => array_function!(StringViewArray), DataType::LargeUtf8 => array_function!(LargeStringArray), DataType::Boolean => array_function!(BooleanArray), DataType::Float32 => array_function!(Float32Array), @@ -89,6 +71,7 @@ macro_rules! call_array_function { match $DATATYPE { DataType::List(_) => array_function!(ListArray), DataType::Utf8 => array_function!(StringArray), + DataType::Utf8View => array_function!(StringViewArray), DataType::LargeUtf8 => array_function!(LargeStringArray), DataType::Boolean => array_function!(BooleanArray), DataType::Float32 => array_function!(Float32Array), @@ -106,6 +89,27 @@ macro_rules! call_array_function { }}; } +macro_rules! to_string { + ($ARG:expr, $ARRAY:expr, $DELIMITER:expr, $NULL_STRING:expr, $WITH_NULL_STRING:expr, $ARRAY_TYPE:ident) => {{ + let arr = downcast_arg!($ARRAY, $ARRAY_TYPE); + for x in arr { + match x { + Some(x) => { + $ARG.push_str(&x.to_string()); + $ARG.push_str($DELIMITER); + } + None => { + if $WITH_NULL_STRING { + $ARG.push_str($NULL_STRING); + $ARG.push_str($DELIMITER); + } + } + } + } + Ok($ARG) + }}; +} + // Create static instances of ScalarUDFs for each function make_udf_expr_and_func!( ArrayToString, @@ -155,7 +159,11 @@ impl ScalarUDFImpl for ArrayToString { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(array_to_string_inner)(args) } @@ -172,12 +180,11 @@ static DOCUMENTATION_ARRAY_TO_STRING: OnceLock = OnceLock::new(); fn get_array_to_string_doc() -> &'static Documentation { DOCUMENTATION_ARRAY_TO_STRING.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Converts each element to its text representation.", - ) - .with_syntax_example("array_to_string(array, delimiter[, null_string])") + + "array_to_string(array, delimiter[, null_string])") .with_sql_example( r#"```sql > select array_to_string([[1, 2, 3, 4], [5, 6, 7, 8]], ','); @@ -201,7 +208,6 @@ fn get_array_to_string_doc() -> &'static Documentation { "Optional. String to replace null values in the array. If not provided, nulls will be handled by default behavior.", ) .build() - .unwrap() }) } @@ -222,10 +228,7 @@ impl StringToArray { pub fn new() -> Self { Self { signature: Signature::one_of( - vec![ - TypeSignature::Uniform(2, vec![Utf8, LargeUtf8]), - TypeSignature::Uniform(3, vec![Utf8, LargeUtf8]), - ], + vec![TypeSignature::String(2), TypeSignature::String(3)], Volatility::Immutable, ), aliases: vec![String::from("string_to_list")], @@ -248,23 +251,27 @@ impl ScalarUDFImpl for StringToArray { fn return_type(&self, arg_types: &[DataType]) -> Result { Ok(match arg_types[0] { - Utf8 | LargeUtf8 => { + Utf8 | Utf8View | LargeUtf8 => { List(Arc::new(Field::new("item", arg_types[0].clone(), true))) } _ => { return plan_err!( - "The string_to_array function can only accept Utf8 or LargeUtf8." + "The string_to_array function can only accept Utf8, Utf8View or LargeUtf8." ); } }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { - Utf8 => make_scalar_function(string_to_array_inner::)(args), + Utf8 | Utf8View => make_scalar_function(string_to_array_inner::)(args), LargeUtf8 => make_scalar_function(string_to_array_inner::)(args), other => { - exec_err!("unsupported type for string_to_array function as {other}") + exec_err!("unsupported type for string_to_array function as {other:?}") } } } @@ -282,12 +289,11 @@ static DOCUMENTATION_STRING_TO_ARRAY: OnceLock = OnceLock::new(); fn get_string_to_array_doc() -> &'static Documentation { DOCUMENTATION_STRING_TO_ARRAY.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ARRAY) - .with_description( + Documentation::builder( + DOC_SECTION_ARRAY, "Splits a string into an array of substrings based on a delimiter. Any substrings matching the optional `null_str` argument are replaced with NULL.", - ) - .with_syntax_example("string_to_array(str, delimiter[, null_str])") + + "string_to_array(str, delimiter[, null_str])") .with_sql_example( r#"```sql > select string_to_array('abc##def', '##'); @@ -317,7 +323,6 @@ fn get_string_to_array_doc() -> &'static Documentation { "Substring values to be replaced with `NULL`.", ) .build() - .unwrap() }) } @@ -329,13 +334,22 @@ pub(super) fn array_to_string_inner(args: &[ArrayRef]) -> Result { let arr = &args[0]; - let delimiters = as_string_array(&args[1])?; - let delimiters: Vec> = delimiters.iter().collect(); + let delimiters: Vec> = match args[1].data_type() { + Utf8 => args[1].as_string::().iter().collect(), + Utf8View => args[1].as_string_view().iter().collect(), + LargeUtf8 => args[1].as_string::().iter().collect(), + other => return exec_err!("unsupported type for second argument to array_to_string function as {other:?}") + }; let mut null_string = String::from(""); let mut with_null_string = false; if args.len() == 3 { - null_string = as_string_array(&args[2])?.value(0).to_string(); + null_string = match args[2].data_type() { + Utf8 => args[2].as_string::().value(0).to_string(), + Utf8View => args[2].as_string_view().value(0).to_string(), + LargeUtf8 => args[2].as_string::().value(0).to_string(), + other => return exec_err!("unsupported type for second argument to array_to_string function as {other:?}") + }; with_null_string = true; } @@ -495,20 +509,173 @@ pub(super) fn array_to_string_inner(args: &[ArrayRef]) -> Result { /// String_to_array SQL function /// Splits string at occurrences of delimiter and returns an array of parts /// string_to_array('abc~@~def~@~ghi', '~@~') = '["abc", "def", "ghi"]' -pub fn string_to_array_inner(args: &[ArrayRef]) -> Result { +fn string_to_array_inner(args: &[ArrayRef]) -> Result { if args.len() < 2 || args.len() > 3 { return exec_err!("string_to_array expects two or three arguments"); } - let string_array = as_generic_string_array::(&args[0])?; - let delimiter_array = as_generic_string_array::(&args[1])?; - let mut list_builder = ListBuilder::new(StringBuilder::with_capacity( - string_array.len(), - string_array.get_buffer_memory_size(), - )); + match args[0].data_type() { + Utf8 => { + let string_array = args[0].as_string::(); + let builder = StringBuilder::with_capacity(string_array.len(), string_array.get_buffer_memory_size()); + string_to_array_inner_2::<&GenericStringArray, StringBuilder>(args, string_array, builder) + } + Utf8View => { + let string_array = args[0].as_string_view(); + let builder = StringViewBuilder::with_capacity(string_array.len()); + string_to_array_inner_2::<&StringViewArray, StringViewBuilder>(args, string_array, builder) + } + LargeUtf8 => { + let string_array = args[0].as_string::(); + let builder = LargeStringBuilder::with_capacity(string_array.len(), string_array.get_buffer_memory_size()); + string_to_array_inner_2::<&GenericStringArray, LargeStringBuilder>(args, string_array, builder) + } + other => exec_err!("unsupported type for first argument to string_to_array function as {other:?}") + } +} - match args.len() { - 2 => { +fn string_to_array_inner_2<'a, StringArrType, StringBuilderType>( + args: &'a [ArrayRef], + string_array: StringArrType, + string_builder: StringBuilderType, +) -> Result +where + StringArrType: StringArrayType<'a>, + StringBuilderType: StringArrayBuilderType, +{ + match args[1].data_type() { + Utf8 => { + let delimiter_array = args[1].as_string::(); + if args.len() == 2 { + string_to_array_impl::< + StringArrType, + &GenericStringArray, + &StringViewArray, + StringBuilderType, + >(string_array, delimiter_array, None, string_builder) + } else { + string_to_array_inner_3::, + StringBuilderType>(args, string_array, delimiter_array, string_builder) + } + } + Utf8View => { + let delimiter_array = args[1].as_string_view(); + + if args.len() == 2 { + string_to_array_impl::< + StringArrType, + &StringViewArray, + &StringViewArray, + StringBuilderType, + >(string_array, delimiter_array, None, string_builder) + } else { + string_to_array_inner_3::(args, string_array, delimiter_array, string_builder) + } + } + LargeUtf8 => { + let delimiter_array = args[1].as_string::(); + if args.len() == 2 { + string_to_array_impl::< + StringArrType, + &GenericStringArray, + &StringViewArray, + StringBuilderType, + >(string_array, delimiter_array, None, string_builder) + } else { + string_to_array_inner_3::, + StringBuilderType>(args, string_array, delimiter_array, string_builder) + } + } + other => exec_err!("unsupported type for second argument to string_to_array function as {other:?}") + } +} + +fn string_to_array_inner_3<'a, StringArrType, DelimiterArrType, StringBuilderType>( + args: &'a [ArrayRef], + string_array: StringArrType, + delimiter_array: DelimiterArrType, + string_builder: StringBuilderType, +) -> Result +where + StringArrType: StringArrayType<'a>, + DelimiterArrType: StringArrayType<'a>, + StringBuilderType: StringArrayBuilderType, +{ + match args[2].data_type() { + Utf8 => { + let null_type_array = Some(args[2].as_string::()); + string_to_array_impl::< + StringArrType, + DelimiterArrType, + &GenericStringArray, + StringBuilderType, + >( + string_array, + delimiter_array, + null_type_array, + string_builder, + ) + } + Utf8View => { + let null_type_array = Some(args[2].as_string_view()); + string_to_array_impl::< + StringArrType, + DelimiterArrType, + &StringViewArray, + StringBuilderType, + >( + string_array, + delimiter_array, + null_type_array, + string_builder, + ) + } + LargeUtf8 => { + let null_type_array = Some(args[2].as_string::()); + string_to_array_impl::< + StringArrType, + DelimiterArrType, + &GenericStringArray, + StringBuilderType, + >( + string_array, + delimiter_array, + null_type_array, + string_builder, + ) + } + other => { + exec_err!("unsupported type for string_to_array function as {other:?}") + } + } +} + +fn string_to_array_impl< + 'a, + StringArrType, + DelimiterArrType, + NullValueArrType, + StringBuilderType, +>( + string_array: StringArrType, + delimiter_array: DelimiterArrType, + null_value_array: Option, + string_builder: StringBuilderType, +) -> Result +where + StringArrType: StringArrayType<'a>, + DelimiterArrType: StringArrayType<'a>, + NullValueArrType: StringArrayType<'a>, + StringBuilderType: StringArrayBuilderType, +{ + let mut list_builder = ListBuilder::new(string_builder); + + match null_value_array { + None => { string_array.iter().zip(delimiter_array.iter()).for_each( |(string, delimiter)| { match (string, delimiter) { @@ -524,63 +691,90 @@ pub fn string_to_array_inner(args: &[ArrayRef]) -> Result { string.chars().map(|c| c.to_string()).for_each(|c| { - list_builder.values().append_value(c); + list_builder.values().append_value(c.as_str()); }); list_builder.append(true); } _ => list_builder.append(false), // null value } }, - ); + ) } - - 3 => { - let null_value_array = as_generic_string_array::(&args[2])?; - string_array - .iter() - .zip(delimiter_array.iter()) - .zip(null_value_array.iter()) - .for_each(|((string, delimiter), null_value)| { - match (string, delimiter) { - (Some(string), Some("")) => { - if Some(string) == null_value { + Some(null_value_array) => string_array + .iter() + .zip(delimiter_array.iter()) + .zip(null_value_array.iter()) + .for_each(|((string, delimiter), null_value)| { + match (string, delimiter) { + (Some(string), Some("")) => { + if Some(string) == null_value { + list_builder.values().append_null(); + } else { + list_builder.values().append_value(string); + } + list_builder.append(true); + } + (Some(string), Some(delimiter)) => { + string.split(delimiter).for_each(|s| { + if Some(s) == null_value { list_builder.values().append_null(); } else { - list_builder.values().append_value(string); + list_builder.values().append_value(s); } - list_builder.append(true); - } - (Some(string), Some(delimiter)) => { - string.split(delimiter).for_each(|s| { - if Some(s) == null_value { - list_builder.values().append_null(); - } else { - list_builder.values().append_value(s); - } - }); - list_builder.append(true); - } - (Some(string), None) => { - string.chars().map(|c| c.to_string()).for_each(|c| { - if Some(c.as_str()) == null_value { - list_builder.values().append_null(); - } else { - list_builder.values().append_value(c); - } - }); - list_builder.append(true); - } - _ => list_builder.append(false), // null value + }); + list_builder.append(true); } - }); - } - _ => { - return exec_err!( - "Expect string_to_array function to take two or three parameters" - ) - } - } + (Some(string), None) => { + string.chars().map(|c| c.to_string()).for_each(|c| { + if Some(c.as_str()) == null_value { + list_builder.values().append_null(); + } else { + list_builder.values().append_value(c.as_str()); + } + }); + list_builder.append(true); + } + _ => list_builder.append(false), // null value + } + }), + }; let list_array = list_builder.finish(); Ok(Arc::new(list_array) as ArrayRef) } + +trait StringArrayBuilderType: ArrayBuilder { + fn append_value(&mut self, val: &str); + + fn append_null(&mut self); +} + +impl StringArrayBuilderType for StringBuilder { + fn append_value(&mut self, val: &str) { + StringBuilder::append_value(self, val); + } + + fn append_null(&mut self) { + StringBuilder::append_null(self); + } +} + +impl StringArrayBuilderType for StringViewBuilder { + fn append_value(&mut self, val: &str) { + StringViewBuilder::append_value(self, val) + } + + fn append_null(&mut self) { + StringViewBuilder::append_null(self) + } +} + +impl StringArrayBuilderType for LargeStringBuilder { + fn append_value(&mut self, val: &str) { + LargeStringBuilder::append_value(self, val); + } + + fn append_null(&mut self) { + LargeStringBuilder::append_null(self); + } +} diff --git a/datafusion/functions-window-common/LICENSE.txt b/datafusion/functions-window-common/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/functions-window-common/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/functions-window-common/NOTICE.txt b/datafusion/functions-window-common/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/functions-window-common/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions-window/LICENSE.txt b/datafusion/functions-window/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/functions-window/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/functions-window/NOTICE.txt b/datafusion/functions-window/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/functions-window/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions-window/src/cume_dist.rs b/datafusion/functions-window/src/cume_dist.rs index 500d96b56323..2523fd1cfe57 100644 --- a/datafusion/functions-window/src/cume_dist.rs +++ b/datafusion/functions-window/src/cume_dist.rs @@ -94,14 +94,8 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_cume_dist_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_RANKING) - .with_description( - "Relative rank of the current row: (number of rows preceding or peer with current row) / (total rows).", - ) - .with_syntax_example("cume_dist()") + Documentation::builder(DOC_SECTION_RANKING, "Relative rank of the current row: (number of rows preceding or peer with current row) / (total rows).", "cume_dist()") .build() - .unwrap() }) } diff --git a/datafusion/functions-window/src/lead_lag.rs b/datafusion/functions-window/src/lead_lag.rs index bbe50cbbdc8a..d73a5d86bbe6 100644 --- a/datafusion/functions-window/src/lead_lag.rs +++ b/datafusion/functions-window/src/lead_lag.rs @@ -152,21 +152,15 @@ static LAG_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_lag_doc() -> &'static Documentation { LAG_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ANALYTICAL) - .with_description( - "Returns value evaluated at the row that is offset rows before the \ + Documentation::builder(DOC_SECTION_ANALYTICAL, "Returns value evaluated at the row that is offset rows before the \ current row within the partition; if there is no such row, instead return default \ - (which must be of the same type as value).", - ) - .with_syntax_example("lag(expression, offset, default)") + (which must be of the same type as value).", "lag(expression, offset, default)") .with_argument("expression", "Expression to operate on") .with_argument("offset", "Integer. Specifies how many rows back \ the value of expression should be retrieved. Defaults to 1.") .with_argument("default", "The default value if the offset is \ not within the partition. Must be of the same type as expression.") .build() - .unwrap() }) } @@ -174,21 +168,17 @@ static LEAD_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_lead_doc() -> &'static Documentation { LEAD_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ANALYTICAL) - .with_description( + Documentation::builder(DOC_SECTION_ANALYTICAL, "Returns value evaluated at the row that is offset rows after the \ current row within the partition; if there is no such row, instead return default \ (which must be of the same type as value).", - ) - .with_syntax_example("lead(expression, offset, default)") + "lead(expression, offset, default)") .with_argument("expression", "Expression to operate on") .with_argument("offset", "Integer. Specifies how many rows \ forward the value of expression should be retrieved. Defaults to 1.") .with_argument("default", "The default value if the offset is \ not within the partition. Must be of the same type as expression.") .build() - .unwrap() }) } diff --git a/datafusion/functions-window/src/nth_value.rs b/datafusion/functions-window/src/nth_value.rs index cff49c1f4c85..d15e76718b02 100644 --- a/datafusion/functions-window/src/nth_value.rs +++ b/datafusion/functions-window/src/nth_value.rs @@ -131,16 +131,14 @@ static FIRST_VALUE_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_first_value_doc() -> &'static Documentation { FIRST_VALUE_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ANALYTICAL) - .with_description( - "Returns value evaluated at the row that is the first row of the window \ + Documentation::builder( + DOC_SECTION_ANALYTICAL, + "Returns value evaluated at the row that is the first row of the window \ frame.", - ) - .with_syntax_example("first_value(expression)") - .with_argument("expression", "Expression to operate on") - .build() - .unwrap() + "first_value(expression)", + ) + .with_argument("expression", "Expression to operate on") + .build() }) } @@ -148,16 +146,14 @@ static LAST_VALUE_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_last_value_doc() -> &'static Documentation { LAST_VALUE_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ANALYTICAL) - .with_description( - "Returns value evaluated at the row that is the last row of the window \ + Documentation::builder( + DOC_SECTION_ANALYTICAL, + "Returns value evaluated at the row that is the last row of the window \ frame.", - ) - .with_syntax_example("last_value(expression)") - .with_argument("expression", "Expression to operate on") - .build() - .unwrap() + "last_value(expression)", + ) + .with_argument("expression", "Expression to operate on") + .build() }) } @@ -165,21 +161,19 @@ static NTH_VALUE_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_nth_value_doc() -> &'static Documentation { NTH_VALUE_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_ANALYTICAL) - .with_description( - "Returns value evaluated at the row that is the nth row of the window \ + Documentation::builder( + DOC_SECTION_ANALYTICAL, + "Returns value evaluated at the row that is the nth row of the window \ frame (counting from 1); null if no such row.", - ) - .with_syntax_example("nth_value(expression, n)") - .with_argument( - "expression", - "The name the column of which nth \ + "nth_value(expression, n)", + ) + .with_argument( + "expression", + "The name the column of which nth \ value to retrieve", - ) - .with_argument("n", "Integer. Specifies the n in nth") - .build() - .unwrap() + ) + .with_argument("n", "Integer. Specifies the n in nth") + .build() }) } diff --git a/datafusion/functions-window/src/ntile.rs b/datafusion/functions-window/src/ntile.rs index b0a7241f24cd..06bf32f9859f 100644 --- a/datafusion/functions-window/src/ntile.rs +++ b/datafusion/functions-window/src/ntile.rs @@ -82,15 +82,9 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_ntile_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_RANKING) - .with_description( - "Integer ranging from 1 to the argument value, dividing the partition as equally as possible", - ) - .with_syntax_example("ntile(expression)") + Documentation::builder(DOC_SECTION_RANKING, "Integer ranging from 1 to the argument value, dividing the partition as equally as possible", "ntile(expression)") .with_argument("expression","An integer describing the number groups the partition should be split into") .build() - .unwrap() }) } diff --git a/datafusion/functions-window/src/rank.rs b/datafusion/functions-window/src/rank.rs index 06945e693eea..dacee90bfad6 100644 --- a/datafusion/functions-window/src/rank.rs +++ b/datafusion/functions-window/src/rank.rs @@ -106,16 +106,14 @@ static RANK_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_rank_doc() -> &'static Documentation { RANK_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_RANKING) - .with_description( + Documentation::builder( + DOC_SECTION_RANKING, "Returns the rank of the current row within its partition, allowing \ gaps between ranks. This function provides a ranking similar to `row_number`, but \ skips ranks for identical values.", - ) - .with_syntax_example("rank()") + + "rank()") .build() - .unwrap() }) } @@ -123,16 +121,10 @@ static DENSE_RANK_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_dense_rank_doc() -> &'static Documentation { DENSE_RANK_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_RANKING) - .with_description( - "Returns the rank of the current row without gaps. This function ranks \ + Documentation::builder(DOC_SECTION_RANKING, "Returns the rank of the current row without gaps. This function ranks \ rows in a dense manner, meaning consecutive ranks are assigned even for identical \ - values.", - ) - .with_syntax_example("dense_rank()") + values.", "dense_rank()") .build() - .unwrap() }) } @@ -140,15 +132,9 @@ static PERCENT_RANK_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_percent_rank_doc() -> &'static Documentation { PERCENT_RANK_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_RANKING) - .with_description( - "Returns the percentage rank of the current row within its partition. \ - The value ranges from 0 to 1 and is computed as `(rank - 1) / (total_rows - 1)`.", - ) - .with_syntax_example("percent_rank()") + Documentation::builder(DOC_SECTION_RANKING, "Returns the percentage rank of the current row within its partition. \ + The value ranges from 0 to 1 and is computed as `(rank - 1) / (total_rows - 1)`.", "percent_rank()") .build() - .unwrap() }) } diff --git a/datafusion/functions-window/src/row_number.rs b/datafusion/functions-window/src/row_number.rs index 68f6fde23280..72d4e0232365 100644 --- a/datafusion/functions-window/src/row_number.rs +++ b/datafusion/functions-window/src/row_number.rs @@ -66,14 +66,12 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_row_number_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_RANKING) - .with_description( - "Number of the current row within its partition, counting from 1.", - ) - .with_syntax_example("row_number()") - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_RANKING, + "Number of the current row within its partition, counting from 1.", + "row_number()", + ) + .build() }) } diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index 70a988dbfefb..36d4af9ab55b 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -72,8 +72,10 @@ blake2 = { version = "^0.10.2", optional = true } blake3 = { version = "1.0", optional = true } chrono = { workspace = true } datafusion-common = { workspace = true } +datafusion-doc = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-macros = { workspace = true } hashbrown = { workspace = true, optional = true } hex = { version = "0.4", optional = true } itertools = { workspace = true } @@ -89,7 +91,6 @@ uuid = { version = "1.7", features = ["v4"], optional = true } arrow = { workspace = true, features = ["test_utils"] } criterion = "0.5" rand = { workspace = true } -rstest = { workspace = true } tokio = { workspace = true, features = ["macros", "rt", "sync"] } [[bench]] diff --git a/datafusion/functions/LICENSE.txt b/datafusion/functions/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/functions/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/functions/NOTICE.txt b/datafusion/functions/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/functions/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/functions/benches/character_length.rs b/datafusion/functions/benches/character_length.rs index 9ba16807de01..b3fdb8dc8561 100644 --- a/datafusion/functions/benches/character_length.rs +++ b/datafusion/functions/benches/character_length.rs @@ -86,8 +86,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("character_length_StringArray_ascii_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(character_length.invoke(&args_string_ascii)) + // TODO use invoke_with_args + black_box(character_length.invoke_batch(&args_string_ascii, n_rows)) }) }, ); @@ -98,8 +98,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("character_length_StringArray_utf8_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(character_length.invoke(&args_string_utf8)) + // TODO use invoke_with_args + black_box(character_length.invoke_batch(&args_string_utf8, n_rows)) }) }, ); @@ -110,8 +110,10 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("character_length_StringViewArray_ascii_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(character_length.invoke(&args_string_view_ascii)) + // TODO use invoke_with_args + black_box( + character_length.invoke_batch(&args_string_view_ascii, n_rows), + ) }) }, ); @@ -122,8 +124,10 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("character_length_StringViewArray_utf8_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(character_length.invoke(&args_string_view_utf8)) + // TODO use invoke_with_args + black_box( + character_length.invoke_batch(&args_string_view_utf8, n_rows), + ) }) }, ); diff --git a/datafusion/functions/benches/concat.rs b/datafusion/functions/benches/concat.rs index 280819778f93..0f287ab36dad 100644 --- a/datafusion/functions/benches/concat.rs +++ b/datafusion/functions/benches/concat.rs @@ -39,8 +39,8 @@ fn criterion_benchmark(c: &mut Criterion) { let mut group = c.benchmark_group("concat function"); group.bench_function(BenchmarkId::new("concat", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - criterion::black_box(concat().invoke(&args).unwrap()) + // TODO use invoke_with_args + criterion::black_box(concat().invoke_batch(&args, size).unwrap()) }) }); group.finish(); diff --git a/datafusion/functions/benches/cot.rs b/datafusion/functions/benches/cot.rs index a33f00b4b73e..bb0585a2de9b 100644 --- a/datafusion/functions/benches/cot.rs +++ b/datafusion/functions/benches/cot.rs @@ -34,16 +34,16 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("cot f32 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(cot_fn.invoke(&f32_args).unwrap()) + // TODO use invoke_with_args + black_box(cot_fn.invoke_batch(&f32_args, size).unwrap()) }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("cot f64 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(cot_fn.invoke(&f64_args).unwrap()) + // TODO use invoke_with_args + black_box(cot_fn.invoke_batch(&f64_args, size).unwrap()) }) }); } diff --git a/datafusion/functions/benches/date_bin.rs b/datafusion/functions/benches/date_bin.rs index 4a8682c42f94..aa7c7710617d 100644 --- a/datafusion/functions/benches/date_bin.rs +++ b/datafusion/functions/benches/date_bin.rs @@ -19,7 +19,7 @@ extern crate criterion; use std::sync::Arc; -use arrow::array::{ArrayRef, TimestampSecondArray}; +use arrow::array::{Array, ArrayRef, TimestampSecondArray}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_common::ScalarValue; use rand::rngs::ThreadRng; @@ -40,14 +40,16 @@ fn timestamps(rng: &mut ThreadRng) -> TimestampSecondArray { fn criterion_benchmark(c: &mut Criterion) { c.bench_function("date_bin_1000", |b| { let mut rng = rand::thread_rng(); + let timestamps_array = Arc::new(timestamps(&mut rng)) as ArrayRef; + let batch_len = timestamps_array.len(); let interval = ColumnarValue::Scalar(ScalarValue::new_interval_dt(0, 1_000_000)); - let timestamps = ColumnarValue::Array(Arc::new(timestamps(&mut rng)) as ArrayRef); + let timestamps = ColumnarValue::Array(timestamps_array); let udf = date_bin(); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( - udf.invoke(&[interval.clone(), timestamps.clone()]) + udf.invoke_batch(&[interval.clone(), timestamps.clone()], batch_len) .expect("date_bin should work on valid values"), ) }) diff --git a/datafusion/functions/benches/encoding.rs b/datafusion/functions/benches/encoding.rs index 0615091e90d4..e37842a62b4a 100644 --- a/datafusion/functions/benches/encoding.rs +++ b/datafusion/functions/benches/encoding.rs @@ -29,29 +29,35 @@ fn criterion_benchmark(c: &mut Criterion) { let str_array = Arc::new(create_string_array_with_len::(size, 0.2, 32)); c.bench_function(&format!("base64_decode/{size}"), |b| { let method = ColumnarValue::Scalar("base64".into()); - #[allow(deprecated)] // TODO use invoke_batch + // TODO: use invoke_with_args let encoded = encoding::encode() - .invoke(&[ColumnarValue::Array(str_array.clone()), method.clone()]) + .invoke_batch( + &[ColumnarValue::Array(str_array.clone()), method.clone()], + size, + ) .unwrap(); let args = vec![encoded, method]; b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(decode.invoke(&args).unwrap()) + // TODO use invoke_with_args + black_box(decode.invoke_batch(&args, size).unwrap()) }) }); c.bench_function(&format!("hex_decode/{size}"), |b| { let method = ColumnarValue::Scalar("hex".into()); - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args let encoded = encoding::encode() - .invoke(&[ColumnarValue::Array(str_array.clone()), method.clone()]) + .invoke_batch( + &[ColumnarValue::Array(str_array.clone()), method.clone()], + size, + ) .unwrap(); let args = vec![encoded, method]; b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(decode.invoke(&args).unwrap()) + // TODO use invoke_with_args + black_box(decode.invoke_batch(&args, size).unwrap()) }) }); } diff --git a/datafusion/functions/benches/isnan.rs b/datafusion/functions/benches/isnan.rs index 3e50de658b36..605a520715f4 100644 --- a/datafusion/functions/benches/isnan.rs +++ b/datafusion/functions/benches/isnan.rs @@ -33,16 +33,16 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("isnan f32 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(isnan.invoke(&f32_args).unwrap()) + // TODO use invoke_with_args + black_box(isnan.invoke_batch(&f32_args, size).unwrap()) }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("isnan f64 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(isnan.invoke(&f64_args).unwrap()) + // TODO use invoke_with_args + black_box(isnan.invoke_batch(&f64_args, size).unwrap()) }) }); } diff --git a/datafusion/functions/benches/iszero.rs b/datafusion/functions/benches/iszero.rs index 3e6ac97063ca..48fb6fbed9c3 100644 --- a/datafusion/functions/benches/iszero.rs +++ b/datafusion/functions/benches/iszero.rs @@ -30,19 +30,21 @@ fn criterion_benchmark(c: &mut Criterion) { let iszero = iszero(); for size in [1024, 4096, 8192] { let f32_array = Arc::new(create_primitive_array::(size, 0.2)); + let batch_len = f32_array.len(); let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("iszero f32 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(iszero.invoke(&f32_args).unwrap()) + // TODO use invoke_with_args + black_box(iszero.invoke_batch(&f32_args, batch_len).unwrap()) }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); + let batch_len = f64_array.len(); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("iszero f64 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(iszero.invoke(&f64_args).unwrap()) + // TODO use invoke_with_args + black_box(iszero.invoke_batch(&f64_args, batch_len).unwrap()) }) }); } diff --git a/datafusion/functions/benches/lower.rs b/datafusion/functions/benches/lower.rs index 6cc67791464f..114ac4a16fe5 100644 --- a/datafusion/functions/benches/lower.rs +++ b/datafusion/functions/benches/lower.rs @@ -125,8 +125,8 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args1(size, 32); c.bench_function(&format!("lower_all_values_are_ascii: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(lower.invoke(&args)) + // TODO use invoke_with_args + black_box(lower.invoke_batch(&args, size)) }) }); @@ -135,8 +135,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("lower_the_first_value_is_nonascii: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(lower.invoke(&args)) + // TODO use invoke_with_args + black_box(lower.invoke_batch(&args, size)) }) }, ); @@ -146,8 +146,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("lower_the_middle_value_is_nonascii: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(lower.invoke(&args)) + // TODO use invoke_with_args + black_box(lower.invoke_batch(&args, size)) }) }, ); @@ -167,8 +167,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", size, str_len, null_density, mixed), |b| b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(lower.invoke(&args)) + // TODO use invoke_with_args + black_box(lower.invoke_batch(&args, size)) }), ); @@ -177,8 +177,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", size, str_len, null_density, mixed), |b| b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(lower.invoke(&args)) + // TODO use invoke_with_args + black_box(lower.invoke_batch(&args, size)) }), ); @@ -187,8 +187,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("lower_some_values_are_nonascii_string_views: size: {}, str_len: {}, non_ascii_density: {}, null_density: {}, mixed: {}", size, str_len, 0.1, null_density, mixed), |b| b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(lower.invoke(&args)) + // TODO use invoke_with_args + black_box(lower.invoke_batch(&args, size)) }), ); } diff --git a/datafusion/functions/benches/ltrim.rs b/datafusion/functions/benches/ltrim.rs index 4f94729b6fef..fed455eeac91 100644 --- a/datafusion/functions/benches/ltrim.rs +++ b/datafusion/functions/benches/ltrim.rs @@ -141,8 +141,8 @@ fn run_with_string_type( ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(ltrim.invoke(&args)) + // TODO use invoke_with_args + black_box(ltrim.invoke_batch(&args, size)) }) }, ); diff --git a/datafusion/functions/benches/make_date.rs b/datafusion/functions/benches/make_date.rs index a9844e4b2541..d9309bcd3db2 100644 --- a/datafusion/functions/benches/make_date.rs +++ b/datafusion/functions/benches/make_date.rs @@ -19,7 +19,7 @@ extern crate criterion; use std::sync::Arc; -use arrow::array::{ArrayRef, Int32Array}; +use arrow::array::{Array, ArrayRef, Int32Array}; use criterion::{black_box, criterion_group, criterion_main, Criterion}; use rand::rngs::ThreadRng; use rand::Rng; @@ -57,15 +57,20 @@ fn days(rng: &mut ThreadRng) -> Int32Array { fn criterion_benchmark(c: &mut Criterion) { c.bench_function("make_date_col_col_col_1000", |b| { let mut rng = rand::thread_rng(); - let years = ColumnarValue::Array(Arc::new(years(&mut rng)) as ArrayRef); + let years_array = Arc::new(years(&mut rng)) as ArrayRef; + let batch_len = years_array.len(); + let years = ColumnarValue::Array(years_array); let months = ColumnarValue::Array(Arc::new(months(&mut rng)) as ArrayRef); let days = ColumnarValue::Array(Arc::new(days(&mut rng)) as ArrayRef); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( make_date() - .invoke(&[years.clone(), months.clone(), days.clone()]) + .invoke_batch( + &[years.clone(), months.clone(), days.clone()], + batch_len, + ) .expect("make_date should work on valid values"), ) }) @@ -74,14 +79,19 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("make_date_scalar_col_col_1000", |b| { let mut rng = rand::thread_rng(); let year = ColumnarValue::Scalar(ScalarValue::Int32(Some(2025))); - let months = ColumnarValue::Array(Arc::new(months(&mut rng)) as ArrayRef); + let months_arr = Arc::new(months(&mut rng)) as ArrayRef; + let batch_len = months_arr.len(); + let months = ColumnarValue::Array(months_arr); let days = ColumnarValue::Array(Arc::new(days(&mut rng)) as ArrayRef); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( make_date() - .invoke(&[year.clone(), months.clone(), days.clone()]) + .invoke_batch( + &[year.clone(), months.clone(), days.clone()], + batch_len, + ) .expect("make_date should work on valid values"), ) }) @@ -91,13 +101,15 @@ fn criterion_benchmark(c: &mut Criterion) { let mut rng = rand::thread_rng(); let year = ColumnarValue::Scalar(ScalarValue::Int32(Some(2025))); let month = ColumnarValue::Scalar(ScalarValue::Int32(Some(11))); - let days = ColumnarValue::Array(Arc::new(days(&mut rng)) as ArrayRef); + let day_arr = Arc::new(days(&mut rng)); + let batch_len = day_arr.len(); + let days = ColumnarValue::Array(day_arr); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( make_date() - .invoke(&[year.clone(), month.clone(), days.clone()]) + .invoke_batch(&[year.clone(), month.clone(), days.clone()], batch_len) .expect("make_date should work on valid values"), ) }) @@ -109,10 +121,10 @@ fn criterion_benchmark(c: &mut Criterion) { let day = ColumnarValue::Scalar(ScalarValue::Int32(Some(26))); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( make_date() - .invoke(&[year.clone(), month.clone(), day.clone()]) + .invoke_batch(&[year.clone(), month.clone(), day.clone()], 1) .expect("make_date should work on valid values"), ) }) diff --git a/datafusion/functions/benches/nullif.rs b/datafusion/functions/benches/nullif.rs index 6e1154cf182a..e29fd03aa819 100644 --- a/datafusion/functions/benches/nullif.rs +++ b/datafusion/functions/benches/nullif.rs @@ -34,8 +34,8 @@ fn criterion_benchmark(c: &mut Criterion) { ]; c.bench_function(&format!("nullif scalar array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(nullif.invoke(&args).unwrap()) + // TODO use invoke_with_args + black_box(nullif.invoke_batch(&args, size).unwrap()) }) }); } diff --git a/datafusion/functions/benches/pad.rs b/datafusion/functions/benches/pad.rs index 4b21ca373047..6f267b350a35 100644 --- a/datafusion/functions/benches/pad.rs +++ b/datafusion/functions/benches/pad.rs @@ -102,24 +102,24 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("utf8 type", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - criterion::black_box(lpad().invoke(&args).unwrap()) + // TODO use invoke_with_args + criterion::black_box(lpad().invoke_batch(&args, size).unwrap()) }) }); let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("largeutf8 type", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - criterion::black_box(lpad().invoke(&args).unwrap()) + // TODO use invoke_with_args + criterion::black_box(lpad().invoke_batch(&args, size).unwrap()) }) }); let args = create_args::(size, 32, true); group.bench_function(BenchmarkId::new("stringview type", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - criterion::black_box(lpad().invoke(&args).unwrap()) + // TODO use invoke_with_args + criterion::black_box(lpad().invoke_batch(&args, size).unwrap()) }) }); @@ -130,16 +130,16 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("utf8 type", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - criterion::black_box(rpad().invoke(&args).unwrap()) + // TODO use invoke_with_args + criterion::black_box(rpad().invoke_batch(&args, size).unwrap()) }) }); let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("largeutf8 type", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - criterion::black_box(rpad().invoke(&args).unwrap()) + // TODO use invoke_with_args + criterion::black_box(rpad().invoke_batch(&args, size).unwrap()) }) }); @@ -147,8 +147,8 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, true); group.bench_function(BenchmarkId::new("stringview type", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - criterion::black_box(rpad().invoke(&args).unwrap()) + // TODO use invoke_with_args + criterion::black_box(rpad().invoke_batch(&args, size).unwrap()) }) }); diff --git a/datafusion/functions/benches/random.rs b/datafusion/functions/benches/random.rs index 5df5d9c7dee2..bc20e0ff11c1 100644 --- a/datafusion/functions/benches/random.rs +++ b/datafusion/functions/benches/random.rs @@ -29,6 +29,7 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("random_1M_rows_batch_8192", |b| { b.iter(|| { for _ in 0..iterations { + #[allow(deprecated)] // TODO: migrate to invoke_with_args black_box(random_func.invoke_batch(&[], 8192).unwrap()); } }) @@ -39,6 +40,7 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("random_1M_rows_batch_128", |b| { b.iter(|| { for _ in 0..iterations_128 { + #[allow(deprecated)] // TODO: migrate to invoke_with_args black_box(random_func.invoke_batch(&[], 128).unwrap()); } }) diff --git a/datafusion/functions/benches/repeat.rs b/datafusion/functions/benches/repeat.rs index 6e54c92b9b26..e7e3c634ea82 100644 --- a/datafusion/functions/benches/repeat.rs +++ b/datafusion/functions/benches/repeat.rs @@ -73,8 +73,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(repeat.invoke(&args)) + // TODO use invoke_with_args + black_box(repeat.invoke_batch(&args, repeat_times as usize)) }) }, ); @@ -87,8 +87,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(repeat.invoke(&args)) + // TODO use invoke_with_args + black_box(repeat.invoke_batch(&args, repeat_times as usize)) }) }, ); @@ -101,8 +101,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(repeat.invoke(&args)) + // TODO use invoke_with_args + black_box(repeat.invoke_batch(&args, repeat_times as usize)) }) }, ); @@ -124,8 +124,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(repeat.invoke(&args)) + // TODO use invoke_with_args + black_box(repeat.invoke_batch(&args, repeat_times as usize)) }) }, ); @@ -138,8 +138,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(repeat.invoke(&args)) + // TODO use invoke_with_args + black_box(repeat.invoke_batch(&args, size)) }) }, ); @@ -152,8 +152,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(repeat.invoke(&args)) + // TODO use invoke_with_args + black_box(repeat.invoke_batch(&args, repeat_times as usize)) }) }, ); diff --git a/datafusion/functions/benches/signum.rs b/datafusion/functions/benches/signum.rs index ea1f5433df4e..a51b2ebe5ab7 100644 --- a/datafusion/functions/benches/signum.rs +++ b/datafusion/functions/benches/signum.rs @@ -30,19 +30,22 @@ fn criterion_benchmark(c: &mut Criterion) { let signum = signum(); for size in [1024, 4096, 8192] { let f32_array = Arc::new(create_primitive_array::(size, 0.2)); + let batch_len = f32_array.len(); let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("signum f32 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(signum.invoke(&f32_args).unwrap()) + // TODO use invoke_with_args + black_box(signum.invoke_batch(&f32_args, batch_len).unwrap()) }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); + let batch_len = f64_array.len(); + let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("signum f64 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(signum.invoke(&f64_args).unwrap()) + // TODO use invoke_with_args + black_box(signum.invoke_batch(&f64_args, batch_len).unwrap()) }) }); } diff --git a/datafusion/functions/benches/strpos.rs b/datafusion/functions/benches/strpos.rs index 31ca61e34c3a..f4962380dfbf 100644 --- a/datafusion/functions/benches/strpos.rs +++ b/datafusion/functions/benches/strpos.rs @@ -114,8 +114,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("strpos_StringArray_ascii_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(strpos.invoke(&args_string_ascii)) + // TODO use invoke_with_args + black_box(strpos.invoke_batch(&args_string_ascii, n_rows)) }) }, ); @@ -126,8 +126,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("strpos_StringArray_utf8_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(strpos.invoke(&args_string_utf8)) + // TODO use invoke_with_args + black_box(strpos.invoke_batch(&args_string_utf8, n_rows)) }) }, ); @@ -138,8 +138,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("strpos_StringViewArray_ascii_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(strpos.invoke(&args_string_view_ascii)) + // TODO use invoke_with_args + black_box(strpos.invoke_batch(&args_string_view_ascii, n_rows)) }) }, ); @@ -150,8 +150,8 @@ fn criterion_benchmark(c: &mut Criterion) { &format!("strpos_StringViewArray_utf8_str_len_{}", str_len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(strpos.invoke(&args_string_view_utf8)) + // TODO use invoke_with_args + black_box(strpos.invoke_batch(&args_string_view_utf8, n_rows)) }) }, ); diff --git a/datafusion/functions/benches/substr.rs b/datafusion/functions/benches/substr.rs index 21020dad31a4..8b8e8dbc4279 100644 --- a/datafusion/functions/benches/substr.rs +++ b/datafusion/functions/benches/substr.rs @@ -109,8 +109,8 @@ fn criterion_benchmark(c: &mut Criterion) { format!("substr_string_view [size={}, strlen={}]", size, len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -120,8 +120,8 @@ fn criterion_benchmark(c: &mut Criterion) { format!("substr_string [size={}, strlen={}]", size, len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -131,8 +131,8 @@ fn criterion_benchmark(c: &mut Criterion) { format!("substr_large_string [size={}, strlen={}]", size, len), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -154,8 +154,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -168,8 +168,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -182,8 +182,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -205,8 +205,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -219,8 +219,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); @@ -233,8 +233,8 @@ fn criterion_benchmark(c: &mut Criterion) { ), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(substr.invoke(&args)) + // TODO use invoke_with_args + black_box(substr.invoke_batch(&args, size)) }) }, ); diff --git a/datafusion/functions/benches/substr_index.rs b/datafusion/functions/benches/substr_index.rs index 1e793cf4db8c..1ea8e2606f0d 100644 --- a/datafusion/functions/benches/substr_index.rs +++ b/datafusion/functions/benches/substr_index.rs @@ -84,16 +84,17 @@ fn data() -> (StringArray, StringArray, Int64Array) { fn criterion_benchmark(c: &mut Criterion) { c.bench_function("substr_index_array_array_1000", |b| { let (strings, delimiters, counts) = data(); + let batch_len = counts.len(); let strings = ColumnarValue::Array(Arc::new(strings) as ArrayRef); let delimiters = ColumnarValue::Array(Arc::new(delimiters) as ArrayRef); let counts = ColumnarValue::Array(Arc::new(counts) as ArrayRef); let args = [strings, delimiters, counts]; b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + #[allow(deprecated)] // TODO: invoke_with_args black_box( substr_index() - .invoke(&args) + .invoke_batch(&args, batch_len) .expect("substr_index should work on valid values"), ) }) diff --git a/datafusion/functions/benches/to_char.rs b/datafusion/functions/benches/to_char.rs index 09032fdf2de1..72eae45b1e1b 100644 --- a/datafusion/functions/benches/to_char.rs +++ b/datafusion/functions/benches/to_char.rs @@ -82,14 +82,16 @@ fn patterns(rng: &mut ThreadRng) -> StringArray { fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_char_array_array_1000", |b| { let mut rng = rand::thread_rng(); - let data = ColumnarValue::Array(Arc::new(data(&mut rng)) as ArrayRef); + let data_arr = data(&mut rng); + let batch_len = data_arr.len(); + let data = ColumnarValue::Array(Arc::new(data_arr) as ArrayRef); let patterns = ColumnarValue::Array(Arc::new(patterns(&mut rng)) as ArrayRef); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_char() - .invoke(&[data.clone(), patterns.clone()]) + .invoke_batch(&[data.clone(), patterns.clone()], batch_len) .expect("to_char should work on valid values"), ) }) @@ -97,15 +99,17 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_char_array_scalar_1000", |b| { let mut rng = rand::thread_rng(); - let data = ColumnarValue::Array(Arc::new(data(&mut rng)) as ArrayRef); + let data_arr = data(&mut rng); + let batch_len = data_arr.len(); + let data = ColumnarValue::Array(Arc::new(data_arr) as ArrayRef); let patterns = ColumnarValue::Scalar(ScalarValue::Utf8(Some("%Y-%m-%d".to_string()))); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_char() - .invoke(&[data.clone(), patterns.clone()]) + .invoke_batch(&[data.clone(), patterns.clone()], batch_len) .expect("to_char should work on valid values"), ) }) @@ -126,10 +130,10 @@ fn criterion_benchmark(c: &mut Criterion) { ))); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_char() - .invoke(&[data.clone(), pattern.clone()]) + .invoke_batch(&[data.clone(), pattern.clone()], 1) .expect("to_char should work on valid values"), ) }) diff --git a/datafusion/functions/benches/to_timestamp.rs b/datafusion/functions/benches/to_timestamp.rs index 11816fe9c64f..9f5f6661f998 100644 --- a/datafusion/functions/benches/to_timestamp.rs +++ b/datafusion/functions/benches/to_timestamp.rs @@ -20,7 +20,7 @@ extern crate criterion; use std::sync::Arc; use arrow::array::builder::StringBuilder; -use arrow::array::{ArrayRef, StringArray}; +use arrow::array::{Array, ArrayRef, StringArray}; use arrow::compute::cast; use arrow::datatypes::DataType; use criterion::{black_box, criterion_group, criterion_main, Criterion}; @@ -110,13 +110,15 @@ fn data_with_formats() -> (StringArray, StringArray, StringArray, StringArray) { } fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_timestamp_no_formats_utf8", |b| { - let string_array = ColumnarValue::Array(Arc::new(data()) as ArrayRef); + let arr_data = data(); + let batch_len = arr_data.len(); + let string_array = ColumnarValue::Array(Arc::new(arr_data) as ArrayRef); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_timestamp() - .invoke(&[string_array.clone()]) + .invoke_batch(&[string_array.clone()], batch_len) .expect("to_timestamp should work on valid values"), ) }) @@ -124,13 +126,14 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_timestamp_no_formats_largeutf8", |b| { let data = cast(&data(), &DataType::LargeUtf8).unwrap(); + let batch_len = data.len(); let string_array = ColumnarValue::Array(Arc::new(data) as ArrayRef); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_timestamp() - .invoke(&[string_array.clone()]) + .invoke_batch(&[string_array.clone()], batch_len) .expect("to_timestamp should work on valid values"), ) }) @@ -138,13 +141,14 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_timestamp_no_formats_utf8view", |b| { let data = cast(&data(), &DataType::Utf8View).unwrap(); + let batch_len = data.len(); let string_array = ColumnarValue::Array(Arc::new(data) as ArrayRef); b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_timestamp() - .invoke(&[string_array.clone()]) + .invoke_batch(&[string_array.clone()], batch_len) .expect("to_timestamp should work on valid values"), ) }) @@ -152,6 +156,7 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_timestamp_with_formats_utf8", |b| { let (inputs, format1, format2, format3) = data_with_formats(); + let batch_len = inputs.len(); let args = [ ColumnarValue::Array(Arc::new(inputs) as ArrayRef), @@ -160,10 +165,10 @@ fn criterion_benchmark(c: &mut Criterion) { ColumnarValue::Array(Arc::new(format3) as ArrayRef), ]; b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_timestamp() - .invoke(&args.clone()) + .invoke_batch(&args.clone(), batch_len) .expect("to_timestamp should work on valid values"), ) }) @@ -171,6 +176,7 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_timestamp_with_formats_largeutf8", |b| { let (inputs, format1, format2, format3) = data_with_formats(); + let batch_len = inputs.len(); let args = [ ColumnarValue::Array( @@ -187,10 +193,10 @@ fn criterion_benchmark(c: &mut Criterion) { ), ]; b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_timestamp() - .invoke(&args.clone()) + .invoke_batch(&args.clone(), batch_len) .expect("to_timestamp should work on valid values"), ) }) @@ -199,6 +205,8 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("to_timestamp_with_formats_utf8view", |b| { let (inputs, format1, format2, format3) = data_with_formats(); + let batch_len = inputs.len(); + let args = [ ColumnarValue::Array( Arc::new(cast(&inputs, &DataType::Utf8View).unwrap()) as ArrayRef @@ -214,10 +222,10 @@ fn criterion_benchmark(c: &mut Criterion) { ), ]; b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch + // TODO use invoke_with_args black_box( to_timestamp() - .invoke(&args.clone()) + .invoke_batch(&args.clone(), batch_len) .expect("to_timestamp should work on valid values"), ) }) diff --git a/datafusion/functions/benches/trunc.rs b/datafusion/functions/benches/trunc.rs index 07ce522eb913..83d5b761e809 100644 --- a/datafusion/functions/benches/trunc.rs +++ b/datafusion/functions/benches/trunc.rs @@ -34,16 +34,16 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("trunc f32 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(trunc.invoke(&f32_args).unwrap()) + // TODO use invoke_with_args + black_box(trunc.invoke_batch(&f32_args, size).unwrap()) }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("trunc f64 array: {}", size), |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(trunc.invoke(&f64_args).unwrap()) + // TODO use invoke_with_args + black_box(trunc.invoke_batch(&f64_args, size).unwrap()) }) }); } diff --git a/datafusion/functions/benches/upper.rs b/datafusion/functions/benches/upper.rs index ac4ecacff941..9b41a15b11c7 100644 --- a/datafusion/functions/benches/upper.rs +++ b/datafusion/functions/benches/upper.rs @@ -38,8 +38,8 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args(size, 32); c.bench_function("upper_all_values_are_ascii", |b| { b.iter(|| { - #[allow(deprecated)] // TODO use invoke_batch - black_box(upper.invoke(&args)) + // TODO use invoke_with_args + black_box(upper.invoke_batch(&args, size)) }) }); } diff --git a/datafusion/functions/src/core/arrow_cast.rs b/datafusion/functions/src/core/arrow_cast.rs index a3e3feaa17e3..3853737d7b5b 100644 --- a/datafusion/functions/src/core/arrow_cast.rs +++ b/datafusion/functions/src/core/arrow_cast.rs @@ -104,7 +104,11 @@ impl ScalarUDFImpl for ArrowCastFunc { data_type_from_args(args) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { internal_err!("arrow_cast should have been simplified to cast") } @@ -143,10 +147,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_arrow_cast_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_OTHER) - .with_description("Casts a value to a specific Arrow data type.") - .with_syntax_example("arrow_cast(expression, datatype)") + Documentation::builder( + DOC_SECTION_OTHER, + "Casts a value to a specific Arrow data type.", + "arrow_cast(expression, datatype)") .with_sql_example( r#"```sql > select arrow_cast(-5, 'Int8') as a, @@ -164,7 +168,6 @@ fn get_arrow_cast_doc() -> &'static Documentation { .with_argument("expression", "Expression to cast. The expression can be a constant, column, or function, and any combination of operators.") .with_argument("datatype", "[Arrow data type](https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html) name to cast to, as a string. The format is the same as that returned by [`arrow_typeof`]") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/arrowtypeof.rs b/datafusion/functions/src/core/arrowtypeof.rs index a425aff6caad..54684c6f0bfb 100644 --- a/datafusion/functions/src/core/arrowtypeof.rs +++ b/datafusion/functions/src/core/arrowtypeof.rs @@ -58,7 +58,11 @@ impl ScalarUDFImpl for ArrowTypeOfFunc { Ok(DataType::Utf8) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 1 { return exec_err!( "arrow_typeof function requires 1 arguments, got {}", @@ -81,12 +85,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_arrowtypeof_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_OTHER) - .with_description( + Documentation::builder( + DOC_SECTION_OTHER, "Returns the name of the underlying [Arrow data type](https://docs.rs/arrow/latest/arrow/datatypes/enum.DataType.html) of the expression.", - ) - .with_syntax_example("arrow_typeof(expression)") + + "arrow_typeof(expression)") .with_sql_example( r#"```sql > select arrow_typeof('foo'), arrow_typeof(1); @@ -100,6 +103,5 @@ fn get_arrowtypeof_doc() -> &'static Documentation { ) .with_argument("expression", "Expression to evaluate. The expression can be a constant, column, or function, and any combination of operators.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/coalesce.rs b/datafusion/functions/src/core/coalesce.rs index a05f3f08232c..4f9e83fbf0d9 100644 --- a/datafusion/functions/src/core/coalesce.rs +++ b/datafusion/functions/src/core/coalesce.rs @@ -74,7 +74,11 @@ impl ScalarUDFImpl for CoalesceFunc { } /// coalesce evaluates to the first value which is not NULL - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { // do not accept 0 arguments. if args.is_empty() { return exec_err!( @@ -150,10 +154,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_coalesce_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_CONDITIONAL) - .with_description("Returns the first of its arguments that is not _null_. Returns _null_ if all arguments are _null_. This function is often used to substitute a default value for _null_ values.") - .with_syntax_example("coalesce(expression1[, ..., expression_n])") + Documentation::builder( + DOC_SECTION_CONDITIONAL, + "Returns the first of its arguments that is not _null_. Returns _null_ if all arguments are _null_. This function is often used to substitute a default value for _null_ values.", + "coalesce(expression1[, ..., expression_n])") .with_sql_example(r#"```sql > select coalesce(null, null, 'datafusion'); +----------------------------------------+ @@ -168,7 +172,6 @@ fn get_coalesce_doc() -> &'static Documentation { "Expression to use if previous expressions are _null_. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary." ) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/getfield.rs b/datafusion/functions/src/core/getfield.rs index c0af4d35966b..5c8e1e803e0f 100644 --- a/datafusion/functions/src/core/getfield.rs +++ b/datafusion/functions/src/core/getfield.rs @@ -160,7 +160,11 @@ impl ScalarUDFImpl for GetFieldFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 2 { return exec_err!( "get_field function requires 2 arguments, got {}", @@ -245,13 +249,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_getfield_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_OTHER) - .with_description(r#"Returns a field within a map or a struct with the given key. + Documentation::builder( + DOC_SECTION_OTHER, + r#"Returns a field within a map or a struct with the given key. Note: most users invoke `get_field` indirectly via field access syntax such as `my_struct_col['field_name']` which results in a call to -`get_field(my_struct_col, 'field_name')`."#) - .with_syntax_example("get_field(expression1, expression2)") +`get_field(my_struct_col, 'field_name')`."#, + "get_field(expression1, expression2)") .with_sql_example(r#"```sql > create table t (idx varchar, v varchar) as values ('data','fusion'), ('apache', 'arrow'); > select struct(idx, v) from t as c; @@ -286,6 +290,5 @@ syntax such as `my_struct_col['field_name']` which results in a call to "The field name in the map or struct to retrieve data for. Must evaluate to a string." ) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/greatest.rs b/datafusion/functions/src/core/greatest.rs new file mode 100644 index 000000000000..3ea2eadf22ee --- /dev/null +++ b/datafusion/functions/src/core/greatest.rs @@ -0,0 +1,272 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use arrow::array::{make_comparator, Array, ArrayRef, BooleanArray}; +use arrow::compute::kernels::cmp; +use arrow::compute::kernels::zip::zip; +use arrow::compute::SortOptions; +use arrow::datatypes::DataType; +use arrow_buffer::BooleanBuffer; +use datafusion_common::{exec_err, plan_err, Result, ScalarValue}; +use datafusion_doc::Documentation; +use datafusion_expr::binary::type_union_resolution; +use datafusion_expr::scalar_doc_sections::DOC_SECTION_CONDITIONAL; +use datafusion_expr::ColumnarValue; +use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; +use std::any::Any; +use std::sync::{Arc, OnceLock}; + +const SORT_OPTIONS: SortOptions = SortOptions { + // We want greatest first + descending: false, + + // NULL will be less than any other value + nulls_first: true, +}; + +#[derive(Debug)] +pub struct GreatestFunc { + signature: Signature, +} + +impl Default for GreatestFunc { + fn default() -> Self { + GreatestFunc::new() + } +} + +impl GreatestFunc { + pub fn new() -> Self { + Self { + signature: Signature::user_defined(Volatility::Immutable), + } + } +} + +fn get_logical_null_count(arr: &dyn Array) -> usize { + arr.logical_nulls() + .map(|n| n.null_count()) + .unwrap_or_default() +} + +/// Return boolean array where `arr[i] = lhs[i] >= rhs[i]` for all i, where `arr` is the result array +/// Nulls are always considered smaller than any other value +fn get_larger(lhs: &dyn Array, rhs: &dyn Array) -> Result { + // Fast path: + // If both arrays are not nested, have the same length and no nulls, we can use the faster vectorised kernel + // - If both arrays are not nested: Nested types, such as lists, are not supported as the null semantics are not well-defined. + // - both array does not have any nulls: cmp::gt_eq will return null if any of the input is null while we want to return false in that case + if !lhs.data_type().is_nested() + && get_logical_null_count(lhs) == 0 + && get_logical_null_count(rhs) == 0 + { + return cmp::gt_eq(&lhs, &rhs).map_err(|e| e.into()); + } + + let cmp = make_comparator(lhs, rhs, SORT_OPTIONS)?; + + if lhs.len() != rhs.len() { + return exec_err!( + "All arrays should have the same length for greatest comparison" + ); + } + + let values = BooleanBuffer::collect_bool(lhs.len(), |i| cmp(i, i).is_ge()); + + // No nulls as we only want to keep the values that are larger, its either true or false + Ok(BooleanArray::new(values, None)) +} + +/// Return array where the largest value at each index is kept +fn keep_larger(lhs: ArrayRef, rhs: ArrayRef) -> Result { + // True for values that we should keep from the left array + let keep_lhs = get_larger(lhs.as_ref(), rhs.as_ref())?; + + let larger = zip(&keep_lhs, &lhs, &rhs)?; + + Ok(larger) +} + +fn keep_larger_scalar<'a>( + lhs: &'a ScalarValue, + rhs: &'a ScalarValue, +) -> Result<&'a ScalarValue> { + if !lhs.data_type().is_nested() { + return if lhs >= rhs { Ok(lhs) } else { Ok(rhs) }; + } + + // If complex type we can't compare directly as we want null values to be smaller + let cmp = make_comparator( + lhs.to_array()?.as_ref(), + rhs.to_array()?.as_ref(), + SORT_OPTIONS, + )?; + + if cmp(0, 0).is_ge() { + Ok(lhs) + } else { + Ok(rhs) + } +} + +fn find_coerced_type(data_types: &[DataType]) -> Result { + if data_types.is_empty() { + plan_err!("greatest was called without any arguments. It requires at least 1.") + } else if let Some(coerced_type) = type_union_resolution(data_types) { + Ok(coerced_type) + } else { + plan_err!("Cannot find a common type for arguments") + } +} + +impl ScalarUDFImpl for GreatestFunc { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "greatest" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, arg_types: &[DataType]) -> Result { + Ok(arg_types[0].clone()) + } + + fn invoke(&self, args: &[ColumnarValue]) -> Result { + if args.is_empty() { + return exec_err!( + "greatest was called with no arguments. It requires at least 1." + ); + } + + // Some engines (e.g. SQL Server) allow greatest with single arg, it's a noop + if args.len() == 1 { + return Ok(args[0].clone()); + } + + // Split to scalars and arrays for later optimization + let (scalars, arrays): (Vec<_>, Vec<_>) = args.iter().partition(|x| match x { + ColumnarValue::Scalar(_) => true, + ColumnarValue::Array(_) => false, + }); + + let mut arrays_iter = arrays.iter().map(|x| match x { + ColumnarValue::Array(a) => a, + _ => unreachable!(), + }); + + let first_array = arrays_iter.next(); + + let mut largest: ArrayRef; + + // Optimization: merge all scalars into one to avoid recomputing + if !scalars.is_empty() { + let mut scalars_iter = scalars.iter().map(|x| match x { + ColumnarValue::Scalar(s) => s, + _ => unreachable!(), + }); + + // We have at least one scalar + let mut largest_scalar = scalars_iter.next().unwrap(); + + for scalar in scalars_iter { + largest_scalar = keep_larger_scalar(largest_scalar, scalar)?; + } + + // If we only have scalars, return the largest one + if arrays.is_empty() { + return Ok(ColumnarValue::Scalar(largest_scalar.clone())); + } + + // We have at least one array + let first_array = first_array.unwrap(); + + // Start with the largest value + largest = keep_larger( + Arc::clone(first_array), + largest_scalar.to_array_of_size(first_array.len())?, + )?; + } else { + // If we only have arrays, start with the first array + // (We must have at least one array) + largest = Arc::clone(first_array.unwrap()); + } + + for array in arrays_iter { + largest = keep_larger(Arc::clone(array), largest)?; + } + + Ok(ColumnarValue::Array(largest)) + } + + fn coerce_types(&self, arg_types: &[DataType]) -> Result> { + let coerced_type = find_coerced_type(arg_types)?; + + Ok(vec![coerced_type; arg_types.len()]) + } + + fn documentation(&self) -> Option<&Documentation> { + Some(get_greatest_doc()) + } +} +static DOCUMENTATION: OnceLock = OnceLock::new(); + +fn get_greatest_doc() -> &'static Documentation { + DOCUMENTATION.get_or_init(|| { + Documentation::builder( + DOC_SECTION_CONDITIONAL, + "Returns the greatest value in a list of expressions. Returns _null_ if all expressions are _null_.", + "greatest(expression1[, ..., expression_n])") + .with_sql_example(r#"```sql +> select greatest(4, 7, 5); ++---------------------------+ +| greatest(4,7,5) | ++---------------------------+ +| 7 | ++---------------------------+ +```"#, + ) + .with_argument( + "expression1, expression_n", + "Expressions to compare and return the greatest value.. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary." + ) + .build() + }) +} + +#[cfg(test)] +mod test { + use crate::core; + use arrow::datatypes::DataType; + use datafusion_expr::ScalarUDFImpl; + + #[test] + fn test_greatest_return_types_without_common_supertype_in_arg_type() { + let greatest = core::greatest::GreatestFunc::new(); + let return_type = greatest + .coerce_types(&[DataType::Decimal128(10, 3), DataType::Decimal128(10, 4)]) + .unwrap(); + assert_eq!( + return_type, + vec![DataType::Decimal128(11, 4), DataType::Decimal128(11, 4)] + ); + } +} diff --git a/datafusion/functions/src/core/mod.rs b/datafusion/functions/src/core/mod.rs index cf64c03766cb..24d26c539539 100644 --- a/datafusion/functions/src/core/mod.rs +++ b/datafusion/functions/src/core/mod.rs @@ -25,6 +25,7 @@ pub mod arrowtypeof; pub mod coalesce; pub mod expr_ext; pub mod getfield; +pub mod greatest; pub mod named_struct; pub mod nullif; pub mod nvl; @@ -43,6 +44,7 @@ make_udf_function!(r#struct::StructFunc, STRUCT, r#struct); make_udf_function!(named_struct::NamedStructFunc, NAMED_STRUCT, named_struct); make_udf_function!(getfield::GetFieldFunc, GET_FIELD, get_field); make_udf_function!(coalesce::CoalesceFunc, COALESCE, coalesce); +make_udf_function!(greatest::GreatestFunc, GREATEST, greatest); make_udf_function!(version::VersionFunc, VERSION, version); pub mod expr_fn { @@ -80,6 +82,10 @@ pub mod expr_fn { coalesce, "Returns `coalesce(args...)`, which evaluates to the value of the first expr which is not NULL", args, + ),( + greatest, + "Returns `greatest(args...)`, which evaluates to the greatest value in the list of expressions or NULL if all the expressions are NULL", + args, )); #[doc = "Returns the value of the field with the given name from the struct"] @@ -106,6 +112,7 @@ pub fn functions() -> Vec> { // calls to `get_field` get_field(), coalesce(), + greatest(), version(), r#struct(), ] diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index 0211ed3fe691..556cad1de1ac 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -158,7 +158,11 @@ impl ScalarUDFImpl for NamedStructFunc { Ok(DataType::Struct(Fields::from(return_fields))) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { named_struct_expr(args) } @@ -171,10 +175,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_named_struct_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRUCT) - .with_description("Returns an Arrow struct using the specified name and input expressions pairs.") - .with_syntax_example("named_struct(expression1_name, expression1_input[, ..., expression_n_name, expression_n_input])") + Documentation::builder( + DOC_SECTION_STRUCT, + "Returns an Arrow struct using the specified name and input expressions pairs.", + "named_struct(expression1_name, expression1_input[, ..., expression_n_name, expression_n_input])") .with_sql_example(r#" For example, this query converts two columns `a` and `b` to a single column with a struct type of fields `field_a` and `field_b`: @@ -201,6 +205,5 @@ a struct type of fields `field_a` and `field_b`: ) .with_argument("expression_n_input", "Expression to include in the output struct. Can be a constant, column, or function, and any combination of arithmetic or string operators.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/nullif.rs b/datafusion/functions/src/core/nullif.rs index 801a80201946..0c2d01376de9 100644 --- a/datafusion/functions/src/core/nullif.rs +++ b/datafusion/functions/src/core/nullif.rs @@ -32,26 +32,6 @@ pub struct NullIfFunc { signature: Signature, } -/// Currently supported types by the nullif function. -/// The order of these types correspond to the order on which coercion applies -/// This should thus be from least informative to most informative -static SUPPORTED_NULLIF_TYPES: &[DataType] = &[ - DataType::Boolean, - DataType::UInt8, - DataType::UInt16, - DataType::UInt32, - DataType::UInt64, - DataType::Int8, - DataType::Int16, - DataType::Int32, - DataType::Int64, - DataType::Float32, - DataType::Float64, - DataType::Utf8View, - DataType::Utf8, - DataType::LargeUtf8, -]; - impl Default for NullIfFunc { fn default() -> Self { Self::new() @@ -61,11 +41,20 @@ impl Default for NullIfFunc { impl NullIfFunc { pub fn new() -> Self { Self { - signature: Signature::uniform( - 2, - SUPPORTED_NULLIF_TYPES.to_vec(), - Volatility::Immutable, - ), + // Documentation mentioned in Postgres, + // The result has the same type as the first argument — but there is a subtlety. + // What is actually returned is the first argument of the implied = operator, + // and in some cases that will have been promoted to match the second argument's type. + // For example, NULLIF(1, 2.2) yields numeric, because there is no integer = numeric operator, only numeric = numeric + // + // We don't strictly follow Postgres or DuckDB for **simplicity**. + // In this function, we will coerce arguments to the same data type for comparison need. Unlike DuckDB + // we don't return the **original** first argument type but return the final coerced type. + // + // In Postgres, nullif('2', 2) returns Null but nullif('2::varchar', 2) returns error. + // While in DuckDB both query returns Null. We follow DuckDB in this case since I think they are equivalent thing and should + // have the same result as well. + signature: Signature::comparable(2, Volatility::Immutable), } } } @@ -83,17 +72,14 @@ impl ScalarUDFImpl for NullIfFunc { } fn return_type(&self, arg_types: &[DataType]) -> Result { - // NULLIF has two args and they might get coerced, get a preview of this - let coerced_types = datafusion_expr::type_coercion::functions::data_types( - arg_types, - &self.signature, - ); - coerced_types - .map(|typs| typs[0].clone()) - .map_err(|e| e.context("Failed to coerce arguments for NULLIF")) + Ok(arg_types[0].to_owned()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { nullif_func(args) } @@ -106,11 +92,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_nullif_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_CONDITIONAL) - .with_description("Returns _null_ if _expression1_ equals _expression2_; otherwise it returns _expression1_. -This can be used to perform the inverse operation of [`coalesce`](#coalesce).") - .with_syntax_example("nullif(expression1, expression2)") + Documentation::builder( + DOC_SECTION_CONDITIONAL, + "Returns _null_ if _expression1_ equals _expression2_; otherwise it returns _expression1_. +This can be used to perform the inverse operation of [`coalesce`](#coalesce).", + "nullif(expression1, expression2)") .with_sql_example(r#"```sql > select nullif('datafusion', 'data'); +-----------------------------------------+ @@ -135,7 +121,6 @@ This can be used to perform the inverse operation of [`coalesce`](#coalesce).") "Expression to compare to expression1. Can be a constant, column, or function, and any combination of operators." ) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/nvl.rs b/datafusion/functions/src/core/nvl.rs index 24b6f5fc14fe..6c470eca3d46 100644 --- a/datafusion/functions/src/core/nvl.rs +++ b/datafusion/functions/src/core/nvl.rs @@ -88,7 +88,11 @@ impl ScalarUDFImpl for NVLFunc { Ok(arg_types[0].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { nvl_func(args) } @@ -105,10 +109,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_nvl_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_CONDITIONAL) - .with_description("Returns _expression2_ if _expression1_ is NULL otherwise it returns _expression1_.") - .with_syntax_example("nvl(expression1, expression2)") + Documentation::builder( + DOC_SECTION_CONDITIONAL, + "Returns _expression2_ if _expression1_ is NULL otherwise it returns _expression1_.", + "nvl(expression1, expression2)") .with_sql_example(r#"```sql > select nvl(null, 'a'); +---------------------+ @@ -133,7 +137,6 @@ fn get_nvl_doc() -> &'static Documentation { "Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators." ) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/nvl2.rs b/datafusion/functions/src/core/nvl2.rs index cfcdb4480787..a37292683260 100644 --- a/datafusion/functions/src/core/nvl2.rs +++ b/datafusion/functions/src/core/nvl2.rs @@ -63,7 +63,11 @@ impl ScalarUDFImpl for NVL2Func { Ok(arg_types[1].clone()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { nvl2_func(args) } @@ -101,10 +105,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_nvl2_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_CONDITIONAL) - .with_description("Returns _expression2_ if _expression1_ is not NULL; otherwise it returns _expression3_.") - .with_syntax_example("nvl2(expression1, expression2, expression3)") + Documentation::builder( + DOC_SECTION_CONDITIONAL, + "Returns _expression2_ if _expression1_ is not NULL; otherwise it returns _expression3_.", + "nvl2(expression1, expression2, expression3)") .with_sql_example(r#"```sql > select nvl2(null, 'a', 'b'); +--------------------------------+ @@ -133,7 +137,6 @@ fn get_nvl2_doc() -> &'static Documentation { "Expression to return if expr1 is null. Can be a constant, column, or function, and any combination of operators." ) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/struct.rs b/datafusion/functions/src/core/struct.rs index 75d1d4eca698..b7579ba5e4a4 100644 --- a/datafusion/functions/src/core/struct.rs +++ b/datafusion/functions/src/core/struct.rs @@ -101,7 +101,11 @@ impl ScalarUDFImpl for StructFunc { Ok(DataType::Struct(Fields::from(return_fields))) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { struct_expr(args) } @@ -114,12 +118,12 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_struct_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRUCT) - .with_description("Returns an Arrow struct using the specified input expressions optionally named. + Documentation::builder( + DOC_SECTION_STRUCT, + "Returns an Arrow struct using the specified input expressions optionally named. Fields in the returned struct use the optional name or the `cN` naming convention. -For example: `c0`, `c1`, `c2`, etc.") - .with_syntax_example("struct(expression1[, ..., expression_n])") +For example: `c0`, `c1`, `c2`, etc.", + "struct(expression1[, ..., expression_n])") .with_sql_example(r#"For example, this query converts two columns `a` and `b` to a single column with a struct type of fields `field_a` and `c1`: ```sql @@ -154,6 +158,5 @@ select struct(a as field_a, b) from t; "expression1, expression_n", "Expression to include in the output struct. Can be a constant, column, or function, any combination of arithmetic or string operators.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/core/version.rs b/datafusion/functions/src/core/version.rs index 36cf07e9e5da..bfc87f28ebeb 100644 --- a/datafusion/functions/src/core/version.rs +++ b/datafusion/functions/src/core/version.rs @@ -94,12 +94,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_version_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_OTHER) - .with_description("Returns the version of DataFusion.") - .with_syntax_example("version()") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_OTHER, + "Returns the version of DataFusion.", + "version()", + ) + .with_sql_example( + r#"```sql > select version(); +--------------------------------------------+ | version() | @@ -107,9 +108,8 @@ fn get_version_doc() -> &'static Documentation { | Apache DataFusion 42.0.0, aarch64 on macos | +--------------------------------------------+ ```"#, - ) - .build() - .unwrap() + ) + .build() }) } @@ -121,6 +121,7 @@ mod test { #[tokio::test] async fn test_version_udf() { let version_udf = ScalarUDF::from(VersionFunc::new()); + #[allow(deprecated)] // TODO: migrate to invoke_with_args let version = version_udf.invoke_batch(&[], 1).unwrap(); if let ColumnarValue::Scalar(ScalarValue::Utf8(Some(version))) = version { diff --git a/datafusion/functions/src/crypto/digest.rs b/datafusion/functions/src/crypto/digest.rs index f738c6e3e40f..0d2d130cdd71 100644 --- a/datafusion/functions/src/crypto/digest.rs +++ b/datafusion/functions/src/crypto/digest.rs @@ -69,7 +69,11 @@ impl ScalarUDFImpl for DigestFunc { fn return_type(&self, arg_types: &[DataType]) -> Result { utf8_or_binary_to_binary_type(&arg_types[0], self.name()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { digest(args) } @@ -82,14 +86,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_digest_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_HASHING) - .with_description( - "Computes the binary hash of an expression using the specified algorithm.", - ) - .with_syntax_example("digest(expression, algorithm)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_HASHING, + "Computes the binary hash of an expression using the specified algorithm.", + "digest(expression, algorithm)", + ) + .with_sql_example( + r#"```sql > select digest('foo', 'sha256'); +------------------------------------------+ | digest(Utf8("foo"), Utf8("sha256")) | @@ -97,12 +100,11 @@ fn get_digest_doc() -> &'static Documentation { | | +------------------------------------------+ ```"#, - ) - .with_standard_argument( - "expression", Some("String")) - .with_argument( - "algorithm", - "String expression specifying algorithm to use. Must be one of: + ) + .with_standard_argument("expression", Some("String")) + .with_argument( + "algorithm", + "String expression specifying algorithm to use. Must be one of: - md5 - sha224 @@ -112,8 +114,7 @@ fn get_digest_doc() -> &'static Documentation { - blake2s - blake2b - blake3", - ) - .build() - .unwrap() + ) + .build() }) } diff --git a/datafusion/functions/src/crypto/md5.rs b/datafusion/functions/src/crypto/md5.rs index 0e8ff1cd3192..e6cc59a4a4f7 100644 --- a/datafusion/functions/src/crypto/md5.rs +++ b/datafusion/functions/src/crypto/md5.rs @@ -85,7 +85,11 @@ impl ScalarUDFImpl for Md5Func { } }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { md5(args) } @@ -98,12 +102,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_md5_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_HASHING) - .with_description("Computes an MD5 128-bit checksum for a string expression.") - .with_syntax_example("md5(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_HASHING, + "Computes an MD5 128-bit checksum for a string expression.", + "md5(expression)", + ) + .with_sql_example( + r#"```sql > select md5('foo'); +-------------------------------------+ | md5(Utf8("foo")) | @@ -111,9 +116,8 @@ fn get_md5_doc() -> &'static Documentation { | | +-------------------------------------+ ```"#, - ) - .with_standard_argument("expression", Some("String")) - .build() - .unwrap() + ) + .with_standard_argument("expression", Some("String")) + .build() }) } diff --git a/datafusion/functions/src/crypto/sha224.rs b/datafusion/functions/src/crypto/sha224.rs index f0bfcb9fab3b..eba22bb3de37 100644 --- a/datafusion/functions/src/crypto/sha224.rs +++ b/datafusion/functions/src/crypto/sha224.rs @@ -54,12 +54,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_sha224_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_HASHING) - .with_description("Computes the SHA-224 hash of a binary string.") - .with_syntax_example("sha224(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_HASHING, + "Computes the SHA-224 hash of a binary string.", + "sha224(expression)", + ) + .with_sql_example( + r#"```sql > select sha224('foo'); +------------------------------------------+ | sha224(Utf8("foo")) | @@ -67,10 +68,9 @@ fn get_sha224_doc() -> &'static Documentation { | | +------------------------------------------+ ```"#, - ) - .with_standard_argument("expression", Some("String")) - .build() - .unwrap() + ) + .with_standard_argument("expression", Some("String")) + .build() }) } @@ -91,7 +91,11 @@ impl ScalarUDFImpl for SHA224Func { utf8_or_binary_to_binary_type(&arg_types[0], self.name()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { sha224(args) } diff --git a/datafusion/functions/src/crypto/sha256.rs b/datafusion/functions/src/crypto/sha256.rs index 0a0044f72206..9343fa0af942 100644 --- a/datafusion/functions/src/crypto/sha256.rs +++ b/datafusion/functions/src/crypto/sha256.rs @@ -65,7 +65,11 @@ impl ScalarUDFImpl for SHA256Func { utf8_or_binary_to_binary_type(&arg_types[0], self.name()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { sha256(args) } @@ -78,12 +82,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_sha256_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_HASHING) - .with_description("Computes the SHA-256 hash of a binary string.") - .with_syntax_example("sha256(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_HASHING, + "Computes the SHA-256 hash of a binary string.", + "sha256(expression)", + ) + .with_sql_example( + r#"```sql > select sha256('foo'); +--------------------------------------+ | sha256(Utf8("foo")) | @@ -91,9 +96,8 @@ fn get_sha256_doc() -> &'static Documentation { | | +--------------------------------------+ ```"#, - ) - .with_standard_argument("expression", Some("String")) - .build() - .unwrap() + ) + .with_standard_argument("expression", Some("String")) + .build() }) } diff --git a/datafusion/functions/src/crypto/sha384.rs b/datafusion/functions/src/crypto/sha384.rs index 7f8220e5f9d5..495036d02474 100644 --- a/datafusion/functions/src/crypto/sha384.rs +++ b/datafusion/functions/src/crypto/sha384.rs @@ -65,7 +65,11 @@ impl ScalarUDFImpl for SHA384Func { utf8_or_binary_to_binary_type(&arg_types[0], self.name()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { sha384(args) } @@ -78,12 +82,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_sha384_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_HASHING) - .with_description("Computes the SHA-384 hash of a binary string.") - .with_syntax_example("sha384(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_HASHING, + "Computes the SHA-384 hash of a binary string.", + "sha384(expression)", + ) + .with_sql_example( + r#"```sql > select sha384('foo'); +-----------------------------------------+ | sha384(Utf8("foo")) | @@ -91,9 +96,8 @@ fn get_sha384_doc() -> &'static Documentation { | | +-----------------------------------------+ ```"#, - ) - .with_standard_argument("expression", Some("String")) - .build() - .unwrap() + ) + .with_standard_argument("expression", Some("String")) + .build() }) } diff --git a/datafusion/functions/src/crypto/sha512.rs b/datafusion/functions/src/crypto/sha512.rs index d2d51bfa53ab..7d454ff8da35 100644 --- a/datafusion/functions/src/crypto/sha512.rs +++ b/datafusion/functions/src/crypto/sha512.rs @@ -65,7 +65,11 @@ impl ScalarUDFImpl for SHA512Func { utf8_or_binary_to_binary_type(&arg_types[0], self.name()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { sha512(args) } @@ -78,12 +82,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_sha512_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_HASHING) - .with_description("Computes the SHA-512 hash of a binary string.") - .with_syntax_example("sha512(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_HASHING, + "Computes the SHA-512 hash of a binary string.", + "sha512(expression)", + ) + .with_sql_example( + r#"```sql > select sha512('foo'); +-------------------------------------------+ | sha512(Utf8("foo")) | @@ -91,9 +96,8 @@ fn get_sha512_doc() -> &'static Documentation { | | +-------------------------------------------+ ```"#, - ) - .with_argument("expression", "String") - .build() - .unwrap() + ) + .with_argument("expression", "String") + .build() }) } diff --git a/datafusion/functions/src/datetime/current_date.rs b/datafusion/functions/src/datetime/current_date.rs index 3b819c470d1e..97d97939d329 100644 --- a/datafusion/functions/src/datetime/current_date.rs +++ b/datafusion/functions/src/datetime/current_date.rs @@ -73,7 +73,11 @@ impl ScalarUDFImpl for CurrentDateFunc { Ok(Date32) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { internal_err!( "invoke should not be called on a simplified current_date() function" ) @@ -109,15 +113,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_current_date_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description(r#" + Documentation::builder( + DOC_SECTION_DATETIME, + r#" Returns the current UTC date. The `current_date()` return value is determined at query time and will return the same date, no matter when in the query plan the function executes. -"#) - .with_syntax_example("current_date()") +"#, + "current_date()") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/datetime/current_time.rs b/datafusion/functions/src/datetime/current_time.rs index ca591f922305..1cd39e5777ea 100644 --- a/datafusion/functions/src/datetime/current_time.rs +++ b/datafusion/functions/src/datetime/current_time.rs @@ -70,7 +70,11 @@ impl ScalarUDFImpl for CurrentTimeFunc { Ok(Time64(Nanosecond)) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { internal_err!( "invoke should not be called on a simplified current_time() function" ) @@ -97,15 +101,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_current_time_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description(r#" + Documentation::builder( + DOC_SECTION_DATETIME, + r#" Returns the current UTC time. The `current_time()` return value is determined at query time and will return the same time, no matter when in the query plan the function executes. -"#) - .with_syntax_example("current_time()") +"#, + "current_time()") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/datetime/date_bin.rs b/datafusion/functions/src/datetime/date_bin.rs index 671967a89325..49bf00d5c17b 100644 --- a/datafusion/functions/src/datetime/date_bin.rs +++ b/datafusion/functions/src/datetime/date_bin.rs @@ -133,7 +133,11 @@ impl ScalarUDFImpl for DateBinFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() == 2 { // Default to unix EPOCH let origin = ColumnarValue::Scalar(ScalarValue::TimestampNanosecond( @@ -173,14 +177,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_date_bin_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description(r#" + Documentation::builder( + DOC_SECTION_DATETIME, + r#" Calculates time intervals and returns the start of the interval nearest to the specified timestamp. Use `date_bin` to downsample time series data by grouping rows into time-based "bins" or "windows" and applying an aggregate or selector function to each window. For example, if you "bin" or "window" data into 15 minute intervals, an input timestamp of `2023-01-01T18:18:18Z` will be updated to the start time of the 15 minute bin it is in: `2023-01-01T18:15:00Z`. -"#) - .with_syntax_example("date_bin(interval, expression, origin-timestamp)") +"#, + "date_bin(interval, expression, origin-timestamp)") .with_sql_example(r#"```sql -- Bin the timestamp into 1 day intervals > SELECT date_bin(interval '1 day', time) as bin @@ -224,7 +228,6 @@ The following intervals are supported: - century ") .build() - .unwrap() }) } @@ -515,6 +518,7 @@ mod tests { use chrono::TimeDelta; #[test] + #[allow(deprecated)] // TODO migrate UDF invoke from invoke_batch fn test_date_bin() { let res = DateBinFunc::new().invoke_batch( &[ @@ -532,7 +536,7 @@ mod tests { assert!(res.is_ok()); let timestamps = Arc::new((1..6).map(Some).collect::()); - let batch_size = timestamps.len(); + let batch_len = timestamps.len(); let res = DateBinFunc::new().invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::IntervalDayTime(Some( @@ -544,7 +548,7 @@ mod tests { ColumnarValue::Array(timestamps), ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), ], - batch_size, + batch_len, ); assert!(res.is_ok()); @@ -720,14 +724,13 @@ mod tests { }) .collect::(), ); - let batch_size = intervals.len(); let res = DateBinFunc::new().invoke_batch( &[ ColumnarValue::Array(intervals), ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), ], - batch_size, + 1, ); assert_eq!( res.err().unwrap().strip_backtrace(), @@ -736,7 +739,7 @@ mod tests { // unsupported array type for origin let timestamps = Arc::new((1..6).map(Some).collect::()); - let batch_size = timestamps.len(); + let batch_len = timestamps.len(); let res = DateBinFunc::new().invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::IntervalDayTime(Some( @@ -748,7 +751,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), ColumnarValue::Array(timestamps), ], - batch_size, + batch_len, ); assert_eq!( res.err().unwrap().strip_backtrace(), @@ -864,7 +867,8 @@ mod tests { .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::() .with_timezone_opt(tz_opt.clone()); - let batch_size = input.len(); + let batch_len = input.len(); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = DateBinFunc::new() .invoke_batch( &[ @@ -875,7 +879,7 @@ mod tests { tz_opt.clone(), )), ], - batch_size, + batch_len, ) .unwrap(); if let ColumnarValue::Array(result) = result { diff --git a/datafusion/functions/src/datetime/date_part.rs b/datafusion/functions/src/datetime/date_part.rs index 01e094bc4e0b..b8c58a11d999 100644 --- a/datafusion/functions/src/datetime/date_part.rs +++ b/datafusion/functions/src/datetime/date_part.rs @@ -19,12 +19,11 @@ use std::any::Any; use std::str::FromStr; use std::sync::{Arc, OnceLock}; -use arrow::array::{Array, ArrayRef, Float64Array}; +use arrow::array::{Array, ArrayRef, Float64Array, Int32Array}; use arrow::compute::kernels::cast_utils::IntervalUnit; -use arrow::compute::{binary, cast, date_part, DatePart}; +use arrow::compute::{binary, date_part, DatePart}; use arrow::datatypes::DataType::{ - Date32, Date64, Duration, Float64, Interval, Time32, Time64, Timestamp, Utf8, - Utf8View, + Date32, Date64, Duration, Interval, Time32, Time64, Timestamp, Utf8, Utf8View, }; use arrow::datatypes::IntervalUnit::{DayTime, MonthDayNano, YearMonth}; use arrow::datatypes::TimeUnit::{Microsecond, Millisecond, Nanosecond, Second}; @@ -36,11 +35,12 @@ use datafusion_common::cast::{ as_timestamp_microsecond_array, as_timestamp_millisecond_array, as_timestamp_nanosecond_array, as_timestamp_second_array, }; -use datafusion_common::{exec_err, Result, ScalarValue}; +use datafusion_common::{exec_err, internal_err, ExprSchema, Result, ScalarValue}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::TypeSignature::Exact; use datafusion_expr::{ - ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, TIMEZONE_WILDCARD, + ColumnarValue, Documentation, Expr, ScalarUDFImpl, Signature, Volatility, + TIMEZONE_WILDCARD, }; #[derive(Debug)] @@ -148,10 +148,28 @@ impl ScalarUDFImpl for DatePartFunc { } fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(Float64) + internal_err!("return_type_from_exprs shoud be called instead") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn return_type_from_exprs( + &self, + args: &[Expr], + _schema: &dyn ExprSchema, + _arg_types: &[DataType], + ) -> Result { + match &args[0] { + Expr::Literal(ScalarValue::Utf8(Some(part))) if is_epoch(part) => { + Ok(DataType::Float64) + } + _ => Ok(DataType::Int32), + } + } + + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 2 { return exec_err!("Expected two arguments in DATE_PART"); } @@ -174,35 +192,31 @@ impl ScalarUDFImpl for DatePartFunc { ColumnarValue::Scalar(scalar) => scalar.to_array()?, }; - // to remove quotes at most 2 characters - let part_trim = part.trim_matches(|c| c == '\'' || c == '\"'); - if ![2, 0].contains(&(part.len() - part_trim.len())) { - return exec_err!("Date part '{part}' not supported"); - } + let part_trim = part_normalization(part); // using IntervalUnit here means we hand off all the work of supporting plurals (like "seconds") // and synonyms ( like "ms,msec,msecond,millisecond") to Arrow let arr = if let Ok(interval_unit) = IntervalUnit::from_str(part_trim) { match interval_unit { - IntervalUnit::Year => date_part_f64(array.as_ref(), DatePart::Year)?, - IntervalUnit::Month => date_part_f64(array.as_ref(), DatePart::Month)?, - IntervalUnit::Week => date_part_f64(array.as_ref(), DatePart::Week)?, - IntervalUnit::Day => date_part_f64(array.as_ref(), DatePart::Day)?, - IntervalUnit::Hour => date_part_f64(array.as_ref(), DatePart::Hour)?, - IntervalUnit::Minute => date_part_f64(array.as_ref(), DatePart::Minute)?, - IntervalUnit::Second => seconds(array.as_ref(), Second)?, - IntervalUnit::Millisecond => seconds(array.as_ref(), Millisecond)?, - IntervalUnit::Microsecond => seconds(array.as_ref(), Microsecond)?, - IntervalUnit::Nanosecond => seconds(array.as_ref(), Nanosecond)?, + IntervalUnit::Year => date_part(array.as_ref(), DatePart::Year)?, + IntervalUnit::Month => date_part(array.as_ref(), DatePart::Month)?, + IntervalUnit::Week => date_part(array.as_ref(), DatePart::Week)?, + IntervalUnit::Day => date_part(array.as_ref(), DatePart::Day)?, + IntervalUnit::Hour => date_part(array.as_ref(), DatePart::Hour)?, + IntervalUnit::Minute => date_part(array.as_ref(), DatePart::Minute)?, + IntervalUnit::Second => seconds_as_i32(array.as_ref(), Second)?, + IntervalUnit::Millisecond => seconds_as_i32(array.as_ref(), Millisecond)?, + IntervalUnit::Microsecond => seconds_as_i32(array.as_ref(), Microsecond)?, + IntervalUnit::Nanosecond => seconds_as_i32(array.as_ref(), Nanosecond)?, // century and decade are not supported by `DatePart`, although they are supported in postgres _ => return exec_err!("Date part '{part}' not supported"), } } else { // special cases that can be extracted (in postgres) but are not interval units match part_trim.to_lowercase().as_str() { - "qtr" | "quarter" => date_part_f64(array.as_ref(), DatePart::Quarter)?, - "doy" => date_part_f64(array.as_ref(), DatePart::DayOfYear)?, - "dow" => date_part_f64(array.as_ref(), DatePart::DayOfWeekSunday0)?, + "qtr" | "quarter" => date_part(array.as_ref(), DatePart::Quarter)?, + "doy" => date_part(array.as_ref(), DatePart::DayOfYear)?, + "dow" => date_part(array.as_ref(), DatePart::DayOfWeekSunday0)?, "epoch" => epoch(array.as_ref())?, _ => return exec_err!("Date part '{part}' not supported"), } @@ -223,14 +237,26 @@ impl ScalarUDFImpl for DatePartFunc { } } +fn is_epoch(part: &str) -> bool { + let part = part_normalization(part); + matches!(part.to_lowercase().as_str(), "epoch") +} + +// Try to remove quote if exist, if the quote is invalid, return original string and let the downstream function handle the error +fn part_normalization(part: &str) -> &str { + part.strip_prefix(|c| c == '\'' || c == '\"') + .and_then(|s| s.strip_suffix(|c| c == '\'' || c == '\"')) + .unwrap_or(part) +} + static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_date_part_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Returns the specified part of the date as an integer.") - .with_syntax_example("date_part(part, expression)") + Documentation::builder( + DOC_SECTION_DATETIME, + "Returns the specified part of the date as an integer.", + "date_part(part, expression)") .with_argument( "part", r#"Part of the date to return. The following date parts are supported: @@ -257,18 +283,66 @@ fn get_date_part_doc() -> &'static Documentation { ) .with_alternative_syntax("extract(field FROM source)") .build() - .unwrap() }) } -/// Invoke [`date_part`] and cast the result to Float64 -fn date_part_f64(array: &dyn Array, part: DatePart) -> Result { - Ok(cast(date_part(array, part)?.as_ref(), &Float64)?) +/// Invoke [`date_part`] on an `array` (e.g. Timestamp) and convert the +/// result to a total number of seconds, milliseconds, microseconds or +/// nanoseconds +fn seconds_as_i32(array: &dyn Array, unit: TimeUnit) -> Result { + // Nanosecond is neither supported in Postgres nor DuckDB, to avoid to deal with overflow and precision issue we don't support nanosecond + if unit == Nanosecond { + return internal_err!("unit {unit:?} not supported"); + } + + let conversion_factor = match unit { + Second => 1_000_000_000, + Millisecond => 1_000_000, + Microsecond => 1_000, + Nanosecond => 1, + }; + + let second_factor = match unit { + Second => 1, + Millisecond => 1_000, + Microsecond => 1_000_000, + Nanosecond => 1_000_000_000, + }; + + let secs = date_part(array, DatePart::Second)?; + // This assumes array is primitive and not a dictionary + let secs = as_int32_array(secs.as_ref())?; + let subsecs = date_part(array, DatePart::Nanosecond)?; + let subsecs = as_int32_array(subsecs.as_ref())?; + + // Special case where there are no nulls. + if subsecs.null_count() == 0 { + let r: Int32Array = binary(secs, subsecs, |secs, subsecs| { + secs * second_factor + (subsecs % 1_000_000_000) / conversion_factor + })?; + Ok(Arc::new(r)) + } else { + // Nulls in secs are preserved, nulls in subsecs are treated as zero to account for the case + // where the number of nanoseconds overflows. + let r: Int32Array = secs + .iter() + .zip(subsecs) + .map(|(secs, subsecs)| { + secs.map(|secs| { + let subsecs = subsecs.unwrap_or(0); + secs * second_factor + (subsecs % 1_000_000_000) / conversion_factor + }) + }) + .collect(); + Ok(Arc::new(r)) + } } /// Invoke [`date_part`] on an `array` (e.g. Timestamp) and convert the /// result to a total number of seconds, milliseconds, microseconds or /// nanoseconds +/// +/// Given epoch return f64, this is a duplicated function to optimize for f64 type fn seconds(array: &dyn Array, unit: TimeUnit) -> Result { let sf = match unit { Second => 1_f64, diff --git a/datafusion/functions/src/datetime/date_trunc.rs b/datafusion/functions/src/datetime/date_trunc.rs index 5ec308ef9c81..b9f3bbf65973 100644 --- a/datafusion/functions/src/datetime/date_trunc.rs +++ b/datafusion/functions/src/datetime/date_trunc.rs @@ -137,7 +137,11 @@ impl ScalarUDFImpl for DateTruncFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let (granularity, array) = (&args[0], &args[1]); let granularity = if let ColumnarValue::Scalar(ScalarValue::Utf8(Some(v))) = @@ -251,13 +255,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_date_trunc_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Truncates a timestamp value to a specified precision.") - .with_syntax_example("date_trunc(precision, expression)") - .with_argument( - "precision", - r#"Time precision to truncate to. The following precisions are supported: + Documentation::builder( + DOC_SECTION_DATETIME, + "Truncates a timestamp value to a specified precision.", + "date_trunc(precision, expression)", + ) + .with_argument( + "precision", + r#"Time precision to truncate to. The following precisions are supported: - year / YEAR - quarter / QUARTER @@ -268,13 +273,12 @@ fn get_date_trunc_doc() -> &'static Documentation { - minute / MINUTE - second / SECOND "#, - ) - .with_argument( - "expression", - "Time expression to operate on. Can be a constant, column, or function.", - ) - .build() - .unwrap() + ) + .with_argument( + "expression", + "Time expression to operate on. Can be a constant, column, or function.", + ) + .build() }) } @@ -724,14 +728,15 @@ mod tests { .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::() .with_timezone_opt(tz_opt.clone()); - let batch_size = input.len(); + let batch_len = input.len(); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = DateTruncFunc::new() .invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::from("day")), ColumnarValue::Array(Arc::new(input)), ], - batch_size, + batch_len, ) .unwrap(); if let ColumnarValue::Array(result) = result { @@ -886,14 +891,15 @@ mod tests { .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::() .with_timezone_opt(tz_opt.clone()); - let batch_size = input.len(); + let batch_len = input.len(); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = DateTruncFunc::new() .invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::from("hour")), ColumnarValue::Array(Arc::new(input)), ], - batch_size, + batch_len, ) .unwrap(); if let ColumnarValue::Array(result) = result { diff --git a/datafusion/functions/src/datetime/from_unixtime.rs b/datafusion/functions/src/datetime/from_unixtime.rs index 29b2f29b14c2..374c744915f7 100644 --- a/datafusion/functions/src/datetime/from_unixtime.rs +++ b/datafusion/functions/src/datetime/from_unixtime.rs @@ -88,7 +88,11 @@ impl ScalarUDFImpl for FromUnixtimeFunc { internal_err!("call return_type_from_exprs instead") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let len = args.len(); if len != 1 && len != 2 { return exec_err!( @@ -129,10 +133,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_from_unixtime_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Converts an integer to RFC3339 timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp.") - .with_syntax_example("from_unixtime(expression[, timezone])") + Documentation::builder( + DOC_SECTION_DATETIME, + "Converts an integer to RFC3339 timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp.", + "from_unixtime(expression[, timezone])") .with_standard_argument("expression", None) .with_argument( "timezone", @@ -147,7 +151,6 @@ fn get_from_unixtime_doc() -> &'static Documentation { +-----------------------------------------------------------+ ```"#) .build() - .unwrap() }) } @@ -162,8 +165,8 @@ mod test { fn test_without_timezone() { let args = [ColumnarValue::Scalar(Int64(Some(1729900800)))]; - #[allow(deprecated)] // TODO use invoke_batch - let result = FromUnixtimeFunc::new().invoke(&args).unwrap(); + // TODO use invoke_with_args + let result = FromUnixtimeFunc::new().invoke_batch(&args, 1).unwrap(); match result { ColumnarValue::Scalar(ScalarValue::TimestampSecond(Some(sec), None)) => { @@ -182,8 +185,8 @@ mod test { ))), ]; - #[allow(deprecated)] // TODO use invoke_batch - let result = FromUnixtimeFunc::new().invoke(&args).unwrap(); + // TODO use invoke_with_args + let result = FromUnixtimeFunc::new().invoke_batch(&args, 2).unwrap(); match result { ColumnarValue::Scalar(ScalarValue::TimestampSecond(Some(sec), Some(tz))) => { diff --git a/datafusion/functions/src/datetime/make_date.rs b/datafusion/functions/src/datetime/make_date.rs index a13511f33398..a9340f82f23d 100644 --- a/datafusion/functions/src/datetime/make_date.rs +++ b/datafusion/functions/src/datetime/make_date.rs @@ -72,7 +72,11 @@ impl ScalarUDFImpl for MakeDateFunc { Ok(Date32) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 3 { return exec_err!( "make_date function requires 3 arguments, got {}", @@ -160,10 +164,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_make_date_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Make a date from year/month/day component parts.") - .with_syntax_example("make_date(year, month, day)") + Documentation::builder( + DOC_SECTION_DATETIME, + "Make a date from year/month/day component parts.", + "make_date(year, month, day)") .with_argument( "year", " Year to use when making the date. Can be a constant, column or function, and any combination of arithmetic operators.", ) @@ -190,7 +194,6 @@ fn get_make_date_doc() -> &'static Documentation { Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/make_date.rs) "#) .build() - .unwrap() }) } @@ -234,6 +237,7 @@ mod tests { #[test] fn test_make_date() { + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new() .invoke_batch( &[ @@ -251,6 +255,7 @@ mod tests { panic!("Expected a scalar value") } + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new() .invoke_batch( &[ @@ -268,6 +273,7 @@ mod tests { panic!("Expected a scalar value") } + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new() .invoke_batch( &[ @@ -288,7 +294,8 @@ mod tests { let years = Arc::new((2021..2025).map(Some).collect::()); let months = Arc::new((1..5).map(Some).collect::()); let days = Arc::new((11..15).map(Some).collect::()); - let batch_size = years.len(); + let batch_len = years.len(); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new() .invoke_batch( &[ @@ -296,7 +303,7 @@ mod tests { ColumnarValue::Array(months), ColumnarValue::Array(days), ], - batch_size, + batch_len, ) .expect("that make_date parsed values without error"); @@ -317,6 +324,7 @@ mod tests { // // invalid number of arguments + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new() .invoke_batch(&[ColumnarValue::Scalar(ScalarValue::Int32(Some(1)))], 1); assert_eq!( @@ -325,6 +333,7 @@ mod tests { ); // invalid type + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new().invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::IntervalYearMonth(Some(1))), @@ -339,6 +348,7 @@ mod tests { ); // overflow of month + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new().invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::Int32(Some(2023))), @@ -353,6 +363,7 @@ mod tests { ); // overflow of day + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let res = MakeDateFunc::new().invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::Int32(Some(2023))), diff --git a/datafusion/functions/src/datetime/now.rs b/datafusion/functions/src/datetime/now.rs index cadc4fce04f1..58381473a9ab 100644 --- a/datafusion/functions/src/datetime/now.rs +++ b/datafusion/functions/src/datetime/now.rs @@ -72,7 +72,11 @@ impl ScalarUDFImpl for NowFunc { Ok(Timestamp(Nanosecond, Some("+00:00".into()))) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { internal_err!("invoke should not be called on a simplified now() function") } @@ -106,15 +110,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_to_unixtime_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description(r#" + Documentation::builder( + DOC_SECTION_DATETIME, + r#" Returns the current UTC timestamp. The `now()` return value is determined at query time and will return the same timestamp, no matter when in the query plan the function executes. -"#) - .with_syntax_example("now()") +"#, + "now()") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index dd4ae7b8464e..4e3fcd1dc13a 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -107,7 +107,11 @@ impl ScalarUDFImpl for ToCharFunc { Ok(Utf8) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 2 { return exec_err!( "to_char function requires 2 arguments, got {}", @@ -147,10 +151,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_to_char_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Returns a string representation of a date, time, timestamp or duration based on a [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html). Unlike the PostgreSQL equivalent of this function numerical formatting is not supported.") - .with_syntax_example("to_char(expression, format)") + Documentation::builder( + DOC_SECTION_DATETIME, + "Returns a string representation of a date, time, timestamp or duration based on a [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html). Unlike the PostgreSQL equivalent of this function numerical formatting is not supported.", + "to_char(expression, format)") .with_argument( "expression", " Expression to operate on. Can be a constant, column, or function that results in a date, time, timestamp or duration." @@ -172,7 +176,6 @@ fn get_to_char_doc() -> &'static Documentation { Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_char.rs) "#) .build() - .unwrap() }) } @@ -384,6 +387,7 @@ mod tests { ]; for (value, format, expected) in scalar_data { + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let result = ToCharFunc::new() .invoke_batch( &[ColumnarValue::Scalar(value), ColumnarValue::Scalar(format)], @@ -461,14 +465,15 @@ mod tests { ]; for (value, format, expected) in scalar_array_data { - let batch_size = format.len(); + let batch_len = format.len(); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let result = ToCharFunc::new() .invoke_batch( &[ ColumnarValue::Scalar(value), ColumnarValue::Array(Arc::new(format) as ArrayRef), ], - batch_size, + batch_len, ) .expect("that to_char parsed values without error"); @@ -590,14 +595,15 @@ mod tests { ]; for (value, format, expected) in array_scalar_data { - let batch_size = value.len(); + let batch_len = value.len(); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let result = ToCharFunc::new() .invoke_batch( &[ ColumnarValue::Array(value as ArrayRef), ColumnarValue::Scalar(format), ], - batch_size, + batch_len, ) .expect("that to_char parsed values without error"); @@ -610,14 +616,15 @@ mod tests { } for (value, format, expected) in array_array_data { - let batch_size = value.len(); + let batch_len = value.len(); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let result = ToCharFunc::new() .invoke_batch( &[ ColumnarValue::Array(value), ColumnarValue::Array(Arc::new(format) as ArrayRef), ], - batch_size, + batch_len, ) .expect("that to_char parsed values without error"); @@ -634,6 +641,7 @@ mod tests { // // invalid number of arguments + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let result = ToCharFunc::new() .invoke_batch(&[ColumnarValue::Scalar(ScalarValue::Int32(Some(1)))], 1); assert_eq!( @@ -642,6 +650,7 @@ mod tests { ); // invalid type + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let result = ToCharFunc::new().invoke_batch( &[ ColumnarValue::Scalar(ScalarValue::Int32(Some(1))), diff --git a/datafusion/functions/src/datetime/to_date.rs b/datafusion/functions/src/datetime/to_date.rs index ff322ce31960..15f1159ef984 100644 --- a/datafusion/functions/src/datetime/to_date.rs +++ b/datafusion/functions/src/datetime/to_date.rs @@ -22,13 +22,46 @@ use arrow::error::ArrowError::ParseError; use arrow::{array::types::Date32Type, compute::kernels::cast_utils::Parser}; use datafusion_common::error::DataFusionError; use datafusion_common::{arrow_err, exec_err, internal_datafusion_err, Result}; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; +use datafusion_macros::user_doc; use std::any::Any; use std::sync::OnceLock; +#[user_doc( + doc_section(label = "Time and Date Functions"), + description = r"Converts a value to a date (`YYYY-MM-DD`). +Supports strings, integer and double types as input. +Strings are parsed as YYYY-MM-DD (e.g. '2023-07-20') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. +Integers and doubles are interpreted as days since the unix epoch (`1970-01-01T00:00:00Z`). +Returns the corresponding date. + +Note: `to_date` returns Date32, which represents its values as the number of days since unix epoch(`1970-01-01`) stored as signed 32 bit value. The largest supported date value is `9999-12-31`.", + syntax_example = "to_date('2017-05-31', '%Y-%m-%d')", + sql_example = "```sql\n\ +> select to_date('2023-01-31');\n\ ++-----------------------------+\n\ +| to_date(Utf8(\"2023-01-31\")) |\n\ ++-----------------------------+\n\ +| 2023-01-31 |\n\ ++-----------------------------+\n\ +> select to_date('2023/01/31', '%Y-%m-%d', '%Y/%m/%d');\n\ ++---------------------------------------------------------------+\n\ +| to_date(Utf8(\"2023/01/31\"),Utf8(\"%Y-%m-%d\"),Utf8(\"%Y/%m/%d\")) |\n\ ++---------------------------------------------------------------+\n\ +| 2023-01-31 |\n\ ++---------------------------------------------------------------+\n\ +```\n\n\ +Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_date.rs)", + standard_argument(name = "expression", prefix = "String"), + argument( + name = "format_n", + description = r"Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order + they appear with the first successful one being returned. If none of the formats successfully parse the expression + an error will be returned." + ) +)] #[derive(Debug)] pub struct ToDateFunc { signature: Signature, @@ -79,50 +112,6 @@ impl ToDateFunc { } } -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_to_date_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description(r#"Converts a value to a date (`YYYY-MM-DD`). -Supports strings, integer and double types as input. -Strings are parsed as YYYY-MM-DD (e.g. '2023-07-20') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. -Integers and doubles are interpreted as days since the unix epoch (`1970-01-01T00:00:00Z`). -Returns the corresponding date. - -Note: `to_date` returns Date32, which represents its values as the number of days since unix epoch(`1970-01-01`) stored as signed 32 bit value. The largest supported date value is `9999-12-31`. -"#) - .with_syntax_example("to_date('2017-05-31', '%Y-%m-%d')") - .with_sql_example(r#"```sql -> select to_date('2023-01-31'); -+-----------------------------+ -| to_date(Utf8("2023-01-31")) | -+-----------------------------+ -| 2023-01-31 | -+-----------------------------+ -> select to_date('2023/01/31', '%Y-%m-%d', '%Y/%m/%d'); -+---------------------------------------------------------------+ -| to_date(Utf8("2023/01/31"),Utf8("%Y-%m-%d"),Utf8("%Y/%m/%d")) | -+---------------------------------------------------------------+ -| 2023-01-31 | -+---------------------------------------------------------------+ -``` - -Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_date.rs) -"#) - .with_standard_argument("expression", Some("String")) - .with_argument( - "format_n", - "Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order - they appear with the first successful one being returned. If none of the formats successfully parse the expression - an error will be returned.", - ) - .build() - .unwrap() - }) -} - impl ScalarUDFImpl for ToDateFunc { fn as_any(&self) -> &dyn Any { self @@ -140,7 +129,11 @@ impl ScalarUDFImpl for ToDateFunc { Ok(Date32) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.is_empty() { return exec_err!("to_date function requires 1 or more arguments, got 0"); } @@ -162,7 +155,7 @@ impl ScalarUDFImpl for ToDateFunc { } fn documentation(&self) -> Option<&Documentation> { - Some(get_to_date_doc()) + self.doc() } } @@ -213,6 +206,7 @@ mod tests { } fn test_scalar(sv: ScalarValue, tc: &TestCase) { + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new().invoke_batch(&[ColumnarValue::Scalar(sv)], 1); @@ -234,9 +228,10 @@ mod tests { A: From> + Array + 'static, { let date_array = A::from(vec![tc.date_str]); - let batch_size = date_array.len(); + let batch_len = date_array.len(); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new() - .invoke_batch(&[ColumnarValue::Array(Arc::new(date_array))], batch_size); + .invoke_batch(&[ColumnarValue::Array(Arc::new(date_array))], batch_len); match to_date_result { Ok(ColumnarValue::Array(a)) => { @@ -325,6 +320,7 @@ mod tests { fn test_scalar(sv: ScalarValue, tc: &TestCase) { let format_scalar = ScalarValue::Utf8(Some(tc.format_str.to_string())); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new().invoke_batch( &[ ColumnarValue::Scalar(sv), @@ -351,14 +347,15 @@ mod tests { { let date_array = A::from(vec![tc.formatted_date]); let format_array = A::from(vec![tc.format_str]); + let batch_len = date_array.len(); - let batch_size = date_array.len(); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new().invoke_batch( &[ ColumnarValue::Array(Arc::new(date_array)), ColumnarValue::Array(Arc::new(format_array)), ], - batch_size, + batch_len, ); match to_date_result { @@ -391,6 +388,7 @@ mod tests { let format1_scalar = ScalarValue::Utf8(Some("%Y-%m-%d".into())); let format2_scalar = ScalarValue::Utf8(Some("%Y/%m/%d".into())); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new().invoke_batch( &[ ColumnarValue::Scalar(formatted_date_scalar), @@ -422,6 +420,7 @@ mod tests { for date_str in test_cases { let formatted_date_scalar = ScalarValue::Utf8(Some(date_str.into())); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new() .invoke_batch(&[ColumnarValue::Scalar(formatted_date_scalar)], 1); @@ -440,6 +439,7 @@ mod tests { let date_str = "20241231"; let date_scalar = ScalarValue::Utf8(Some(date_str.into())); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new().invoke_batch(&[ColumnarValue::Scalar(date_scalar)], 1); @@ -461,12 +461,13 @@ mod tests { let date_str = "202412311"; let date_scalar = ScalarValue::Utf8(Some(date_str.into())); + #[allow(deprecated)] // TODO migrate UDF to invoke from invoke_batch let to_date_result = ToDateFunc::new().invoke_batch(&[ColumnarValue::Scalar(date_scalar)], 1); if let Ok(ColumnarValue::Scalar(ScalarValue::Date32(_))) = to_date_result { panic!( - "Conversion of {} succeded, but should have failed, ", + "Conversion of {} succeeded, but should have failed, ", date_str ); } diff --git a/datafusion/functions/src/datetime/to_local_time.rs b/datafusion/functions/src/datetime/to_local_time.rs index fef1eb9a60c8..eaa91d1140ba 100644 --- a/datafusion/functions/src/datetime/to_local_time.rs +++ b/datafusion/functions/src/datetime/to_local_time.rs @@ -320,7 +320,11 @@ impl ScalarUDFImpl for ToLocalTimeFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 1 { return exec_err!( "to_local_time function requires 1 argument, got {:?}", @@ -363,10 +367,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_to_local_time_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Converts a timestamp with a timezone to a timestamp without a timezone (with no offset or timezone information). This function handles daylight saving time changes.") - .with_syntax_example("to_local_time(expression)") + Documentation::builder( + DOC_SECTION_DATETIME, + "Converts a timestamp with a timezone to a timestamp without a timezone (with no offset or timezone information). This function handles daylight saving time changes.", + "to_local_time(expression)") .with_argument( "expression", "Time expression to operate on. Can be a constant, column, or function." @@ -418,7 +422,6 @@ FROM ( +---------------------------+ ```"#) .build() - .unwrap() }) } @@ -431,7 +434,7 @@ mod tests { use arrow::datatypes::{DataType, TimeUnit}; use chrono::NaiveDateTime; use datafusion_common::ScalarValue; - use datafusion_expr::{ColumnarValue, ScalarUDFImpl}; + use datafusion_expr::{ColumnarValue, ScalarFunctionArgs, ScalarUDFImpl}; use super::{adjust_to_local_time, ToLocalTimeFunc}; @@ -558,7 +561,11 @@ mod tests { fn test_to_local_time_helper(input: ScalarValue, expected: ScalarValue) { let res = ToLocalTimeFunc::new() - .invoke_batch(&[ColumnarValue::Scalar(input)], 1) + .invoke_with_args(ScalarFunctionArgs { + args: &[ColumnarValue::Scalar(input)], + number_rows: 1, + return_type: &expected.data_type(), + }) .unwrap(); match res { ColumnarValue::Scalar(res) => { @@ -617,6 +624,7 @@ mod tests { .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::(); let batch_size = input.len(); + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = ToLocalTimeFunc::new() .invoke_batch(&[ColumnarValue::Array(Arc::new(input))], batch_size) .unwrap(); diff --git a/datafusion/functions/src/datetime/to_timestamp.rs b/datafusion/functions/src/datetime/to_timestamp.rs index f15fad701c55..430d4a970810 100644 --- a/datafusion/functions/src/datetime/to_timestamp.rs +++ b/datafusion/functions/src/datetime/to_timestamp.rs @@ -150,7 +150,11 @@ impl ScalarUDFImpl for ToTimestampFunc { Ok(return_type_for(&arg_types[0], Nanosecond)) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.is_empty() { return exec_err!( "to_timestamp function requires 1 or more arguments, got {}", @@ -193,14 +197,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_to_timestamp_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description(r#" + Documentation::builder( + DOC_SECTION_DATETIME, + r#" Converts a value to a timestamp (`YYYY-MM-DDT00:00:00Z`). Supports strings, integer, unsigned integer, and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats] are provided. Integers, unsigned integers, and doubles are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp. Note: `to_timestamp` returns `Timestamp(Nanosecond)`. The supported range for integer input is between `-9223372037` and `9223372036`. Supported range for string input is between `1677-09-21T00:12:44.0` and `2262-04-11T23:47:16.0`. Please use `to_timestamp_seconds` for the input outside of supported bounds. -"#) - .with_syntax_example("to_timestamp(expression[, ..., format_n])") +"#, + "to_timestamp(expression[, ..., format_n])") .with_argument( "expression", "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." @@ -226,7 +230,6 @@ Note: `to_timestamp` returns `Timestamp(Nanosecond)`. The supported range for in Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) "#) .build() - .unwrap() }) } @@ -247,7 +250,11 @@ impl ScalarUDFImpl for ToTimestampSecondsFunc { Ok(return_type_for(&arg_types[0], Second)) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.is_empty() { return exec_err!( "to_timestamp_seconds function requires 1 or more arguments, got {}", @@ -285,10 +292,10 @@ static TO_TIMESTAMP_SECONDS_DOC: OnceLock = OnceLock::new(); fn get_to_timestamp_seconds_doc() -> &'static Documentation { TO_TIMESTAMP_SECONDS_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.") - .with_syntax_example("to_timestamp_seconds(expression[, ..., format_n])") + Documentation::builder( + DOC_SECTION_DATETIME, + "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as seconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", + "to_timestamp_seconds(expression[, ..., format_n])") .with_argument( "expression", "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." @@ -314,7 +321,6 @@ fn get_to_timestamp_seconds_doc() -> &'static Documentation { Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) "#) .build() - .unwrap() }) } @@ -335,7 +341,11 @@ impl ScalarUDFImpl for ToTimestampMillisFunc { Ok(return_type_for(&arg_types[0], Millisecond)) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.is_empty() { return exec_err!( "to_timestamp_millis function requires 1 or more arguments, got {}", @@ -375,10 +385,10 @@ static TO_TIMESTAMP_MILLIS_DOC: OnceLock = OnceLock::new(); fn get_to_timestamp_millis_doc() -> &'static Documentation { TO_TIMESTAMP_MILLIS_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.") - .with_syntax_example("to_timestamp_millis(expression[, ..., format_n])") + Documentation::builder( + DOC_SECTION_DATETIME, + "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided. Integers and unsigned integers are interpreted as milliseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", + "to_timestamp_millis(expression[, ..., format_n])") .with_argument( "expression", "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." @@ -404,7 +414,6 @@ fn get_to_timestamp_millis_doc() -> &'static Documentation { Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) "#) .build() - .unwrap() }) } @@ -425,7 +434,11 @@ impl ScalarUDFImpl for ToTimestampMicrosFunc { Ok(return_type_for(&arg_types[0], Microsecond)) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.is_empty() { return exec_err!( "to_timestamp_micros function requires 1 or more arguments, got {}", @@ -465,10 +478,10 @@ static TO_TIMESTAMP_MICROS_DOC: OnceLock = OnceLock::new(); fn get_to_timestamp_micros_doc() -> &'static Documentation { TO_TIMESTAMP_MICROS_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as microseconds since the unix epoch (`1970-01-01T00:00:00Z`) Returns the corresponding timestamp.") - .with_syntax_example("to_timestamp_micros(expression[, ..., format_n])") + Documentation::builder( + DOC_SECTION_DATETIME, + "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as microseconds since the unix epoch (`1970-01-01T00:00:00Z`) Returns the corresponding timestamp.", + "to_timestamp_micros(expression[, ..., format_n])") .with_argument( "expression", "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." @@ -494,7 +507,6 @@ fn get_to_timestamp_micros_doc() -> &'static Documentation { Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) "#) .build() - .unwrap() }) } @@ -515,7 +527,11 @@ impl ScalarUDFImpl for ToTimestampNanosFunc { Ok(return_type_for(&arg_types[0], Nanosecond)) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.is_empty() { return exec_err!( "to_timestamp_nanos function requires 1 or more arguments, got {}", @@ -555,10 +571,10 @@ static TO_TIMESTAMP_NANOS_DOC: OnceLock = OnceLock::new(); fn get_to_timestamp_nanos_doc() -> &'static Documentation { TO_TIMESTAMP_NANOS_DOC.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.") - .with_syntax_example("to_timestamp_nanos(expression[, ..., format_n])") + Documentation::builder( + DOC_SECTION_DATETIME, + "Converts a value to a timestamp (`YYYY-MM-DDT00:00:00.000000000Z`). Supports strings, integer, and unsigned integer types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html)s are provided. Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`). Returns the corresponding timestamp.", + "to_timestamp_nanos(expression[, ..., format_n])") .with_argument( "expression", "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." @@ -584,7 +600,6 @@ fn get_to_timestamp_nanos_doc() -> &'static Documentation { Additional examples can be found [here](https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/to_timestamp.rs) "#) .build() - .unwrap() }) } @@ -1008,7 +1023,7 @@ mod tests { for array in arrays { let rt = udf.return_type(&[array.data_type()]).unwrap(); assert!(matches!(rt, Timestamp(_, Some(_)))); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let res = udf .invoke_batch(&[array.clone()], 1) .expect("that to_timestamp parsed values without error"); @@ -1051,7 +1066,7 @@ mod tests { for array in arrays { let rt = udf.return_type(&[array.data_type()]).unwrap(); assert!(matches!(rt, Timestamp(_, None))); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let res = udf .invoke_batch(&[array.clone()], 1) .expect("that to_timestamp parsed values without error"); diff --git a/datafusion/functions/src/datetime/to_unixtime.rs b/datafusion/functions/src/datetime/to_unixtime.rs index dd90ce6a6c96..9e6453a597c3 100644 --- a/datafusion/functions/src/datetime/to_unixtime.rs +++ b/datafusion/functions/src/datetime/to_unixtime.rs @@ -83,6 +83,7 @@ impl ScalarUDFImpl for ToUnixtimeFunc { DataType::Date64 | DataType::Date32 | DataType::Timestamp(_, None) => args[0] .cast_to(&DataType::Timestamp(TimeUnit::Second, None), None)? .cast_to(&DataType::Int64, None), + #[allow(deprecated)] // TODO: migrate to invoke_with_args DataType::Utf8 => ToTimestampSecondsFunc::new() .invoke_batch(args, batch_size)? .cast_to(&DataType::Int64, None), @@ -100,10 +101,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_to_unixtime_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_DATETIME) - .with_description("Converts a value to seconds since the unix epoch (`1970-01-01T00:00:00Z`). Supports strings, dates, timestamps and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided.") - .with_syntax_example("to_unixtime(expression[, ..., format_n])") + Documentation::builder( + DOC_SECTION_DATETIME, + "Converts a value to seconds since the unix epoch (`1970-01-01T00:00:00Z`). Supports strings, dates, timestamps and double types as input. Strings are parsed as RFC3339 (e.g. '2023-07-20T05:44:00') if no [Chrono formats](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) are provided.", + "to_unixtime(expression[, ..., format_n])") .with_argument( "expression", "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." @@ -127,6 +128,5 @@ fn get_to_unixtime_doc() -> &'static Documentation { ``` "#) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/encoding/inner.rs b/datafusion/functions/src/encoding/inner.rs index 0649c7cbb5c0..42d2ff98c39d 100644 --- a/datafusion/functions/src/encoding/inner.rs +++ b/datafusion/functions/src/encoding/inner.rs @@ -62,15 +62,15 @@ static ENCODE_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_encode_doc() -> &'static Documentation { ENCODE_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_BINARY_STRING) - .with_description("Encode binary data into a textual representation.") - .with_syntax_example("encode(expression, format)") - .with_argument("expression", "Expression containing string or binary data") - .with_argument("format", "Supported formats are: `base64`, `hex`") - .with_related_udf("decode") - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_BINARY_STRING, + "Encode binary data into a textual representation.", + "encode(expression, format)", + ) + .with_argument("expression", "Expression containing string or binary data") + .with_argument("format", "Supported formats are: `base64`, `hex`") + .with_related_udf("decode") + .build() }) } @@ -90,7 +90,11 @@ impl ScalarUDFImpl for EncodeFunc { Ok(arg_types[0].to_owned()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { encode(args) } @@ -149,15 +153,15 @@ static DECODE_DOCUMENTATION: OnceLock = OnceLock::new(); fn get_decode_doc() -> &'static Documentation { DECODE_DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_BINARY_STRING) - .with_description("Decode binary data from textual representation in string.") - .with_syntax_example("decode(expression, format)") - .with_argument("expression", "Expression containing encoded string data") - .with_argument("format", "Same arguments as [encode](#encode)") - .with_related_udf("encode") - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_BINARY_STRING, + "Decode binary data from textual representation in string.", + "decode(expression, format)", + ) + .with_argument("expression", "Expression containing encoded string data") + .with_argument("format", "Same arguments as [encode](#encode)") + .with_related_udf("encode") + .build() }) } @@ -177,7 +181,11 @@ impl ScalarUDFImpl for DecodeFunc { Ok(arg_types[0].to_owned()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { decode(args) } diff --git a/datafusion/functions/src/macros.rs b/datafusion/functions/src/macros.rs index 9bc038e71edc..bc6685589091 100644 --- a/datafusion/functions/src/macros.rs +++ b/datafusion/functions/src/macros.rs @@ -208,7 +208,11 @@ macro_rules! make_math_unary_udf { $EVALUATE_BOUNDS(inputs) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; let arr: ArrayRef = match args[0].data_type() { DataType::Float64 => Arc::new( @@ -316,7 +320,11 @@ macro_rules! make_math_binary_udf { $OUTPUT_ORDERING(input) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; let arr: ArrayRef = match args[0].data_type() { DataType::Float64 => { diff --git a/datafusion/functions/src/math/abs.rs b/datafusion/functions/src/math/abs.rs index 798939162a63..3565afbe6b48 100644 --- a/datafusion/functions/src/math/abs.rs +++ b/datafusion/functions/src/math/abs.rs @@ -160,7 +160,11 @@ impl ScalarUDFImpl for AbsFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; if args.len() != 1 { @@ -197,12 +201,12 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_abs_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the absolute value of a number.") - .with_syntax_example("abs(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the absolute value of a number.", + "abs(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } diff --git a/datafusion/functions/src/math/cot.rs b/datafusion/functions/src/math/cot.rs index eded50a20d8d..2355696a8be7 100644 --- a/datafusion/functions/src/math/cot.rs +++ b/datafusion/functions/src/math/cot.rs @@ -43,13 +43,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_cot_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the cotangent of a number.") - .with_syntax_example(r#"cot(numeric_expression)"#) - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the cotangent of a number.", + r#"cot(numeric_expression)"#, + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -95,7 +95,11 @@ impl ScalarUDFImpl for CotFunc { Some(get_cot_doc()) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(cot, vec![])(args) } } diff --git a/datafusion/functions/src/math/factorial.rs b/datafusion/functions/src/math/factorial.rs index bacdf47524f4..083936eb185a 100644 --- a/datafusion/functions/src/math/factorial.rs +++ b/datafusion/functions/src/math/factorial.rs @@ -68,7 +68,11 @@ impl ScalarUDFImpl for FactorialFunc { Ok(Int64) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(factorial, vec![])(args) } @@ -81,13 +85,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_factorial_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Factorial. Returns 1 if value is less than 2.") - .with_syntax_example("factorial(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Factorial. Returns 1 if value is less than 2.", + "factorial(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } diff --git a/datafusion/functions/src/math/gcd.rs b/datafusion/functions/src/math/gcd.rs index f4edef3acca3..f4119cd975ab 100644 --- a/datafusion/functions/src/math/gcd.rs +++ b/datafusion/functions/src/math/gcd.rs @@ -68,7 +68,11 @@ impl ScalarUDFImpl for GcdFunc { Ok(Int64) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(gcd, vec![])(args) } @@ -81,16 +85,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_gcd_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( + Documentation::builder( + DOC_SECTION_MATH, "Returns the greatest common divisor of `expression_x` and `expression_y`. Returns 0 if both inputs are zero.", - ) - .with_syntax_example("gcd(expression_x, expression_y)") + + "gcd(expression_x, expression_y)") .with_standard_argument("expression_x", Some("First numeric")) .with_standard_argument("expression_y", Some("Second numeric")) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/math/iszero.rs b/datafusion/functions/src/math/iszero.rs index 7e5d4fe77ffa..c1498ae36222 100644 --- a/datafusion/functions/src/math/iszero.rs +++ b/datafusion/functions/src/math/iszero.rs @@ -71,7 +71,11 @@ impl ScalarUDFImpl for IsZeroFunc { Ok(Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(iszero, vec![])(args) } @@ -84,15 +88,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_iszero_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - "Returns true if a given number is +0.0 or -0.0 otherwise returns false.", - ) - .with_syntax_example("iszero(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns true if a given number is +0.0 or -0.0 otherwise returns false.", + "iszero(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } diff --git a/datafusion/functions/src/math/lcm.rs b/datafusion/functions/src/math/lcm.rs index 64b07ce606f2..4e5a9b64f6f5 100644 --- a/datafusion/functions/src/math/lcm.rs +++ b/datafusion/functions/src/math/lcm.rs @@ -69,7 +69,11 @@ impl ScalarUDFImpl for LcmFunc { Ok(Int64) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(lcm, vec![])(args) } @@ -82,16 +86,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_lcm_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( + Documentation::builder( + DOC_SECTION_MATH, "Returns the least common multiple of `expression_x` and `expression_y`. Returns 0 if either input is zero.", - ) - .with_syntax_example("lcm(expression_x, expression_y)") + + "lcm(expression_x, expression_y)") .with_standard_argument("expression_x", Some("First numeric")) .with_standard_argument("expression_y", Some("Second numeric")) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index 9110f9f532d8..d4bb8ec13b0b 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -51,15 +51,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_log_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the base-x logarithm of a number. Can either provide a specified base, or if omitted then takes the base-10 of a number.") - .with_syntax_example(r#"log(base, numeric_expression) + Documentation::builder( + DOC_SECTION_MATH, + "Returns the base-x logarithm of a number. Can either provide a specified base, or if omitted then takes the base-10 of a number.", + r#"log(base, numeric_expression) log(numeric_expression)"#) .with_standard_argument("base", Some("Base numeric")) .with_standard_argument("numeric_expression", Some("Numeric")) .build() - .unwrap() }) } @@ -125,7 +124,11 @@ impl ScalarUDFImpl for LogFunc { } // Support overloaded log(base, x) and log(x) which defaults to log(10, x) - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; let mut base = ColumnarValue::Scalar(ScalarValue::Float32(Some(10.0))); @@ -277,7 +280,7 @@ mod tests { ]))), // num ColumnarValue::Array(Arc::new(Int64Array::from(vec![5, 10, 15, 20]))), ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let _ = LogFunc::new().invoke_batch(&args, 4); } @@ -286,7 +289,7 @@ mod tests { let args = [ ColumnarValue::Array(Arc::new(Int64Array::from(vec![10]))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new().invoke_batch(&args, 1); result.expect_err("expected error"); } @@ -296,7 +299,7 @@ mod tests { let args = [ ColumnarValue::Scalar(ScalarValue::Float32(Some(10.0))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 1) .expect("failed to initialize function log"); @@ -320,7 +323,7 @@ mod tests { let args = [ ColumnarValue::Scalar(ScalarValue::Float64(Some(10.0))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 1) .expect("failed to initialize function log"); @@ -345,7 +348,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Float32(Some(2.0))), // num ColumnarValue::Scalar(ScalarValue::Float32(Some(32.0))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 1) .expect("failed to initialize function log"); @@ -370,7 +373,7 @@ mod tests { ColumnarValue::Scalar(ScalarValue::Float64(Some(2.0))), // num ColumnarValue::Scalar(ScalarValue::Float64(Some(64.0))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 1) .expect("failed to initialize function log"); @@ -396,7 +399,7 @@ mod tests { 10.0, 100.0, 1000.0, 10000.0, ]))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 4) .expect("failed to initialize function log"); @@ -425,7 +428,7 @@ mod tests { 10.0, 100.0, 1000.0, 10000.0, ]))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 4) .expect("failed to initialize function log"); @@ -455,7 +458,7 @@ mod tests { 8.0, 4.0, 81.0, 625.0, ]))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 4) .expect("failed to initialize function log"); @@ -485,7 +488,7 @@ mod tests { 8.0, 4.0, 81.0, 625.0, ]))), // num ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = LogFunc::new() .invoke_batch(&args, 4) .expect("failed to initialize function log"); diff --git a/datafusion/functions/src/math/monotonicity.rs b/datafusion/functions/src/math/monotonicity.rs index 19c85f4b6e3c..46c670b8e651 100644 --- a/datafusion/functions/src/math/monotonicity.rs +++ b/datafusion/functions/src/math/monotonicity.rs @@ -42,13 +42,13 @@ static DOCUMENTATION_ACOS: OnceLock = OnceLock::new(); pub fn get_acos_doc() -> &'static Documentation { DOCUMENTATION_ACOS.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the arc cosine or inverse cosine of a number.") - .with_syntax_example("acos(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the arc cosine or inverse cosine of a number.", + "acos(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -73,15 +73,13 @@ static DOCUMENTATION_ACOSH: OnceLock = OnceLock::new(); pub fn get_acosh_doc() -> &'static Documentation { DOCUMENTATION_ACOSH.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( + Documentation::builder( + DOC_SECTION_MATH, "Returns the area hyperbolic cosine or inverse hyperbolic cosine of a number.", - ) - .with_syntax_example("acosh(numeric_expression)") + + "acosh(numeric_expression)") .with_standard_argument("numeric_expression", Some("Numeric")) .build() - .unwrap() }) } @@ -104,13 +102,13 @@ static DOCUMENTATION_ASIN: OnceLock = OnceLock::new(); pub fn get_asin_doc() -> &'static Documentation { DOCUMENTATION_ASIN.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the arc sine or inverse sine of a number.") - .with_syntax_example("asin(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the arc sine or inverse sine of a number.", + "asin(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -123,15 +121,13 @@ static DOCUMENTATION_ASINH: OnceLock = OnceLock::new(); pub fn get_asinh_doc() -> &'static Documentation { DOCUMENTATION_ASINH.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - "Returns the area hyperbolic sine or inverse hyperbolic sine of a number.", - ) - .with_syntax_example("asinh(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the area hyperbolic sine or inverse hyperbolic sine of a number.", + "asinh(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -144,13 +140,13 @@ static DOCUMENTATION_ATAN: OnceLock = OnceLock::new(); pub fn get_atan_doc() -> &'static Documentation { DOCUMENTATION_ATAN.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the arc tangent or inverse tangent of a number.") - .with_syntax_example("atan(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the arc tangent or inverse tangent of a number.", + "atan(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -173,15 +169,13 @@ static DOCUMENTATION_ATANH: OnceLock = OnceLock::new(); pub fn get_atanh_doc() -> &'static Documentation { DOCUMENTATION_ATANH.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( + Documentation::builder( + DOC_SECTION_MATH, "Returns the area hyperbolic tangent or inverse hyperbolic tangent of a number.", - ) - .with_syntax_example("atanh(numeric_expression)") + + "atanh(numeric_expression)") .with_standard_argument("numeric_expression", Some("Numeric")) .build() - .unwrap() }) } @@ -195,18 +189,16 @@ static DOCUMENTATION_ATANH2: OnceLock = OnceLock::new(); pub fn get_atan2_doc() -> &'static Documentation { DOCUMENTATION_ATANH2.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( + Documentation::builder( + DOC_SECTION_MATH, "Returns the arc tangent or inverse tangent of `expression_y / expression_x`.", - ) - .with_syntax_example("atan2(expression_y, expression_x)") + + "atan2(expression_y, expression_x)") .with_argument("expression_y", r#"First numeric expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators."#) .with_argument("expression_x", r#"Second numeric expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators."#) .build() - .unwrap() }) } @@ -219,13 +211,13 @@ static DOCUMENTATION_CBRT: OnceLock = OnceLock::new(); pub fn get_cbrt_doc() -> &'static Documentation { DOCUMENTATION_CBRT.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the cube root of a number.") - .with_syntax_example("cbrt(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the cube root of a number.", + "cbrt(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -238,15 +230,13 @@ static DOCUMENTATION_CEIL: OnceLock = OnceLock::new(); pub fn get_ceil_doc() -> &'static Documentation { DOCUMENTATION_CEIL.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - "Returns the nearest integer greater than or equal to a number.", - ) - .with_syntax_example("ceil(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the nearest integer greater than or equal to a number.", + "ceil(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -261,13 +251,13 @@ static DOCUMENTATION_COS: OnceLock = OnceLock::new(); pub fn get_cos_doc() -> &'static Documentation { DOCUMENTATION_COS.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the cosine of a number.") - .with_syntax_example("cos(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the cosine of a number.", + "cos(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -291,13 +281,13 @@ static DOCUMENTATION_COSH: OnceLock = OnceLock::new(); pub fn get_cosh_doc() -> &'static Documentation { DOCUMENTATION_COSH.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the hyperbolic cosine of a number.") - .with_syntax_example("cosh(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the hyperbolic cosine of a number.", + "cosh(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -310,13 +300,13 @@ static DOCUMENTATION_DEGREES: OnceLock = OnceLock::new(); pub fn get_degrees_doc() -> &'static Documentation { DOCUMENTATION_DEGREES.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Converts radians to degrees.") - .with_syntax_example("degrees(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Converts radians to degrees.", + "degrees(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -329,13 +319,13 @@ static DOCUMENTATION_EXP: OnceLock = OnceLock::new(); pub fn get_exp_doc() -> &'static Documentation { DOCUMENTATION_EXP.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the base-e exponential of a number.") - .with_syntax_example("exp(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the base-e exponential of a number.", + "exp(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -348,15 +338,13 @@ static DOCUMENTATION_FLOOR: OnceLock = OnceLock::new(); pub fn get_floor_doc() -> &'static Documentation { DOCUMENTATION_FLOOR.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - "Returns the nearest integer less than or equal to a number.", - ) - .with_syntax_example("floor(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the nearest integer less than or equal to a number.", + "floor(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -378,13 +366,13 @@ static DOCUMENTATION_LN: OnceLock = OnceLock::new(); pub fn get_ln_doc() -> &'static Documentation { DOCUMENTATION_LN.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the natural logarithm of a number.") - .with_syntax_example("ln(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the natural logarithm of a number.", + "ln(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -406,13 +394,13 @@ static DOCUMENTATION_LOG2: OnceLock = OnceLock::new(); pub fn get_log2_doc() -> &'static Documentation { DOCUMENTATION_LOG2.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the base-2 logarithm of a number.") - .with_syntax_example("log2(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the base-2 logarithm of a number.", + "log2(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -434,13 +422,13 @@ static DOCUMENTATION_LOG10: OnceLock = OnceLock::new(); pub fn get_log10_doc() -> &'static Documentation { DOCUMENTATION_LOG10.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the base-10 logarithm of a number.") - .with_syntax_example("log10(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the base-10 logarithm of a number.", + "log10(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -453,13 +441,13 @@ static DOCUMENTATION_RADIONS: OnceLock = OnceLock::new(); pub fn get_radians_doc() -> &'static Documentation { DOCUMENTATION_RADIONS.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Converts degrees to radians.") - .with_syntax_example("radians(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Converts degrees to radians.", + "radians(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -474,13 +462,13 @@ static DOCUMENTATION_SIN: OnceLock = OnceLock::new(); pub fn get_sin_doc() -> &'static Documentation { DOCUMENTATION_SIN.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the sine of a number.") - .with_syntax_example("sin(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the sine of a number.", + "sin(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -493,13 +481,13 @@ static DOCUMENTATION_SINH: OnceLock = OnceLock::new(); pub fn get_sinh_doc() -> &'static Documentation { DOCUMENTATION_SINH.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the hyperbolic sine of a number.") - .with_syntax_example("sinh(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the hyperbolic sine of a number.", + "sinh(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -521,13 +509,13 @@ static DOCUMENTATION_SQRT: OnceLock = OnceLock::new(); pub fn get_sqrt_doc() -> &'static Documentation { DOCUMENTATION_SQRT.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the square root of a number.") - .with_syntax_example("sqrt(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the square root of a number.", + "sqrt(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -542,13 +530,13 @@ static DOCUMENTATION_TAN: OnceLock = OnceLock::new(); pub fn get_tan_doc() -> &'static Documentation { DOCUMENTATION_TAN.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the tangent of a number.") - .with_syntax_example("tan(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the tangent of a number.", + "tan(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -561,12 +549,12 @@ static DOCUMENTATION_TANH: OnceLock = OnceLock::new(); pub fn get_tanh_doc() -> &'static Documentation { DOCUMENTATION_TANH.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns the hyperbolic tangent of a number.") - .with_syntax_example("tanh(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns the hyperbolic tangent of a number.", + "tanh(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } diff --git a/datafusion/functions/src/math/nans.rs b/datafusion/functions/src/math/nans.rs index c1dd1aacc35a..4cfbf0494812 100644 --- a/datafusion/functions/src/math/nans.rs +++ b/datafusion/functions/src/math/nans.rs @@ -69,7 +69,11 @@ impl ScalarUDFImpl for IsNanFunc { Ok(DataType::Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; let arr: ArrayRef = match args[0].data_type() { @@ -101,14 +105,12 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_isnan_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - "Returns true if a given number is +NaN or -NaN otherwise returns false.", - ) - .with_syntax_example("isnan(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns true if a given number is +NaN or -NaN otherwise returns false.", + "isnan(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } diff --git a/datafusion/functions/src/math/nanvl.rs b/datafusion/functions/src/math/nanvl.rs index cfd21256dd96..0715dc7f7eac 100644 --- a/datafusion/functions/src/math/nanvl.rs +++ b/datafusion/functions/src/math/nanvl.rs @@ -73,7 +73,11 @@ impl ScalarUDFImpl for NanvlFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(nanvl, vec![])(args) } @@ -86,17 +90,15 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_nanvl_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( + Documentation::builder( + DOC_SECTION_MATH, r#"Returns the first argument if it's not _NaN_. Returns the second argument otherwise."#, - ) - .with_syntax_example("nanvl(expression_x, expression_y)") + + "nanvl(expression_x, expression_y)") .with_argument("expression_x", "Numeric expression to return if it's not _NaN_. Can be a constant, column, or function, and any combination of arithmetic operators.") .with_argument("expression_y", "Numeric expression to return if the first expression is _NaN_. Can be a constant, column, or function, and any combination of arithmetic operators.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/math/pi.rs b/datafusion/functions/src/math/pi.rs index 70cc76f03c58..a96ca176622d 100644 --- a/datafusion/functions/src/math/pi.rs +++ b/datafusion/functions/src/math/pi.rs @@ -20,7 +20,7 @@ use std::sync::OnceLock; use arrow::datatypes::DataType; use arrow::datatypes::DataType::Float64; -use datafusion_common::{internal_err, not_impl_err, Result, ScalarValue}; +use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_MATH; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ @@ -63,10 +63,6 @@ impl ScalarUDFImpl for PiFunc { Ok(Float64) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { - not_impl_err!("{} function does not accept arguments", self.name()) - } - fn invoke_batch( &self, args: &[ColumnarValue], @@ -94,11 +90,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_pi_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Returns an approximate value of π.") - .with_syntax_example("pi()") - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns an approximate value of π.", + "pi()", + ) + .build() }) } diff --git a/datafusion/functions/src/math/power.rs b/datafusion/functions/src/math/power.rs index a24c613f5259..92dd8966b66c 100644 --- a/datafusion/functions/src/math/power.rs +++ b/datafusion/functions/src/math/power.rs @@ -84,7 +84,11 @@ impl ScalarUDFImpl for PowerFunc { &self.aliases } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; let arr: ArrayRef = match args[0].data_type() { @@ -174,16 +178,14 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_power_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - "Returns a base expression raised to the power of an exponent.", - ) - .with_syntax_example("power(base, exponent)") - .with_standard_argument("base", Some("Numeric")) - .with_standard_argument("exponent", Some("Exponent numeric")) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Returns a base expression raised to the power of an exponent.", + "power(base, exponent)", + ) + .with_standard_argument("base", Some("Numeric")) + .with_standard_argument("exponent", Some("Exponent numeric")) + .build() }) } @@ -205,7 +207,7 @@ mod tests { ColumnarValue::Array(Arc::new(Float64Array::from(vec![2.0, 2.0, 3.0, 5.0]))), // base ColumnarValue::Array(Arc::new(Float64Array::from(vec![3.0, 2.0, 4.0, 4.0]))), // exponent ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = PowerFunc::new() .invoke_batch(&args, 4) .expect("failed to initialize function power"); @@ -232,7 +234,7 @@ mod tests { ColumnarValue::Array(Arc::new(Int64Array::from(vec![2, 2, 3, 5]))), // base ColumnarValue::Array(Arc::new(Int64Array::from(vec![3, 2, 4, 4]))), // exponent ]; - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = PowerFunc::new() .invoke_batch(&args, 4) .expect("failed to initialize function power"); diff --git a/datafusion/functions/src/math/random.rs b/datafusion/functions/src/math/random.rs index 0026037c95bd..e34db023ed9a 100644 --- a/datafusion/functions/src/math/random.rs +++ b/datafusion/functions/src/math/random.rs @@ -90,14 +90,12 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_random_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - r#"Returns a random float value in the range [0, 1). + Documentation::builder( + DOC_SECTION_MATH, + r#"Returns a random float value in the range [0, 1). The random seed is unique to each row."#, - ) - .with_syntax_example("random()") - .build() - .unwrap() + "random()", + ) + .build() }) } diff --git a/datafusion/functions/src/math/round.rs b/datafusion/functions/src/math/round.rs index 6000e5d765de..cfbf083fcb1e 100644 --- a/datafusion/functions/src/math/round.rs +++ b/datafusion/functions/src/math/round.rs @@ -80,7 +80,11 @@ impl ScalarUDFImpl for RoundFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(round, vec![])(args) } @@ -108,17 +112,17 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_round_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description("Rounds a number to the nearest integer.") - .with_syntax_example("round(numeric_expression[, decimal_places])") - .with_standard_argument("numeric_expression", Some("Numeric")) - .with_argument( - "decimal_places", - "Optional. The number of decimal places to round to. Defaults to 0.", - ) - .build() - .unwrap() + Documentation::builder( + DOC_SECTION_MATH, + "Rounds a number to the nearest integer.", + "round(numeric_expression[, decimal_places])", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .with_argument( + "decimal_places", + "Optional. The number of decimal places to round to. Defaults to 0.", + ) + .build() }) } diff --git a/datafusion/functions/src/math/signum.rs b/datafusion/functions/src/math/signum.rs index 7f21297712c7..eda9df49fbac 100644 --- a/datafusion/functions/src/math/signum.rs +++ b/datafusion/functions/src/math/signum.rs @@ -80,7 +80,11 @@ impl ScalarUDFImpl for SignumFunc { Ok(input[0].sort_properties) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(signum, vec![])(args) } @@ -93,17 +97,15 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_signum_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( - r#"Returns the sign of a number. + Documentation::builder( + DOC_SECTION_MATH, + r#"Returns the sign of a number. Negative numbers return `-1`. Zero and positive numbers return `1`."#, - ) - .with_syntax_example("signum(numeric_expression)") - .with_standard_argument("numeric_expression", Some("Numeric")) - .build() - .unwrap() + "signum(numeric_expression)", + ) + .with_standard_argument("numeric_expression", Some("Numeric")) + .build() }) } @@ -167,6 +169,7 @@ mod test { f32::NEG_INFINITY, ])); let batch_size = array.len(); + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = SignumFunc::new() .invoke_batch(&[ColumnarValue::Array(array)], batch_size) .expect("failed to initialize function signum"); @@ -207,6 +210,7 @@ mod test { f64::NEG_INFINITY, ])); let batch_size = array.len(); + #[allow(deprecated)] // TODO: migrate to invoke_with_args let result = SignumFunc::new() .invoke_batch(&[ColumnarValue::Array(array)], batch_size) .expect("failed to initialize function signum"); diff --git a/datafusion/functions/src/math/trunc.rs b/datafusion/functions/src/math/trunc.rs index 9a05684d238e..c2787c4577d0 100644 --- a/datafusion/functions/src/math/trunc.rs +++ b/datafusion/functions/src/math/trunc.rs @@ -85,7 +85,11 @@ impl ScalarUDFImpl for TruncFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(trunc, vec![])(args) } @@ -113,12 +117,11 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_trunc_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_MATH) - .with_description( + Documentation::builder( + DOC_SECTION_MATH, "Truncates a number to a whole number or truncated to the specified decimal places.", - ) - .with_syntax_example("trunc(numeric_expression[, decimal_places])") + + "trunc(numeric_expression[, decimal_places])") .with_standard_argument("numeric_expression", Some("Numeric")) .with_argument("decimal_places", r#"Optional. The number of decimal places to truncate to. Defaults to 0 (truncate to a whole number). If @@ -126,7 +129,6 @@ fn get_trunc_doc() -> &'static Documentation { right of the decimal point. If `decimal_places` is a negative integer, replaces digits to the left of the decimal point with `0`."#) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/regex/regexpcount.rs b/datafusion/functions/src/regex/regexpcount.rs index 8da154430fc5..a667d70e7bb2 100644 --- a/datafusion/functions/src/regex/regexpcount.rs +++ b/datafusion/functions/src/regex/regexpcount.rs @@ -81,7 +81,11 @@ impl ScalarUDFImpl for RegexpCountFunc { Ok(Int64) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let len = args .iter() .fold(Option::::None, |acc, arg| match arg { @@ -115,10 +119,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_regexp_count_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_REGEX) - .with_description("Returns the number of matches that a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has in a string.") - .with_syntax_example("regexp_count(str, regexp[, start, flags])") + Documentation::builder( + DOC_SECTION_REGEX, + "Returns the number of matches that a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has in a string.", + "regexp_count(str, regexp[, start, flags])") .with_sql_example(r#"```sql > select regexp_count('abcAbAbc', 'abc', 2, 'i'); +---------------------------------------------------------------+ @@ -138,7 +142,6 @@ fn get_regexp_count_doc() -> &'static Documentation { - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used - **U**: swap the meaning of x* and x*?"#) .build() - .unwrap() }) } @@ -655,7 +658,7 @@ mod tests { let v_sv = ScalarValue::Utf8(Some(v.to_string())); let regex_sv = ScalarValue::Utf8(Some(regex.to_string())); let expected = expected.get(pos).cloned(); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)], 1, @@ -670,7 +673,7 @@ mod tests { // largeutf8 let v_sv = ScalarValue::LargeUtf8(Some(v.to_string())); let regex_sv = ScalarValue::LargeUtf8(Some(regex.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)], 1, @@ -685,7 +688,7 @@ mod tests { // utf8view let v_sv = ScalarValue::Utf8View(Some(v.to_string())); let regex_sv = ScalarValue::Utf8View(Some(regex.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)], 1, @@ -711,7 +714,7 @@ mod tests { let regex_sv = ScalarValue::Utf8(Some(regex.to_string())); let start_sv = ScalarValue::Int64(Some(start)); let expected = expected.get(pos).cloned(); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -730,7 +733,7 @@ mod tests { // largeutf8 let v_sv = ScalarValue::LargeUtf8(Some(v.to_string())); let regex_sv = ScalarValue::LargeUtf8(Some(regex.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -749,7 +752,7 @@ mod tests { // utf8view let v_sv = ScalarValue::Utf8View(Some(v.to_string())); let regex_sv = ScalarValue::Utf8View(Some(regex.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -781,7 +784,7 @@ mod tests { let start_sv = ScalarValue::Int64(Some(start)); let flags_sv = ScalarValue::Utf8(Some(flags.to_string())); let expected = expected.get(pos).cloned(); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -802,7 +805,7 @@ mod tests { let v_sv = ScalarValue::LargeUtf8(Some(v.to_string())); let regex_sv = ScalarValue::LargeUtf8(Some(regex.to_string())); let flags_sv = ScalarValue::LargeUtf8(Some(flags.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -823,7 +826,7 @@ mod tests { let v_sv = ScalarValue::Utf8View(Some(v.to_string())); let regex_sv = ScalarValue::Utf8View(Some(regex.to_string())); let flags_sv = ScalarValue::Utf8View(Some(flags.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -905,7 +908,7 @@ mod tests { let start_sv = ScalarValue::Int64(Some(start)); let flags_sv = ScalarValue::Utf8(flags.get(pos).map(|f| f.to_string())); let expected = expected.get(pos).cloned(); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -926,7 +929,7 @@ mod tests { let v_sv = ScalarValue::LargeUtf8(Some(v.to_string())); let regex_sv = ScalarValue::LargeUtf8(regex.get(pos).map(|s| s.to_string())); let flags_sv = ScalarValue::LargeUtf8(flags.get(pos).map(|f| f.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), @@ -947,7 +950,7 @@ mod tests { let v_sv = ScalarValue::Utf8View(Some(v.to_string())); let regex_sv = ScalarValue::Utf8View(regex.get(pos).map(|s| s.to_string())); let flags_sv = ScalarValue::Utf8View(flags.get(pos).map(|f| f.to_string())); - + #[allow(deprecated)] // TODO: migrate to invoke_with_args let re = RegexpCountFunc::new().invoke_batch( &[ ColumnarValue::Scalar(v_sv), diff --git a/datafusion/functions/src/regex/regexplike.rs b/datafusion/functions/src/regex/regexplike.rs index 13de7888aa5f..adbd6ef94d84 100644 --- a/datafusion/functions/src/regex/regexplike.rs +++ b/datafusion/functions/src/regex/regexplike.rs @@ -47,10 +47,7 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_regexp_like_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_REGEX) - .with_description("Returns true if a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has at least one match in a string, false otherwise.") - .with_syntax_example("regexp_like(str, regexp[, flags])") + Documentation::builder(DOC_SECTION_REGEX,"Returns true if a [regular expression](https://docs.rs/regex/latest/regex/#syntax) has at least one match in a string, false otherwise.","regexp_like(str, regexp[, flags])") .with_sql_example(r#"```sql select regexp_like('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); +--------------------------------------------------------+ @@ -77,7 +74,6 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used - **U**: swap the meaning of x* and x*?"#) .build() - .unwrap() }) } @@ -135,7 +131,11 @@ impl ScalarUDFImpl for RegexpLikeFunc { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let len = args .iter() .fold(Option::::None, |acc, arg| match arg { diff --git a/datafusion/functions/src/regex/regexpmatch.rs b/datafusion/functions/src/regex/regexpmatch.rs index 019666bd7b2d..df4f294bb950 100644 --- a/datafusion/functions/src/regex/regexpmatch.rs +++ b/datafusion/functions/src/regex/regexpmatch.rs @@ -83,7 +83,11 @@ impl ScalarUDFImpl for RegexpMatchFunc { other => DataType::List(Arc::new(Field::new("item", other.clone(), true))), }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let len = args .iter() .fold(Option::::None, |acc, arg| match arg { @@ -117,10 +121,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_regexp_match_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_REGEX) - .with_description("Returns the first [regular expression](https://docs.rs/regex/latest/regex/#syntax) matches in a string.") - .with_syntax_example("regexp_match(str, regexp[, flags])") + Documentation::builder( + DOC_SECTION_REGEX, + "Returns the first [regular expression](https://docs.rs/regex/latest/regex/#syntax) matches in a string.", + "regexp_match(str, regexp[, flags])") .with_sql_example(r#"```sql > select regexp_match('Köln', '[a-zA-Z]ö[a-zA-Z]{2}'); +---------------------------------------------------------+ @@ -148,7 +152,6 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used - **U**: swap the meaning of x* and x*?"#) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/regex/regexpreplace.rs b/datafusion/functions/src/regex/regexpreplace.rs index 4d8e5e5fe3e3..3f289e7c1511 100644 --- a/datafusion/functions/src/regex/regexpreplace.rs +++ b/datafusion/functions/src/regex/regexpreplace.rs @@ -106,7 +106,11 @@ impl ScalarUDFImpl for RegexpReplaceFunc { } }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let len = args .iter() .fold(Option::::None, |acc, arg| match arg { @@ -134,10 +138,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_regexp_replace_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_REGEX) - .with_description("Replaces substrings in a string that match a [regular expression](https://docs.rs/regex/latest/regex/#syntax).") - .with_syntax_example("regexp_replace(str, regexp, replacement[, flags])") + Documentation::builder( + DOC_SECTION_REGEX, + "Replaces substrings in a string that match a [regular expression](https://docs.rs/regex/latest/regex/#syntax).", + "regexp_replace(str, regexp, replacement[, flags])") .with_sql_example(r#"```sql > select regexp_replace('foobarbaz', 'b(..)', 'X\\1Y', 'g'); +------------------------------------------------------------------------+ @@ -167,7 +171,6 @@ Additional examples can be found [here](https://github.com/apache/datafusion/blo - **R**: enables CRLF mode: when multi-line mode is enabled, \r\n is used - **U**: swap the meaning of x* and x*?"#) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/ascii.rs b/datafusion/functions/src/string/ascii.rs index b76d70d7e9d2..4f615b5b2c58 100644 --- a/datafusion/functions/src/string/ascii.rs +++ b/datafusion/functions/src/string/ascii.rs @@ -64,7 +64,11 @@ impl ScalarUDFImpl for AsciiFunc { Ok(Int32) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(ascii, vec![])(args) } @@ -77,14 +81,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_ascii_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description( - "Returns the Unicode character code of the first character in a string.", - ) - .with_syntax_example("ascii(str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns the Unicode character code of the first character in a string.", + "ascii(str)", + ) + .with_sql_example( + r#"```sql > select ascii('abc'); +--------------------+ | ascii(Utf8("abc")) | @@ -98,11 +101,10 @@ fn get_ascii_doc() -> &'static Documentation { | 128640 | +-------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_related_udf("chr") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_related_udf("chr") + .build() }) } diff --git a/datafusion/functions/src/string/bit_length.rs b/datafusion/functions/src/string/bit_length.rs index cb815df15e4b..5a23692d85c7 100644 --- a/datafusion/functions/src/string/bit_length.rs +++ b/datafusion/functions/src/string/bit_length.rs @@ -62,7 +62,11 @@ impl ScalarUDFImpl for BitLengthFunc { utf8_to_int_type(&arg_types[0], "bit_length") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 1 { return exec_err!( "bit_length function requires 1 argument, got {}", @@ -96,12 +100,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_bit_length_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns the bit length of a string.") - .with_syntax_example("bit_length(str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns the bit length of a string.", + "bit_length(str)", + ) + .with_sql_example( + r#"```sql > select bit_length('datafusion'); +--------------------------------+ | bit_length(Utf8("datafusion")) | @@ -109,11 +114,10 @@ fn get_bit_length_doc() -> &'static Documentation { | 80 | +--------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_related_udf("length") - .with_related_udf("octet_length") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_related_udf("length") + .with_related_udf("octet_length") + .build() }) } diff --git a/datafusion/functions/src/string/btrim.rs b/datafusion/functions/src/string/btrim.rs index e215b18d9c3c..ae79bb59f9c7 100644 --- a/datafusion/functions/src/string/btrim.rs +++ b/datafusion/functions/src/string/btrim.rs @@ -80,7 +80,11 @@ impl ScalarUDFImpl for BTrimFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8 | DataType::Utf8View => make_scalar_function( btrim::, @@ -110,10 +114,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_btrim_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Trims the specified trim string from the start and end of a string. If no trim string is provided, all whitespace is removed from the start and end of the input string.") - .with_syntax_example("btrim(str[, trim_str])") + Documentation::builder( + DOC_SECTION_STRING, + "Trims the specified trim string from the start and end of a string. If no trim string is provided, all whitespace is removed from the start and end of the input string.", + "btrim(str[, trim_str])") .with_sql_example(r#"```sql > select btrim('__datafusion____', '_'); +-------------------------------------------+ @@ -129,7 +133,6 @@ fn get_btrim_doc() -> &'static Documentation { .with_related_udf("ltrim") .with_related_udf("rtrim") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/chr.rs b/datafusion/functions/src/string/chr.rs index 0d94cab08d91..127b02cdf733 100644 --- a/datafusion/functions/src/string/chr.rs +++ b/datafusion/functions/src/string/chr.rs @@ -96,7 +96,11 @@ impl ScalarUDFImpl for ChrFunc { Ok(Utf8) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(chr, vec![])(args) } @@ -109,14 +113,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_chr_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description( - "Returns the character with the specified ASCII or Unicode code value.", - ) - .with_syntax_example("chr(expression)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns the character with the specified ASCII or Unicode code value.", + "chr(expression)", + ) + .with_sql_example( + r#"```sql > select chr(128640); +--------------------+ | chr(Int64(128640)) | @@ -124,10 +127,9 @@ fn get_chr_doc() -> &'static Documentation { | 🚀 | +--------------------+ ```"#, - ) - .with_standard_argument("expression", Some("String")) - .with_related_udf("ascii") - .build() - .unwrap() + ) + .with_standard_argument("expression", Some("String")) + .with_related_udf("ascii") + .build() }) } diff --git a/datafusion/functions/src/string/concat.rs b/datafusion/functions/src/string/concat.rs index d49a2777b4ff..576c891ce467 100644 --- a/datafusion/functions/src/string/concat.rs +++ b/datafusion/functions/src/string/concat.rs @@ -85,7 +85,11 @@ impl ScalarUDFImpl for ConcatFunc { /// Concatenates the text representations of all the arguments. NULL arguments are ignored. /// concat('abcde', 2, NULL, 22) = 'abcde222' - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let mut return_datatype = DataType::Utf8; args.iter().for_each(|col| { if col.data_type() == DataType::Utf8View { @@ -267,12 +271,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_concat_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Concatenates multiple strings together.") - .with_syntax_example("concat(str[, ..., str_n])") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Concatenates multiple strings together.", + "concat(str[, ..., str_n])", + ) + .with_sql_example( + r#"```sql > select concat('data', 'f', 'us', 'ion'); +-------------------------------------------------------+ | concat(Utf8("data"),Utf8("f"),Utf8("us"),Utf8("ion")) | @@ -280,12 +285,11 @@ fn get_concat_doc() -> &'static Documentation { | datafusion | +-------------------------------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_argument("str_n", "Subsequent string expressions to concatenate.") - .with_related_udf("concat_ws") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_argument("str_n", "Subsequent string expressions to concatenate.") + .with_related_udf("concat_ws") + .build() }) } @@ -472,6 +476,7 @@ mod tests { ]))); let args = &[c0, c1, c2, c3, c4]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ConcatFunc::new().invoke_batch(args, 3)?; let expected = Arc::new(StringViewArray::from(vec!["foo,x,a", "bar,,", "baz,z,b"])) diff --git a/datafusion/functions/src/string/concat_ws.rs b/datafusion/functions/src/string/concat_ws.rs index 98a75f121c35..610c4f0be697 100644 --- a/datafusion/functions/src/string/concat_ws.rs +++ b/datafusion/functions/src/string/concat_ws.rs @@ -75,7 +75,11 @@ impl ScalarUDFImpl for ConcatWsFunc { /// Concatenates all but the first argument, with separators. The first argument is used as the separator string, and should not be NULL. Other NULL arguments are ignored. /// concat_ws(',', 'abcde', 2, NULL, 22) = 'abcde,2,22' - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { // do not accept 0 arguments. if args.len() < 2 { return exec_err!( @@ -275,14 +279,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_concat_ws_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description( - "Concatenates multiple strings together with a specified separator.", - ) - .with_syntax_example("concat_ws(separator, str[, ..., str_n])") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Concatenates multiple strings together with a specified separator.", + "concat_ws(separator, str[, ..., str_n])", + ) + .with_sql_example( + r#"```sql > select concat_ws('_', 'data', 'fusion'); +--------------------------------------------------+ | concat_ws(Utf8("_"),Utf8("data"),Utf8("fusion")) | @@ -290,16 +293,15 @@ fn get_concat_ws_doc() -> &'static Documentation { | data_fusion | +--------------------------------------------------+ ```"#, - ) - .with_argument( - "separator", - "Separator to insert between concatenated strings.", - ) - .with_standard_argument("str", Some("String")) - .with_argument("str_n", "Subsequent string expressions to concatenate.") - .with_related_udf("concat") - .build() - .unwrap() + ) + .with_argument( + "separator", + "Separator to insert between concatenated strings.", + ) + .with_standard_argument("str", Some("String")) + .with_argument("str_n", "Subsequent string expressions to concatenate.") + .with_related_udf("concat") + .build() }) } @@ -467,6 +469,7 @@ mod tests { ]))); let args = &[c0, c1, c2]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ConcatWsFunc::new().invoke_batch(args, 3)?; let expected = Arc::new(StringArray::from(vec!["foo,x", "bar", "baz,z"])) as ArrayRef; @@ -492,6 +495,7 @@ mod tests { ]))); let args = &[c0, c1, c2]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ConcatWsFunc::new().invoke_batch(args, 3)?; let expected = Arc::new(StringArray::from(vec![Some("foo,x"), None, Some("baz+z")])) diff --git a/datafusion/functions/src/string/contains.rs b/datafusion/functions/src/string/contains.rs index 3acd2464524d..3e5c72ac20e9 100644 --- a/datafusion/functions/src/string/contains.rs +++ b/datafusion/functions/src/string/contains.rs @@ -66,7 +66,11 @@ impl ScalarUDFImpl for ContainsFunc { Ok(Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(contains, vec![])(args) } @@ -79,14 +83,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_contains_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description( - "Return true if search_str is found within string (case-sensitive).", - ) - .with_syntax_example("contains(str, search_str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Return true if search_str is found within string (case-sensitive).", + "contains(str, search_str)", + ) + .with_sql_example( + r#"```sql > select contains('the quick brown fox', 'row'); +---------------------------------------------------+ | contains(Utf8("the quick brown fox"),Utf8("row")) | @@ -94,11 +97,10 @@ fn get_contains_doc() -> &'static Documentation { | true | +---------------------------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_argument("search_str", "The string to search for in str.") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_argument("search_str", "The string to search for in str.") + .build() }) } @@ -145,6 +147,7 @@ mod test { Some("yyy?()"), ]))); let scalar = ColumnarValue::Scalar(ScalarValue::Utf8(Some("x?(".to_string()))); + #[allow(deprecated)] // TODO migrate UDF to invoke let actual = udf.invoke_batch(&[array, scalar], 2).unwrap(); let expect = ColumnarValue::Array(Arc::new(BooleanArray::from(vec![ Some(true), diff --git a/datafusion/functions/src/string/ends_with.rs b/datafusion/functions/src/string/ends_with.rs index 88978a35c0b7..fc7fc04f4363 100644 --- a/datafusion/functions/src/string/ends_with.rs +++ b/datafusion/functions/src/string/ends_with.rs @@ -63,7 +63,11 @@ impl ScalarUDFImpl for EndsWithFunc { Ok(DataType::Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8View | DataType::Utf8 | DataType::LargeUtf8 => { make_scalar_function(ends_with, vec![])(args) @@ -83,12 +87,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_ends_with_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Tests if a string ends with a substring.") - .with_syntax_example("ends_with(str, substr)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Tests if a string ends with a substring.", + "ends_with(str, substr)", + ) + .with_sql_example( + r#"```sql > select ends_with('datafusion', 'soin'); +--------------------------------------------+ | ends_with(Utf8("datafusion"),Utf8("soin")) | @@ -102,11 +107,10 @@ fn get_ends_with_doc() -> &'static Documentation { | true | +--------------------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_argument("substr", "Substring to test for.") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_argument("substr", "Substring to test for.") + .build() }) } diff --git a/datafusion/functions/src/string/initcap.rs b/datafusion/functions/src/string/initcap.rs index 5fd1e7929881..a9090b0a6f43 100644 --- a/datafusion/functions/src/string/initcap.rs +++ b/datafusion/functions/src/string/initcap.rs @@ -64,7 +64,11 @@ impl ScalarUDFImpl for InitcapFunc { utf8_to_str_type(&arg_types[0], "initcap") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8 => make_scalar_function(initcap::, vec![])(args), DataType::LargeUtf8 => make_scalar_function(initcap::, vec![])(args), @@ -84,10 +88,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_initcap_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Capitalizes the first character in each word in the input string. Words are delimited by non-alphanumeric characters.") - .with_syntax_example("initcap(str)") + Documentation::builder( + DOC_SECTION_STRING, + "Capitalizes the first character in each word in the input string. Words are delimited by non-alphanumeric characters.", + "initcap(str)") .with_sql_example(r#"```sql > select initcap('apache datafusion'); +------------------------------------+ @@ -100,7 +104,6 @@ fn get_initcap_doc() -> &'static Documentation { .with_related_udf("lower") .with_related_udf("upper") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/levenshtein.rs b/datafusion/functions/src/string/levenshtein.rs index 558e71239f84..51ff428055e4 100644 --- a/datafusion/functions/src/string/levenshtein.rs +++ b/datafusion/functions/src/string/levenshtein.rs @@ -65,7 +65,11 @@ impl ScalarUDFImpl for LevenshteinFunc { utf8_to_int_type(&arg_types[0], "levenshtein") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8View | DataType::Utf8 => { make_scalar_function(levenshtein::, vec![])(args) @@ -86,10 +90,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_levenshtein_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns the [`Levenshtein distance`](https://en.wikipedia.org/wiki/Levenshtein_distance) between the two given strings.") - .with_syntax_example("levenshtein(str1, str2)") + Documentation::builder( + DOC_SECTION_STRING, + "Returns the [`Levenshtein distance`](https://en.wikipedia.org/wiki/Levenshtein_distance) between the two given strings.", + "levenshtein(str1, str2)") .with_sql_example(r#"```sql > select levenshtein('kitten', 'sitting'); +---------------------------------------------+ @@ -101,7 +105,6 @@ fn get_levenshtein_doc() -> &'static Documentation { .with_argument("str1", "String expression to compute Levenshtein distance with str2.") .with_argument("str2", "String expression to compute Levenshtein distance with str1.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/lower.rs b/datafusion/functions/src/string/lower.rs index 78887fde0a8e..67c80cb785b6 100644 --- a/datafusion/functions/src/string/lower.rs +++ b/datafusion/functions/src/string/lower.rs @@ -62,7 +62,11 @@ impl ScalarUDFImpl for LowerFunc { utf8_to_str_type(&arg_types[0], "lower") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { to_lower(args, "lower") } @@ -75,12 +79,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_lower_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Converts a string to lower-case.") - .with_syntax_example("lower(str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Converts a string to lower-case.", + "lower(str)", + ) + .with_sql_example( + r#"```sql > select lower('Ångström'); +-------------------------+ | lower(Utf8("Ångström")) | @@ -88,25 +93,25 @@ fn get_lower_doc() -> &'static Documentation { | ångström | +-------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_related_udf("initcap") - .with_related_udf("upper") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_related_udf("initcap") + .with_related_udf("upper") + .build() }) } #[cfg(test)] mod tests { use super::*; - use arrow::array::{ArrayRef, StringArray}; + use arrow::array::{Array, ArrayRef, StringArray}; use std::sync::Arc; fn to_lower(input: ArrayRef, expected: ArrayRef) -> Result<()> { let func = LowerFunc::new(); - let batch_size = input.len(); + let batch_len = input.len(); let args = vec![ColumnarValue::Array(input)]; - let result = match func.invoke_batch(&args, batch_size)? { + #[allow(deprecated)] // TODO migrate UDF to invoke + let result = match func.invoke_batch(&args, batch_len)? { ColumnarValue::Array(result) => result, _ => unreachable!("lower"), }; diff --git a/datafusion/functions/src/string/ltrim.rs b/datafusion/functions/src/string/ltrim.rs index 0b4c197646b6..93470368803a 100644 --- a/datafusion/functions/src/string/ltrim.rs +++ b/datafusion/functions/src/string/ltrim.rs @@ -18,15 +18,15 @@ use arrow::array::{ArrayRef, OffsetSizeTrait}; use arrow::datatypes::DataType; use std::any::Any; -use std::sync::OnceLock; use crate::string::common::*; use crate::utils::{make_scalar_function, utf8_to_str_type}; use datafusion_common::{exec_err, Result}; use datafusion_expr::function::Hint; -use datafusion_expr::scalar_doc_sections::DOC_SECTION_STRING; use datafusion_expr::{ColumnarValue, Documentation, TypeSignature, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; +use datafusion_macros::user_doc; +use std::sync::OnceLock; /// Returns the longest string with leading characters removed. If the characters are not specified, whitespace is removed. /// ltrim('zzzytest', 'xyz') = 'test' @@ -35,6 +35,33 @@ fn ltrim(args: &[ArrayRef]) -> Result { general_trim::(args, TrimType::Left, use_string_view) } +#[user_doc( + doc_section(label = "String Functions"), + description = "Trims the specified trim string from the beginning of a string. If no trim string is provided, all whitespace is removed from the start of the input string.", + syntax_example = "ltrim(str[, trim_str])", + sql_example = r#"```sql +> select ltrim(' datafusion '); ++-------------------------------+ +| ltrim(Utf8(" datafusion ")) | ++-------------------------------+ +| datafusion | ++-------------------------------+ +> select ltrim('___datafusion___', '_'); ++-------------------------------------------+ +| ltrim(Utf8("___datafusion___"),Utf8("_")) | ++-------------------------------------------+ +| datafusion___ | ++-------------------------------------------+ +```"#, + standard_argument(name = "str", prefix = "String"), + argument( + name = "trim_str", + description = r"String expression to trim from the beginning of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._" + ), + alternative_syntax = "trim(LEADING trim_str FROM str)", + related_udf(name = "btrim"), + related_udf(name = "rtrim") +)] #[derive(Debug)] pub struct LtrimFunc { signature: Signature, @@ -78,7 +105,11 @@ impl ScalarUDFImpl for LtrimFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8 | DataType::Utf8View => make_scalar_function( ltrim::, @@ -96,42 +127,10 @@ impl ScalarUDFImpl for LtrimFunc { } fn documentation(&self) -> Option<&Documentation> { - Some(get_ltrim_doc()) + self.doc() } } -static DOCUMENTATION: OnceLock = OnceLock::new(); - -fn get_ltrim_doc() -> &'static Documentation { - DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Trims the specified trim string from the beginning of a string. If no trim string is provided, all whitespace is removed from the start of the input string.") - .with_syntax_example("ltrim(str[, trim_str])") - .with_sql_example(r#"```sql -> select ltrim(' datafusion '); -+-------------------------------+ -| ltrim(Utf8(" datafusion ")) | -+-------------------------------+ -| datafusion | -+-------------------------------+ -> select ltrim('___datafusion___', '_'); -+-------------------------------------------+ -| ltrim(Utf8("___datafusion___"),Utf8("_")) | -+-------------------------------------------+ -| datafusion___ | -+-------------------------------------------+ -```"#) - .with_standard_argument("str", Some("String")) - .with_argument("trim_str", "String expression to trim from the beginning of the input string. Can be a constant, column, or function, and any combination of arithmetic operators. _Default is whitespace characters._") - .with_alternative_syntax("trim(LEADING trim_str FROM str)") - .with_related_udf("btrim") - .with_related_udf("rtrim") - .build() - .unwrap() - }) -} - #[cfg(test)] mod tests { use arrow::array::{Array, StringArray, StringViewArray}; diff --git a/datafusion/functions/src/string/octet_length.rs b/datafusion/functions/src/string/octet_length.rs index 89f71d457199..2dbfa6746d61 100644 --- a/datafusion/functions/src/string/octet_length.rs +++ b/datafusion/functions/src/string/octet_length.rs @@ -62,7 +62,11 @@ impl ScalarUDFImpl for OctetLengthFunc { utf8_to_int_type(&arg_types[0], "octet_length") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { if args.len() != 1 { return exec_err!( "octet_length function requires 1 argument, got {}", @@ -96,12 +100,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_octet_length_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns the length of a string in bytes.") - .with_syntax_example("octet_length(str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns the length of a string in bytes.", + "octet_length(str)", + ) + .with_sql_example( + r#"```sql > select octet_length('Ångström'); +--------------------------------+ | octet_length(Utf8("Ångström")) | @@ -109,12 +114,11 @@ fn get_octet_length_doc() -> &'static Documentation { | 10 | +--------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_related_udf("bit_length") - .with_related_udf("length") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_related_udf("bit_length") + .with_related_udf("length") + .build() }) } diff --git a/datafusion/functions/src/string/overlay.rs b/datafusion/functions/src/string/overlay.rs index 796776304f4a..ced263456802 100644 --- a/datafusion/functions/src/string/overlay.rs +++ b/datafusion/functions/src/string/overlay.rs @@ -77,7 +77,11 @@ impl ScalarUDFImpl for OverlayFunc { utf8_to_str_type(&arg_types[0], "overlay") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8View | DataType::Utf8 => { make_scalar_function(overlay::, vec![])(args) @@ -96,10 +100,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_overlay_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns the string which is replaced by another string from the specified position and specified count length.") - .with_syntax_example("overlay(str PLACING substr FROM pos [FOR count])") + Documentation::builder( + DOC_SECTION_STRING, + "Returns the string which is replaced by another string from the specified position and specified count length.", + "overlay(str PLACING substr FROM pos [FOR count])") .with_sql_example(r#"```sql > select overlay('Txxxxas' placing 'hom' from 2 for 4); +--------------------------------------------------------+ @@ -113,7 +117,6 @@ fn get_overlay_doc() -> &'static Documentation { .with_argument("pos", "The start position to start the replace in str.") .with_argument("count", "The count of characters to be replaced from start position of str. If not specified, will use substr length instead.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/repeat.rs b/datafusion/functions/src/string/repeat.rs index 249ce15d6dbe..4140a9b913ff 100644 --- a/datafusion/functions/src/string/repeat.rs +++ b/datafusion/functions/src/string/repeat.rs @@ -72,7 +72,11 @@ impl ScalarUDFImpl for RepeatFunc { utf8_to_str_type(&arg_types[0], "repeat") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(repeat, vec![])(args) } @@ -85,14 +89,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_repeat_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description( - "Returns a string with an input string repeated a specified number.", - ) - .with_syntax_example("repeat(str, n)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns a string with an input string repeated a specified number.", + "repeat(str, n)", + ) + .with_sql_example( + r#"```sql > select repeat('data', 3); +-------------------------------+ | repeat(Utf8("data"),Int64(3)) | @@ -100,11 +103,10 @@ fn get_repeat_doc() -> &'static Documentation { | datadatadata | +-------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_argument("n", "Number of times to repeat the input string.") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_argument("n", "Number of times to repeat the input string.") + .build() }) } diff --git a/datafusion/functions/src/string/replace.rs b/datafusion/functions/src/string/replace.rs index 91abc39da058..2439799f96d7 100644 --- a/datafusion/functions/src/string/replace.rs +++ b/datafusion/functions/src/string/replace.rs @@ -64,7 +64,11 @@ impl ScalarUDFImpl for ReplaceFunc { utf8_to_str_type(&arg_types[0], "replace") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8 => make_scalar_function(replace::, vec![])(args), DataType::LargeUtf8 => make_scalar_function(replace::, vec![])(args), @@ -84,10 +88,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_replace_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Replaces all occurrences of a specified substring in a string with a new substring.") - .with_syntax_example("replace(str, substr, replacement)") + Documentation::builder( + DOC_SECTION_STRING, + "Replaces all occurrences of a specified substring in a string with a new substring.", + "replace(str, substr, replacement)") .with_sql_example(r#"```sql > select replace('ABabbaBA', 'ab', 'cd'); +-------------------------------------------------+ @@ -100,7 +104,6 @@ fn get_replace_doc() -> &'static Documentation { .with_standard_argument("substr", Some("Substring expression to replace in the input string. Substring")) .with_standard_argument("replacement", Some("Replacement substring")) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/rtrim.rs b/datafusion/functions/src/string/rtrim.rs index e934147efbbe..b4fe8d432432 100644 --- a/datafusion/functions/src/string/rtrim.rs +++ b/datafusion/functions/src/string/rtrim.rs @@ -78,7 +78,11 @@ impl ScalarUDFImpl for RtrimFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8 | DataType::Utf8View => make_scalar_function( rtrim::, @@ -104,10 +108,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_rtrim_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Trims the specified trim string from the end of a string. If no trim string is provided, all whitespace is removed from the end of the input string.") - .with_syntax_example("rtrim(str[, trim_str])") + Documentation::builder( + DOC_SECTION_STRING, + "Trims the specified trim string from the end of a string. If no trim string is provided, all whitespace is removed from the end of the input string.", + "rtrim(str[, trim_str])") .with_sql_example(r#"```sql > select rtrim(' datafusion '); +-------------------------------+ @@ -128,7 +132,6 @@ fn get_rtrim_doc() -> &'static Documentation { .with_related_udf("btrim") .with_related_udf("ltrim") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/split_part.rs b/datafusion/functions/src/string/split_part.rs index ea01cb1f56f9..e55325db756d 100644 --- a/datafusion/functions/src/string/split_part.rs +++ b/datafusion/functions/src/string/split_part.rs @@ -81,7 +81,11 @@ impl ScalarUDFImpl for SplitPartFunc { utf8_to_str_type(&arg_types[0], "split_part") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { // First, determine if any of the arguments is an Array let len = args.iter().find_map(|arg| match arg { ColumnarValue::Array(a) => Some(a.len()), @@ -186,10 +190,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_split_part_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Splits a string based on a specified delimiter and returns the substring in the specified position.") - .with_syntax_example("split_part(str, delimiter, pos)") + Documentation::builder( + DOC_SECTION_STRING, + "Splits a string based on a specified delimiter and returns the substring in the specified position.", + "split_part(str, delimiter, pos)") .with_sql_example(r#"```sql > select split_part('1.2.3.4.5', '.', 3); +--------------------------------------------------+ @@ -202,7 +206,6 @@ fn get_split_part_doc() -> &'static Documentation { .with_argument("delimiter", "String or character to split on.") .with_argument("pos", "Position of the part to return.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/string/starts_with.rs b/datafusion/functions/src/string/starts_with.rs index dce161a2e14b..36dbd8167b4e 100644 --- a/datafusion/functions/src/string/starts_with.rs +++ b/datafusion/functions/src/string/starts_with.rs @@ -70,7 +70,11 @@ impl ScalarUDFImpl for StartsWithFunc { Ok(DataType::Boolean) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8View | DataType::Utf8 | DataType::LargeUtf8 => { make_scalar_function(starts_with, vec![])(args) @@ -88,12 +92,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_starts_with_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Tests if a string starts with a substring.") - .with_syntax_example("starts_with(str, substr)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Tests if a string starts with a substring.", + "starts_with(str, substr)", + ) + .with_sql_example( + r#"```sql > select starts_with('datafusion','data'); +----------------------------------------------+ | starts_with(Utf8("datafusion"),Utf8("data")) | @@ -101,11 +106,10 @@ fn get_starts_with_doc() -> &'static Documentation { | true | +----------------------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_argument("substr", "Substring to test for.") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_argument("substr", "Substring to test for.") + .build() }) } diff --git a/datafusion/functions/src/string/to_hex.rs b/datafusion/functions/src/string/to_hex.rs index e0033d2d1cb0..04907af14ade 100644 --- a/datafusion/functions/src/string/to_hex.rs +++ b/datafusion/functions/src/string/to_hex.rs @@ -103,7 +103,11 @@ impl ScalarUDFImpl for ToHexFunc { }) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Int32 => make_scalar_function(to_hex::, vec![])(args), DataType::Int64 => make_scalar_function(to_hex::, vec![])(args), @@ -120,12 +124,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_to_hex_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Converts an integer to a hexadecimal string.") - .with_syntax_example("to_hex(int)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Converts an integer to a hexadecimal string.", + "to_hex(int)", + ) + .with_sql_example( + r#"```sql > select to_hex(12345689); +-------------------------+ | to_hex(Int64(12345689)) | @@ -133,10 +138,9 @@ fn get_to_hex_doc() -> &'static Documentation { | bc6159 | +-------------------------+ ```"#, - ) - .with_standard_argument("int", Some("Integer")) - .build() - .unwrap() + ) + .with_standard_argument("int", Some("Integer")) + .build() }) } diff --git a/datafusion/functions/src/string/upper.rs b/datafusion/functions/src/string/upper.rs index 5039d094f2d6..1d05c42394a7 100644 --- a/datafusion/functions/src/string/upper.rs +++ b/datafusion/functions/src/string/upper.rs @@ -61,7 +61,11 @@ impl ScalarUDFImpl for UpperFunc { utf8_to_str_type(&arg_types[0], "upper") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { to_upper(args, "upper") } @@ -74,12 +78,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_upper_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Converts a string to upper-case.") - .with_syntax_example("upper(str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Converts a string to upper-case.", + "upper(str)", + ) + .with_sql_example( + r#"```sql > select upper('dataFusion'); +---------------------------+ | upper(Utf8("dataFusion")) | @@ -87,26 +92,26 @@ fn get_upper_doc() -> &'static Documentation { | DATAFUSION | +---------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_related_udf("initcap") - .with_related_udf("lower") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_related_udf("initcap") + .with_related_udf("lower") + .build() }) } #[cfg(test)] mod tests { use super::*; - use arrow::array::{ArrayRef, StringArray}; + use arrow::array::{Array, ArrayRef, StringArray}; use std::sync::Arc; fn to_upper(input: ArrayRef, expected: ArrayRef) -> Result<()> { let func = UpperFunc::new(); - let batch_size = input.len(); + let batch_len = input.len(); let args = vec![ColumnarValue::Array(input)]; - let result = match func.invoke_batch(&args, batch_size)? { + #[allow(deprecated)] // TODO migrate UDF to invoke + let result = match func.invoke_batch(&args, batch_len)? { ColumnarValue::Array(result) => result, _ => unreachable!("upper"), }; diff --git a/datafusion/functions/src/string/uuid.rs b/datafusion/functions/src/string/uuid.rs index 2282670dac79..6048a70bd8c5 100644 --- a/datafusion/functions/src/string/uuid.rs +++ b/datafusion/functions/src/string/uuid.rs @@ -88,10 +88,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_uuid_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns [`UUID v4`](https://en.wikipedia.org/wiki/Universally_unique_identifier#Version_4_(random)) string value which is unique per row.") - .with_syntax_example("uuid()") + Documentation::builder( + DOC_SECTION_STRING, + "Returns [`UUID v4`](https://en.wikipedia.org/wiki/Universally_unique_identifier#Version_4_(random)) string value which is unique per row.", + "uuid()") .with_sql_example(r#"```sql > select uuid(); +--------------------------------------+ @@ -101,6 +101,5 @@ fn get_uuid_doc() -> &'static Documentation { +--------------------------------------+ ```"#) .build() - .unwrap() }) } diff --git a/datafusion/functions/src/strings.rs b/datafusion/functions/src/strings.rs index e0cec3cb5756..d2fb5d58519e 100644 --- a/datafusion/functions/src/strings.rs +++ b/datafusion/functions/src/strings.rs @@ -395,7 +395,7 @@ pub enum ColumnarValueRef<'a> { NonNullableStringViewArray(&'a StringViewArray), } -impl<'a> ColumnarValueRef<'a> { +impl ColumnarValueRef<'_> { #[inline] pub fn is_valid(&self, i: usize) -> bool { match &self { diff --git a/datafusion/functions/src/unicode/character_length.rs b/datafusion/functions/src/unicode/character_length.rs index eca8d3fd493d..726822a8f887 100644 --- a/datafusion/functions/src/unicode/character_length.rs +++ b/datafusion/functions/src/unicode/character_length.rs @@ -72,7 +72,11 @@ impl ScalarUDFImpl for CharacterLengthFunc { utf8_to_int_type(&arg_types[0], "character_length") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(character_length, vec![])(args) } @@ -89,12 +93,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_character_length_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns the number of characters in a string.") - .with_syntax_example("character_length(str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns the number of characters in a string.", + "character_length(str)", + ) + .with_sql_example( + r#"```sql > select character_length('Ångström'); +------------------------------------+ | character_length(Utf8("Ångström")) | @@ -102,12 +107,11 @@ fn get_character_length_doc() -> &'static Documentation { | 8 | +------------------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .with_related_udf("bit_length") - .with_related_udf("octet_length") - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .with_related_udf("bit_length") + .with_related_udf("octet_length") + .build() }) } diff --git a/datafusion/functions/src/unicode/find_in_set.rs b/datafusion/functions/src/unicode/find_in_set.rs index cad860e41088..38efb408c1d3 100644 --- a/datafusion/functions/src/unicode/find_in_set.rs +++ b/datafusion/functions/src/unicode/find_in_set.rs @@ -76,7 +76,11 @@ impl ScalarUDFImpl for FindInSetFunc { utf8_to_int_type(&arg_types[0], "find_in_set") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(find_in_set, vec![])(args) } @@ -89,10 +93,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_find_in_set_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns a value in the range of 1 to N if the string str is in the string list strlist consisting of N substrings.") - .with_syntax_example("find_in_set(str, strlist)") + Documentation::builder( + DOC_SECTION_STRING, + "Returns a value in the range of 1 to N if the string str is in the string list strlist consisting of N substrings.", + "find_in_set(str, strlist)") .with_sql_example(r#"```sql > select find_in_set('b', 'a,b,c,d'); +----------------------------------------+ @@ -104,7 +108,6 @@ fn get_find_in_set_doc() -> &'static Documentation { .with_argument("str", "String expression to find in strlist.") .with_argument("strlist", "A string list is a string composed of substrings separated by , characters.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/unicode/left.rs b/datafusion/functions/src/unicode/left.rs index a6c2b9768f0b..ef2802340b14 100644 --- a/datafusion/functions/src/unicode/left.rs +++ b/datafusion/functions/src/unicode/left.rs @@ -81,7 +81,11 @@ impl ScalarUDFImpl for LeftFunc { utf8_to_str_type(&arg_types[0], "left") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8 | DataType::Utf8View => { make_scalar_function(left::, vec![])(args) @@ -103,23 +107,25 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_left_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns a specified number of characters from the left side of a string.") - .with_syntax_example("left(str, n)") - .with_sql_example(r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns a specified number of characters from the left side of a string.", + "left(str, n)", + ) + .with_sql_example( + r#"```sql > select left('datafusion', 4); +-----------------------------------+ | left(Utf8("datafusion"),Int64(4)) | +-----------------------------------+ | data | +-----------------------------------+ -```"#) - .with_standard_argument("str", Some("String")) - .with_argument("n", "Number of characters to return.") - .with_related_udf("right") - .build() - .unwrap() +```"#, + ) + .with_standard_argument("str", Some("String")) + .with_argument("n", "Number of characters to return.") + .with_related_udf("right") + .build() }) } diff --git a/datafusion/functions/src/unicode/lpad.rs b/datafusion/functions/src/unicode/lpad.rs index a639bcedcd1f..6c8a4ec97bb0 100644 --- a/datafusion/functions/src/unicode/lpad.rs +++ b/datafusion/functions/src/unicode/lpad.rs @@ -90,7 +90,11 @@ impl ScalarUDFImpl for LPadFunc { utf8_to_str_type(&arg_types[0], "lpad") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { Utf8 | Utf8View => make_scalar_function(lpad::, vec![])(args), LargeUtf8 => make_scalar_function(lpad::, vec![])(args), @@ -107,10 +111,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_lpad_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Pads the left side of a string with another string to a specified string length.") - .with_syntax_example("lpad(str, n[, padding_str])") + Documentation::builder( + DOC_SECTION_STRING, + "Pads the left side of a string with another string to a specified string length.", + "lpad(str, n[, padding_str])") .with_sql_example(r#"```sql > select lpad('Dolly', 10, 'hello'); +---------------------------------------------+ @@ -124,7 +128,6 @@ fn get_lpad_doc() -> &'static Documentation { .with_argument("padding_str", "Optional string expression to pad with. Can be a constant, column, or function, and any combination of string operators. _Default is a space._") .with_related_udf("rpad") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/unicode/reverse.rs b/datafusion/functions/src/unicode/reverse.rs index baf3b56636e2..38c1f23cbd5a 100644 --- a/datafusion/functions/src/unicode/reverse.rs +++ b/datafusion/functions/src/unicode/reverse.rs @@ -72,7 +72,11 @@ impl ScalarUDFImpl for ReverseFunc { utf8_to_str_type(&arg_types[0], "reverse") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { Utf8 | Utf8View => make_scalar_function(reverse::, vec![])(args), LargeUtf8 => make_scalar_function(reverse::, vec![])(args), @@ -91,12 +95,13 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_reverse_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Reverses the character order of a string.") - .with_syntax_example("reverse(str)") - .with_sql_example( - r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Reverses the character order of a string.", + "reverse(str)", + ) + .with_sql_example( + r#"```sql > select reverse('datafusion'); +-----------------------------+ | reverse(Utf8("datafusion")) | @@ -104,10 +109,9 @@ fn get_reverse_doc() -> &'static Documentation { | noisufatad | +-----------------------------+ ```"#, - ) - .with_standard_argument("str", Some("String")) - .build() - .unwrap() + ) + .with_standard_argument("str", Some("String")) + .build() }) } diff --git a/datafusion/functions/src/unicode/right.rs b/datafusion/functions/src/unicode/right.rs index ab3b7ba1a27e..1586e23eb8aa 100644 --- a/datafusion/functions/src/unicode/right.rs +++ b/datafusion/functions/src/unicode/right.rs @@ -81,7 +81,11 @@ impl ScalarUDFImpl for RightFunc { utf8_to_str_type(&arg_types[0], "right") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match args[0].data_type() { DataType::Utf8 | DataType::Utf8View => { make_scalar_function(right::, vec![])(args) @@ -103,23 +107,25 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_right_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns a specified number of characters from the right side of a string.") - .with_syntax_example("right(str, n)") - .with_sql_example(r#"```sql + Documentation::builder( + DOC_SECTION_STRING, + "Returns a specified number of characters from the right side of a string.", + "right(str, n)", + ) + .with_sql_example( + r#"```sql > select right('datafusion', 6); +------------------------------------+ | right(Utf8("datafusion"),Int64(6)) | +------------------------------------+ | fusion | +------------------------------------+ -```"#) - .with_standard_argument("str", Some("String")) - .with_argument("n", "Number of characters to return") - .with_related_udf("left") - .build() - .unwrap() +```"#, + ) + .with_standard_argument("str", Some("String")) + .with_argument("n", "Number of characters to return") + .with_related_udf("left") + .build() }) } diff --git a/datafusion/functions/src/unicode/rpad.rs b/datafusion/functions/src/unicode/rpad.rs index bd9d625105e9..6e6bde3e177c 100644 --- a/datafusion/functions/src/unicode/rpad.rs +++ b/datafusion/functions/src/unicode/rpad.rs @@ -89,7 +89,11 @@ impl ScalarUDFImpl for RPadFunc { utf8_to_str_type(&arg_types[0], "rpad") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { match ( args.len(), args[0].data_type(), @@ -126,10 +130,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_rpad_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Pads the right side of a string with another string to a specified string length.") - .with_syntax_example("rpad(str, n[, padding_str])") + Documentation::builder( + DOC_SECTION_STRING, + "Pads the right side of a string with another string to a specified string length.", + "rpad(str, n[, padding_str])") .with_sql_example(r#"```sql > select rpad('datafusion', 20, '_-'); +-----------------------------------------------+ @@ -147,7 +151,6 @@ fn get_rpad_doc() -> &'static Documentation { "String expression to pad with. Can be a constant, column, or function, and any combination of string operators. _Default is a space._") .with_related_udf("lpad") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/unicode/strpos.rs b/datafusion/functions/src/unicode/strpos.rs index 9c84590f7f94..1917cd7291cc 100644 --- a/datafusion/functions/src/unicode/strpos.rs +++ b/datafusion/functions/src/unicode/strpos.rs @@ -66,7 +66,11 @@ impl ScalarUDFImpl for StrposFunc { utf8_to_int_type(&arg_types[0], "strpos/instr/position") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(strpos, vec![])(args) } @@ -83,10 +87,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_strpos_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Returns the starting position of a specified substring in a string. Positions begin at 1. If the substring does not exist in the string, the function returns 0.") - .with_syntax_example("strpos(str, substr)") + Documentation::builder( + DOC_SECTION_STRING, + "Returns the starting position of a specified substring in a string. Positions begin at 1. If the substring does not exist in the string, the function returns 0.", + "strpos(str, substr)") .with_sql_example(r#"```sql > select strpos('datafusion', 'fus'); +----------------------------------------+ @@ -99,7 +103,6 @@ fn get_strpos_doc() -> &'static Documentation { .with_argument("substr", "Substring expression to search for.") .with_alternative_syntax("position(substr in origstr)") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/unicode/substr.rs b/datafusion/functions/src/unicode/substr.rs index edfe57210b71..0ac050c707bf 100644 --- a/datafusion/functions/src/unicode/substr.rs +++ b/datafusion/functions/src/unicode/substr.rs @@ -75,7 +75,11 @@ impl ScalarUDFImpl for SubstrFunc { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(substr, vec![])(args) } @@ -158,10 +162,10 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_substr_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Extracts a substring of a specified number of characters from a specific starting position in a string.") - .with_syntax_example("substr(str, start_pos[, length])") + Documentation::builder( + DOC_SECTION_STRING, + "Extracts a substring of a specified number of characters from a specific starting position in a string.", + "substr(str, start_pos[, length])") .with_sql_example(r#"```sql > select substr('datafusion', 5, 3); +----------------------------------------------+ @@ -175,7 +179,6 @@ fn get_substr_doc() -> &'static Documentation { .with_argument("length", "Number of characters to extract. If not specified, returns the rest of the string after the start position.") .with_alternative_syntax("substring(str from start_pos for length)") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/unicode/substrindex.rs b/datafusion/functions/src/unicode/substrindex.rs index c04839783f58..825666b0455e 100644 --- a/datafusion/functions/src/unicode/substrindex.rs +++ b/datafusion/functions/src/unicode/substrindex.rs @@ -78,7 +78,11 @@ impl ScalarUDFImpl for SubstrIndexFunc { utf8_to_str_type(&arg_types[0], "substr_index") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(substr_index, vec![])(args) } @@ -95,12 +99,12 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_substr_index_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description(r#"Returns the substring from str before count occurrences of the delimiter delim. + Documentation::builder( + DOC_SECTION_STRING, + r#"Returns the substring from str before count occurrences of the delimiter delim. If count is positive, everything to the left of the final delimiter (counting from the left) is returned. -If count is negative, everything to the right of the final delimiter (counting from the right) is returned."#) - .with_syntax_example("substr_index(str, delim, count)") +If count is negative, everything to the right of the final delimiter (counting from the right) is returned."#, + "substr_index(str, delim, count)") .with_sql_example(r#"```sql > select substr_index('www.apache.org', '.', 1); +---------------------------------------------------------+ @@ -119,7 +123,6 @@ If count is negative, everything to the right of the final delimiter (counting f .with_argument("delim", "The string to find in str to split str.") .with_argument("count", "The number of times to search for the delimiter. Can be either a positive or negative number.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/unicode/translate.rs b/datafusion/functions/src/unicode/translate.rs index 845d34c708d4..780603777133 100644 --- a/datafusion/functions/src/unicode/translate.rs +++ b/datafusion/functions/src/unicode/translate.rs @@ -76,7 +76,11 @@ impl ScalarUDFImpl for TranslateFunc { utf8_to_str_type(&arg_types[0], "translate") } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(invoke_translate, vec![])(args) } @@ -89,10 +93,7 @@ static DOCUMENTATION: OnceLock = OnceLock::new(); fn get_translate_doc() -> &'static Documentation { DOCUMENTATION.get_or_init(|| { - Documentation::builder() - .with_doc_section(DOC_SECTION_STRING) - .with_description("Translates characters in a string to specified translation characters.") - .with_syntax_example("translate(str, chars, translation)") + Documentation::builder(DOC_SECTION_STRING,"Translates characters in a string to specified translation characters.","translate(str, chars, translation)") .with_sql_example(r#"```sql > select translate('twice', 'wic', 'her'); +--------------------------------------------------+ @@ -105,7 +106,6 @@ fn get_translate_doc() -> &'static Documentation { .with_argument("chars", "Characters to translate.") .with_argument("translation", "Translation characters. Translation characters replace only characters at the same position in the **chars** string.") .build() - .unwrap() }) } diff --git a/datafusion/functions/src/utils.rs b/datafusion/functions/src/utils.rs index 87180cb77de7..8b473500416b 100644 --- a/datafusion/functions/src/utils.rs +++ b/datafusion/functions/src/utils.rs @@ -146,9 +146,10 @@ pub mod test { match expected { Ok(expected) => { assert_eq!(return_type.is_ok(), true); - assert_eq!(return_type.unwrap(), $EXPECTED_DATA_TYPE); + let return_type = return_type.unwrap(); + assert_eq!(return_type, $EXPECTED_DATA_TYPE); - let result = func.invoke_batch($ARGS, cardinality); + let result = func.invoke_with_args(datafusion_expr::ScalarFunctionArgs{args: $ARGS, number_rows: cardinality, return_type: &return_type}); assert_eq!(result.is_ok(), true, "function returned an error: {}", result.unwrap_err()); let result = result.unwrap().clone().into_array(cardinality).expect("Failed to convert to array"); @@ -169,7 +170,7 @@ pub mod test { } else { // invoke is expected error - cannot use .expect_err() due to Debug not being implemented - match func.invoke_batch($ARGS, cardinality) { + match func.invoke_with_args(datafusion_expr::ScalarFunctionArgs{args: $ARGS, number_rows: cardinality, return_type: &return_type.unwrap()}) { Ok(_) => assert!(false, "expected error"), Err(error) => { assert!(expected_error.strip_backtrace().starts_with(&error.strip_backtrace())); diff --git a/datafusion/macros/Cargo.toml b/datafusion/macros/Cargo.toml new file mode 100644 index 000000000000..c5ac9d08dffa --- /dev/null +++ b/datafusion/macros/Cargo.toml @@ -0,0 +1,42 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[package] +name = "datafusion-macros" +description = "Procedural macros for DataFusion query engine" +keywords = ["datafusion", "query", "sql"] +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +authors = { workspace = true } +rust-version = { workspace = true } + +[lints] +workspace = true + +[lib] +name = "datafusion_macros" +path = "src/lib.rs" +proc-macro = true + +[dependencies] +datafusion-doc = { workspace = true } +proc-macro2 = "1.0" +quote = "1.0.37" +syn = { version = "2.0.79", features = ["full"] } diff --git a/datafusion/macros/src/lib.rs b/datafusion/macros/src/lib.rs new file mode 100644 index 000000000000..e4eeeba7a4ab --- /dev/null +++ b/datafusion/macros/src/lib.rs @@ -0,0 +1,255 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +extern crate proc_macro; +use proc_macro::TokenStream; +use quote::quote; +use syn::{parse_macro_input, DeriveInput, LitStr}; + +/// This procedural macro is intended to parse a rust custom attribute and create user documentation +/// from it by constructing a `DocumentBuilder()` automatically. The `Documentation` can be +/// retrieved from the `documentation()` method +/// declared on `AggregateUDF`, `WindowUDFImpl`, `ScalarUDFImpl` traits. +/// +/// Example: +/// #[user_doc( +/// doc_section(include = "true", label = "Time and Date Functions"), +/// description = r"Converts a value to a date (`YYYY-MM-DD`)." +/// sql_example = "```sql\n\ +/// \> select to_date('2023-01-31');\n\ +/// +-----------------------------+\n\ +/// | to_date(Utf8(\"2023-01-31\")) |\n\ +/// +-----------------------------+\n\ +/// | 2023-01-31 |\n\ +/// +-----------------------------+\n\"), +/// standard_argument(name = "expression", prefix = "String"), +/// argument( +/// name = "format_n", +/// description = r"Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order +/// they appear with the first successful one being returned. If none of the formats successfully parse the expression +/// an error will be returned." +/// ) +/// )] +/// #[derive(Debug)] +/// pub struct ToDateFunc { +/// signature: Signature, +/// } +/// +/// will generate the following code +/// +/// #[derive(Debug)] pub struct ToDateFunc { signature : Signature, } +/// use datafusion_doc :: DocSection; +/// use datafusion_doc :: DocumentationBuilder; +/// static DOCUMENTATION : OnceLock < Documentation > = OnceLock :: new(); +/// impl ToDateFunc +/// { +/// fn doc(& self) -> Option < & Documentation > +/// { +/// Some(DOCUMENTATION.get_or_init(|| +/// { +/// Documentation :: +/// builder(DocSection +/// { +/// include : true, label : "Time and Date Functions", description +/// : None +/// }, r"Converts a value to a date (`YYYY-MM-DD`).") +/// .with_syntax_example("to_date('2017-05-31', '%Y-%m-%d')".to_string(),"```sql\n\ +/// \> select to_date('2023-01-31');\n\ +/// +-----------------------------+\n\ +/// | to_date(Utf8(\"2023-01-31\")) |\n\ +/// +-----------------------------+\n\ +/// | 2023-01-31 |\n\ +/// +-----------------------------+\n\) +/// .with_standard_argument("expression", "String".into()) +/// .with_argument("format_n", +/// r"Optional [Chrono format](https://docs.rs/chrono/latest/chrono/format/strftime/index.html) strings to use to parse the expression. Formats will be tried in the order +/// they appear with the first successful one being returned. If none of the formats successfully parse the expression +/// an error will be returned.").build() +/// })) +/// } +/// } +#[proc_macro_attribute] +pub fn user_doc(args: TokenStream, input: TokenStream) -> TokenStream { + let mut doc_section_include: Option = None; + let mut doc_section_lbl: Option = None; + let mut doc_section_desc: Option = None; + + let mut description: Option = None; + let mut syntax_example: Option = None; + let mut alt_syntax_example: Option = None; + let mut sql_example: Option = None; + let mut standard_args: Vec<(Option, Option)> = vec![]; + let mut udf_args: Vec<(Option, Option)> = vec![]; + let mut related_udfs: Vec> = vec![]; + + let parser = syn::meta::parser(|meta| { + if meta.path.is_ident("doc_section") { + meta.parse_nested_meta(|meta| { + if meta.path.is_ident("include") { + doc_section_include = meta.value()?.parse()?; + return Ok(()); + } else if meta.path.is_ident("label") { + doc_section_lbl = meta.value()?.parse()?; + return Ok(()); + } else if meta.path.is_ident("description") { + doc_section_desc = meta.value()?.parse()?; + return Ok(()); + } + Ok(()) + }) + } else if meta.path.is_ident("description") { + description = Some(meta.value()?.parse()?); + Ok(()) + } else if meta.path.is_ident("syntax_example") { + syntax_example = Some(meta.value()?.parse()?); + Ok(()) + } else if meta.path.is_ident("alternative_syntax") { + alt_syntax_example = Some(meta.value()?.parse()?); + Ok(()) + } else if meta.path.is_ident("sql_example") { + sql_example = Some(meta.value()?.parse()?); + Ok(()) + } else if meta.path.is_ident("standard_argument") { + let mut standard_arg: (Option, Option) = (None, None); + let m = meta.parse_nested_meta(|meta| { + if meta.path.is_ident("name") { + standard_arg.0 = meta.value()?.parse()?; + return Ok(()); + } else if meta.path.is_ident("prefix") { + standard_arg.1 = meta.value()?.parse()?; + return Ok(()); + } + Ok(()) + }); + + standard_args.push(standard_arg.clone()); + + m + } else if meta.path.is_ident("argument") { + let mut arg: (Option, Option) = (None, None); + let m = meta.parse_nested_meta(|meta| { + if meta.path.is_ident("name") { + arg.0 = meta.value()?.parse()?; + return Ok(()); + } else if meta.path.is_ident("description") { + arg.1 = meta.value()?.parse()?; + return Ok(()); + } + Ok(()) + }); + + udf_args.push(arg.clone()); + + m + } else if meta.path.is_ident("related_udf") { + let mut arg: Option = None; + let m = meta.parse_nested_meta(|meta| { + if meta.path.is_ident("name") { + arg = meta.value()?.parse()?; + return Ok(()); + } + Ok(()) + }); + + related_udfs.push(arg.clone()); + + m + } else { + Err(meta.error(format!("Unsupported property: {:?}", meta.path.get_ident()))) + } + }); + + parse_macro_input!(args with parser); + + // Parse the input struct + let input = parse_macro_input!(input as DeriveInput); + let name = input.clone().ident; + + let doc_section_include: bool = if let Some(doc_section_include) = doc_section_include + { + doc_section_include.value().parse().unwrap() + } else { + true + }; + + let doc_section_description = doc_section_desc + .map(|desc| quote! { Some(#desc)}) + .unwrap_or(quote! { None }); + + let udf_args = udf_args + .iter() + .map(|(name, desc)| { + quote! { + .with_argument(#name, #desc) + } + }) + .collect::>(); + + let standard_args = standard_args + .iter() + .map(|(name, desc)| { + quote! { + .with_standard_argument(#name, #desc.into()) + } + }) + .collect::>(); + + let related_udfs = related_udfs + .iter() + .map(|name| { + quote! { + .with_related_udf(#name) + } + }) + .collect::>(); + + let alt_syntax_example = alt_syntax_example.map(|syn| { + quote! { + .with_alternative_syntax(#syn) + } + }); + + let generated = quote! { + #input + + use datafusion_doc::DocSection; + use datafusion_doc::DocumentationBuilder; + + static DOCUMENTATION: OnceLock = OnceLock::new(); + + impl #name { + fn doc(&self) -> Option<&Documentation> { + Some(DOCUMENTATION.get_or_init(|| { + Documentation::builder(DocSection { include: #doc_section_include, label: #doc_section_lbl, description: #doc_section_description }, + #description.to_string(), #syntax_example.to_string()) + .with_sql_example(#sql_example.to_string()) + #alt_syntax_example + #(#standard_args)* + #(#udf_args)* + #(#related_udfs)* + .build() + })) + } + } + }; + + // Debug the generated code if needed + //eprintln!("Generated code: {}", generated); + + // Return the generated code + TokenStream::from(generated) +} diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index 34e35c66107a..c0f17de6c5c5 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -42,17 +42,14 @@ chrono = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } datafusion-physical-expr = { workspace = true } -hashbrown = { workspace = true } indexmap = { workspace = true } itertools = { workspace = true } log = { workspace = true } -paste = "1.0.14" recursive = { workspace = true } regex = { workspace = true } regex-syntax = "0.8.0" [dev-dependencies] -arrow-buffer = { workspace = true } ctor = { workspace = true } datafusion-functions-aggregate = { workspace = true } datafusion-functions-window-common = { workspace = true } diff --git a/datafusion/optimizer/LICENSE.txt b/datafusion/optimizer/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/optimizer/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/optimizer/NOTICE.txt b/datafusion/optimizer/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/optimizer/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/optimizer/src/analyzer/subquery.rs b/datafusion/optimizer/src/analyzer/subquery.rs index 0b54b302c2df..fee06eeb9f75 100644 --- a/datafusion/optimizer/src/analyzer/subquery.rs +++ b/datafusion/optimizer/src/analyzer/subquery.rs @@ -163,7 +163,8 @@ fn check_inner_plan(inner_plan: &LogicalPlan, can_contain_outer_ref: bool) -> Re | LogicalPlan::Limit(_) | LogicalPlan::Values(_) | LogicalPlan::Subquery(_) - | LogicalPlan::SubqueryAlias(_) => { + | LogicalPlan::SubqueryAlias(_) + | LogicalPlan::Unnest(_) => { inner_plan.apply_children(|plan| { check_inner_plan(plan, can_contain_outer_ref)?; Ok(TreeNodeRecursion::Continue) diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index b56c2dc604a9..58e390ee8bdb 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -290,7 +290,7 @@ impl<'a> TypeCoercionRewriter<'a> { } } -impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { +impl TreeNodeRewriter for TypeCoercionRewriter<'_> { type Node = Expr; fn f_up(&mut self, expr: Expr) -> Result> { @@ -692,6 +692,7 @@ fn extract_window_frame_target_type(col_type: &DataType) -> Result { if col_type.is_numeric() || is_utf8_or_large_utf8(col_type) || matches!(col_type, DataType::Null) + || matches!(col_type, DataType::Boolean) { Ok(col_type.clone()) } else if is_datetime(col_type) { @@ -1250,7 +1251,11 @@ mod test { Ok(Utf8) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { Ok(ColumnarValue::Scalar(ScalarValue::from("a"))) } } diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 16a4fa6be38d..0ea2d24effbb 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -1450,7 +1450,11 @@ mod test { Ok(DataType::Float64) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { unimplemented!() } } diff --git a/datafusion/optimizer/src/eliminate_group_by_constant.rs b/datafusion/optimizer/src/eliminate_group_by_constant.rs index 13d03d647fe2..035a1d2da229 100644 --- a/datafusion/optimizer/src/eliminate_group_by_constant.rs +++ b/datafusion/optimizer/src/eliminate_group_by_constant.rs @@ -155,7 +155,11 @@ mod tests { fn return_type(&self, _args: &[DataType]) -> Result { Ok(DataType::Int32) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { unimplemented!() } } diff --git a/datafusion/optimizer/src/join_key_set.rs b/datafusion/optimizer/src/join_key_set.rs index c0eec78b183d..0a97173b3096 100644 --- a/datafusion/optimizer/src/join_key_set.rs +++ b/datafusion/optimizer/src/join_key_set.rs @@ -148,7 +148,7 @@ impl<'a> ExprPair<'a> { } } -impl<'a> Equivalent<(Expr, Expr)> for ExprPair<'a> { +impl Equivalent<(Expr, Expr)> for ExprPair<'_> { fn equivalent(&self, other: &(Expr, Expr)) -> bool { self.0 == &other.0 && self.1 == &other.1 } diff --git a/datafusion/optimizer/src/optimize_projections/required_indices.rs b/datafusion/optimizer/src/optimize_projections/required_indices.rs index 60d8ef1a8e6c..55e8081eaeb2 100644 --- a/datafusion/optimizer/src/optimize_projections/required_indices.rs +++ b/datafusion/optimizer/src/optimize_projections/required_indices.rs @@ -33,7 +33,7 @@ use datafusion_expr::{Expr, LogicalPlan}; /// /// Indices are always in order and without duplicates. For example, if these /// indices were added `[3, 2, 4, 3, 6, 1]`, the instance would be represented -/// by `[1, 2, 3, 6]`. +/// by `[1, 2, 3, 4, 6]`. #[derive(Debug, Clone, Default)] pub(super) struct RequiredIndicies { /// The indices of the required columns in the diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 975150cd6122..dfdd0c110c22 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -68,7 +68,6 @@ use crate::utils::log_plan; /// /// [`AnalyzerRule`]: crate::analyzer::AnalyzerRule /// [`SessionState::add_optimizer_rule`]: https://docs.rs/datafusion/latest/datafusion/execution/session_state/struct.SessionState.html#method.add_optimizer_rule - pub trait OptimizerRule: Debug { /// Try and rewrite `plan` to an optimized form, returning None if the plan /// cannot be optimized by this rule. @@ -302,7 +301,7 @@ impl<'a> Rewriter<'a> { } } -impl<'a> TreeNodeRewriter for Rewriter<'a> { +impl TreeNodeRewriter for Rewriter<'_> { type Node = LogicalPlan; fn f_down(&mut self, node: LogicalPlan) -> Result> { diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 23cd46803c78..fe751a5fb583 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -988,22 +988,32 @@ impl OptimizerRule for PushDownFilter { LogicalPlan::Join(join) => push_down_join(join, Some(&filter.predicate)), LogicalPlan::TableScan(scan) => { let filter_predicates = split_conjunction(&filter.predicate); - let results = scan + + let (volatile_filters, non_volatile_filters): (Vec<&Expr>, Vec<&Expr>) = + filter_predicates + .into_iter() + .partition(|pred| pred.is_volatile()); + + // Check which non-volatile filters are supported by source + let supported_filters = scan .source - .supports_filters_pushdown(filter_predicates.as_slice())?; - if filter_predicates.len() != results.len() { + .supports_filters_pushdown(non_volatile_filters.as_slice())?; + if non_volatile_filters.len() != supported_filters.len() { return internal_err!( "Vec returned length: {} from supports_filters_pushdown is not the same size as the filters passed, which length is: {}", - results.len(), - filter_predicates.len()); + supported_filters.len(), + non_volatile_filters.len()); } - let zip = filter_predicates.into_iter().zip(results); + // Compose scan filters from non-volatile filters of `Exact` or `Inexact` pushdown type + let zip = non_volatile_filters.into_iter().zip(supported_filters); let new_scan_filters = zip .clone() .filter(|(_, res)| res != &TableProviderFilterPushDown::Unsupported) .map(|(pred, _)| pred); + + // Add new scan filters let new_scan_filters: Vec = scan .filters .iter() @@ -1011,9 +1021,13 @@ impl OptimizerRule for PushDownFilter { .unique() .cloned() .collect(); + + // Compose predicates to be of `Unsupported` or `Inexact` pushdown type, and also include volatile filters let new_predicate: Vec = zip .filter(|(_, res)| res != &TableProviderFilterPushDown::Exact) - .map(|(pred, _)| pred.clone()) + .map(|(pred, _)| pred) + .chain(volatile_filters) + .cloned() .collect(); let new_scan = LogicalPlan::TableScan(TableScan { @@ -2515,23 +2529,31 @@ mod tests { } } - fn table_scan_with_pushdown_provider( + fn table_scan_with_pushdown_provider_builder( filter_support: TableProviderFilterPushDown, - ) -> Result { + filters: Vec, + projection: Option>, + ) -> Result { let test_provider = PushDownProvider { filter_support }; let table_scan = LogicalPlan::TableScan(TableScan { table_name: "test".into(), - filters: vec![], + filters, projected_schema: Arc::new(DFSchema::try_from( (*test_provider.schema()).clone(), )?), - projection: None, + projection, source: Arc::new(test_provider), fetch: None, }); - LogicalPlanBuilder::from(table_scan) + Ok(LogicalPlanBuilder::from(table_scan)) + } + + fn table_scan_with_pushdown_provider( + filter_support: TableProviderFilterPushDown, + ) -> Result { + table_scan_with_pushdown_provider_builder(filter_support, vec![], None)? .filter(col("a").eq(lit(1i64)))? .build() } @@ -2588,25 +2610,14 @@ mod tests { #[test] fn multi_combined_filter() -> Result<()> { - let test_provider = PushDownProvider { - filter_support: TableProviderFilterPushDown::Inexact, - }; - - let table_scan = LogicalPlan::TableScan(TableScan { - table_name: "test".into(), - filters: vec![col("a").eq(lit(10i64)), col("b").gt(lit(11i64))], - projected_schema: Arc::new(DFSchema::try_from( - (*test_provider.schema()).clone(), - )?), - projection: Some(vec![0]), - source: Arc::new(test_provider), - fetch: None, - }); - - let plan = LogicalPlanBuilder::from(table_scan) - .filter(and(col("a").eq(lit(10i64)), col("b").gt(lit(11i64))))? - .project(vec![col("a"), col("b")])? - .build()?; + let plan = table_scan_with_pushdown_provider_builder( + TableProviderFilterPushDown::Inexact, + vec![col("a").eq(lit(10i64)), col("b").gt(lit(11i64))], + Some(vec![0]), + )? + .filter(and(col("a").eq(lit(10i64)), col("b").gt(lit(11i64))))? + .project(vec![col("a"), col("b")])? + .build()?; let expected = "Projection: a, b\ \n Filter: a = Int64(10) AND b > Int64(11)\ @@ -2617,25 +2628,14 @@ mod tests { #[test] fn multi_combined_filter_exact() -> Result<()> { - let test_provider = PushDownProvider { - filter_support: TableProviderFilterPushDown::Exact, - }; - - let table_scan = LogicalPlan::TableScan(TableScan { - table_name: "test".into(), - filters: vec![], - projected_schema: Arc::new(DFSchema::try_from( - (*test_provider.schema()).clone(), - )?), - projection: Some(vec![0]), - source: Arc::new(test_provider), - fetch: None, - }); - - let plan = LogicalPlanBuilder::from(table_scan) - .filter(and(col("a").eq(lit(10i64)), col("b").gt(lit(11i64))))? - .project(vec![col("a"), col("b")])? - .build()?; + let plan = table_scan_with_pushdown_provider_builder( + TableProviderFilterPushDown::Exact, + vec![], + Some(vec![0]), + )? + .filter(and(col("a").eq(lit(10i64)), col("b").gt(lit(11i64))))? + .project(vec![col("a"), col("b")])? + .build()?; let expected = r#" Projection: a, b @@ -3301,7 +3301,11 @@ Projection: a, b Ok(DataType::Int32) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { Ok(ColumnarValue::Scalar(ScalarValue::from(1))) } } @@ -3385,4 +3389,87 @@ Projection: a, b \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } + + #[test] + fn test_push_down_volatile_table_scan() -> Result<()> { + // SELECT test.a, test.b FROM test as t WHERE TestScalarUDF() > 0.1; + let table_scan = test_table_scan()?; + let fun = ScalarUDF::new_from_impl(TestScalarUDF { + signature: Signature::exact(vec![], Volatility::Volatile), + }); + let expr = Expr::ScalarFunction(ScalarFunction::new_udf(Arc::new(fun), vec![])); + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a"), col("b")])? + .filter(expr.gt(lit(0.1)))? + .build()?; + + let expected_before = "Filter: TestScalarUDF() > Float64(0.1)\ + \n Projection: test.a, test.b\ + \n TableScan: test"; + assert_eq!(format!("{plan}"), expected_before); + + let expected_after = "Projection: test.a, test.b\ + \n Filter: TestScalarUDF() > Float64(0.1)\ + \n TableScan: test"; + assert_optimized_plan_eq(plan, expected_after) + } + + #[test] + fn test_push_down_volatile_mixed_table_scan() -> Result<()> { + // SELECT test.a, test.b FROM test as t WHERE TestScalarUDF() > 0.1 and test.a > 5 and test.b > 10; + let table_scan = test_table_scan()?; + let fun = ScalarUDF::new_from_impl(TestScalarUDF { + signature: Signature::exact(vec![], Volatility::Volatile), + }); + let expr = Expr::ScalarFunction(ScalarFunction::new_udf(Arc::new(fun), vec![])); + let plan = LogicalPlanBuilder::from(table_scan) + .project(vec![col("a"), col("b")])? + .filter( + expr.gt(lit(0.1)) + .and(col("t.a").gt(lit(5))) + .and(col("t.b").gt(lit(10))), + )? + .build()?; + + let expected_before = "Filter: TestScalarUDF() > Float64(0.1) AND t.a > Int32(5) AND t.b > Int32(10)\ + \n Projection: test.a, test.b\ + \n TableScan: test"; + assert_eq!(format!("{plan}"), expected_before); + + let expected_after = "Projection: test.a, test.b\ + \n Filter: TestScalarUDF() > Float64(0.1)\ + \n TableScan: test, full_filters=[t.a > Int32(5), t.b > Int32(10)]"; + assert_optimized_plan_eq(plan, expected_after) + } + + #[test] + fn test_push_down_volatile_mixed_unsupported_table_scan() -> Result<()> { + // SELECT test.a, test.b FROM test as t WHERE TestScalarUDF() > 0.1 and test.a > 5 and test.b > 10; + let fun = ScalarUDF::new_from_impl(TestScalarUDF { + signature: Signature::exact(vec![], Volatility::Volatile), + }); + let expr = Expr::ScalarFunction(ScalarFunction::new_udf(Arc::new(fun), vec![])); + let plan = table_scan_with_pushdown_provider_builder( + TableProviderFilterPushDown::Unsupported, + vec![], + None, + )? + .project(vec![col("a"), col("b")])? + .filter( + expr.gt(lit(0.1)) + .and(col("t.a").gt(lit(5))) + .and(col("t.b").gt(lit(10))), + )? + .build()?; + + let expected_before = "Filter: TestScalarUDF() > Float64(0.1) AND t.a > Int32(5) AND t.b > Int32(10)\ + \n Projection: a, b\ + \n TableScan: test"; + assert_eq!(format!("{plan}"), expected_before); + + let expected_after = "Projection: a, b\ + \n Filter: t.a > Int32(5) AND t.b > Int32(10) AND TestScalarUDF() > Float64(0.1)\ + \n TableScan: test"; + assert_optimized_plan_eq(plan, expected_after) + } } diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index f3e1673e7211..48b2828faf45 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -54,8 +54,6 @@ use datafusion_expr::{Aggregate, Distinct, DistinctOn, Expr, LogicalPlan}; /// ) /// ORDER BY a DESC /// ``` - -/// Optimizer that replaces logical [[Distinct]] with a logical [[Aggregate]] #[derive(Default, Debug)] pub struct ReplaceDistinctWithAggregate {} diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index 6564e722eaf8..60de3f6b3673 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -489,7 +489,7 @@ enum ConstSimplifyResult { SimplifyRuntimeError(DataFusionError, Expr), } -impl<'a> TreeNodeRewriter for ConstEvaluator<'a> { +impl TreeNodeRewriter for ConstEvaluator<'_> { type Node = Expr; fn f_down(&mut self, expr: Expr) -> Result> { @@ -710,7 +710,7 @@ impl<'a, S> Simplifier<'a, S> { } } -impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { +impl TreeNodeRewriter for Simplifier<'_, S> { type Node = Expr; /// rewrite the expression simplifying any constant expressions diff --git a/datafusion/optimizer/src/simplify_expressions/guarantees.rs b/datafusion/optimizer/src/simplify_expressions/guarantees.rs index afcbe528083b..4700ab97b5f3 100644 --- a/datafusion/optimizer/src/simplify_expressions/guarantees.rs +++ b/datafusion/optimizer/src/simplify_expressions/guarantees.rs @@ -57,7 +57,7 @@ impl<'a> GuaranteeRewriter<'a> { } } -impl<'a> TreeNodeRewriter for GuaranteeRewriter<'a> { +impl TreeNodeRewriter for GuaranteeRewriter<'_> { type Node = Expr; fn f_up(&mut self, expr: Expr) -> Result> { diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index ad27c9d49cf7..14d6ca64d15e 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -42,4 +42,3 @@ datafusion-common = { workspace = true, default-features = true } datafusion-expr-common = { workspace = true } hashbrown = { workspace = true } itertools = { workspace = true } -rand = { workspace = true } diff --git a/datafusion/physical-expr-common/LICENSE.txt b/datafusion/physical-expr-common/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/physical-expr-common/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/physical-expr-common/NOTICE.txt b/datafusion/physical-expr-common/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/physical-expr-common/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/physical-expr-common/src/binary_map.rs b/datafusion/physical-expr-common/src/binary_map.rs index 59280a3abbdb..8febbdd5b1f9 100644 --- a/datafusion/physical-expr-common/src/binary_map.rs +++ b/datafusion/physical-expr-common/src/binary_map.rs @@ -28,7 +28,7 @@ use arrow::array::{ use arrow::buffer::{NullBuffer, OffsetBuffer, ScalarBuffer}; use arrow::datatypes::DataType; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::utils::proxy::{RawTableAllocExt, VecAllocExt}; +use datafusion_common::utils::proxy::{HashTableAllocExt, VecAllocExt}; use std::any::type_name; use std::fmt::Debug; use std::mem::{size_of, swap}; @@ -215,7 +215,7 @@ where /// Should the output be String or Binary? output_type: OutputType, /// Underlying hash set for each distinct value - map: hashbrown::raw::RawTable>, + map: hashbrown::hash_table::HashTable>, /// Total size of the map in bytes map_size: usize, /// In progress arrow `Buffer` containing all values @@ -246,7 +246,7 @@ where pub fn new(output_type: OutputType) -> Self { Self { output_type, - map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), + map: hashbrown::hash_table::HashTable::with_capacity(INITIAL_MAP_CAPACITY), map_size: 0, buffer: BufferBuilder::new(INITIAL_BUFFER_CAPACITY), offsets: vec![O::default()], // first offset is always 0 @@ -387,7 +387,7 @@ where let inline = value.iter().fold(0usize, |acc, &x| acc << 8 | x as usize); // is value is already present in the set? - let entry = self.map.get_mut(hash, |header| { + let entry = self.map.find_mut(hash, |header| { // compare value if hashes match if header.len != value_len { return false; @@ -425,7 +425,7 @@ where // value is not "small" else { // Check if the value is already present in the set - let entry = self.map.get_mut(hash, |header| { + let entry = self.map.find_mut(hash, |header| { // compare value if hashes match if header.len != value_len { return false; diff --git a/datafusion/physical-expr-common/src/binary_view_map.rs b/datafusion/physical-expr-common/src/binary_view_map.rs index 8af35510dd6c..7ce943030a45 100644 --- a/datafusion/physical-expr-common/src/binary_view_map.rs +++ b/datafusion/physical-expr-common/src/binary_view_map.rs @@ -24,7 +24,7 @@ use arrow::array::cast::AsArray; use arrow::array::{Array, ArrayBuilder, ArrayRef, GenericByteViewBuilder}; use arrow::datatypes::{BinaryViewType, ByteViewType, DataType, StringViewType}; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::utils::proxy::{RawTableAllocExt, VecAllocExt}; +use datafusion_common::utils::proxy::{HashTableAllocExt, VecAllocExt}; use std::fmt::Debug; use std::sync::Arc; @@ -114,7 +114,6 @@ impl ArrowBytesViewSet { /// This map is used by the special `COUNT DISTINCT` aggregate function to /// store the distinct values, and by the `GROUP BY` operator to store /// group values when they are a single string array. - pub struct ArrowBytesViewMap where V: Debug + PartialEq + Eq + Clone + Copy + Default, @@ -122,7 +121,7 @@ where /// Should the output be StringView or BinaryView? output_type: OutputType, /// Underlying hash set for each distinct value - map: hashbrown::raw::RawTable>, + map: hashbrown::hash_table::HashTable>, /// Total size of the map in bytes map_size: usize, @@ -148,7 +147,7 @@ where pub fn new(output_type: OutputType) -> Self { Self { output_type, - map: hashbrown::raw::RawTable::with_capacity(INITIAL_MAP_CAPACITY), + map: hashbrown::hash_table::HashTable::with_capacity(INITIAL_MAP_CAPACITY), map_size: 0, builder: GenericByteViewBuilder::new(), random_state: RandomState::new(), @@ -274,7 +273,7 @@ where // get the value as bytes let value: &[u8] = value.as_ref(); - let entry = self.map.get_mut(hash, |header| { + let entry = self.map.find_mut(hash, |header| { let v = self.builder.get_value(header.view_idx); if v.len() != value.len() { diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index aa816cfa4469..8ab7030dd8a1 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -219,7 +219,7 @@ pub fn with_new_children_if_necessary( /// Example output: `[a + 1, b]` pub fn format_physical_expr_list(exprs: &[Arc]) -> impl Display + '_ { struct DisplayWrapper<'a>(&'a [Arc]); - impl<'a> Display for DisplayWrapper<'a> { + impl Display for DisplayWrapper<'_> { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { let mut iter = self.0.iter(); write!(f, "[")?; diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 9ae12fa9f608..bf06296ba3e2 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -251,7 +251,7 @@ pub fn format_physical_sort_requirement_list( exprs: &[PhysicalSortRequirement], ) -> impl Display + '_ { struct DisplayWrapper<'a>(&'a [PhysicalSortRequirement]); - impl<'a> Display for DisplayWrapper<'a> { + impl Display for DisplayWrapper<'_> { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { let mut iter = self.0.iter(); write!(f, "[")?; diff --git a/datafusion/physical-expr/Cargo.toml b/datafusion/physical-expr/Cargo.toml index 4195e684381f..db3e0e10d816 100644 --- a/datafusion/physical-expr/Cargo.toml +++ b/datafusion/physical-expr/Cargo.toml @@ -40,10 +40,7 @@ ahash = { workspace = true } arrow = { workspace = true } arrow-array = { workspace = true } arrow-buffer = { workspace = true } -arrow-ord = { workspace = true } arrow-schema = { workspace = true } -arrow-string = { workspace = true } -chrono = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true } @@ -62,7 +59,6 @@ arrow = { workspace = true, features = ["test_utils"] } criterion = "0.5" rand = { workspace = true } rstest = { workspace = true } -tokio = { workspace = true, features = ["rt-multi-thread"] } [[bench]] harness = false diff --git a/datafusion/physical-expr/LICENSE.txt b/datafusion/physical-expr/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/physical-expr/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/physical-expr/NOTICE.txt b/datafusion/physical-expr/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/physical-expr/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 5dc138933430..4eaabace7257 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -28,6 +28,7 @@ pub(crate) mod stats { pub use datafusion_functions_aggregate_common::stats::StatsType; } pub mod utils { + #[allow(deprecated)] // allow adjust_output_array pub use datafusion_functions_aggregate_common::utils::{ adjust_output_array, get_accum_scalar_values_as_arrays, get_sort_options, ordering_fields, DecimalAverager, Hashable, diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index e4185ad44d65..9e00b756b42a 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -125,7 +125,7 @@ impl ConstExpr { /// Returns a [`Display`]able list of `ConstExpr`. pub fn format_list(input: &[ConstExpr]) -> impl Display + '_ { struct DisplayableList<'a>(&'a [ConstExpr]); - impl<'a> Display for DisplayableList<'a> { + impl Display for DisplayableList<'_> { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { let mut first = true; for const_expr in self.0 { diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 06f1e24ed202..fe866450b2b2 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -883,9 +883,11 @@ impl EquivalenceProperties { if self.is_expr_constant(source) && !const_exprs_contains(&projected_constants, target) { + let across_partitions = self.is_expr_constant_accross_partitions(source); // Expression evaluates to single value - projected_constants - .push(ConstExpr::from(target).with_across_partitions(true)); + projected_constants.push( + ConstExpr::from(target).with_across_partitions(across_partitions), + ); } } projected_constants @@ -1014,6 +1016,37 @@ impl EquivalenceProperties { is_constant_recurse(&normalized_constants, &normalized_expr) } + /// This function determines whether the provided expression is constant + /// across partitions based on the known constants. + /// + /// # Arguments + /// + /// - `expr`: A reference to a `Arc` representing the + /// expression to be checked. + /// + /// # Returns + /// + /// Returns `true` if the expression is constant across all partitions according + /// to equivalence group, `false` otherwise. + pub fn is_expr_constant_accross_partitions( + &self, + expr: &Arc, + ) -> bool { + // As an example, assume that we know columns `a` and `b` are constant. + // Then, `a`, `b` and `a + b` will all return `true` whereas `c` will + // return `false`. + let const_exprs = self.constants.iter().flat_map(|const_expr| { + if const_expr.across_partitions() { + Some(Arc::clone(const_expr.expr())) + } else { + None + } + }); + let normalized_constants = self.eq_group.normalize_exprs(const_exprs); + let normalized_expr = self.eq_group.normalize_expr(Arc::clone(expr)); + is_constant_recurse(&normalized_constants, &normalized_expr) + } + /// Retrieves the properties for a given physical expression. /// /// This function constructs an [`ExprProperties`] object for the given diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 663045fcad3f..29577740aab4 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -244,7 +244,7 @@ trait IsEqual: HashValue { fn is_equal(&self, other: &Self) -> bool; } -impl<'a, T: IsEqual + ?Sized> IsEqual for &'a T { +impl IsEqual for &T { fn is_equal(&self, other: &Self) -> bool { T::is_equal(self, other) } diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index bffc2c46fc1e..add6c18b329c 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -200,8 +200,11 @@ pub fn create_physical_expr( escape_char, case_insensitive, }) => { - if escape_char.is_some() { - return exec_err!("LIKE does not support escape_char"); + // `\` is the implicit escape, see https://github.com/apache/datafusion/issues/13291 + if escape_char.unwrap_or('\\') != '\\' { + return exec_err!( + "LIKE does not support escape_char other than the backslash (\\)" + ); } let physical_expr = create_physical_expr(expr, input_dfschema, execution_props)?; diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index 9bf168e8a199..138774d806f2 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -43,7 +43,7 @@ use datafusion_common::{internal_err, DFSchema, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::ExprProperties; use datafusion_expr::type_coercion::functions::data_types_with_scalar_udf; -use datafusion_expr::{expr_vec_fmt, ColumnarValue, Expr, ScalarUDF}; +use datafusion_expr::{expr_vec_fmt, ColumnarValue, Expr, ScalarFunctionArgs, ScalarUDF}; /// Physical expression of a scalar function #[derive(Eq, PartialEq, Hash)] @@ -140,18 +140,24 @@ impl PhysicalExpr for ScalarFunctionExpr { .map(|e| e.evaluate(batch)) .collect::>>()?; + let input_empty = inputs.is_empty(); + let input_all_scalar = inputs + .iter() + .all(|arg| matches!(arg, ColumnarValue::Scalar(_))); + // evaluate the function - let output = self.fun.invoke_batch(&inputs, batch.num_rows())?; + let output = self.fun.invoke_with_args(ScalarFunctionArgs { + args: inputs.as_slice(), + number_rows: batch.num_rows(), + return_type: &self.return_type, + })?; if let ColumnarValue::Array(array) = &output { if array.len() != batch.num_rows() { // If the arguments are a non-empty slice of scalar values, we can assume that // returning a one-element array is equivalent to returning a scalar. - let preserve_scalar = array.len() == 1 - && !inputs.is_empty() - && inputs - .iter() - .all(|arg| matches!(arg, ColumnarValue::Scalar(_))); + let preserve_scalar = + array.len() == 1 && !input_empty && input_all_scalar; return if preserve_scalar { ScalarValue::try_from_array(array, 0).map(ColumnarValue::Scalar) } else { diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index 2c37c4d8b394..75ad5d22afe6 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -124,7 +124,7 @@ impl LiteralGuarantee { // for an `AND` conjunction to be true, all terms individually must be true .fold(GuaranteeBuilder::new(), |builder, expr| { if let Some(cel) = ColOpLit::try_new(expr) { - return builder.aggregate_conjunct(cel); + builder.aggregate_conjunct(cel) } else if let Some(inlist) = expr .as_any() .downcast_ref::() diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 1abb11137a52..c06efd554098 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -146,9 +146,7 @@ struct PhysicalExprDAEGBuilder<'a, T, F: Fn(&ExprTreeNode) -> Result< constructor: &'a F, } -impl<'a, T, F: Fn(&ExprTreeNode) -> Result> - PhysicalExprDAEGBuilder<'a, T, F> -{ +impl) -> Result> PhysicalExprDAEGBuilder<'_, T, F> { // This method mutates an expression node by transforming it to a physical expression // and adding it to the graph. The method returns the mutated expression node. fn mutate( @@ -311,7 +309,11 @@ pub(crate) mod tests { Ok(input[0].sort_properties) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { let args = ColumnarValue::values_to_arrays(args)?; let arr: ArrayRef = match args[0].data_type() { diff --git a/datafusion/physical-expr/src/window/mod.rs b/datafusion/physical-expr/src/window/mod.rs index e7a318b860fd..bc7c716783bd 100644 --- a/datafusion/physical-expr/src/window/mod.rs +++ b/datafusion/physical-expr/src/window/mod.rs @@ -16,15 +16,21 @@ // under the License. mod aggregate; -mod built_in; -mod built_in_window_function_expr; mod sliding_aggregate; +mod standard; +mod standard_window_function_expr; mod window_expr; +#[deprecated(since = "44.0.0", note = "use StandardWindowExpr")] +pub type BuiltInWindowExpr = StandardWindowExpr; + +#[deprecated(since = "44.0.0", note = "use StandardWindowFunctionExpr")] +pub type BuiltInWindowFunctionExpr = dyn StandardWindowFunctionExpr; + pub use aggregate::PlainAggregateWindowExpr; -pub use built_in::BuiltInWindowExpr; -pub use built_in_window_function_expr::BuiltInWindowFunctionExpr; pub use sliding_aggregate::SlidingAggregateWindowExpr; +pub use standard::StandardWindowExpr; +pub use standard_window_function_expr::StandardWindowFunctionExpr; pub use window_expr::PartitionBatches; pub use window_expr::PartitionKey; pub use window_expr::PartitionWindowAggStates; diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/standard.rs similarity index 93% rename from datafusion/physical-expr/src/window/built_in.rs rename to datafusion/physical-expr/src/window/standard.rs index 0f6c3f921892..82e48a5f9338 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -15,13 +15,13 @@ // specific language governing permissions and limitations // under the License. -//! Physical exec for built-in window function expressions. +//! Physical exec for standard window function expressions. use std::any::Any; use std::ops::Range; use std::sync::Arc; -use super::{BuiltInWindowFunctionExpr, WindowExpr}; +use super::{StandardWindowFunctionExpr, WindowExpr}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; @@ -35,19 +35,19 @@ use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; use datafusion_physical_expr_common::sort_expr::LexOrdering; -/// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. +/// A window expr that takes the form of a [`StandardWindowFunctionExpr`]. #[derive(Debug)] -pub struct BuiltInWindowExpr { - expr: Arc, +pub struct StandardWindowExpr { + expr: Arc, partition_by: Vec>, order_by: LexOrdering, window_frame: Arc, } -impl BuiltInWindowExpr { - /// create a new built-in window function expression +impl StandardWindowExpr { + /// create a new standard window function expression pub fn new( - expr: Arc, + expr: Arc, partition_by: &[Arc], order_by: &LexOrdering, window_frame: Arc, @@ -60,8 +60,8 @@ impl BuiltInWindowExpr { } } - /// Get BuiltInWindowFunction expr of BuiltInWindowExpr - pub fn get_built_in_func_expr(&self) -> &Arc { + /// Get StandardWindowFunction expr of StandardWindowExpr + pub fn get_standard_func_expr(&self) -> &Arc { &self.expr } @@ -79,7 +79,7 @@ impl BuiltInWindowExpr { eq_properties .add_new_orderings([LexOrdering::new(vec![fn_res_ordering])]); } else { - // If we have a PARTITION BY, built-in functions can not introduce + // If we have a PARTITION BY, standard functions can not introduce // a global ordering unless the existing ordering is compatible // with PARTITION BY expressions. To elaborate, when PARTITION BY // expressions and existing ordering expressions are equal (w.r.t. @@ -96,7 +96,7 @@ impl BuiltInWindowExpr { } } -impl WindowExpr for BuiltInWindowExpr { +impl WindowExpr for StandardWindowExpr { /// Return a reference to Any that can be used for downcasting fn as_any(&self) -> &dyn Any { self @@ -264,7 +264,7 @@ impl WindowExpr for BuiltInWindowExpr { fn get_reverse_expr(&self) -> Option> { self.expr.reverse_expr().map(|reverse_expr| { - Arc::new(BuiltInWindowExpr::new( + Arc::new(StandardWindowExpr::new( reverse_expr, &self.partition_by.clone(), reverse_order_bys(self.order_by.as_ref()).as_ref(), diff --git a/datafusion/physical-expr/src/window/built_in_window_function_expr.rs b/datafusion/physical-expr/src/window/standard_window_function_expr.rs similarity index 92% rename from datafusion/physical-expr/src/window/built_in_window_function_expr.rs rename to datafusion/physical-expr/src/window/standard_window_function_expr.rs index 7aa4f6536a6e..d308812a0e35 100644 --- a/datafusion/physical-expr/src/window/built_in_window_function_expr.rs +++ b/datafusion/physical-expr/src/window/standard_window_function_expr.rs @@ -36,7 +36,7 @@ use std::sync::Arc; /// but others such as `first_value`, `last_value`, and /// `nth_value` need the value. #[allow(rustdoc::private_intra_doc_links)] -pub trait BuiltInWindowFunctionExpr: Send + Sync + std::fmt::Debug { +pub trait StandardWindowFunctionExpr: Send + Sync + std::fmt::Debug { /// Returns the aggregate expression as [`Any`] so that it can be /// downcast to a specific implementation. fn as_any(&self) -> &dyn Any; @@ -50,7 +50,7 @@ pub trait BuiltInWindowFunctionExpr: Send + Sync + std::fmt::Debug { /// Human readable name such as `"MIN(c2)"` or `"RANK()"`. The default /// implementation returns placeholder text. fn name(&self) -> &str { - "BuiltInWindowFunctionExpr: default name" + "StandardWindowFunctionExpr: default name" } /// Evaluate window function's arguments against the input window @@ -71,7 +71,7 @@ pub trait BuiltInWindowFunctionExpr: Send + Sync + std::fmt::Debug { /// a particular partition. fn create_evaluator(&self) -> Result>; - /// Construct a new [`BuiltInWindowFunctionExpr`] that produces + /// Construct a new [`StandardWindowFunctionExpr`] that produces /// the same result as this function on a window with reverse /// order. The return value of this function is used by the /// DataFusion optimizer to avoid re-sorting the data when @@ -80,7 +80,7 @@ pub trait BuiltInWindowFunctionExpr: Send + Sync + std::fmt::Debug { /// Returns `None` (the default) if no reverse is known (or possible). /// /// For example, the reverse of `lead(10)` is `lag(10)`. - fn reverse_expr(&self) -> Option> { + fn reverse_expr(&self) -> Option> { None } diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index 718567de8df4..838617ae9889 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -33,11 +33,9 @@ workspace = true [dependencies] arrow = { workspace = true } -arrow-schema = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr-common = { workspace = true, default-features = true } -datafusion-optimizer = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-plan = { workspace = true } itertools = { workspace = true } diff --git a/datafusion/physical-optimizer/LICENSE.txt b/datafusion/physical-optimizer/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/physical-optimizer/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/physical-optimizer/NOTICE.txt b/datafusion/physical-optimizer/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/physical-optimizer/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 64fd0f49a233..bb0e21fdfd15 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -51,7 +51,6 @@ datafusion-common = { workspace = true, default-features = true } datafusion-common-runtime = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } -datafusion-functions-aggregate-common = { workspace = true } datafusion-functions-window-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-physical-expr-common = { workspace = true } diff --git a/datafusion/physical-plan/LICENSE.txt b/datafusion/physical-plan/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/physical-plan/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/physical-plan/NOTICE.txt b/datafusion/physical-plan/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/physical-plan/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index ae528daad53c..58bc7bb90a88 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -164,7 +164,7 @@ pub(crate) fn new_group_values( TimeUnit::Nanosecond => downcast_helper!(Time64NanosecondType, d), _ => {} }, - DataType::Timestamp(t, _) => match t { + DataType::Timestamp(t, _tz) => match t { TimeUnit::Second => downcast_helper!(TimestampSecondType, d), TimeUnit::Millisecond => downcast_helper!(TimestampMillisecondType, d), TimeUnit::Microsecond => downcast_helper!(TimestampMicrosecondType, d), diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs index 10b00cf74fdb..89041eb0f04e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs @@ -880,12 +880,12 @@ impl GroupValuesColumn { /// `$t`: the primitive type of the builder /// macro_rules! instantiate_primitive { - ($v:expr, $nullable:expr, $t:ty) => { + ($v:expr, $nullable:expr, $t:ty, $data_type:ident) => { if $nullable { - let b = PrimitiveGroupValueBuilder::<$t, true>::new(); + let b = PrimitiveGroupValueBuilder::<$t, true>::new($data_type.to_owned()); $v.push(Box::new(b) as _) } else { - let b = PrimitiveGroupValueBuilder::<$t, false>::new(); + let b = PrimitiveGroupValueBuilder::<$t, false>::new($data_type.to_owned()); $v.push(Box::new(b) as _) } }; @@ -898,53 +898,114 @@ impl GroupValues for GroupValuesColumn { for f in self.schema.fields().iter() { let nullable = f.is_nullable(); - match f.data_type() { - &DataType::Int8 => instantiate_primitive!(v, nullable, Int8Type), - &DataType::Int16 => instantiate_primitive!(v, nullable, Int16Type), - &DataType::Int32 => instantiate_primitive!(v, nullable, Int32Type), - &DataType::Int64 => instantiate_primitive!(v, nullable, Int64Type), - &DataType::UInt8 => instantiate_primitive!(v, nullable, UInt8Type), - &DataType::UInt16 => instantiate_primitive!(v, nullable, UInt16Type), - &DataType::UInt32 => instantiate_primitive!(v, nullable, UInt32Type), - &DataType::UInt64 => instantiate_primitive!(v, nullable, UInt64Type), + let data_type = f.data_type(); + match data_type { + &DataType::Int8 => { + instantiate_primitive!(v, nullable, Int8Type, data_type) + } + &DataType::Int16 => { + instantiate_primitive!(v, nullable, Int16Type, data_type) + } + &DataType::Int32 => { + instantiate_primitive!(v, nullable, Int32Type, data_type) + } + &DataType::Int64 => { + instantiate_primitive!(v, nullable, Int64Type, data_type) + } + &DataType::UInt8 => { + instantiate_primitive!(v, nullable, UInt8Type, data_type) + } + &DataType::UInt16 => { + instantiate_primitive!(v, nullable, UInt16Type, data_type) + } + &DataType::UInt32 => { + instantiate_primitive!(v, nullable, UInt32Type, data_type) + } + &DataType::UInt64 => { + instantiate_primitive!(v, nullable, UInt64Type, data_type) + } &DataType::Float32 => { - instantiate_primitive!(v, nullable, Float32Type) + instantiate_primitive!(v, nullable, Float32Type, data_type) } &DataType::Float64 => { - instantiate_primitive!(v, nullable, Float64Type) + instantiate_primitive!(v, nullable, Float64Type, data_type) + } + &DataType::Date32 => { + instantiate_primitive!(v, nullable, Date32Type, data_type) + } + &DataType::Date64 => { + instantiate_primitive!(v, nullable, Date64Type, data_type) } - &DataType::Date32 => instantiate_primitive!(v, nullable, Date32Type), - &DataType::Date64 => instantiate_primitive!(v, nullable, Date64Type), &DataType::Time32(t) => match t { TimeUnit::Second => { - instantiate_primitive!(v, nullable, Time32SecondType) + instantiate_primitive!( + v, + nullable, + Time32SecondType, + data_type + ) } TimeUnit::Millisecond => { - instantiate_primitive!(v, nullable, Time32MillisecondType) + instantiate_primitive!( + v, + nullable, + Time32MillisecondType, + data_type + ) } _ => {} }, &DataType::Time64(t) => match t { TimeUnit::Microsecond => { - instantiate_primitive!(v, nullable, Time64MicrosecondType) + instantiate_primitive!( + v, + nullable, + Time64MicrosecondType, + data_type + ) } TimeUnit::Nanosecond => { - instantiate_primitive!(v, nullable, Time64NanosecondType) + instantiate_primitive!( + v, + nullable, + Time64NanosecondType, + data_type + ) } _ => {} }, &DataType::Timestamp(t, _) => match t { TimeUnit::Second => { - instantiate_primitive!(v, nullable, TimestampSecondType) + instantiate_primitive!( + v, + nullable, + TimestampSecondType, + data_type + ) } TimeUnit::Millisecond => { - instantiate_primitive!(v, nullable, TimestampMillisecondType) + instantiate_primitive!( + v, + nullable, + TimestampMillisecondType, + data_type + ) } TimeUnit::Microsecond => { - instantiate_primitive!(v, nullable, TimestampMicrosecondType) + instantiate_primitive!( + v, + nullable, + TimestampMicrosecondType, + data_type + ) } TimeUnit::Nanosecond => { - instantiate_primitive!(v, nullable, TimestampNanosecondType) + instantiate_primitive!( + v, + nullable, + TimestampNanosecondType, + data_type + ) } }, &DataType::Utf8 => { diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs index 4da482247458..4686a78f24b0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/primitive.rs @@ -20,6 +20,7 @@ use crate::aggregates::group_values::null_builder::MaybeNullBufferBuilder; use arrow::buffer::ScalarBuffer; use arrow_array::cast::AsArray; use arrow_array::{Array, ArrayRef, ArrowPrimitiveType, PrimitiveArray}; +use arrow_schema::DataType; use datafusion_execution::memory_pool::proxy::VecAllocExt; use itertools::izip; use std::iter; @@ -35,6 +36,7 @@ use std::sync::Arc; /// `NULLABLE`: if the data can contain any nulls #[derive(Debug)] pub struct PrimitiveGroupValueBuilder { + data_type: DataType, group_values: Vec, nulls: MaybeNullBufferBuilder, } @@ -44,8 +46,9 @@ where T: ArrowPrimitiveType, { /// Create a new `PrimitiveGroupValueBuilder` - pub fn new() -> Self { + pub fn new(data_type: DataType) -> Self { Self { + data_type, group_values: vec![], nulls: MaybeNullBufferBuilder::new(), } @@ -177,6 +180,7 @@ impl GroupColumn fn build(self: Box) -> ArrayRef { let Self { + data_type, group_values, nulls, } = *self; @@ -186,10 +190,9 @@ impl GroupColumn assert!(nulls.is_none(), "unexpected nulls in non nullable input"); } - Arc::new(PrimitiveArray::::new( - ScalarBuffer::from(group_values), - nulls, - )) + let arr = PrimitiveArray::::new(ScalarBuffer::from(group_values), nulls); + // Set timezone information for timestamp + Arc::new(arr.with_data_type(data_type)) } fn take_n(&mut self, n: usize) -> ArrayRef { @@ -212,6 +215,7 @@ mod tests { use arrow::datatypes::Int64Type; use arrow_array::{ArrayRef, Int64Array}; use arrow_buffer::{BooleanBufferBuilder, NullBuffer}; + use arrow_schema::DataType; use super::GroupColumn; @@ -283,7 +287,8 @@ mod tests { // - exist not null, input not null; values equal // Define PrimitiveGroupValueBuilder - let mut builder = PrimitiveGroupValueBuilder::::new(); + let mut builder = + PrimitiveGroupValueBuilder::::new(DataType::Int64); let builder_array = Arc::new(Int64Array::from(vec![ None, None, @@ -392,7 +397,8 @@ mod tests { // - values not equal // Define PrimitiveGroupValueBuilder - let mut builder = PrimitiveGroupValueBuilder::::new(); + let mut builder = + PrimitiveGroupValueBuilder::::new(DataType::Int64); let builder_array = Arc::new(Int64Array::from(vec![Some(0), Some(1)])) as ArrayRef; append(&mut builder, &builder_array, &[0, 1]); @@ -419,7 +425,8 @@ mod tests { // Test the special `all nulls` or `not nulls` input array case // for vectorized append and equal to - let mut builder = PrimitiveGroupValueBuilder::::new(); + let mut builder = + PrimitiveGroupValueBuilder::::new(DataType::Int64); // All nulls input array let all_nulls_input_array = Arc::new(Int64Array::from(vec![ diff --git a/datafusion/physical-plan/src/aggregates/topk/heap.rs b/datafusion/physical-plan/src/aggregates/topk/heap.rs index e694422e443d..ec1277f8fd55 100644 --- a/datafusion/physical-plan/src/aggregates/topk/heap.rs +++ b/datafusion/physical-plan/src/aggregates/topk/heap.rs @@ -20,11 +20,11 @@ use arrow::datatypes::i256; use arrow_array::cast::AsArray; use arrow_array::{downcast_primitive, ArrayRef, ArrowPrimitiveType, PrimitiveArray}; -use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; +use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano, ScalarBuffer}; use arrow_schema::DataType; use datafusion_common::DataFusionError; use datafusion_common::Result; -use datafusion_physical_expr::aggregate::utils::adjust_output_array; + use half::f16; use std::cmp::Ordering; use std::fmt::{Debug, Display, Formatter}; @@ -151,10 +151,11 @@ where } fn drain(&mut self) -> (ArrayRef, Vec) { + let nulls = None; let (vals, map_idxs) = self.heap.drain(); - let vals = Arc::new(PrimitiveArray::::from_iter_values(vals)); - let vals = adjust_output_array(&self.data_type, vals).expect("Type is incorrect"); - (vals, map_idxs) + let arr = PrimitiveArray::::new(ScalarBuffer::from(vals), nulls) + .with_data_type(self.data_type.clone()); + (Arc::new(arr), map_idxs) } } @@ -366,7 +367,7 @@ impl TopKHeap { impl Display for TopKHeap { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { let mut output = String::new(); - if self.heap.first().is_some() { + if !self.heap.is_empty() { self._tree_print(0, String::new(), true, &mut output); } write!(f, "{}", output) diff --git a/datafusion/physical-plan/src/aggregates/topk_stream.rs b/datafusion/physical-plan/src/aggregates/topk_stream.rs index 075d8c5f2883..5d18f40d13bc 100644 --- a/datafusion/physical-plan/src/aggregates/topk_stream.rs +++ b/datafusion/physical-plan/src/aggregates/topk_stream.rs @@ -123,7 +123,7 @@ impl Stream for GroupedTopKAggregateStream { batch.num_rows() ); if log::log_enabled!(Level::Trace) && batch.num_rows() < 20 { - print_batches(&[batch.clone()])?; + print_batches(std::slice::from_ref(&batch))?; } self.row_count += batch.num_rows(); let batches = &[batch]; @@ -165,7 +165,7 @@ impl Stream for GroupedTopKAggregateStream { batch.num_rows() ); if log::log_enabled!(Level::Trace) { - print_batches(&[batch.clone()])?; + print_batches(std::slice::from_ref(&batch))?; } return Poll::Ready(Some(Ok(batch))); } diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index 9f3a76e28577..9c5e95d56e60 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -124,7 +124,7 @@ impl<'a> DisplayableExecutionPlan<'a> { show_statistics: bool, show_schema: bool, } - impl<'a> fmt::Display for Wrapper<'a> { + impl fmt::Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut visitor = IndentVisitor { t: self.format_type, @@ -163,7 +163,7 @@ impl<'a> DisplayableExecutionPlan<'a> { show_metrics: ShowMetrics, show_statistics: bool, } - impl<'a> fmt::Display for Wrapper<'a> { + impl fmt::Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let t = DisplayFormatType::Default; @@ -202,7 +202,7 @@ impl<'a> DisplayableExecutionPlan<'a> { show_schema: bool, } - impl<'a> fmt::Display for Wrapper<'a> { + impl fmt::Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let mut visitor = IndentVisitor { f, @@ -268,7 +268,7 @@ struct IndentVisitor<'a, 'b> { show_schema: bool, } -impl<'a, 'b> ExecutionPlanVisitor for IndentVisitor<'a, 'b> { +impl ExecutionPlanVisitor for IndentVisitor<'_, '_> { type Error = fmt::Error; fn pre_visit(&mut self, plan: &dyn ExecutionPlan) -> Result { write!(self.f, "{:indent$}", "", indent = self.indent * 2)?; @@ -349,7 +349,7 @@ impl ExecutionPlanVisitor for GraphvizVisitor<'_, '_> { struct Wrapper<'a>(&'a dyn ExecutionPlan, DisplayFormatType); - impl<'a> fmt::Display for Wrapper<'a> { + impl fmt::Display for Wrapper<'_> { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { self.0.fmt_as(self.1, f) } @@ -447,7 +447,7 @@ impl fmt::Display for VerboseDisplay { #[derive(Debug)] pub struct ProjectSchemaDisplay<'a>(pub &'a SchemaRef); -impl<'a> fmt::Display for ProjectSchemaDisplay<'a> { +impl fmt::Display for ProjectSchemaDisplay<'_> { fn fmt(&self, f: &mut Formatter) -> fmt::Result { let parts: Vec<_> = self .0 diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index 7220e7594ea6..4b91cfff062b 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -1098,9 +1098,9 @@ mod tests { &vec![0], ); assert!(result.is_err()); - assert_starts_with( - result.err().unwrap().message().as_ref(), - "Invalid batch column at '0' has null but schema specifies non-nullable", + assert_eq!( + result.err().unwrap().strip_backtrace(), + "Execution error: Invalid batch column at '0' has null but schema specifies non-nullable", ); Ok(()) } @@ -1123,9 +1123,9 @@ mod tests { &vec![0], ); assert!(result.is_err()); - assert_starts_with( - result.err().unwrap().message().as_ref(), - "Invalid batch column at '0' has null but schema specifies non-nullable", + assert_eq!( + result.err().unwrap().strip_backtrace(), + "Execution error: Invalid batch column at '0' has null but schema specifies non-nullable", ); Ok(()) } @@ -1147,9 +1147,9 @@ mod tests { &vec![0], ); assert!(result.is_err()); - assert_starts_with( - result.err().unwrap().message().as_ref(), - "Invalid batch column at '0' has null but schema specifies non-nullable", + assert_eq!( + result.err().unwrap().strip_backtrace(), + "Execution error: Invalid batch column at '0' has null but schema specifies non-nullable", ); Ok(()) } @@ -1190,21 +1190,10 @@ mod tests { &vec![0], ); assert!(result.is_err()); - assert_starts_with( - result.err().unwrap().message().as_ref(), - "Invalid batch column at '0' has null but schema specifies non-nullable", + assert_eq!( + result.err().unwrap().strip_backtrace(), + "Execution error: Invalid batch column at '0' has null but schema specifies non-nullable", ); Ok(()) } - - fn assert_starts_with(actual: impl AsRef, expected_prefix: impl AsRef) { - let actual = actual.as_ref(); - let expected_prefix = expected_prefix.as_ref(); - assert!( - actual.starts_with(expected_prefix), - "Expected '{}' to start with '{}'", - actual, - expected_prefix - ); - } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 8ab292c14269..f7bf039ee7b5 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -1560,7 +1560,7 @@ mod tests { use rstest_reuse::*; fn div_ceil(a: usize, b: usize) -> usize { - (a + b - 1) / b + a.div_ceil(b) } #[template] diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 5b1a29665868..43f698c24d05 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -68,8 +68,54 @@ use crate::{ RecordBatchStream, SendableRecordBatchStream, Statistics, }; -/// join execution plan executes partitions in parallel and combines them into a set of -/// partitions. +/// Join execution plan that executes equi-join predicates on multiple partitions using Sort-Merge +/// join algorithm and applies an optional filter post join. Can be used to join arbitrarily large +/// inputs where one or both of the inputs don't fit in the available memory. +/// +/// # Join Expressions +/// +/// Equi-join predicate (e.g. ` = `) expressions are represented by [`Self::on`]. +/// +/// Non-equality predicates, which can not be pushed down to join inputs (e.g. +/// ` != `) are known as "filter expressions" and are evaluated +/// after the equijoin predicates. They are represented by [`Self::filter`]. These are optional +/// expressions. +/// +/// # Sorting +/// +/// Assumes that both the left and right input to the join are pre-sorted. It is not the +/// responisibility of this execution plan to sort the inputs. +/// +/// # "Streamed" vs "Buffered" +/// +/// The number of record batches of streamed input currently present in the memory will depend +/// on the output batch size of the execution plan. There is no spilling support for streamed input. +/// The comparisons are performed from values of join keys in streamed input with the values of +/// join keys in buffered input. One row in streamed record batch could be matched with multiple rows in +/// buffered input batches. The streamed input is managed through the states in `StreamedState` +/// and streamed input batches are represented by `StreamedBatch`. +/// +/// Buffered input is buffered for all record batches having the same value of join key. +/// If the memory limit increases beyond the specified value and spilling is enabled, +/// buffered batches could be spilled to disk. If spilling is disabled, the execution +/// will fail under the same conditions. Multiple record batches of buffered could currently reside +/// in memory/disk during the exectution. The number of buffered batches residing in +/// memory/disk depends on the number of rows of buffered input having the same value +/// of join key as that of streamed input rows currently present in memory. Due to pre-sorted inputs, +/// the algorithm understands when it is not needed anymore, and releases the buffered batches +/// from memory/disk. The buffered input is managed through the states in `BufferedState` +/// and buffered input batches are represented by `BufferedBatch`. +/// +/// Depending on the type of join, left or right input may be selected as streamed or buffered +/// respectively. For example, in a left-outer join, the left execution plan will be selected as +/// streamed input while in a right-outer join, the right execution plan will be selected as the +/// streamed input. +/// +/// Reference for the algorithm: +/// . +/// +/// Helpful short video demonstration: +/// . #[derive(Debug, Clone)] pub struct SortMergeJoinExec { /// Left sorted joining execution plan @@ -529,6 +575,9 @@ struct StreamedJoinedChunk { buffered_indices: UInt64Builder, } +/// Represents a record batch from streamed input. +/// +/// Also stores information of matching rows from buffered batches. struct StreamedBatch { /// The streamed record batch pub batch: RecordBatch, @@ -667,8 +716,8 @@ impl BufferedBatch { } } -/// Sort-merge join stream that consumes streamed and buffered data stream -/// and produces joined output +/// Sort-Merge join stream that consumes streamed and buffered data streams +/// and produces joined output stream. struct SortMergeJoinStream { /// Current state of the stream pub state: SortMergeJoinState, diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 421fd0da808c..37d6c0aff850 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -47,21 +47,23 @@ use rand::prelude::StdRng; use rand::{Rng, SeedableRng}; pub fn compare_batches(collected_1: &[RecordBatch], collected_2: &[RecordBatch]) { + let left_row_num: usize = collected_1.iter().map(|batch| batch.num_rows()).sum(); + let right_row_num: usize = collected_2.iter().map(|batch| batch.num_rows()).sum(); + if left_row_num == 0 && right_row_num == 0 { + return; + } // compare let first_formatted = pretty_format_batches(collected_1).unwrap().to_string(); let second_formatted = pretty_format_batches(collected_2).unwrap().to_string(); - let mut first_formatted_sorted: Vec<&str> = first_formatted.trim().lines().collect(); - first_formatted_sorted.sort_unstable(); + let mut first_lines: Vec<&str> = first_formatted.trim().lines().collect(); + first_lines.sort_unstable(); - let mut second_formatted_sorted: Vec<&str> = - second_formatted.trim().lines().collect(); - second_formatted_sorted.sort_unstable(); + let mut second_lines: Vec<&str> = second_formatted.trim().lines().collect(); + second_lines.sort_unstable(); - for (i, (first_line, second_line)) in first_formatted_sorted - .iter() - .zip(&second_formatted_sorted) - .enumerate() + for (i, (first_line, second_line)) in + first_lines.iter().zip(&second_lines).enumerate() { assert_eq!((i, first_line), (i, second_line)); } diff --git a/datafusion/physical-plan/src/metrics/value.rs b/datafusion/physical-plan/src/metrics/value.rs index 2eb01914ee0a..decf77369db4 100644 --- a/datafusion/physical-plan/src/metrics/value.rs +++ b/datafusion/physical-plan/src/metrics/value.rs @@ -313,7 +313,7 @@ pub struct ScopedTimerGuard<'a> { start: Option, } -impl<'a> ScopedTimerGuard<'a> { +impl ScopedTimerGuard<'_> { /// Stop the timer timing and record the time taken pub fn stop(&mut self) { if let Some(start) = self.start.take() { @@ -332,7 +332,7 @@ impl<'a> ScopedTimerGuard<'a> { } } -impl<'a> Drop for ScopedTimerGuard<'a> { +impl Drop for ScopedTimerGuard<'_> { fn drop(&mut self) { self.stop() } diff --git a/datafusion/physical-plan/src/repartition/distributor_channels.rs b/datafusion/physical-plan/src/repartition/distributor_channels.rs index 2e5ef24beac3..8d09c664fbb2 100644 --- a/datafusion/physical-plan/src/repartition/distributor_channels.rs +++ b/datafusion/physical-plan/src/repartition/distributor_channels.rs @@ -203,7 +203,7 @@ pub struct SendFuture<'a, T> { element: Box>, } -impl<'a, T> Future for SendFuture<'a, T> { +impl Future for SendFuture<'_, T> { type Output = Result<(), SendError>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -295,7 +295,7 @@ pub struct RecvFuture<'a, T> { rdy: bool, } -impl<'a, T> Future for RecvFuture<'a, T> { +impl Future for RecvFuture<'_, T> { type Output = Option; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 9a89db9a5893..906164f21b8c 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -746,7 +746,7 @@ mod tests { // Split the provided record batch into multiple batch_size record batches fn split_batch(sorted: &RecordBatch, batch_size: usize) -> Vec { - let batches = (sorted.num_rows() + batch_size - 1) / batch_size; + let batches = sorted.num_rows().div_ceil(batch_size); // Split the sorted RecordBatch into multiple (0..batches) diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 4350235ef47d..2178cc012a10 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -62,7 +62,7 @@ pub struct StreamingMergeBuilder<'a> { enable_round_robin_tie_breaker: bool, } -impl<'a> Default for StreamingMergeBuilder<'a> { +impl Default for StreamingMergeBuilder<'_> { fn default() -> Self { Self { streams: vec![], diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index ec4c9dd502a6..b3054299b7f7 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -48,7 +48,6 @@ use tokio::task::JoinSet; /// 3. Automatically cancels any outstanding tasks when the receiver stream is dropped. /// /// [`ReceiverStream` from tokio-stream]: https://docs.rs/tokio-stream/latest/tokio_stream/wrappers/struct.ReceiverStream.html - pub(crate) struct ReceiverStreamBuilder { tx: Sender>, rx: Receiver>, diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index cf1c0e313733..cc0a7cbd9b52 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -714,8 +714,6 @@ pub async fn assert_strong_count_converges_to_zero(refs: Weak) { .unwrap(); } -/// - /// Execution plan that emits streams that panics. /// /// This is useful to test panic handling of certain execution plans. diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 50666f6cc368..398b5eb292d7 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1189,7 +1189,7 @@ mod tests { use crate::common::collect; use crate::memory::MemoryExec; - use datafusion_physical_expr::window::BuiltInWindowExpr; + use datafusion_physical_expr::window::StandardWindowExpr; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1562,7 +1562,7 @@ mod tests { let window_exprs = vec![ // LAST_VALUE(a) - Arc::new(BuiltInWindowExpr::new( + Arc::new(StandardWindowExpr::new( last_value_func, &[], &LexOrdering::default(), @@ -1573,7 +1573,7 @@ mod tests { )), )) as _, // NTH_VALUE(a, -1) - Arc::new(BuiltInWindowExpr::new( + Arc::new(StandardWindowExpr::new( nth_value_func1, &[], &LexOrdering::default(), @@ -1584,7 +1584,7 @@ mod tests { )), )) as _, // NTH_VALUE(a, -2) - Arc::new(BuiltInWindowExpr::new( + Arc::new(StandardWindowExpr::new( nth_value_func2, &[], &LexOrdering::default(), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 32173c3ef17d..222a8bb71a02 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -35,7 +35,7 @@ use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctio use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, - window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, + window::{SlidingAggregateWindowExpr, StandardWindowFunctionExpr}, ConstExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; use itertools::Itertools; @@ -50,7 +50,7 @@ use datafusion_functions_window_common::field::WindowUDFFieldArgs; use datafusion_functions_window_common::partition::PartitionEvaluatorArgs; use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::{ - BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, + PlainAggregateWindowExpr, StandardWindowExpr, WindowExpr, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; pub use window_agg_exec::WindowAggExec; @@ -117,7 +117,7 @@ pub fn create_window_expr( aggregate, ) } - WindowFunctionDefinition::WindowUDF(fun) => Arc::new(BuiltInWindowExpr::new( + WindowFunctionDefinition::WindowUDF(fun) => Arc::new(StandardWindowExpr::new( create_udwf_window_expr(fun, args, input_schema, name, ignore_nulls)?, partition_by, order_by, @@ -153,14 +153,14 @@ fn window_expr_from_aggregate_expr( } } -/// Creates a `BuiltInWindowFunctionExpr` suitable for a user defined window function +/// Creates a `StandardWindowFunctionExpr` suitable for a user defined window function pub fn create_udwf_window_expr( fun: &Arc, args: &[Arc], input_schema: &Schema, name: String, ignore_nulls: bool, -) -> Result> { +) -> Result> { // need to get the types into an owned vec for some reason let input_types: Vec<_> = args .iter() @@ -192,7 +192,7 @@ pub fn create_udwf_window_expr( Ok(udwf_expr) } -/// Implements [`BuiltInWindowFunctionExpr`] for [`WindowUDF`] +/// Implements [`StandardWindowFunctionExpr`] for [`WindowUDF`] #[derive(Clone, Debug)] pub struct WindowUDFExpr { fun: Arc, @@ -215,7 +215,7 @@ impl WindowUDFExpr { } } -impl BuiltInWindowFunctionExpr for WindowUDFExpr { +impl StandardWindowFunctionExpr for WindowUDFExpr { fn as_any(&self) -> &dyn std::any::Any { self } @@ -244,7 +244,7 @@ impl BuiltInWindowFunctionExpr for WindowUDFExpr { &self.name } - fn reverse_expr(&self) -> Option> { + fn reverse_expr(&self) -> Option> { match self.fun.reverse_expr() { ReversedUDWF::Identical => Some(Arc::new(self.clone())), ReversedUDWF::NotSupported => None, @@ -345,10 +345,9 @@ pub(crate) fn window_equivalence_properties( .extend(input.equivalence_properties().clone()); for expr in window_expr { - if let Some(builtin_window_expr) = - expr.as_any().downcast_ref::() + if let Some(udf_window_expr) = expr.as_any().downcast_ref::() { - builtin_window_expr.add_equal_orderings(&mut window_eq_properties); + udf_window_expr.add_equal_orderings(&mut window_eq_properties); } } window_eq_properties diff --git a/datafusion/proto-common/LICENSE.txt b/datafusion/proto-common/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/proto-common/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/proto-common/NOTICE.txt b/datafusion/proto-common/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/proto-common/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/proto/LICENSE.txt b/datafusion/proto/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/proto/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/proto/NOTICE.txt b/datafusion/proto/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/proto/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 7d9a524af828..c1bbb3ad26ce 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -19,7 +19,7 @@ use std::sync::Arc; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; -use datafusion::physical_expr::window::{BuiltInWindowExpr, SlidingAggregateWindowExpr}; +use datafusion::physical_expr::window::{SlidingAggregateWindowExpr, StandardWindowExpr}; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, @@ -120,9 +120,9 @@ pub fn serialize_physical_window_expr( window_frame, codec, )? - } else if let Some(built_in_window_expr) = expr.downcast_ref::() { - if let Some(expr) = built_in_window_expr - .get_built_in_func_expr() + } else if let Some(udf_window_expr) = expr.downcast_ref::() { + if let Some(expr) = udf_window_expr + .get_standard_func_expr() .as_any() .downcast_ref::() { diff --git a/datafusion/proto/tests/cases/mod.rs b/datafusion/proto/tests/cases/mod.rs index 4d69ca075483..f36b7178313a 100644 --- a/datafusion/proto/tests/cases/mod.rs +++ b/datafusion/proto/tests/cases/mod.rs @@ -69,9 +69,10 @@ impl ScalarUDFImpl for MyRegexUdf { plan_err!("regex_udf only accepts Utf8 arguments") } } - fn invoke( + fn invoke_batch( &self, _args: &[ColumnarValue], + _number_rows: usize, ) -> datafusion_common::Result { unimplemented!() } diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index efa462aa7a85..3311b57f5d6b 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -54,7 +54,7 @@ use datafusion::functions_window::nth_value::nth_value_udwf; use datafusion::functions_window::row_number::row_number_udwf; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; -use datafusion::physical_expr::window::{BuiltInWindowExpr, SlidingAggregateWindowExpr}; +use datafusion::physical_expr::window::{SlidingAggregateWindowExpr, StandardWindowExpr}; use datafusion::physical_expr::{ LexOrdering, LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, }; @@ -279,7 +279,7 @@ fn roundtrip_udwf() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let udwf_expr = Arc::new(BuiltInWindowExpr::new( + let udwf_expr = Arc::new(StandardWindowExpr::new( create_udwf_window_expr( &row_number_udwf(), &[], @@ -326,7 +326,7 @@ fn roundtrip_window() -> Result<()> { "NTH_VALUE(a, 2) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW".to_string(), false, )?; - let udwf_expr = Arc::new(BuiltInWindowExpr::new( + let udwf_expr = Arc::new(StandardWindowExpr::new( nth_value_window, &[col("b", &schema)?], &LexOrdering { @@ -1125,7 +1125,7 @@ fn roundtrip_udwf_extension_codec() -> Result<()> { WindowFrameBound::CurrentRow, ); - let udwf_expr = Arc::new(BuiltInWindowExpr::new( + let udwf_expr = Arc::new(StandardWindowExpr::new( udwf, &[col("b", &schema)?], &LexOrdering { diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index 94c3ce97a441..01bf92043816 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -51,7 +51,6 @@ log = { workspace = true } recursive = { workspace = true } regex = { workspace = true } sqlparser = { workspace = true } -strum = { version = "0.26.1", features = ["derive"] } [dev-dependencies] ctor = { workspace = true } diff --git a/datafusion/sql/LICENSE.txt b/datafusion/sql/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/sql/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/sql/NOTICE.txt b/datafusion/sql/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/sql/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/sql/src/cte.rs b/datafusion/sql/src/cte.rs index c288d6ca7067..3650aea9c3c2 100644 --- a/datafusion/sql/src/cte.rs +++ b/datafusion/sql/src/cte.rs @@ -28,7 +28,7 @@ use datafusion_common::{ use datafusion_expr::{LogicalPlan, LogicalPlanBuilder, TableSource}; use sqlparser::ast::{Query, SetExpr, SetOperator, With}; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(super) fn plan_with_clause( &self, with: With, diff --git a/datafusion/sql/src/expr/binary_op.rs b/datafusion/sql/src/expr/binary_op.rs index fcb57e8a82e4..eaf28adf4ea8 100644 --- a/datafusion/sql/src/expr/binary_op.rs +++ b/datafusion/sql/src/expr/binary_op.rs @@ -20,7 +20,7 @@ use datafusion_common::{not_impl_err, Result}; use datafusion_expr::Operator; use sqlparser::ast::BinaryOperator; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(crate) fn parse_sql_binary_op(&self, op: BinaryOperator) -> Result { match op { BinaryOperator::Gt => Ok(Operator::Gt), diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index cb7255bb7873..67fa23b86990 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -190,7 +190,7 @@ impl FunctionArgs { } } -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(super) fn sql_function_to_expr( &self, function: SQLFunction, diff --git a/datafusion/sql/src/expr/grouping_set.rs b/datafusion/sql/src/expr/grouping_set.rs index a8b3ef7e20ec..bedbf2a7d347 100644 --- a/datafusion/sql/src/expr/grouping_set.rs +++ b/datafusion/sql/src/expr/grouping_set.rs @@ -21,7 +21,7 @@ use datafusion_common::{DFSchema, Result}; use datafusion_expr::{Expr, GroupingSet}; use sqlparser::ast::Expr as SQLExpr; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(super) fn sql_grouping_sets_to_expr( &self, exprs: Vec>, diff --git a/datafusion/sql/src/expr/identifier.rs b/datafusion/sql/src/expr/identifier.rs index e103f68fc927..9adf14459081 100644 --- a/datafusion/sql/src/expr/identifier.rs +++ b/datafusion/sql/src/expr/identifier.rs @@ -28,7 +28,7 @@ use datafusion_expr::{Case, Expr}; use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_expr::UNNAMED_TABLE; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(super) fn sql_identifier_to_expr( &self, id: Ident, diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 8c8d716a6665..57ac96951f1f 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -49,7 +49,7 @@ mod substring; mod unary_op; mod value; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(crate) fn sql_expr_to_logical_expr( &self, sql: SQLExpr, @@ -140,7 +140,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { let mut expr = self.sql_expr_to_logical_expr(sql, schema, planner_context)?; expr = self.rewrite_partial_qualifier(expr, schema); - self.validate_schema_satisfies_exprs(schema, &[expr.clone()])?; + self.validate_schema_satisfies_exprs(schema, std::slice::from_ref(&expr))?; let (expr, _) = expr.infer_placeholder_types(schema)?; Ok(expr) } diff --git a/datafusion/sql/src/expr/order_by.rs b/datafusion/sql/src/expr/order_by.rs index 00289806876f..b7ed04326f40 100644 --- a/datafusion/sql/src/expr/order_by.rs +++ b/datafusion/sql/src/expr/order_by.rs @@ -23,7 +23,7 @@ use datafusion_expr::expr::Sort; use datafusion_expr::{Expr, SortExpr}; use sqlparser::ast::{Expr as SQLExpr, OrderByExpr, Value}; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { /// Convert sql [OrderByExpr] to `Vec`. /// /// `input_schema` and `additional_schema` are used to resolve column references in the order-by expressions. diff --git a/datafusion/sql/src/expr/subquery.rs b/datafusion/sql/src/expr/subquery.rs index ff161c6ed644..481f024787fe 100644 --- a/datafusion/sql/src/expr/subquery.rs +++ b/datafusion/sql/src/expr/subquery.rs @@ -24,7 +24,7 @@ use sqlparser::ast::Expr as SQLExpr; use sqlparser::ast::Query; use std::sync::Arc; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(super) fn parse_exists_subquery( &self, subquery: Query, diff --git a/datafusion/sql/src/expr/substring.rs b/datafusion/sql/src/expr/substring.rs index f58ab5ff3612..59c78bc713cc 100644 --- a/datafusion/sql/src/expr/substring.rs +++ b/datafusion/sql/src/expr/substring.rs @@ -22,7 +22,7 @@ use datafusion_expr::planner::PlannerResult; use datafusion_expr::Expr; use sqlparser::ast::Expr as SQLExpr; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(super) fn sql_substring_to_expr( &self, expr: Box, diff --git a/datafusion/sql/src/expr/unary_op.rs b/datafusion/sql/src/expr/unary_op.rs index 06988eb03893..a4096ec2355b 100644 --- a/datafusion/sql/src/expr/unary_op.rs +++ b/datafusion/sql/src/expr/unary_op.rs @@ -23,7 +23,7 @@ use datafusion_expr::{ }; use sqlparser::ast::{Expr as SQLExpr, UnaryOperator, Value}; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(crate) fn parse_sql_unary_op( &self, op: UnaryOperator, diff --git a/datafusion/sql/src/expr/value.rs b/datafusion/sql/src/expr/value.rs index 1cf090aa64aa..a651567abe22 100644 --- a/datafusion/sql/src/expr/value.rs +++ b/datafusion/sql/src/expr/value.rs @@ -32,7 +32,7 @@ use sqlparser::ast::{BinaryOperator, Expr as SQLExpr, Interval, UnaryOperator, V use sqlparser::parser::ParserError::ParserError; use std::borrow::Cow; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(crate) fn parse_value( &self, value: Value, diff --git a/datafusion/sql/src/query.rs b/datafusion/sql/src/query.rs index 740f9ad3b42c..2e115d140ea8 100644 --- a/datafusion/sql/src/query.rs +++ b/datafusion/sql/src/query.rs @@ -29,7 +29,7 @@ use sqlparser::ast::{ SetExpr, }; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { /// Generate a logical plan from an SQL query/subquery pub(crate) fn query_to_plan( &self, diff --git a/datafusion/sql/src/relation/join.rs b/datafusion/sql/src/relation/join.rs index 3f34608e3756..2ed1197e8fbf 100644 --- a/datafusion/sql/src/relation/join.rs +++ b/datafusion/sql/src/relation/join.rs @@ -21,7 +21,7 @@ use datafusion_expr::{JoinType, LogicalPlan, LogicalPlanBuilder}; use sqlparser::ast::{Join, JoinConstraint, JoinOperator, TableFactor, TableWithJoins}; use std::collections::HashSet; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(crate) fn plan_table_with_joins( &self, t: TableWithJoins, diff --git a/datafusion/sql/src/relation/mod.rs b/datafusion/sql/src/relation/mod.rs index 256cc58e71dc..45a617daae96 100644 --- a/datafusion/sql/src/relation/mod.rs +++ b/datafusion/sql/src/relation/mod.rs @@ -28,7 +28,7 @@ use sqlparser::ast::{FunctionArg, FunctionArgExpr, TableFactor}; mod join; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { /// Create a `LogicalPlan` that scans the named relation fn create_relation( &self, diff --git a/datafusion/sql/src/select.rs b/datafusion/sql/src/select.rs index 80a08da5e35d..39b6eb6e8132 100644 --- a/datafusion/sql/src/select.rs +++ b/datafusion/sql/src/select.rs @@ -25,7 +25,7 @@ use crate::utils::{ }; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion}; -use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; +use datafusion_common::{not_impl_err, plan_err, Result}; use datafusion_common::{RecursionUnnestOption, UnnestOptions}; use datafusion_expr::expr::{Alias, PlannedReplaceSelectItem, WildcardOptions}; use datafusion_expr::expr_rewriter::{ @@ -45,7 +45,7 @@ use sqlparser::ast::{ }; use sqlparser::ast::{NamedWindowDefinition, Select, SelectItem, TableWithJoins}; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { /// Generate a logic plan from an SQL select pub(super) fn select_to_plan( &self, @@ -167,7 +167,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let group_by_expr = normalize_col(group_by_expr, &projected_plan)?; self.validate_schema_satisfies_exprs( base_plan.schema(), - &[group_by_expr.clone()], + std::slice::from_ref(&group_by_expr), )?; Ok(group_by_expr) }) @@ -657,9 +657,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } = options; if opt_rename.is_some() { - Err(DataFusionError::NotImplemented( - "wildcard * with RENAME not supported ".to_string(), - )) + not_impl_err!("wildcard * with RENAME not supported ") } else { Ok(()) } @@ -815,7 +813,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { check_columns_satisfy_exprs( &column_exprs_post_aggr, - &[having_expr_post_aggr.clone()], + std::slice::from_ref(&having_expr_post_aggr), "HAVING clause references non-aggregate values", )?; diff --git a/datafusion/sql/src/set_expr.rs b/datafusion/sql/src/set_expr.rs index e56ebb4d323f..3b1201d3dd59 100644 --- a/datafusion/sql/src/set_expr.rs +++ b/datafusion/sql/src/set_expr.rs @@ -21,7 +21,7 @@ use datafusion_expr::{LogicalPlan, LogicalPlanBuilder}; use recursive::recursive; use sqlparser::ast::{SetExpr, SetOperator, SetQuantifier}; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { #[recursive] pub(super) fn set_expr_to_plan( &self, diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 31b836f32b24..38695f98b5fe 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -163,7 +163,7 @@ fn calc_inline_constraints_from_columns(columns: &[ColumnDef]) -> Vec SqlToRel<'a, S> { +impl SqlToRel<'_, S> { /// Generate a logical plan from an DataFusion SQL statement pub fn statement_to_plan(&self, statement: DFStatement) -> Result { match statement { diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index fbaa402e703c..57c89f74f5c9 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -467,7 +467,7 @@ impl Default for CustomDialect { impl CustomDialect { // Create a CustomDialect - #[deprecated(note = "please use `CustomDialectBuilder` instead")] + #[deprecated(since = "41.0.0", note = "please use `CustomDialectBuilder` instead")] pub fn new(identifier_quote_style: Option) -> Self { Self { identifier_quote_style, diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index f1f28258f9bd..70246360450d 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -462,7 +462,10 @@ impl Unparser<'_> { match func_name { "make_array" => self.make_array_to_sql(args), "array_element" => self.array_element_to_sql(args), - // TODO: support for the construct and access functions of the `map` and `struct` types + "named_struct" => self.named_struct_to_sql(args), + "get_field" => self.get_field_to_sql(args), + "map" => self.map_to_sql(args), + // TODO: support for the construct and access functions of the `map` type _ => self.scalar_function_to_sql_internal(func_name, args), } } @@ -514,6 +517,90 @@ impl Unparser<'_> { }) } + fn named_struct_to_sql(&self, args: &[Expr]) -> Result { + if args.len() % 2 != 0 { + return internal_err!("named_struct must have an even number of arguments"); + } + + let args = args + .chunks_exact(2) + .map(|chunk| { + let key = match &chunk[0] { + Expr::Literal(ScalarValue::Utf8(Some(s))) => self.new_ident_quoted_if_needs(s.to_string()), + _ => return internal_err!("named_struct expects even arguments to be strings, but received: {:?}", &chunk[0]) + }; + + Ok(ast::DictionaryField { + key, + value: Box::new(self.expr_to_sql(&chunk[1])?), + }) + }) + .collect::>>()?; + + Ok(ast::Expr::Dictionary(args)) + } + + fn get_field_to_sql(&self, args: &[Expr]) -> Result { + if args.len() != 2 { + return internal_err!("get_field must have exactly 2 arguments"); + } + + let mut id = match &args[0] { + Expr::Column(col) => match self.col_to_sql(col)? { + ast::Expr::Identifier(ident) => vec![ident], + ast::Expr::CompoundIdentifier(idents) => idents, + other => return internal_err!("expected col_to_sql to return an Identifier or CompoundIdentifier, but received: {:?}", other), + }, + _ => return internal_err!("get_field expects first argument to be column, but received: {:?}", &args[0]), + }; + + let field = match &args[1] { + Expr::Literal(lit) => self.new_ident_quoted_if_needs(lit.to_string()), + _ => { + return internal_err!( + "get_field expects second argument to be a string, but received: {:?}", + &args[0] + ) + } + }; + id.push(field); + + Ok(ast::Expr::CompoundIdentifier(id)) + } + + fn map_to_sql(&self, args: &[Expr]) -> Result { + if args.len() != 2 { + return internal_err!("map must have exactly 2 arguments"); + } + + let ast::Expr::Array(Array { elem: keys, .. }) = self.expr_to_sql(&args[0])? + else { + return internal_err!( + "map expects first argument to be an array, but received: {:?}", + &args[0] + ); + }; + + let ast::Expr::Array(Array { elem: values, .. }) = self.expr_to_sql(&args[1])? + else { + return internal_err!( + "map expects second argument to be an array, but received: {:?}", + &args[1] + ); + }; + + let entries = keys + .into_iter() + .zip(values) + .map(|(key, value)| ast::MapEntry { + key: Box::new(key), + value: Box::new(value), + }) + .collect(); + + Ok(ast::Expr::Map(ast::Map { entries })) + } + pub fn sort_to_sql(&self, sort: &Sort) -> Result { let Sort { expr, @@ -1524,9 +1611,11 @@ mod tests { Signature, Volatility, WindowFrame, WindowFunctionDefinition, }; use datafusion_expr::{interval_month_day_nano_lit, ExprFunctionExt}; + use datafusion_functions::expr_fn::{get_field, named_struct}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::expr_fn::sum; use datafusion_functions_nested::expr_fn::{array_element, make_array}; + use datafusion_functions_nested::map::map; use datafusion_functions_window::row_number::row_number_udwf; use crate::unparser::dialect::{ @@ -1567,7 +1656,11 @@ mod tests { Ok(DataType::Int32) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { unimplemented!("DummyUDF::invoke") } } @@ -1937,6 +2030,15 @@ mod tests { array_element(make_array(vec![lit(1), lit(2), lit(3)]), lit(1)), "[1, 2, 3][1]", ), + ( + named_struct(vec![lit("a"), lit("1"), lit("b"), lit(2)]), + "{a: '1', b: 2}", + ), + (get_field(col("a.b"), "c"), "a.b.c"), + ( + map(vec![lit("a"), lit("b")], vec![lit(1), lit(2)]), + "MAP {'a': 1, 'b': 2}", + ), ]; for (expr, expected) in tests { diff --git a/datafusion/sql/src/unparser/mod.rs b/datafusion/sql/src/unparser/mod.rs index 83ae64ba238b..2c2530ade7fb 100644 --- a/datafusion/sql/src/unparser/mod.rs +++ b/datafusion/sql/src/unparser/mod.rs @@ -107,7 +107,7 @@ impl<'a> Unparser<'a> { } } -impl<'a> Default for Unparser<'a> { +impl Default for Unparser<'_> { fn default() -> Self { Self { dialect: &DefaultDialect {}, diff --git a/datafusion/sql/src/unparser/utils.rs b/datafusion/sql/src/unparser/utils.rs index d0f80da83d63..c77af7924384 100644 --- a/datafusion/sql/src/unparser/utils.rs +++ b/datafusion/sql/src/unparser/utils.rs @@ -26,6 +26,7 @@ use datafusion_expr::{ expr, utils::grouping_set_to_exprlist, Aggregate, Expr, LogicalPlan, LogicalPlanBuilder, Projection, SortExpr, Unnest, Window, }; +use indexmap::IndexSet; use sqlparser::ast; use super::{ @@ -310,7 +311,7 @@ pub(crate) fn unproject_sort_expr( pub(crate) fn try_transform_to_simple_table_scan_with_filters( plan: &LogicalPlan, ) -> Result)>> { - let mut filters: Vec = vec![]; + let mut filters: IndexSet = IndexSet::new(); let mut plan_stack = vec![plan]; let mut table_alias = None; @@ -321,7 +322,9 @@ pub(crate) fn try_transform_to_simple_table_scan_with_filters( plan_stack.push(alias.input.as_ref()); } LogicalPlan::Filter(filter) => { - filters.push(filter.predicate.clone()); + if !filters.contains(&filter.predicate) { + filters.insert(filter.predicate.clone()); + } plan_stack.push(filter.input.as_ref()); } LogicalPlan::TableScan(table_scan) => { @@ -347,7 +350,11 @@ pub(crate) fn try_transform_to_simple_table_scan_with_filters( }) .collect::, DataFusionError>>()?; - filters.extend(table_scan_filters); + for table_scan_filter in table_scan_filters { + if !filters.contains(&table_scan_filter) { + filters.insert(table_scan_filter); + } + } let mut builder = LogicalPlanBuilder::scan( table_scan.table_name.clone(), @@ -360,6 +367,7 @@ pub(crate) fn try_transform_to_simple_table_scan_with_filters( } let plan = builder.build()?; + let filters = filters.into_iter().collect(); return Ok(Some((plan, filters))); } diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index e479bdbacd83..e29a2299b41c 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -328,7 +328,7 @@ struct RecursiveUnnestRewriter<'a> { columns_unnestings: &'a mut IndexMap>>, transformed_root_exprs: Option>, } -impl<'a> RecursiveUnnestRewriter<'a> { +impl RecursiveUnnestRewriter<'_> { /// This struct stores the history of expr /// during its tree-traversal with a notation of /// \[None,**Unnest(exprA)**,**Unnest(exprB)**,None,None\] @@ -416,7 +416,7 @@ impl<'a> RecursiveUnnestRewriter<'a> { } } -impl<'a> TreeNodeRewriter for RecursiveUnnestRewriter<'a> { +impl TreeNodeRewriter for RecursiveUnnestRewriter<'_> { type Node = Expr; /// This downward traversal needs to keep track of: diff --git a/datafusion/sql/src/values.rs b/datafusion/sql/src/values.rs index a4001bea7dea..dd8957c95470 100644 --- a/datafusion/sql/src/values.rs +++ b/datafusion/sql/src/values.rs @@ -22,7 +22,7 @@ use datafusion_common::{DFSchema, Result}; use datafusion_expr::{LogicalPlan, LogicalPlanBuilder}; use sqlparser::ast::Values as SQLValues; -impl<'a, S: ContextProvider> SqlToRel<'a, S> { +impl SqlToRel<'_, S> { pub(super) fn sql_values_to_plan( &self, values: SQLValues, diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index f9d97cdc74af..8e89323204a3 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -27,6 +27,7 @@ use datafusion_expr::{col, lit, table_scan, wildcard, LogicalPlanBuilder}; use datafusion_functions::unicode; use datafusion_functions_aggregate::grouping::grouping_udaf; use datafusion_functions_nested::make_array::make_array_udf; +use datafusion_functions_nested::map::map_udf; use datafusion_functions_window::rank::rank_udwf; use datafusion_sql::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_sql::unparser::dialect::{ @@ -188,7 +189,10 @@ fn roundtrip_statement() -> Result<()> { "SELECT ARRAY[1, 2, 3][1]", "SELECT [1, 2, 3]", "SELECT [1, 2, 3][1]", - "SELECT left[1] FROM array" + "SELECT left[1] FROM array", + "SELECT {a:1, b:2}", + "SELECT s.a FROM (SELECT {a:1, b:2} AS s)", + "SELECT MAP {'a': 1, 'b': 2}" ]; // For each test sql string, we transform as follows: @@ -204,6 +208,7 @@ fn roundtrip_statement() -> Result<()> { let state = MockSessionState::default() .with_scalar_function(make_array_udf()) .with_scalar_function(array_element_udf()) + .with_scalar_function(map_udf()) .with_aggregate_function(sum_udaf()) .with_aggregate_function(count_udaf()) .with_aggregate_function(max_udaf()) @@ -1157,6 +1162,36 @@ fn test_join_with_table_scan_filters() -> Result<()> { assert_eq!(sql.to_string(), expected_sql); + let right_plan_with_filter_schema = table_scan_with_filters( + Some("right_table"), + &schema_right, + None, + vec![ + col("right_table.age").gt(lit(10)), + col("right_table.age").lt(lit(11)), + ], + )? + .build()?; + let right_plan_with_duplicated_filter = + LogicalPlanBuilder::from(right_plan_with_filter_schema.clone()) + .filter(col("right_table.age").gt(lit(10)))? + .build()?; + + let join_plan_duplicated_filter = LogicalPlanBuilder::from(left_plan) + .join( + right_plan_with_duplicated_filter, + datafusion_expr::JoinType::Inner, + (vec!["left.id"], vec!["right_table.id"]), + Some(col("left.id").gt(lit(5))), + )? + .build()?; + + let sql = plan_to_sql(&join_plan_duplicated_filter)?; + + let expected_sql = r#"SELECT * FROM left_table AS "left" JOIN right_table ON "left".id = right_table.id AND (("left".id > 5) AND (("left"."name" LIKE 'some_name' AND (right_table.age > 10)) AND (right_table.age < 11)))"#; + + assert_eq!(sql.to_string(), expected_sql); + Ok(()) } diff --git a/datafusion/sql/tests/sql_integration.rs b/datafusion/sql/tests/sql_integration.rs index ab7e6c8d0bb7..8f2325fa2d78 100644 --- a/datafusion/sql/tests/sql_integration.rs +++ b/datafusion/sql/tests/sql_integration.rs @@ -2684,7 +2684,11 @@ impl ScalarUDFImpl for DummyUDF { Ok(self.return_type.clone()) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { unimplemented!("DummyUDF::invoke") } } diff --git a/datafusion/sqllogictest/Cargo.toml b/datafusion/sqllogictest/Cargo.toml index ed2b9c49715e..849003f8eeac 100644 --- a/datafusion/sqllogictest/Cargo.toml +++ b/datafusion/sqllogictest/Cargo.toml @@ -70,7 +70,6 @@ postgres = [ [dev-dependencies] env_logger = { workspace = true } -num_cpus = { workspace = true } tokio = { workspace = true, features = ["rt-multi-thread"] } [[test]] diff --git a/datafusion/sqllogictest/bin/sqllogictests.rs b/datafusion/sqllogictest/bin/sqllogictests.rs index c3e739d146c6..12c0e27ea911 100644 --- a/datafusion/sqllogictest/bin/sqllogictests.rs +++ b/datafusion/sqllogictest/bin/sqllogictests.rs @@ -20,6 +20,7 @@ use std::fs; use std::path::{Path, PathBuf}; use clap::Parser; +use datafusion_common::utils::get_available_parallelism; use datafusion_sqllogictest::{DataFusion, TestContext}; use futures::stream::StreamExt; use itertools::Itertools; @@ -112,7 +113,7 @@ async fn run_tests() -> Result<()> { .join() }) // run up to num_cpus streams in parallel - .buffer_unordered(num_cpus::get()) + .buffer_unordered(get_available_parallelism()) .flat_map(|result| { // Filter out any Ok() leaving only the DataFusionErrors futures::stream::iter(match result { diff --git a/datafusion/sqllogictest/src/engines/conversion.rs b/datafusion/sqllogictest/src/engines/conversion.rs index 909539b3131b..8d2fd1e6d0f2 100644 --- a/datafusion/sqllogictest/src/engines/conversion.rs +++ b/datafusion/sqllogictest/src/engines/conversion.rs @@ -101,5 +101,70 @@ pub(crate) fn decimal_to_str(value: Decimal) -> String { } pub(crate) fn big_decimal_to_str(value: BigDecimal) -> String { - value.round(12).normalized().to_string() + // Round the value to limit the number of decimal places + let value = value.round(12).normalized(); + // Format the value to a string + format_big_decimal(value) +} + +fn format_big_decimal(value: BigDecimal) -> String { + let (integer, scale) = value.into_bigint_and_exponent(); + let mut str = integer.to_str_radix(10); + if scale <= 0 { + // Append zeros to the right of the integer part + str.extend(std::iter::repeat('0').take(scale.unsigned_abs() as usize)); + str + } else { + let (sign, unsigned_len, unsigned_str) = if integer.is_negative() { + ("-", str.len() - 1, &str[1..]) + } else { + ("", str.len(), &str[..]) + }; + let scale = scale as usize; + if unsigned_len <= scale { + format!("{}0.{:0>scale$}", sign, unsigned_str) + } else { + str.insert(str.len() - scale, '.'); + str + } + } +} + +#[cfg(test)] +mod tests { + use super::big_decimal_to_str; + use bigdecimal::{num_bigint::BigInt, BigDecimal}; + + macro_rules! assert_decimal_str_eq { + ($integer:expr, $scale:expr, $expected:expr) => { + assert_eq!( + big_decimal_to_str(BigDecimal::from_bigint( + BigInt::from($integer), + $scale + )), + $expected + ); + }; + } + + #[test] + fn test_big_decimal_to_str() { + assert_decimal_str_eq!(11, 3, "0.011"); + assert_decimal_str_eq!(11, 2, "0.11"); + assert_decimal_str_eq!(11, 1, "1.1"); + assert_decimal_str_eq!(11, 0, "11"); + assert_decimal_str_eq!(11, -1, "110"); + assert_decimal_str_eq!(0, 0, "0"); + + // Negative cases + assert_decimal_str_eq!(-11, 3, "-0.011"); + assert_decimal_str_eq!(-11, 2, "-0.11"); + assert_decimal_str_eq!(-11, 1, "-1.1"); + assert_decimal_str_eq!(-11, 0, "-11"); + assert_decimal_str_eq!(-11, -1, "-110"); + + // Round to 12 decimal places + // 1.0000000000011 -> 1.000000000001 + assert_decimal_str_eq!(10_i128.pow(13) + 11, 13, "1.000000000001"); + } } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index e6676d683f91..7578692ddb0f 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -3985,6 +3985,30 @@ ORDER BY column1; 3 [bar] bar NULL [baz] baz +# verify make_array does force to Utf8View +query T +SELECT arrow_typeof(make_array(arrow_cast('a', 'Utf8View'), 'b', 'c', 'd')); +---- +List(Field { name: "item", data_type: Utf8View, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }) + +# expect a,b,c,d. make_array forces all types to be of a common type (see above) +query T +SELECT array_to_string(make_array(arrow_cast('a', 'Utf8View'), 'b', 'c', 'd'), ','); +---- +a,b,c,d + +# array_to_string using largeutf8 for second arg +query TTT +select array_to_string(['h', 'e', 'l', 'l', 'o'], arrow_cast(',', 'LargeUtf8')), array_to_string([1, 2, 3, 4, 5], arrow_cast('-', 'LargeUtf8')), array_to_string([1.0, 2.0, 3.0], arrow_cast('|', 'LargeUtf8')); +---- +h,e,l,l,o 1-2-3-4-5 1|2|3 + +# array_to_string using utf8view for second arg +query TTT +select array_to_string(['h', 'e', 'l', 'l', 'o'], arrow_cast(',', 'Utf8View')), array_to_string([1, 2, 3, 4, 5], arrow_cast('-', 'Utf8View')), array_to_string([1.0, 2.0, 3.0], arrow_cast('|', 'Utf8View')); +---- +h,e,l,l,o 1-2-3-4-5 1|2|3 + statement ok drop table table1; @@ -6928,6 +6952,79 @@ select string_to_array(e, ',') from values; [adipiscing] NULL +# karge string tests for string_to_array + +# string_to_array scalar function +query ? +SELECT string_to_array(arrow_cast('abcxxxdef', 'LargeUtf8'), 'xxx') +---- +[abc, def] + +# string_to_array scalar function +query ? +SELECT string_to_array(arrow_cast('abcxxxdef', 'LargeUtf8'), arrow_cast('xxx', 'LargeUtf8')) +---- +[abc, def] + +query ? +SELECT string_to_array(arrow_cast('abc', 'LargeUtf8'), NULL) +---- +[a, b, c] + +query ? +select string_to_array(arrow_cast(e, 'LargeUtf8'), ',') from values; +---- +[Lorem] +[ipsum] +[dolor] +[sit] +[amet] +[, ] +[consectetur] +[adipiscing] +NULL + +query ? +select string_to_array(arrow_cast(e, 'LargeUtf8'), ',', arrow_cast('Lorem', 'LargeUtf8')) from values; +---- +[] +[ipsum] +[dolor] +[sit] +[amet] +[, ] +[consectetur] +[adipiscing] +NULL + +# string view tests for string_to_array + +# string_to_array scalar function +query ? +SELECT string_to_array(arrow_cast('abcxxxdef', 'Utf8View'), 'xxx') +---- +[abc, def] + +query ? +SELECT string_to_array(arrow_cast('abc', 'Utf8View'), NULL) +---- +[a, b, c] + +query ? +select string_to_array(arrow_cast(e, 'Utf8View'), ',') from values; +---- +[Lorem] +[ipsum] +[dolor] +[sit] +[amet] +[, ] +[consectetur] +[adipiscing] +NULL + +# test string_to_array aliases + query ? select string_to_list(e, 'm') from values; ---- diff --git a/datafusion/sqllogictest/test_files/clickbench.slt b/datafusion/sqllogictest/test_files/clickbench.slt index 733c0a3cd972..dfcd92475857 100644 --- a/datafusion/sqllogictest/test_files/clickbench.slt +++ b/datafusion/sqllogictest/test_files/clickbench.slt @@ -136,7 +136,7 @@ SELECT "UserID", "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", "SearchPh 519640690937130534 (empty) 2 7418527520126366595 (empty) 1 -query IRTI rowsort +query IITI rowsort SELECT "UserID", extract(minute FROM to_timestamp_seconds("EventTime")) AS m, "SearchPhrase", COUNT(*) FROM hits GROUP BY "UserID", m, "SearchPhrase" ORDER BY COUNT(*) DESC LIMIT 10; ---- -2461439046089301801 18 (empty) 1 diff --git a/datafusion/sqllogictest/test_files/decimal.slt b/datafusion/sqllogictest/test_files/decimal.slt index 8db28c32f13b..f082a79c5508 100644 --- a/datafusion/sqllogictest/test_files/decimal.slt +++ b/datafusion/sqllogictest/test_files/decimal.slt @@ -733,3 +733,10 @@ true 2 statement ok drop table decimal256_simple; + + +# https://github.com/apache/datafusion/issues/12870 +query R +SELECT CAST('0' AS decimal(38,0)); +---- +0 diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 31467072dd3e..499d279515c3 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -840,349 +840,340 @@ SELECT EXTRACT("'''year'''" FROM timestamp '2020-09-08T12:00:00+00:00') query error SELECT EXTRACT("'year'" FROM timestamp '2020-09-08T12:00:00+00:00') -query R +query I SELECT date_part('YEAR', CAST('2000-01-01' AS DATE)) ---- 2000 -query R +query I SELECT EXTRACT(year FROM timestamp '2020-09-08T12:00:00+00:00') ---- 2020 -query R +query I SELECT EXTRACT("year" FROM timestamp '2020-09-08T12:00:00+00:00') ---- 2020 -query R +query I SELECT EXTRACT('year' FROM timestamp '2020-09-08T12:00:00+00:00') ---- 2020 -query R +query I SELECT date_part('QUARTER', CAST('2000-01-01' AS DATE)) ---- 1 -query R +query I SELECT EXTRACT(quarter FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 3 -query R +query I SELECT EXTRACT("quarter" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 3 -query R +query I SELECT EXTRACT('quarter' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 3 -query R +query I SELECT date_part('MONTH', CAST('2000-01-01' AS DATE)) ---- 1 -query R +query I SELECT EXTRACT(month FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 9 -query R +query I SELECT EXTRACT("month" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 9 -query R +query I SELECT EXTRACT('month' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 9 -query R +query I SELECT date_part('WEEK', CAST('2003-01-01' AS DATE)) ---- 1 -query R +query I SELECT EXTRACT(WEEK FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 37 -query R +query I SELECT EXTRACT("WEEK" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 37 -query R +query I SELECT EXTRACT('WEEK' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 37 -query R +query I SELECT date_part('DAY', CAST('2000-01-01' AS DATE)) ---- 1 -query R +query I SELECT EXTRACT(day FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 8 -query R +query I SELECT EXTRACT("day" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 8 -query R +query I SELECT EXTRACT('day' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 8 -query R +query I SELECT date_part('DOY', CAST('2000-01-01' AS DATE)) ---- 1 -query R +query I SELECT EXTRACT(doy FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 252 -query R +query I SELECT EXTRACT("doy" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 252 -query R +query I SELECT EXTRACT('doy' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 252 -query R +query I SELECT date_part('DOW', CAST('2000-01-01' AS DATE)) ---- 6 -query R +query I SELECT EXTRACT(dow FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 2 -query R +query I SELECT EXTRACT("dow" FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 2 -query R +query I SELECT EXTRACT('dow' FROM to_timestamp('2020-09-08T12:00:00+00:00')) ---- 2 -query R +query I SELECT date_part('HOUR', CAST('2000-01-01' AS DATE)) ---- 0 -query R +query I SELECT EXTRACT(hour FROM to_timestamp('2020-09-08T12:03:03+00:00')) ---- 12 -query R +query I SELECT EXTRACT("hour" FROM to_timestamp('2020-09-08T12:03:03+00:00')) ---- 12 -query R +query I SELECT EXTRACT('hour' FROM to_timestamp('2020-09-08T12:03:03+00:00')) ---- 12 -query R +query I SELECT EXTRACT(minute FROM to_timestamp('2020-09-08T12:12:00+00:00')) ---- 12 -query R +query I SELECT EXTRACT("minute" FROM to_timestamp('2020-09-08T12:12:00+00:00')) ---- 12 -query R +query I SELECT EXTRACT('minute' FROM to_timestamp('2020-09-08T12:12:00+00:00')) ---- 12 -query R +query I SELECT date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00')) ---- 12 -query R +# make sure the return type is integer +query T +SELECT arrow_typeof(date_part('minute', to_timestamp('2020-09-08T12:12:00+00:00'))) +---- +Int32 + +query I SELECT EXTRACT(second FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12.12345678 +12 -query R +query I SELECT EXTRACT(millisecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123.45678 +12123 -query R +query I SELECT EXTRACT(microsecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123456.78 +12123456 -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT EXTRACT(nanosecond FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456780 -query R +query I SELECT EXTRACT("second" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12.12345678 +12 -query R +query I SELECT EXTRACT("millisecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123.45678 +12123 -query R +query I SELECT EXTRACT("microsecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123456.78 +12123456 -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT EXTRACT("nanosecond" FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456780 -query R +query I SELECT EXTRACT('second' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12.12345678 +12 -query R +query I SELECT EXTRACT('millisecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123.45678 +12123 -query R +query I SELECT EXTRACT('microsecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123456.78 +12123456 -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT EXTRACT('nanosecond' FROM timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456780 + # Keep precision when coercing Utf8 to Timestamp -query R +query I SELECT date_part('second', timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12.12345678 +12 -query R +query I SELECT date_part('millisecond', timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123.45678 +12123 -query R +query I SELECT date_part('microsecond', timestamp '2020-09-08T12:00:12.12345678+00:00') ---- -12123456.78 +12123456 -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT date_part('nanosecond', timestamp '2020-09-08T12:00:12.12345678+00:00') ----- -12123456780 -query R + +query I SELECT date_part('second', '2020-09-08T12:00:12.12345678+00:00') ---- -12.12345678 +12 -query R +query I SELECT date_part('millisecond', '2020-09-08T12:00:12.12345678+00:00') ---- -12123.45678 +12123 -query R +query I SELECT date_part('microsecond', '2020-09-08T12:00:12.12345678+00:00') ---- -12123456.78 +12123456 -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT date_part('nanosecond', '2020-09-08T12:00:12.12345678+00:00') ----- -12123456780 # test_date_part_time ## time32 seconds -query R +query I SELECT date_part('hour', arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 23 -query R +query I SELECT extract(hour from arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 23 -query R +query I SELECT date_part('minute', arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 32 -query R +query I SELECT extract(minute from arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 32 -query R +query I SELECT date_part('second', arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 50 -query R +query I SELECT extract(second from arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 50 -query R +query I SELECT date_part('millisecond', arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 50000 -query R +query I SELECT extract(millisecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 50000 -query R +query I SELECT date_part('microsecond', arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 50000000 -query R +query I SELECT extract(microsecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) ---- 50000000 -query R -SELECT date_part('nanosecond', arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50000000000 - -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT extract(nanosecond from arrow_cast('23:32:50'::time, 'Time32(Second)')) ----- -50000000000 query R SELECT date_part('epoch', arrow_cast('23:32:50'::time, 'Time32(Second)')) @@ -1195,65 +1186,58 @@ SELECT extract(epoch from arrow_cast('23:32:50'::time, 'Time32(Second)')) 84770 ## time32 milliseconds -query R +query I SELECT date_part('hour', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 23 -query R +query I SELECT extract(hour from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 23 -query R +query I SELECT date_part('minute', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 32 -query R +query I SELECT extract(minute from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 32 -query R +query I SELECT date_part('second', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- -50.123 +50 -query R +query I SELECT extract(second from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- -50.123 +50 -query R +query I SELECT date_part('millisecond', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 50123 -query R +query I SELECT extract(millisecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 50123 -query R +query I SELECT date_part('microsecond', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 50123000 -query R +query I SELECT extract(microsecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ---- 50123000 -query R -SELECT date_part('nanosecond', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50123000000 - -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT extract(nanosecond from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) ----- -50123000000 query R SELECT date_part('epoch', arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)')) @@ -1266,65 +1250,58 @@ SELECT extract(epoch from arrow_cast('23:32:50.123'::time, 'Time32(Millisecond)' 84770.123 ## time64 microseconds -query R +query I SELECT date_part('hour', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- 23 -query R +query I SELECT extract(hour from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- 23 -query R +query I SELECT date_part('minute', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- 32 -query R +query I SELECT extract(minute from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- 32 -query R +query I SELECT date_part('second', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- -50.123456 +50 -query R +query I SELECT extract(second from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- -50.123456 +50 -query R +query I SELECT date_part('millisecond', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- -50123.456 +50123 -query R +query I SELECT extract(millisecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- -50123.456 +50123 -query R +query I SELECT date_part('microsecond', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- 50123456 -query R +query I SELECT extract(microsecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ---- 50123456 -query R -SELECT date_part('nanosecond', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50123456000 - -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT extract(nanosecond from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) ----- -50123456000 query R SELECT date_part('epoch', arrow_cast('23:32:50.123456'::time, 'Time64(Microsecond)')) @@ -1337,81 +1314,74 @@ SELECT extract(epoch from arrow_cast('23:32:50.123456'::time, 'Time64(Microsecon 84770.123456 ## time64 nanoseconds -query R +query I SELECT date_part('hour', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- 23 -query R +query I SELECT extract(hour from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- 23 -query R +query I SELECT date_part('minute', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- 32 -query R +query I SELECT extract(minute from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- 32 -query R +query I SELECT date_part('second', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- -50.123456789 +50 -query R +query I select extract(second from '2024-08-09T12:13:14') ---- 14 -query R +query I select extract(seconds from '2024-08-09T12:13:14') ---- 14 -query R +query I SELECT extract(second from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- -50.123456789 +50 -query R +query I SELECT date_part('millisecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- -50123.456789 +50123 -query R +query I SELECT extract(millisecond from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- -50123.456789 +50123 # just some floating point stuff happening in the result here -query R +query I SELECT date_part('microsecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- -50123456.789000005 +50123456 -query R +query I SELECT extract(microsecond from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- -50123456.789000005 +50123456 -query R +query I SELECT extract(us from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ---- -50123456.789000005 +50123456 -query R +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT date_part('nanosecond', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50123456789 - -query R -SELECT extract(nanosecond from arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) ----- -50123456789 query R SELECT date_part('epoch', arrow_cast('23:32:50.123456789'::time, 'Time64(Nanosecond)')) @@ -1487,32 +1457,32 @@ SELECT extract(epoch from arrow_cast('1969-12-31', 'Date64')) # test_extract_interval -query R +query I SELECT extract(year from arrow_cast('10 years', 'Interval(YearMonth)')) ---- 10 -query R +query I SELECT extract(month from arrow_cast('10 years', 'Interval(YearMonth)')) ---- 0 -query R +query I SELECT extract(year from arrow_cast('10 months', 'Interval(YearMonth)')) ---- 0 -query R +query I SELECT extract(month from arrow_cast('10 months', 'Interval(YearMonth)')) ---- 10 -query R +query I SELECT extract(year from arrow_cast('20 months', 'Interval(YearMonth)')) ---- 1 -query R +query I SELECT extract(month from arrow_cast('20 months', 'Interval(YearMonth)')) ---- 8 @@ -1523,47 +1493,47 @@ SELECT extract(year from arrow_cast('10 days', 'Interval(DayTime)')) query error DataFusion error: Arrow error: Compute error: Month does not support: Interval\(DayTime\) SELECT extract(month from arrow_cast('10 days', 'Interval(DayTime)')) -query R +query I SELECT extract(day from arrow_cast('10 days', 'Interval(DayTime)')) ---- 10 -query R +query I SELECT extract(day from arrow_cast('14400 minutes', 'Interval(DayTime)')) ---- 0 -query R +query I SELECT extract(minute from arrow_cast('14400 minutes', 'Interval(DayTime)')) ---- 14400 -query R +query I SELECT extract(second from arrow_cast('5.1 seconds', 'Interval(DayTime)')) ---- 5 -query R +query I SELECT extract(second from arrow_cast('14400 minutes', 'Interval(DayTime)')) ---- 864000 -query R +query I SELECT extract(second from arrow_cast('2 months', 'Interval(MonthDayNano)')) ---- 0 -query R +query I SELECT extract(second from arrow_cast('2 days', 'Interval(MonthDayNano)')) ---- 0 -query R +query I SELECT extract(second from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ---- 2 -query R +query I SELECT extract(seconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ---- 2 @@ -1573,17 +1543,17 @@ SELECT extract(epoch from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ---- 2 -query R +query I SELECT extract(milliseconds from arrow_cast('2 seconds', 'Interval(MonthDayNano)')) ---- 2000 -query R +query I SELECT extract(second from arrow_cast('2030 milliseconds', 'Interval(MonthDayNano)')) ---- -2.03 +2 -query R +query I SELECT extract(second from arrow_cast(NULL, 'Interval(MonthDayNano)')) ---- NULL @@ -1597,7 +1567,7 @@ create table t (id int, i interval) as values (4, interval '8 months'), (5, NULL); -query IRR rowsort +query III select id, extract(second from i), @@ -1605,9 +1575,9 @@ select from t order by id; ---- -0 0.00000001 5 +0 0 5 1 0 15 -2 0.002 0 +2 0 0 3 2 0 4 0 8 5 NULL NULL @@ -1617,12 +1587,12 @@ drop table t; # test_extract_duration -query R +query I SELECT extract(second from arrow_cast(2, 'Duration(Second)')) ---- 2 -query R +query I SELECT extract(seconds from arrow_cast(2, 'Duration(Second)')) ---- 2 @@ -1632,27 +1602,27 @@ SELECT extract(epoch from arrow_cast(2, 'Duration(Second)')) ---- 2 -query R +query I SELECT extract(millisecond from arrow_cast(2, 'Duration(Second)')) ---- 2000 -query R +query I SELECT extract(second from arrow_cast(2, 'Duration(Millisecond)')) ---- -0.002 +0 -query R +query I SELECT extract(second from arrow_cast(2002, 'Duration(Millisecond)')) ---- -2.002 +2 -query R +query I SELECT extract(millisecond from arrow_cast(2002, 'Duration(Millisecond)')) ---- 2002 -query R +query I SELECT extract(day from arrow_cast(864000, 'Duration(Second)')) ---- 10 @@ -1663,7 +1633,7 @@ SELECT extract(month from arrow_cast(864000, 'Duration(Second)')) query error DataFusion error: Arrow error: Compute error: Year does not support: Duration\(Second\) SELECT extract(year from arrow_cast(864000, 'Duration(Second)')) -query R +query I SELECT extract(day from arrow_cast(NULL, 'Duration(Second)')) ---- NULL @@ -1720,10 +1690,8 @@ SELECT (date_part('microsecond', now()) = EXTRACT(microsecond FROM now())) ---- true -query B +query error DataFusion error: Internal error: unit Nanosecond not supported SELECT (date_part('nanosecond', now()) = EXTRACT(nanosecond FROM now())) ----- -true query B SELECT 'a' IN ('a','b') @@ -2230,7 +2198,7 @@ SELECT digest('','blake3'); ---- af1349b9f5f9a1a6a0404dea36dcc9499bcb25c9adc112b7cc9a93cae41f3262 -# vverify utf8view +# vverify utf8view query ? SELECT sha224(arrow_cast('tom', 'Utf8View')); ---- diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index a7568d88f797..4b770a19fe20 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -756,3 +756,202 @@ query TT select substr('Andrew Lamb', 1, 6), '|' ---- Andrew | + + +# test for greatest +statement ok +CREATE TABLE t1 (a int, b int, c int) as VALUES +(4, NULL, NULL), +(1, 2, 3), +(3, 1, 2), +(1, NULL, -1), +(NULL, NULL, NULL), +(3, 0, -1); + +query I +SELECT greatest(a, b, c) FROM t1 +---- +4 +3 +3 +1 +NULL +3 + +statement ok +drop table t1 + +query I +SELECT greatest(1) +---- +1 + +query I +SELECT greatest(1, 2) +---- +2 + +query I +SELECT greatest(3, 1) +---- +3 + +query ? +SELECT greatest(NULL) +---- +NULL + +query I +SELECT greatest(1, NULL, -1) +---- +1 + +query I +SELECT greatest((3), (0), (-1)); +---- +3 + +query ? +SELECT greatest([4, 3], [4, 2], [4, 4]); +---- +[4, 4] + +query ? +SELECT greatest([2, 3], [1, 4], [5, 0]); +---- +[5, 0] + +query I +SELECT greatest(1::int, 2::text) +---- +2 + +query R +SELECT greatest(-1, 1, 2.3, 123456789, 3 + 5, -(-4)) +---- +123456789 + +query R +SELECT greatest(-1.123, 1.21313, 2.3, 123456789.321, 3 + 5.3213, -(-4.3213), abs(-9)) +---- +123456789.321 + +query R +SELECT greatest(-1, 1, 2.3, 123456789, 3 + 5, -(-4), abs(-9.0)) +---- +123456789 + + +query error greatest does not support zero arguments +SELECT greatest() + +query I +SELECT greatest(4, 5, 7, 1, 2) +---- +7 + +query I +SELECT greatest(4, NULL, 7, 1, 2) +---- +7 + +query I +SELECT greatest(NULL, NULL, 7, NULL, 2) +---- +7 + +query I +SELECT greatest(NULL, NULL, NULL, NULL, 2) +---- +2 + +query I +SELECT greatest(2, NULL, NULL, NULL, NULL) +---- +2 + +query ? +SELECT greatest(NULL, NULL, NULL) +---- +NULL + +query I +SELECT greatest(2, '4') +---- +4 + +query T +SELECT greatest('foo', 'bar', 'foobar') +---- +foobar + +query R +SELECT greatest(1, 1.2) +---- +1.2 + +statement ok +CREATE TABLE foo (a int) + +statement ok +INSERT INTO foo (a) VALUES (1) + +# Test homogenous functions that can't be constant folded. +query I +SELECT greatest(NULL, a, 5, NULL) FROM foo +---- +5 + +query I +SELECT greatest(NULL, NULL, NULL, a, -1) FROM foo +---- +1 + +statement ok +drop table foo + +query R +select greatest(arrow_cast('NAN','Float64'), arrow_cast('NAN','Float64')) +---- +NaN + +query R +select greatest(arrow_cast('NAN','Float64'), arrow_cast('NAN','Float32')) +---- +NaN + +query R +select greatest(arrow_cast('NAN','Float64'), '+Inf'::Double) +---- +NaN + +query R +select greatest(arrow_cast('NAN','Float64'), NULL) +---- +NaN + +query R +select greatest(NULL, '+Inf'::Double) +---- +Infinity + +query R +select greatest(NULL, '-Inf'::Double) +---- +-Infinity + +statement ok +CREATE TABLE t1 (a double, b double, c double) as VALUES +(1, arrow_cast('NAN', 'Float64'), '+Inf'::Double), +(NULL, arrow_cast('NAN','Float64'), '+Inf'::Double), +(1, '+Inf'::Double, NULL); + +query R +SELECT greatest(a, b, c) FROM t1 +---- +NaN +NaN +Infinity + +statement ok +drop table t1 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index f74e1006f7f6..4acf519c5de4 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4294,7 +4294,7 @@ physical_plan 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false -query R +query I SELECT extract(month from ts) as months FROM csv_with_timestamps GROUP BY extract(month from ts) @@ -4344,7 +4344,7 @@ create table t1(state string, city string, min_temp float, area int, time timest ('MA', 'Boston', 70.4, 1, 50), ('MA', 'Bedford', 71.59, 2, 150); -query RI +query II select date_part('year', time) as bla, count(distinct state) as count from t1 group by bla; ---- 1970 1 @@ -5483,3 +5483,19 @@ SELECT max(input_table.x), min(input_table.x) from input_table GROUP BY input_ta ---- NaN NaN +# Group by timestamp +query TP +SELECT + 'foo' AS text, + arrow_cast('2024-01-01T00:00:00Z'::timestamptz, 'Timestamp(Microsecond, Some("UTC"))') AS ts +GROUP BY ts, text +---- +foo 2024-01-01T00:00:00Z + +query TP +SELECT + 'foo' AS text, + arrow_cast('2024-01-01T00:00:00Z'::timestamptz, 'Timestamp(Second, Some("+08:00"))') AS ts +GROUP BY ts, text +---- +foo 2024-01-01T08:00:00+08:00 diff --git a/datafusion/sqllogictest/test_files/map.slt b/datafusion/sqllogictest/test_files/map.slt index 10ca3ae881bf..28fc2f4b0b80 100644 --- a/datafusion/sqllogictest/test_files/map.slt +++ b/datafusion/sqllogictest/test_files/map.slt @@ -185,7 +185,7 @@ SELECT MAP([[1,2], [3,4]], ['a', 'b']); query error SELECT MAP() -query error DataFusion error: Execution error: map requires an even number of arguments, got 1 instead +query error DataFusion error: Execution error: map requires exactly 2 arguments, got 1 instead SELECT MAP(['POST', 'HEAD']) query error DataFusion error: Execution error: Expected list, large_list or fixed_size_list, got Null diff --git a/datafusion/sqllogictest/test_files/nullif.slt b/datafusion/sqllogictest/test_files/nullif.slt index a5060077fe77..18642f6971ca 100644 --- a/datafusion/sqllogictest/test_files/nullif.slt +++ b/datafusion/sqllogictest/test_files/nullif.slt @@ -97,11 +97,54 @@ SELECT NULLIF(1, 3); ---- 1 -query I +query T SELECT NULLIF(NULL, NULL); ---- NULL +query R +select nullif(1, 1.2); +---- +1 + +query R +select nullif(1.0, 2); +---- +1 + +query error DataFusion error: Arrow error: Cast error: Cannot cast string 'a' to value of Int64 type +select nullif(2, 'a'); + +query T +select nullif('2', '3'); +---- +2 + +query I +select nullif(2, '1'); +---- +2 + +query I +select nullif('2', 2); +---- +NULL + +query I +select nullif('1', 2); +---- +1 + +statement ok +create table t(a varchar, b int) as values ('1', 2), ('2', 2), ('3', 2); + +query I +select nullif(a, b) from t; +---- +1 +NULL +3 + query T SELECT NULLIF(arrow_cast('a', 'Utf8View'), 'a'); ---- @@ -130,4 +173,4 @@ NULL query T SELECT NULLIF(arrow_cast('a', 'Utf8View'), null); ---- -a \ No newline at end of file +a diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index d5f0521407c5..a46040aa532e 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -1260,3 +1260,49 @@ limit 2; statement ok drop table ordered_table; + +query TT +EXPLAIN SELECT + CASE + WHEN name = 'name1' THEN 0.0 + WHEN name = 'name2' THEN 0.5 + END AS a +FROM ( + SELECT 'name1' AS name + UNION ALL + SELECT 'name2' +) +ORDER BY a DESC; +---- +logical_plan +01)Sort: a DESC NULLS FIRST +02)--Projection: CASE WHEN name = Utf8("name1") THEN Float64(0) WHEN name = Utf8("name2") THEN Float64(0.5) END AS a +03)----Union +04)------Projection: Utf8("name1") AS name +05)--------EmptyRelation +06)------Projection: Utf8("name2") AS name +07)--------EmptyRelation +physical_plan +01)SortPreservingMergeExec: [a@0 DESC] +02)--ProjectionExec: expr=[CASE WHEN name@0 = name1 THEN 0 WHEN name@0 = name2 THEN 0.5 END as a] +03)----UnionExec +04)------ProjectionExec: expr=[name1 as name] +05)--------PlaceholderRowExec +06)------ProjectionExec: expr=[name2 as name] +07)--------PlaceholderRowExec + +query R +SELECT + CASE + WHEN name = 'name1' THEN 0.0 + WHEN name = 'name2' THEN 0.5 + END AS a +FROM ( + SELECT 'name1' AS name + UNION ALL + SELECT 'name2' +) +ORDER BY a DESC; +---- +0.5 +0 diff --git a/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt b/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt index c16cfc2ca38e..01071f03dce6 100644 --- a/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt +++ b/datafusion/sqllogictest/test_files/string/dictionary_utf8.slt @@ -42,43 +42,6 @@ SELECT arrow_cast('', 'Dictionary(Int32, Utf8)'); ---- (empty) -# TODO: move it back to `string_query.slt.part` after fixing the issue -# see detail: https://github.com/apache/datafusion/issues/12637 -# Test pattern with wildcard characters -query TTBBBB -select ascii_1, unicode_1, - ascii_1 like 'An%' as ascii_like, - unicode_1 like '%ion数据%' as unicode_like, - ascii_1 ilike 'An%' as ascii_ilike, - unicode_1 ilike '%ion数据%' as unicode_ilik -from test_basic_operator; ----- -Andrew datafusion📊🔥 true false true false -Xiangpeng datafusion数据融合 false true false true -Raphael datafusionДатаФусион false false false false -under_score un iść core false false false false -percent pan Tadeusz ma iść w kąt false false false false -(empty) (empty) false false false false -% (empty) false false false false -_ (empty) false false false false -NULL NULL NULL NULL NULL NULL -NULL NULL NULL NULL NULL NULL - -# TODO: move it back to `string_query.slt.part` after fixing the issue -# see issue https://github.com/apache/datafusion/issues/13329 -query IIII -select bit_length(ascii_1), bit_length(ascii_2), bit_length(unicode_1), bit_length(unicode_2) from test_basic_operator; ----- -48 8 144 32 -72 72 176 176 -56 8 240 64 -88 88 104 256 -56 24 216 288 -0 8 0 0 -8 16 0 0 -8 16 0 0 -NULL 8 NULL NULL -NULL 8 NULL 32 # # common test for string-like functions and operators diff --git a/datafusion/sqllogictest/test_files/string/init_data.slt.part b/datafusion/sqllogictest/test_files/string/init_data.slt.part index 9cdeff1977ee..06b65ff8e72a 100644 --- a/datafusion/sqllogictest/test_files/string/init_data.slt.part +++ b/datafusion/sqllogictest/test_files/string/init_data.slt.part @@ -15,7 +15,6 @@ # specific language governing permissions and limitations # under the License. -# TODO (https://github.com/apache/datafusion/issues/12637): add a row with '%%' pattern statement ok create table test_source as values ('Andrew', 'X', 'datafusion📊🔥', '🔥'), @@ -24,6 +23,7 @@ create table test_source as values ('under_score', 'un_____core', 'un iść core', 'chrząszcz na łące w 東京都'), ('percent', 'p%t', 'pan Tadeusz ma iść w kąt', 'Pan Tadeusz ma frunąć stąd w kąt'), ('', '%', '', ''), + ('', '%%', '', ''), ('%', '\%', '', ''), ('_', '\_', '', ''), (NULL, '%', NULL, NULL), diff --git a/datafusion/sqllogictest/test_files/string/large_string.slt b/datafusion/sqllogictest/test_files/string/large_string.slt index 35fc5694df6f..84f1e8382e53 100644 --- a/datafusion/sqllogictest/test_files/string/large_string.slt +++ b/datafusion/sqllogictest/test_files/string/large_string.slt @@ -44,49 +44,12 @@ Raphael R datafusionДатаФусион аФус under_score un_____core un iść core chrząszcz na łące w 東京都 percent p%t pan Tadeusz ma iść w kąt Pan Tadeusz ma frunąć stąd w kąt (empty) % (empty) (empty) +(empty) %% (empty) (empty) % \% (empty) (empty) _ \_ (empty) (empty) NULL % NULL NULL NULL R NULL 🔥 -# TODO: move it back to `string_query.slt.part` after fixing the issue -# see detail: https://github.com/apache/datafusion/issues/12637 -# Test pattern with wildcard characters -query TTBBBB -select ascii_1, unicode_1, - ascii_1 like 'An%' as ascii_like, - unicode_1 like '%ion数据%' as unicode_like, - ascii_1 ilike 'An%' as ascii_ilike, - unicode_1 ilike '%ion数据%' as unicode_ilik -from test_basic_operator; ----- -Andrew datafusion📊🔥 true false true false -Xiangpeng datafusion数据融合 false true false true -Raphael datafusionДатаФусион false false false false -under_score un iść core false false false false -percent pan Tadeusz ma iść w kąt false false false false -(empty) (empty) false false false false -% (empty) false false false false -_ (empty) false false false false -NULL NULL NULL NULL NULL NULL -NULL NULL NULL NULL NULL NULL - -# TODO: move it back to `string_query.slt.part` after fixing the issue -# see issue https://github.com/apache/datafusion/issues/13329 -query IIII -select bit_length(ascii_1), bit_length(ascii_2), bit_length(unicode_1), bit_length(unicode_2) from test_basic_operator; ----- -48 8 144 32 -72 72 176 176 -56 8 240 64 -88 88 104 256 -56 24 216 288 -0 8 0 0 -8 16 0 0 -8 16 0 0 -NULL 8 NULL NULL -NULL 8 NULL 32 - # # common test for string-like functions and operators # diff --git a/datafusion/sqllogictest/test_files/string/string.slt b/datafusion/sqllogictest/test_files/string/string.slt index db2b67d66333..55f0c034f5f9 100644 --- a/datafusion/sqllogictest/test_files/string/string.slt +++ b/datafusion/sqllogictest/test_files/string/string.slt @@ -34,27 +34,7 @@ statement ok create table test_substr as select arrow_cast(col1, 'Utf8') as c1 from test_substr_base; -# TODO: move it back to `string_query.slt.part` after fixing the issue -# see detail: https://github.com/apache/datafusion/issues/12637 -# Test pattern with wildcard characters -query TTBBBB -select ascii_1, unicode_1, - ascii_1 like 'An%' as ascii_like, - unicode_1 like '%ion数据%' as unicode_like, - ascii_1 ilike 'An%' as ascii_ilike, - unicode_1 ilike '%ion数据%' as unicode_ilik -from test_basic_operator; ----- -Andrew datafusion📊🔥 true false true false -Xiangpeng datafusion数据融合 false true false true -Raphael datafusionДатаФусион false false false false -under_score un iść core false false false false -percent pan Tadeusz ma iść w kąt false false false false -(empty) (empty) false false false false -% (empty) false false false false -_ (empty) false false false false -NULL NULL NULL NULL NULL NULL -NULL NULL NULL NULL NULL NULL + # # common test for string-like functions and operators @@ -80,6 +60,9 @@ SELECT unicode_2, 'is NOT LIKE', ascii_2 FROM test_basic_operator WHERE unicode_ (empty) is LIKE % (empty) is LIKE % (empty) is LIKE % +(empty) is LIKE %% +(empty) is LIKE %% +(empty) is LIKE %% (empty) is NOT LIKE \% (empty) is NOT LIKE \% (empty) is NOT LIKE \_ @@ -115,6 +98,7 @@ FROM test_basic_operator ---- % \% (empty) (empty) true true false false (empty) % (empty) (empty) true false true true +(empty) %% (empty) (empty) true false true true Andrew X datafusion📊🔥 🔥 false false false false NULL % NULL NULL NULL NULL NULL NULL NULL R NULL 🔥 NULL NULL NULL false @@ -143,6 +127,9 @@ SELECT unicode_2, 'is NOT ILIKE', ascii_2 FROM test_basic_operator WHERE unicode (empty) is ILIKE % (empty) is ILIKE % (empty) is ILIKE % +(empty) is ILIKE %% +(empty) is ILIKE %% +(empty) is ILIKE %% (empty) is NOT ILIKE \% (empty) is NOT ILIKE \% (empty) is NOT ILIKE \_ @@ -178,6 +165,7 @@ FROM test_basic_operator ---- % \% (empty) (empty) true true false false (empty) % (empty) (empty) true false true true +(empty) %% (empty) (empty) true false true true Andrew X datafusion📊🔥 🔥 false false false false NULL % NULL NULL NULL NULL NULL NULL NULL R NULL 🔥 NULL NULL NULL false @@ -187,21 +175,7 @@ _ \_ (empty) (empty) true false false false percent p%t pan Tadeusz ma iść w kąt Pan Tadeusz ma frunąć stąd w kąt true false true true under_score un_____core un iść core chrząszcz na łące w 東京都 true false true false -# TODO: move it back to `string_query.slt.part` after fixing the issue -# see issue https://github.com/apache/datafusion/issues/13329 -query IIII -select bit_length(ascii_1), bit_length(ascii_2), bit_length(unicode_1), bit_length(unicode_2) from test_basic_operator; ----- -48 8 144 32 -72 72 176 176 -56 8 240 64 -88 88 104 256 -56 24 216 288 -0 8 0 0 -8 16 0 0 -8 16 0 0 -NULL 8 NULL NULL -NULL 8 NULL 32 + # # Clean up diff --git a/datafusion/sqllogictest/test_files/string/string_literal.slt b/datafusion/sqllogictest/test_files/string/string_literal.slt index 145081f91a30..738cb7f27054 100644 --- a/datafusion/sqllogictest/test_files/string/string_literal.slt +++ b/datafusion/sqllogictest/test_files/string/string_literal.slt @@ -861,16 +861,16 @@ SELECT ---- false false true false true false -# \ as an explicit escape character is currently not supported -query error DataFusion error: Execution error: LIKE does not support escape_char +query BBBB SELECT 'a' LIKE '\%' ESCAPE '\', '\a' LIKE '\%' ESCAPE '\', '%' LIKE '\%' ESCAPE '\', '\%' LIKE '\%' ESCAPE '\' +---- +false false true false -# \ as an explicit escape character is currently not supported -query error DataFusion error: Execution error: LIKE does not support escape_char +query BBBBBB SELECT 'a' LIKE '\_' ESCAPE '\', '\a' LIKE '\_' ESCAPE '\', @@ -878,6 +878,26 @@ SELECT '\_' LIKE '\_' ESCAPE '\', 'abc' LIKE 'a_c' ESCAPE '\', 'abc' LIKE 'a\_c' ESCAPE '\' +---- +false false true false true false + +# Only \ is currently supported as an explicit escape character +query error DataFusion error: Execution error: LIKE does not support escape_char other than the backslash \(\\\) +SELECT + 'a' LIKE '$%' ESCAPE '$', + '\a' LIKE '$%' ESCAPE '$', + '%' LIKE '$%' ESCAPE '$', + '\%' LIKE '$%' ESCAPE '$' + +# Only \ is currently supported as an explicit escape character +query error DataFusion error: Execution error: LIKE does not support escape_char other than the backslash \(\\\) +SELECT + 'a' LIKE '$_' ESCAPE '$', + '\a' LIKE '$_' ESCAPE '$', + '_' LIKE '$_' ESCAPE '$', + '\_' LIKE '$_' ESCAPE '$', + 'abc' LIKE 'a_c' ESCAPE '$', + 'abc' LIKE 'a$_c' ESCAPE '$' # a LIKE pattern containing escape can never match an empty string query BBBBB diff --git a/datafusion/sqllogictest/test_files/string/string_query.slt.part b/datafusion/sqllogictest/test_files/string/string_query.slt.part index c42a9384c5d0..80fcc0102887 100644 --- a/datafusion/sqllogictest/test_files/string/string_query.slt.part +++ b/datafusion/sqllogictest/test_files/string/string_query.slt.part @@ -29,6 +29,7 @@ Raphael R datafusionДатаФусион аФус under_score un_____core un iść core chrząszcz na łące w 東京都 percent p%t pan Tadeusz ma iść w kąt Pan Tadeusz ma frunąć stąd w kąt (empty) % (empty) (empty) +(empty) %% (empty) (empty) % \% (empty) (empty) _ \_ (empty) (empty) NULL % NULL NULL @@ -51,6 +52,7 @@ Raphael R under_score un_____core percent p%t (empty) % +(empty) %% % \% _ \_ @@ -61,6 +63,7 @@ datafusion数据融合 datafusion数据融合 (empty) (empty) (empty) (empty) (empty) (empty) +(empty) (empty) query TT select unicode_1, unicode_2 from test_basic_operator where unicode_1 <> unicode_2 @@ -74,6 +77,7 @@ query TT select ascii_1, unicode_1 from test_basic_operator where ascii_1 = unicode_1 ---- (empty) (empty) +(empty) (empty) query TT select ascii_1, unicode_1 from test_basic_operator where ascii_1 <> unicode_1 @@ -106,6 +110,7 @@ Raphael R datafusionДатаФусион аФус false true false true false tr under_score un_____core un iść core chrząszcz na łące w 東京都 false true false true false true percent p%t pan Tadeusz ma iść w kąt Pan Tadeusz ma frunąć stąd w kąt false true false true false true (empty) % (empty) (empty) false true true false true false +(empty) %% (empty) (empty) false true true false true false % \% (empty) (empty) false true true false false true _ \_ (empty) (empty) false true true false false true NULL % NULL NULL NULL NULL NULL NULL NULL NULL @@ -129,6 +134,7 @@ Raphael datafusionДатаФусион false true false true under_score un iść core false true false true percent pan Tadeusz ma iść w kąt false true false true (empty) (empty) false true false true +(empty) (empty) false true false true % (empty) false true false true _ (empty) false true false true NULL NULL NULL NULL NULL NULL @@ -152,6 +158,7 @@ Raphael datafusionДатаФусион false true false true under_score un iść core false true false true percent pan Tadeusz ma iść w kąt false true false true (empty) (empty) false true false true +(empty) (empty) false true false true % (empty) false true false true _ (empty) false true false true NULL NULL NULL NULL NULL NULL @@ -175,6 +182,7 @@ Raphael datafusionДатаФусион false true false true under_score un iść core false true false true percent pan Tadeusz ma iść w kąt false true false true (empty) (empty) false true false true +(empty) (empty) false true false true % (empty) false true false true _ (empty) false true false true NULL NULL NULL NULL NULL NULL @@ -223,6 +231,7 @@ Rap R dat аФу und un_ un chr per p%t pan Pan (empty) % (empty) (empty) +(empty) %% (empty) (empty) % \% (empty) (empty) _ \_ (empty) (empty) NULL % NULL NULL @@ -256,6 +265,7 @@ GROUP BY ascii_2; 1 1 1 1 1 1 +1 1 query II rowsort SELECT @@ -293,6 +303,7 @@ false false false false false true true false false true true false false true true false +false true true false NULL NULL NULL NULL NULL NULL NULL NULL @@ -312,6 +323,7 @@ false false false false false false false false false false false false false false false false +false false false false NULL false NULL NULL NULL false NULL false @@ -330,6 +342,7 @@ Raphael under_scrre percent (empty) +(empty) % _ NULL @@ -348,6 +361,7 @@ pan Tadeusz ma iść w kąt (empty) (empty) (empty) +(empty) NULL NULL @@ -367,6 +381,7 @@ Raphafl undfr_score pfrcent (empty) +(empty) % _ NULL @@ -384,6 +399,7 @@ Raphafl undfr_score pfrcent (empty) +(empty) % _ NULL @@ -403,6 +419,7 @@ bar bar bar bar +bar NULL NULL @@ -420,6 +437,7 @@ bar bar bar bar +bar NULL NULL @@ -444,6 +462,7 @@ Raphael Datafusionдатафусион Under_Score Un Iść Core Percent Pan Tadeusz Ma Iść W KąT (empty) (empty) +(empty) (empty) % (empty) _ (empty) NULL NULL @@ -470,6 +489,7 @@ FROM test_basic_operator; 117 117 117 99 112 112 112 80 0 37 0 0 +0 37 0 0 37 92 0 0 95 92 0 0 NULL 37 NULL NULL @@ -496,6 +516,7 @@ Raphael Raphael NULL datafusionДатаФусион datafusionДатаФусио under_score under_score NULL un iść core un iść core NULL percent percent NULL pan Tadeusz ma iść w kąt pan Tadeusz ma iść w kąt NULL (empty) (empty) NULL (empty) (empty) NULL +(empty) (empty) NULL (empty) (empty) NULL % % NULL (empty) (empty) NULL _ _ NULL (empty) (empty) NULL NULL NULL NULL NULL NULL NULL @@ -522,6 +543,7 @@ Raphael aphael NULL datafusionДатаФусион NULL datafusionДатаФус under_score der_score NULL un iść core NULL un iść core percent ercent NULL pan Tadeusz ma iść w kąt NULL pan Tadeusz ma iść w kąt (empty) (empty) NULL (empty) NULL (empty) +(empty) (empty) NULL (empty) NULL (empty) % (empty) NULL (empty) NULL (empty) _ (empty) NULL (empty) NULL (empty) NULL NULL NULL NULL NULL NULL @@ -547,6 +569,7 @@ Raphael Raphael Raphael NULL datafusionДатаФусион under_sco under_s under_score NULL un iść core percent percen percent NULL pan Tadeusz ma iść w kąt (empty) (empty) (empty) NULL (empty) +(empty) (empty) (empty) NULL (empty) % (empty) % NULL (empty) _ (empty) _ NULL (empty) NULL NULL NULL NULL NULL @@ -574,6 +597,7 @@ false false NULL false NULL false false false NULL true NULL false false false NULL true NULL false false false NULL true NULL false +false false NULL true NULL false NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -590,6 +614,7 @@ raphael datafusionдатафусион under_score un iść core percent pan tadeusz ma iść w kąt (empty) (empty) +(empty) (empty) % (empty) _ (empty) NULL NULL @@ -608,6 +633,7 @@ RAPHAEL DATAFUSIONДАТАФУСИОН UNDER_SCORE UN IŚĆ CORE PERCENT PAN TADEUSZ MA IŚĆ W KĄT (empty) (empty) +(empty) (empty) % (empty) _ (empty) NULL NULL @@ -639,6 +665,7 @@ Raphael:Data RaphaelR Raphael RaphaeldatafusionДатаФусион RaphaelаФ under_score:Data under_scoreun_____core under_score under_scoreun iść core under_scorechrząszcz na łące w 東京都 un iść coreunder_score un iść corechrząszcz na łące w 東京都 un iść core un iść core🔥 🔥 (empty) under_score,un iść core percent:Data percentp%t percent percentpan Tadeusz ma iść w kąt percentPan Tadeusz ma frunąć stąd w kąt pan Tadeusz ma iść w kątpercent pan Tadeusz ma iść w kątPan Tadeusz ma frunąć stąd w kąt pan Tadeusz ma iść w kąt pan Tadeusz ma iść w kąt🔥 🔥 (empty) percent,pan Tadeusz ma iść w kąt :Data % (empty) (empty) (empty) (empty) (empty) (empty) 🔥 🔥 (empty) , +:Data %% (empty) (empty) (empty) (empty) (empty) (empty) 🔥 🔥 (empty) , %:Data %\% % % % % (empty) (empty) 🔥 🔥 (empty) %, _:Data _\_ _ _ _ _ (empty) (empty) 🔥 🔥 (empty) _, :Data % (empty) (empty) (empty) (empty) (empty) (empty) 🔥 🔥 (empty) , @@ -666,6 +693,7 @@ pfooent pfooTadeusz ma iść w kąt p🔥rcent p🔥n Tadeusz ma iść w kąt NU foo foo 🔥 🔥 NULL NULL foo foo 🔥 🔥 NULL NULL foo foo 🔥 🔥 NULL NULL +foo foo 🔥 🔥 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -689,6 +717,7 @@ Raphael baraphael NULL datafusionДатbarион NULL datafusionДатаФуси under_score under_score NULL un iść core NULL un iść core percent percent NULL pan Tadeusz ma iść w kąt NULL pan Tadeusz ma iść w kąt (empty) (empty) NULL bar NULL (empty) +(empty) (empty) NULL bar NULL (empty) % % NULL bar NULL (empty) _ _ NULL bar NULL (empty) NULL NULL NULL NULL NULL NULL @@ -714,6 +743,7 @@ ael (empty) hael ион (empty) afusionДатаФусион ore (empty) er_score ore (empty) iść core ent (empty) cent kąt (empty) Tadeusz ma iść w kąt (empty) (empty) (empty) (empty) (empty) (empty) +(empty) (empty) (empty) (empty) (empty) (empty) % (empty) (empty) (empty) (empty) (empty) _ (empty) (empty) (empty) (empty) (empty) NULL NULL NULL NULL NULL NULL @@ -740,6 +770,7 @@ Rap (empty) Raph dat (empty) datafusionДатаФус und (empty) under_sc un (empty) un iść c per (empty) perc pan (empty) pan Tadeusz ma iść w (empty) (empty) (empty) (empty) (empty) (empty) +(empty) (empty) (empty) (empty) (empty) (empty) % (empty) (empty) (empty) (empty) (empty) _ (empty) (empty) (empty) (empty) (empty) NULL NULL NULL NULL NULL NULL @@ -763,6 +794,7 @@ R Raph datafusionД datafusionДат under_score under_score un iść core un iść core percent percent pan Tadeusz ma iść w kąt pan Tadeusz ma iść w kąt (empty) (empty) (empty) (empty) +(empty) (empty) (empty) (empty) % % (empty) (empty) _ _ (empty) (empty) NULL NULL NULL NULL @@ -788,6 +820,7 @@ FROM test_basic_operator; 0 0 0 0 0 0 0 0 0 0 0 0 +0 0 0 0 NULL NULL NULL NULL NULL NULL NULL NULL @@ -811,6 +844,7 @@ Raphaelfoo Raphael🔥 datafusionДатаФусионfoo datafusionДатаФу under_scorefoo under_score🔥 un iść corefoo un iść core🔥 percentfoo percent🔥 pan Tadeusz ma iść w kątfoo pan Tadeusz ma iść w kąt🔥 foo 🔥 foo 🔥 +foo 🔥 foo 🔥 %foo %🔥 foo 🔥 _foo _🔥 foo 🔥 NULL NULL NULL NULL @@ -832,6 +866,7 @@ RaphaelR RaphaelаФус datafusionДатаФусионR datafusionДатаФу under_scoreun_____core under_scorechrząszcz na łące w 東京都 un iść coreun_____core un iść corechrząszcz na łące w 東京都 percentp%t percentPan Tadeusz ma frunąć stąd w kąt pan Tadeusz ma iść w kątp%t pan Tadeusz ma iść w kątPan Tadeusz ma frunąć stąd w kąt % (empty) % (empty) +%% (empty) %% (empty) %\% % \% (empty) _\_ _ \_ (empty) NULL NULL NULL NULL @@ -855,6 +890,7 @@ false false false false false false false false +false false NULL NULL NULL NULL @@ -872,6 +908,7 @@ false false false false false false false false +false false NULL NULL NULL NULL @@ -889,6 +926,7 @@ true true true true true true true true +true true NULL NULL NULL NULL @@ -906,6 +944,7 @@ true true true true true true true true +true true NULL NULL NULL NULL @@ -928,6 +967,7 @@ Raphael nice Raphael and R datafusionДатаФусион cool datafusionДат under_score nice under_score and un_____core un iść core cool un iść core and chrząszcz na łące w 東京都 under_score 🔥 un iść core percent nice percent and p%t pan Tadeusz ma iść w kąt cool pan Tadeusz ma iść w kąt and Pan Tadeusz ma frunąć stąd w kąt percent 🔥 pan Tadeusz ma iść w kąt nice and % cool and 🔥 + nice and %% cool and 🔥 % nice % and \% cool and % 🔥 _ nice _ and \_ cool and _ 🔥 NULL NULL NULL NULL NULL @@ -937,21 +977,27 @@ NULL NULL NULL NULL NULL # Test LIKE / ILIKE # -------------------------------------- -# TODO: StringView has wrong behavior for LIKE/ILIKE. Enable this after fixing the issue -# see issue: https://github.com/apache/datafusion/issues/12637 # Test pattern with wildcard characters -#query TTBBBB -#select ascii_1, unicode_1, -# ascii_1 like 'An%' as ascii_like, -# unicode_1 like '%ion数据%' as unicode_like, -# ascii_1 ilike 'An%' as ascii_ilike, -# unicode_1 ilike '%ion数据%' as unicode_ilik -#from test_basic_operator; -#---- -#Andrew datafusion📊🔥 true false true false -#Xiangpeng datafusion数据融合 false true false true -#Raphael datafusionДатаФусион false false false false -#NULL NULL NULL NULL NULL NULL +query TTBBBB +select ascii_1, unicode_1, + ascii_1 like 'An%' as ascii_like, + unicode_1 like '%ion数据%' as unicode_like, + ascii_1 ilike 'An%' as ascii_ilike, + unicode_1 ilike '%ion数据%' as unicode_ilik +from test_basic_operator; +---- +Andrew datafusion📊🔥 true false true false +Xiangpeng datafusion数据融合 false true false true +Raphael datafusionДатаФусион false false false false +under_score un iść core false false false false +percent pan Tadeusz ma iść w kąt false false false false +(empty) (empty) false false false false +(empty) (empty) false false false false +% (empty) false false false false +_ (empty) false false false false +NULL NULL NULL NULL NULL NULL +NULL NULL NULL NULL NULL NULL + query TTBBBB SELECT ascii_1, unicode_1, @@ -967,6 +1013,7 @@ Raphael datafusionДатаФусион true true true true under_score un iść core true true true true percent pan Tadeusz ma iść w kąt true true true true (empty) (empty) true true true true +(empty) (empty) true true true true % (empty) true true true true _ (empty) true true true true NULL NULL NULL NULL NULL NULL @@ -986,6 +1033,7 @@ Raphael datafusionДатаФусион false false false false under_score un iść core false false false false percent pan Tadeusz ma iść w kąt false false false false (empty) (empty) false false false false +(empty) (empty) false false false false % (empty) false false false false _ (empty) false false false false NULL NULL NULL NULL NULL NULL @@ -1000,6 +1048,7 @@ Raphael under_score percent (empty) +(empty) % _ @@ -1022,6 +1071,7 @@ Raphael datafusionДатаФусион false false false false under_score un iść core false false false false percent pan Tadeusz ma iść w kąt false false false false (empty) (empty) false false false false +(empty) (empty) false false false false % (empty) false false false false _ (empty) false false false false NULL NULL NULL NULL NULL NULL @@ -1044,11 +1094,32 @@ FROM 11 11 7 24 0 0 +0 0 1 0 1 0 NULL NULL NULL NULL +# -------------------------------------- +# Test BIT_LENGTH +# -------------------------------------- + +query IIII +select bit_length(ascii_1), bit_length(ascii_2), bit_length(unicode_1), bit_length(unicode_2) from test_basic_operator; +---- +48 8 144 32 +72 72 176 176 +56 8 240 64 +88 88 104 256 +56 24 216 288 +0 8 0 0 +0 16 0 0 +8 16 0 0 +8 16 0 0 +NULL 8 NULL NULL +NULL 8 NULL 32 + + # -------------------------------------- # Test Start_With # -------------------------------------- @@ -1069,6 +1140,7 @@ false false NULL NULL false false NULL NULL false false NULL NULL false false NULL NULL +false false NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -1092,6 +1164,7 @@ false false NULL NULL false false NULL NULL false false NULL NULL false false NULL NULL +false false NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -1115,6 +1188,7 @@ FROM test_basic_operator; 6 14 NULL NULL 6 14 NULL NULL 6 14 NULL NULL +6 14 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -1136,6 +1210,7 @@ xxxxxxxxxxxxxRaphael NULL datafusionДатаФусион NULL xxxxxxxxxunder_score NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥un iść core NULL xxxxxxxxxxxxxpercent NULL pan Tadeusz ma iść w NULL xxxxxxxxxxxxxxxxxxxx NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL +xxxxxxxxxxxxxxxxxxxx NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL xxxxxxxxxxxxxxxxxxx% NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL xxxxxxxxxxxxxxxxxxx_ NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL NULL NULL NULL NULL @@ -1154,6 +1229,7 @@ FROM test_basic_operator; under_score un iść core | percent pan Tadeusz ma iść w | | + | % | _ | NULL NULL | @@ -1177,6 +1253,7 @@ Raphaelxxxxxxxxxxxxx NULL datafusionДатаФусион NULL under_scorexxxxxxxxx NULL un iść core🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL percentxxxxxxxxxxxxx NULL pan Tadeusz ma iść w NULL xxxxxxxxxxxxxxxxxxxx NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL +xxxxxxxxxxxxxxxxxxxx NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL %xxxxxxxxxxxxxxxxxxx NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL _xxxxxxxxxxxxxxxxxxx NULL 🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥🔥 NULL NULL NULL NULL NULL @@ -1194,6 +1271,7 @@ Raphael datafusionДатаФусион under_score un iść core percent pan Tadeusz ma iść w + % _ NULL NULL @@ -1225,6 +1303,7 @@ false false NULL NULL false false NULL NULL false false NULL NULL false false NULL NULL false false NULL NULL false false NULL NULL false false NULL NULL false false NULL NULL +false false NULL NULL false false NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -1256,6 +1335,7 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +NULL NULL NULL NULL NULL NULL NULL NULL # -------------------------------------- # Test REPEAT @@ -1273,6 +1353,7 @@ RaphaelRaphaelRaphael datafusionДатаФусионdatafusionДатаФусио under_scoreunder_scoreunder_score un iść coreun iść coreun iść core percentpercentpercent pan Tadeusz ma iść w kątpan Tadeusz ma iść w kątpan Tadeusz ma iść w kąt (empty) (empty) +(empty) (empty) %%% (empty) ___ (empty) NULL NULL @@ -1298,6 +1379,7 @@ Rapha l NULL datafusionДатаФус он NULL und r_scor NULL un iść core (empty) NULL p rc NULL pan Tadeusz ma iść w kąt (empty) NULL (empty) (empty) NULL (empty) (empty) NULL +(empty) (empty) NULL (empty) (empty) NULL % (empty) NULL (empty) (empty) NULL _ (empty) NULL (empty) (empty) NULL NULL NULL NULL NULL NULL NULL @@ -1319,6 +1401,7 @@ leahpaR ноисуФатаДnoisufatad erocs_rednu eroc ćśi nu tnecrep tąk w ćśi am zsuedaT nap (empty) (empty) +(empty) (empty) % (empty) _ (empty) NULL NULL @@ -1346,6 +1429,7 @@ FROM test_basic_operator; 0 0 NULL 0 0 NULL 0 0 NULL 0 0 NULL 0 0 NULL 0 0 NULL +0 0 NULL 0 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -1369,6 +1453,7 @@ Rapha Raphael NULL datafusionДатаФус datafusionДатаФусион NULL und under_score NULL un iść core un iść core NULL p percent NULL pan Tadeusz ma iść w kąt pan Tadeusz ma iść w kąt NULL (empty) (empty) NULL (empty) (empty) NULL +(empty) (empty) NULL (empty) (empty) NULL % % NULL (empty) (empty) NULL _ _ NULL (empty) (empty) NULL NULL NULL NULL NULL NULL NULL @@ -1387,6 +1472,7 @@ select md5(ascii_1) from test_basic_operator; e0c4c75d58916b22a41b6ea9bc46231f 354f047ba64552895b016bbdd60ab174 d41d8cd98f00b204e9800998ecf8427e +d41d8cd98f00b204e9800998ecf8427e 0bcef9c45bd8a48eda1b26eb0c61c869 b14a7b8059d9c055954c92674ce60032 NULL @@ -1405,6 +1491,7 @@ abd8be3961e5dbe324bc67f9a0211d5f7d81e556baadaff6218e4bfa 8f6caa44143a080541f083bb762107ce12224b271bfa8b36ece002ab 951336d101e034714ba1ca0535688f0300613e235814ed938cd25115 d14a028c2a3a2bc9476102bb288234c415a2b01f828ea62ac5b3e42f +d14a028c2a3a2bc9476102bb288234c415a2b01f828ea62ac5b3e42f fda2a4d4c5fb67cfd7fc817f59b543ae42f650aa4abd79934ca5ac55 d365e3c7512c311d0df0528a850e6c827cbe508d13235fa91b545389 NULL @@ -1423,6 +1510,7 @@ ec792d2e89af0d5b05c88ee1e5fe041ce2db94f84c3aabac4f7cfe20f00cd032 d8071166bbe6131a0acaf86019eeeca31c87ee4fda23b80eda0d094dbffee521 fd86717aca41c558c78c19ab2b50691179a57ba5200bc7e3317be70efd4043ad e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855 +e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855 bbf3f11cb5b43e700273a78d12de55e4a7eab741ed2abf13787a4d2dc832b8ec d2e2adf7177b7a8afddbc12d1634cf23ea1a71020f6a1308070a16400fb68fde NULL @@ -1441,6 +1529,7 @@ select sha384(ascii_1) from test_basic_operator; fe417fcff1b9b8cdbc4fba45fedcd882ccbeef438497647052809fd73f43bcf1a6214f543a91e7183d56c6ae8e7cb30e 7791b34dcc841235a8a074052bc12aa7090c0d72f09ec41b1521a67fa09b026a9c02d159b42428d7b528aa5ff7598fd4 38b060a751ac96384cd9327eb1b1e36a21fdb71114be07434c0cc7bf63f6e1da274edebfe76f65fbd51ad2f14898b95b +38b060a751ac96384cd9327eb1b1e36a21fdb71114be07434c0cc7bf63f6e1da274edebfe76f65fbd51ad2f14898b95b bba987e661a4158451c5e9870fe91f483064574a0d7485caef40f48d7846579859c7dddebd418cbc99ccaa1ebd3619ea 586b0fd9f8ec935c69a7dceb5560742f368962833023906d30fe1cf49c96ea6d22cea8c2b63cd18e7af08fbf9e47c3f9 NULL @@ -1460,6 +1549,7 @@ f02c73afb1e433d6cc7e9137bb4ed40791e8c6e7877ae26e7a1edc4ce98a945a61bdf883d985adbc cafc51edc3a949179a74a805be8d0c7991bfc849b01f773f4bcd5e7dbe51b6d71d65921d8025d375d501af6a1c1026ab76cd7f4811b91bb4544f7dcbb710fa1f 2f845edf0e9c9728fae627d4678dc8c35c9a7f22809d355aa5ddf96d9ca3539973ac7ff96bfc6720ce6a973f93b716e265ad719ee38a85e44d9316ac1b6c89a4 cf83e1357eefb8bdf1542850d66d8007d620e4050b5715dc83f4a921d36ce9ce47d0d13c5d85f2b0ff8318d2877eec2f63b931bd47417a81a538327af927da3e +cf83e1357eefb8bdf1542850d66d8007d620e4050b5715dc83f4a921d36ce9ce47d0d13c5d85f2b0ff8318d2877eec2f63b931bd47417a81a538327af927da3e 91972aa34055bca20ddb643b9f817a547e5d4ad49b7ff16a7f828a8d72c4cb4a5679cff4da00f9fb6b2833de7eb3480b3b4a7c7c7b85a39028de55acaf2d8812 bbbe7f2559c7953d281fba7f25258063dbc8a55c5b9fdfcd334ecd64a8d7d8980c6f6ee0457bf496bcff747991f741446f1814222678dfa7457f1ad3a6f848b3 NULL @@ -1478,7 +1568,8 @@ ec792d2e89af0d5b05c88ee1e5fe041ce2db94f84c3aabac4f7cfe20f00cd032 d8071166bbe6131a0acaf86019eeeca31c87ee4fda23b80eda0d094dbffee521 fd86717aca41c558c78c19ab2b50691179a57ba5200bc7e3317be70efd4043ad e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855 +e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855 bbf3f11cb5b43e700273a78d12de55e4a7eab741ed2abf13787a4d2dc832b8ec d2e2adf7177b7a8afddbc12d1634cf23ea1a71020f6a1308070a16400fb68fde NULL -NULL \ No newline at end of file +NULL diff --git a/datafusion/sqllogictest/test_files/string/string_view.slt b/datafusion/sqllogictest/test_files/string/string_view.slt index aa41cbb8119e..ebabaf7655ff 100644 --- a/datafusion/sqllogictest/test_files/string/string_view.slt +++ b/datafusion/sqllogictest/test_files/string/string_view.slt @@ -37,22 +37,6 @@ select arrow_cast(col1, 'Utf8View') as c1 from test_substr_base; statement ok drop table test_source -# TODO: Revisit this issue after upgrading to the arrow-rs version that includes apache/arrow-rs#6671. -# see issue https://github.com/apache/datafusion/issues/13329 -query IIII -select bit_length(ascii_1), bit_length(ascii_2), bit_length(unicode_1), bit_length(unicode_2) from test_basic_operator; ----- -48 8 144 32 -72 72 176 176 -56 8 240 64 -88 88 104 256 -56 24 216 288 -0 8 0 0 -8 16 0 0 -8 16 0 0 -NULL 8 NULL NULL -NULL 8 NULL 32 - # # common test for string-like functions and operators # @@ -1034,6 +1018,27 @@ logical_plan 01)Projection: digest(test.column1_utf8view, Utf8View("md5")) AS c 02)--TableScan: test projection=[column1_utf8view] +## Ensure no unexpected casts for string_to_array +query TT +EXPLAIN SELECT + string_to_array(column1_utf8view, ',') AS c +FROM test; +---- +logical_plan +01)Projection: string_to_array(test.column1_utf8view, Utf8View(",")) AS c +02)--TableScan: test projection=[column1_utf8view] + +## Ensure no unexpected casts for array_to_string +query TT +EXPLAIN SELECT + array_to_string(string_to_array(column1_utf8view, NULL), ',') AS c +FROM test; +---- +logical_plan +01)Projection: array_to_string(string_to_array(test.column1_utf8view, Utf8View(NULL)), Utf8(",")) AS c +02)--TableScan: test projection=[column1_utf8view] + + ## Ensure no casts for binary operators # `~` operator (regex match) query TT diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index a80036df2ca8..b713008d2c3b 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -1756,13 +1756,13 @@ SELECT date_bin('1 day', TIMESTAMPTZ '2022-01-01 20:10:00Z', TIMESTAMP '2020-01- 2022-01-01T07:00:00+07:00 # postgresql: 1 -query R +query I SELECT date_part('hour', TIMESTAMPTZ '2000-01-01T01:01:01') as part ---- 1 # postgresql: 8 -query R +query I SELECT date_part('hour', TIMESTAMPTZ '2000-01-01T01:01:01Z') as part ---- 8 @@ -1839,13 +1839,13 @@ SELECT date_bin('2 hour', TIMESTAMPTZ '2022-01-01 01:10:00+07', '2020-01-01T00:0 2021-12-31T18:00:00Z # postgresql: 1 -query R +query I SELECT date_part('hour', TIMESTAMPTZ '2000-01-01T01:01:01') as part ---- 1 # postgresql: 18 -query R +query I SELECT date_part('hour', TIMESTAMPTZ '2000-01-01T01:01:01+07') as part ---- 18 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q1.slt.part new file mode 100644 index 000000000000..bd8761bbb7fb --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q1.slt.part @@ -0,0 +1,45 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TTRRRRRRRI +select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order +from + lineitem +where + l_shipdate <= date '1998-09-02' +group by + l_returnflag, + l_linestatus +order by + l_returnflag, + l_linestatus; +---- +A F 3774200 5320753880.69 5054096266.6828 5256751331.449234 25.537587 36002.123829 0.050144 147790 +N F 95257 133737795.84 127132372.6512 132286291.229445 25.300664 35521.326916 0.049394 3765 +N O 7459297 10512270008.9 9986238338.3847 10385578376.585467 25.545537 36000.924688 0.050095 292000 +R F 3785523 5337950526.47 5071818532.942 5274405503.049367 25.525943 35994.029214 0.049989 148301 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q10.slt.part new file mode 100644 index 000000000000..8a8d1a00d9fd --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q10.slt.part @@ -0,0 +1,62 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query ITRRTTTT +select + c_custkey, + c_name, + sum(l_extendedprice * (1 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment +from + customer, + orders, + lineitem, + nation +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1993-10-01' + and o_orderdate < date '1994-01-01' + and l_returnflag = 'R' + and c_nationkey = n_nationkey +group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment +order by + revenue desc +limit 10; +---- +8242 Customer#000008242 622786.7297 6322.09 ETHIOPIA P2n4nJhy,UqSo2s43YfSvYJDZ6lk 15-792-676-1184 slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i +7714 Customer#000007714 557400.3053 9799.98 IRAN SnnIGB,SkmnWpX3 20-922-418-6024 arhorses according to the blithely express re +11032 Customer#000011032 512500.9641 8496.93 UNITED KINGDOM WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly 33-102-772-3533 posits-- furiously ironic accounts are again +2455 Customer#000002455 481592.4053 2070.99 GERMANY RVn1ZSRtLqPlJLIZxvpmsbgC02 17-946-225-9977 al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro +12106 Customer#000012106 479414.2133 5342.11 UNITED STATES wth3twOmu6vy 34-905-346-4472 ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. +8530 Customer#000008530 457855.9467 9734.95 MOROCCO GMQyte94oDM7eD7exnkj 4hH9yq3 25-736-932-5850 slyly asymptotes. quickly final deposits in +13984 Customer#000013984 446316.5104 3482.28 IRAN qZXwuapCHvxbX 20-981-264-2952 y unusual courts could wake furiously +1966 Customer#000001966 444059.0382 1937.72 ALGERIA jPv1 UHra5JLALR5Isci5u0636RoAu7t vH 10-973-269-8886 the blithely even accounts. final deposits cajole around the blithely final packages. +11026 Customer#000011026 417913.4142 7738.76 ALGERIA XorIktoJOAEJkpNNMx 10-184-163-4632 ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in +8501 Customer#000008501 412797.51 6906.7 ARGENTINA 776af4rOa mZ66hczs 11-317-552-5840 y final deposits after the fluffily even accounts are slyly final, regular diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q11.slt.part new file mode 100644 index 000000000000..53fb6bca2968 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q11.slt.part @@ -0,0 +1,58 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query IR +select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value +from + partsupp, + supplier, + nation +where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'GERMANY' +group by + ps_partkey having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + ) +order by + value desc +limit 10; +---- +12098 16227681.21 +5134 15709338.52 +13334 15023662.41 +17052 14351644.2 +3452 14070870.14 +12552 13332469.18 +1084 13170428.29 +5797 13038622.72 +12633 12892561.61 +403 12856217.34 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q12.slt.part new file mode 100644 index 000000000000..4d8e9a1a7e3d --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q12.slt.part @@ -0,0 +1,52 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TII +select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count +from + lineitem + join + orders + on + l_orderkey = o_orderkey +where + l_shipmode in ('MAIL', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1994-01-01' + and l_receiptdate < date '1995-01-01' +group by + l_shipmode +order by + l_shipmode; +---- +MAIL 647 945 +SHIP 620 943 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q13.slt.part new file mode 100644 index 000000000000..40971133d2e0 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q13.slt.part @@ -0,0 +1,51 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query II +select + c_count, + count(*) as custdist +from + ( + select + c_custkey, + count(o_orderkey) + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%special%requests%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) +group by + c_count +order by + custdist desc, + c_count desc +limit 10; +---- +0 5000 +10 665 +9 657 +11 621 +12 567 +8 564 +13 492 +18 482 +7 480 +20 456 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q14.slt.part new file mode 100644 index 000000000000..e3a72548bcfa --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q14.slt.part @@ -0,0 +1,34 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query R +select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue +from + lineitem, + part +where + l_partkey = p_partkey + and l_shipdate >= date '1995-09-01' + and l_shipdate < date '1995-10-01'; +---- +16.283855689006 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q15.slt.part new file mode 100644 index 000000000000..bfee9de11108 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q15.slt.part @@ -0,0 +1,52 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query ITTTR +with revenue0 (supplier_no, total_revenue) as ( + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1996-01-01' + and l_shipdate < date '1996-01-01' + interval '3' month + group by + l_suppkey +) +select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue +from + supplier, + revenue0 +where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) +order by + s_suppkey; +---- +677 Supplier#000000677 8mhrffG7D2WJBSQbOGstQ 23-290-639-3315 1614410.2928 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q16.slt.part new file mode 100644 index 000000000000..4412eda050c4 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q16.slt.part @@ -0,0 +1,61 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TTII +select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt +from + partsupp, + part +where + p_partkey = ps_partkey + and p_brand <> 'Brand#45' + and p_type not like 'MEDIUM POLISHED%' + and p_size in (49, 14, 23, 45, 19, 3, 36, 9) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' +) +group by + p_brand, + p_type, + p_size +order by + supplier_cnt desc, + p_brand, + p_type, + p_size +limit 10; +---- +Brand#14 SMALL ANODIZED NICKEL 45 12 +Brand#22 SMALL BURNISHED BRASS 19 12 +Brand#25 PROMO POLISHED COPPER 14 12 +Brand#35 LARGE ANODIZED STEEL 45 12 +Brand#35 PROMO BRUSHED COPPER 9 12 +Brand#51 ECONOMY ANODIZED STEEL 9 12 +Brand#53 LARGE BRUSHED NICKEL 45 12 +Brand#11 ECONOMY POLISHED COPPER 14 8 +Brand#11 LARGE PLATED STEEL 23 8 +Brand#11 PROMO POLISHED STEEL 23 8 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q17.slt.part new file mode 100644 index 000000000000..cdf7f1de8374 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q17.slt.part @@ -0,0 +1,38 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query R +select + sum(l_extendedprice) / 7.0 as avg_yearly +from + lineitem, + part +where + p_partkey = l_partkey + and p_brand = 'Brand#23' + and p_container = 'MED BOX' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey +); +---- +23512.752857142856 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q18.slt.part new file mode 100644 index 000000000000..93da210f51db --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q18.slt.part @@ -0,0 +1,57 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TIIDRR +select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) +from + customer, + orders, + lineitem +where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey having + sum(l_quantity) > 300 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey +group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice +order by + o_totalprice desc, + o_orderdate; +---- +Customer#000001639 1639 502886 1994-04-12 456423.88 312 +Customer#000006655 6655 29158 1995-10-21 452805.02 305 +Customer#000014110 14110 565574 1995-09-24 425099.85 301 +Customer#000001775 1775 6882 1997-04-09 408368.1 303 +Customer#000011459 11459 551136 1993-05-19 386812.74 308 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q19.slt.part new file mode 100644 index 000000000000..a8c6a5032db8 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q19.slt.part @@ -0,0 +1,56 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query R +select + sum(l_extendedprice* (1 - l_discount)) as revenue +from + lineitem, + part +where + ( + p_partkey = l_partkey + and p_brand = 'Brand#12' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 1 and l_quantity <= 1 + 10 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#23' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 10 and l_quantity <= 10 + 10 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#34' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 20 and l_quantity <= 20 + 10 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); +---- +168597.286 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q2.slt.part new file mode 100644 index 000000000000..db4966c47be0 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q2.slt.part @@ -0,0 +1,74 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query RTTITTTT +select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment +from + part, + supplier, + partsupp, + nation, + region +where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 15 + and p_type like '%BRASS' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' +) +order by + s_acctbal desc, + n_name, + s_name, + p_partkey +limit 10; +---- +9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily +9508.37 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T +9508.37 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T +9453.01 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs +9453.01 Supplier#000000802 ROMANIA 13275 Manufacturer#4 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs +9192.1 Supplier#000000115 UNITED KINGDOM 13325 Manufacturer#1 nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV 33-597-248-1220 es across the carefully express accounts boost caref +9032.15 Supplier#000000959 GERMANY 4958 Manufacturer#4 8grA EHBnwOZhO 17-108-642-3106 nding dependencies nag furiou +8702.02 Supplier#000000333 RUSSIA 11810 Manufacturer#3 MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH 32-508-202-6136 oss the deposits cajole carefully even pinto beans. regular foxes detect alo +8615.5 Supplier#000000812 FRANCE 10551 Manufacturer#2 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref +8615.5 Supplier#000000812 FRANCE 13811 Manufacturer#4 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q20.slt.part new file mode 100644 index 000000000000..85711da7bd88 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q20.slt.part @@ -0,0 +1,66 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TT +select + s_name, + s_address +from + supplier, + nation +where + s_suppkey in ( + select + ps_suppkey + from + partsupp + where + ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'forest%' + ) + and ps_availqty > ( + select + 0.5 * sum(l_quantity) + from + lineitem + where + l_partkey = ps_partkey + and l_suppkey = ps_suppkey + and l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + ) + ) + and s_nationkey = n_nationkey + and n_name = 'CANADA' +order by + s_name; +---- +Supplier#000000157 ,mEGorBfVIm +Supplier#000000197 YC2Acon6kjY3zj3Fbxs2k4Vdf7X0cd2F +Supplier#000000287 7a9SP7qW5Yku5PvSg +Supplier#000000378 FfbhyCxWvcPrO8ltp9 +Supplier#000000530 0qwCMwobKY OcmLyfRXlagA8ukENJv, +Supplier#000000555 TfB,a5bfl3Ah 3Z 74GqnNs6zKVGM +Supplier#000000557 jj0wUYh9K3fG5Jhdhrkuy ,4 +Supplier#000000729 pqck2ppy758TQpZCUAjPvlU55K3QjfL7Bi +Supplier#000000935 ij98czM 2KzWe7dDTOxB8sq0UfCdvrX diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q21.slt.part new file mode 100644 index 000000000000..8d92dae05ec6 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q21.slt.part @@ -0,0 +1,106 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TI +select + s_name, + count(*) as numwait +from + supplier, + lineitem l1, + orders, + nation +where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'SAUDI ARABIA' +group by + s_name +order by + numwait desc, + s_name; +---- +Supplier#000000445 16 +Supplier#000000825 16 +Supplier#000000709 15 +Supplier#000000762 15 +Supplier#000000357 14 +Supplier#000000399 14 +Supplier#000000496 14 +Supplier#000000977 13 +Supplier#000000144 12 +Supplier#000000188 12 +Supplier#000000415 12 +Supplier#000000472 12 +Supplier#000000633 12 +Supplier#000000708 12 +Supplier#000000889 12 +Supplier#000000380 11 +Supplier#000000602 11 +Supplier#000000659 11 +Supplier#000000821 11 +Supplier#000000929 11 +Supplier#000000262 10 +Supplier#000000460 10 +Supplier#000000486 10 +Supplier#000000669 10 +Supplier#000000718 10 +Supplier#000000778 10 +Supplier#000000167 9 +Supplier#000000578 9 +Supplier#000000673 9 +Supplier#000000687 9 +Supplier#000000074 8 +Supplier#000000565 8 +Supplier#000000648 8 +Supplier#000000918 8 +Supplier#000000427 7 +Supplier#000000503 7 +Supplier#000000610 7 +Supplier#000000670 7 +Supplier#000000811 7 +Supplier#000000114 6 +Supplier#000000379 6 +Supplier#000000436 6 +Supplier#000000500 6 +Supplier#000000660 6 +Supplier#000000788 6 +Supplier#000000846 6 +Supplier#000000920 4 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q22.slt.part new file mode 100644 index 000000000000..22338269120f --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q22.slt.part @@ -0,0 +1,64 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TIR +select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal +from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('13', '31', '23', '29', '30', '18', '17') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00 + and substring(c_phone from 1 for 2) in + ('13', '31', '23', '29', '30', '18', '17') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale +group by + cntrycode +order by + cntrycode; +---- +13 94 714035.05 +17 96 722560.15 +18 99 738012.52 +23 93 708285.25 +29 85 632693.46 +30 87 646748.02 +31 87 647372.5 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q3.slt.part new file mode 100644 index 000000000000..1e7fc0a9619a --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q3.slt.part @@ -0,0 +1,53 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query IRDI +select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority +from + customer, + orders, + lineitem +where + c_mktsegment = 'BUILDING' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-15' + and l_shipdate > date '1995-03-15' +group by + l_orderkey, + o_orderdate, + o_shippriority +order by + revenue desc, + o_orderdate + limit 10; +---- +223140 355369.0698 1995-03-14 0 +584291 354494.7318 1995-02-21 0 +405063 353125.4577 1995-03-03 0 +573861 351238.277 1995-03-09 0 +554757 349181.7426 1995-03-14 0 +506021 321075.581 1995-03-10 0 +121604 318576.4154 1995-03-07 0 +108514 314967.0754 1995-02-20 0 +462502 312604.542 1995-03-08 0 +178727 309728.9306 1995-02-25 0 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q4.slt.part new file mode 100644 index 000000000000..a37e76f9709d --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q4.slt.part @@ -0,0 +1,46 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TI +select + o_orderpriority, + count(*) as order_count +from + orders +where + o_orderdate >= '1993-07-01' + and o_orderdate < date '1993-07-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) +group by + o_orderpriority +order by + o_orderpriority; +---- +1-URGENT 999 +2-HIGH 997 +3-MEDIUM 1031 +4-NOT SPECIFIED 989 +5-LOW 1077 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q5.slt.part new file mode 100644 index 000000000000..75e2fa316fdc --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q5.slt.part @@ -0,0 +1,49 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TR +select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue +from + customer, + orders, + lineitem, + supplier, + nation, + region +where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1995-01-01' +group by + n_name +order by + revenue desc; +---- +CHINA 7822103 +INDIA 6376121.5085 +JAPAN 6000077.2184 +INDONESIA 5580475.4027 +VIETNAM 4497840.5466 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q6.slt.part new file mode 100644 index 000000000000..57967f4d9e5c --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q6.slt.part @@ -0,0 +1,30 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query R +select + sum(l_extendedprice * l_discount) as revenue +from + lineitem +where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1995-01-01' + and l_discount between 0.06 - 0.01 and 0.06 + 0.01 + and l_quantity < 24; +---- +11803420.2534 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q7.slt.part new file mode 100644 index 000000000000..31923ffb5f7a --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q7.slt.part @@ -0,0 +1,63 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TTIR +select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue +from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY') + or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE') + ) + and l_shipdate between date '1995-01-01' and date '1996-12-31' + ) as shipping +group by + supp_nation, + cust_nation, + l_year +order by + supp_nation, + cust_nation, + l_year; +---- +FRANCE GERMANY 1995 4637235.1501 +FRANCE GERMANY 1996 5224779.5736 +GERMANY FRANCE 1995 6232818.7037 +GERMANY FRANCE 1996 5557312.1121 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q8.slt.part new file mode 100644 index 000000000000..270a7c912aa7 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q8.slt.part @@ -0,0 +1,59 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query IR +select + o_year, + cast(cast(sum(case + when nation = 'BRAZIL' then volume + else 0 + end) as decimal(12,2)) / cast(sum(volume) as decimal(12,2)) as decimal(15,2)) as mkt_share +from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'AMERICA' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'ECONOMY ANODIZED STEEL' + ) as all_nations +group by + o_year +order by + o_year; +---- +1995 0.03 +1996 0.02 diff --git a/datafusion/sqllogictest/test_files/tpch/answers/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/answers/q9.slt.part new file mode 100644 index 000000000000..71ae90119b51 --- /dev/null +++ b/datafusion/sqllogictest/test_files/tpch/answers/q9.slt.part @@ -0,0 +1,63 @@ + +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at + +# http://www.apache.org/licenses/LICENSE-2.0 + +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +query TIR +select + nation, + o_year, + sum(amount) as sum_profit +from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%green%' + ) as profit +group by + nation, + o_year +order by + nation, + o_year desc +limit 10; +---- +ALGERIA 1998 2321785.3682 +ALGERIA 1997 3685016.8589 +ALGERIA 1996 4276597.4253 +ALGERIA 1995 4418370.4154 +ALGERIA 1994 3864849.9521 +ALGERIA 1993 3541051.3865 +ALGERIA 1992 4310013.3482 +ARGENTINA 1998 2685983.8005 +ARGENTINA 1997 4242147.8124 +ARGENTINA 1996 3907867.0103 diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part similarity index 85% rename from datafusion/sqllogictest/test_files/tpch/q1.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part index 4d4323e93e9e..c60848cb56c1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q1.slt.part @@ -57,32 +57,4 @@ physical_plan 08)--------------ProjectionExec: expr=[l_extendedprice@1 * (Some(1),20,0 - l_discount@2) as __common_expr_1, l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_tax@3 as l_tax, l_returnflag@4 as l_returnflag, l_linestatus@5 as l_linestatus] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------FilterExec: l_shipdate@6 <= 1998-09-02, projection=[l_quantity@0, l_extendedprice@1, l_discount@2, l_tax@3, l_returnflag@4, l_linestatus@5] -11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false - -query TTRRRRRRRI -select - l_returnflag, - l_linestatus, - sum(l_quantity) as sum_qty, - sum(l_extendedprice) as sum_base_price, - sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, - sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, - avg(l_quantity) as avg_qty, - avg(l_extendedprice) as avg_price, - avg(l_discount) as avg_disc, - count(*) as count_order -from - lineitem -where - l_shipdate <= date '1998-09-02' -group by - l_returnflag, - l_linestatus -order by - l_returnflag, - l_linestatus; ----- -A F 3774200 5320753880.69 5054096266.6828 5256751331.449234 25.537587 36002.123829 0.050144 147790 -N F 95257 133737795.84 127132372.6512 132286291.229445 25.300664 35521.326916 0.049394 3765 -N O 7459297 10512270008.9 9986238338.3847 10385578376.585467 25.545537 36000.924688 0.050095 292000 -R F 3785523 5337950526.47 5071818532.942 5274405503.049367 25.525943 35994.029214 0.049989 148301 +11)--------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], has_header=false \ No newline at end of file diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part similarity index 79% rename from datafusion/sqllogictest/test_files/tpch/q10.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part index 73593a470c9a..e4376437ad4f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q10.slt.part @@ -104,50 +104,3 @@ physical_plan 33)--------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 34)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 35)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - - -query ITRRTTTT -select - c_custkey, - c_name, - sum(l_extendedprice * (1 - l_discount)) as revenue, - c_acctbal, - n_name, - c_address, - c_phone, - c_comment -from - customer, - orders, - lineitem, - nation -where - c_custkey = o_custkey - and l_orderkey = o_orderkey - and o_orderdate >= date '1993-10-01' - and o_orderdate < date '1994-01-01' - and l_returnflag = 'R' - and c_nationkey = n_nationkey -group by - c_custkey, - c_name, - c_acctbal, - c_phone, - n_name, - c_address, - c_comment -order by - revenue desc -limit 10; ----- -8242 Customer#000008242 622786.7297 6322.09 ETHIOPIA P2n4nJhy,UqSo2s43YfSvYJDZ6lk 15-792-676-1184 slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i -7714 Customer#000007714 557400.3053 9799.98 IRAN SnnIGB,SkmnWpX3 20-922-418-6024 arhorses according to the blithely express re -11032 Customer#000011032 512500.9641 8496.93 UNITED KINGDOM WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly 33-102-772-3533 posits-- furiously ironic accounts are again -2455 Customer#000002455 481592.4053 2070.99 GERMANY RVn1ZSRtLqPlJLIZxvpmsbgC02 17-946-225-9977 al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro -12106 Customer#000012106 479414.2133 5342.11 UNITED STATES wth3twOmu6vy 34-905-346-4472 ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. -8530 Customer#000008530 457855.9467 9734.95 MOROCCO GMQyte94oDM7eD7exnkj 4hH9yq3 25-736-932-5850 slyly asymptotes. quickly final deposits in -13984 Customer#000013984 446316.5104 3482.28 IRAN qZXwuapCHvxbX 20-981-264-2952 y unusual courts could wake furiously -1966 Customer#000001966 444059.0382 1937.72 ALGERIA jPv1 UHra5JLALR5Isci5u0636RoAu7t vH 10-973-269-8886 the blithely even accounts. final deposits cajole around the blithely final packages. -11026 Customer#000011026 417913.4142 7738.76 ALGERIA XorIktoJOAEJkpNNMx 10-184-163-4632 ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in -8501 Customer#000008501 412797.51 6906.7 ARGENTINA 776af4rOa mZ66hczs 11-317-552-5840 y final deposits after the fluffily even accounts are slyly final, regular diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part similarity index 93% rename from datafusion/sqllogictest/test_files/tpch/q11.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part index adaf391de0a2..80f9919cb46b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q11.slt.part @@ -124,45 +124,3 @@ physical_plan 49)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 50)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - -query IR -select - ps_partkey, - sum(ps_supplycost * ps_availqty) as value -from - partsupp, - supplier, - nation -where - ps_suppkey = s_suppkey - and s_nationkey = n_nationkey - and n_name = 'GERMANY' -group by - ps_partkey having - sum(ps_supplycost * ps_availqty) > ( - select - sum(ps_supplycost * ps_availqty) * 0.0001 - from - partsupp, - supplier, - nation - where - ps_suppkey = s_suppkey - and s_nationkey = n_nationkey - and n_name = 'GERMANY' - ) -order by - value desc -limit 10; ----- -12098 16227681.21 -5134 15709338.52 -13334 15023662.41 -17052 14351644.2 -3452 14070870.14 -12552 13332469.18 -1084 13170428.29 -5797 13038622.72 -12633 12892561.61 -403 12856217.34 diff --git a/datafusion/sqllogictest/test_files/tpch/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part similarity index 89% rename from datafusion/sqllogictest/test_files/tpch/q12.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part index b0d0baba90b0..34e5dd69f410 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q12.slt.part @@ -76,40 +76,3 @@ physical_plan 15)------------------CoalesceBatchesExec: target_batch_size=8192 16)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 17)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_orderpriority], has_header=false - - - -query TII -select - l_shipmode, - sum(case - when o_orderpriority = '1-URGENT' - or o_orderpriority = '2-HIGH' - then 1 - else 0 - end) as high_line_count, - sum(case - when o_orderpriority <> '1-URGENT' - and o_orderpriority <> '2-HIGH' - then 1 - else 0 - end) as low_line_count -from - lineitem - join - orders - on - l_orderkey = o_orderkey -where - l_shipmode in ('MAIL', 'SHIP') - and l_commitdate < l_receiptdate - and l_shipdate < l_commitdate - and l_receiptdate >= date '1994-01-01' - and l_receiptdate < date '1995-01-01' -group by - l_shipmode -order by - l_shipmode; ----- -MAIL 647 945 -SHIP 620 943 diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part similarity index 89% rename from datafusion/sqllogictest/test_files/tpch/q13.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part index 2a9fb12a31c2..2667bb23a459 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q13.slt.part @@ -73,39 +73,3 @@ physical_plan 18)--------------------------CoalesceBatchesExec: target_batch_size=8192 19)----------------------------FilterExec: o_comment@2 NOT LIKE %special%requests%, projection=[o_orderkey@0, o_custkey@1] 20)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:0..4223281], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:4223281..8446562], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:8446562..12669843], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/orders.tbl:12669843..16893122]]}, projection=[o_orderkey, o_custkey, o_comment], has_header=false - - - -query II -select - c_count, - count(*) as custdist -from - ( - select - c_custkey, - count(o_orderkey) - from - customer left outer join orders on - c_custkey = o_custkey - and o_comment not like '%special%requests%' - group by - c_custkey - ) as c_orders (c_custkey, c_count) -group by - c_count -order by - custdist desc, - c_count desc -limit 10; ----- -0 5000 -10 665 -9 657 -11 621 -12 567 -8 564 -13 492 -18 482 -7 480 -20 456 diff --git a/datafusion/sqllogictest/test_files/tpch/q14.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part similarity index 91% rename from datafusion/sqllogictest/test_files/tpch/q14.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part index eee10cb3f8e2..96cfbb02d07c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q14.slt.part @@ -57,22 +57,3 @@ physical_plan 14)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 15)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 16)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false - - - -query R -select - 100.00 * sum(case - when p_type like 'PROMO%' - then l_extendedprice * (1 - l_discount) - else 0 - end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue -from - lineitem, - part -where - l_partkey = p_partkey - and l_shipdate >= date '1995-09-01' - and l_shipdate < date '1995-10-01'; ----- -16.283855689006 diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part similarity index 93% rename from datafusion/sqllogictest/test_files/tpch/q15.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part index 2374fd8430a4..4cf4c722eaf8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q15.slt.part @@ -104,38 +104,3 @@ physical_plan 31)----------------------------CoalesceBatchesExec: target_batch_size=8192 32)------------------------------FilterExec: l_shipdate@3 >= 1996-01-01 AND l_shipdate@3 < 1996-04-01, projection=[l_suppkey@0, l_extendedprice@1, l_discount@2] 33)--------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false - -query ITTTR -with revenue0 (supplier_no, total_revenue) as ( - select - l_suppkey, - sum(l_extendedprice * (1 - l_discount)) - from - lineitem - where - l_shipdate >= date '1996-01-01' - and l_shipdate < date '1996-01-01' + interval '3' month - group by - l_suppkey -) -select - s_suppkey, - s_name, - s_address, - s_phone, - total_revenue -from - supplier, - revenue0 -where - s_suppkey = supplier_no - and total_revenue = ( - select - max(total_revenue) - from - revenue0 - ) -order by - s_suppkey; ----- -677 Supplier#000000677 8mhrffG7D2WJBSQbOGstQ 23-290-639-3315 1614410.2928 diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part similarity index 88% rename from datafusion/sqllogictest/test_files/tpch/q16.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part index 6b2c2f7fdc3e..e9486c39f08c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q16.slt.part @@ -98,48 +98,3 @@ physical_plan 31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 32)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false - - -query TTII -select - p_brand, - p_type, - p_size, - count(distinct ps_suppkey) as supplier_cnt -from - partsupp, - part -where - p_partkey = ps_partkey - and p_brand <> 'Brand#45' - and p_type not like 'MEDIUM POLISHED%' - and p_size in (49, 14, 23, 45, 19, 3, 36, 9) - and ps_suppkey not in ( - select - s_suppkey - from - supplier - where - s_comment like '%Customer%Complaints%' -) -group by - p_brand, - p_type, - p_size -order by - supplier_cnt desc, - p_brand, - p_type, - p_size -limit 10; ----- -Brand#14 SMALL ANODIZED NICKEL 45 12 -Brand#22 SMALL BURNISHED BRASS 19 12 -Brand#25 PROMO POLISHED COPPER 14 12 -Brand#35 LARGE ANODIZED STEEL 45 12 -Brand#35 PROMO BRUSHED COPPER 9 12 -Brand#51 ECONOMY ANODIZED STEEL 9 12 -Brand#53 LARGE BRUSHED NICKEL 45 12 -Brand#11 ECONOMY POLISHED COPPER 14 8 -Brand#11 LARGE PLATED STEEL 23 8 -Brand#11 PROMO POLISHED STEEL 23 8 diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part similarity index 94% rename from datafusion/sqllogictest/test_files/tpch/q17.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part index 058bcb5f4962..f28a5ef54885 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q17.slt.part @@ -75,25 +75,3 @@ physical_plan 22)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[avg(lineitem.l_quantity)] 23)----------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity], has_header=false - - -query R -select - sum(l_extendedprice) / 7.0 as avg_yearly -from - lineitem, - part -where - p_partkey = l_partkey - and p_brand = 'Brand#23' - and p_container = 'MED BOX' - and l_quantity < ( - select - 0.2 * avg(l_quantity) - from - lineitem - where - l_partkey = p_partkey -); ----- -23512.752857142856 diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part similarity index 89% rename from datafusion/sqllogictest/test_files/tpch/q18.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part index c80352c5d36a..283de4dddfe3 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q18.slt.part @@ -98,45 +98,3 @@ physical_plan 29)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 30)--------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[sum(lineitem.l_quantity)] 31)----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_quantity], has_header=false - - - -query TIIDRR -select - c_name, - c_custkey, - o_orderkey, - o_orderdate, - o_totalprice, - sum(l_quantity) -from - customer, - orders, - lineitem -where - o_orderkey in ( - select - l_orderkey - from - lineitem - group by - l_orderkey having - sum(l_quantity) > 300 - ) - and c_custkey = o_custkey - and o_orderkey = l_orderkey -group by - c_name, - c_custkey, - o_orderkey, - o_orderdate, - o_totalprice -order by - o_totalprice desc, - o_orderdate; ----- -Customer#000001639 1639 502886 1994-04-12 456423.88 312 -Customer#000006655 6655 29158 1995-10-21 452805.02 305 -Customer#000014110 14110 565574 1995-09-24 425099.85 301 -Customer#000001775 1775 6882 1997-04-09 408368.1 303 -Customer#000011459 11459 551136 1993-05-19 386812.74 308 diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part similarity index 88% rename from datafusion/sqllogictest/test_files/tpch/q19.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part index 70465ea065a1..9eb1f8638292 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q19.slt.part @@ -81,44 +81,3 @@ physical_plan 15)------------------FilterExec: (p_brand@1 = Brand#12 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("SM CASE") }, Literal { value: Utf8("SM BOX") }, Literal { value: Utf8("SM PACK") }, Literal { value: Utf8("SM PKG") }]) AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("MED BAG") }, Literal { value: Utf8("MED BOX") }, Literal { value: Utf8("MED PKG") }, Literal { value: Utf8("MED PACK") }]) AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND Use p_container@3 IN (SET) ([Literal { value: Utf8("LG CASE") }, Literal { value: Utf8("LG BOX") }, Literal { value: Utf8("LG PACK") }, Literal { value: Utf8("LG PKG") }]) AND p_size@2 <= 15) AND p_size@2 >= 1 16)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 17)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false - - - -query R -select - sum(l_extendedprice* (1 - l_discount)) as revenue -from - lineitem, - part -where - ( - p_partkey = l_partkey - and p_brand = 'Brand#12' - and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') - and l_quantity >= 1 and l_quantity <= 1 + 10 - and p_size between 1 and 5 - and l_shipmode in ('AIR', 'AIR REG') - and l_shipinstruct = 'DELIVER IN PERSON' - ) - or - ( - p_partkey = l_partkey - and p_brand = 'Brand#23' - and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') - and l_quantity >= 10 and l_quantity <= 10 + 10 - and p_size between 1 and 10 - and l_shipmode in ('AIR', 'AIR REG') - and l_shipinstruct = 'DELIVER IN PERSON' - ) - or - ( - p_partkey = l_partkey - and p_brand = 'Brand#34' - and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') - and l_quantity >= 20 and l_quantity <= 20 + 10 - and p_size between 1 and 15 - and l_shipmode in ('AIR', 'AIR REG') - and l_shipinstruct = 'DELIVER IN PERSON' - ); ----- -168597.286 diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part similarity index 86% rename from datafusion/sqllogictest/test_files/tpch/q2.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part index 23ffa0d226b8..8fdbb6884d14 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q2.slt.part @@ -178,62 +178,3 @@ physical_plan 77)----------------------------------FilterExec: r_name@1 = EUROPE, projection=[r_regionkey@0] 78)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 79)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - - - -query RTTITTTT -select - s_acctbal, - s_name, - n_name, - p_partkey, - p_mfgr, - s_address, - s_phone, - s_comment -from - part, - supplier, - partsupp, - nation, - region -where - p_partkey = ps_partkey - and s_suppkey = ps_suppkey - and p_size = 15 - and p_type like '%BRASS' - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'EUROPE' - and ps_supplycost = ( - select - min(ps_supplycost) - from - partsupp, - supplier, - nation, - region - where - p_partkey = ps_partkey - and s_suppkey = ps_suppkey - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'EUROPE' -) -order by - s_acctbal desc, - n_name, - s_name, - p_partkey -limit 10; ----- -9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily -9508.37 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T -9508.37 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T -9453.01 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs -9453.01 Supplier#000000802 ROMANIA 13275 Manufacturer#4 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs -9192.1 Supplier#000000115 UNITED KINGDOM 13325 Manufacturer#1 nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV 33-597-248-1220 es across the carefully express accounts boost caref -9032.15 Supplier#000000959 GERMANY 4958 Manufacturer#4 8grA EHBnwOZhO 17-108-642-3106 nding dependencies nag furiou -8702.02 Supplier#000000333 RUSSIA 11810 Manufacturer#3 MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH 32-508-202-6136 oss the deposits cajole carefully even pinto beans. regular foxes detect alo -8615.5 Supplier#000000812 FRANCE 10551 Manufacturer#2 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref -8615.5 Supplier#000000812 FRANCE 13811 Manufacturer#4 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part similarity index 87% rename from datafusion/sqllogictest/test_files/tpch/q20.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part index 177e38e51ca4..e3918ba62b02 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q20.slt.part @@ -125,53 +125,3 @@ physical_plan 42)----------------------------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01, projection=[l_partkey@0, l_suppkey@1, l_quantity@2] 43)------------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false - - -query TT -select - s_name, - s_address -from - supplier, - nation -where - s_suppkey in ( - select - ps_suppkey - from - partsupp - where - ps_partkey in ( - select - p_partkey - from - part - where - p_name like 'forest%' - ) - and ps_availqty > ( - select - 0.5 * sum(l_quantity) - from - lineitem - where - l_partkey = ps_partkey - and l_suppkey = ps_suppkey - and l_shipdate >= date '1994-01-01' - and l_shipdate < date '1994-01-01' + interval '1' year - ) - ) - and s_nationkey = n_nationkey - and n_name = 'CANADA' -order by - s_name; ----- -Supplier#000000157 ,mEGorBfVIm -Supplier#000000197 YC2Acon6kjY3zj3Fbxs2k4Vdf7X0cd2F -Supplier#000000287 7a9SP7qW5Yku5PvSg -Supplier#000000378 FfbhyCxWvcPrO8ltp9 -Supplier#000000530 0qwCMwobKY OcmLyfRXlagA8ukENJv, -Supplier#000000555 TfB,a5bfl3Ah 3Z 74GqnNs6zKVGM -Supplier#000000557 jj0wUYh9K3fG5Jhdhrkuy ,4 -Supplier#000000729 pqck2ppy758TQpZCUAjPvlU55K3QjfL7Bi -Supplier#000000935 ij98czM 2KzWe7dDTOxB8sq0UfCdvrX diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part similarity index 85% rename from datafusion/sqllogictest/test_files/tpch/q21.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part index 93dcd4c68052..939e01b1784e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q21.slt.part @@ -141,94 +141,3 @@ physical_plan 49)----------------------CoalesceBatchesExec: target_batch_size=8192 50)------------------------FilterExec: l_receiptdate@3 > l_commitdate@2, projection=[l_orderkey@0, l_suppkey@1] 51)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], has_header=false - - - -query TI -select - s_name, - count(*) as numwait -from - supplier, - lineitem l1, - orders, - nation -where - s_suppkey = l1.l_suppkey - and o_orderkey = l1.l_orderkey - and o_orderstatus = 'F' - and l1.l_receiptdate > l1.l_commitdate - and exists ( - select - * - from - lineitem l2 - where - l2.l_orderkey = l1.l_orderkey - and l2.l_suppkey <> l1.l_suppkey - ) - and not exists ( - select - * - from - lineitem l3 - where - l3.l_orderkey = l1.l_orderkey - and l3.l_suppkey <> l1.l_suppkey - and l3.l_receiptdate > l3.l_commitdate - ) - and s_nationkey = n_nationkey - and n_name = 'SAUDI ARABIA' -group by - s_name -order by - numwait desc, - s_name; ----- -Supplier#000000445 16 -Supplier#000000825 16 -Supplier#000000709 15 -Supplier#000000762 15 -Supplier#000000357 14 -Supplier#000000399 14 -Supplier#000000496 14 -Supplier#000000977 13 -Supplier#000000144 12 -Supplier#000000188 12 -Supplier#000000415 12 -Supplier#000000472 12 -Supplier#000000633 12 -Supplier#000000708 12 -Supplier#000000889 12 -Supplier#000000380 11 -Supplier#000000602 11 -Supplier#000000659 11 -Supplier#000000821 11 -Supplier#000000929 11 -Supplier#000000262 10 -Supplier#000000460 10 -Supplier#000000486 10 -Supplier#000000669 10 -Supplier#000000718 10 -Supplier#000000778 10 -Supplier#000000167 9 -Supplier#000000578 9 -Supplier#000000673 9 -Supplier#000000687 9 -Supplier#000000074 8 -Supplier#000000565 8 -Supplier#000000648 8 -Supplier#000000918 8 -Supplier#000000427 7 -Supplier#000000503 7 -Supplier#000000610 7 -Supplier#000000670 7 -Supplier#000000811 7 -Supplier#000000114 6 -Supplier#000000379 6 -Supplier#000000436 6 -Supplier#000000500 6 -Supplier#000000660 6 -Supplier#000000788 6 -Supplier#000000846 6 -Supplier#000000920 4 diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part similarity index 87% rename from datafusion/sqllogictest/test_files/tpch/q22.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part index 2955748160ea..a55d1e8fdfb1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q22.slt.part @@ -103,51 +103,3 @@ physical_plan 27)----------------------------FilterExec: c_acctbal@1 > Some(0),15,2 AND Use substr(c_phone@0, 1, 2) IN (SET) ([Literal { value: Utf8("13") }, Literal { value: Utf8("31") }, Literal { value: Utf8("23") }, Literal { value: Utf8("29") }, Literal { value: Utf8("30") }, Literal { value: Utf8("18") }, Literal { value: Utf8("17") }]), projection=[c_acctbal@1] 28)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 29)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false - - -query TIR -select - cntrycode, - count(*) as numcust, - sum(c_acctbal) as totacctbal -from - ( - select - substring(c_phone from 1 for 2) as cntrycode, - c_acctbal - from - customer - where - substring(c_phone from 1 for 2) in - ('13', '31', '23', '29', '30', '18', '17') - and c_acctbal > ( - select - avg(c_acctbal) - from - customer - where - c_acctbal > 0.00 - and substring(c_phone from 1 for 2) in - ('13', '31', '23', '29', '30', '18', '17') - ) - and not exists ( - select - * - from - orders - where - o_custkey = c_custkey - ) - ) as custsale -group by - cntrycode -order by - cntrycode; ----- -13 94 714035.05 -17 96 722560.15 -18 99 738012.52 -23 93 708285.25 -29 85 632693.46 -30 87 646748.02 -31 87 647372.5 diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part similarity index 89% rename from datafusion/sqllogictest/test_files/tpch/q3.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part index 289e9c7732bb..0f35239bfd9d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q3.slt.part @@ -87,41 +87,3 @@ physical_plan 27)----------------------CoalesceBatchesExec: target_batch_size=8192 28)------------------------FilterExec: l_shipdate@3 > 1995-03-15, projection=[l_orderkey@0, l_extendedprice@1, l_discount@2] 29)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], has_header=false - - - -query IRDI -select - l_orderkey, - sum(l_extendedprice * (1 - l_discount)) as revenue, - o_orderdate, - o_shippriority -from - customer, - orders, - lineitem -where - c_mktsegment = 'BUILDING' - and c_custkey = o_custkey - and l_orderkey = o_orderkey - and o_orderdate < date '1995-03-15' - and l_shipdate > date '1995-03-15' -group by - l_orderkey, - o_orderdate, - o_shippriority -order by - revenue desc, - o_orderdate - limit 10; ----- -223140 355369.0698 1995-03-14 0 -584291 354494.7318 1995-02-21 0 -405063 353125.4577 1995-03-03 0 -573861 351238.277 1995-03-09 0 -554757 349181.7426 1995-03-14 0 -506021 321075.581 1995-03-10 0 -121604 318576.4154 1995-03-07 0 -108514 314967.0754 1995-02-20 0 -462502 312604.542 1995-03-08 0 -178727 309728.9306 1995-02-25 0 diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part similarity index 90% rename from datafusion/sqllogictest/test_files/tpch/q4.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part index a68b745c366c..693d37a5b53d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q4.slt.part @@ -72,34 +72,3 @@ physical_plan 17)----------------------CoalesceBatchesExec: target_batch_size=8192 18)------------------------FilterExec: l_receiptdate@2 > l_commitdate@1, projection=[l_orderkey@0] 19)--------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_orderkey, l_commitdate, l_receiptdate], has_header=false - - - -query TI -select - o_orderpriority, - count(*) as order_count -from - orders -where - o_orderdate >= '1993-07-01' - and o_orderdate < date '1993-07-01' + interval '3' month - and exists ( - select - * - from - lineitem - where - l_orderkey = o_orderkey - and l_commitdate < l_receiptdate - ) -group by - o_orderpriority -order by - o_orderpriority; ----- -1-URGENT 999 -2-HIGH 997 -3-MEDIUM 1031 -4-NOT SPECIFIED 989 -5-LOW 1077 diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part similarity index 94% rename from datafusion/sqllogictest/test_files/tpch/q5.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part index e59daf4943e8..0c03e0498d64 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q5.slt.part @@ -118,37 +118,3 @@ physical_plan 49)------------------------FilterExec: r_name@1 = ASIA, projection=[r_regionkey@0] 50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - - - -query TR -select - n_name, - sum(l_extendedprice * (1 - l_discount)) as revenue -from - customer, - orders, - lineitem, - supplier, - nation, - region -where - c_custkey = o_custkey - and l_orderkey = o_orderkey - and l_suppkey = s_suppkey - and c_nationkey = s_nationkey - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'ASIA' - and o_orderdate >= date '1994-01-01' - and o_orderdate < date '1995-01-01' -group by - n_name -order by - revenue desc; ----- -CHINA 7822103 -INDIA 6376121.5085 -JAPAN 6000077.2184 -INDONESIA 5580475.4027 -VIETNAM 4497840.5466 diff --git a/datafusion/sqllogictest/test_files/tpch/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part similarity index 92% rename from datafusion/sqllogictest/test_files/tpch/q6.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part index 548d26972f14..6c818a1e2f42 100644 --- a/datafusion/sqllogictest/test_files/tpch/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q6.slt.part @@ -41,18 +41,3 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: l_shipdate@3 >= 1994-01-01 AND l_shipdate@3 < 1995-01-01 AND l_discount@2 >= Some(5),15,2 AND l_discount@2 <= Some(7),15,2 AND l_quantity@0 < Some(2400),15,2, projection=[l_extendedprice@1, l_discount@2] 07)------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], has_header=false - - - -query R -select - sum(l_extendedprice * l_discount) as revenue -from - lineitem -where - l_shipdate >= date '1994-01-01' - and l_shipdate < date '1995-01-01' - and l_discount between 0.06 - 0.01 and 0.06 + 0.01 - and l_quantity < 24; ----- -11803420.2534 diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part similarity index 91% rename from datafusion/sqllogictest/test_files/tpch/q7.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part index a16af4710478..023af0ea9fbd 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q7.slt.part @@ -138,51 +138,3 @@ physical_plan 52)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE 53)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 54)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - - -query TTRR -select - supp_nation, - cust_nation, - l_year, - sum(volume) as revenue -from - ( - select - n1.n_name as supp_nation, - n2.n_name as cust_nation, - extract(year from l_shipdate) as l_year, - l_extendedprice * (1 - l_discount) as volume - from - supplier, - lineitem, - orders, - customer, - nation n1, - nation n2 - where - s_suppkey = l_suppkey - and o_orderkey = l_orderkey - and c_custkey = o_custkey - and s_nationkey = n1.n_nationkey - and c_nationkey = n2.n_nationkey - and ( - (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY') - or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE') - ) - and l_shipdate between date '1995-01-01' and date '1996-12-31' - ) as shipping -group by - supp_nation, - cust_nation, - l_year -order by - supp_nation, - cust_nation, - l_year; ----- -FRANCE GERMANY 1995 4637235.1501 -FRANCE GERMANY 1996 5224779.5736 -GERMANY FRANCE 1995 6232818.7037 -GERMANY FRANCE 1996 5557312.1121 diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part similarity index 93% rename from datafusion/sqllogictest/test_files/tpch/q8.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part index fd5773438466..2bcab40dc985 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q8.slt.part @@ -160,47 +160,3 @@ physical_plan 68)--------------------------FilterExec: r_name@1 = AMERICA, projection=[r_regionkey@0] 69)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 70)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - - - -query RR -select - o_year, - cast(cast(sum(case - when nation = 'BRAZIL' then volume - else 0 - end) as decimal(12,2)) / cast(sum(volume) as decimal(12,2)) as decimal(15,2)) as mkt_share -from - ( - select - extract(year from o_orderdate) as o_year, - l_extendedprice * (1 - l_discount) as volume, - n2.n_name as nation - from - part, - supplier, - lineitem, - orders, - customer, - nation n1, - nation n2, - region - where - p_partkey = l_partkey - and s_suppkey = l_suppkey - and l_orderkey = o_orderkey - and o_custkey = c_custkey - and c_nationkey = n1.n_nationkey - and n1.n_regionkey = r_regionkey - and r_name = 'AMERICA' - and s_nationkey = n2.n_nationkey - and o_orderdate between date '1995-01-01' and date '1996-12-31' - and p_type = 'ECONOMY ANODIZED STEEL' - ) as all_nations -group by - o_year -order by - o_year; ----- -1995 0.03 -1996 0.02 diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part similarity index 91% rename from datafusion/sqllogictest/test_files/tpch/q9.slt.part rename to datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part index c4910beb842b..4a288893da95 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/plans/q9.slt.part @@ -124,51 +124,3 @@ physical_plan 47)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 48)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 49)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - - -query TRR -select - nation, - o_year, - sum(amount) as sum_profit -from - ( - select - n_name as nation, - extract(year from o_orderdate) as o_year, - l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount - from - part, - supplier, - lineitem, - partsupp, - orders, - nation - where - s_suppkey = l_suppkey - and ps_suppkey = l_suppkey - and ps_partkey = l_partkey - and p_partkey = l_partkey - and o_orderkey = l_orderkey - and s_nationkey = n_nationkey - and p_name like '%green%' - ) as profit -group by - nation, - o_year -order by - nation, - o_year desc -limit 10; ----- -ALGERIA 1998 2321785.3682 -ALGERIA 1997 3685016.8589 -ALGERIA 1996 4276597.4253 -ALGERIA 1995 4418370.4154 -ALGERIA 1994 3864849.9521 -ALGERIA 1993 3541051.3865 -ALGERIA 1992 4310013.3482 -ARGENTINA 1998 2685983.8005 -ARGENTINA 1997 4242147.8124 -ARGENTINA 1996 3907867.0103 diff --git a/datafusion/sqllogictest/test_files/tpch/tpch.slt b/datafusion/sqllogictest/test_files/tpch/tpch.slt index 9a876acf088f..127152daa580 100644 --- a/datafusion/sqllogictest/test_files/tpch/tpch.slt +++ b/datafusion/sqllogictest/test_files/tpch/tpch.slt @@ -18,5 +18,14 @@ # under the License. include ./create_tables.slt.part -include ./q*.slt.part +include ./plans/q*.slt.part +include ./answers/q*.slt.part + +# test answers with sort merge join +statement ok +set datafusion.optimizer.prefer_hash_join = false; + +include ./answers/q*.slt.part + include ./drop_tables.slt.part + diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index b5e82f613a46..d94780744db9 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -766,13 +766,13 @@ SELECT NULL WHERE FALSE; query error DataFusion error: type_coercion\ncaused by\nError during planning: Incompatible inputs for Union: Previous inputs were of type List(.*), but got incompatible type List(.*) on column 'x' SELECT make_array(2) x UNION ALL SELECT make_array(now()) x; -query ? +query ? rowsort select make_array(arrow_cast(2, 'UInt8')) x UNION ALL SELECT make_array(arrow_cast(-2, 'Int8')) x; ---- [-2] [2] -query ? +query ? rowsort select make_array(make_array(1)) x UNION ALL SELECT make_array(arrow_cast(make_array(-1), 'LargeList(Int8)')) x; ---- [[-1]] diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index 2e1b8b87cc42..d409e0902f7e 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -859,3 +859,30 @@ query I select count(*) from (select unnest(range(0, 100000)) id) t inner join (select unnest(range(0, 100000)) id) t1 on t.id = t1.id; ---- 100000 + + +## Unnest in subquery +query IIII +with t as ( + select + left1, + width1, + min(column3) as min_height + from + unnest_table a + cross join unnest(ARRAY[1,2,3,4,5,6,7,8,9,10]) as t(left1) + cross join unnest(ARRAY[1,2,3,4,5,6,7,8,9,10]) as t1(width1) + where + left1 + width1 - 1 <= 10 + and column3 between left1 and left1 + width1 - 1 + group by + left1, width1 +) +select + left1, width1, min_height, min_height * width1 as area +from t +where min_height * width1 = ( + select max(min_height * width1) from t +) +---- +4 7 4 28 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 8e3559a32684..6c48ac68ab6b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -5046,3 +5046,85 @@ statement ok DROP TABLE t1; ## end test handle NULL of lead + +# row_number on bool column +query II +select b, row_number() over (order by a) from (select TRUE as a, 1 as b); +---- +1 1 + +# test window functions on boolean columns +query T +create table t1 (id int, bool_col boolean) as values + (1, true), + (2, false), + (3, true), + (4, false); + +query I +select row_number() over (order by bool_col) from t1; +---- +1 +2 +3 +4 + +query I +select rank() over (order by bool_col) from t1; +---- +1 +1 +3 +3 + +query I +select dense_rank() over (order by bool_col) from t1; +---- +1 +1 +2 +2 + +query R +select cume_dist() over (order by bool_col) from t1; +---- +0.5 +0.5 +1 +1 + +query R +select percent_rank() over (order by bool_col) from t1; +---- +0 +0 +0.666666666667 +0.666666666667 + +query I +select ntile(2) over (order by bool_col) from t1; +---- +1 +1 +2 +2 + +query IIIRRI +select + row_number() over (order by bool_col) as row_num, + rank() over (order by bool_col) as rank, + dense_rank() over (order by bool_col) as dense_rank, + cume_dist() over (order by bool_col) as cume_dist, + percent_rank() over (order by bool_col) as percent_rank, + ntile(2) over (order by bool_col) as ntile +from t1 +order by id; +---- +3 3 2 1 0.666666666667 2 +1 1 1 0.5 0 1 +4 3 2 1 0.666666666667 2 +2 1 1 0.5 0 1 + +statement ok +drop table t1; + diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 61cdf3e91e3c..79e5a3206cad 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -42,7 +42,7 @@ object_store = { workspace = true } pbjson-types = "0.7" # TODO use workspace version prost = "0.13" -substrait = { version = "0.47", features = ["serde"] } +substrait = { version = "0.49", features = ["serde"] } url = { workspace = true } [dev-dependencies] diff --git a/datafusion/substrait/LICENSE.txt b/datafusion/substrait/LICENSE.txt new file mode 120000 index 000000000000..1ef648f64b34 --- /dev/null +++ b/datafusion/substrait/LICENSE.txt @@ -0,0 +1 @@ +../../LICENSE.txt \ No newline at end of file diff --git a/datafusion/substrait/NOTICE.txt b/datafusion/substrait/NOTICE.txt new file mode 120000 index 000000000000..fb051c92b10b --- /dev/null +++ b/datafusion/substrait/NOTICE.txt @@ -0,0 +1 @@ +../../NOTICE.txt \ No newline at end of file diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 77e9eb81f546..a9e411e35ae8 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -31,7 +31,7 @@ use datafusion::logical_expr::expr::{Exists, InSubquery, Sort}; use datafusion::logical_expr::{ Aggregate, BinaryExpr, Case, EmptyRelation, Expr, ExprSchemable, LogicalPlan, - Operator, Projection, SortExpr, Values, + Operator, Projection, SortExpr, TryCast, Values, }; use substrait::proto::aggregate_rel::Grouping; use substrait::proto::expression::subquery::set_predicate::PredicateOp; @@ -71,6 +71,7 @@ use datafusion::{ use std::collections::HashSet; use std::sync::Arc; use substrait::proto::exchange_rel::ExchangeKind; +use substrait::proto::expression::cast::FailureBehavior::ReturnNull; use substrait::proto::expression::literal::user_defined::Val; use substrait::proto::expression::literal::{ interval_day_to_second, IntervalCompound, IntervalDayToSecond, IntervalYearToMonth, @@ -1147,6 +1148,7 @@ fn retrieve_rel_common(rel: &Rel) -> Option<&RelCommon> { RelType::Window(w) => w.common.as_ref(), RelType::Exchange(e) => e.common.as_ref(), RelType::Expand(e) => e.common.as_ref(), + RelType::Update(_) => None, }, } } @@ -1681,8 +1683,8 @@ pub async fn from_substrait_rex( Ok(Expr::Literal(scalar_value)) } Some(RexType::Cast(cast)) => match cast.as_ref().r#type.as_ref() { - Some(output_type) => Ok(Expr::Cast(Cast::new( - Box::new( + Some(output_type) => { + let input_expr = Box::new( from_substrait_rex( state, cast.as_ref().input.as_ref().unwrap().as_ref(), @@ -1690,9 +1692,15 @@ pub async fn from_substrait_rex( extensions, ) .await?, - ), - from_substrait_type_without_names(output_type, extensions)?, - ))), + ); + let data_type = + from_substrait_type_without_names(output_type, extensions)?; + if cast.failure_behavior() == ReturnNull { + Ok(Expr::TryCast(TryCast::new(input_expr, data_type))) + } else { + Ok(Expr::Cast(Cast::new(input_expr, data_type))) + } + } None => substrait_err!("Cast expression without output type is not allowed"), }, Some(RexType::WindowFunction(window)) => { diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 29019dfd74f3..c432a144127c 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -23,7 +23,7 @@ use substrait::proto::expression_reference::ExprType; use datafusion::arrow::datatypes::{Field, IntervalUnit}; use datafusion::logical_expr::{ - Distinct, FetchType, Like, Partitioning, SkipType, WindowFrameUnits, + Distinct, FetchType, Like, Partitioning, SkipType, TryCast, WindowFrameUnits, }; use datafusion::{ arrow::datatypes::{DataType, TimeUnit}, @@ -55,6 +55,7 @@ use datafusion::logical_expr::{expr, Between, JoinConstraint, LogicalPlan, Opera use datafusion::prelude::Expr; use pbjson_types::Any as ProtoAny; use substrait::proto::exchange_rel::{ExchangeKind, RoundRobin, ScatterFields}; +use substrait::proto::expression::cast::FailureBehavior; use substrait::proto::expression::literal::interval_day_to_second::PrecisionMode; use substrait::proto::expression::literal::map::KeyValue; use substrait::proto::expression::literal::{ @@ -1182,23 +1183,36 @@ pub fn to_substrait_rex( rex_type: Some(RexType::IfThen(Box::new(IfThen { ifs, r#else }))), }) } - Expr::Cast(Cast { expr, data_type }) => { - Ok(Expression { - rex_type: Some(RexType::Cast(Box::new( - substrait::proto::expression::Cast { - r#type: Some(to_substrait_type(data_type, true)?), - input: Some(Box::new(to_substrait_rex( - state, - expr, - schema, - col_ref_offset, - extensions, - )?)), - failure_behavior: 0, // FAILURE_BEHAVIOR_UNSPECIFIED - }, - ))), - }) - } + Expr::Cast(Cast { expr, data_type }) => Ok(Expression { + rex_type: Some(RexType::Cast(Box::new( + substrait::proto::expression::Cast { + r#type: Some(to_substrait_type(data_type, true)?), + input: Some(Box::new(to_substrait_rex( + state, + expr, + schema, + col_ref_offset, + extensions, + )?)), + failure_behavior: FailureBehavior::ThrowException.into(), + }, + ))), + }), + Expr::TryCast(TryCast { expr, data_type }) => Ok(Expression { + rex_type: Some(RexType::Cast(Box::new( + substrait::proto::expression::Cast { + r#type: Some(to_substrait_type(data_type, true)?), + input: Some(Box::new(to_substrait_rex( + state, + expr, + schema, + col_ref_offset, + extensions, + )?)), + failure_behavior: FailureBehavior::ReturnNull.into(), + }, + ))), + }), Expr::Literal(value) => to_substrait_literal_expr(value, extensions), Expr::Alias(Alias { expr, .. }) => { to_substrait_rex(state, expr, schema, col_ref_offset, extensions) diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index d03ab5182028..0141e6a08647 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -411,6 +411,16 @@ async fn implicit_cast() -> Result<()> { roundtrip("SELECT * FROM data WHERE a = b").await } +#[tokio::test] +async fn try_cast_decimal_to_int() -> Result<()> { + roundtrip("SELECT * FROM data WHERE a = TRY_CAST(b AS int)").await +} + +#[tokio::test] +async fn try_cast_decimal_to_string() -> Result<()> { + roundtrip("SELECT * FROM data WHERE a = TRY_CAST(b AS string)").await +} + #[tokio::test] async fn aggregate_case() -> Result<()> { assert_expected_plan( diff --git a/dev/release/README.md b/dev/release/README.md index cac8ea654f40..5dd9b4fc59fd 100644 --- a/dev/release/README.md +++ b/dev/release/README.md @@ -30,7 +30,7 @@ Patch releases are made on an adhoc basis, but we try and avoid them given the f - Once the PR is approved and merged, we tag the rc in the release branch, and release from the release branch - Bug fixes can be merged to the release branch and patch releases can be created from the release branch -#### How to add changes to `branch-*` branch? +#### How to backport (add changes) to `branch-*` branch If you would like to propose your change for inclusion in a release branch for a patch release: @@ -39,6 +39,16 @@ patch release: 1. Follow normal workflow to create PR to `main` branch and wait for its approval and merge. 1. After PR is squash merged to `main`, branch from most recent release branch (e.g. `branch-37`), cherry-pick the commit and create a PR targeting the release branch [example backport PR]. +For example, to backport commit `12345` from `main` to `branch-43`: + +```shell +git checkout branch-43 +git checkout -b backport_to_43 +git cherry-pick 12345 +git push -u +# make a PR as normal +``` + [example release issue]: https://github.com/apache/datafusion/issues/9904 [example backport pr]: https://github.com/apache/datafusion/pull/10123 diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index fe3990b90c3c..2044cfaa70fc 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -30,7 +30,8 @@ This page covers how to add UDFs to DataFusion. In particular, it covers how to | Aggregate | A function that takes a group of rows and returns a single value. | [simple_udaf.rs][3] | | Table | A function that takes parameters and returns a `TableProvider` to be used in an query plan. | [simple_udtf.rs][4] | -First we'll talk about adding an Scalar UDF end-to-end, then we'll talk about the differences between the different types of UDFs. +First we'll talk about adding an Scalar UDF end-to-end, then we'll talk about the differences between the different +types of UDFs. ## Adding a Scalar UDF @@ -40,12 +41,14 @@ an Arrow Array with the same number of rows as output. To create a Scalar UDF, you -1. Implement the `ScalarUDFImpl` trait to tell DataFusion about your function such as what types of arguments it takes and how to calculate the results. -2. Create a `ScalarUDF` and register it with `SessionContext::register_udf` so it can be invoked by name. +1. Implement the `ScalarUDFImpl` trait to tell DataFusion about your function such as what types of arguments it takes + and how to calculate the results. +2. Create a `ScalarUDF` and register it with `SessionContext::register_udf` so it can be invoked by name. In the following example, we will add a function takes a single i64 and returns a single i64 with 1 added to it: -For brevity, we'll skipped some error handling, but e.g. you may want to check that `args.len()` is the expected number of arguments. +For brevity, we'll skipped some error handling, but e.g. you may want to check that `args.len()` is the expected number +of arguments. ### Adding by `impl ScalarUDFImpl` @@ -77,20 +80,20 @@ impl ScalarUDFImpl for AddOne { fn name(&self) -> &str { "add_one" } fn signature(&self) -> &Signature { &self.signature } fn return_type(&self, args: &[DataType]) -> Result { - if !matches!(args.get(0), Some(&DataType::Int32)) { - return plan_err!("add_one only accepts Int32 arguments"); - } - Ok(DataType::Int32) + if !matches!(args.get(0), Some(&DataType::Int32)) { + return plan_err!("add_one only accepts Int32 arguments"); + } + Ok(DataType::Int32) } // The actual implementation would add one to the argument - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch(&self, args: &[ColumnarValue], _number_rows: usize) -> Result { let args = columnar_values_to_array(args)?; let i64s = as_int64_array(&args[0])?; let new_array = i64s - .iter() - .map(|array_elem| array_elem.map(|value| value + 1)) - .collect::(); + .iter() + .map(|array_elem| array_elem.map(|value| value + 1)) + .collect::(); Ok(Arc::new(new_array)) } } @@ -130,31 +133,34 @@ pub fn add_one(args: &[ColumnarValue]) -> Result { let i64s = as_int64_array(&args[0])?; let new_array = i64s - .iter() - .map(|array_elem| array_elem.map(|value| value + 1)) - .collect::(); + .iter() + .map(|array_elem| array_elem.map(|value| value + 1)) + .collect::(); Ok(Arc::new(new_array)) } ``` -This "works" in isolation, i.e. if you have a slice of `ArrayRef`s, you can call `add_one` and it will return a new `ArrayRef` with 1 added to each value. +This "works" in isolation, i.e. if you have a slice of `ArrayRef`s, you can call `add_one` and it will return a new +`ArrayRef` with 1 added to each value. ```rust let input = vec![Some(1), None, Some(3)]; let input = Arc::new(Int64Array::from(input)) as ArrayRef; -let result = add_one(&[input]).unwrap(); +let result = add_one( & [input]).unwrap(); let result = result.as_any().downcast_ref::().unwrap(); assert_eq!(result, &Int64Array::from(vec![Some(2), None, Some(4)])); ``` -The challenge however is that DataFusion doesn't know about this function. We need to register it with DataFusion so that it can be used in the context of a query. +The challenge however is that DataFusion doesn't know about this function. We need to register it with DataFusion so +that it can be used in the context of a query. #### Registering a Scalar UDF -To register a Scalar UDF, you need to wrap the function implementation in a [`ScalarUDF`] struct and then register it with the `SessionContext`. +To register a Scalar UDF, you need to wrap the function implementation in a [`ScalarUDF`] struct and then register it +with the `SessionContext`. DataFusion provides the [`create_udf`] and helper functions to make this easier. ```rust @@ -163,11 +169,11 @@ use datafusion::arrow::datatypes::DataType; use std::sync::Arc; let udf = create_udf( - "add_one", - vec![DataType::Int64], - Arc::new(DataType::Int64), - Volatility::Immutable, - Arc::new(add_one), +"add_one", +vec![DataType::Int64], +Arc::new(DataType::Int64), +Volatility::Immutable, +Arc::new(add_one), ); ``` @@ -179,9 +185,13 @@ let udf = create_udf( A few things to note: - The first argument is the name of the function. This is the name that will be used in SQL queries. -- The second argument is a vector of `DataType`s. This is the list of argument types that the function accepts. I.e. in this case, the function accepts a single `Int64` argument. +- The second argument is a vector of `DataType`s. This is the list of argument types that the function accepts. I.e. in + this case, the function accepts a single `Int64` argument. - The third argument is the return type of the function. I.e. in this case, the function returns an `Int64`. -- The fourth argument is the volatility of the function. In short, this is used to determine if the function's performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns the same value for the same input. A random number generator would be `Volatile` because it returns a different value for the same input. +- The fourth argument is the volatility of the function. In short, this is used to determine if the function's + performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns + the same value for the same input. A random number generator would be `Volatile` because it returns a different value + for the same input. - The fifth argument is the function implementation. This is the function that we defined above. That gives us a `ScalarUDF` that we can register with the `SessionContext`: @@ -199,12 +209,13 @@ At this point, you can use the `add_one` function in your query: ```rust let sql = "SELECT add_one(1)"; -let df = ctx.sql(&sql).await.unwrap(); +let df = ctx.sql( & sql).await.unwrap(); ``` ## Adding a Window UDF -Scalar UDFs are functions that take a row of data and return a single value. Window UDFs are similar, but they also have access to the rows around them. Access to the proximal rows is helpful, but adds some complexity to the implementation. +Scalar UDFs are functions that take a row of data and return a single value. Window UDFs are similar, but they also have +access to the rows around them. Access to the proximal rows is helpful, but adds some complexity to the implementation. For example, we will declare a user defined window function that computes a moving average. @@ -277,7 +288,8 @@ fn make_partition_evaluator() -> Result> { ### Registering a Window UDF -To register a Window UDF, you need to wrap the function implementation in a [`WindowUDF`] struct and then register it with the `SessionContext`. DataFusion provides the [`create_udwf`] helper functions to make this easier. +To register a Window UDF, you need to wrap the function implementation in a [`WindowUDF`] struct and then register it +with the `SessionContext`. DataFusion provides the [`create_udwf`] helper functions to make this easier. There is a lower level API with more functionality but is more complex, that is documented in [`advanced_udwf.rs`]. ```rust @@ -287,11 +299,11 @@ use std::sync::Arc; // here is where we define the UDWF. We also declare its signature: let smooth_it = create_udwf( - "smooth_it", - DataType::Float64, - Arc::new(DataType::Float64), - Volatility::Immutable, - Arc::new(make_partition_evaluator), +"smooth_it", +DataType::Float64, +Arc::new(DataType::Float64), +Volatility::Immutable, +Arc::new(make_partition_evaluator), ); ``` @@ -302,9 +314,13 @@ let smooth_it = create_udwf( The `create_udwf` has five arguments to check: - The first argument is the name of the function. This is the name that will be used in SQL queries. -- **The second argument** is the `DataType` of input array (attention: this is not a list of arrays). I.e. in this case, the function accepts `Float64` as argument. +- **The second argument** is the `DataType` of input array (attention: this is not a list of arrays). I.e. in this case, + the function accepts `Float64` as argument. - The third argument is the return type of the function. I.e. in this case, the function returns an `Float64`. -- The fourth argument is the volatility of the function. In short, this is used to determine if the function's performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns the same value for the same input. A random number generator would be `Volatile` because it returns a different value for the same input. +- The fourth argument is the volatility of the function. In short, this is used to determine if the function's + performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns + the same value for the same input. A random number generator would be `Volatile` because it returns a different value + for the same input. - **The fifth argument** is the function implementation. This is the function that we defined above. That gives us a `WindowUDF` that we can register with the `SessionContext`: @@ -319,7 +335,8 @@ ctx.register_udwf(smooth_it); At this point, you can use the `smooth_it` function in your query: -For example, if we have a [`cars.csv`](https://github.com/apache/datafusion/blob/main/datafusion/core/tests/data/cars.csv) whose contents like +For example, if we have a [ +`cars.csv`](https://github.com/apache/datafusion/blob/main/datafusion/core/tests/data/cars.csv) whose contents like ``` car,speed,time @@ -336,11 +353,11 @@ Then, we can query like below: use datafusion::datasource::file_format::options::CsvReadOptions; // register csv table first let csv_path = "cars.csv".to_string(); -ctx.register_csv("cars", &csv_path, CsvReadOptions::default().has_header(true)).await?; +ctx.register_csv("cars", & csv_path, CsvReadOptions::default ().has_header(true)).await?; // do query with smooth_it let df = ctx - .sql( - "SELECT \ +.sql( +"SELECT \ car, \ speed, \ smooth_it(speed) OVER (PARTITION BY car ORDER BY time) as smooth_speed,\ @@ -348,8 +365,8 @@ let df = ctx from cars \ ORDER BY \ car", - ) - .await?; +) +.await?; // print the results df.show().await?; ``` @@ -379,7 +396,8 @@ the output will be like: ## Adding an Aggregate UDF -Aggregate UDFs are functions that take a group of rows and return a single value. These are akin to SQL's `SUM` or `COUNT` functions. +Aggregate UDFs are functions that take a group of rows and return a single value. These are akin to SQL's `SUM` or +`COUNT` functions. For example, we will declare a single-type, single return type UDAF that computes the geometric mean. @@ -474,7 +492,8 @@ impl Accumulator for GeometricMean { ### registering an Aggregate UDF -To register a Aggregate UDF, you need to wrap the function implementation in a [`AggregateUDF`] struct and then register it with the `SessionContext`. DataFusion provides the [`create_udaf`] helper functions to make this easier. +To register a Aggregate UDF, you need to wrap the function implementation in a [`AggregateUDF`] struct and then register +it with the `SessionContext`. DataFusion provides the [`create_udaf`] helper functions to make this easier. There is a lower level API with more functionality but is more complex, that is documented in [`advanced_udaf.rs`]. ```rust @@ -484,17 +503,17 @@ use std::sync::Arc; // here is where we define the UDAF. We also declare its signature: let geometric_mean = create_udaf( - // the name; used to represent it in plan descriptions and in the registry, to use in SQL. - "geo_mean", - // the input type; DataFusion guarantees that the first entry of `values` in `update` has this type. - vec![DataType::Float64], - // the return type; DataFusion expects this to match the type returned by `evaluate`. - Arc::new(DataType::Float64), - Volatility::Immutable, - // This is the accumulator factory; DataFusion uses it to create new accumulators. - Arc::new(|_| Ok(Box::new(GeometricMean::new()))), - // This is the description of the state. `state()` must match the types here. - Arc::new(vec![DataType::Float64, DataType::UInt32]), +// the name; used to represent it in plan descriptions and in the registry, to use in SQL. +"geo_mean", +// the input type; DataFusion guarantees that the first entry of `values` in `update` has this type. +vec![DataType::Float64], +// the return type; DataFusion expects this to match the type returned by `evaluate`. +Arc::new(DataType::Float64), +Volatility::Immutable, +// This is the accumulator factory; DataFusion uses it to create new accumulators. +Arc::new( | _ | Ok(Box::new(GeometricMean::new()))), +// This is the description of the state. `state()` must match the types here. +Arc::new(vec![DataType::Float64, DataType::UInt32]), ); ``` @@ -505,9 +524,13 @@ let geometric_mean = create_udaf( The `create_udaf` has six arguments to check: - The first argument is the name of the function. This is the name that will be used in SQL queries. -- The second argument is a vector of `DataType`s. This is the list of argument types that the function accepts. I.e. in this case, the function accepts a single `Float64` argument. +- The second argument is a vector of `DataType`s. This is the list of argument types that the function accepts. I.e. in + this case, the function accepts a single `Float64` argument. - The third argument is the return type of the function. I.e. in this case, the function returns an `Int64`. -- The fourth argument is the volatility of the function. In short, this is used to determine if the function's performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns the same value for the same input. A random number generator would be `Volatile` because it returns a different value for the same input. +- The fourth argument is the volatility of the function. In short, this is used to determine if the function's + performance can be optimized in some situations. In this case, the function is `Immutable` because it always returns + the same value for the same input. A random number generator would be `Volatile` because it returns a different value + for the same input. - The fifth argument is the function implementation. This is the function that we defined above. - The sixth argument is the description of the state, which will by passed between execution stages. @@ -531,9 +554,14 @@ let df = ctx.sql("SELECT geo_mean(a) FROM t").await?; A User-Defined Table Function (UDTF) is a function that takes parameters and returns a `TableProvider`. -Because we're returning a `TableProvider`, in this example we'll use the `MemTable` data source to represent a table. This is a simple struct that holds a set of RecordBatches in memory and treats them as a table. In your case, this would be replaced with your own struct that implements `TableProvider`. +Because we're returning a `TableProvider`, in this example we'll use the `MemTable` data source to represent a table. +This is a simple struct that holds a set of RecordBatches in memory and treats them as a table. In your case, this would +be replaced with your own struct that implements `TableProvider`. -While this is a simple example for illustrative purposes, UDTFs have a lot of potential use cases. And can be particularly useful for reading data from external sources and interactive analysis. For example, see the [example][4] for a working example that reads from a CSV file. As another example, you could use the built-in UDTF `parquet_metadata` in the CLI to read the metadata from a Parquet file. +While this is a simple example for illustrative purposes, UDTFs have a lot of potential use cases. And can be +particularly useful for reading data from external sources and interactive analysis. For example, see the [example][4] +for a working example that reads from a CSV file. As another example, you could use the built-in UDTF `parquet_metadata` +in the CLI to read the metadata from a Parquet file. ```console > select filename, row_group_id, row_group_num_rows, row_group_bytes, stats_min, stats_max from parquet_metadata('./benchmarks/data/hits.parquet') where column_id = 17 limit 10; @@ -555,9 +583,12 @@ While this is a simple example for illustrative purposes, UDTFs have a lot of po ### Writing the UDTF -The simple UDTF used here takes a single `Int64` argument and returns a table with a single column with the value of the argument. To create a function in DataFusion, you need to implement the `TableFunctionImpl` trait. This trait has a single method, `call`, that takes a slice of `Expr`s and returns a `Result>`. +The simple UDTF used here takes a single `Int64` argument and returns a table with a single column with the value of the +argument. To create a function in DataFusion, you need to implement the `TableFunctionImpl` trait. This trait has a +single method, `call`, that takes a slice of `Expr`s and returns a `Result>`. -In the `call` method, you parse the input `Expr`s and return a `TableProvider`. You might also want to do some validation of the input `Expr`s, e.g. checking that the number of arguments is correct. +In the `call` method, you parse the input `Expr`s and return a `TableProvider`. You might also want to do some +validation of the input `Expr`s, e.g. checking that the number of arguments is correct. ```rust use datafusion::common::plan_err; @@ -600,7 +631,7 @@ use datafusion::execution::context::SessionContext; let ctx = SessionContext::new(); -ctx.register_udtf("echo", Arc::new(EchoFunction::default())); +ctx.register_udtf("echo", Arc::new(EchoFunction::default ())); ``` And if all goes well, you can use it in your query: @@ -611,7 +642,7 @@ use datafusion::arrow::util::pretty; let df = ctx.sql("SELECT * FROM echo(1)").await?; let results = df.collect().await?; -pretty::print_batches(&results)?; +pretty::print_batches( & results) ?; // +---+ // | a | // +---+ diff --git a/docs/source/library-user-guide/catalogs.md b/docs/source/library-user-guide/catalogs.md index e9d157df5f2a..13158d656423 100644 --- a/docs/source/library-user-guide/catalogs.md +++ b/docs/source/library-user-guide/catalogs.md @@ -73,9 +73,9 @@ impl SchemaProvider for MemorySchemaProvider { table: Arc, ) -> Result>> { if self.table_exist(name.as_str()) { - return Err(DataFusionError::Execution(format!( + return exec_err!( "The table {name} already exists" - ))); + ); } Ok(self.tables.insert(name, table)) } diff --git a/docs/source/user-guide/introduction.md b/docs/source/user-guide/introduction.md index 7c975055d152..02d53a713e4f 100644 --- a/docs/source/user-guide/introduction.md +++ b/docs/source/user-guide/introduction.md @@ -110,6 +110,7 @@ Here are some active projects using DataFusion: - [Kamu](https://github.com/kamu-data/kamu-cli/) Planet-scale streaming data pipeline - [LakeSoul](https://github.com/lakesoul-io/LakeSoul) Open source LakeHouse framework with native IO in Rust. - [Lance](https://github.com/lancedb/lance) Modern columnar data format for ML +- [OpenObserve](https://github.com/openobserve/openobserve) Distributed cloud native observability platform - [ParadeDB](https://github.com/paradedb/paradedb) PostgreSQL for Search & Analytics - [Parseable](https://github.com/parseablehq/parseable) Log storage and observability platform - [qv](https://github.com/timvw/qv) Quickly view your data @@ -120,7 +121,6 @@ Here are some active projects using DataFusion: - [Spice.ai](https://github.com/spiceai/spiceai) Unified SQL query interface & materialization engine - [Synnada](https://synnada.ai/) Streaming-first framework for data products - [VegaFusion](https://vegafusion.io/) Server-side acceleration for the [Vega](https://vega.github.io/) visualization grammar -- [ZincObserve](https://github.com/zinclabs/zincobserve) Distributed cloud native observability platform Here are some less active projects that used DataFusion: @@ -143,6 +143,7 @@ Here are some less active projects that used DataFusion: [greptime db]: https://github.com/GreptimeTeam/greptimedb [horaedb]: https://github.com/apache/incubator-horaedb [influxdb]: https://github.com/influxdata/influxdb +[openobserve]: https://github.com/openobserve/openobserve [parseable]: https://github.com/parseablehq/parseable [prql-query]: https://github.com/prql/prql-query [qv]: https://github.com/timvw/qv @@ -151,8 +152,7 @@ Here are some less active projects that used DataFusion: [spice.ai]: https://github.com/spiceai/spiceai [synnada]: https://synnada.ai/ [tensorbase]: https://github.com/tensorbase/tensorbase -[vegafusion]: https://vegafusion.io/ -[zincobserve]: https://github.com/zinclabs/zincobserve "if you know of another project, please submit a PR to add a link!" +[vegafusion]: https://vegafusion.io/ "if you know of another project, please submit a PR to add a link!" ## Integrations and Extensions diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index a379dfc9ec29..208d18f0e5ab 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -547,6 +547,7 @@ trunc(numeric_expression[, decimal_places]) ## Conditional Functions - [coalesce](#coalesce) +- [greatest](#greatest) - [ifnull](#ifnull) - [nullif](#nullif) - [nvl](#nvl) @@ -575,6 +576,29 @@ coalesce(expression1[, ..., expression_n]) +----------------------------------------+ ``` +### `greatest` + +Returns the greatest value in a list of expressions. Returns _null_ if all expressions are _null_. + +``` +greatest(expression1[, ..., expression_n]) +``` + +#### Arguments + +- **expression1, expression_n**: Expressions to compare and return the greatest value.. Can be a constant, column, or function, and any combination of arithmetic operators. Pass as many expression arguments as necessary. + +#### Example + +```sql +> select greatest(4, 7, 5); ++---------------------------+ +| greatest(4,7,5) | ++---------------------------+ +| 7 | ++---------------------------+ +``` + ### `ifnull` _Alias of [nvl](#nvl)._