diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index ce4b4b06cf44..2ddeebbc558e 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -581,4 +581,4 @@ jobs: run: cargo msrv verify - name: Check datafusion-cli working-directory: datafusion-cli - run: cargo msrv verify + run: cargo msrv verify \ No newline at end of file diff --git a/Cargo.toml b/Cargo.toml index 968a74e37f10..f87205f0d067 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -53,7 +53,7 @@ license = "Apache-2.0" readme = "README.md" repository = "https://github.com/apache/datafusion" rust-version = "1.76" -version = "39.0.0" +version = "40.0.0" [workspace.dependencies] # We turn off default-features for some dependencies here so the workspaces which inherit them can @@ -86,23 +86,23 @@ bytes = "1.4" chrono = { version = "0.4.34", default-features = false } ctor = "0.2.0" dashmap = "5.5.0" -datafusion = { path = "datafusion/core", version = "39.0.0", default-features = false } -datafusion-common = { path = "datafusion/common", version = "39.0.0", default-features = false } -datafusion-common-runtime = { path = "datafusion/common-runtime", version = "39.0.0" } -datafusion-execution = { path = "datafusion/execution", version = "39.0.0" } -datafusion-expr = { path = "datafusion/expr", version = "39.0.0" } -datafusion-functions = { path = "datafusion/functions", version = "39.0.0" } -datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "39.0.0" } -datafusion-functions-array = { path = "datafusion/functions-array", version = "39.0.0" } -datafusion-optimizer = { path = "datafusion/optimizer", version = "39.0.0", default-features = false } -datafusion-physical-expr = { path = "datafusion/physical-expr", version = "39.0.0", default-features = false } -datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "39.0.0", default-features = false } -datafusion-physical-plan = { path = "datafusion/physical-plan", version = "39.0.0" } -datafusion-proto = { path = "datafusion/proto", version = "39.0.0" } -datafusion-proto-common = { path = "datafusion/proto-common", version = "39.0.0" } -datafusion-sql = { path = "datafusion/sql", version = "39.0.0" } -datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "39.0.0" } -datafusion-substrait = { path = "datafusion/substrait", version = "39.0.0" } +datafusion = { path = "datafusion/core", version = "40.0.0", default-features = false } +datafusion-common = { path = "datafusion/common", version = "40.0.0", default-features = false } +datafusion-common-runtime = { path = "datafusion/common-runtime", version = "40.0.0" } +datafusion-execution = { path = "datafusion/execution", version = "40.0.0" } +datafusion-expr = { path = "datafusion/expr", version = "40.0.0" } +datafusion-functions = { path = "datafusion/functions", version = "40.0.0" } +datafusion-functions-aggregate = { path = "datafusion/functions-aggregate", version = "40.0.0" } +datafusion-functions-array = { path = "datafusion/functions-array", version = "40.0.0" } +datafusion-optimizer = { path = "datafusion/optimizer", version = "40.0.0", default-features = false } +datafusion-physical-expr = { path = "datafusion/physical-expr", version = "40.0.0", default-features = false } +datafusion-physical-expr-common = { path = "datafusion/physical-expr-common", version = "40.0.0", default-features = false } +datafusion-physical-plan = { path = "datafusion/physical-plan", version = "40.0.0" } +datafusion-proto = { path = "datafusion/proto", version = "40.0.0" } +datafusion-proto-common = { path = "datafusion/proto-common", version = "40.0.0" } +datafusion-sql = { path = "datafusion/sql", version = "40.0.0" } +datafusion-sqllogictest = { path = "datafusion/sqllogictest", version = "40.0.0" } +datafusion-substrait = { path = "datafusion/substrait", version = "40.0.0" } doc-comment = "0.3" env_logger = "0.11" futures = "0.3" diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 4fce2ec500e4..42ec5922a73f 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -381,13 +381,13 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.80" +version = "0.1.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6fa2087f2753a7da8cc1c0dbfcf89579dd57458e36769de5ac750b4671737ca" +checksum = "6e0c28dcc82d7c8ead5cb13beb15405b57b8546e93215673ff8ca0349a028107" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -875,9 +875,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.0.104" +version = "1.0.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "74b6a57f98764a267ff415d50a25e6e166f3831a5071af4995296ea97d210490" +checksum = "066fce287b1d4eafef758e89e09d724a24808a9196fe9756b8ca90e86d0719a2" dependencies = [ "jobserver", "libc", @@ -1099,7 +1099,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "edb49164822f3ee45b17acd4a208cfc1251410cf0cad9a833234c9890774dd9f" dependencies = [ "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -1123,7 +1123,7 @@ dependencies = [ [[package]] name = "datafusion" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "apache-avro", @@ -1177,7 +1177,7 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "assert_cmd", @@ -1204,7 +1204,7 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "apache-avro", @@ -1225,14 +1225,14 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "39.0.0" +version = "40.0.0" dependencies = [ "tokio", ] [[package]] name = "datafusion-execution" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "chrono", @@ -1251,7 +1251,7 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1268,7 +1268,7 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "base64 0.22.1", @@ -1292,7 +1292,7 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1308,7 +1308,7 @@ dependencies = [ [[package]] name = "datafusion-functions-array" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "arrow-array", @@ -1319,6 +1319,7 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-functions", + "datafusion-functions-aggregate", "itertools", "log", "paste", @@ -1326,7 +1327,7 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "async-trait", @@ -1344,7 +1345,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1372,7 +1373,7 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1384,7 +1385,7 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "39.0.0" +version = "40.0.0" dependencies = [ "ahash", "arrow", @@ -1416,7 +1417,7 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "39.0.0" +version = "40.0.0" dependencies = [ "arrow", "arrow-array", @@ -1685,7 +1686,7 @@ checksum = "87750cf4b7a4c0625b1529e4c543c2182106e4dedc60a2a6455e00d212c489ac" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2007,7 +2008,7 @@ dependencies = [ "http 1.1.0", "hyper 1.4.0", "hyper-util", - "rustls 0.23.10", + "rustls 0.23.11", "rustls-native-certs 0.7.1", "rustls-pki-types", "tokio", @@ -2698,7 +2699,7 @@ checksum = "2f38a4412a78282e09a2cf38d195ea5420d15ba0602cb375210efbc877243965" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -2821,7 +2822,7 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.10", + "rustls 0.23.11", "thiserror", "tokio", "tracing", @@ -2837,7 +2838,7 @@ dependencies = [ "rand", "ring 0.17.8", "rustc-hash", - "rustls 0.23.10", + "rustls 0.23.11", "slab", "thiserror", "tinyvec", @@ -2986,7 +2987,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.10", + "rustls 0.23.11", "rustls-native-certs 0.7.1", "rustls-pemfile 2.1.2", "rustls-pki-types", @@ -3116,9 +3117,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.10" +version = "0.23.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05cff451f60db80f490f3c182b77c35260baace73209e9cdbbe526bfe3a4d402" +checksum = "4828ea528154ae444e5a642dbb7d5623354030dc9822b83fd9bb79683c7399d0" dependencies = [ "once_cell", "ring 0.17.8", @@ -3295,22 +3296,22 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "bc76f558e0cbb2a839d37354c575f1dc3fdc6546b5be373ba43d95f231bf7c12" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.204" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "e0cd7e117be63d3c3678776753929474f3b04a43a080c744d6b0ae2a8c28e222" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3445,7 +3446,7 @@ checksum = "01b2e185515564f15375f593fb966b5718bc624ba77fe49fa4616ad619690554" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3491,7 +3492,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3504,7 +3505,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3526,9 +3527,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.68" +version = "2.0.70" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "901fa70d88b9d6c98022e23b4136f9f3e54e4662c3bc1bd1d84a42a9a0f0c1e9" +checksum = "2f0209b68b3613b093e0ec905354eccaedcfe83b8cb37cbdeae64026c3064c16" dependencies = [ "proc-macro2", "quote", @@ -3591,7 +3592,7 @@ checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3646,9 +3647,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.7.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce6b6a2fb3a985e99cebfaefa9faa3024743da73304ca1c683a36429613d3d22" +checksum = "445e881f4f6d382d5f27c034e25eb92edd7c784ceab92a0937db7f2e9471b938" dependencies = [ "tinyvec_macros", ] @@ -3686,7 +3687,7 @@ checksum = "5f5ae998a069d4b5aba8ee9dad856af7d520c3699e6159b185c2acd48155d39a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3706,7 +3707,7 @@ version = "0.26.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c7bc40d0e5a97695bb96e27995cd3a08538541b0a846f65bba7a359f36700d4" dependencies = [ - "rustls 0.23.10", + "rustls 0.23.11", "rustls-pki-types", "tokio", ] @@ -3783,7 +3784,7 @@ checksum = "34704c8d6ebcbc939824180af020566b01a7c01f80641264eba0999f6c2b6be7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3828,7 +3829,7 @@ checksum = "f03ca4cb38206e2bef0700092660bb74d696f808514dae47fa1467cbfe26e96e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] @@ -3982,7 +3983,7 @@ dependencies = [ "once_cell", "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-shared", ] @@ -4016,7 +4017,7 @@ checksum = "e94f17b526d0a461a191c78ea52bbce64071ed5c04c9ffe424dcb38f74171bb7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -4281,7 +4282,7 @@ checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.68", + "syn 2.0.70", ] [[package]] diff --git a/datafusion-cli/Cargo.toml b/datafusion-cli/Cargo.toml index bcacf1d52a9b..860dc123fa94 100644 --- a/datafusion-cli/Cargo.toml +++ b/datafusion-cli/Cargo.toml @@ -18,7 +18,7 @@ [package] name = "datafusion-cli" description = "Command Line Client for DataFusion query engine." -version = "39.0.0" +version = "40.0.0" authors = ["Apache DataFusion "] edition = "2021" keywords = ["arrow", "datafusion", "query", "sql"] @@ -35,7 +35,7 @@ async-trait = "0.1.41" aws-config = "0.55" aws-credential-types = "0.55" clap = { version = "3", features = ["derive", "cargo"] } -datafusion = { path = "../datafusion/core", version = "39.0.0", features = [ +datafusion = { path = "../datafusion/core", version = "40.0.0", features = [ "avro", "crypto_expressions", "datetime_expressions", diff --git a/datafusion-cli/src/helper.rs b/datafusion-cli/src/helper.rs index f93aaec4218d..9b1f2aa125c2 100644 --- a/datafusion-cli/src/helper.rs +++ b/datafusion-cli/src/helper.rs @@ -252,7 +252,7 @@ mod tests { fn unescape_readline_input() -> Result<()> { let validator = CliHelper::default(); - // shoule be valid + // should be valid let result = readline_direct( Cursor::new( r"create external table test stored as csv location 'data.csv' options ('format.delimiter' ',');" @@ -326,7 +326,7 @@ mod tests { fn sql_dialect() -> Result<()> { let mut validator = CliHelper::default(); - // shoule be invalid in generic dialect + // should be invalid in generic dialect let result = readline_direct(Cursor::new(r"select 1 # 2;".as_bytes()), &validator)?; assert!( diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 48da09a51236..1259f90d6449 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -339,7 +339,7 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { Ok(()) } - /// Generate output, as specififed by `emit_to` and update the intermediate state + /// Generate output, as specified by `emit_to` and update the intermediate state fn evaluate(&mut self, emit_to: datafusion_expr::EmitTo) -> Result { let counts = emit_to.take_needed(&mut self.counts); let prods = emit_to.take_needed(&mut self.prods); diff --git a/datafusion-examples/examples/advanced_udwf.rs b/datafusion-examples/examples/advanced_udwf.rs index 41c6381df5d4..11fb6f6ccc48 100644 --- a/datafusion-examples/examples/advanced_udwf.rs +++ b/datafusion-examples/examples/advanced_udwf.rs @@ -75,7 +75,7 @@ impl WindowUDFImpl for SmoothItUdf { Ok(DataType::Float64) } - /// Create a `PartitionEvalutor` to evaluate this function on a new + /// Create a `PartitionEvaluator` to evaluate this function on a new /// partition. fn partition_evaluator(&self) -> Result> { Ok(Box::new(MyPartitionEvaluator::new())) diff --git a/datafusion-examples/examples/catalog.rs b/datafusion-examples/examples/catalog.rs index 5bc2cadac128..b9188e1cd5e0 100644 --- a/datafusion-examples/examples/catalog.rs +++ b/datafusion-examples/examples/catalog.rs @@ -83,7 +83,7 @@ async fn main() -> Result<()> { // register our catalog in the context ctx.register_catalog("dircat", Arc::new(catalog)); { - // catalog was passed down into our custom catalog list since we overide the ctx's default + // catalog was passed down into our custom catalog list since we override the ctx's default let catalogs = catlist.catalogs.read().unwrap(); assert!(catalogs.contains_key("dircat")); }; diff --git a/datafusion-examples/examples/expr_api.rs b/datafusion-examples/examples/expr_api.rs index 36ce3badcb5e..43729a913e5d 100644 --- a/datafusion-examples/examples/expr_api.rs +++ b/datafusion-examples/examples/expr_api.rs @@ -46,7 +46,7 @@ use datafusion_expr::{AggregateExt, ColumnarValue, ExprSchemable, Operator}; /// /// The code in this example shows how to: /// 1. Create [`Expr`]s using different APIs: [`main`]` -/// 2. Use the fluent API to easly create complex [`Expr`]s: [`expr_fn_demo`] +/// 2. Use the fluent API to easily create complex [`Expr`]s: [`expr_fn_demo`] /// 3. Evaluate [`Expr`]s against data: [`evaluate_demo`] /// 4. Simplify expressions: [`simplify_demo`] /// 5. Analyze predicates for boundary ranges: [`range_analysis_demo`] diff --git a/datafusion-examples/examples/parse_sql_expr.rs b/datafusion-examples/examples/parse_sql_expr.rs index 6444eb68b6b2..a1fc5d269a04 100644 --- a/datafusion-examples/examples/parse_sql_expr.rs +++ b/datafusion-examples/examples/parse_sql_expr.rs @@ -113,7 +113,7 @@ async fn query_parquet_demo() -> Result<()> { vec![df.parse_sql_expr("SUM(int_col) as sum_int_col")?], )? // Directly parsing the SQL text into a sort expression is not supported yet, so - // construct it programatically + // construct it programmatically .sort(vec![col("double_col").sort(false, false)])? .limit(0, Some(1))?; diff --git a/datafusion-examples/examples/simple_udwf.rs b/datafusion-examples/examples/simple_udwf.rs index 95339eff1cae..563f02cee6a6 100644 --- a/datafusion-examples/examples/simple_udwf.rs +++ b/datafusion-examples/examples/simple_udwf.rs @@ -132,7 +132,7 @@ async fn main() -> Result<()> { Ok(()) } -/// Create a `PartitionEvalutor` to evaluate this function on a new +/// Create a `PartitionEvaluator` to evaluate this function on a new /// partition. fn make_partition_evaluator() -> Result> { Ok(Box::new(MyPartitionEvaluator::new())) diff --git a/datafusion-examples/examples/sql_analysis.rs b/datafusion-examples/examples/sql_analysis.rs index 3995988751c7..9a2aabaa79c2 100644 --- a/datafusion-examples/examples/sql_analysis.rs +++ b/datafusion-examples/examples/sql_analysis.rs @@ -87,7 +87,7 @@ fn count_trees(plan: &LogicalPlan) -> (usize, Vec) { let mut groups = vec![]; while let Some(node) = to_visit.pop() { - // if we encouter a join, we know were at the root of the tree + // if we encounter a join, we know were at the root of the tree // count this tree and recurse on it's inputs if matches!(node, LogicalPlan::Join(_) | LogicalPlan::CrossJoin(_)) { let (group_count, inputs) = count_tree(node); diff --git a/datafusion/common/src/scalar/mod.rs b/datafusion/common/src/scalar/mod.rs index 55ce76c4b939..26e03a3b9893 100644 --- a/datafusion/common/src/scalar/mod.rs +++ b/datafusion/common/src/scalar/mod.rs @@ -1459,7 +1459,10 @@ impl ScalarValue { ScalarValue::DurationMillisecond(v) => v.is_none(), ScalarValue::DurationMicrosecond(v) => v.is_none(), ScalarValue::DurationNanosecond(v) => v.is_none(), - ScalarValue::Union(v, _, _) => v.is_none(), + ScalarValue::Union(v, _, _) => match v { + Some((_, s)) => s.is_null(), + None => true, + }, ScalarValue::Dictionary(_, v) => v.is_null(), } } @@ -6514,4 +6517,33 @@ mod tests { } intervals } + + fn union_fields() -> UnionFields { + [ + (0, Arc::new(Field::new("A", DataType::Int32, true))), + (1, Arc::new(Field::new("B", DataType::Float64, true))), + ] + .into_iter() + .collect() + } + + #[test] + fn sparse_scalar_union_is_null() { + let sparse_scalar = ScalarValue::Union( + Some((0_i8, Box::new(ScalarValue::Int32(None)))), + union_fields(), + UnionMode::Sparse, + ); + assert!(sparse_scalar.is_null()); + } + + #[test] + fn dense_scalar_union_is_null() { + let dense_scalar = ScalarValue::Union( + Some((0_i8, Box::new(ScalarValue::Int32(None)))), + union_fields(), + UnionMode::Dense, + ); + assert!(dense_scalar.is_null()); + } } diff --git a/datafusion/core/benches/sql_planner.rs b/datafusion/core/benches/sql_planner.rs index c0e02d388af4..00f6d5916751 100644 --- a/datafusion/core/benches/sql_planner.rs +++ b/datafusion/core/benches/sql_planner.rs @@ -82,7 +82,7 @@ fn register_defs(ctx: SessionContext, defs: Vec) -> SessionContext { defs.iter().for_each(|TableDef { name, schema }| { ctx.register_table( name, - Arc::new(MemTable::try_new(Arc::new(schema.clone()), vec![]).unwrap()), + Arc::new(MemTable::try_new(Arc::new(schema.clone()), vec![vec![]]).unwrap()), ) .unwrap(); }); diff --git a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs index b9aca2ac2cc9..59369aba57a9 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet/statistics.rs @@ -19,17 +19,19 @@ // TODO: potentially move this to arrow-rs: https://github.com/apache/arrow-rs/issues/4328 -use arrow::array::builder::FixedSizeBinaryBuilder; +use arrow::array::{ + BooleanBuilder, FixedSizeBinaryBuilder, LargeStringBuilder, StringBuilder, +}; use arrow::datatypes::i256; use arrow::{array::ArrayRef, datatypes::DataType}; use arrow_array::{ new_empty_array, new_null_array, BinaryArray, BooleanArray, Date32Array, Date64Array, - Decimal128Array, Decimal256Array, FixedSizeBinaryArray, Float16Array, Float32Array, - Float64Array, Int16Array, Int32Array, Int64Array, Int8Array, LargeBinaryArray, - LargeStringArray, StringArray, Time32MillisecondArray, Time32SecondArray, - Time64MicrosecondArray, Time64NanosecondArray, TimestampMicrosecondArray, - TimestampMillisecondArray, TimestampNanosecondArray, TimestampSecondArray, - UInt16Array, UInt32Array, UInt64Array, UInt8Array, + Decimal128Array, Decimal256Array, Float16Array, Float32Array, Float64Array, + Int16Array, Int32Array, Int64Array, Int8Array, LargeBinaryArray, + Time32MillisecondArray, Time32SecondArray, Time64MicrosecondArray, + Time64NanosecondArray, TimestampMicrosecondArray, TimestampMillisecondArray, + TimestampNanosecondArray, TimestampSecondArray, UInt16Array, UInt32Array, + UInt64Array, UInt8Array, }; use arrow_schema::{Field, FieldRef, Schema, TimeUnit}; use datafusion_common::{internal_datafusion_err, internal_err, plan_err, Result}; @@ -393,51 +395,73 @@ macro_rules! get_statistics { }) }, DataType::Binary => Ok(Arc::new(BinaryArray::from_iter( - [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator).map(|x| x.map(|x| x.to_vec())), + [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator) ))), DataType::LargeBinary => Ok(Arc::new(LargeBinaryArray::from_iter( - [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator).map(|x| x.map(|x|x.to_vec())), - ))), - DataType::Utf8 => Ok(Arc::new(StringArray::from_iter( - [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| { - let res = std::str::from_utf8(x).map(|s| s.to_string()).ok(); - if res.is_none() { - log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); - } - res - }) - }), + [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator) ))), + DataType::Utf8 => { + let iterator = [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator); + let mut builder = StringBuilder::new(); + for x in iterator { + let Some(x) = x else { + builder.append_null(); // no statistics value + continue; + }; + + let Ok(x) = std::str::from_utf8(x) else { + log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); + builder.append_null(); + continue; + }; + + builder.append_value(x); + } + Ok(Arc::new(builder.finish())) + }, DataType::LargeUtf8 => { - Ok(Arc::new(LargeStringArray::from_iter( - [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| { - let res = std::str::from_utf8(x).map(|s| s.to_string()).ok(); - if res.is_none() { - log::debug!("LargeUtf8 statistics is a non-UTF8 value, ignoring it."); - } - res - }) - }), - ))) - } - DataType::FixedSizeBinary(size) => Ok(Arc::new(FixedSizeBinaryArray::from( - [<$stat_type_prefix FixedLenByteArrayStatsIterator>]::new($iterator).map(|x| { - x.and_then(|x| { - if x.len().try_into() == Ok(*size) { - Some(x) - } else { - log::debug!( - "FixedSizeBinary({}) statistics is a binary of size {}, ignoring it.", - size, - x.len(), - ); - None - } - }) - }).collect::>(), - ))), + let iterator = [<$stat_type_prefix ByteArrayStatsIterator>]::new($iterator); + let mut builder = LargeStringBuilder::new(); + for x in iterator { + let Some(x) = x else { + builder.append_null(); // no statistics value + continue; + }; + + let Ok(x) = std::str::from_utf8(x) else { + log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); + builder.append_null(); + continue; + }; + + builder.append_value(x); + } + Ok(Arc::new(builder.finish())) + }, + DataType::FixedSizeBinary(size) => { + let iterator = [<$stat_type_prefix FixedLenByteArrayStatsIterator>]::new($iterator); + let mut builder = FixedSizeBinaryBuilder::new(*size); + for x in iterator { + let Some(x) = x else { + builder.append_null(); // no statistics value + continue; + }; + + // ignore invalid values + if x.len().try_into() != Ok(*size){ + log::debug!( + "FixedSizeBinary({}) statistics is a binary of size {}, ignoring it.", + size, + x.len(), + ); + builder.append_null(); + continue; + } + + builder.append_value(x).expect("ensure to append successfully here, because size have been checked before"); + } + Ok(Arc::new(builder.finish())) + }, DataType::Decimal128(precision, scale) => { let arr = Decimal128Array::from_iter( [<$stat_type_prefix Decimal128StatsIterator>]::new($iterator) @@ -740,15 +764,20 @@ macro_rules! get_data_page_statistics { ($stat_type_prefix: ident, $data_type: ident, $iterator: ident) => { paste! { match $data_type { - Some(DataType::Boolean) => Ok(Arc::new( - BooleanArray::from_iter( - [<$stat_type_prefix BooleanDataPageStatsIterator>]::new($iterator) - .flatten() - // BooleanArray::from_iter required a sized iterator, so collect into Vec first - .collect::>() - .into_iter() - ) - )), + Some(DataType::Boolean) => { + let iterator = [<$stat_type_prefix BooleanDataPageStatsIterator>]::new($iterator); + let mut builder = BooleanBuilder::new(); + for x in iterator { + for x in x.into_iter() { + let Some(x) = x else { + builder.append_null(); // no statistics value + continue; + }; + builder.append_value(x); + } + } + Ok(Arc::new(builder.finish())) + }, Some(DataType::UInt8) => Ok(Arc::new( UInt8Array::from_iter( [<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) @@ -830,32 +859,48 @@ macro_rules! get_data_page_statistics { Some(DataType::Float64) => Ok(Arc::new(Float64Array::from_iter([<$stat_type_prefix Float64DataPageStatsIterator>]::new($iterator).flatten()))), Some(DataType::Binary) => Ok(Arc::new(BinaryArray::from_iter([<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator).flatten()))), Some(DataType::LargeBinary) => Ok(Arc::new(LargeBinaryArray::from_iter([<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator).flatten()))), - Some(DataType::Utf8) => Ok(Arc::new(StringArray::from( - [<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator).map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| { - let res = std::str::from_utf8(x.data()).map(|s| s.to_string()).ok(); - if res.is_none() { + Some(DataType::Utf8) => { + let mut builder = StringBuilder::new(); + let iterator = [<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator); + for x in iterator { + for x in x.into_iter() { + let Some(x) = x else { + builder.append_null(); // no statistics value + continue; + }; + + let Ok(x) = std::str::from_utf8(x.data()) else { log::debug!("Utf8 statistics is a non-UTF8 value, ignoring it."); - } - res - }) - }) - }).flatten().collect::>(), - ))), - Some(DataType::LargeUtf8) => Ok(Arc::new(LargeStringArray::from( - [<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator).map(|x| { - x.into_iter().map(|x| { - x.and_then(|x| { - let res = std::str::from_utf8(x.data()).map(|s| s.to_string()).ok(); - if res.is_none() { - log::debug!("LargeUtf8 statistics is a non-UTF8 value, ignoring it."); - } - res - }) - }) - }).flatten().collect::>(), - ))), + builder.append_null(); + continue; + }; + + builder.append_value(x); + } + } + Ok(Arc::new(builder.finish())) + }, + Some(DataType::LargeUtf8) => { + let mut builder = LargeStringBuilder::new(); + let iterator = [<$stat_type_prefix ByteArrayDataPageStatsIterator>]::new($iterator); + for x in iterator { + for x in x.into_iter() { + let Some(x) = x else { + builder.append_null(); // no statistics value + continue; + }; + + let Ok(x) = std::str::from_utf8(x.data()) else { + log::debug!("LargeUtf8 statistics is a non-UTF8 value, ignoring it."); + builder.append_null(); + continue; + }; + + builder.append_value(x); + } + } + Ok(Arc::new(builder.finish())) + }, Some(DataType::Dictionary(_, value_type)) => { [<$stat_type_prefix:lower _ page_statistics>](Some(value_type), $iterator) }, @@ -871,14 +916,14 @@ macro_rules! get_data_page_statistics { Some(DataType::Date32) => Ok(Arc::new(Date32Array::from_iter([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator).flatten()))), Some(DataType::Date64) => Ok( Arc::new( - Date64Array::from([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) + Date64Array::from_iter([<$stat_type_prefix Int32DataPageStatsIterator>]::new($iterator) .map(|x| { x.into_iter() .map(|x| { x.and_then(|x| i64::try_from(x).ok()) - .map(|x| x * 24 * 60 * 60 * 1000) }) - }).flatten().collect::>() + .map(|x| x.map(|x| x * 24 * 60 * 60 * 1000)) + }).flatten() ) ) ), diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index 04debf498aa9..4b9e3e843341 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -60,7 +60,7 @@ use datafusion_execution::registry::SerializerRegistry; use datafusion_expr::{ expr_rewriter::FunctionRewrite, logical_plan::{DdlStatement, Statement}, - planner::UserDefinedSQLPlanner, + planner::ExprPlanner, Expr, UserDefinedLogicalNode, WindowUDF, }; @@ -1392,17 +1392,15 @@ impl FunctionRegistry for SessionContext { self.state.write().register_function_rewrite(rewrite) } - fn expr_planners(&self) -> Vec> { + fn expr_planners(&self) -> Vec> { self.state.read().expr_planners() } - fn register_user_defined_sql_planner( + fn register_expr_planner( &mut self, - user_defined_sql_planner: Arc, + expr_planner: Arc, ) -> Result<()> { - self.state - .write() - .register_user_defined_sql_planner(user_defined_sql_planner) + self.state.write().register_expr_planner(expr_planner) } } diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index d056b91c2747..c123ebb22ecb 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -60,7 +60,7 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr_rewriter::FunctionRewrite; -use datafusion_expr::planner::UserDefinedSQLPlanner; +use datafusion_expr::planner::ExprPlanner; use datafusion_expr::registry::{FunctionRegistry, SerializerRegistry}; use datafusion_expr::simplify::SimplifyInfo; use datafusion_expr::var_provider::{is_system_variables, VarType}; @@ -101,7 +101,7 @@ pub struct SessionState { /// Responsible for analyzing and rewrite a logical plan before optimization analyzer: Analyzer, /// Provides support for customising the SQL planner, e.g. to add support for custom operators like `->>` or `?` - user_defined_sql_planners: Vec>, + expr_planners: Vec>, /// Responsible for optimizing a logical plan optimizer: Optimizer, /// Responsible for optimizing a physical execution plan @@ -231,7 +231,7 @@ impl SessionState { ); } - let user_defined_sql_planners: Vec> = vec![ + let expr_planners: Vec> = vec![ Arc::new(functions::core::planner::CoreFunctionPlanner::default()), // register crate of array expressions (if enabled) #[cfg(feature = "array_expressions")] @@ -248,7 +248,7 @@ impl SessionState { let mut new_self = SessionState { session_id, analyzer: Analyzer::new(), - user_defined_sql_planners, + expr_planners, optimizer: Optimizer::new(), physical_optimizers: PhysicalOptimizer::new(), query_planner: Arc::new(DefaultQueryPlanner {}), @@ -968,7 +968,7 @@ impl SessionState { let mut query = SqlToRel::new_with_options(provider, self.get_parser_options()); // custom planners are registered first, so they're run first and take precedence over built-in planners - for planner in self.user_defined_sql_planners.iter() { + for planner in self.expr_planners.iter() { query = query.with_user_defined_planner(planner.clone()); } @@ -1184,16 +1184,15 @@ impl FunctionRegistry for SessionState { Ok(()) } - fn expr_planners(&self) -> Vec> { - self.user_defined_sql_planners.clone() + fn expr_planners(&self) -> Vec> { + self.expr_planners.clone() } - fn register_user_defined_sql_planner( + fn register_expr_planner( &mut self, - user_defined_sql_planner: Arc, + expr_planner: Arc, ) -> datafusion_common::Result<()> { - self.user_defined_sql_planners - .push(user_defined_sql_planner); + self.expr_planners.push(expr_planner); Ok(()) } } diff --git a/datafusion/core/src/lib.rs b/datafusion/core/src/lib.rs index fb7abcd795e8..956e9f7246a3 100644 --- a/datafusion/core/src/lib.rs +++ b/datafusion/core/src/lib.rs @@ -626,6 +626,12 @@ doc_comment::doctest!( user_guide_configs ); +#[cfg(doctest)] +doc_comment::doctest!( + "../../../docs/source/user-guide/dataframe.md", + user_guide_dataframe +); + #[cfg(doctest)] doc_comment::doctest!( "../../../docs/source/user-guide/expressions.md", diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index e8f2f34abda0..a7ce29bdc7e3 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -987,8 +987,8 @@ impl<'a> PruningExpressionBuilder<'a> { }) } - fn op(&self) -> &Operator { - &self.op + fn op(&self) -> Operator { + self.op } fn scalar_expr(&self) -> &Arc { @@ -1064,7 +1064,7 @@ fn rewrite_expr_to_prunable( scalar_expr: &PhysicalExprRef, schema: DFSchema, ) -> Result<(PhysicalExprRef, Operator, PhysicalExprRef)> { - if !is_compare_op(&op) { + if !is_compare_op(op) { return plan_err!("rewrite_expr_to_prunable only support compare expression"); } @@ -1131,7 +1131,7 @@ fn rewrite_expr_to_prunable( } } -fn is_compare_op(op: &Operator) -> bool { +fn is_compare_op(op: Operator) -> bool { matches!( op, Operator::Eq @@ -1358,13 +1358,11 @@ fn build_predicate_expression( .map(|e| { Arc::new(phys_expr::BinaryExpr::new( in_list.expr().clone(), - eq_op.clone(), + eq_op, e.clone(), )) as _ }) - .reduce(|a, b| { - Arc::new(phys_expr::BinaryExpr::new(a, re_op.clone(), b)) as _ - }) + .reduce(|a, b| Arc::new(phys_expr::BinaryExpr::new(a, re_op, b)) as _) .unwrap(); return build_predicate_expression(&change_expr, schema, required_columns); } else { @@ -1376,7 +1374,7 @@ fn build_predicate_expression( if let Some(bin_expr) = expr_any.downcast_ref::() { ( bin_expr.left().clone(), - bin_expr.op().clone(), + *bin_expr.op(), bin_expr.right().clone(), ) } else { @@ -1388,7 +1386,7 @@ fn build_predicate_expression( let left_expr = build_predicate_expression(&left, schema, required_columns); let right_expr = build_predicate_expression(&right, schema, required_columns); // simplify boolean expression if applicable - let expr = match (&left_expr, &op, &right_expr) { + let expr = match (&left_expr, op, &right_expr) { (left, Operator::And, _) if is_always_true(left) => right_expr, (_, Operator::And, right) if is_always_true(right) => left_expr, (left, Operator::Or, right) @@ -1396,11 +1394,7 @@ fn build_predicate_expression( { unhandled } - _ => Arc::new(phys_expr::BinaryExpr::new( - left_expr, - op.clone(), - right_expr, - )), + _ => Arc::new(phys_expr::BinaryExpr::new(left_expr, op, right_expr)), }; return expr; } diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index e46a92e92818..2d1904d9e166 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -29,8 +29,9 @@ use arrow::{ }, record_batch::RecordBatch, }; -use arrow_array::Float32Array; -use arrow_schema::ArrowError; +use arrow_array::{Array, Float32Array, Float64Array, UnionArray}; +use arrow_buffer::ScalarBuffer; +use arrow_schema::{ArrowError, UnionFields, UnionMode}; use datafusion_functions_aggregate::count::count_udaf; use object_store::local::LocalFileSystem; use std::fs; @@ -2195,3 +2196,163 @@ async fn write_parquet_results() -> Result<()> { Ok(()) } + +fn union_fields() -> UnionFields { + [ + (0, Arc::new(Field::new("A", DataType::Int32, true))), + (1, Arc::new(Field::new("B", DataType::Float64, true))), + (2, Arc::new(Field::new("C", DataType::Utf8, true))), + ] + .into_iter() + .collect() +} + +#[tokio::test] +async fn sparse_union_is_null() { + // union of [{A=1}, {A=}, {B=3.2}, {B=}, {C="a"}, {C=}] + let int_array = Int32Array::from(vec![Some(1), None, None, None, None, None]); + let float_array = Float64Array::from(vec![None, None, Some(3.2), None, None, None]); + let str_array = StringArray::from(vec![None, None, None, None, Some("a"), None]); + let type_ids = [0, 0, 1, 1, 2, 2].into_iter().collect::>(); + + let children = vec![ + Arc::new(int_array) as Arc, + Arc::new(float_array), + Arc::new(str_array), + ]; + + let array = UnionArray::try_new(union_fields(), type_ids, None, children).unwrap(); + + let field = Field::new( + "my_union", + DataType::Union(union_fields(), UnionMode::Sparse), + true, + ); + let schema = Arc::new(Schema::new(vec![field])); + + let batch = RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap(); + + let ctx = SessionContext::new(); + + ctx.register_batch("union_batch", batch).unwrap(); + + let df = ctx.table("union_batch").await.unwrap(); + + // view_all + let expected = [ + "+----------+", + "| my_union |", + "+----------+", + "| {A=1} |", + "| {A=} |", + "| {B=3.2} |", + "| {B=} |", + "| {C=a} |", + "| {C=} |", + "+----------+", + ]; + assert_batches_sorted_eq!(expected, &df.clone().collect().await.unwrap()); + + // filter where is null + let result_df = df.clone().filter(col("my_union").is_null()).unwrap(); + let expected = [ + "+----------+", + "| my_union |", + "+----------+", + "| {A=} |", + "| {B=} |", + "| {C=} |", + "+----------+", + ]; + assert_batches_sorted_eq!(expected, &result_df.collect().await.unwrap()); + + // filter where is not null + let result_df = df.filter(col("my_union").is_not_null()).unwrap(); + let expected = [ + "+----------+", + "| my_union |", + "+----------+", + "| {A=1} |", + "| {B=3.2} |", + "| {C=a} |", + "+----------+", + ]; + assert_batches_sorted_eq!(expected, &result_df.collect().await.unwrap()); +} + +#[tokio::test] +async fn dense_union_is_null() { + // union of [{A=1}, null, {B=3.2}, {A=34}] + let int_array = Int32Array::from(vec![Some(1), None]); + let float_array = Float64Array::from(vec![Some(3.2), None]); + let str_array = StringArray::from(vec![Some("a"), None]); + let type_ids = [0, 0, 1, 1, 2, 2].into_iter().collect::>(); + let offsets = [0, 1, 0, 1, 0, 1] + .into_iter() + .collect::>(); + + let children = vec![ + Arc::new(int_array) as Arc, + Arc::new(float_array), + Arc::new(str_array), + ]; + + let array = + UnionArray::try_new(union_fields(), type_ids, Some(offsets), children).unwrap(); + + let field = Field::new( + "my_union", + DataType::Union(union_fields(), UnionMode::Dense), + true, + ); + let schema = Arc::new(Schema::new(vec![field])); + + let batch = RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap(); + + let ctx = SessionContext::new(); + + ctx.register_batch("union_batch", batch).unwrap(); + + let df = ctx.table("union_batch").await.unwrap(); + + // view_all + let expected = [ + "+----------+", + "| my_union |", + "+----------+", + "| {A=1} |", + "| {A=} |", + "| {B=3.2} |", + "| {B=} |", + "| {C=a} |", + "| {C=} |", + "+----------+", + ]; + assert_batches_sorted_eq!(expected, &df.clone().collect().await.unwrap()); + + // filter where is null + let result_df = df.clone().filter(col("my_union").is_null()).unwrap(); + let expected = [ + "+----------+", + "| my_union |", + "+----------+", + "| {A=} |", + "| {B=} |", + "| {C=} |", + "+----------+", + ]; + assert_batches_sorted_eq!(expected, &result_df.collect().await.unwrap()); + + // filter where is not null + let result_df = df.filter(col("my_union").is_not_null()).unwrap(); + let expected = [ + "+----------+", + "| my_union |", + "+----------+", + "| {A=1} |", + "| {B=3.2} |", + "| {C=a} |", + "+----------+", + ]; + assert_batches_sorted_eq!(expected, &result_df.collect().await.unwrap()); +} diff --git a/datafusion/core/tests/user_defined/user_defined_sql_planner.rs b/datafusion/core/tests/user_defined/expr_planner.rs similarity index 68% rename from datafusion/core/tests/user_defined/user_defined_sql_planner.rs rename to datafusion/core/tests/user_defined/expr_planner.rs index 37df7e0900b4..1b23bf9ab2ef 100644 --- a/datafusion/core/tests/user_defined/user_defined_sql_planner.rs +++ b/datafusion/core/tests/user_defined/expr_planner.rs @@ -24,12 +24,14 @@ use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::Operator; use datafusion::prelude::*; use datafusion::sql::sqlparser::ast::BinaryOperator; -use datafusion_expr::planner::{PlannerResult, RawBinaryExpr, UserDefinedSQLPlanner}; +use datafusion_common::ScalarValue; +use datafusion_expr::expr::Alias; +use datafusion_expr::planner::{ExprPlanner, PlannerResult, RawBinaryExpr}; use datafusion_expr::BinaryExpr; struct MyCustomPlanner; -impl UserDefinedSQLPlanner for MyCustomPlanner { +impl ExprPlanner for MyCustomPlanner { fn plan_binary_op( &self, expr: RawBinaryExpr, @@ -50,14 +52,23 @@ impl UserDefinedSQLPlanner for MyCustomPlanner { op: Operator::Plus, }))) } + BinaryOperator::Question => { + Ok(PlannerResult::Planned(Expr::Alias(Alias::new( + Expr::Literal(ScalarValue::Boolean(Some(true))), + None::<&str>, + format!("{} ? {}", expr.left, expr.right), + )))) + } _ => Ok(PlannerResult::Original(expr)), } } } async fn plan_and_collect(sql: &str) -> Result> { - let mut ctx = SessionContext::new(); - ctx.register_user_defined_sql_planner(Arc::new(MyCustomPlanner))?; + let config = + SessionConfig::new().set_str("datafusion.sql_parser.dialect", "postgres"); + let mut ctx = SessionContext::new_with_config(config); + ctx.register_expr_planner(Arc::new(MyCustomPlanner))?; ctx.sql(sql).await?.collect().await } @@ -86,3 +97,27 @@ async fn test_custom_operators_long_arrow() { ]; assert_batches_eq!(&expected, &actual); } + +#[tokio::test] +async fn test_question_select() { + let actual = plan_and_collect("select a ? 2 from (select 1 as a);") + .await + .unwrap(); + let expected = [ + "+--------------+", + "| a ? Int64(2) |", + "+--------------+", + "| true |", + "+--------------+", + ]; + assert_batches_eq!(&expected, &actual); +} + +#[tokio::test] +async fn test_question_filter() { + let actual = plan_and_collect("select a from (select 1 as a) where a ? 2;") + .await + .unwrap(); + let expected = ["+---+", "| a |", "+---+", "| 1 |", "+---+"]; + assert_batches_eq!(&expected, &actual); +} diff --git a/datafusion/core/tests/user_defined/mod.rs b/datafusion/core/tests/user_defined/mod.rs index 9b83a9fdd408..56cec8df468b 100644 --- a/datafusion/core/tests/user_defined/mod.rs +++ b/datafusion/core/tests/user_defined/mod.rs @@ -30,5 +30,5 @@ mod user_defined_window_functions; /// Tests for User Defined Table Functions mod user_defined_table_functions; -/// Tests for User Defined SQL Planner -mod user_defined_sql_planner; +/// Tests for Expression Planner +mod expr_planner; 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 ae8a009c6292..1733068debb9 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -575,9 +575,9 @@ async fn test_user_defined_functions_cast_to_i64() -> Result<()> { async fn test_user_defined_sql_functions() -> Result<()> { let ctx = SessionContext::new(); - let sql_planners = ctx.expr_planners(); + let expr_planners = ctx.expr_planners(); - assert!(!sql_planners.is_empty()); + assert!(!expr_planners.is_empty()); Ok(()) } diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index 24d61e6a8b72..df7fd0dbd92c 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -27,7 +27,7 @@ use crate::{ runtime_env::{RuntimeConfig, RuntimeEnv}, }; use datafusion_common::{plan_datafusion_err, DataFusionError, Result}; -use datafusion_expr::planner::UserDefinedSQLPlanner; +use datafusion_expr::planner::ExprPlanner; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; /// Task Execution Context @@ -192,7 +192,7 @@ impl FunctionRegistry for TaskContext { Ok(self.scalar_functions.insert(udf.name().into(), udf)) } - fn expr_planners(&self) -> Vec> { + fn expr_planners(&self) -> Vec> { vec![] } } diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 760952d94815..23e98714dfa4 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -39,8 +39,6 @@ pub enum AggregateFunction { Max, /// Aggregation into an array ArrayAgg, - /// N'th value in a group according to some ordering - NthValue, } impl AggregateFunction { @@ -50,7 +48,6 @@ impl AggregateFunction { Min => "MIN", Max => "MAX", ArrayAgg => "ARRAY_AGG", - NthValue => "NTH_VALUE", } } } @@ -69,7 +66,6 @@ impl FromStr for AggregateFunction { "max" => AggregateFunction::Max, "min" => AggregateFunction::Min, "array_agg" => AggregateFunction::ArrayAgg, - "nth_value" => AggregateFunction::NthValue, _ => { return plan_err!("There is no built-in function named {name}"); } @@ -114,7 +110,6 @@ impl AggregateFunction { coerced_data_types[0].clone(), input_expr_nullable[0], )))), - AggregateFunction::NthValue => Ok(coerced_data_types[0].clone()), } } @@ -124,7 +119,6 @@ impl AggregateFunction { match self { AggregateFunction::Max | AggregateFunction::Min => Ok(true), AggregateFunction::ArrayAgg => Ok(false), - AggregateFunction::NthValue => Ok(true), } } } @@ -147,7 +141,6 @@ impl AggregateFunction { .collect::>(); Signature::uniform(1, valid, Volatility::Immutable) } - AggregateFunction::NthValue => Signature::any(2, Volatility::Immutable), } } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 579f5fed578f..ecece6dbfce7 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1413,12 +1413,19 @@ impl Expr { .unwrap() } + /// Returns true if the expression node is volatile, i.e. whether it can return + /// different results when evaluated multiple times with the same input. + /// Note: unlike [`Self::is_volatile`], this function does not consider inputs: + /// - `rand()` returns `true`, + /// - `a + rand()` returns `false` + pub fn is_volatile_node(&self) -> bool { + matches!(self, Expr::ScalarFunction(func) if func.func.signature().volatility == Volatility::Volatile) + } + /// Returns true if the expression is volatile, i.e. whether it can return different /// results when evaluated multiple times with the same input. pub fn is_volatile(&self) -> Result { - self.exists(|expr| { - Ok(matches!(expr, Expr::ScalarFunction(func) if func.func.signature().volatility == Volatility::Volatile )) - }) + self.exists(|expr| Ok(expr.is_volatile_node())) } /// Recursively find all [`Expr::Placeholder`] expressions, and diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 8fd5982a0f2e..bda03fb7087a 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -25,7 +25,7 @@ use std::sync::Arc; use super::dml::CopyTo; use super::DdlStatement; use crate::builder::{change_redundant_column, unnest_with_options}; -use crate::expr::{Alias, Placeholder, Sort as SortExpr, WindowFunction}; +use crate::expr::{Placeholder, Sort as SortExpr, WindowFunction}; use crate::expr_rewriter::{create_col_from_scalar_expr, normalize_cols}; use crate::logical_plan::display::{GraphvizVisitor, IndentVisitor}; use crate::logical_plan::extension::UserDefinedLogicalNode; @@ -2130,16 +2130,10 @@ impl Filter { } } - // filter predicates should not be aliased - if let Expr::Alias(Alias { expr, name, .. }) = predicate { - return plan_err!( - "Attempted to create Filter predicate with \ - expression `{expr}` aliased as '{name}'. Filter predicates should not be \ - aliased." - ); - } - - Ok(Self { predicate, input }) + Ok(Self { + predicate: predicate.unalias_nested().data, + input, + }) } /// Is this filter guaranteed to return 0 or 1 row in a given instantiation? diff --git a/datafusion/expr/src/operator.rs b/datafusion/expr/src/operator.rs index 742511822a0f..a10312e23446 100644 --- a/datafusion/expr/src/operator.rs +++ b/datafusion/expr/src/operator.rs @@ -25,7 +25,7 @@ use std::ops; use std::ops::Not; /// Operators applied to expressions -#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] +#[derive(Debug, Copy, Clone, PartialEq, Eq, PartialOrd, Hash)] pub enum Operator { /// Expressions are equal Eq, diff --git a/datafusion/expr/src/planner.rs b/datafusion/expr/src/planner.rs index bcbf5eb203ac..aeb8ed8372b7 100644 --- a/datafusion/expr/src/planner.rs +++ b/datafusion/expr/src/planner.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! [`ContextProvider`] and [`UserDefinedSQLPlanner`] APIs to customize SQL query planning +//! [`ContextProvider`] and [`ExprPlanner`] APIs to customize SQL query planning use std::sync::Arc; @@ -83,7 +83,7 @@ pub trait ContextProvider { } /// This trait allows users to customize the behavior of the SQL planner -pub trait UserDefinedSQLPlanner: Send + Sync { +pub trait ExprPlanner: Send + Sync { /// Plan the binary operation between two expressions, returns original /// BinaryExpr if not possible fn plan_binary_op( @@ -139,6 +139,28 @@ pub trait UserDefinedSQLPlanner: Send + Sync { fn plan_extract(&self, args: Vec) -> Result>> { Ok(PlannerResult::Original(args)) } + + /// Plan an substring expression, e.g., `SUBSTRING( [FROM ] [FOR ])` + /// + /// Returns origin expression arguments if not possible + fn plan_substring(&self, args: Vec) -> Result>> { + Ok(PlannerResult::Original(args)) + } + + /// Plans a struct `struct(expression1[, ..., expression_n])` + /// literal based on the given input expressions. + /// This function takes a vector of expressions and a boolean flag indicating whether + /// the struct uses the optional name + /// + /// Returns a `PlannerResult` containing either the planned struct expressions or the original + /// input expressions if planning is not possible. + fn plan_struct_literal( + &self, + args: Vec, + _is_named_struct: bool, + ) -> Result>> { + Ok(PlannerResult::Original(args)) + } } /// An operator with two arguments to plan @@ -146,7 +168,7 @@ pub trait UserDefinedSQLPlanner: Send + Sync { /// Note `left` and `right` are DataFusion [`Expr`]s but the `op` is the SQL AST /// operator. /// -/// This structure is used by [`UserDefinedSQLPlanner`] to plan operators with +/// This structure is used by [`ExprPlanner`] to plan operators with /// custom expressions. #[derive(Debug, Clone)] pub struct RawBinaryExpr { @@ -157,7 +179,7 @@ pub struct RawBinaryExpr { /// An expression with GetFieldAccess to plan /// -/// This structure is used by [`UserDefinedSQLPlanner`] to plan operators with +/// This structure is used by [`ExprPlanner`] to plan operators with /// custom expressions. #[derive(Debug, Clone)] pub struct RawFieldAccessExpr { @@ -167,7 +189,7 @@ pub struct RawFieldAccessExpr { /// A Dictionary literal expression `{ key: value, ...}` /// -/// This structure is used by [`UserDefinedSQLPlanner`] to plan operators with +/// This structure is used by [`ExprPlanner`] to plan operators with /// custom expressions. #[derive(Debug, Clone)] pub struct RawDictionaryExpr { @@ -175,7 +197,7 @@ pub struct RawDictionaryExpr { pub values: Vec, } -/// Result of planning a raw expr with [`UserDefinedSQLPlanner`] +/// Result of planning a raw expr with [`ExprPlanner`] #[derive(Debug, Clone)] pub enum PlannerResult { /// The raw expression was successfully planned as a new [`Expr`] diff --git a/datafusion/expr/src/registry.rs b/datafusion/expr/src/registry.rs index 6a27c05bb451..988dc0f5aeda 100644 --- a/datafusion/expr/src/registry.rs +++ b/datafusion/expr/src/registry.rs @@ -18,7 +18,7 @@ //! FunctionRegistry trait use crate::expr_rewriter::FunctionRewrite; -use crate::planner::UserDefinedSQLPlanner; +use crate::planner::ExprPlanner; use crate::{AggregateUDF, ScalarUDF, UserDefinedLogicalNode, WindowUDF}; use datafusion_common::{not_impl_err, plan_datafusion_err, Result}; use std::collections::HashMap; @@ -110,15 +110,15 @@ pub trait FunctionRegistry { not_impl_err!("Registering FunctionRewrite") } - /// Set of all registered [`UserDefinedSQLPlanner`]s - fn expr_planners(&self) -> Vec>; + /// Set of all registered [`ExprPlanner`]s + fn expr_planners(&self) -> Vec>; - /// Registers a new [`UserDefinedSQLPlanner`] with the registry. - fn register_user_defined_sql_planner( + /// Registers a new [`ExprPlanner`] with the registry. + fn register_expr_planner( &mut self, - _user_defined_sql_planner: Arc, + _expr_planner: Arc, ) -> Result<()> { - not_impl_err!("Registering UserDefinedSQLPlanner") + not_impl_err!("Registering ExprPlanner") } } @@ -196,7 +196,7 @@ impl FunctionRegistry for MemoryFunctionRegistry { Ok(self.udwfs.insert(udaf.name().into(), udaf)) } - fn expr_planners(&self) -> Vec> { + fn expr_planners(&self) -> Vec> { vec![] } } diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 0f7464b96b3e..fbec6e2f8024 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -101,7 +101,6 @@ pub fn coerce_types( // unpack the dictionary to get the value get_min_max_result_type(input_types) } - AggregateFunction::NthValue => Ok(input_types.to_vec()), } } diff --git a/datafusion/expr/src/type_coercion/binary.rs b/datafusion/expr/src/type_coercion/binary.rs index 83a7da046844..4f79f3fa2b22 100644 --- a/datafusion/expr/src/type_coercion/binary.rs +++ b/datafusion/expr/src/type_coercion/binary.rs @@ -1152,8 +1152,8 @@ mod tests { ]; for (i, input_type) in input_types.iter().enumerate() { let expect_type = &result_types[i]; - for op in &comparison_op_types { - let (lhs, rhs) = get_input_types(&input_decimal, op, input_type)?; + for op in comparison_op_types { + let (lhs, rhs) = get_input_types(&input_decimal, &op, input_type)?; assert_eq!(expect_type, &lhs); assert_eq!(expect_type, &rhs); } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 34e007207427..45155cbd2c27 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -997,7 +997,7 @@ fn split_conjunction_impl<'a>(expr: &'a Expr, mut exprs: Vec<&'a Expr>) -> Vec<& /// assert_eq!(split_conjunction_owned(expr), split); /// ``` pub fn split_conjunction_owned(expr: Expr) -> Vec { - split_binary_owned(expr, &Operator::And) + split_binary_owned(expr, Operator::And) } /// Splits an owned binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` @@ -1020,19 +1020,19 @@ pub fn split_conjunction_owned(expr: Expr) -> Vec { /// ]; /// /// // use split_binary_owned to split them -/// assert_eq!(split_binary_owned(expr, &Operator::Plus), split); +/// assert_eq!(split_binary_owned(expr, Operator::Plus), split); /// ``` -pub fn split_binary_owned(expr: Expr, op: &Operator) -> Vec { +pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec { split_binary_owned_impl(expr, op, vec![]) } fn split_binary_owned_impl( expr: Expr, - operator: &Operator, + operator: Operator, mut exprs: Vec, ) -> Vec { match expr { - Expr::BinaryExpr(BinaryExpr { right, op, left }) if &op == operator => { + Expr::BinaryExpr(BinaryExpr { right, op, left }) if op == operator => { let exprs = split_binary_owned_impl(*left, operator, exprs); split_binary_owned_impl(*right, operator, exprs) } @@ -1049,17 +1049,17 @@ fn split_binary_owned_impl( /// Splits an binary operator tree [`Expr`] such as `A B C` => `[A, B, C]` /// /// See [`split_binary_owned`] for more details and an example. -pub fn split_binary<'a>(expr: &'a Expr, op: &Operator) -> Vec<&'a Expr> { +pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> { split_binary_impl(expr, op, vec![]) } fn split_binary_impl<'a>( expr: &'a Expr, - operator: &Operator, + operator: Operator, mut exprs: Vec<&'a Expr>, ) -> Vec<&'a Expr> { match expr { - Expr::BinaryExpr(BinaryExpr { right, op, left }) if op == operator => { + Expr::BinaryExpr(BinaryExpr { right, op, left }) if *op == operator => { let exprs = split_binary_impl(left, operator, exprs); split_binary_impl(right, operator, exprs) } @@ -1613,13 +1613,13 @@ mod tests { #[test] fn test_split_binary_owned() { let expr = col("a"); - assert_eq!(split_binary_owned(expr.clone(), &Operator::And), vec![expr]); + assert_eq!(split_binary_owned(expr.clone(), Operator::And), vec![expr]); } #[test] fn test_split_binary_owned_two() { assert_eq!( - split_binary_owned(col("a").eq(lit(5)).and(col("b")), &Operator::And), + split_binary_owned(col("a").eq(lit(5)).and(col("b")), Operator::And), vec![col("a").eq(lit(5)), col("b")] ); } @@ -1629,7 +1629,7 @@ mod tests { let expr = col("a").eq(lit(5)).or(col("b")); assert_eq!( // expr is connected by OR, but pass in AND - split_binary_owned(expr.clone(), &Operator::And), + split_binary_owned(expr.clone(), Operator::And), vec![expr] ); } diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index fc485a284ab4..6ae2dfb3697c 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -74,6 +74,7 @@ pub mod average; pub mod bit_and_or_xor; pub mod bool_and_or; pub mod grouping; +pub mod nth_value; pub mod string_agg; use crate::approx_percentile_cont::approx_percentile_cont_udaf; @@ -105,6 +106,7 @@ pub mod expr_fn { pub use super::first_last::last_value; pub use super::grouping::grouping; pub use super::median::median; + pub use super::nth_value::nth_value; pub use super::regr::regr_avgx; pub use super::regr::regr_avgy; pub use super::regr::regr_count; @@ -157,6 +159,7 @@ pub fn all_default_aggregate_functions() -> Vec> { bool_and_or::bool_or_udaf(), average::avg_udaf(), grouping::grouping_udaf(), + nth_value::nth_value_udaf(), ] } diff --git a/datafusion/physical-expr/src/aggregate/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs similarity index 77% rename from datafusion/physical-expr/src/aggregate/nth_value.rs rename to datafusion/functions-aggregate/src/nth_value.rs index b75ecd1066ca..6719c673c55b 100644 --- a/datafusion/physical-expr/src/aggregate/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -22,149 +22,149 @@ use std::any::Any; use std::collections::VecDeque; use std::sync::Arc; -use crate::aggregate::array_agg_ordered::merge_ordered_arrays; -use crate::aggregate::utils::{down_cast_any_ref, ordering_fields}; -use crate::expressions::{format_state_name, Literal}; -use crate::{ - reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, -}; - -use arrow_array::cast::AsArray; -use arrow_array::{new_empty_array, ArrayRef, StructArray}; +use arrow::array::{new_empty_array, ArrayRef, AsArray, StructArray}; use arrow_schema::{DataType, Field, Fields}; + use datafusion_common::utils::{array_into_list_array_nullable, get_row_at_idx}; -use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; -use datafusion_expr::utils::AggregateOrderSensitivity; -use datafusion_expr::Accumulator; +use datafusion_common::{exec_err, internal_err, not_impl_err, Result, ScalarValue}; +use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::utils::format_state_name; +use datafusion_expr::{ + Accumulator, AggregateUDF, AggregateUDFImpl, Expr, ReversedUDAF, Signature, + Volatility, +}; +use datafusion_physical_expr_common::aggregate::merge_arrays::merge_ordered_arrays; +use datafusion_physical_expr_common::aggregate::utils::ordering_fields; +use datafusion_physical_expr_common::sort_expr::{ + limited_convert_logical_sort_exprs_to_physical, LexOrdering, PhysicalSortExpr, +}; + +make_udaf_expr_and_func!( + NthValueAgg, + nth_value, + "Returns the nth value in a group of values.", + nth_value_udaf +); /// Expression for a `NTH_VALUE(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, /// and then their results are merged. #[derive(Debug)] pub struct NthValueAgg { - /// Column name - name: String, - /// The `DataType` for the input expression - input_data_type: DataType, - /// The input expression - expr: Arc, - /// The `N` value. - n: i64, - /// If the input expression can have `NULL`s - nullable: bool, - /// Ordering data types - order_by_data_types: Vec, - /// Ordering requirement - ordering_req: LexOrdering, + signature: Signature, + /// Determines whether `N` is relative to the beginning or the end + /// of the aggregation. When set to `true`, then `N` is from the end. + reversed: bool, } impl NthValueAgg { /// Create a new `NthValueAgg` aggregate function - pub fn new( - expr: Arc, - n: i64, - name: impl Into, - input_data_type: DataType, - nullable: bool, - order_by_data_types: Vec, - ordering_req: LexOrdering, - ) -> Self { + pub fn new() -> Self { Self { - name: name.into(), - input_data_type, - expr, - n, - nullable, - order_by_data_types, - ordering_req, + signature: Signature::any(2, Volatility::Immutable), + reversed: false, } } + + pub fn with_reversed(mut self, reversed: bool) -> Self { + self.reversed = reversed; + self + } } -impl AggregateExpr for NthValueAgg { +impl Default for NthValueAgg { + fn default() -> Self { + Self::new() + } +} + +impl AggregateUDFImpl for NthValueAgg { fn as_any(&self) -> &dyn Any { self } - fn field(&self) -> Result { - Ok(Field::new(&self.name, self.input_data_type.clone(), true)) + fn name(&self) -> &str { + "nth_value" + } + + fn signature(&self) -> &Signature { + &self.signature } - fn create_accumulator(&self) -> Result> { - Ok(Box::new(NthValueAccumulator::try_new( - self.n, - &self.input_data_type, - &self.order_by_data_types, - self.ordering_req.clone(), - )?)) + fn return_type(&self, arg_types: &[DataType]) -> Result { + Ok(arg_types[0].clone()) } - fn state_fields(&self) -> Result> { + fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { + let n = match acc_args.input_exprs[1] { + Expr::Literal(ScalarValue::Int64(Some(value))) => { + if self.reversed { + Ok(-value) + } else { + Ok(value) + } + } + _ => not_impl_err!( + "{} not supported for n: {}", + self.name(), + &acc_args.input_exprs[1] + ), + }?; + + let ordering_req = limited_convert_logical_sort_exprs_to_physical( + acc_args.sort_exprs, + acc_args.schema, + )?; + + let ordering_dtypes = ordering_req + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; + + NthValueAccumulator::try_new( + n, + acc_args.input_type, + &ordering_dtypes, + ordering_req, + ) + .map(|acc| Box::new(acc) as _) + } + + fn state_fields(&self, args: StateFieldsArgs) -> Result> { let mut fields = vec![Field::new_list( - format_state_name(&self.name, "nth_value"), - Field::new("item", self.input_data_type.clone(), true), - self.nullable, // This should be the same as field() + format_state_name(self.name(), "nth_value"), + // TODO: The nullability of the list element should be configurable. + // The hard-coded `true` should be changed once the field for + // nullability is added to `StateFieldArgs` struct. + // See: https://github.com/apache/datafusion/pull/11063 + Field::new("item", args.input_type.clone(), true), + false, )]; - if !self.ordering_req.is_empty() { - let orderings = - ordering_fields(&self.ordering_req, &self.order_by_data_types); + let orderings = args.ordering_fields.to_vec(); + if !orderings.is_empty() { fields.push(Field::new_list( - format_state_name(&self.name, "nth_value_orderings"), + format_state_name(self.name(), "nth_value_orderings"), Field::new("item", DataType::Struct(Fields::from(orderings)), true), - self.nullable, + false, )); } Ok(fields) } - fn expressions(&self) -> Vec> { - let n = Arc::new(Literal::new(ScalarValue::Int64(Some(self.n)))) as _; - vec![Arc::clone(&self.expr), n] + fn aliases(&self) -> &[String] { + &[] } - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - (!self.ordering_req.is_empty()).then_some(&self.ordering_req) - } - - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::HardRequirement - } - - fn name(&self) -> &str { - &self.name - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(Self { - name: self.name.to_string(), - input_data_type: self.input_data_type.clone(), - expr: Arc::clone(&self.expr), - // index should be from the opposite side - n: -self.n, - nullable: self.nullable, - order_by_data_types: self.order_by_data_types.clone(), - // reverse requirement - ordering_req: reverse_order_bys(&self.ordering_req), - }) as _) - } -} - -impl PartialEq for NthValueAgg { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.input_data_type == x.input_data_type - && self.order_by_data_types == x.order_by_data_types - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) + fn reverse_expr(&self) -> ReversedUDAF { + ReversedUDAF::Reversed(Arc::from(AggregateUDF::from( + Self::new().with_reversed(!self.reversed), + ))) } } #[derive(Debug)] -pub(crate) struct NthValueAccumulator { +pub struct NthValueAccumulator { + /// The `N` value. n: i64, /// Stores entries in the `NTH_VALUE` result. values: VecDeque, diff --git a/datafusion/functions-array/Cargo.toml b/datafusion/functions-array/Cargo.toml index eb1ef9e03f31..73c5b9114a2c 100644 --- a/datafusion/functions-array/Cargo.toml +++ b/datafusion/functions-array/Cargo.toml @@ -49,6 +49,7 @@ datafusion-common = { workspace = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-functions = { workspace = true } +datafusion-functions-aggregate = { workspace = true } itertools = { version = "0.12", features = ["use_std"] } log = { workspace = true } paste = "1.0.14" diff --git a/datafusion/functions-array/src/planner.rs b/datafusion/functions-array/src/planner.rs index cfb3e5ed0729..cfbe99b4b7fd 100644 --- a/datafusion/functions-array/src/planner.rs +++ b/datafusion/functions-array/src/planner.rs @@ -19,10 +19,11 @@ use datafusion_common::{utils::list_ndims, DFSchema, Result}; use datafusion_expr::{ - planner::{PlannerResult, RawBinaryExpr, RawFieldAccessExpr, UserDefinedSQLPlanner}, + planner::{ExprPlanner, PlannerResult, RawBinaryExpr, RawFieldAccessExpr}, sqlparser, AggregateFunction, Expr, ExprSchemable, GetFieldAccess, }; use datafusion_functions::expr_fn::get_field; +use datafusion_functions_aggregate::nth_value::nth_value_udaf; use crate::{ array_has::array_has_all, @@ -33,7 +34,7 @@ use crate::{ pub struct ArrayFunctionPlanner; -impl UserDefinedSQLPlanner for ArrayFunctionPlanner { +impl ExprPlanner for ArrayFunctionPlanner { fn plan_binary_op( &self, expr: RawBinaryExpr, @@ -100,7 +101,7 @@ impl UserDefinedSQLPlanner for ArrayFunctionPlanner { pub struct FieldAccessPlanner; -impl UserDefinedSQLPlanner for FieldAccessPlanner { +impl ExprPlanner for FieldAccessPlanner { fn plan_field_access( &self, expr: RawFieldAccessExpr, @@ -119,8 +120,8 @@ impl UserDefinedSQLPlanner for FieldAccessPlanner { // Special case for array_agg(expr)[index] to NTH_VALUE(expr, index) Expr::AggregateFunction(agg_func) if is_array_agg(&agg_func) => { Ok(PlannerResult::Planned(Expr::AggregateFunction( - datafusion_expr::expr::AggregateFunction::new( - AggregateFunction::NthValue, + datafusion_expr::expr::AggregateFunction::new_udf( + nth_value_udaf(), agg_func .args .into_iter() diff --git a/datafusion/functions/src/core/mod.rs b/datafusion/functions/src/core/mod.rs index 5761a4bd39cc..062a4a104d54 100644 --- a/datafusion/functions/src/core/mod.rs +++ b/datafusion/functions/src/core/mod.rs @@ -93,7 +93,6 @@ pub fn functions() -> Vec> { nvl(), nvl2(), arrow_typeof(), - r#struct(), named_struct(), get_field(), coalesce(), diff --git a/datafusion/functions/src/core/planner.rs b/datafusion/functions/src/core/planner.rs index 71f6c7f8ac72..748b598d292f 100644 --- a/datafusion/functions/src/core/planner.rs +++ b/datafusion/functions/src/core/planner.rs @@ -17,14 +17,16 @@ use datafusion_common::DFSchema; use datafusion_common::Result; -use datafusion_expr::planner::{PlannerResult, RawDictionaryExpr, UserDefinedSQLPlanner}; +use datafusion_expr::expr::ScalarFunction; +use datafusion_expr::planner::{ExprPlanner, PlannerResult, RawDictionaryExpr}; +use datafusion_expr::Expr; use super::named_struct; #[derive(Default)] pub struct CoreFunctionPlanner {} -impl UserDefinedSQLPlanner for CoreFunctionPlanner { +impl ExprPlanner for CoreFunctionPlanner { fn plan_dictionary_literal( &self, expr: RawDictionaryExpr, @@ -37,4 +39,21 @@ impl UserDefinedSQLPlanner for CoreFunctionPlanner { } Ok(PlannerResult::Planned(named_struct().call(args))) } + + fn plan_struct_literal( + &self, + args: Vec, + is_named_struct: bool, + ) -> Result>> { + Ok(PlannerResult::Planned(Expr::ScalarFunction( + ScalarFunction::new_udf( + if is_named_struct { + crate::core::named_struct() + } else { + crate::core::r#struct() + }, + args, + ), + ))) + } } diff --git a/datafusion/functions/src/planner.rs b/datafusion/functions/src/planner.rs index b00d5cf60810..ad42c5edd6e6 100644 --- a/datafusion/functions/src/planner.rs +++ b/datafusion/functions/src/planner.rs @@ -20,14 +20,14 @@ use datafusion_common::Result; use datafusion_expr::{ expr::ScalarFunction, - planner::{PlannerResult, UserDefinedSQLPlanner}, + planner::{ExprPlanner, PlannerResult}, Expr, }; #[derive(Default)] pub struct UserDefinedFunctionPlanner; -impl UserDefinedSQLPlanner for UserDefinedFunctionPlanner { +impl ExprPlanner for UserDefinedFunctionPlanner { #[cfg(feature = "datetime_expressions")] fn plan_extract(&self, args: Vec) -> Result>> { Ok(PlannerResult::Planned(Expr::ScalarFunction( @@ -41,4 +41,11 @@ impl UserDefinedSQLPlanner for UserDefinedFunctionPlanner { ScalarFunction::new_udf(crate::unicode::strpos(), args), ))) } + + #[cfg(feature = "unicode_expressions")] + fn plan_substring(&self, args: Vec) -> Result>> { + Ok(PlannerResult::Planned(Expr::ScalarFunction( + ScalarFunction::new_udf(crate::unicode::substr(), args), + ))) + } } diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 64d9c508f3f6..6c08b3e998b3 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -146,7 +146,7 @@ impl<'a> TypeCoercionRewriter<'a> { .map(|(lhs, rhs)| { // coerce the arguments as though they were a single binary equality // expression - let (lhs, rhs) = self.coerce_binary_op(lhs, &Operator::Eq, rhs)?; + let (lhs, rhs) = self.coerce_binary_op(lhs, Operator::Eq, rhs)?; Ok((lhs, rhs)) }) .collect::>>()?; @@ -157,12 +157,12 @@ impl<'a> TypeCoercionRewriter<'a> { fn coerce_binary_op( &self, left: Expr, - op: &Operator, + op: Operator, right: Expr, ) -> Result<(Expr, Expr)> { let (left_type, right_type) = get_input_types( &left.get_type(self.schema)?, - op, + &op, &right.get_type(self.schema)?, )?; Ok(( @@ -279,7 +279,7 @@ impl<'a> TreeNodeRewriter for TypeCoercionRewriter<'a> { )))) } Expr::BinaryExpr(BinaryExpr { left, op, right }) => { - let (left, right) = self.coerce_binary_op(*left, &op, *right)?; + let (left, right) = self.coerce_binary_op(*left, op, *right)?; Ok(Transformed::yes(Expr::BinaryExpr(BinaryExpr::new( Box::new(left), op, diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index cebae410f309..4a4933fe9cfd 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -191,24 +191,19 @@ impl CommonSubexprEliminate { id_array: &mut IdArray<'n>, expr_mask: ExprMask, ) -> Result { - // Don't consider volatile expressions for CSE. - Ok(if expr.is_volatile()? { - false - } else { - let mut visitor = ExprIdentifierVisitor { - expr_stats, - id_array, - visit_stack: vec![], - down_index: 0, - up_index: 0, - expr_mask, - random_state: &self.random_state, - found_common: false, - }; - expr.visit(&mut visitor)?; + let mut visitor = ExprIdentifierVisitor { + expr_stats, + id_array, + visit_stack: vec![], + down_index: 0, + up_index: 0, + expr_mask, + random_state: &self.random_state, + found_common: false, + }; + expr.visit(&mut visitor)?; - visitor.found_common - }) + Ok(visitor.found_common) } /// Rewrites `exprs_list` with common sub-expressions replaced with a new @@ -917,27 +912,50 @@ struct ExprIdentifierVisitor<'a, 'n> { /// Record item that used when traversing an expression tree. enum VisitRecord<'n> { - /// Contains the post-order index assigned in during the first, visiting traversal and - /// a boolean flag to indicate if the record marks an expression subtree (not just a - /// single node). + /// Marks the beginning of expression. It contains: + /// - The post-order index assigned during the first, visiting traversal. + /// - A boolean flag if the record marks an expression subtree (not just a single + /// node). EnterMark(usize, bool), - /// Accumulated identifier of sub expression. - ExprItem(Identifier<'n>), + + /// Marks an accumulated subexpression tree. It contains: + /// - The accumulated identifier of a subexpression. + /// - A boolean flag if the expression is valid for subexpression elimination. + /// The flag is propagated up from children to parent. (E.g. volatile expressions + /// are not valid and can't be extracted, but non-volatile children of volatile + /// expressions can be extracted.) + ExprItem(Identifier<'n>, bool), } impl<'n> ExprIdentifierVisitor<'_, 'n> { - /// Find the first `EnterMark` in the stack, and accumulates every `ExprItem` - /// before it. - fn pop_enter_mark(&mut self) -> (usize, bool, Option>) { + /// Find the first `EnterMark` in the stack, and accumulates every `ExprItem` before + /// it. Returns a tuple that contains: + /// - The pre-order index of the expression we marked. + /// - A boolean flag if we marked an expression subtree (not just a single node). + /// If true we didn't recurse into the node's children, so we need to calculate the + /// hash of the marked expression tree (not just the node) and we need to validate + /// the expression tree (not just the node). + /// - The accumulated identifier of the children of the marked expression. + /// - An accumulated boolean flag from the children of the marked expression if all + /// children are valid for subexpression elimination (i.e. it is safe to extract the + /// expression as a common expression from its children POV). + /// (E.g. if any of the children of the marked expression is not valid (e.g. is + /// volatile) then the expression is also not valid, so we can propagate this + /// information up from children to parents via `visit_stack` during the first, + /// visiting traversal and no need to test the expression's validity beforehand with + /// an extra traversal). + fn pop_enter_mark(&mut self) -> (usize, bool, Option>, bool) { let mut expr_id = None; + let mut is_valid = true; while let Some(item) = self.visit_stack.pop() { match item { - VisitRecord::EnterMark(down_index, tree) => { - return (down_index, tree, expr_id); + VisitRecord::EnterMark(down_index, is_tree) => { + return (down_index, is_tree, expr_id, is_valid); } - VisitRecord::ExprItem(id) => { - expr_id = Some(id.combine(expr_id)); + VisitRecord::ExprItem(sub_expr_id, sub_expr_is_valid) => { + expr_id = Some(sub_expr_id.combine(expr_id)); + is_valid &= sub_expr_is_valid; } } } @@ -949,8 +967,6 @@ impl<'n> TreeNodeVisitor<'n> for ExprIdentifierVisitor<'_, 'n> { type Node = Expr; fn f_down(&mut self, expr: &'n Expr) -> Result { - // TODO: consider non-volatile sub-expressions for CSE - // If an expression can short circuit its children then don't consider its // children for CSE (https://github.com/apache/arrow-datafusion/issues/8814). // This means that we don't recurse into its children, but handle the expression @@ -972,13 +988,22 @@ impl<'n> TreeNodeVisitor<'n> for ExprIdentifierVisitor<'_, 'n> { } fn f_up(&mut self, expr: &'n Expr) -> Result { - let (down_index, is_tree, sub_expr_id) = self.pop_enter_mark(); + let (down_index, is_tree, sub_expr_id, sub_expr_is_valid) = self.pop_enter_mark(); - let expr_id = - Identifier::new(expr, is_tree, self.random_state).combine(sub_expr_id); + let (expr_id, is_valid) = if is_tree { + ( + Identifier::new(expr, true, self.random_state), + !expr.is_volatile()?, + ) + } else { + ( + Identifier::new(expr, false, self.random_state).combine(sub_expr_id), + !expr.is_volatile_node() && sub_expr_is_valid, + ) + }; self.id_array[down_index].0 = self.up_index; - if !self.expr_mask.ignores(expr) { + if is_valid && !self.expr_mask.ignores(expr) { self.id_array[down_index].1 = Some(expr_id); let count = self.expr_stats.entry(expr_id).or_insert(0); *count += 1; @@ -986,7 +1011,8 @@ impl<'n> TreeNodeVisitor<'n> for ExprIdentifierVisitor<'_, 'n> { self.found_common = true; } } - self.visit_stack.push(VisitRecord::ExprItem(expr_id)); + self.visit_stack + .push(VisitRecord::ExprItem(expr_id, is_valid)); self.up_index += 1; Ok(TreeNodeRecursion::Continue) @@ -1101,6 +1127,7 @@ fn replace_common_expr<'n>( #[cfg(test)] mod test { + use std::any::Any; use std::collections::HashSet; use std::iter; @@ -1108,8 +1135,9 @@ mod test { use datafusion_expr::expr::AggregateFunction; use datafusion_expr::logical_plan::{table_scan, JoinType}; use datafusion_expr::{ - grouping_set, AccumulatorFactoryFunction, AggregateUDF, BinaryExpr, Signature, - SimpleAggregateUDF, Volatility, + grouping_set, AccumulatorFactoryFunction, AggregateUDF, BinaryExpr, + ColumnarValue, ScalarUDF, ScalarUDFImpl, Signature, SimpleAggregateUDF, + Volatility, }; use datafusion_expr::{lit, logical_plan::builder::LogicalPlanBuilder}; @@ -1838,4 +1866,96 @@ mod test { Ok(()) } + + #[test] + fn test_volatile() -> Result<()> { + let table_scan = test_table_scan()?; + + let extracted_child = col("a") + col("b"); + let rand = rand_func().call(vec![]); + let not_extracted_volatile = extracted_child + rand; + let plan = LogicalPlanBuilder::from(table_scan.clone()) + .project(vec![ + not_extracted_volatile.clone().alias("c1"), + not_extracted_volatile.alias("c2"), + ])? + .build()?; + + let expected = "Projection: __common_expr_1 + random() AS c1, __common_expr_1 + random() AS c2\ + \n Projection: test.a + test.b AS __common_expr_1, test.a, test.b, test.c\ + \n TableScan: test"; + + assert_optimized_plan_eq(expected, plan, None); + + Ok(()) + } + + #[test] + fn test_volatile_short_circuits() -> Result<()> { + let table_scan = test_table_scan()?; + + let rand = rand_func().call(vec![]); + let not_extracted_volatile_short_circuit_2 = + rand.clone().eq(lit(0)).or(col("b").eq(lit(0))); + let not_extracted_volatile_short_circuit_1 = + col("a").eq(lit(0)).or(rand.eq(lit(0))); + let plan = LogicalPlanBuilder::from(table_scan.clone()) + .project(vec![ + not_extracted_volatile_short_circuit_1.clone().alias("c1"), + not_extracted_volatile_short_circuit_1.alias("c2"), + not_extracted_volatile_short_circuit_2.clone().alias("c3"), + not_extracted_volatile_short_circuit_2.alias("c4"), + ])? + .build()?; + + let expected = "Projection: test.a = Int32(0) OR random() = Int32(0) AS c1, test.a = Int32(0) OR random() = Int32(0) AS c2, random() = Int32(0) OR test.b = Int32(0) AS c3, random() = Int32(0) OR test.b = Int32(0) AS c4\ + \n TableScan: test"; + + assert_non_optimized_plan_eq(expected, plan, None); + + Ok(()) + } + + /// returns a "random" function that is marked volatile (aka each invocation + /// returns a different value) + /// + /// Does not use datafusion_functions::rand to avoid introducing a + /// dependency on that crate. + fn rand_func() -> ScalarUDF { + ScalarUDF::new_from_impl(RandomStub::new()) + } + + #[derive(Debug)] + struct RandomStub { + signature: Signature, + } + + impl RandomStub { + fn new() -> Self { + Self { + signature: Signature::exact(vec![], Volatility::Volatile), + } + } + } + impl ScalarUDFImpl for RandomStub { + fn as_any(&self) -> &dyn Any { + self + } + + fn name(&self) -> &str { + "random" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Float64) + } + + fn invoke(&self, _args: &[ColumnarValue]) -> Result { + unimplemented!() + } + } } diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index ed3fd75f3efd..5da727cb5990 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -69,8 +69,8 @@ pub static POWS_OF_TEN: [i128; 38] = [ /// expressions. Such as: (A AND B) AND C pub fn expr_contains(expr: &Expr, needle: &Expr, search_op: Operator) -> bool { match expr { - Expr::BinaryExpr(BinaryExpr { left, op, right }) if op == &search_op => { - expr_contains(left, needle, search_op.clone()) + Expr::BinaryExpr(BinaryExpr { left, op, right }) if *op == search_op => { + expr_contains(left, needle, search_op) || expr_contains(right, needle, search_op) } _ => expr == needle, @@ -88,7 +88,7 @@ pub fn delete_xor_in_complex_expr(expr: &Expr, needle: &Expr, is_left: bool) -> ) -> Expr { match expr { Expr::BinaryExpr(BinaryExpr { left, op, right }) - if op == &Operator::BitwiseXor => + if *op == Operator::BitwiseXor => { let left_expr = recursive_delete_xor_in_expr(left, needle, xor_counter); let right_expr = recursive_delete_xor_in_expr(right, needle, xor_counter); @@ -102,7 +102,7 @@ pub fn delete_xor_in_complex_expr(expr: &Expr, needle: &Expr, is_left: bool) -> Expr::BinaryExpr(BinaryExpr::new( Box::new(left_expr), - op.clone(), + *op, Box::new(right_expr), )) } diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 0549845430a6..05b1744d90c5 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -177,13 +177,13 @@ pub fn split_conjunction_owned(expr: Expr) -> Vec { /// ]; /// /// // use split_binary_owned to split them -/// assert_eq!(split_binary_owned(expr, &Operator::Plus), split); +/// assert_eq!(split_binary_owned(expr, Operator::Plus), split); /// ``` #[deprecated( since = "34.0.0", note = "use `datafusion_expr::utils::split_binary_owned` instead" )] -pub fn split_binary_owned(expr: Expr, op: &Operator) -> Vec { +pub fn split_binary_owned(expr: Expr, op: Operator) -> Vec { expr_utils::split_binary_owned(expr, op) } @@ -194,7 +194,7 @@ pub fn split_binary_owned(expr: Expr, op: &Operator) -> Vec { since = "34.0.0", note = "use `datafusion_expr::utils::split_binary` instead" )] -pub fn split_binary<'a>(expr: &'a Expr, op: &Operator) -> Vec<&'a Expr> { +pub fn split_binary(expr: &Expr, op: Operator) -> Vec<&Expr> { expr_utils::split_binary(expr, op) } diff --git a/datafusion/physical-expr-common/src/aggregate/merge_arrays.rs b/datafusion/physical-expr-common/src/aggregate/merge_arrays.rs new file mode 100644 index 000000000000..544bdc182829 --- /dev/null +++ b/datafusion/physical-expr-common/src/aggregate/merge_arrays.rs @@ -0,0 +1,195 @@ +// 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::compute::SortOptions; +use datafusion_common::utils::compare_rows; +use datafusion_common::{exec_err, ScalarValue}; +use std::cmp::Ordering; +use std::collections::{BinaryHeap, VecDeque}; + +/// This is a wrapper struct to be able to correctly merge `ARRAY_AGG` data from +/// multiple partitions using `BinaryHeap`. When used inside `BinaryHeap`, this +/// struct returns smallest `CustomElement`, where smallest is determined by +/// `ordering` values (`Vec`) according to `sort_options`. +#[derive(Debug, PartialEq, Eq)] +struct CustomElement<'a> { + /// Stores the partition this entry came from + branch_idx: usize, + /// Values to merge + value: ScalarValue, + // Comparison "key" + ordering: Vec, + /// Options defining the ordering semantics + sort_options: &'a [SortOptions], +} + +impl<'a> CustomElement<'a> { + fn new( + branch_idx: usize, + value: ScalarValue, + ordering: Vec, + sort_options: &'a [SortOptions], + ) -> Self { + Self { + branch_idx, + value, + ordering, + sort_options, + } + } + + fn ordering( + &self, + current: &[ScalarValue], + target: &[ScalarValue], + ) -> datafusion_common::Result { + // Calculate ordering according to `sort_options` + compare_rows(current, target, self.sort_options) + } +} + +// 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> { + fn cmp(&self, other: &Self) -> Ordering { + // Compares according to custom ordering + self.ordering(&self.ordering, &other.ordering) + // Convert max heap to min heap + .map(|ordering| ordering.reverse()) + // This function return error, when `self.ordering` and `other.ordering` + // have different types (such as one is `ScalarValue::Int64`, other is `ScalarValue::Float32`) + // Here this case won't happen, because data from each partition will have same type + .unwrap() + } +} + +impl<'a> PartialOrd for CustomElement<'a> { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +/// This functions merges `values` array (`&[Vec]`) into single array `Vec` +/// Merging done according to ordering values stored inside `ordering_values` (`&[Vec>]`) +/// Inner `Vec` in the `ordering_values` can be thought as ordering information for the +/// each `ScalarValue` in the `values` array. +/// Desired ordering specified by `sort_options` argument (Should have same size with inner `Vec` +/// of the `ordering_values` array). +/// +/// As an example +/// values can be \[ +/// \[1, 2, 3, 4, 5\], +/// \[1, 2, 3, 4\], +/// \[1, 2, 3, 4, 5, 6\], +/// \] +/// In this case we will be merging three arrays (doesn't have to be same size) +/// and produce a merged array with size 15 (sum of 5+4+6) +/// Merging will be done according to ordering at `ordering_values` vector. +/// As an example `ordering_values` can be [ +/// \[(1, a), (2, b), (3, b), (4, a), (5, b) \], +/// \[(1, a), (2, b), (3, b), (4, a) \], +/// \[(1, b), (2, c), (3, d), (4, e), (5, a), (6, b) \], +/// ] +/// For each ScalarValue in the `values` we have a corresponding `Vec` (like timestamp of it) +/// for the example above `sort_options` will have size two, that defines ordering requirement of the merge. +/// Inner `Vec`s of the `ordering_values` will be compared according `sort_options` (Their sizes should match) +pub fn merge_ordered_arrays( + // We will merge values into single `Vec`. + values: &mut [VecDeque], + // `values` will be merged according to `ordering_values`. + // Inner `Vec` can be thought as ordering information for the + // each `ScalarValue` in the values`. + ordering_values: &mut [VecDeque>], + // Defines according to which ordering comparisons should be done. + sort_options: &[SortOptions], +) -> datafusion_common::Result<(Vec, Vec>)> { + // Keep track the most recent data of each branch, in binary heap data structure. + let mut heap = BinaryHeap::::new(); + + if values.len() != ordering_values.len() + || values + .iter() + .zip(ordering_values.iter()) + .any(|(vals, ordering_vals)| vals.len() != ordering_vals.len()) + { + return exec_err!( + "Expects values arguments and/or ordering_values arguments to have same size" + ); + } + let n_branch = values.len(); + let mut merged_values = vec![]; + let mut merged_orderings = vec![]; + // Continue iterating the loop until consuming data of all branches. + loop { + let minimum = if let Some(minimum) = heap.pop() { + minimum + } else { + // Heap is empty, fill it with the next entries from each branch. + for branch_idx in 0..n_branch { + if let Some(orderings) = ordering_values[branch_idx].pop_front() { + // Their size should be same, we can safely .unwrap here. + let value = values[branch_idx].pop_front().unwrap(); + // Push the next element to the heap: + heap.push(CustomElement::new( + branch_idx, + value, + orderings, + sort_options, + )); + } + // If None, we consumed this branch, skip it. + } + + // Now we have filled the heap, get the largest entry (this will be + // the next element in merge). + if let Some(minimum) = heap.pop() { + minimum + } else { + // Heap is empty, this means that all indices are same with + // `end_indices`. We have consumed all of the branches, merge + // is completed, exit from the loop: + break; + } + }; + let CustomElement { + branch_idx, + value, + ordering, + .. + } = minimum; + // Add minimum value in the heap to the result + merged_values.push(value); + merged_orderings.push(ordering); + + // If there is an available entry, push next entry in the most + // recently consumed branch to the heap. + if let Some(orderings) = ordering_values[branch_idx].pop_front() { + // Their size should be same, we can safely .unwrap here. + let value = values[branch_idx].pop_front().unwrap(); + // Push the next element to the heap: + heap.push(CustomElement::new( + branch_idx, + value, + orderings, + sort_options, + )); + } + } + + Ok((merged_values, merged_orderings)) +} diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index cd309b7f7d29..35666f199ace 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -17,6 +17,7 @@ pub mod count_distinct; pub mod groups_accumulator; +pub mod merge_arrays; pub mod stats; pub mod tdigest; pub mod utils; diff --git a/datafusion/physical-expr-common/src/datum.rs b/datafusion/physical-expr-common/src/datum.rs index 790e742c4221..d0ba5f113b6f 100644 --- a/datafusion/physical-expr-common/src/datum.rs +++ b/datafusion/physical-expr-common/src/datum.rs @@ -63,7 +63,7 @@ pub fn apply_cmp( /// Applies a binary [`Datum`] comparison kernel `f` to `lhs` and `rhs` for nested type like /// List, FixedSizeList, LargeList, Struct, Union, Map, or a dictionary of a nested type pub fn apply_cmp_for_nested( - op: &Operator, + op: Operator, lhs: &ColumnarValue, rhs: &ColumnarValue, ) -> Result { @@ -88,7 +88,7 @@ pub fn apply_cmp_for_nested( /// Compare on nested type List, Struct, and so on pub fn compare_op_for_nested( - op: &Operator, + op: Operator, lhs: &dyn Datum, rhs: &dyn Datum, ) -> Result { diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 3b122fe9f82b..a64d97637c3b 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -19,8 +19,7 @@ //! that can evaluated at runtime during query execution use std::any::Any; -use std::cmp::Ordering; -use std::collections::{BinaryHeap, VecDeque}; +use std::collections::VecDeque; use std::fmt::Debug; use std::sync::Arc; @@ -33,11 +32,12 @@ use crate::{ use arrow::datatypes::{DataType, Field}; use arrow_array::cast::AsArray; use arrow_array::{new_empty_array, Array, ArrayRef, StructArray}; -use arrow_schema::{Fields, SortOptions}; -use datafusion_common::utils::{array_into_list_array, compare_rows, get_row_at_idx}; +use arrow_schema::Fields; +use datafusion_common::utils::{array_into_list_array, get_row_at_idx}; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::utils::AggregateOrderSensitivity; use datafusion_expr::Accumulator; +use datafusion_physical_expr_common::aggregate::merge_arrays::merge_ordered_arrays; /// Expression for a `ARRAY_AGG(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, @@ -384,179 +384,6 @@ impl OrderSensitiveArrayAggAccumulator { } } -/// This is a wrapper struct to be able to correctly merge `ARRAY_AGG` data from -/// multiple partitions using `BinaryHeap`. When used inside `BinaryHeap`, this -/// struct returns smallest `CustomElement`, where smallest is determined by -/// `ordering` values (`Vec`) according to `sort_options`. -#[derive(Debug, PartialEq, Eq)] -struct CustomElement<'a> { - /// Stores the partition this entry came from - branch_idx: usize, - /// Values to merge - value: ScalarValue, - // Comparison "key" - ordering: Vec, - /// Options defining the ordering semantics - sort_options: &'a [SortOptions], -} - -impl<'a> CustomElement<'a> { - fn new( - branch_idx: usize, - value: ScalarValue, - ordering: Vec, - sort_options: &'a [SortOptions], - ) -> Self { - Self { - branch_idx, - value, - ordering, - sort_options, - } - } - - fn ordering( - &self, - current: &[ScalarValue], - target: &[ScalarValue], - ) -> Result { - // Calculate ordering according to `sort_options` - compare_rows(current, target, self.sort_options) - } -} - -// 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> { - fn cmp(&self, other: &Self) -> Ordering { - // Compares according to custom ordering - self.ordering(&self.ordering, &other.ordering) - // Convert max heap to min heap - .map(|ordering| ordering.reverse()) - // This function return error, when `self.ordering` and `other.ordering` - // have different types (such as one is `ScalarValue::Int64`, other is `ScalarValue::Float32`) - // Here this case won't happen, because data from each partition will have same type - .unwrap() - } -} - -impl<'a> PartialOrd for CustomElement<'a> { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -/// This functions merges `values` array (`&[Vec]`) into single array `Vec` -/// Merging done according to ordering values stored inside `ordering_values` (`&[Vec>]`) -/// Inner `Vec` in the `ordering_values` can be thought as ordering information for the -/// each `ScalarValue` in the `values` array. -/// Desired ordering specified by `sort_options` argument (Should have same size with inner `Vec` -/// of the `ordering_values` array). -/// -/// As an example -/// values can be \[ -/// \[1, 2, 3, 4, 5\], -/// \[1, 2, 3, 4\], -/// \[1, 2, 3, 4, 5, 6\], -/// \] -/// In this case we will be merging three arrays (doesn't have to be same size) -/// and produce a merged array with size 15 (sum of 5+4+6) -/// Merging will be done according to ordering at `ordering_values` vector. -/// As an example `ordering_values` can be [ -/// \[(1, a), (2, b), (3, b), (4, a), (5, b) \], -/// \[(1, a), (2, b), (3, b), (4, a) \], -/// \[(1, b), (2, c), (3, d), (4, e), (5, a), (6, b) \], -/// ] -/// For each ScalarValue in the `values` we have a corresponding `Vec` (like timestamp of it) -/// for the example above `sort_options` will have size two, that defines ordering requirement of the merge. -/// Inner `Vec`s of the `ordering_values` will be compared according `sort_options` (Their sizes should match) -pub(crate) fn merge_ordered_arrays( - // We will merge values into single `Vec`. - values: &mut [VecDeque], - // `values` will be merged according to `ordering_values`. - // Inner `Vec` can be thought as ordering information for the - // each `ScalarValue` in the values`. - ordering_values: &mut [VecDeque>], - // Defines according to which ordering comparisons should be done. - sort_options: &[SortOptions], -) -> Result<(Vec, Vec>)> { - // Keep track the most recent data of each branch, in binary heap data structure. - let mut heap = BinaryHeap::::new(); - - if values.len() != ordering_values.len() - || values - .iter() - .zip(ordering_values.iter()) - .any(|(vals, ordering_vals)| vals.len() != ordering_vals.len()) - { - return exec_err!( - "Expects values arguments and/or ordering_values arguments to have same size" - ); - } - let n_branch = values.len(); - let mut merged_values = vec![]; - let mut merged_orderings = vec![]; - // Continue iterating the loop until consuming data of all branches. - loop { - let minimum = if let Some(minimum) = heap.pop() { - minimum - } else { - // Heap is empty, fill it with the next entries from each branch. - for branch_idx in 0..n_branch { - if let Some(orderings) = ordering_values[branch_idx].pop_front() { - // Their size should be same, we can safely .unwrap here. - let value = values[branch_idx].pop_front().unwrap(); - // Push the next element to the heap: - heap.push(CustomElement::new( - branch_idx, - value, - orderings, - sort_options, - )); - } - // If None, we consumed this branch, skip it. - } - - // Now we have filled the heap, get the largest entry (this will be - // the next element in merge). - if let Some(minimum) = heap.pop() { - minimum - } else { - // Heap is empty, this means that all indices are same with - // `end_indices`. We have consumed all of the branches, merge - // is completed, exit from the loop: - break; - } - }; - let CustomElement { - branch_idx, - value, - ordering, - .. - } = minimum; - // Add minimum value in the heap to the result - merged_values.push(value); - merged_orderings.push(ordering); - - // If there is an available entry, push next entry in the most - // recently consumed branch to the heap. - if let Some(orderings) = ordering_values[branch_idx].pop_front() { - // Their size should be same, we can safely .unwrap here. - let value = values[branch_idx].pop_front().unwrap(); - // Push the next element to the heap: - heap.push(CustomElement::new( - branch_idx, - value, - orderings, - sort_options, - )); - } - } - - Ok((merged_values, merged_orderings)) -} - #[cfg(test)] mod tests { use std::collections::VecDeque; diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 1eadf7247f7c..d4cd3d51d174 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -30,10 +30,10 @@ use std::sync::Arc; use arrow::datatypes::Schema; -use datafusion_common::{exec_err, not_impl_err, Result}; +use datafusion_common::{not_impl_err, Result}; use datafusion_expr::AggregateFunction; -use crate::expressions::{self, Literal}; +use crate::expressions::{self}; use crate::{AggregateExpr, PhysicalExpr, PhysicalSortExpr}; /// Create a physical aggregation expression. @@ -102,26 +102,6 @@ pub fn create_aggregate_expr( name, data_type, )), - (AggregateFunction::NthValue, _) => { - let expr = &input_phy_exprs[0]; - let Some(n) = input_phy_exprs[1] - .as_any() - .downcast_ref::() - .map(|literal| literal.value()) - else { - return exec_err!("Second argument of NTH_VALUE needs to be a literal"); - }; - let nullable = expr.nullable(input_schema)?; - Arc::new(expressions::NthValueAgg::new( - Arc::clone(expr), - n.clone().try_into()?, - name, - input_phy_types[0].clone(), - nullable, - ordering_types, - ordering_req.to_vec(), - )) - } }) } diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index f0de7446f6f1..b9d803900f53 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -20,7 +20,6 @@ pub use datafusion_physical_expr_common::aggregate::AggregateExpr; pub(crate) mod array_agg; pub(crate) mod array_agg_distinct; pub(crate) mod array_agg_ordered; -pub(crate) mod nth_value; #[macro_use] pub(crate) mod min_max; pub(crate) mod groups_accumulator; diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index f1e40575bc64..c153ead9639f 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -269,7 +269,7 @@ impl PhysicalExpr for BinaryExpr { if right_data_type != left_data_type { return internal_err!("type mismatch"); } - return apply_cmp_for_nested(&self.op, &lhs, &rhs); + return apply_cmp_for_nested(self.op, &lhs, &rhs); } match self.op { @@ -329,7 +329,7 @@ impl PhysicalExpr for BinaryExpr { ) -> Result> { Ok(Arc::new(BinaryExpr::new( Arc::clone(&children[0]), - self.op.clone(), + self.op, Arc::clone(&children[1]), ))) } diff --git a/datafusion/physical-expr/src/expressions/is_not_null.rs b/datafusion/physical-expr/src/expressions/is_not_null.rs index d8fa77585b5d..9f7438d13e05 100644 --- a/datafusion/physical-expr/src/expressions/is_not_null.rs +++ b/datafusion/physical-expr/src/expressions/is_not_null.rs @@ -73,9 +73,11 @@ impl PhysicalExpr for IsNotNullExpr { fn evaluate(&self, batch: &RecordBatch) -> Result { let arg = self.arg.evaluate(batch)?; match arg { - ColumnarValue::Array(array) => Ok(ColumnarValue::Array(Arc::new( - compute::is_not_null(array.as_ref())?, - ))), + ColumnarValue::Array(array) => { + let is_null = super::is_null::compute_is_null(array)?; + let is_not_null = compute::not(&is_null)?; + Ok(ColumnarValue::Array(Arc::new(is_not_null))) + } ColumnarValue::Scalar(scalar) => Ok(ColumnarValue::Scalar( ScalarValue::Boolean(Some(!scalar.is_null())), )), @@ -120,6 +122,8 @@ mod tests { array::{BooleanArray, StringArray}, datatypes::*, }; + use arrow_array::{Array, Float64Array, Int32Array, UnionArray}; + use arrow_buffer::ScalarBuffer; use datafusion_common::cast::as_boolean_array; #[test] @@ -143,4 +147,48 @@ mod tests { Ok(()) } + + #[test] + fn union_is_not_null_op() { + // union of [{A=1}, {A=}, {B=1.1}, {B=1.2}, {B=}] + let int_array = Int32Array::from(vec![Some(1), None, None, None, None]); + let float_array = + Float64Array::from(vec![None, None, Some(1.1), Some(1.2), None]); + let type_ids = [0, 0, 1, 1, 1].into_iter().collect::>(); + + let children = vec![Arc::new(int_array) as Arc, Arc::new(float_array)]; + + let union_fields: UnionFields = [ + (0, Arc::new(Field::new("A", DataType::Int32, true))), + (1, Arc::new(Field::new("B", DataType::Float64, true))), + ] + .into_iter() + .collect(); + + let array = + UnionArray::try_new(union_fields.clone(), type_ids, None, children).unwrap(); + + let field = Field::new( + "my_union", + DataType::Union(union_fields, UnionMode::Sparse), + true, + ); + + let schema = Schema::new(vec![field]); + let expr = is_not_null(col("my_union", &schema).unwrap()).unwrap(); + let batch = + RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)]).unwrap(); + + // expression: "a is not null" + let actual = expr + .evaluate(&batch) + .unwrap() + .into_array(batch.num_rows()) + .expect("Failed to convert to array"); + let actual = as_boolean_array(&actual).unwrap(); + + let expected = &BooleanArray::from(vec![true, false, true, true, false]); + + assert_eq!(expected, actual); + } } diff --git a/datafusion/physical-expr/src/expressions/is_null.rs b/datafusion/physical-expr/src/expressions/is_null.rs index 41becafde6de..e2dc941e26bc 100644 --- a/datafusion/physical-expr/src/expressions/is_null.rs +++ b/datafusion/physical-expr/src/expressions/is_null.rs @@ -25,6 +25,9 @@ use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; +use arrow_array::{Array, ArrayRef, BooleanArray, Int8Array, UnionArray}; +use arrow_buffer::{BooleanBuffer, ScalarBuffer}; +use arrow_ord::cmp; use crate::physical_expr::down_cast_any_ref; use crate::PhysicalExpr; @@ -74,9 +77,9 @@ impl PhysicalExpr for IsNullExpr { fn evaluate(&self, batch: &RecordBatch) -> Result { let arg = self.arg.evaluate(batch)?; match arg { - ColumnarValue::Array(array) => Ok(ColumnarValue::Array(Arc::new( - compute::is_null(array.as_ref())?, - ))), + ColumnarValue::Array(array) => { + Ok(ColumnarValue::Array(Arc::new(compute_is_null(array)?))) + } ColumnarValue::Scalar(scalar) => Ok(ColumnarValue::Scalar( ScalarValue::Boolean(Some(scalar.is_null())), )), @@ -100,6 +103,55 @@ impl PhysicalExpr for IsNullExpr { } } +/// workaround , +/// this can be replaced with a direct call to `arrow::compute::is_null` once it's fixed. +pub(crate) fn compute_is_null(array: ArrayRef) -> Result { + if let Some(union_array) = array.as_any().downcast_ref::() { + if let Some(offsets) = union_array.offsets() { + dense_union_is_null(union_array, offsets) + } else { + sparse_union_is_null(union_array) + } + } else { + compute::is_null(array.as_ref()).map_err(Into::into) + } +} + +fn dense_union_is_null( + union_array: &UnionArray, + offsets: &ScalarBuffer, +) -> Result { + let child_arrays = (0..union_array.type_names().len()) + .map(|type_id| { + compute::is_null(&union_array.child(type_id as i8)).map_err(Into::into) + }) + .collect::>>()?; + + let buffer: BooleanBuffer = offsets + .iter() + .zip(union_array.type_ids()) + .map(|(offset, type_id)| child_arrays[*type_id as usize].value(*offset as usize)) + .collect(); + + Ok(BooleanArray::new(buffer, None)) +} + +fn sparse_union_is_null(union_array: &UnionArray) -> Result { + let type_ids = Int8Array::new(union_array.type_ids().clone(), None); + + let mut union_is_null = + BooleanArray::new(BooleanBuffer::new_unset(union_array.len()), None); + for type_id in 0..union_array.type_names().len() { + let type_id = type_id as i8; + let union_is_child = cmp::eq(&type_ids, &Int8Array::new_scalar(type_id))?; + let child = union_array.child(type_id); + let child_array_is_null = compute::is_null(&child)?; + let child_is_null = compute::and(&union_is_child, &child_array_is_null)?; + union_is_null = compute::or(&union_is_null, &child_is_null)?; + } + Ok(union_is_null) +} + impl PartialEq for IsNullExpr { fn eq(&self, other: &dyn Any) -> bool { down_cast_any_ref(other) @@ -108,6 +160,7 @@ impl PartialEq for IsNullExpr { .unwrap_or(false) } } + /// Create an IS NULL expression pub fn is_null(arg: Arc) -> Result> { Ok(Arc::new(IsNullExpr::new(arg))) @@ -121,6 +174,8 @@ mod tests { array::{BooleanArray, StringArray}, datatypes::*, }; + use arrow_array::{Float64Array, Int32Array}; + use arrow_buffer::ScalarBuffer; use datafusion_common::cast::as_boolean_array; #[test] @@ -145,4 +200,72 @@ mod tests { Ok(()) } + + fn union_fields() -> UnionFields { + [ + (0, Arc::new(Field::new("A", DataType::Int32, true))), + (1, Arc::new(Field::new("B", DataType::Float64, true))), + (2, Arc::new(Field::new("C", DataType::Utf8, true))), + ] + .into_iter() + .collect() + } + + #[test] + fn sparse_union_is_null() { + // union of [{A=1}, {A=}, {B=1.1}, {B=1.2}, {B=}, {C=}, {C="a"}] + let int_array = + Int32Array::from(vec![Some(1), None, None, None, None, None, None]); + let float_array = + Float64Array::from(vec![None, None, Some(1.1), Some(1.2), None, None, None]); + let str_array = + StringArray::from(vec![None, None, None, None, None, None, Some("a")]); + let type_ids = [0, 0, 1, 1, 1, 2, 2] + .into_iter() + .collect::>(); + + let children = vec![ + Arc::new(int_array) as Arc, + Arc::new(float_array), + Arc::new(str_array), + ]; + + let array = + UnionArray::try_new(union_fields(), type_ids, None, children).unwrap(); + + let array_ref = Arc::new(array) as ArrayRef; + let result = compute_is_null(array_ref).unwrap(); + + let expected = + &BooleanArray::from(vec![false, true, false, false, true, true, false]); + assert_eq!(expected, &result); + } + + #[test] + fn dense_union_is_null() { + // union of [{A=1}, {A=}, {B=3.2}, {B=}, {C="a"}, {C=}] + let int_array = Int32Array::from(vec![Some(1), None]); + let float_array = Float64Array::from(vec![Some(3.2), None]); + let str_array = StringArray::from(vec![Some("a"), None]); + let type_ids = [0, 0, 1, 1, 2, 2].into_iter().collect::>(); + let offsets = [0, 1, 0, 1, 0, 1] + .into_iter() + .collect::>(); + + let children = vec![ + Arc::new(int_array) as Arc, + Arc::new(float_array), + Arc::new(str_array), + ]; + + let array = + UnionArray::try_new(union_fields(), type_ids, Some(offsets), children) + .unwrap(); + + let array_ref = Arc::new(array) as ArrayRef; + let result = compute_is_null(array_ref).unwrap(); + + let expected = &BooleanArray::from(vec![false, true, false, true, false, true]); + assert_eq!(expected, &result); + } } diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 1f2c955ad07e..7d8f12091f46 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -39,7 +39,6 @@ pub use crate::aggregate::array_agg_distinct::DistinctArrayAgg; pub use crate::aggregate::array_agg_ordered::OrderSensitiveArrayAgg; pub use crate::aggregate::build_in::create_aggregate_expr; pub use crate::aggregate::min_max::{Max, MaxAccumulator, Min, MinAccumulator}; -pub use crate::aggregate::nth_value::NthValueAgg; pub use crate::aggregate::stats::StatsType; pub use crate::window::cume_dist::{cume_dist, CumeDist}; pub use crate::window::lead_lag::{lag, lead, WindowShift}; diff --git a/datafusion/physical-expr/src/intervals/cp_solver.rs b/datafusion/physical-expr/src/intervals/cp_solver.rs index ef9dd36cfb50..fc4950ae4e7c 100644 --- a/datafusion/physical-expr/src/intervals/cp_solver.rs +++ b/datafusion/physical-expr/src/intervals/cp_solver.rs @@ -222,7 +222,7 @@ pub fn propagate_arithmetic( left_child: &Interval, right_child: &Interval, ) -> Result> { - let inverse_op = get_inverse_op(op)?; + let inverse_op = get_inverse_op(*op)?; match (left_child.data_type(), right_child.data_type()) { // If we have a child whose type is a time interval (i.e. DataType::Interval), // we need special handling since timestamp differencing results in a diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 37527802f84d..b426a656fba9 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -63,7 +63,7 @@ pub fn check_support(expr: &Arc, schema: &SchemaRef) -> bool { } // This function returns the inverse operator of the given operator. -pub fn get_inverse_op(op: &Operator) -> Result { +pub fn get_inverse_op(op: Operator) -> Result { match op { Operator::Plus => Ok(Operator::Minus), Operator::Minus => Ok(Operator::Plus), diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index dbebf4c18b79..a975f0c6ef83 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -195,7 +195,7 @@ pub fn create_physical_expr( // // There should be no coercion during physical // planning. - binary(lhs, op.clone(), rhs, input_schema) + binary(lhs, *op, rhs, input_schema) } Expr::Like(Like { negated, diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index a33f65f92a61..6c4791b158c8 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -44,7 +44,7 @@ use petgraph::stable_graph::StableGraph; pub fn split_conjunction( predicate: &Arc, ) -> Vec<&Arc> { - split_impl(&Operator::And, predicate, vec![]) + split_impl(Operator::And, predicate, vec![]) } /// Assume the predicate is in the form of DNF, split the predicate to a Vec of PhysicalExprs. @@ -53,16 +53,16 @@ pub fn split_conjunction( pub fn split_disjunction( predicate: &Arc, ) -> Vec<&Arc> { - split_impl(&Operator::Or, predicate, vec![]) + split_impl(Operator::Or, predicate, vec![]) } fn split_impl<'a>( - operator: &Operator, + operator: Operator, predicate: &'a Arc, mut exprs: Vec<&'a Arc>, ) -> Vec<&'a Arc> { match predicate.as_any().downcast_ref::() { - Some(binary) if binary.op() == operator => { + Some(binary) if binary.op() == &operator => { let exprs = split_impl(operator, binary.left(), exprs); split_impl(operator, binary.right(), exprs) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6a0ae202c067..8caf10acf09b 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -188,7 +188,7 @@ impl PhysicalGroupBy { pub fn input_exprs(&self) -> Vec> { self.expr .iter() - .map(|(expr, _alias)| expr.clone()) + .map(|(expr, _alias)| Arc::clone(expr)) .collect() } @@ -283,9 +283,9 @@ impl AggregateExec { group_by: self.group_by.clone(), filter_expr: self.filter_expr.clone(), limit: self.limit, - input: self.input.clone(), - schema: self.schema.clone(), - input_schema: self.input_schema.clone(), + input: Arc::clone(&self.input), + schema: Arc::clone(&self.schema), + input_schema: Arc::clone(&self.input_schema), } } @@ -355,7 +355,7 @@ impl AggregateExec { let mut new_requirement = indices .iter() .map(|&idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), + expr: Arc::clone(&groupby_exprs[idx]), options: None, }) .collect::>(); @@ -399,7 +399,7 @@ impl AggregateExec { let cache = Self::compute_properties( &input, - schema.clone(), + Arc::clone(&schema), &projection_mapping, &mode, &input_order_mode, @@ -458,7 +458,7 @@ impl AggregateExec { /// Get the input schema before any aggregates are applied pub fn input_schema(&self) -> SchemaRef { - self.input_schema.clone() + Arc::clone(&self.input_schema) } /// number of rows soft limit of the AggregateExec @@ -706,9 +706,9 @@ impl ExecutionPlan for AggregateExec { self.group_by.clone(), self.aggr_expr.clone(), self.filter_expr.clone(), - children[0].clone(), - self.input_schema.clone(), - self.schema.clone(), + Arc::clone(&children[0]), + Arc::clone(&self.input_schema), + Arc::clone(&self.schema), )?; me.limit = self.limit; @@ -1005,7 +1005,7 @@ fn aggregate_expressions( // way order sensitive aggregators can satisfy requirement // themselves. if let Some(ordering_req) = agg.order_bys() { - result.extend(ordering_req.iter().map(|item| item.expr.clone())); + result.extend(ordering_req.iter().map(|item| Arc::clone(&item.expr))); } result }) @@ -1165,9 +1165,9 @@ pub(crate) fn evaluate_group_by( .enumerate() .map(|(idx, is_null)| { if *is_null { - null_exprs[idx].clone() + Arc::clone(&null_exprs[idx]) } else { - exprs[idx].clone() + Arc::clone(&exprs[idx]) } }) .collect() @@ -1235,10 +1235,10 @@ mod tests { // define data. ( - schema.clone(), + Arc::clone(&schema), vec![ RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(UInt32Array::from(vec![2, 3, 4, 4])), Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])), @@ -1270,10 +1270,10 @@ mod tests { // the expected result by accident, but merging actually works properly; // i.e. it doesn't depend on the data insertion order. ( - schema.clone(), + Arc::clone(&schema), vec![ RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(UInt32Array::from(vec![2, 3, 4, 4])), Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])), @@ -1281,7 +1281,7 @@ mod tests { ) .unwrap(), RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(UInt32Array::from(vec![2, 3, 3, 4])), Arc::new(Float64Array::from(vec![0.0, 1.0, 2.0, 3.0])), @@ -1289,7 +1289,7 @@ mod tests { ) .unwrap(), RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(UInt32Array::from(vec![2, 3, 3, 4])), Arc::new(Float64Array::from(vec![3.0, 4.0, 5.0, 6.0])), @@ -1369,11 +1369,11 @@ mod tests { aggregates.clone(), vec![None], input, - input_schema.clone(), + Arc::clone(&input_schema), )?); let result = - common::collect(partial_aggregate.execute(0, task_ctx.clone())?).await?; + common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; let expected = if spill { vec![ @@ -1451,7 +1451,7 @@ mod tests { )?); let result = - common::collect(merged_aggregate.execute(0, task_ctx.clone())?).await?; + common::collect(merged_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; let batch = concat_batches(&result[0].schema(), &result)?; assert_eq!(batch.num_columns(), 3); assert_eq!(batch.num_rows(), 12); @@ -1519,11 +1519,11 @@ mod tests { aggregates.clone(), vec![None], input, - input_schema.clone(), + Arc::clone(&input_schema), )?); let result = - common::collect(partial_aggregate.execute(0, task_ctx.clone())?).await?; + common::collect(partial_aggregate.execute(0, Arc::clone(&task_ctx))?).await?; let expected = if spill { vec![ @@ -1573,7 +1573,7 @@ mod tests { // enlarge memory limit to let the final aggregation finish new_spill_ctx(2, 2600) } else { - task_ctx.clone() + Arc::clone(&task_ctx) }; let result = common::collect(merged_aggregate.execute(0, task_ctx)?).await?; let batch = concat_batches(&result[0].schema(), &result)?; @@ -1856,11 +1856,11 @@ mod tests { groups, aggregates, vec![None; n_aggr], - input.clone(), - input_schema.clone(), + Arc::clone(&input), + Arc::clone(&input_schema), )?); - let stream = partial_aggregate.execute_typed(0, task_ctx.clone())?; + let stream = partial_aggregate.execute_typed(0, Arc::clone(&task_ctx))?; // ensure that we really got the version we wanted match version { @@ -2112,7 +2112,7 @@ mod tests { vec![partition3], vec![partition4], ], - schema.clone(), + Arc::clone(&schema), None, )?); let aggregate_exec = Arc::new(AggregateExec::try_new( @@ -2121,7 +2121,7 @@ mod tests { aggregates.clone(), vec![None], memory_exec, - schema.clone(), + Arc::clone(&schema), )?); let coalesce = if use_coalesce_batches { let coalesce = Arc::new(CoalescePartitionsExec::new(aggregate_exec)); @@ -2186,41 +2186,41 @@ mod tests { let order_by_exprs = vec![ None, Some(vec![PhysicalSortExpr { - expr: col_a.clone(), + expr: Arc::clone(col_a), options: options1, }]), Some(vec![ PhysicalSortExpr { - expr: col_a.clone(), + expr: Arc::clone(col_a), options: options1, }, PhysicalSortExpr { - expr: col_b.clone(), + expr: Arc::clone(col_b), options: options1, }, PhysicalSortExpr { - expr: col_c.clone(), + expr: Arc::clone(col_c), options: options1, }, ]), Some(vec![ PhysicalSortExpr { - expr: col_a.clone(), + expr: Arc::clone(col_a), options: options1, }, PhysicalSortExpr { - expr: col_b.clone(), + expr: Arc::clone(col_b), options: options1, }, ]), ]; let common_requirement = vec![ PhysicalSortExpr { - expr: col_a.clone(), + expr: Arc::clone(col_a), options: options1, }, PhysicalSortExpr { - expr: col_c.clone(), + expr: Arc::clone(col_c), options: options1, }, ]; @@ -2228,7 +2228,7 @@ mod tests { .into_iter() .map(|order_by_expr| { Arc::new(OrderSensitiveArrayAgg::new( - col_a.clone(), + Arc::clone(col_a), "array_agg", DataType::Int32, false, @@ -2273,12 +2273,11 @@ mod tests { groups, aggregates.clone(), vec![None, None], - blocking_exec.clone(), + Arc::clone(&blocking_exec) as Arc, schema, )?); - let new_agg = aggregate_exec - .clone() - .with_new_children(vec![blocking_exec])?; + let new_agg = + Arc::clone(&aggregate_exec).with_new_children(vec![blocking_exec])?; assert_eq!(new_agg.schema(), aggregate_exec.schema()); Ok(()) } @@ -2340,12 +2339,15 @@ mod tests { let b = Arc::new(Float32Array::from(vec![0.; 8192])); let c = Arc::new(Int32Array::from(vec![1; 8192])); - RecordBatch::try_new(schema.clone(), vec![a, b, c]).unwrap() + RecordBatch::try_new(Arc::clone(&schema), vec![a, b, c]).unwrap() }) .collect(); - let input = - Arc::new(MemoryExec::try_new(&[input_batches], schema.clone(), None)?); + let input = Arc::new(MemoryExec::try_new( + &[input_batches], + Arc::clone(&schema), + None, + )?); let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 5ec95bd79942..f85164f7f1e2 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -140,8 +140,11 @@ impl AggregateStream { let result = finalize_aggregation(&mut this.accumulators, &this.mode) .and_then(|columns| { - RecordBatch::try_new(this.schema.clone(), columns) - .map_err(Into::into) + RecordBatch::try_new( + Arc::clone(&this.schema), + columns, + ) + .map_err(Into::into) }) .record_output(&this.baseline_metrics); @@ -181,7 +184,7 @@ impl Stream for AggregateStream { impl RecordBatchStream for AggregateStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index ecd37c913e98..f8fd86ff8b50 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -22,6 +22,7 @@ use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; use datafusion_physical_expr::PhysicalSortExpr; +use std::sync::Arc; /// Tracks grouping state when the data is ordered by some subset of /// the group keys. @@ -138,7 +139,7 @@ impl GroupOrderingPartial { let sort_values: Vec<_> = self .order_indices .iter() - .map(|&idx| group_values[idx].clone()) + .map(|&idx| Arc::clone(&group_values[idx])) .collect(); Ok(self.row_converter.convert_columns(&sort_values)?) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 27577e6c8bf8..a1d3378181c2 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -358,7 +358,7 @@ impl GroupedHashAggregateStream { let spill_state = SpillState { spills: vec![], spill_expr, - spill_schema: agg_schema.clone(), + spill_schema: Arc::clone(&agg_schema), is_stream_merging: false, merging_aggregate_arguments, merging_group_by: PhysicalGroupBy::new_single(agg_group_by.expr.clone()), @@ -401,7 +401,7 @@ pub(crate) fn create_group_accumulator( "Creating GroupsAccumulatorAdapter for {}: {agg_expr:?}", agg_expr.name() ); - let agg_expr_captured = agg_expr.clone(); + let agg_expr_captured = Arc::clone(agg_expr); let factory = move || agg_expr_captured.create_accumulator(); Ok(Box::new(GroupsAccumulatorAdapter::new(factory))) } @@ -515,7 +515,7 @@ impl Stream for GroupedHashAggregateStream { impl RecordBatchStream for GroupedHashAggregateStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -625,7 +625,7 @@ impl GroupedHashAggregateStream { /// accumulator states/values specified in emit_to fn emit(&mut self, emit_to: EmitTo, spilling: bool) -> Result { let schema = if spilling { - self.spill_state.spill_schema.clone() + Arc::clone(&self.spill_state.spill_schema) } else { self.schema() }; @@ -746,13 +746,13 @@ impl GroupedHashAggregateStream { let expr = self.spill_state.spill_expr.clone(); let schema = batch.schema(); streams.push(Box::pin(RecordBatchStreamAdapter::new( - schema.clone(), + Arc::clone(&schema), futures::stream::once(futures::future::lazy(move |_| { sort_batch(&batch, &expr, None) })), ))); for spill in self.spill_state.spills.drain(..) { - let stream = read_spill_as_stream(spill, schema.clone(), 2)?; + let stream = read_spill_as_stream(spill, Arc::clone(&schema), 2)?; streams.push(stream); } self.spill_state.is_stream_merging = true; diff --git a/datafusion/physical-plan/src/aggregates/topk_stream.rs b/datafusion/physical-plan/src/aggregates/topk_stream.rs index 9f25473cb9b4..075d8c5f2883 100644 --- a/datafusion/physical-plan/src/aggregates/topk_stream.rs +++ b/datafusion/physical-plan/src/aggregates/topk_stream.rs @@ -84,14 +84,14 @@ impl GroupedTopKAggregateStream { impl RecordBatchStream for GroupedTopKAggregateStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } impl GroupedTopKAggregateStream { fn intern(&mut self, ids: ArrayRef, vals: ArrayRef) -> Result<()> { let len = ids.len(); - self.priority_map.set_batch(ids, vals.clone()); + self.priority_map.set_batch(ids, Arc::clone(&vals)); let has_nulls = vals.null_count() > 0; for row_idx in 0..len { @@ -139,14 +139,14 @@ impl Stream for GroupedTopKAggregateStream { 1, "Exactly 1 group value required" ); - let group_by_values = group_by_values[0][0].clone(); + let group_by_values = Arc::clone(&group_by_values[0][0]); let input_values = evaluate_many( &self.aggregate_arguments, batches.first().unwrap(), )?; assert_eq!(input_values.len(), 1, "Exactly 1 input required"); assert_eq!(input_values[0].len(), 1, "Exactly 1 input required"); - let input_values = input_values[0][0].clone(); + let input_values = Arc::clone(&input_values[0][0]); // iterate over each column of group_by values (*self).intern(group_by_values, input_values)?; @@ -158,7 +158,7 @@ impl Stream for GroupedTopKAggregateStream { return Poll::Ready(None); } let cols = self.priority_map.emit()?; - let batch = RecordBatch::try_new(self.schema.clone(), cols)?; + let batch = RecordBatch::try_new(Arc::clone(&self.schema), cols)?; trace!( "partition {} emit batch with {} rows", self.partition, diff --git a/datafusion/physical-plan/src/analyze.rs b/datafusion/physical-plan/src/analyze.rs index 5b859804163b..b4c1e25e6191 100644 --- a/datafusion/physical-plan/src/analyze.rs +++ b/datafusion/physical-plan/src/analyze.rs @@ -59,7 +59,7 @@ impl AnalyzeExec { input: Arc, schema: SchemaRef, ) -> Self { - let cache = Self::compute_properties(&input, schema.clone()); + let cache = Self::compute_properties(&input, Arc::clone(&schema)); AnalyzeExec { verbose, show_statistics, @@ -141,7 +141,7 @@ impl ExecutionPlan for AnalyzeExec { self.verbose, self.show_statistics, children.pop().unwrap(), - self.schema.clone(), + Arc::clone(&self.schema), ))) } @@ -164,13 +164,17 @@ impl ExecutionPlan for AnalyzeExec { RecordBatchReceiverStream::builder(self.schema(), num_input_partitions); for input_partition in 0..num_input_partitions { - builder.run_input(self.input.clone(), input_partition, context.clone()); + builder.run_input( + Arc::clone(&self.input), + input_partition, + Arc::clone(&context), + ); } // Create future that computes thefinal output let start = Instant::now(); - let captured_input = self.input.clone(); - let captured_schema = self.schema.clone(); + let captured_input = Arc::clone(&self.input); + let captured_schema = Arc::clone(&self.schema); let verbose = self.verbose; let show_statistics = self.show_statistics; @@ -196,7 +200,7 @@ impl ExecutionPlan for AnalyzeExec { }; Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema.clone(), + Arc::clone(&self.schema), futures::stream::once(output), ))) } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 804fabff71ac..b9bdfcdee712 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -134,7 +134,7 @@ impl ExecutionPlan for CoalesceBatchesExec { children: Vec>, ) -> Result> { Ok(Arc::new(CoalesceBatchesExec::new( - children[0].clone(), + Arc::clone(&children[0]), self.target_batch_size, ))) } @@ -272,7 +272,7 @@ impl CoalesceBatchesStream { impl RecordBatchStream for CoalesceBatchesStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -329,7 +329,7 @@ mod tests { target_batch_size: usize, ) -> Result>> { // create physical plan - let exec = MemoryExec::try_new(&input_partitions, schema.clone(), None)?; + let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?; let exec = RepartitionExec::try_new(Arc::new(exec), Partitioning::RoundRobinBatch(1))?; let exec: Arc = @@ -341,7 +341,7 @@ mod tests { for i in 0..output_partition_count { // execute this *output* partition and collect all batches let task_ctx = Arc::new(TaskContext::default()); - let mut stream = exec.execute(i, task_ctx.clone())?; + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; let mut batches = vec![]; while let Some(result) = stream.next().await { batches.push(result?); diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 93f449f2d39b..ef6afee80307 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -114,7 +114,9 @@ impl ExecutionPlan for CoalescePartitionsExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(CoalescePartitionsExec::new(children[0].clone()))) + Ok(Arc::new(CoalescePartitionsExec::new(Arc::clone( + &children[0], + )))) } fn execute( @@ -152,7 +154,11 @@ impl ExecutionPlan for CoalescePartitionsExec { // spawn independent tasks whose resulting streams (of batches) // are sent to the channel for consumption. for part_i in 0..input_partitions { - builder.run_input(self.input.clone(), part_i, context.clone()); + builder.run_input( + Arc::clone(&self.input), + part_i, + Arc::clone(&context), + ); } let stream = builder.build(); diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index c61e9a05bfa6..bf9d14e73dd8 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -618,16 +618,17 @@ mod tests { expr: col("f32", &schema).unwrap(), options: SortOptions::default(), }]; - let memory_exec = Arc::new(MemoryExec::try_new(&[], schema.clone(), None)?) as _; + let memory_exec = + Arc::new(MemoryExec::try_new(&[], Arc::clone(&schema), None)?) as _; let sort_exec = Arc::new(SortExec::new(sort_expr.clone(), memory_exec)) as Arc; let memory_exec2 = Arc::new(MemoryExec::try_new(&[], schema, None)?) as _; // memory_exec2 doesn't have output ordering - let union_exec = UnionExec::new(vec![sort_exec.clone(), memory_exec2]); + let union_exec = UnionExec::new(vec![Arc::clone(&sort_exec), memory_exec2]); let res = get_meet_of_orderings(union_exec.inputs()); assert!(res.is_none()); - let union_exec = UnionExec::new(vec![sort_exec.clone(), sort_exec]); + let union_exec = UnionExec::new(vec![Arc::clone(&sort_exec), sort_exec]); let res = get_meet_of_orderings(union_exec.inputs()); assert_eq!(res, Some(&sort_expr[..])); Ok(()) diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index 11af0624db15..4bacea48c347 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -47,7 +47,7 @@ pub struct EmptyExec { impl EmptyExec { /// Create a new EmptyExec pub fn new(schema: SchemaRef) -> Self { - let cache = Self::compute_properties(schema.clone(), 1); + let cache = Self::compute_properties(Arc::clone(&schema), 1); EmptyExec { schema, partitions: 1, @@ -142,7 +142,7 @@ impl ExecutionPlan for EmptyExec { Ok(Box::pin(MemoryStream::try_new( self.data()?, - self.schema.clone(), + Arc::clone(&self.schema), None, )?)) } @@ -170,7 +170,7 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let schema = test::aggr_test_schema(); - let empty = EmptyExec::new(schema.clone()); + let empty = EmptyExec::new(Arc::clone(&schema)); assert_eq!(empty.schema(), schema); // we should have no results @@ -184,9 +184,12 @@ mod tests { #[test] fn with_new_children() -> Result<()> { let schema = test::aggr_test_schema(); - let empty = Arc::new(EmptyExec::new(schema.clone())); + let empty = Arc::new(EmptyExec::new(Arc::clone(&schema))); - let empty2 = with_new_children_if_necessary(empty.clone(), vec![])?; + let empty2 = with_new_children_if_necessary( + Arc::clone(&empty) as Arc, + vec![], + )?; assert_eq!(empty.schema(), empty2.schema()); let too_many_kids = vec![empty2]; @@ -204,7 +207,7 @@ mod tests { let empty = EmptyExec::new(schema); // ask for the wrong partition - assert!(empty.execute(1, task_ctx.clone()).is_err()); + assert!(empty.execute(1, Arc::clone(&task_ctx)).is_err()); assert!(empty.execute(20, task_ctx).is_err()); Ok(()) } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 4b2edbf2045d..56dc35e8819d 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -53,7 +53,7 @@ impl ExplainExec { stringified_plans: Vec, verbose: bool, ) -> Self { - let cache = Self::compute_properties(schema.clone()); + let cache = Self::compute_properties(Arc::clone(&schema)); ExplainExec { schema, stringified_plans, @@ -160,7 +160,7 @@ impl ExecutionPlan for ExplainExec { } let record_batch = RecordBatch::try_new( - self.schema.clone(), + Arc::clone(&self.schema), vec![ Arc::new(type_builder.finish()), Arc::new(plan_builder.finish()), @@ -171,7 +171,7 @@ impl ExecutionPlan for ExplainExec { "Before returning RecordBatchStream in ExplainExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema.clone(), + Arc::clone(&self.schema), futures::stream::iter(vec![Ok(record_batch)]), ))) } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index ab7a63e44550..96ec6c0cf34d 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -78,7 +78,7 @@ impl FilterExec { Self::compute_properties(&input, &predicate, default_selectivity)?; Ok(Self { predicate, - input: input.clone(), + input: Arc::clone(&input), metrics: ExecutionPlanMetricsSet::new(), default_selectivity, cache, @@ -263,7 +263,7 @@ impl ExecutionPlan for FilterExec { self: Arc, mut children: Vec>, ) -> Result> { - FilterExec::try_new(self.predicate.clone(), children.swap_remove(0)) + FilterExec::try_new(Arc::clone(&self.predicate), children.swap_remove(0)) .and_then(|e| { let selectivity = e.default_selectivity(); e.with_default_selectivity(selectivity) @@ -280,7 +280,7 @@ impl ExecutionPlan for FilterExec { let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); Ok(Box::pin(FilterExecStream { schema: self.input.schema(), - predicate: self.predicate.clone(), + predicate: Arc::clone(&self.predicate), input: self.input.execute(partition, context)?, baseline_metrics, })) @@ -405,7 +405,7 @@ impl Stream for FilterExecStream { impl RecordBatchStream for FilterExecStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -1124,7 +1124,7 @@ mod tests { binary(col("c1", &schema)?, Operator::LtEq, lit(4i32), &schema)?, &schema, )?, - Arc::new(EmptyExec::new(schema.clone())), + Arc::new(EmptyExec::new(Arc::clone(&schema))), )?; exec.statistics().unwrap(); diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 30c3353d4b71..1c21991d93c5 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -153,7 +153,7 @@ impl DataSinkExec { } else { // Check not null constraint on the input stream Ok(Box::pin(RecordBatchStreamAdapter::new( - self.sink_schema.clone(), + Arc::clone(&self.sink_schema), input_stream .map(move |batch| check_not_null_contraits(batch?, &risky_columns)), ))) @@ -252,9 +252,9 @@ impl ExecutionPlan for DataSinkExec { children: Vec>, ) -> Result> { Ok(Arc::new(Self::new( - children[0].clone(), - self.sink.clone(), - self.sink_schema.clone(), + Arc::clone(&children[0]), + Arc::clone(&self.sink), + Arc::clone(&self.sink_schema), self.sort_order.clone(), ))) } @@ -269,10 +269,10 @@ impl ExecutionPlan for DataSinkExec { if partition != 0 { return internal_err!("DataSinkExec can only be called on partition 0!"); } - let data = self.execute_input_stream(0, context.clone())?; + let data = self.execute_input_stream(0, Arc::clone(&context))?; - let count_schema = self.count_schema.clone(); - let sink = self.sink.clone(); + let count_schema = Arc::clone(&self.count_schema); + let sink = Arc::clone(&self.sink); let stream = futures::stream::once(async move { sink.write_all(data, &context).await.map(make_count_batch) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 92443d06856a..33a9c061bf31 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -79,7 +79,7 @@ impl CrossJoinExec { }; let schema = Arc::new(Schema::new(all_columns)); - let cache = Self::compute_properties(&left, &right, schema.clone()); + let cache = Self::compute_properties(&left, &right, Arc::clone(&schema)); CrossJoinExec { left, right, @@ -220,8 +220,8 @@ impl ExecutionPlan for CrossJoinExec { children: Vec>, ) -> Result> { Ok(Arc::new(CrossJoinExec::new( - children[0].clone(), - children[1].clone(), + Arc::clone(&children[0]), + Arc::clone(&children[1]), ))) } @@ -237,7 +237,7 @@ impl ExecutionPlan for CrossJoinExec { partition: usize, context: Arc, ) -> Result { - let stream = self.right.execute(partition, context.clone())?; + let stream = self.right.execute(partition, Arc::clone(&context))?; let join_metrics = BuildProbeJoinMetrics::new(partition, &self.metrics); @@ -247,7 +247,7 @@ impl ExecutionPlan for CrossJoinExec { let left_fut = self.left_fut.once(|| { load_left_input( - self.left.clone(), + Arc::clone(&self.left), context, join_metrics.clone(), reservation, @@ -255,7 +255,7 @@ impl ExecutionPlan for CrossJoinExec { }); Ok(Box::pin(CrossJoinStream { - schema: self.schema.clone(), + schema: Arc::clone(&self.schema), left_fut, right: stream, left_index: 0, @@ -337,7 +337,7 @@ struct CrossJoinStream { impl RecordBatchStream for CrossJoinStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index c6ef9936b9c5..16b3a4f2febd 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -367,7 +367,7 @@ impl HashJoinExec { let cache = Self::compute_properties( &left, &right, - join_schema.clone(), + Arc::clone(&join_schema), *join_type, &on, partition_mode, @@ -433,7 +433,10 @@ impl HashJoinExec { false, matches!( join_type, - JoinType::Inner | JoinType::RightAnti | JoinType::RightSemi + JoinType::Inner + | JoinType::Right + | JoinType::RightAnti + | JoinType::RightSemi ), ] } @@ -461,8 +464,8 @@ impl HashJoinExec { None => None, }; Self::try_new( - self.left.clone(), - self.right.clone(), + Arc::clone(&self.left), + Arc::clone(&self.right), self.on.clone(), self.filter.clone(), &self.join_type, @@ -487,7 +490,7 @@ impl HashJoinExec { left.equivalence_properties().clone(), right.equivalence_properties().clone(), &join_type, - schema.clone(), + Arc::clone(&schema), &Self::maintains_input_order(join_type), Some(Self::probe_side()), on, @@ -635,8 +638,11 @@ impl ExecutionPlan for HashJoinExec { Distribution::UnspecifiedDistribution, ], PartitionMode::Partitioned => { - let (left_expr, right_expr) = - self.on.iter().map(|(l, r)| (l.clone(), r.clone())).unzip(); + let (left_expr, right_expr) = self + .on + .iter() + .map(|(l, r)| (Arc::clone(l), Arc::clone(r))) + .unzip(); vec![ Distribution::HashPartitioned(left_expr), Distribution::HashPartitioned(right_expr), @@ -678,8 +684,8 @@ impl ExecutionPlan for HashJoinExec { children: Vec>, ) -> Result> { Ok(Arc::new(HashJoinExec::try_new( - children[0].clone(), - children[1].clone(), + Arc::clone(&children[0]), + Arc::clone(&children[1]), self.on.clone(), self.filter.clone(), &self.join_type, @@ -694,8 +700,16 @@ impl ExecutionPlan for HashJoinExec { partition: usize, context: Arc, ) -> Result { - let on_left = self.on.iter().map(|on| on.0.clone()).collect::>(); - let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); + let on_left = self + .on + .iter() + .map(|on| Arc::clone(&on.0)) + .collect::>(); + let on_right = self + .on + .iter() + .map(|on| Arc::clone(&on.1)) + .collect::>(); let left_partitions = self.left.output_partitioning().partition_count(); let right_partitions = self.right.output_partitioning().partition_count(); @@ -715,9 +729,9 @@ impl ExecutionPlan for HashJoinExec { collect_left_input( None, self.random_state.clone(), - self.left.clone(), + Arc::clone(&self.left), on_left.clone(), - context.clone(), + Arc::clone(&context), join_metrics.clone(), reservation, need_produce_result_in_final(self.join_type), @@ -732,9 +746,9 @@ impl ExecutionPlan for HashJoinExec { OnceFut::new(collect_left_input( Some(partition), self.random_state.clone(), - self.left.clone(), + Arc::clone(&self.left), on_left.clone(), - context.clone(), + Arc::clone(&context), join_metrics.clone(), reservation, need_produce_result_in_final(self.join_type), @@ -779,6 +793,7 @@ impl ExecutionPlan for HashJoinExec { build_side: BuildSide::Initial(BuildSideInitialState { left_fut }), batch_size, hashes_buffer: vec![], + right_side_ordered: self.right.output_ordering().is_some(), })) } @@ -791,8 +806,8 @@ impl ExecutionPlan for HashJoinExec { // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` let mut stats = estimate_join_statistics( - self.left.clone(), - self.right.clone(), + Arc::clone(&self.left), + Arc::clone(&self.right), self.on.clone(), &self.join_type, &self.join_schema, @@ -836,7 +851,7 @@ async fn collect_left_input( }; // Depending on partition argument load single partition or whole left side in memory - let stream = left_input.execute(left_input_partition, context.clone())?; + let stream = left_input.execute(left_input_partition, Arc::clone(&context))?; // This operation performs 2 steps at once: // 1. creates a [JoinHashMap] of all batches from the stream @@ -1107,11 +1122,13 @@ struct HashJoinStream { batch_size: usize, /// Scratch space for computing hashes hashes_buffer: Vec, + /// Specifies whether the right side has an ordering to potentially preserve + right_side_ordered: bool, } impl RecordBatchStream for HashJoinStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -1221,7 +1238,7 @@ fn eq_dyn_null( } else { Operator::Eq }; - return Ok(compare_op_for_nested(&op, &left, &right)?); + return Ok(compare_op_for_nested(op, &left, &right)?); } match (left.data_type(), right.data_type()) { _ if null_equals_null => not_distinct(&left, &right), @@ -1449,6 +1466,7 @@ impl HashJoinStream { right_indices, index_alignment_range_start..index_alignment_range_end, self.join_type, + self.right_side_ordered, ); let result = build_batch_from_indices( @@ -1542,7 +1560,6 @@ impl Stream for HashJoinStream { #[cfg(test)] mod tests { - use super::*; use crate::{ common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, @@ -1676,8 +1693,10 @@ mod tests { ) -> Result<(Vec, Vec)> { let partition_count = 4; - let (left_expr, right_expr) = - on.iter().map(|(l, r)| (l.clone(), r.clone())).unzip(); + let (left_expr, right_expr) = on + .iter() + .map(|(l, r)| (Arc::clone(l), Arc::clone(r))) + .unzip(); let left_repartitioned: Arc = match partition_mode { PartitionMode::CollectLeft => Arc::new(CoalescePartitionsExec::new(left)), @@ -1726,7 +1745,7 @@ mod tests { let mut batches = vec![]; for i in 0..partition_count { - let stream = join.execute(i, context.clone())?; + let stream = join.execute(i, Arc::clone(&context))?; let more_batches = common::collect(stream).await?; batches.extend( more_batches @@ -1760,8 +1779,8 @@ mod tests { )]; let (columns, batches) = join_collect( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), &JoinType::Inner, false, @@ -1807,8 +1826,8 @@ mod tests { )]; let (columns, batches) = partitioned_join_collect( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), &JoinType::Inner, false, @@ -2111,7 +2130,7 @@ mod tests { assert_eq!(columns, vec!["a1", "b1", "c1", "a2", "b1", "c2"]); // first part - let stream = join.execute(0, task_ctx.clone())?; + let stream = join.execute(0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // expected joined records = 1 (first right batch) @@ -2134,7 +2153,7 @@ mod tests { assert_batches_eq!(expected, &batches); // second part - let stream = join.execute(1, task_ctx.clone())?; + let stream = join.execute(1, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; // expected joined records = 2 (second right batch) @@ -2349,8 +2368,8 @@ mod tests { )]; let (columns, batches) = join_collect( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), &JoinType::Left, false, @@ -2393,8 +2412,8 @@ mod tests { )]; let (columns, batches) = partitioned_join_collect( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), &JoinType::Left, false, @@ -2505,8 +2524,8 @@ mod tests { ); let join = join_with_filter( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), filter, &JoinType::LeftSemi, @@ -2516,7 +2535,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header.clone(), vec!["a1", "b1", "c1"]); - let stream = join.execute(0, task_ctx.clone())?; + let stream = join.execute(0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; let expected = [ @@ -2629,8 +2648,8 @@ mod tests { ); let join = join_with_filter( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), filter, &JoinType::RightSemi, @@ -2640,7 +2659,7 @@ mod tests { let columns = columns(&join.schema()); assert_eq!(columns, vec!["a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx.clone())?; + let stream = join.execute(0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; let expected = [ @@ -2751,8 +2770,8 @@ mod tests { ); let join = join_with_filter( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), filter, &JoinType::LeftAnti, @@ -2762,7 +2781,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header, vec!["a1", "b1", "c1"]); - let stream = join.execute(0, task_ctx.clone())?; + let stream = join.execute(0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; let expected = [ @@ -2880,8 +2899,8 @@ mod tests { ); let join = join_with_filter( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), filter, &JoinType::RightAnti, @@ -2891,7 +2910,7 @@ mod tests { let columns_header = columns(&join.schema()); assert_eq!(columns_header, vec!["a2", "b2", "c2"]); - let stream = join.execute(0, task_ctx.clone())?; + let stream = join.execute(0, Arc::clone(&task_ctx))?; let batches = common::collect(stream).await?; let expected = [ @@ -3081,8 +3100,11 @@ mod tests { let random_state = RandomState::with_seeds(0, 0, 0, 0); let hashes_buff = &mut vec![0; left.num_rows()]; - let hashes = - create_hashes(&[left.columns()[0].clone()], &random_state, hashes_buff)?; + let hashes = create_hashes( + &[Arc::clone(&left.columns()[0])], + &random_state, + hashes_buff, + )?; // Create hash collisions (same hashes) hashmap_left.insert(hashes[0], (hashes[0], 1), |(h, _)| *h); @@ -3110,7 +3132,7 @@ mod tests { &join_hash_map, &left, &right, - &[key_column.clone()], + &[Arc::clone(&key_column)], &[key_column], false, &hashes_buffer, @@ -3470,13 +3492,13 @@ mod tests { for (join_type, expected) in test_cases { let (_, batches) = join_collect_with_partition_mode( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), &join_type, PartitionMode::CollectLeft, false, - task_ctx.clone(), + Arc::clone(&task_ctx), ) .await?; assert_batches_sorted_eq!(expected, &batches); @@ -3494,13 +3516,14 @@ mod tests { let dates: ArrayRef = Arc::new(Date32Array::from(vec![19107, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 3])); - let batch = RecordBatch::try_new(schema.clone(), vec![dates, n])?; - let left = - Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None).unwrap()); + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; + let left = Arc::new( + MemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None).unwrap(), + ); let dates: ArrayRef = Arc::new(Date32Array::from(vec![19108, 19108, 19109])); let n: ArrayRef = Arc::new(Int32Array::from(vec![4, 5, 6])); - let batch = RecordBatch::try_new(schema.clone(), vec![dates, n])?; + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![dates, n])?; let right = Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None).unwrap()); let on = vec![( @@ -3562,8 +3585,8 @@ mod tests { for join_type in join_types { let join = join( - left.clone(), - right_input.clone(), + Arc::clone(&left), + Arc::clone(&right_input) as Arc, on.clone(), &join_type, false, @@ -3678,9 +3701,14 @@ mod tests { for batch_size in (1..21).rev() { let task_ctx = prepare_task_ctx(batch_size); - let join = - join(left.clone(), right.clone(), on.clone(), &join_type, false) - .unwrap(); + let join = join( + Arc::clone(&left), + Arc::clone(&right), + on.clone(), + &join_type, + false, + ) + .unwrap(); let stream = join.execute(0, task_ctx).unwrap(); let batches = common::collect(stream).await.unwrap(); @@ -3753,7 +3781,13 @@ mod tests { let task_ctx = TaskContext::default().with_runtime(runtime); let task_ctx = Arc::new(task_ctx); - let join = join(left.clone(), right.clone(), on.clone(), &join_type, false)?; + let join = join( + Arc::clone(&left), + Arc::clone(&right), + on.clone(), + &join_type, + false, + )?; let stream = join.execute(0, task_ctx)?; let err = common::collect(stream).await.unwrap_err(); @@ -3826,8 +3860,8 @@ mod tests { let task_ctx = Arc::new(task_ctx); let join = HashJoinExec::try_new( - left.clone(), - right.clone(), + Arc::clone(&left) as Arc, + Arc::clone(&right) as Arc, on.clone(), None, &join_type, @@ -3921,12 +3955,12 @@ mod tests { )]; let (_, batches_null_eq) = join_collect( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), &JoinType::Inner, true, - task_ctx.clone(), + Arc::clone(&task_ctx), ) .await?; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 6be124cce06f..754e55e49650 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -173,7 +173,8 @@ impl NestedLoopJoinExec { let (schema, column_indices) = build_join_schema(&left_schema, &right_schema, join_type); let schema = Arc::new(schema); - let cache = Self::compute_properties(&left, &right, schema.clone(), *join_type); + let cache = + Self::compute_properties(&left, &right, Arc::clone(&schema), *join_type); Ok(NestedLoopJoinExec { left, @@ -287,8 +288,8 @@ impl ExecutionPlan for NestedLoopJoinExec { children: Vec>, ) -> Result> { Ok(Arc::new(NestedLoopJoinExec::try_new( - children[0].clone(), - children[1].clone(), + Arc::clone(&children[0]), + Arc::clone(&children[1]), self.filter.clone(), &self.join_type, )?)) @@ -308,8 +309,8 @@ impl ExecutionPlan for NestedLoopJoinExec { let inner_table = self.inner_table.once(|| { collect_left_input( - self.left.clone(), - context.clone(), + Arc::clone(&self.left), + Arc::clone(&context), join_metrics.clone(), load_reservation, need_produce_result_in_final(self.join_type), @@ -319,7 +320,7 @@ impl ExecutionPlan for NestedLoopJoinExec { let outer_table = self.right.execute(partition, context)?; Ok(Box::pin(NestedLoopJoinStream { - schema: self.schema.clone(), + schema: Arc::clone(&self.schema), filter: self.filter.clone(), join_type: self.join_type, outer_table, @@ -336,8 +337,8 @@ impl ExecutionPlan for NestedLoopJoinExec { fn statistics(&self) -> Result { estimate_join_statistics( - self.left.clone(), - self.right.clone(), + Arc::clone(&self.left), + Arc::clone(&self.right), vec![], &self.join_type, &self.schema, @@ -604,6 +605,7 @@ fn join_left_and_right_batch( right_side, 0..right_batch.num_rows(), join_type, + false, ); build_batch_from_indices( @@ -641,13 +643,12 @@ impl Stream for NestedLoopJoinStream { impl RecordBatchStream for NestedLoopJoinStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } #[cfg(test)] mod tests { - use super::*; use crate::{ common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, @@ -752,7 +753,7 @@ mod tests { let columns = columns(&nested_loop_join.schema()); let mut batches = vec![]; for i in 0..partition_count { - let stream = nested_loop_join.execute(i, context.clone())?; + let stream = nested_loop_join.execute(i, Arc::clone(&context))?; let more_batches = common::collect(stream).await?; batches.extend( more_batches @@ -1037,8 +1038,8 @@ mod tests { let task_ctx = Arc::new(task_ctx); let err = multi_partitioned_join_collect( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), &join_type, Some(filter.clone()), task_ctx, diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 91b2151d32e7..e9124a72970a 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -126,11 +126,11 @@ impl SortMergeJoinExec { .zip(sort_options.iter()) .map(|((l, r), sort_op)| { let left = PhysicalSortExpr { - expr: l.clone(), + expr: Arc::clone(l), options: *sort_op, }; let right = PhysicalSortExpr { - expr: r.clone(), + expr: Arc::clone(r), options: *sort_op, }; (left, right) @@ -140,7 +140,7 @@ impl SortMergeJoinExec { let schema = Arc::new(build_join_schema(&left_schema, &right_schema, &join_type).0); let cache = - Self::compute_properties(&left, &right, schema.clone(), join_type, &on); + Self::compute_properties(&left, &right, Arc::clone(&schema), join_type, &on); Ok(Self { left, right, @@ -271,8 +271,11 @@ impl ExecutionPlan for SortMergeJoinExec { } fn required_input_distribution(&self) -> Vec { - let (left_expr, right_expr) = - self.on.iter().map(|(l, r)| (l.clone(), r.clone())).unzip(); + let (left_expr, right_expr) = self + .on + .iter() + .map(|(l, r)| (Arc::clone(l), Arc::clone(r))) + .unzip(); vec![ Distribution::HashPartitioned(left_expr), Distribution::HashPartitioned(right_expr), @@ -304,8 +307,8 @@ impl ExecutionPlan for SortMergeJoinExec { ) -> Result> { match &children[..] { [left, right] => Ok(Arc::new(SortMergeJoinExec::try_new( - left.clone(), - right.clone(), + Arc::clone(left), + Arc::clone(right), self.on.clone(), self.filter.clone(), self.join_type, @@ -332,14 +335,24 @@ impl ExecutionPlan for SortMergeJoinExec { let (on_left, on_right) = self.on.iter().cloned().unzip(); let (streamed, buffered, on_streamed, on_buffered) = if SortMergeJoinExec::probe_side(&self.join_type) == JoinSide::Left { - (self.left.clone(), self.right.clone(), on_left, on_right) + ( + Arc::clone(&self.left), + Arc::clone(&self.right), + on_left, + on_right, + ) } else { - (self.right.clone(), self.left.clone(), on_right, on_left) + ( + Arc::clone(&self.right), + Arc::clone(&self.left), + on_right, + on_left, + ) }; // execute children plans - let streamed = streamed.execute(partition, context.clone())?; - let buffered = buffered.execute(partition, context.clone())?; + let streamed = streamed.execute(partition, Arc::clone(&context))?; + let buffered = buffered.execute(partition, Arc::clone(&context))?; // create output buffer let batch_size = context.session_config().batch_size(); @@ -350,7 +363,7 @@ impl ExecutionPlan for SortMergeJoinExec { // create join stream Ok(Box::pin(SMJStream::try_new( - self.schema.clone(), + Arc::clone(&self.schema), self.sort_options.clone(), self.null_equals_null, streamed, @@ -374,8 +387,8 @@ impl ExecutionPlan for SortMergeJoinExec { // There are some special cases though, for example: // - `A LEFT JOIN B ON A.col=B.col` with `COUNT_DISTINCT(B.col)=COUNT(B.col)` estimate_join_statistics( - self.left.clone(), - self.right.clone(), + Arc::clone(&self.left), + Arc::clone(&self.right), self.on.clone(), &self.join_type, &self.schema, @@ -657,7 +670,7 @@ struct SMJStream { impl RecordBatchStream for SMJStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -780,7 +793,7 @@ impl SMJStream { sort_options, null_equals_null, schema, - streamed_schema: streamed_schema.clone(), + streamed_schema: Arc::clone(&streamed_schema), buffered_schema, streamed, buffered, @@ -1233,7 +1246,7 @@ impl SMJStream { }; let output_batch = - RecordBatch::try_new(self.schema.clone(), columns.clone())?; + RecordBatch::try_new(Arc::clone(&self.schema), columns.clone())?; // Apply join filter if any if !filter_columns.is_empty() { @@ -1353,8 +1366,10 @@ impl SMJStream { }; // Push the streamed/buffered batch joined nulls to the output - let null_joined_streamed_batch = - RecordBatch::try_new(self.schema.clone(), columns.clone())?; + let null_joined_streamed_batch = RecordBatch::try_new( + Arc::clone(&self.schema), + columns.clone(), + )?; self.output_record_batches.push(null_joined_streamed_batch); // For full join, we also need to output the null joined rows from the buffered side. @@ -1430,14 +1445,14 @@ fn get_filter_column( .column_indices() .iter() .filter(|col_index| col_index.side == JoinSide::Left) - .map(|i| streamed_columns[i.index].clone()) + .map(|i| Arc::clone(&streamed_columns[i.index])) .collect::>(); let right_columns = f .column_indices() .iter() .filter(|col_index| col_index.side == JoinSide::Right) - .map(|i| buffered_columns[i.index].clone()) + .map(|i| Arc::clone(&buffered_columns[i.index])) .collect::>(); filter_columns.extend(left_columns); @@ -1476,7 +1491,7 @@ fn produce_buffered_null_batch( streamed_columns.extend(buffered_columns); Ok(Some(RecordBatch::try_new( - schema.clone(), + Arc::clone(schema), streamed_columns, )?)) } @@ -1927,7 +1942,7 @@ mod tests { Field::new(c.0, DataType::Int32, true), ])); let batch = RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(Int32Array::from(a.1.clone())), Arc::new(Int32Array::from(b.1.clone())), @@ -2771,8 +2786,8 @@ mod tests { let task_ctx = Arc::new(task_ctx); let join = join_with_options( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), join_type, sort_options.clone(), @@ -2849,8 +2864,8 @@ mod tests { .with_runtime(runtime); let task_ctx = Arc::new(task_ctx); let join = join_with_options( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), join_type, sort_options.clone(), diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 46d3ac5acf1e..ba9384aef1a6 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -270,7 +270,7 @@ pub fn convert_sort_expr_with_filter_schema( sort_expr: &PhysicalSortExpr, ) -> Result>> { let column_map = map_origin_col_to_filter_col(filter, schema, side)?; - let expr = sort_expr.expr.clone(); + let expr = Arc::clone(&sort_expr.expr); // Get main schema columns: let expr_columns = collect_columns(&expr); // Calculation is possible with `column_map` since sort exprs belong to a child. @@ -697,7 +697,7 @@ fn update_sorted_exprs_with_node_indices( // Extract filter expressions from the sorted expressions: let filter_exprs = sorted_exprs .iter() - .map(|expr| expr.filter_expr().clone()) + .map(|expr| Arc::clone(expr.filter_expr())) .collect::>(); // Gather corresponding node indices for the extracted filter expressions from the graph: @@ -756,7 +756,7 @@ pub fn prepare_sorted_exprs( // Build the expression interval graph let mut graph = - ExprIntervalGraph::try_new(filter.expression().clone(), filter.schema())?; + ExprIntervalGraph::try_new(Arc::clone(filter.expression()), filter.schema())?; // Update sorted expressions with node indices update_sorted_exprs_with_node_indices(&mut graph, &mut sorted_exprs); @@ -818,9 +818,9 @@ pub mod tests { &intermediate_schema, )?; let filter_expr = binary( - filter_left.clone(), + Arc::clone(&filter_left), Operator::Gt, - filter_right.clone(), + Arc::clone(&filter_right), &intermediate_schema, )?; let column_indices = vec![ diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 813f670147bc..c23dc2032c4b 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -233,7 +233,7 @@ impl SymmetricHashJoinExec { let random_state = RandomState::with_seeds(0, 0, 0, 0); let schema = Arc::new(schema); let cache = - Self::compute_properties(&left, &right, schema.clone(), *join_type, &on); + Self::compute_properties(&left, &right, Arc::clone(&schema), *join_type, &on); Ok(SymmetricHashJoinExec { left, right, @@ -397,7 +397,7 @@ impl ExecutionPlan for SymmetricHashJoinExec { let (left_expr, right_expr) = self .on .iter() - .map(|(l, r)| (l.clone() as _, r.clone() as _)) + .map(|(l, r)| (Arc::clone(l) as _, Arc::clone(r) as _)) .unzip(); vec![ Distribution::HashPartitioned(left_expr), @@ -430,8 +430,8 @@ impl ExecutionPlan for SymmetricHashJoinExec { children: Vec>, ) -> Result> { Ok(Arc::new(SymmetricHashJoinExec::try_new( - children[0].clone(), - children[1].clone(), + Arc::clone(&children[0]), + Arc::clone(&children[1]), self.on.clone(), self.filter.clone(), &self.join_type, @@ -489,9 +489,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { let right_side_joiner = OneSideHashJoiner::new(JoinSide::Right, on_right, self.right.schema()); - let left_stream = self.left.execute(partition, context.clone())?; + let left_stream = self.left.execute(partition, Arc::clone(&context))?; - let right_stream = self.right.execute(partition, context.clone())?; + let right_stream = self.right.execute(partition, Arc::clone(&context))?; let reservation = Arc::new(Mutex::new( MemoryConsumer::new(format!("SymmetricHashJoinStream[{partition}]")) @@ -559,7 +559,7 @@ struct SymmetricHashJoinStream { impl RecordBatchStream for SymmetricHashJoinStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -1634,13 +1634,13 @@ mod tests { task_ctx: Arc, ) -> Result<()> { let first_batches = partitioned_sym_join_with_filter( - left.clone(), - right.clone(), + Arc::clone(&left), + Arc::clone(&right), on.clone(), filter.clone(), &join_type, false, - task_ctx.clone(), + Arc::clone(&task_ctx), ) .await?; let second_batches = partitioned_hash_join_with_filter( diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 7e05ded6f69d..264f297ffb4c 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -78,17 +78,23 @@ pub async fn partitioned_sym_join_with_filter( ) -> Result> { let partition_count = 4; - let left_expr = on.iter().map(|(l, _)| l.clone() as _).collect::>(); + let left_expr = on + .iter() + .map(|(l, _)| Arc::clone(l) as _) + .collect::>(); - let right_expr = on.iter().map(|(_, r)| r.clone() as _).collect::>(); + let right_expr = on + .iter() + .map(|(_, r)| Arc::clone(r) as _) + .collect::>(); let join = SymmetricHashJoinExec::try_new( Arc::new(RepartitionExec::try_new( - left.clone(), + Arc::clone(&left), Partitioning::Hash(left_expr, partition_count), )?), Arc::new(RepartitionExec::try_new( - right.clone(), + Arc::clone(&right), Partitioning::Hash(right_expr, partition_count), )?), on, @@ -102,7 +108,7 @@ pub async fn partitioned_sym_join_with_filter( let mut batches = vec![]; for i in 0..partition_count { - let stream = join.execute(i, context.clone())?; + let stream = join.execute(i, Arc::clone(&context))?; let more_batches = common::collect(stream).await?; batches.extend( more_batches @@ -127,7 +133,7 @@ pub async fn partitioned_hash_join_with_filter( let partition_count = 4; let (left_expr, right_expr) = on .iter() - .map(|(l, r)| (l.clone() as _, r.clone() as _)) + .map(|(l, r)| (Arc::clone(l) as _, Arc::clone(r) as _)) .unzip(); let join = Arc::new(HashJoinExec::try_new( @@ -149,7 +155,7 @@ pub async fn partitioned_hash_join_with_filter( let mut batches = vec![]; for i in 0..partition_count { - let stream = join.execute(i, context.clone())?; + let stream = join.execute(i, Arc::clone(&context))?; let more_batches = common::collect(stream).await?; batches.extend( more_batches @@ -475,20 +481,29 @@ pub fn build_sides_record_batches( )); let left = RecordBatch::try_from_iter(vec![ - ("la1", ordered.clone()), - ("lb1", cardinality.clone()), + ("la1", Arc::clone(&ordered)), + ("lb1", Arc::clone(&cardinality) as ArrayRef), ("lc1", cardinality_key_left), - ("lt1", time.clone()), - ("la2", ordered.clone()), - ("la1_des", ordered_des.clone()), - ("l_asc_null_first", ordered_asc_null_first.clone()), - ("l_asc_null_last", ordered_asc_null_last.clone()), - ("l_desc_null_first", ordered_desc_null_first.clone()), - ("li1", interval_time.clone()), - ("l_float", float_asc.clone()), + ("lt1", Arc::clone(&time) as ArrayRef), + ("la2", Arc::clone(&ordered)), + ("la1_des", Arc::clone(&ordered_des) as ArrayRef), + ( + "l_asc_null_first", + Arc::clone(&ordered_asc_null_first) as ArrayRef, + ), + ( + "l_asc_null_last", + Arc::clone(&ordered_asc_null_last) as ArrayRef, + ), + ( + "l_desc_null_first", + Arc::clone(&ordered_desc_null_first) as ArrayRef, + ), + ("li1", Arc::clone(&interval_time)), + ("l_float", Arc::clone(&float_asc) as ArrayRef), ])?; let right = RecordBatch::try_from_iter(vec![ - ("ra1", ordered.clone()), + ("ra1", Arc::clone(&ordered)), ("rb1", cardinality), ("rc1", cardinality_key_right), ("rt1", time), diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index dfa1fd4763f4..e3ec242ce8de 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -34,8 +34,9 @@ use arrow::array::{ UInt32BufferBuilder, UInt32Builder, UInt64Array, UInt64BufferBuilder, }; use arrow::compute; -use arrow::datatypes::{Field, Schema, SchemaBuilder}; +use arrow::datatypes::{Field, Schema, SchemaBuilder, UInt32Type, UInt64Type}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; +use arrow_array::builder::UInt64Builder; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray}; use arrow_buffer::ArrowNativeType; use datafusion_common::cast::as_boolean_array; @@ -439,7 +440,7 @@ pub fn adjust_right_output_partitioning( Partitioning::Hash(exprs, size) => { let new_exprs = exprs .iter() - .map(|expr| add_offset_to_expr(expr.clone(), left_columns_len)) + .map(|expr| add_offset_to_expr(Arc::clone(expr), left_columns_len)) .collect(); Partitioning::Hash(new_exprs, *size) } @@ -455,12 +456,10 @@ fn replace_on_columns_of_right_ordering( ) -> Result<()> { for (left_col, right_col) in on_columns { for item in right_ordering.iter_mut() { - let new_expr = item - .expr - .clone() + let new_expr = Arc::clone(&item.expr) .transform(|e| { if e.eq(right_col) { - Ok(Transformed::yes(left_col.clone())) + Ok(Transformed::yes(Arc::clone(left_col))) } else { Ok(Transformed::no(e)) } @@ -483,7 +482,7 @@ fn offset_ordering( JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => ordering .iter() .map(|sort_expr| PhysicalSortExpr { - expr: add_offset_to_expr(sort_expr.expr.clone(), offset), + expr: add_offset_to_expr(Arc::clone(&sort_expr.expr), offset), options: sort_expr.options, }) .collect(), @@ -1121,7 +1120,7 @@ impl OnceFut { OnceFutState::Ready(r) => Poll::Ready( r.as_ref() .map(|r| r.as_ref()) - .map_err(|e| DataFusionError::External(Box::new(e.clone()))), + .map_err(|e| DataFusionError::External(Box::new(Arc::clone(e)))), ), } } @@ -1284,6 +1283,7 @@ pub(crate) fn adjust_indices_by_join_type( right_indices: UInt32Array, adjust_range: Range, join_type: JoinType, + preserve_order_for_right: bool, ) -> (UInt64Array, UInt32Array) { match join_type { JoinType::Inner => { @@ -1295,12 +1295,17 @@ pub(crate) fn adjust_indices_by_join_type( (left_indices, right_indices) // unmatched left row will be produced in the end of loop, and it has been set in the left visited bitmap } - JoinType::Right | JoinType::Full => { - // matched - // unmatched right row will be produced in this batch - let right_unmatched_indices = get_anti_indices(adjust_range, &right_indices); + JoinType::Right => { // combine the matched and unmatched right result together - append_right_indices(left_indices, right_indices, right_unmatched_indices) + append_right_indices( + left_indices, + right_indices, + adjust_range, + preserve_order_for_right, + ) + } + JoinType::Full => { + append_right_indices(left_indices, right_indices, adjust_range, false) } JoinType::RightSemi => { // need to remove the duplicated record in the right side @@ -1326,30 +1331,48 @@ pub(crate) fn adjust_indices_by_join_type( } } -/// Appends the `right_unmatched_indices` to the `right_indices`, -/// and fills Null to tail of `left_indices` to -/// keep the length of `right_indices` and `left_indices` consistent. +/// Appends right indices to left indices based on the specified order mode. +/// +/// The function operates in two modes: +/// 1. If `preserve_order_for_right` is true, probe matched and unmatched indices +/// are inserted in order using the `append_probe_indices_in_order()` method. +/// 2. Otherwise, unmatched probe indices are simply appended after matched ones. +/// +/// # Parameters +/// - `left_indices`: UInt64Array of left indices. +/// - `right_indices`: UInt32Array of right indices. +/// - `adjust_range`: Range to adjust the right indices. +/// - `preserve_order_for_right`: Boolean flag to determine the mode of operation. +/// +/// # Returns +/// A tuple of updated `UInt64Array` and `UInt32Array`. pub(crate) fn append_right_indices( left_indices: UInt64Array, right_indices: UInt32Array, - right_unmatched_indices: UInt32Array, + adjust_range: Range, + preserve_order_for_right: bool, ) -> (UInt64Array, UInt32Array) { - // left_indices, right_indices and right_unmatched_indices must not contain the null value - if right_unmatched_indices.is_empty() { - (left_indices, right_indices) + if preserve_order_for_right { + append_probe_indices_in_order(left_indices, right_indices, adjust_range) } else { - let unmatched_size = right_unmatched_indices.len(); - // the new left indices: left_indices + null array - // the new right indices: right_indices + right_unmatched_indices - let new_left_indices = left_indices - .iter() - .chain(std::iter::repeat(None).take(unmatched_size)) - .collect::(); - let new_right_indices = right_indices - .iter() - .chain(right_unmatched_indices.iter()) - .collect::(); - (new_left_indices, new_right_indices) + let right_unmatched_indices = get_anti_indices(adjust_range, &right_indices); + + if right_unmatched_indices.is_empty() { + (left_indices, right_indices) + } else { + let unmatched_size = right_unmatched_indices.len(); + // the new left indices: left_indices + null array + // the new right indices: right_indices + right_unmatched_indices + let new_left_indices = left_indices + .iter() + .chain(std::iter::repeat(None).take(unmatched_size)) + .collect(); + let new_right_indices = right_indices + .iter() + .chain(right_unmatched_indices.iter()) + .collect(); + (new_left_indices, new_right_indices) + } } } @@ -1379,7 +1402,7 @@ where .filter_map(|idx| { (!bitmap.get_bit(idx - offset)).then_some(T::Native::from_usize(idx)) }) - .collect::>() + .collect() } /// Returns intersection of `range` and `input_indices` omitting duplicates @@ -1408,7 +1431,61 @@ where .filter_map(|idx| { (bitmap.get_bit(idx - offset)).then_some(T::Native::from_usize(idx)) }) - .collect::>() + .collect() +} + +/// Appends probe indices in order by considering the given build indices. +/// +/// This function constructs new build and probe indices by iterating through +/// the provided indices, and appends any missing values between previous and +/// current probe index with a corresponding null build index. +/// +/// # Parameters +/// +/// - `build_indices`: `PrimitiveArray` of `UInt64Type` containing build indices. +/// - `probe_indices`: `PrimitiveArray` of `UInt32Type` containing probe indices. +/// - `range`: The range of indices to consider. +/// +/// # Returns +/// +/// A tuple of two arrays: +/// - A `PrimitiveArray` of `UInt64Type` with the newly constructed build indices. +/// - A `PrimitiveArray` of `UInt32Type` with the newly constructed probe indices. +fn append_probe_indices_in_order( + build_indices: PrimitiveArray, + probe_indices: PrimitiveArray, + range: Range, +) -> (PrimitiveArray, PrimitiveArray) { + // Builders for new indices: + let mut new_build_indices = UInt64Builder::new(); + let mut new_probe_indices = UInt32Builder::new(); + // Set previous index as the start index for the initial loop: + let mut prev_index = range.start as u32; + // Zip the two iterators. + debug_assert!(build_indices.len() == probe_indices.len()); + for (build_index, probe_index) in build_indices + .values() + .into_iter() + .zip(probe_indices.values().into_iter()) + { + // Append values between previous and current probe index with null build index: + for value in prev_index..*probe_index { + new_probe_indices.append_value(value); + new_build_indices.append_null(); + } + // Append current indices: + new_probe_indices.append_value(*probe_index); + new_build_indices.append_value(*build_index); + // Set current probe index as previous for the next iteration: + prev_index = probe_index + 1; + } + // Append remaining probe indices after the last valid probe index with null build index. + for value in prev_index..range.end as u32 { + new_probe_indices.append_value(value); + new_build_indices.append_null(); + } + // Build arrays and return: + (new_build_indices.finish(), new_probe_indices.finish()) } /// Metrics for build & probe joins @@ -2475,7 +2552,7 @@ mod tests { &on_columns, left_columns_len, maintains_input_order, - probe_side + probe_side, ), expected[i] ); diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index aef5b307968c..f3a709ff7670 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -14,6 +14,8 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. +// Make cheap clones clear: https://github.com/apache/datafusion/issues/11143 +#![deny(clippy::clone_on_ref_ptr)] //! Traits for physical query plan, supporting parallel execution for partitioned relations. @@ -155,7 +157,7 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { /// Get the schema for this execution plan fn schema(&self) -> SchemaRef { - self.properties().schema().clone() + Arc::clone(self.properties().schema()) } /// Return properties of the output of the `ExecutionPlan`, such as output @@ -736,7 +738,7 @@ pub fn execute_stream( 1 => plan.execute(0, context), _ => { // merge into a single partition - let plan = CoalescePartitionsExec::new(plan.clone()); + let plan = CoalescePartitionsExec::new(Arc::clone(&plan)); // CoalescePartitionsExec must produce a single partition assert_eq!(1, plan.properties().output_partitioning().partition_count()); plan.execute(0, context) @@ -798,7 +800,7 @@ pub fn execute_stream_partitioned( let num_partitions = plan.output_partitioning().partition_count(); let mut streams = Vec::with_capacity(num_partitions); for i in 0..num_partitions { - streams.push(plan.execute(i, context.clone())?); + streams.push(plan.execute(i, Arc::clone(&context))?); } Ok(streams) } diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 4c6d1b3674d5..9c77a3d05cc2 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -145,7 +145,7 @@ impl ExecutionPlan for GlobalLimitExec { children: Vec>, ) -> Result> { Ok(Arc::new(GlobalLimitExec::new( - children[0].clone(), + Arc::clone(&children[0]), self.skip, self.fetch, ))) @@ -352,7 +352,7 @@ impl ExecutionPlan for LocalLimitExec { ) -> Result> { match children.len() { 1 => Ok(Arc::new(LocalLimitExec::new( - children[0].clone(), + Arc::clone(&children[0]), self.fetch, ))), _ => internal_err!("LocalLimitExec wrong number of children"), @@ -551,7 +551,7 @@ impl Stream for LimitStream { impl RecordBatchStream for LimitStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -864,11 +864,11 @@ mod tests { // Adding a "GROUP BY i" changes the input stats from Exact to Inexact. let agg = AggregateExec::try_new( AggregateMode::Final, - build_group_by(&csv.schema().clone(), vec!["i".to_string()]), + build_group_by(&csv.schema(), vec!["i".to_string()]), vec![], vec![], - csv.clone(), - csv.schema().clone(), + Arc::clone(&csv), + Arc::clone(&csv.schema()), )?; let agg_exec: Arc = Arc::new(agg); diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 39ae8d551f4b..6b2c78902eae 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -140,7 +140,7 @@ impl ExecutionPlan for MemoryExec { ) -> Result { Ok(Box::pin(MemoryStream::try_new( self.partitions[partition].clone(), - self.projected_schema.clone(), + Arc::clone(&self.projected_schema), self.projection.clone(), )?)) } @@ -164,7 +164,8 @@ impl MemoryExec { projection: Option>, ) -> Result { let projected_schema = project_schema(&schema, projection.as_ref())?; - let cache = Self::compute_properties(projected_schema.clone(), &[], partitions); + let cache = + Self::compute_properties(Arc::clone(&projected_schema), &[], partitions); Ok(Self { partitions: partitions.to_vec(), schema, @@ -219,7 +220,7 @@ impl MemoryExec { } pub fn original_schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. @@ -305,7 +306,7 @@ impl Stream for MemoryStream { impl RecordBatchStream for MemoryStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 3b10cc0ac435..272211d5056e 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -50,7 +50,7 @@ impl PlaceholderRowExec { /// Create a new PlaceholderRowExec pub fn new(schema: SchemaRef) -> Self { let partitions = 1; - let cache = Self::compute_properties(schema.clone(), partitions); + let cache = Self::compute_properties(Arc::clone(&schema), partitions); PlaceholderRowExec { schema, partitions, @@ -160,7 +160,7 @@ impl ExecutionPlan for PlaceholderRowExec { Ok(Box::pin(MemoryStream::try_new( self.data()?, - self.schema.clone(), + Arc::clone(&self.schema), None, )?)) } @@ -188,7 +188,10 @@ mod tests { let placeholder = Arc::new(PlaceholderRowExec::new(schema)); - let placeholder_2 = with_new_children_if_necessary(placeholder.clone(), vec![])?; + let placeholder_2 = with_new_children_if_necessary( + Arc::clone(&placeholder) as Arc, + vec![], + )?; assert_eq!(placeholder.schema(), placeholder_2.schema()); let too_many_kids = vec![placeholder_2]; @@ -206,7 +209,7 @@ mod tests { let placeholder = PlaceholderRowExec::new(schema); // ask for the wrong partition - assert!(placeholder.execute(1, task_ctx.clone()).is_err()); + assert!(placeholder.execute(1, Arc::clone(&task_ctx)).is_err()); assert!(placeholder.execute(20, task_ctx).is_err()); Ok(()) } @@ -234,7 +237,7 @@ mod tests { let placeholder = PlaceholderRowExec::new(schema).with_partitions(partitions); for n in 0..partitions { - let iter = placeholder.execute(n, task_ctx.clone())?; + let iter = placeholder.execute(n, Arc::clone(&task_ctx))?; let batches = common::collect(iter).await?; // should have one item diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 8341549340dd..9efa0422ec75 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -94,7 +94,7 @@ impl ProjectionExec { // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; let cache = - Self::compute_properties(&input, &projection_mapping, schema.clone())?; + Self::compute_properties(&input, &projection_mapping, Arc::clone(&schema))?; Ok(Self { expr, schema, @@ -227,8 +227,8 @@ impl ExecutionPlan for ProjectionExec { ) -> Result { trace!("Start ProjectionExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); Ok(Box::pin(ProjectionStream { - schema: self.schema.clone(), - expr: self.expr.iter().map(|x| x.0.clone()).collect(), + schema: Arc::clone(&self.schema), + expr: self.expr.iter().map(|x| Arc::clone(&x.0)).collect(), input: self.input.execute(partition, context)?, baseline_metrics: BaselineMetrics::new(&self.metrics, partition), })) @@ -242,7 +242,7 @@ impl ExecutionPlan for ProjectionExec { Ok(stats_projection( self.input.statistics()?, self.expr.iter().map(|(e, _)| Arc::clone(e)), - self.schema.clone(), + Arc::clone(&self.schema), )) } } @@ -311,10 +311,10 @@ impl ProjectionStream { if arrays.is_empty() { let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows())); - RecordBatch::try_new_with_options(self.schema.clone(), arrays, &options) + RecordBatch::try_new_with_options(Arc::clone(&self.schema), arrays, &options) .map_err(Into::into) } else { - RecordBatch::try_new(self.schema.clone(), arrays).map_err(Into::into) + RecordBatch::try_new(Arc::clone(&self.schema), arrays).map_err(Into::into) } } } @@ -351,7 +351,7 @@ impl Stream for ProjectionStream { impl RecordBatchStream for ProjectionStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -370,10 +370,12 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let exec = test::scan_partitioned(1); - let expected = collect(exec.execute(0, task_ctx.clone())?).await.unwrap(); + let expected = collect(exec.execute(0, Arc::clone(&task_ctx))?) + .await + .unwrap(); let projection = ProjectionExec::try_new(vec![], exec)?; - let stream = projection.execute(0, task_ctx.clone())?; + let stream = projection.execute(0, Arc::clone(&task_ctx))?; let output = collect(stream).await.unwrap(); assert_eq!(output.len(), expected.len()); diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index 9a0b66caba31..bd9303f97db0 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -82,7 +82,7 @@ impl RecursiveQueryExec { // Each recursive query needs its own work table let work_table = Arc::new(WorkTable::new()); // Use the same work table for both the WorkTableExec and the recursive term - let recursive_term = assign_work_table(recursive_term, work_table.clone())?; + let recursive_term = assign_work_table(recursive_term, Arc::clone(&work_table))?; let cache = Self::compute_properties(static_term.schema()); Ok(RecursiveQueryExec { name, @@ -147,8 +147,8 @@ impl ExecutionPlan for RecursiveQueryExec { ) -> Result> { RecursiveQueryExec::try_new( self.name.clone(), - children[0].clone(), - children[1].clone(), + Arc::clone(&children[0]), + Arc::clone(&children[1]), self.is_distinct, ) .map(|e| Arc::new(e) as _) @@ -167,12 +167,12 @@ impl ExecutionPlan for RecursiveQueryExec { ))); } - let static_stream = self.static_term.execute(partition, context.clone())?; + let static_stream = self.static_term.execute(partition, Arc::clone(&context))?; let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); Ok(Box::pin(RecursiveQueryStream::new( context, - self.work_table.clone(), - self.recursive_term.clone(), + Arc::clone(&self.work_table), + Arc::clone(&self.recursive_term), static_stream, baseline_metrics, ))) @@ -313,9 +313,9 @@ impl RecursiveQueryStream { // Downstream plans should not expect any partitioning. let partition = 0; - let recursive_plan = reset_plan_states(self.recursive_term.clone())?; + let recursive_plan = reset_plan_states(Arc::clone(&self.recursive_term))?; self.recursive_stream = - Some(recursive_plan.execute(partition, self.task_context.clone())?); + Some(recursive_plan.execute(partition, Arc::clone(&self.task_context))?); self.poll_next(cx) } } @@ -334,7 +334,7 @@ fn assign_work_table( } else { work_table_refs += 1; Ok(Transformed::yes(Arc::new( - exec.with_work_table(work_table.clone()), + exec.with_work_table(Arc::clone(&work_table)), ))) } } else if plan.as_any().is::() { @@ -358,8 +358,7 @@ fn reset_plan_states(plan: Arc) -> Result() { Ok(Transformed::no(plan)) } else { - let new_plan = plan - .clone() + let new_plan = Arc::clone(&plan) .with_new_children(plan.children().into_iter().cloned().collect())?; Ok(Transformed::yes(new_plan)) } @@ -407,7 +406,7 @@ impl Stream for RecursiveQueryStream { impl RecordBatchStream for RecursiveQueryStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index d9e16c98eee8..3d4d3058393e 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -133,12 +133,12 @@ impl RepartitionExecState { let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics); let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input( - input.clone(), + Arc::clone(&input), i, txs.clone(), partitioning.clone(), r_metrics, - context.clone(), + Arc::clone(&context), )); // In a separate task, wait for each input to be done @@ -616,7 +616,7 @@ impl ExecutionPlan for RepartitionExec { schema: Arc::clone(&schema_captured), receiver, drop_helper: Arc::clone(&abort_helper), - reservation: reservation.clone(), + reservation: Arc::clone(&reservation), }) as SendableRecordBatchStream }) .collect::>(); @@ -866,7 +866,7 @@ impl RepartitionExec { for (_, tx) in txs { // wrap it because need to send error to all output partitions - let err = Err(DataFusionError::External(Box::new(e.clone()))); + let err = Err(DataFusionError::External(Box::new(Arc::clone(&e)))); tx.send(Some(err)).await.ok(); } } @@ -945,7 +945,7 @@ impl Stream for RepartitionStream { impl RecordBatchStream for RepartitionStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -995,7 +995,7 @@ impl Stream for PerPartitionStream { impl RecordBatchStream for PerPartitionStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -1117,14 +1117,14 @@ mod tests { ) -> Result>> { let task_ctx = Arc::new(TaskContext::default()); // create physical plan - let exec = MemoryExec::try_new(&input_partitions, schema.clone(), None)?; + let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?; let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; // execute and collect results let mut output_partitions = vec![]; for i in 0..exec.partitioning.partition_count() { // execute this *output* partition and collect all batches - let mut stream = exec.execute(i, task_ctx.clone())?; + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; let mut batches = vec![]; while let Some(result) = stream.next().await { batches.push(result?); @@ -1301,10 +1301,14 @@ mod tests { let input = Arc::new(make_barrier_exec()); // partition into two output streams - let exec = RepartitionExec::try_new(input.clone(), partitioning).unwrap(); + let exec = RepartitionExec::try_new( + Arc::clone(&input) as Arc, + partitioning, + ) + .unwrap(); - let output_stream0 = exec.execute(0, task_ctx.clone()).unwrap(); - let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap(); + let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); + let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); // now, purposely drop output stream 0 // *before* any outputs are produced @@ -1349,8 +1353,12 @@ mod tests { // We first collect the results without droping the output stream. let input = Arc::new(make_barrier_exec()); - let exec = RepartitionExec::try_new(input.clone(), partitioning.clone()).unwrap(); - let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap(); + let exec = RepartitionExec::try_new( + Arc::clone(&input) as Arc, + partitioning.clone(), + ) + .unwrap(); + let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); let mut background_task = JoinSet::new(); background_task.spawn(async move { input.wait().await; @@ -1370,9 +1378,13 @@ mod tests { // Now do the same but dropping the stream before waiting for the barrier let input = Arc::new(make_barrier_exec()); - let exec = RepartitionExec::try_new(input.clone(), partitioning).unwrap(); - let output_stream0 = exec.execute(0, task_ctx.clone()).unwrap(); - let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap(); + let exec = RepartitionExec::try_new( + Arc::clone(&input) as Arc, + partitioning, + ) + .unwrap(); + let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); + let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); // now, purposely drop output stream 0 // *before* any outputs are produced std::mem::drop(output_stream0); @@ -1471,9 +1483,9 @@ mod tests { let schema = batch.schema(); let input = MockExec::new(vec![Ok(batch)], schema); let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); - let output_stream0 = exec.execute(0, task_ctx.clone()).unwrap(); + let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); let batch0 = crate::common::collect(output_stream0).await.unwrap(); - let output_stream1 = exec.execute(1, task_ctx.clone()).unwrap(); + let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); let batch1 = crate::common::collect(output_stream1).await.unwrap(); assert!(batch0.is_empty() || batch1.is_empty()); Ok(()) @@ -1496,12 +1508,12 @@ mod tests { let task_ctx = Arc::new(task_ctx); // create physical plan - let exec = MemoryExec::try_new(&input_partitions, schema.clone(), None)?; + let exec = MemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?; // pull partitions for i in 0..exec.partitioning.partition_count() { - let mut stream = exec.execute(i, task_ctx.clone())?; + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; let err = arrow_datafusion_err!(stream.next().await.unwrap().unwrap_err().into()); let err = err.find_root(); @@ -1642,7 +1654,7 @@ mod test { } fn memory_exec(schema: &SchemaRef) -> Arc { - Arc::new(MemoryExec::try_new(&[vec![]], schema.clone(), None).unwrap()) + Arc::new(MemoryExec::try_new(&[vec![]], Arc::clone(schema), None).unwrap()) } fn sorted_memory_exec( @@ -1650,7 +1662,7 @@ mod test { sort_exprs: Vec, ) -> Arc { Arc::new( - MemoryExec::try_new(&[vec![]], schema.clone(), None) + MemoryExec::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .with_sort_information(vec![sort_exprs]), ) diff --git a/datafusion/physical-plan/src/sorts/builder.rs b/datafusion/physical-plan/src/sorts/builder.rs index 3527d5738223..d32c60697ec8 100644 --- a/datafusion/physical-plan/src/sorts/builder.rs +++ b/datafusion/physical-plan/src/sorts/builder.rs @@ -20,6 +20,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; +use std::sync::Arc; #[derive(Debug, Copy, Clone, Default)] struct BatchCursor { @@ -145,6 +146,9 @@ impl BatchBuilder { retain }); - Ok(Some(RecordBatch::try_new(self.schema.clone(), columns)?)) + Ok(Some(RecordBatch::try_new( + Arc::clone(&self.schema), + columns, + )?)) } } diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 422ff3aebdb3..85418ff36119 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -29,6 +29,7 @@ use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; use std::pin::Pin; +use std::sync::Arc; use std::task::{ready, Context, Poll}; /// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] @@ -324,6 +325,6 @@ impl Stream for SortPreservingMergeStream { impl RecordBatchStream for SortPreservingMergeStream { fn schema(&self) -> SchemaRef { - self.in_progress.schema().clone() + Arc::clone(self.in_progress.schema()) } } diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index ad5d485cffc9..fe6b744935fb 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -260,7 +260,7 @@ impl ExecutionPlan for PartialSortExec { ) -> Result> { let new_partial_sort = PartialSortExec::new( self.expr.clone(), - children[0].clone(), + Arc::clone(&children[0]), self.common_prefix_length, ) .with_fetch(self.fetch) @@ -276,7 +276,7 @@ impl ExecutionPlan for PartialSortExec { ) -> Result { trace!("Start PartialSortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); - let input = self.input.execute(partition, context.clone())?; + let input = self.input.execute(partition, Arc::clone(&context))?; trace!( "End PartialSortExec's input.execute for partition: {}", @@ -485,11 +485,11 @@ mod tests { options: option_asc, }, ], - source.clone(), + Arc::clone(&source), 2, )) as Arc; - let result = collect(partial_sort_exec, task_ctx.clone()).await?; + let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; let expected_after_sort = [ "+---+---+---+", @@ -549,13 +549,13 @@ mod tests { options: option_asc, }, ], - source.clone(), + Arc::clone(&source), common_prefix_length, ) .with_fetch(Some(4)), ) as Arc; - let result = collect(partial_sort_exec, task_ctx.clone()).await?; + let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; let expected_after_sort = [ "+---+---+---+", @@ -621,11 +621,11 @@ mod tests { options: option_asc, }, ], - source.clone(), + Arc::clone(source), common_prefix_length, )); - let result = collect(partial_sort_exec, task_ctx.clone()).await?; + let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; assert_eq!(2, result.len()); assert_eq!( task_ctx.runtime_env().memory_pool.reserved(), @@ -676,7 +676,7 @@ mod tests { Arc::new( MemoryExec::try_new( &[vec![batch1, batch2, batch3, batch4]], - schema.clone(), + Arc::clone(&schema), None, ) .unwrap(), @@ -711,7 +711,7 @@ mod tests { options: option_asc, }, ], - mem_exec.clone(), + Arc::clone(&mem_exec), 1, ); let partial_sort_exec = @@ -720,7 +720,7 @@ mod tests { partial_sort_executor.expr, partial_sort_executor.input, )) as Arc; - let result = collect(partial_sort_exec, task_ctx.clone()).await?; + let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; assert_eq!( result.iter().map(|r| r.num_rows()).collect_vec(), [0, 125, 125, 0, 150] @@ -732,7 +732,7 @@ mod tests { "The sort should have returned all memory used back to the memory manager" ); let partial_sort_result = concat_batches(&schema, &result).unwrap(); - let sort_result = collect(sort_exec, task_ctx.clone()).await?; + let sort_result = collect(sort_exec, Arc::clone(&task_ctx)).await?; assert_eq!(sort_result[0], partial_sort_result); Ok(()) @@ -772,7 +772,7 @@ mod tests { options: option_asc, }, ], - mem_exec.clone(), + Arc::clone(&mem_exec), 1, ) .with_fetch(fetch_size); @@ -783,7 +783,7 @@ mod tests { SortExec::new(partial_sort_executor.expr, partial_sort_executor.input) .with_fetch(fetch_size), ) as Arc; - let result = collect(partial_sort_exec, task_ctx.clone()).await?; + let result = collect(partial_sort_exec, Arc::clone(&task_ctx)).await?; assert_eq!( result.iter().map(|r| r.num_rows()).collect_vec(), expected_batch_num_rows @@ -795,7 +795,7 @@ mod tests { "The sort should have returned all memory used back to the memory manager" ); let partial_sort_result = concat_batches(&schema, &result)?; - let sort_result = collect(sort_exec, task_ctx.clone()).await?; + let sort_result = collect(sort_exec, Arc::clone(&task_ctx)).await?; assert_eq!(sort_result[0], partial_sort_result); } @@ -822,8 +822,12 @@ mod tests { let data: ArrayRef = Arc::new(vec![1, 1, 2].into_iter().map(Some).collect::()); - let batch = RecordBatch::try_new(schema.clone(), vec![data])?; - let input = Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None)?); + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data])?; + let input = Arc::new(MemoryExec::try_new( + &[vec![batch]], + Arc::clone(&schema), + None, + )?); let partial_sort_exec = Arc::new(PartialSortExec::new( vec![PhysicalSortExpr { @@ -837,13 +841,13 @@ mod tests { let result: Vec = collect(partial_sort_exec, task_ctx).await?; let expected_batch = vec![ RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![Arc::new( vec![1, 1].into_iter().map(Some).collect::(), )], )?, RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![Arc::new( vec![2].into_iter().map(Some).collect::(), )], @@ -879,7 +883,7 @@ mod tests { // define data. let batch = RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(Float32Array::from(vec![ Some(1.0_f32), @@ -961,8 +965,11 @@ mod tests { *partial_sort_exec.schema().field(2).data_type() ); - let result: Vec = - collect(partial_sort_exec.clone(), task_ctx).await?; + let result: Vec = collect( + Arc::clone(&partial_sort_exec) as Arc, + task_ctx, + ) + .await?; assert_batches_eq!(expected, &result); assert_eq!(result.len(), 2); let metrics = partial_sort_exec.metrics().unwrap(); @@ -997,7 +1004,7 @@ mod tests { 1, )); - let fut = collect(sort_exec, task_ctx.clone()); + let fut = collect(sort_exec, Arc::clone(&task_ctx)); let mut fut = fut.boxed(); assert_is_pending(&mut fut); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 0bf66bc6e522..f347a0f5b6d5 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -338,13 +338,13 @@ impl ExternalSorter { spill.path() ))); } - let stream = read_spill_as_stream(spill, self.schema.clone(), 2)?; + let stream = read_spill_as_stream(spill, Arc::clone(&self.schema), 2)?; streams.push(stream); } streaming_merge( streams, - self.schema.clone(), + Arc::clone(&self.schema), &self.expr, self.metrics.baseline.clone(), self.batch_size, @@ -354,7 +354,9 @@ impl ExternalSorter { } else if !self.in_mem_batches.is_empty() { self.in_mem_sort_stream(self.metrics.baseline.clone()) } else { - Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone()))) + Ok(Box::pin(EmptyRecordBatchStream::new(Arc::clone( + &self.schema, + )))) } } @@ -394,8 +396,11 @@ impl ExternalSorter { let spill_file = self.runtime.disk_manager.create_tmp_file("Sorting")?; let batches = std::mem::take(&mut self.in_mem_batches); - let spilled_rows = - spill_record_batches(batches, spill_file.path().into(), self.schema.clone())?; + let spilled_rows = spill_record_batches( + batches, + spill_file.path().into(), + Arc::clone(&self.schema), + )?; let used = self.reservation.free(); self.metrics.spill_count.add(1); self.metrics.spilled_bytes.add(used); @@ -525,7 +530,7 @@ impl ExternalSorter { streaming_merge( streams, - self.schema.clone(), + Arc::clone(&self.schema), &self.expr, metrics, self.batch_size, @@ -548,7 +553,7 @@ impl ExternalSorter { let schema = batch.schema(); let fetch = self.fetch; - let expressions = self.expr.clone(); + let expressions = Arc::clone(&self.expr); let stream = futures::stream::once(futures::future::lazy(move |_| { let sorted = sort_batch(&batch, &expressions, fetch)?; metrics.record_output(sorted.num_rows()); @@ -844,7 +849,7 @@ impl ExecutionPlan for SortExec { self: Arc, children: Vec>, ) -> Result> { - let new_sort = SortExec::new(self.expr.clone(), children[0].clone()) + let new_sort = SortExec::new(self.expr.clone(), Arc::clone(&children[0])) .with_fetch(self.fetch) .with_preserve_partitioning(self.preserve_partitioning); @@ -858,7 +863,7 @@ impl ExecutionPlan for SortExec { ) -> Result { trace!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); - let mut input = self.input.execute(partition, context.clone())?; + let mut input = self.input.execute(partition, Arc::clone(&context))?; let execution_options = &context.session_config().options().execution; @@ -962,7 +967,7 @@ mod tests { Arc::new(CoalescePartitionsExec::new(csv)), )); - let result = collect(sort_exec, task_ctx.clone()).await?; + let result = collect(sort_exec, Arc::clone(&task_ctx)).await?; assert_eq!(result.len(), 1); assert_eq!(result[0].num_rows(), 400); @@ -1005,7 +1010,11 @@ mod tests { Arc::new(CoalescePartitionsExec::new(input)), )); - let result = collect(sort_exec.clone(), task_ctx.clone()).await?; + let result = collect( + Arc::clone(&sort_exec) as Arc, + Arc::clone(&task_ctx), + ) + .await?; assert_eq!(result.len(), 2); @@ -1081,7 +1090,11 @@ mod tests { .with_fetch(fetch), ); - let result = collect(sort_exec.clone(), task_ctx.clone()).await?; + let result = collect( + Arc::clone(&sort_exec) as Arc, + Arc::clone(&task_ctx), + ) + .await?; assert_eq!(result.len(), 1); let metrics = sort_exec.metrics().unwrap(); @@ -1111,9 +1124,10 @@ mod tests { let data: ArrayRef = Arc::new(vec![3, 2, 1].into_iter().map(Some).collect::()); - let batch = RecordBatch::try_new(schema.clone(), vec![data]).unwrap(); - let input = - Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None).unwrap()); + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap(); + let input = Arc::new( + MemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None).unwrap(), + ); let sort_exec = Arc::new(SortExec::new( vec![PhysicalSortExpr { @@ -1128,7 +1142,7 @@ mod tests { let expected_data: ArrayRef = Arc::new(vec![1, 2, 3].into_iter().map(Some).collect::()); let expected_batch = - RecordBatch::try_new(schema.clone(), vec![expected_data]).unwrap(); + RecordBatch::try_new(Arc::clone(&schema), vec![expected_data]).unwrap(); // Data is correct assert_eq!(&vec![expected_batch], &result); @@ -1154,7 +1168,7 @@ mod tests { // define data. let batch = RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(Int32Array::from(vec![Some(2), None, Some(1), Some(2)])), Arc::new(ListArray::from_iter_primitive::(vec![ @@ -1183,7 +1197,11 @@ mod tests { }, }, ], - Arc::new(MemoryExec::try_new(&[vec![batch]], schema.clone(), None)?), + Arc::new(MemoryExec::try_new( + &[vec![batch]], + Arc::clone(&schema), + None, + )?), )); assert_eq!(DataType::Int32, *sort_exec.schema().field(0).data_type()); @@ -1192,7 +1210,8 @@ mod tests { *sort_exec.schema().field(1).data_type() ); - let result: Vec = collect(sort_exec.clone(), task_ctx).await?; + let result: Vec = + collect(Arc::clone(&sort_exec) as Arc, task_ctx).await?; let metrics = sort_exec.metrics().unwrap(); assert!(metrics.elapsed_compute().unwrap() > 0); assert_eq!(metrics.output_rows().unwrap(), 4); @@ -1226,7 +1245,7 @@ mod tests { // define data. let batch = RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![ Arc::new(Float32Array::from(vec![ Some(f32::NAN), @@ -1274,7 +1293,8 @@ mod tests { assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type()); assert_eq!(DataType::Float64, *sort_exec.schema().field(1).data_type()); - let result: Vec = collect(sort_exec.clone(), task_ctx).await?; + let result: Vec = + collect(Arc::clone(&sort_exec) as Arc, task_ctx).await?; let metrics = sort_exec.metrics().unwrap(); assert!(metrics.elapsed_compute().unwrap() > 0); assert_eq!(metrics.output_rows().unwrap(), 8); @@ -1337,7 +1357,7 @@ mod tests { blocking_exec, )); - let fut = collect(sort_exec, task_ctx.clone()); + let fut = collect(sort_exec, Arc::clone(&task_ctx)); let mut fut = fut.boxed(); assert_is_pending(&mut fut); @@ -1358,7 +1378,8 @@ mod tests { let schema = Arc::new(Schema::empty()); let options = RecordBatchOptions::new().with_row_count(Some(1)); let batch = - RecordBatch::try_new_with_options(schema.clone(), vec![], &options).unwrap(); + RecordBatch::try_new_with_options(Arc::clone(&schema), vec![], &options) + .unwrap(); let expressions = vec![PhysicalSortExpr { expr: Arc::new(Literal::new(ScalarValue::Int64(Some(1)))), diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index e364aca3791c..41dfd449dd82 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -188,7 +188,7 @@ impl ExecutionPlan for SortPreservingMergeExec { children: Vec>, ) -> Result> { Ok(Arc::new( - SortPreservingMergeExec::new(self.expr.clone(), children[0].clone()) + SortPreservingMergeExec::new(self.expr.clone(), Arc::clone(&children[0])) .with_fetch(self.fetch), )) } @@ -232,7 +232,8 @@ impl ExecutionPlan for SortPreservingMergeExec { _ => { let receivers = (0..input_partitions) .map(|partition| { - let stream = self.input.execute(partition, context.clone())?; + let stream = + self.input.execute(partition, Arc::clone(&context))?; Ok(spawn_buffered(stream, 1)) }) .collect::>()?; @@ -587,8 +588,9 @@ mod tests { }, }]; - let basic = basic_sort(csv.clone(), sort.clone(), task_ctx.clone()).await; - let partition = partition_sort(csv, sort, task_ctx.clone()).await; + let basic = + basic_sort(Arc::clone(&csv), sort.clone(), Arc::clone(&task_ctx)).await; + let partition = partition_sort(csv, sort, Arc::clone(&task_ctx)).await; let basic = arrow::util::pretty::pretty_format_batches(&[basic]) .unwrap() @@ -654,10 +656,11 @@ mod tests { }]; let input = - sorted_partitioned_input(sort.clone(), &[10, 3, 11], task_ctx.clone()) + sorted_partitioned_input(sort.clone(), &[10, 3, 11], Arc::clone(&task_ctx)) .await?; - let basic = basic_sort(input.clone(), sort.clone(), task_ctx.clone()).await; - let partition = sorted_merge(input, sort, task_ctx.clone()).await; + let basic = + basic_sort(Arc::clone(&input), sort.clone(), Arc::clone(&task_ctx)).await; + let partition = sorted_merge(input, sort, Arc::clone(&task_ctx)).await; assert_eq!(basic.num_rows(), 1200); assert_eq!(partition.num_rows(), 1200); @@ -685,9 +688,9 @@ mod tests { // Test streaming with default batch size let task_ctx = Arc::new(TaskContext::default()); let input = - sorted_partitioned_input(sort.clone(), &[10, 5, 13], task_ctx.clone()) + sorted_partitioned_input(sort.clone(), &[10, 5, 13], Arc::clone(&task_ctx)) .await?; - let basic = basic_sort(input.clone(), sort.clone(), task_ctx).await; + let basic = basic_sort(Arc::clone(&input), sort.clone(), task_ctx).await; // batch size of 23 let task_ctx = TaskContext::default() @@ -805,17 +808,18 @@ mod tests { }]; let batches = - sorted_partitioned_input(sort.clone(), &[5, 7, 3], task_ctx.clone()).await?; + sorted_partitioned_input(sort.clone(), &[5, 7, 3], Arc::clone(&task_ctx)) + .await?; let partition_count = batches.output_partitioning().partition_count(); let mut streams = Vec::with_capacity(partition_count); for partition in 0..partition_count { - let mut builder = RecordBatchReceiverStream::builder(schema.clone(), 1); + let mut builder = RecordBatchReceiverStream::builder(Arc::clone(&schema), 1); let sender = builder.tx(); - let mut stream = batches.execute(partition, task_ctx.clone()).unwrap(); + let mut stream = batches.execute(partition, Arc::clone(&task_ctx)).unwrap(); builder.spawn(async move { while let Some(batch) = stream.next().await { sender.send(batch).await.unwrap(); @@ -849,7 +853,7 @@ mod tests { assert_eq!(merged.len(), 1); let merged = merged.remove(0); - let basic = basic_sort(batches, sort.clone(), task_ctx.clone()).await; + let basic = basic_sort(batches, sort.clone(), Arc::clone(&task_ctx)).await; let basic = arrow::util::pretty::pretty_format_batches(&[basic]) .unwrap() @@ -885,7 +889,9 @@ mod tests { let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); - let collected = collect(merge.clone(), task_ctx).await.unwrap(); + let collected = collect(Arc::clone(&merge) as Arc, task_ctx) + .await + .unwrap(); let expected = [ "+----+---+", "| a | b |", diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index 135b4fbdece4..c7924edfb1eb 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -109,7 +109,7 @@ impl RowCursorStream { Ok(Self { converter, reservation, - column_expressions: expressions.iter().map(|x| x.expr.clone()).collect(), + column_expressions: expressions.iter().map(|x| Arc::clone(&x.expr)).collect(), streams: FusedStreams(streams), }) } diff --git a/datafusion/physical-plan/src/stream.rs b/datafusion/physical-plan/src/stream.rs index 99d9367740be..faeb4799f5af 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -382,7 +382,7 @@ where S: Stream>, { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -402,7 +402,7 @@ impl EmptyRecordBatchStream { impl RecordBatchStream for EmptyRecordBatchStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -474,7 +474,7 @@ mod test { let schema = schema(); let num_partitions = 10; - let input = PanicExec::new(schema.clone(), num_partitions); + let input = PanicExec::new(Arc::clone(&schema), num_partitions); consume(input, 10).await } @@ -485,7 +485,7 @@ mod test { // make 2 partitions, second partition panics before the first let num_partitions = 2; - let input = PanicExec::new(schema.clone(), num_partitions) + let input = PanicExec::new(Arc::clone(&schema), num_partitions) .with_partition_panic(0, 10) .with_partition_panic(1, 3); // partition 1 should panic first (after 3 ) @@ -504,12 +504,12 @@ mod test { let schema = schema(); // Make an input that never proceeds - let input = BlockingExec::new(schema.clone(), 1); + let input = BlockingExec::new(Arc::clone(&schema), 1); let refs = input.refs(); // Configure a RecordBatchReceiverStream to consume the input let mut builder = RecordBatchReceiverStream::builder(schema, 2); - builder.run_input(Arc::new(input), 0, task_ctx.clone()); + builder.run_input(Arc::new(input), 0, Arc::clone(&task_ctx)); let stream = builder.build(); // input should still be present @@ -529,12 +529,14 @@ mod test { let schema = schema(); // make an input that will error twice - let error_stream = - MockExec::new(vec![exec_err!("Test1"), exec_err!("Test2")], schema.clone()) - .with_use_task(false); + let error_stream = MockExec::new( + vec![exec_err!("Test1"), exec_err!("Test2")], + Arc::clone(&schema), + ) + .with_use_task(false); let mut builder = RecordBatchReceiverStream::builder(schema, 2); - builder.run_input(Arc::new(error_stream), 0, task_ctx.clone()); + builder.run_input(Arc::new(error_stream), 0, Arc::clone(&task_ctx)); let mut stream = builder.build(); // get the first result, which should be an error @@ -560,7 +562,11 @@ mod test { let mut builder = RecordBatchReceiverStream::builder(input.schema(), num_partitions); for partition in 0..num_partitions { - builder.run_input(input.clone(), partition, task_ctx.clone()); + builder.run_input( + Arc::clone(&input) as Arc, + partition, + Arc::clone(&task_ctx), + ); } let mut stream = builder.build(); diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index ff57adde4e2e..5a9035c8dbfc 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -93,7 +93,7 @@ impl StreamingTableExec { let projected_output_ordering = projected_output_ordering.into_iter().collect::>(); let cache = Self::compute_properties( - projected_schema.clone(), + Arc::clone(&projected_schema), &projected_output_ordering, &partitions, infinite, @@ -240,7 +240,7 @@ impl ExecutionPlan for StreamingTableExec { let stream = self.partitions[partition].execute(ctx); let projected_stream = match self.projection.clone() { Some(projection) => Box::pin(RecordBatchStreamAdapter::new( - self.projected_schema.clone(), + Arc::clone(&self.projected_schema), stream.map(move |x| { x.and_then(|b| b.project(projection.as_ref()).map_err(Into::into)) }), @@ -327,7 +327,7 @@ mod test { /// Set the batches for the stream fn with_batches(mut self, batches: Vec) -> Self { let stream = TestPartitionStream::new_with_batches(batches); - self.schema = Some(stream.schema().clone()); + self.schema = Some(Arc::clone(stream.schema())); self.partitions = vec![Arc::new(stream)]; self } diff --git a/datafusion/physical-plan/src/test.rs b/datafusion/physical-plan/src/test.rs index 377b919bb407..f5b4a096018f 100644 --- a/datafusion/physical-plan/src/test.rs +++ b/datafusion/physical-plan/src/test.rs @@ -144,6 +144,9 @@ impl PartitionStream for TestPartitionStream { } fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { let stream = futures::stream::iter(self.batches.clone().into_iter().map(Ok)); - Box::pin(RecordBatchStreamAdapter::new(self.schema.clone(), stream)) + Box::pin(RecordBatchStreamAdapter::new( + Arc::clone(&self.schema), + stream, + )) } } diff --git a/datafusion/physical-plan/src/test/exec.rs b/datafusion/physical-plan/src/test/exec.rs index ad47a484c9f7..ac4eb1ca9e58 100644 --- a/datafusion/physical-plan/src/test/exec.rs +++ b/datafusion/physical-plan/src/test/exec.rs @@ -133,7 +133,7 @@ impl MockExec { /// ensure any poll loops are correct. This behavior can be /// changed with `with_use_task` pub fn new(data: Vec>, schema: SchemaRef) -> Self { - let cache = Self::compute_properties(schema.clone()); + let cache = Self::compute_properties(Arc::clone(&schema)); Self { data, schema, @@ -294,7 +294,7 @@ impl BarrierExec { pub fn new(data: Vec>, schema: SchemaRef) -> Self { // wait for all streams and the input let barrier = Arc::new(Barrier::new(data.len() + 1)); - let cache = Self::compute_properties(schema.clone(), &data); + let cache = Self::compute_properties(Arc::clone(&schema), &data); Self { data, schema, @@ -374,7 +374,7 @@ impl ExecutionPlan for BarrierExec { // task simply sends data in order after barrier is reached let data = self.data[partition].clone(); - let b = self.barrier.clone(); + let b = Arc::clone(&self.barrier); let tx = builder.tx(); builder.spawn(async move { println!("Partition {partition} waiting on barrier"); @@ -421,7 +421,7 @@ impl ErrorExec { DataType::Int64, true, )])); - let cache = Self::compute_properties(schema.clone()); + let cache = Self::compute_properties(schema); Self { cache } } @@ -591,7 +591,7 @@ pub struct BlockingExec { impl BlockingExec { /// Create new [`BlockingExec`] with a give schema and number of partitions. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { - let cache = Self::compute_properties(schema.clone(), n_partitions); + let cache = Self::compute_properties(Arc::clone(&schema), n_partitions); Self { schema, refs: Default::default(), @@ -735,7 +735,7 @@ impl PanicExec { /// partitions, which will each panic immediately. pub fn new(schema: SchemaRef, n_partitions: usize) -> Self { let batches_until_panics = vec![0; n_partitions]; - let cache = Self::compute_properties(schema.clone(), &batches_until_panics); + let cache = Self::compute_properties(Arc::clone(&schema), &batches_until_panics); Self { schema, batches_until_panics, @@ -845,7 +845,7 @@ impl Stream for PanicStream { if self.ready { self.batches_until_panic -= 1; self.ready = false; - let batch = RecordBatch::new_empty(self.schema.clone()); + let batch = RecordBatch::new_empty(Arc::clone(&self.schema)); return Poll::Ready(Some(Ok(batch))); } else { self.ready = true; diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 6a77bfaf3ccd..5366a5707696 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -131,7 +131,7 @@ impl TopK { ); Ok(Self { - schema: schema.clone(), + schema: Arc::clone(&schema), metrics: TopKMetrics::new(metrics, partition), reservation, batch_size, @@ -355,7 +355,7 @@ impl TopKHeap { /// high, as a single [`RecordBatch`], and a sorted vec of the /// current heap's contents pub fn emit_with_state(&mut self) -> Result<(RecordBatch, Vec)> { - let schema = self.store.schema().clone(); + let schema = Arc::clone(self.store.schema()); // generate sorted rows let topk_rows = std::mem::take(&mut self.inner).into_sorted_vec(); diff --git a/datafusion/physical-plan/src/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs index 1570778be69b..96bd0de3d37c 100644 --- a/datafusion/physical-plan/src/tree_node.rs +++ b/datafusion/physical-plan/src/tree_node.rs @@ -62,7 +62,7 @@ impl PlanContext { } pub fn update_plan_from_children(mut self) -> Result { - let children_plans = self.children.iter().map(|c| c.plan.clone()).collect(); + let children_plans = self.children.iter().map(|c| Arc::clone(&c.plan)).collect(); self.plan = with_new_children_if_necessary(self.plan, children_plans)?; Ok(self) diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 867cddeb7b41..b39c6aee82b9 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -449,7 +449,7 @@ impl ExecutionPlan for InterleaveExec { let mut input_stream_vec = vec![]; for input in self.inputs.iter() { if partition < input.output_partitioning().partition_count() { - input_stream_vec.push(input.execute(partition, context.clone())?); + input_stream_vec.push(input.execute(partition, Arc::clone(&context))?); } else { // Do not find a partition to execute break; @@ -550,7 +550,7 @@ impl CombinedRecordBatchStream { impl RecordBatchStream for CombinedRecordBatchStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -657,7 +657,7 @@ mod tests { in_data .iter() .map(|(expr, options)| PhysicalSortExpr { - expr: (*expr).clone(), + expr: Arc::clone(*expr), options: *options, }) .collect::>() @@ -842,11 +842,11 @@ mod tests { .map(|ordering| convert_to_sort_exprs(ordering)) .collect::>(); let child1 = Arc::new( - MemoryExec::try_new(&[], schema.clone(), None)? + MemoryExec::try_new(&[], Arc::clone(&schema), None)? .with_sort_information(first_orderings), ); let child2 = Arc::new( - MemoryExec::try_new(&[], schema.clone(), None)? + MemoryExec::try_new(&[], Arc::clone(&schema), None)? .with_sort_information(second_orderings), ); diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index e072b214fd36..bdd56f4b5aa4 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -83,7 +83,7 @@ impl UnnestExec { schema: SchemaRef, options: UnnestOptions, ) -> Self { - let cache = Self::compute_properties(&input, schema.clone()); + let cache = Self::compute_properties(&input, Arc::clone(&schema)); UnnestExec { input, @@ -147,10 +147,10 @@ impl ExecutionPlan for UnnestExec { children: Vec>, ) -> Result> { Ok(Arc::new(UnnestExec::new( - children[0].clone(), + Arc::clone(&children[0]), self.list_column_indices.clone(), self.struct_column_indices.clone(), - self.schema.clone(), + Arc::clone(&self.schema), self.options.clone(), ))) } @@ -169,7 +169,7 @@ impl ExecutionPlan for UnnestExec { Ok(Box::pin(UnnestStream { input, - schema: self.schema.clone(), + schema: Arc::clone(&self.schema), list_type_columns: self.list_column_indices.clone(), struct_column_indices: self.struct_column_indices.iter().copied().collect(), options: self.options.clone(), @@ -237,7 +237,7 @@ struct UnnestStream { impl RecordBatchStream for UnnestStream { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -330,13 +330,13 @@ fn flatten_struct_cols( data_type ), }, - None => Ok(vec![column_data.clone()]), + None => Ok(vec![Arc::clone(column_data)]), }) .collect::>>()? .into_iter() .flatten() .collect(); - Ok(RecordBatch::try_new(schema.clone(), columns_expanded)?) + Ok(RecordBatch::try_new(Arc::clone(schema), columns_expanded)?) } /// For each row in a `RecordBatch`, some list/struct columns need to be unnested. @@ -357,7 +357,7 @@ fn build_batch( let list_arrays: Vec = list_type_columns .iter() .map(|index| { - ColumnarValue::Array(batch.column(*index).clone()) + ColumnarValue::Array(Arc::clone(batch.column(*index))) .into_array(batch.num_rows()) }) .collect::>()?; @@ -372,7 +372,7 @@ fn build_batch( })? as usize }; if total_length == 0 { - return Ok(RecordBatch::new_empty(schema.clone())); + return Ok(RecordBatch::new_empty(Arc::clone(schema))); } // Unnest all the list arrays @@ -444,7 +444,7 @@ fn find_longest_length( .collect::>()?; let longest_length = list_lengths.iter().skip(1).try_fold( - list_lengths[0].clone(), + Arc::clone(&list_lengths[0]), |longest, current| { let is_lt = lt(&longest, ¤t)?; zip(&is_lt, ¤t, &longest) @@ -649,7 +649,7 @@ fn flatten_list_cols_from_indices( .iter() .enumerate() .map(|(col_idx, arr)| match unnested_list_arrays.get(&col_idx) { - Some(unnested_array) => Ok(unnested_array.clone()), + Some(unnested_array) => Ok(Arc::clone(unnested_array)), None => Ok(kernels::take::take(arr, indices, None)?), }) .collect::>>()?; @@ -813,27 +813,27 @@ mod tests { // Test with single ListArray // [A, B, C], [], NULL, [D], NULL, [NULL, F] let list_array = Arc::new(make_generic_array::()) as ArrayRef; - verify_longest_length(&[list_array.clone()], false, vec![3, 0, 0, 1, 0, 2])?; - verify_longest_length(&[list_array.clone()], true, vec![3, 0, 1, 1, 1, 2])?; + verify_longest_length(&[Arc::clone(&list_array)], false, vec![3, 0, 0, 1, 0, 2])?; + verify_longest_length(&[Arc::clone(&list_array)], true, vec![3, 0, 1, 1, 1, 2])?; // Test with single LargeListArray // [A, B, C], [], NULL, [D], NULL, [NULL, F] let list_array = Arc::new(make_generic_array::()) as ArrayRef; - verify_longest_length(&[list_array.clone()], false, vec![3, 0, 0, 1, 0, 2])?; - verify_longest_length(&[list_array.clone()], true, vec![3, 0, 1, 1, 1, 2])?; + verify_longest_length(&[Arc::clone(&list_array)], false, vec![3, 0, 0, 1, 0, 2])?; + verify_longest_length(&[Arc::clone(&list_array)], true, vec![3, 0, 1, 1, 1, 2])?; // Test with single FixedSizeListArray // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] let list_array = Arc::new(make_fixed_list()) as ArrayRef; - verify_longest_length(&[list_array.clone()], false, vec![2, 0, 2, 0, 2, 2])?; - verify_longest_length(&[list_array.clone()], true, vec![2, 1, 2, 1, 2, 2])?; + verify_longest_length(&[Arc::clone(&list_array)], false, vec![2, 0, 2, 0, 2, 2])?; + verify_longest_length(&[Arc::clone(&list_array)], true, vec![2, 1, 2, 1, 2, 2])?; // Test with multiple list arrays // [A, B, C], [], NULL, [D], NULL, [NULL, F] // [A, B], NULL, [C, D], NULL, [NULL, F], [NULL, NULL] let list1 = Arc::new(make_generic_array::()) as ArrayRef; let list2 = Arc::new(make_fixed_list()) as ArrayRef; - let list_arrays = vec![list1.clone(), list2.clone()]; + let list_arrays = vec![Arc::clone(&list1), Arc::clone(&list2)]; verify_longest_length(&list_arrays, false, vec![3, 0, 2, 1, 2, 2])?; verify_longest_length(&list_arrays, true, vec![3, 1, 2, 1, 2, 2])?; diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 4d385812d4a8..3ea27d62d80b 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -88,7 +88,7 @@ impl ValuesExec { .and_then(ScalarValue::iter_to_array) }) .collect::>>()?; - let batch = RecordBatch::try_new(schema.clone(), arr)?; + let batch = RecordBatch::try_new(Arc::clone(&schema), arr)?; let data: Vec = vec![batch]; Self::try_new_from_batches(schema, data) } @@ -114,7 +114,7 @@ impl ValuesExec { } } - let cache = Self::compute_properties(schema.clone()); + let cache = Self::compute_properties(Arc::clone(&schema)); Ok(ValuesExec { schema, data: batches, @@ -175,7 +175,7 @@ impl ExecutionPlan for ValuesExec { self: Arc, _: Vec>, ) -> Result> { - ValuesExec::try_new_from_batches(self.schema.clone(), self.data.clone()) + ValuesExec::try_new_from_batches(Arc::clone(&self.schema), self.data.clone()) .map(|e| Arc::new(e) as _) } @@ -193,7 +193,7 @@ impl ExecutionPlan for ValuesExec { Ok(Box::pin(MemoryStream::try_new( self.data(), - self.schema.clone(), + Arc::clone(&self.schema), None, )?)) } @@ -260,7 +260,7 @@ mod tests { DataType::UInt32, false, )])); - let _ = ValuesExec::try_new(schema.clone(), vec![vec![lit(1u32)]]).unwrap(); + let _ = ValuesExec::try_new(Arc::clone(&schema), vec![vec![lit(1u32)]]).unwrap(); // Test that a null value is rejected let _ = ValuesExec::try_new(schema, vec![vec![lit(ScalarValue::UInt32(None))]]) .unwrap_err(); 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 9eb29891703e..6311107f7b58 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -289,7 +289,7 @@ impl ExecutionPlan for BoundedWindowAggExec { ) -> Result> { Ok(Arc::new(BoundedWindowAggExec::try_new( self.window_expr.clone(), - children[0].clone(), + Arc::clone(&children[0]), self.partition_keys.clone(), self.input_order_mode.clone(), )?)) @@ -303,7 +303,7 @@ impl ExecutionPlan for BoundedWindowAggExec { let input = self.input.execute(partition, context)?; let search_mode = self.get_search_algo()?; let stream = Box::pin(BoundedWindowAggStream::new( - self.schema.clone(), + Arc::clone(&self.schema), self.window_expr.clone(), input, BaselineMetrics::new(&self.metrics, partition), @@ -394,7 +394,9 @@ trait PartitionSearcher: Send { // as it may not have the "correct" schema in terms of output // nullability constraints. For details, see the following issue: // https://github.com/apache/datafusion/issues/9320 - .or_insert_with(|| PartitionBatchState::new(self.input_schema().clone())); + .or_insert_with(|| { + PartitionBatchState::new(Arc::clone(self.input_schema())) + }); partition_batch_state.extend(&partition_batch)?; } @@ -513,7 +515,7 @@ impl PartitionSearcher for LinearSearch { let length = indices.len(); for (idx, window_agg_state) in window_agg_states.iter().enumerate() { let partition = &window_agg_state[&row]; - let values = partition.state.out_col.slice(0, length).clone(); + let values = Arc::clone(&partition.state.out_col.slice(0, length)); new_columns[idx].push(values); } let partition_batch_state = &mut partition_buffers[&row]; @@ -935,7 +937,7 @@ impl BoundedWindowAggStream { search_mode: Box, ) -> Result { let state = window_expr.iter().map(|_| IndexMap::new()).collect(); - let empty_batch = RecordBatch::new_empty(schema.clone()); + let empty_batch = RecordBatch::new_empty(Arc::clone(&schema)); Ok(Self { schema, input, @@ -957,7 +959,7 @@ impl BoundedWindowAggStream { cur_window_expr.evaluate_stateful(&self.partition_buffers, state)?; } - let schema = self.schema.clone(); + let schema = Arc::clone(&self.schema); let window_expr_out = self.search_mode.calculate_out_columns( &self.input_buffer, &self.window_agg_states, @@ -1114,7 +1116,7 @@ impl BoundedWindowAggStream { impl RecordBatchStream for BoundedWindowAggStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -1287,7 +1289,7 @@ mod tests { impl RecordBatchStream for TestStreamPartition { fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } @@ -1467,7 +1469,7 @@ mod tests { } let batch = RecordBatch::try_new( - schema.clone(), + Arc::clone(schema), vec![Arc::new(sn1_array.finish()), Arc::new(hash_array.finish())], )?; batches.push(batch); @@ -1500,7 +1502,7 @@ mod tests { // Source has 2 partitions let partitions = vec![ Arc::new(TestStreamPartition { - schema: schema.clone(), + schema: Arc::clone(&schema), batches: batches.clone(), idx: 0, state: PolingState::BatchReturn, @@ -1510,7 +1512,7 @@ mod tests { n_partition ]; let source = Arc::new(StreamingTableExec::try_new( - schema.clone(), + Arc::clone(&schema), partitions, None, orderings, @@ -1533,28 +1535,38 @@ mod tests { let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)])); // Create a new batch of data to insert into the table let batch = RecordBatch::try_new( - schema.clone(), + Arc::clone(&schema), vec![Arc::new(arrow_array::Int32Array::from(vec![1, 2, 3]))], )?; let memory_exec = MemoryExec::try_new( &[vec![batch.clone(), batch.clone(), batch.clone()]], - schema.clone(), + Arc::clone(&schema), None, ) .map(|e| Arc::new(e) as Arc)?; let col_a = col("a", &schema)?; - let nth_value_func1 = - NthValue::nth("nth_value(-1)", col_a.clone(), DataType::Int32, 1, false)? - .reverse_expr() - .unwrap(); - let nth_value_func2 = - NthValue::nth("nth_value(-2)", col_a.clone(), DataType::Int32, 2, false)? - .reverse_expr() - .unwrap(); + let nth_value_func1 = NthValue::nth( + "nth_value(-1)", + Arc::clone(&col_a), + DataType::Int32, + 1, + false, + )? + .reverse_expr() + .unwrap(); + let nth_value_func2 = NthValue::nth( + "nth_value(-2)", + Arc::clone(&col_a), + DataType::Int32, + 2, + false, + )? + .reverse_expr() + .unwrap(); let last_value_func = Arc::new(NthValue::last( "last", - col_a.clone(), + Arc::clone(&col_a), DataType::Int32, false, )) as _; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0622aad74cad..7f794556a241 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -64,11 +64,11 @@ pub fn schema_add_window_field( ) -> Result> { let data_types = args .iter() - .map(|e| e.clone().as_ref().data_type(schema)) + .map(|e| Arc::clone(e).as_ref().data_type(schema)) .collect::>>()?; let nullability = args .iter() - .map(|e| e.clone().as_ref().nullable(schema)) + .map(|e| Arc::clone(e).as_ref().nullable(schema)) .collect::>>()?; let window_expr_return_type = window_fn.return_type(&data_types, &nullability)?; let mut window_fields = schema @@ -288,7 +288,7 @@ fn create_built_in_window_expr( } } BuiltInWindowFunction::Lag => { - let arg = args[0].clone(); + let arg = Arc::clone(&args[0]); let shift_offset = get_scalar_value_from_args(args, 1)? .map(get_signed_integer) .map_or(Ok(None), |v| v.map(Some))?; @@ -304,7 +304,7 @@ fn create_built_in_window_expr( )) } BuiltInWindowFunction::Lead => { - let arg = args[0].clone(); + let arg = Arc::clone(&args[0]); let shift_offset = get_scalar_value_from_args(args, 1)? .map(get_signed_integer) .map_or(Ok(None), |v| v.map(Some))?; @@ -320,7 +320,7 @@ fn create_built_in_window_expr( )) } BuiltInWindowFunction::NthValue => { - let arg = args[0].clone(); + let arg = Arc::clone(&args[0]); let n = get_signed_integer( args[1] .as_any() @@ -338,7 +338,7 @@ fn create_built_in_window_expr( )?) } BuiltInWindowFunction::FirstValue => { - let arg = args[0].clone(); + let arg = Arc::clone(&args[0]); Arc::new(NthValue::first( name, arg, @@ -347,7 +347,7 @@ fn create_built_in_window_expr( )) } BuiltInWindowFunction::LastValue => { - let arg = args[0].clone(); + let arg = Arc::clone(&args[0]); Arc::new(NthValue::last( name, arg, @@ -429,13 +429,16 @@ pub(crate) fn calc_requirements< let mut sort_reqs = partition_by_exprs .into_iter() .map(|partition_by| { - PhysicalSortRequirement::new(partition_by.borrow().clone(), None) + PhysicalSortRequirement::new(Arc::clone(partition_by.borrow()), None) }) .collect::>(); for element in orderby_sort_exprs.into_iter() { let PhysicalSortExpr { expr, options } = element.borrow(); if !sort_reqs.iter().any(|e| e.expr.eq(expr)) { - sort_reqs.push(PhysicalSortRequirement::new(expr.clone(), Some(*options))); + sort_reqs.push(PhysicalSortRequirement::new( + Arc::clone(expr), + Some(*options), + )); } } // Convert empty result to None. Otherwise wrap result inside Some() @@ -464,7 +467,7 @@ pub(crate) fn get_partition_by_sort_exprs( ) -> Result { let ordered_partition_exprs = ordered_partition_by_indices .iter() - .map(|idx| partition_by_exprs[*idx].clone()) + .map(|idx| Arc::clone(&partition_by_exprs[*idx])) .collect::>(); // Make sure ordered section doesn't move over the partition by expression assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); @@ -485,7 +488,7 @@ pub(crate) fn window_equivalence_properties( ) -> EquivalenceProperties { // We need to update the schema, so we can not directly use // `input.equivalence_properties()`. - let mut window_eq_properties = EquivalenceProperties::new(schema.clone()) + let mut window_eq_properties = EquivalenceProperties::new(Arc::clone(schema)) .extend(input.equivalence_properties().clone()); for expr in window_expr { @@ -556,7 +559,7 @@ pub fn get_best_fitting_window( if window_expr.iter().all(|e| e.uses_bounded_memory()) { Ok(Some(Arc::new(BoundedWindowAggExec::try_new( window_expr, - input.clone(), + Arc::clone(input), physical_partition_keys.to_vec(), input_order_mode, )?) as _)) @@ -569,7 +572,7 @@ pub fn get_best_fitting_window( } else { Ok(Some(Arc::new(WindowAggExec::try_new( window_expr, - input.clone(), + Arc::clone(input), physical_partition_keys.to_vec(), )?) as _)) } @@ -594,7 +597,7 @@ pub fn get_window_mode( let mut partition_by_reqs: Vec = vec![]; let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement { - expr: partitionby_exprs[idx].clone(), + expr: Arc::clone(&partitionby_exprs[idx]), options: None, })); // Treat partition by exprs as constant. During analysis of requirements are satisfied. @@ -694,7 +697,7 @@ mod tests { let sort_exprs = sort_exprs.into_iter().collect(); Ok(Arc::new(StreamingTableExec::try_new( - schema.clone(), + Arc::clone(schema), vec![], None, Some(sort_exprs), diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index eb01da2ec094..b6330f65e0b7 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -79,7 +79,7 @@ impl WindowAggExec { let ordered_partition_by_indices = get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); - let cache = Self::compute_properties(schema.clone(), &input, &window_expr); + let cache = Self::compute_properties(Arc::clone(&schema), &input, &window_expr); Ok(Self { input, window_expr, @@ -220,7 +220,7 @@ impl ExecutionPlan for WindowAggExec { ) -> Result> { Ok(Arc::new(WindowAggExec::try_new( self.window_expr.clone(), - children[0].clone(), + Arc::clone(&children[0]), self.partition_keys.clone(), )?)) } @@ -232,7 +232,7 @@ impl ExecutionPlan for WindowAggExec { ) -> Result { let input = self.input.execute(partition, context)?; let stream = Box::pin(WindowAggStream::new( - self.schema.clone(), + Arc::clone(&self.schema), self.window_expr.clone(), input, BaselineMetrics::new(&self.metrics, partition), @@ -333,7 +333,7 @@ impl WindowAggStream { let _timer = self.baseline_metrics.elapsed_compute().timer(); let batch = concat_batches(&self.input.schema(), &self.batches)?; if batch.num_rows() == 0 { - return Ok(RecordBatch::new_empty(self.schema.clone())); + return Ok(RecordBatch::new_empty(Arc::clone(&self.schema))); } let partition_by_sort_keys = self @@ -366,7 +366,10 @@ impl WindowAggStream { let mut batch_columns = batch.columns().to_vec(); // calculate window cols batch_columns.extend_from_slice(&columns); - Ok(RecordBatch::try_new(self.schema.clone(), batch_columns)?) + Ok(RecordBatch::try_new( + Arc::clone(&self.schema), + batch_columns, + )?) } } @@ -412,6 +415,6 @@ impl WindowAggStream { impl RecordBatchStream for WindowAggStream { /// Get the schema fn schema(&self) -> SchemaRef { - self.schema.clone() + Arc::clone(&self.schema) } } diff --git a/datafusion/physical-plan/src/work_table.rs b/datafusion/physical-plan/src/work_table.rs index 003957947fec..5f3cf6e2aee8 100644 --- a/datafusion/physical-plan/src/work_table.rs +++ b/datafusion/physical-plan/src/work_table.rs @@ -110,7 +110,7 @@ pub struct WorkTableExec { impl WorkTableExec { /// Create a new execution plan for a worktable exec. pub fn new(name: String, schema: SchemaRef) -> Self { - let cache = Self::compute_properties(schema.clone()); + let cache = Self::compute_properties(Arc::clone(&schema)); Self { name, schema, @@ -123,7 +123,7 @@ impl WorkTableExec { pub(super) fn with_work_table(&self, work_table: Arc) -> Self { Self { name: self.name.clone(), - schema: self.schema.clone(), + schema: Arc::clone(&self.schema), metrics: ExecutionPlanMetricsSet::new(), work_table, cache: self.cache.clone(), @@ -185,7 +185,7 @@ impl ExecutionPlan for WorkTableExec { self: Arc, _: Vec>, ) -> Result> { - Ok(self.clone()) + Ok(Arc::clone(&self) as Arc) } /// Stream the batches that were written to the work table. @@ -202,7 +202,7 @@ impl ExecutionPlan for WorkTableExec { } let batch = self.work_table.take()?; Ok(Box::pin( - MemoryStream::try_new(batch.batches, self.schema.clone(), None)? + MemoryStream::try_new(batch.batches, Arc::clone(&self.schema), None)? .with_reservation(batch.reservation), )) } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index ce6c0c53c3fc..345765b08be3 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -500,7 +500,7 @@ enum AggregateFunction { // REGR_SYY = 33; // REGR_SXY = 34; // STRING_AGG = 35; - NTH_VALUE_AGG = 36; + // NTH_VALUE_AGG = 36; } message AggregateExprNode { diff --git a/datafusion/proto/src/bytes/mod.rs b/datafusion/proto/src/bytes/mod.rs index 83210cb4e41f..9188480431aa 100644 --- a/datafusion/proto/src/bytes/mod.rs +++ b/datafusion/proto/src/bytes/mod.rs @@ -39,7 +39,7 @@ use std::sync::Arc; use datafusion::execution::registry::FunctionRegistry; use datafusion::physical_plan::ExecutionPlan; use datafusion::prelude::SessionContext; -use datafusion_expr::planner::UserDefinedSQLPlanner; +use datafusion_expr::planner::ExprPlanner; mod registry; @@ -167,7 +167,7 @@ impl Serializeable for Expr { ) } - fn expr_planners(&self) -> Vec> { + fn expr_planners(&self) -> Vec> { vec![] } } diff --git a/datafusion/proto/src/bytes/registry.rs b/datafusion/proto/src/bytes/registry.rs index 075993e2ba76..eae2425f8ac1 100644 --- a/datafusion/proto/src/bytes/registry.rs +++ b/datafusion/proto/src/bytes/registry.rs @@ -20,7 +20,7 @@ use std::{collections::HashSet, sync::Arc}; use datafusion::execution::registry::FunctionRegistry; use datafusion_common::plan_err; use datafusion_common::Result; -use datafusion_expr::planner::UserDefinedSQLPlanner; +use datafusion_expr::planner::ExprPlanner; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; /// A default [`FunctionRegistry`] registry that does not resolve any @@ -56,7 +56,7 @@ impl FunctionRegistry for NoRegistry { plan_err!("No function registry provided to deserialize, so can not deserialize User Defined Window Function '{}'", udwf.inner().name()) } - fn expr_planners(&self) -> Vec> { + fn expr_planners(&self) -> Vec> { vec![] } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 347654e52b73..905f0d984955 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -535,7 +535,6 @@ impl serde::Serialize for AggregateFunction { Self::Min => "MIN", Self::Max => "MAX", Self::ArrayAgg => "ARRAY_AGG", - Self::NthValueAgg => "NTH_VALUE_AGG", }; serializer.serialize_str(variant) } @@ -550,7 +549,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "MIN", "MAX", "ARRAY_AGG", - "NTH_VALUE_AGG", ]; struct GeneratedVisitor; @@ -594,7 +592,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "MIN" => Ok(AggregateFunction::Min), "MAX" => Ok(AggregateFunction::Max), "ARRAY_AGG" => Ok(AggregateFunction::ArrayAgg), - "NTH_VALUE_AGG" => Ok(AggregateFunction::NthValueAgg), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index c74f172482b7..b16d26ee6e1e 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1924,7 +1924,7 @@ pub enum AggregateFunction { /// AVG = 3; /// COUNT = 4; /// APPROX_DISTINCT = 5; - ArrayAgg = 6, + /// /// VARIANCE = 7; /// VARIANCE_POP = 8; /// COVARIANCE = 9; @@ -1952,7 +1952,8 @@ pub enum AggregateFunction { /// REGR_SYY = 33; /// REGR_SXY = 34; /// STRING_AGG = 35; - NthValueAgg = 36, + /// NTH_VALUE_AGG = 36; + ArrayAgg = 6, } impl AggregateFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -1964,7 +1965,6 @@ impl AggregateFunction { AggregateFunction::Min => "MIN", AggregateFunction::Max => "MAX", AggregateFunction::ArrayAgg => "ARRAY_AGG", - AggregateFunction::NthValueAgg => "NTH_VALUE_AGG", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -1973,7 +1973,6 @@ impl AggregateFunction { "MIN" => Some(Self::Min), "MAX" => Some(Self::Max), "ARRAY_AGG" => Some(Self::ArrayAgg), - "NTH_VALUE_AGG" => Some(Self::NthValueAgg), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index f4fb69280436..095c6a50973a 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -145,7 +145,6 @@ impl From for AggregateFunction { protobuf::AggregateFunction::Min => Self::Min, protobuf::AggregateFunction::Max => Self::Max, protobuf::AggregateFunction::ArrayAgg => Self::ArrayAgg, - protobuf::AggregateFunction::NthValueAgg => Self::NthValue, } } } @@ -270,11 +269,7 @@ pub fn parse_expr( Ok(operands .into_iter() .reduce(|left, right| { - Expr::BinaryExpr(BinaryExpr::new( - Box::new(left), - op.clone(), - Box::new(right), - )) + Expr::BinaryExpr(BinaryExpr::new(Box::new(left), op, Box::new(right))) }) .expect("Binary expression could not be reduced to a single expression.")) } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 7570040a1d08..d8f8ea002b2d 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -117,7 +117,6 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::Min => Self::Min, AggregateFunction::Max => Self::Max, AggregateFunction::ArrayAgg => Self::ArrayAgg, - AggregateFunction::NthValue => Self::NthValueAgg, } } } @@ -377,9 +376,6 @@ pub fn serialize_expr( AggregateFunction::ArrayAgg => protobuf::AggregateFunction::ArrayAgg, AggregateFunction::Min => protobuf::AggregateFunction::Min, AggregateFunction::Max => protobuf::AggregateFunction::Max, - AggregateFunction::NthValue => { - protobuf::AggregateFunction::NthValueAgg - } }; let aggregate_expr = protobuf::AggregateExprNode { diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 23cdc666e701..5e982ad2afde 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -25,8 +25,8 @@ use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ ArrayAgg, BinaryExpr, CaseExpr, CastExpr, Column, CumeDist, DistinctArrayAgg, InListExpr, IsNotNullExpr, IsNullExpr, Literal, Max, Min, NegativeExpr, NotExpr, - NthValue, NthValueAgg, Ntile, OrderSensitiveArrayAgg, Rank, RankType, RowNumber, - TryCastExpr, WindowShift, + NthValue, Ntile, OrderSensitiveArrayAgg, Rank, RankType, RowNumber, TryCastExpr, + WindowShift, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; @@ -255,8 +255,6 @@ fn aggr_expr_to_aggr_fn(expr: &dyn AggregateExpr) -> Result { protobuf::AggregateFunction::Min } else if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::Max - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::NthValueAgg } else { return not_impl_err!("Aggregate function not supported: {expr:?}"); }; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 106247b2d441..d8d85ace1a29 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -38,7 +38,7 @@ use datafusion::datasource::physical_plan::{ }; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; -use datafusion::physical_expr::expressions::{Max, NthValueAgg}; +use datafusion::physical_expr::expressions::Max; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{PhysicalSortRequirement, ScalarFunctionExpr}; use datafusion::physical_plan::aggregates::{ @@ -81,6 +81,7 @@ use datafusion_expr::{ ScalarUDFImpl, Signature, SimpleAggregateUDF, WindowFrame, WindowFrameBound, }; use datafusion_functions_aggregate::average::avg_udaf; +use datafusion_functions_aggregate::nth_value::nth_value_udaf; use datafusion_functions_aggregate::string_agg::StringAgg; use datafusion_proto::physical_plan::{ AsExecutionPlan, DefaultPhysicalExtensionCodec, PhysicalExtensionCodec, @@ -362,15 +363,17 @@ fn rountrip_aggregate() -> Result<()> { false, )?], // NTH_VALUE - vec![Arc::new(NthValueAgg::new( - col("b", &schema)?, - 1, - "NTH_VALUE(b, 1)".to_string(), - DataType::Int64, + vec![udaf::create_aggregate_expr( + &nth_value_udaf(), + &[col("b", &schema)?, lit(1u64)], + &[], + &[], + &[], + &schema, + "NTH_VALUE(b, 1)", false, - Vec::new(), - Vec::new(), - ))], + false, + )?], // STRING_AGG vec![udaf::create_aggregate_expr( &AggregateUDF::new_from_impl(StringAgg::new()), diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index ea460cb3efc2..d9ddf57eb192 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -415,9 +415,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { // check udaf first let udaf = self.context_provider.get_aggregate_meta(name); - // Skip first value and last value, since we expect window builtin first/last value not udaf version + // Use the builtin window function instead of the user-defined aggregate function if udaf.as_ref().is_some_and(|udaf| { - udaf.name() != "first_value" && udaf.name() != "last_value" + udaf.name() != "first_value" + && udaf.name() != "last_value" + && udaf.name() != "nth_value" }) { Ok(WindowFunctionDefinition::AggregateUDF(udaf.unwrap())) } else { diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 6295821fa944..0546a101fcb2 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -21,7 +21,8 @@ use datafusion_expr::planner::PlannerResult; use datafusion_expr::planner::RawDictionaryExpr; use datafusion_expr::planner::RawFieldAccessExpr; use sqlparser::ast::{ - CastKind, DictionaryField, Expr as SQLExpr, Subscript, TrimWhereField, Value, + CastKind, DictionaryField, Expr as SQLExpr, StructField, Subscript, TrimWhereField, + Value, }; use datafusion_common::{ @@ -597,7 +598,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } SQLExpr::Struct { values, fields } => { - self.parse_struct(values, fields, schema, planner_context) + self.parse_struct(schema, planner_context, values, fields) } SQLExpr::Position { expr, r#in } => { self.sql_position_to_expr(*expr, *r#in, schema, planner_context) @@ -629,6 +630,36 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } } + /// Parses a struct(..) expression and plans it creation + fn parse_struct( + &self, + schema: &DFSchema, + planner_context: &mut PlannerContext, + values: Vec, + fields: Vec, + ) -> Result { + if !fields.is_empty() { + return not_impl_err!("Struct fields are not supported yet"); + } + let is_named_struct = values + .iter() + .any(|value| matches!(value, SQLExpr::Named { .. })); + + let mut create_struct_args = if is_named_struct { + self.create_named_struct_expr(values, schema, planner_context)? + } else { + self.create_struct_expr(values, schema, planner_context)? + }; + + for planner in self.planners.iter() { + match planner.plan_struct_literal(create_struct_args, is_named_struct)? { + PlannerResult::Planned(expr) => return Ok(expr), + PlannerResult::Original(args) => create_struct_args = args, + } + } + not_impl_err!("Struct not supported by UserDefinedExtensionPlanners: {create_struct_args:?}") + } + fn sql_position_to_expr( &self, substr_expr: SQLExpr, @@ -639,18 +670,18 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let substr = self.sql_expr_to_logical_expr(substr_expr, schema, planner_context)?; let fullstr = self.sql_expr_to_logical_expr(str_expr, schema, planner_context)?; - let mut extract_args = vec![fullstr, substr]; + let mut position_args = vec![fullstr, substr]; for planner in self.planners.iter() { - match planner.plan_position(extract_args)? { + match planner.plan_position(position_args)? { PlannerResult::Planned(expr) => return Ok(expr), PlannerResult::Original(args) => { - extract_args = args; + position_args = args; } } } not_impl_err!( - "Position not supported by UserDefinedExtensionPlanners: {extract_args:?}" + "Position not supported by UserDefinedExtensionPlanners: {position_args:?}" ) } @@ -683,37 +714,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { not_impl_err!("Unsupported dictionary literal: {raw_expr:?}") } - /// Parses a struct(..) expression - fn parse_struct( - &self, - values: Vec, - fields: Vec, - input_schema: &DFSchema, - planner_context: &mut PlannerContext, - ) -> Result { - if !fields.is_empty() { - return not_impl_err!("Struct fields are not supported yet"); - } - - if values - .iter() - .any(|value| matches!(value, SQLExpr::Named { .. })) - { - self.create_named_struct(values, input_schema, planner_context) - } else { - self.create_struct(values, input_schema, planner_context) - } - } - // Handles a call to struct(...) where the arguments are named. For example // `struct (v as foo, v2 as bar)` by creating a call to the `named_struct` function - fn create_named_struct( + fn create_named_struct_expr( &self, values: Vec, input_schema: &DFSchema, planner_context: &mut PlannerContext, - ) -> Result { - let args = values + ) -> Result> { + Ok(values .into_iter() .enumerate() .map(|(i, value)| { @@ -742,47 +751,24 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .collect::>>()? .into_iter() .flatten() - .collect(); - - let named_struct_func = self - .context_provider - .get_function_meta("named_struct") - .ok_or_else(|| { - internal_datafusion_err!("Unable to find expected 'named_struct' function") - })?; - - Ok(Expr::ScalarFunction(ScalarFunction::new_udf( - named_struct_func, - args, - ))) + .collect()) } // Handles a call to struct(...) where the arguments are not named. For example // `struct (v, v2)` by creating a call to the `struct` function // which will create a struct with fields named `c0`, `c1`, etc. - fn create_struct( + fn create_struct_expr( &self, values: Vec, input_schema: &DFSchema, planner_context: &mut PlannerContext, - ) -> Result { - let args = values + ) -> Result> { + values .into_iter() .map(|value| { self.sql_expr_to_logical_expr(value, input_schema, planner_context) }) - .collect::>>()?; - let struct_func = self - .context_provider - .get_function_meta("struct") - .ok_or_else(|| { - internal_datafusion_err!("Unable to find expected 'struct' function") - })?; - - Ok(Expr::ScalarFunction(ScalarFunction::new_udf( - struct_func, - args, - ))) + .collect::>>() } fn sql_in_list_to_expr( diff --git a/datafusion/sql/src/expr/substring.rs b/datafusion/sql/src/expr/substring.rs index f58c6f3b94d0..a0dfee1b9d90 100644 --- a/datafusion/sql/src/expr/substring.rs +++ b/datafusion/sql/src/expr/substring.rs @@ -16,9 +16,9 @@ // under the License. use crate::planner::{ContextProvider, PlannerContext, SqlToRel}; -use datafusion_common::{internal_datafusion_err, plan_err}; +use datafusion_common::{not_impl_err, plan_err}; use datafusion_common::{DFSchema, Result, ScalarValue}; -use datafusion_expr::expr::ScalarFunction; +use datafusion_expr::planner::PlannerResult; use datafusion_expr::Expr; use sqlparser::ast::Expr as SQLExpr; @@ -31,7 +31,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema: &DFSchema, planner_context: &mut PlannerContext, ) -> Result { - let args = match (substring_from, substring_for) { + let mut substring_args = match (substring_from, substring_for) { (Some(from_expr), Some(for_expr)) => { let arg = self.sql_expr_to_logical_expr(*expr, schema, planner_context)?; @@ -68,13 +68,17 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } }; - let fun = self - .context_provider - .get_function_meta("substr") - .ok_or_else(|| { - internal_datafusion_err!("Unable to find expected 'substr' function") - })?; + for planner in self.planners.iter() { + match planner.plan_substring(substring_args)? { + PlannerResult::Planned(expr) => return Ok(expr), + PlannerResult::Original(args) => { + substring_args = args; + } + } + } - Ok(Expr::ScalarFunction(ScalarFunction::new_udf(fun, args))) + not_impl_err!( + "Substring not supported by UserDefinedExtensionPlanners: {substring_args:?}" + ) } } diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 443cd64a940c..a77f0003f738 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -24,7 +24,7 @@ use arrow_schema::*; use datafusion_common::{ field_not_found, internal_err, plan_datafusion_err, DFSchemaRef, SchemaError, }; -use datafusion_expr::planner::UserDefinedSQLPlanner; +use datafusion_expr::planner::ExprPlanner; use sqlparser::ast::TimezoneInfo; use sqlparser::ast::{ArrayElemTypeDef, ExactNumberInfo}; use sqlparser::ast::{ColumnDef as SQLColumnDef, ColumnOption}; @@ -187,7 +187,7 @@ pub struct SqlToRel<'a, S: ContextProvider> { pub(crate) options: ParserOptions, pub(crate) normalizer: IdentNormalizer, /// user defined planner extensions - pub(crate) planners: Vec>, + pub(crate) planners: Vec>, } impl<'a, S: ContextProvider> SqlToRel<'a, S> { @@ -197,10 +197,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } /// add an user defined planner - pub fn with_user_defined_planner( - mut self, - planner: Arc, - ) -> Self { + pub fn with_user_defined_planner(mut self, planner: Arc) -> Self { self.planners.push(planner); self } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index 07af4bfeba9c..198186934c84 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -977,8 +977,13 @@ impl Unparser<'_> { } DataType::Float32 => Ok(ast::DataType::Float(None)), DataType::Float64 => Ok(ast::DataType::Double), - DataType::Timestamp(_, _) => { - not_impl_err!("Unsupported DataType: conversion: {data_type:?}") + DataType::Timestamp(_, tz) => { + let tz_info = match tz { + Some(_) => TimezoneInfo::WithTimeZone, + None => TimezoneInfo::None, + }; + + Ok(ast::DataType::Timestamp(None, tz_info)) } DataType::Date32 => Ok(ast::DataType::Date), DataType::Date64 => Ok(ast::DataType::Datetime(None)), @@ -1063,9 +1068,9 @@ mod tests { use std::ops::{Add, Sub}; use std::{any::Any, sync::Arc, vec}; + use arrow::datatypes::TimeUnit; use arrow::datatypes::{Field, Schema}; use arrow_schema::DataType::Int8; - use datafusion_common::TableReference; use datafusion_expr::{ case, col, cube, exists, grouping_set, interval_datetime_lit, @@ -1158,6 +1163,23 @@ mod tests { }), r#"CAST(a AS DATETIME)"#, ), + ( + Expr::Cast(Cast { + expr: Box::new(col("a")), + data_type: DataType::Timestamp( + TimeUnit::Nanosecond, + Some("+08:00".into()), + ), + }), + r#"CAST(a AS TIMESTAMP WITH TIME ZONE)"#, + ), + ( + Expr::Cast(Cast { + expr: Box::new(col("a")), + data_type: DataType::Timestamp(TimeUnit::Millisecond, None), + }), + r#"CAST(a AS TIMESTAMP)"#, + ), ( Expr::Cast(Cast { expr: Box::new(col("a")), diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index 342d45e7fb24..9a0a1d587433 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -39,16 +39,16 @@ EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1] as result GROUP BY a; ---- logical_plan -01)Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result -02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] +01)Projection: multiple_ordered_table.a, nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result +02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[nth_value(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] 03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan -01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] -02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +01)ProjectionExec: expr=[a@0 as a, nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -59,16 +59,16 @@ EXPLAIN SELECT a, NTH_VALUE(c, 1 ORDER BY c) as result GROUP BY a; ---- logical_plan -01)Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result -02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] +01)Projection: multiple_ordered_table.a, nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result +02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[nth_value(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] 03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan -01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] -02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +01)ProjectionExec: expr=[a@0 as a, nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -78,16 +78,16 @@ EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1 + 100] as result GROUP BY a; ---- logical_plan -01)Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result -02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(101)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] +01)Projection: multiple_ordered_table.a, nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS result +02)--Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[nth_value(multiple_ordered_table.c, Int64(101)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] AS nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] 03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan -01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] -02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +01)ProjectionExec: expr=[a@0 as a, nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 03)----SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true diff --git a/datafusion/sqllogictest/test_files/cse.slt b/datafusion/sqllogictest/test_files/cse.slt new file mode 100644 index 000000000000..3579c1c1635c --- /dev/null +++ b/datafusion/sqllogictest/test_files/cse.slt @@ -0,0 +1,173 @@ +# 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. + +statement ok +CREATE TABLE IF NOT EXISTS t1(a DOUBLE, b DOUBLE) + +# Trivial common expression +query TT +EXPLAIN SELECT + a + 1 AS c1, + a + 1 AS c2 +FROM t1 +---- +logical_plan +01)Projection: __common_expr_1 AS c1, __common_expr_1 AS c2 +02)--Projection: t1.a + Float64(1) AS __common_expr_1 +03)----TableScan: t1 projection=[a] +physical_plan +01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2] +02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] +03)----MemoryExec: partitions=1, partition_sizes=[0] + +# Common volatile expression +query TT +EXPLAIN SELECT + a + random() AS c1, + a + random() AS c2 +FROM t1 +---- +logical_plan +01)Projection: t1.a + random() AS c1, t1.a + random() AS c2 +02)--TableScan: t1 projection=[a] +physical_plan +01)ProjectionExec: expr=[a@0 + random() as c1, a@0 + random() as c2] +02)--MemoryExec: partitions=1, partition_sizes=[0] + +# Volatile expression with non-volatile common child +query TT +EXPLAIN SELECT + a + 1 + random() AS c1, + a + 1 + random() AS c2 +FROM t1 +---- +logical_plan +01)Projection: __common_expr_1 + random() AS c1, __common_expr_1 + random() AS c2 +02)--Projection: t1.a + Float64(1) AS __common_expr_1 +03)----TableScan: t1 projection=[a] +physical_plan +01)ProjectionExec: expr=[__common_expr_1@0 + random() as c1, __common_expr_1@0 + random() as c2] +02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1] +03)----MemoryExec: partitions=1, partition_sizes=[0] + +# Volatile expression with non-volatile common children +query TT +EXPLAIN SELECT + a + 1 + random() + (a + 2) AS c1, + a + 1 + random() + (a + 2) AS c2 +FROM t1 +---- +logical_plan +01)Projection: __common_expr_1 + random() + __common_expr_2 AS c1, __common_expr_1 + random() + __common_expr_2 AS c2 +02)--Projection: t1.a + Float64(1) AS __common_expr_1, t1.a + Float64(2) AS __common_expr_2 +03)----TableScan: t1 projection=[a] +physical_plan +01)ProjectionExec: expr=[__common_expr_1@0 + random() + __common_expr_2@1 as c1, __common_expr_1@0 + random() + __common_expr_2@1 as c2] +02)--ProjectionExec: expr=[a@0 + 1 as __common_expr_1, a@0 + 2 as __common_expr_2] +03)----MemoryExec: partitions=1, partition_sizes=[0] + +# Common short-circuit expression +query TT +EXPLAIN SELECT + a = 0 AND b = 0 AS c1, + a = 0 AND b = 0 AS c2, + a = 0 OR b = 0 AS c3, + a = 0 OR b = 0 AS c4, + CASE WHEN (a = 0) THEN 0 ELSE 1 END AS c5, + CASE WHEN (a = 0) THEN 0 ELSE 1 END AS c6 +FROM t1 +---- +logical_plan +01)Projection: __common_expr_1 AS c1, __common_expr_1 AS c2, __common_expr_2 AS c3, __common_expr_2 AS c4, __common_expr_3 AS c5, __common_expr_3 AS c6 +02)--Projection: t1.a = Float64(0) AND t1.b = Float64(0) AS __common_expr_1, t1.a = Float64(0) OR t1.b = Float64(0) AS __common_expr_2, CASE WHEN t1.a = Float64(0) THEN Int64(0) ELSE Int64(1) END AS __common_expr_3 +03)----TableScan: t1 projection=[a, b] +physical_plan +01)ProjectionExec: expr=[__common_expr_1@0 as c1, __common_expr_1@0 as c2, __common_expr_2@1 as c3, __common_expr_2@1 as c4, __common_expr_3@2 as c5, __common_expr_3@2 as c6] +02)--ProjectionExec: expr=[a@0 = 0 AND b@1 = 0 as __common_expr_1, a@0 = 0 OR b@1 = 0 as __common_expr_2, CASE WHEN a@0 = 0 THEN 0 ELSE 1 END as __common_expr_3] +03)----MemoryExec: partitions=1, partition_sizes=[0] + +# Common children of short-circuit expression +# TODO: consider surely executed children of "short circuited"s for CSE. i.e. `a = 0`, `a = 2`, `a = 4` should be extracted +query TT +EXPLAIN SELECT + a = 0 AND b = 0 AS c1, + a = 0 AND b = 1 AS c2, + b = 2 AND a = 1 AS c3, + b = 3 AND a = 1 AS c4, + a = 2 OR b = 4 AS c5, + a = 2 OR b = 5 AS c6, + b = 6 OR a = 3 AS c7, + b = 7 OR a = 3 AS c8, + CASE WHEN (a = 4) THEN 0 ELSE 1 END AS c9, + CASE WHEN (a = 4) THEN 0 ELSE 2 END AS c10, + CASE WHEN (b = 8) THEN a + 1 ELSE 0 END AS c11, + CASE WHEN (b = 9) THEN a + 1 ELSE 0 END AS c12, + CASE WHEN (b = 10) THEN 0 ELSE a + 2 END AS c13, + CASE WHEN (b = 11) THEN 0 ELSE a + 2 END AS c14 +FROM t1 +---- +logical_plan +01)Projection: t1.a = Float64(0) AND t1.b = Float64(0) AS c1, t1.a = Float64(0) AND t1.b = Float64(1) AS c2, t1.b = Float64(2) AND t1.a = Float64(1) AS c3, t1.b = Float64(3) AND t1.a = Float64(1) AS c4, t1.a = Float64(2) OR t1.b = Float64(4) AS c5, t1.a = Float64(2) OR t1.b = Float64(5) AS c6, t1.b = Float64(6) OR t1.a = Float64(3) AS c7, t1.b = Float64(7) OR t1.a = Float64(3) AS c8, CASE WHEN t1.a = Float64(4) THEN Int64(0) ELSE Int64(1) END AS c9, CASE WHEN t1.a = Float64(4) THEN Int64(0) ELSE Int64(2) END AS c10, CASE WHEN t1.b = Float64(8) THEN t1.a + Float64(1) ELSE Float64(0) END AS c11, CASE WHEN t1.b = Float64(9) THEN t1.a + Float64(1) ELSE Float64(0) END AS c12, CASE WHEN t1.b = Float64(10) THEN Float64(0) ELSE t1.a + Float64(2) END AS c13, CASE WHEN t1.b = Float64(11) THEN Float64(0) ELSE t1.a + Float64(2) END AS c14 +02)--TableScan: t1 projection=[a, b] +physical_plan +01)ProjectionExec: expr=[a@0 = 0 AND b@1 = 0 as c1, a@0 = 0 AND b@1 = 1 as c2, b@1 = 2 AND a@0 = 1 as c3, b@1 = 3 AND a@0 = 1 as c4, a@0 = 2 OR b@1 = 4 as c5, a@0 = 2 OR b@1 = 5 as c6, b@1 = 6 OR a@0 = 3 as c7, b@1 = 7 OR a@0 = 3 as c8, CASE WHEN a@0 = 4 THEN 0 ELSE 1 END as c9, CASE WHEN a@0 = 4 THEN 0 ELSE 2 END as c10, CASE WHEN b@1 = 8 THEN a@0 + 1 ELSE 0 END as c11, CASE WHEN b@1 = 9 THEN a@0 + 1 ELSE 0 END as c12, CASE WHEN b@1 = 10 THEN 0 ELSE a@0 + 2 END as c13, CASE WHEN b@1 = 11 THEN 0 ELSE a@0 + 2 END as c14] +02)--MemoryExec: partitions=1, partition_sizes=[0] + +# Common children of volatile, short-circuit expression +# TODO: consider surely executed children of "short circuited"s for CSE. i.e. `a = 0`, `a = 2`, `a = 4` should be extracted +query TT +EXPLAIN SELECT + a = 0 AND b = random() AS c1, + a = 0 AND b = 1 + random() AS c2, + b = 2 + random() AND a = 1 AS c3, + b = 3 + random() AND a = 1 AS c4, + a = 2 OR b = 4 + random() AS c5, + a = 2 OR b = 5 + random() AS c6, + b = 6 + random() OR a = 3 AS c7, + b = 7 + random() OR a = 3 AS c8, + CASE WHEN (a = 4) THEN random() ELSE 1 END AS c9, + CASE WHEN (a = 4) THEN random() ELSE 2 END AS c10, + CASE WHEN (b = 8 + random()) THEN a + 1 ELSE 0 END AS c11, + CASE WHEN (b = 9 + random()) THEN a + 1 ELSE 0 END AS c12, + CASE WHEN (b = 10 + random()) THEN 0 ELSE a + 2 END AS c13, + CASE WHEN (b = 11 + random()) THEN 0 ELSE a + 2 END AS c14 +FROM t1 +---- +logical_plan +01)Projection: t1.a = Float64(0) AND t1.b = random() AS c1, t1.a = Float64(0) AND t1.b = Float64(1) + random() AS c2, t1.b = Float64(2) + random() AND t1.a = Float64(1) AS c3, t1.b = Float64(3) + random() AND t1.a = Float64(1) AS c4, t1.a = Float64(2) OR t1.b = Float64(4) + random() AS c5, t1.a = Float64(2) OR t1.b = Float64(5) + random() AS c6, t1.b = Float64(6) + random() OR t1.a = Float64(3) AS c7, t1.b = Float64(7) + random() OR t1.a = Float64(3) AS c8, CASE WHEN t1.a = Float64(4) THEN random() ELSE Float64(1) END AS c9, CASE WHEN t1.a = Float64(4) THEN random() ELSE Float64(2) END AS c10, CASE WHEN t1.b = Float64(8) + random() THEN t1.a + Float64(1) ELSE Float64(0) END AS c11, CASE WHEN t1.b = Float64(9) + random() THEN t1.a + Float64(1) ELSE Float64(0) END AS c12, CASE WHEN t1.b = Float64(10) + random() THEN Float64(0) ELSE t1.a + Float64(2) END AS c13, CASE WHEN t1.b = Float64(11) + random() THEN Float64(0) ELSE t1.a + Float64(2) END AS c14 +02)--TableScan: t1 projection=[a, b] +physical_plan +01)ProjectionExec: expr=[a@0 = 0 AND b@1 = random() as c1, a@0 = 0 AND b@1 = 1 + random() as c2, b@1 = 2 + random() AND a@0 = 1 as c3, b@1 = 3 + random() AND a@0 = 1 as c4, a@0 = 2 OR b@1 = 4 + random() as c5, a@0 = 2 OR b@1 = 5 + random() as c6, b@1 = 6 + random() OR a@0 = 3 as c7, b@1 = 7 + random() OR a@0 = 3 as c8, CASE WHEN a@0 = 4 THEN random() ELSE 1 END as c9, CASE WHEN a@0 = 4 THEN random() ELSE 2 END as c10, CASE WHEN b@1 = 8 + random() THEN a@0 + 1 ELSE 0 END as c11, CASE WHEN b@1 = 9 + random() THEN a@0 + 1 ELSE 0 END as c12, CASE WHEN b@1 = 10 + random() THEN 0 ELSE a@0 + 2 END as c13, CASE WHEN b@1 = 11 + random() THEN 0 ELSE a@0 + 2 END as c14] +02)--MemoryExec: partitions=1, partition_sizes=[0] + +# Common volatile children of short-circuit expression +query TT +EXPLAIN SELECT + a = random() AND b = 0 AS c1, + a = random() AND b = 1 AS c2, + a = 2 + random() OR b = 4 AS c3, + a = 2 + random() OR b = 5 AS c4, + CASE WHEN (a = 4 + random()) THEN 0 ELSE 1 END AS c5, + CASE WHEN (a = 4 + random()) THEN 0 ELSE 2 END AS c6 +FROM t1 +---- +logical_plan +01)Projection: t1.a = random() AND t1.b = Float64(0) AS c1, t1.a = random() AND t1.b = Float64(1) AS c2, t1.a = Float64(2) + random() OR t1.b = Float64(4) AS c3, t1.a = Float64(2) + random() OR t1.b = Float64(5) AS c4, CASE WHEN t1.a = Float64(4) + random() THEN Int64(0) ELSE Int64(1) END AS c5, CASE WHEN t1.a = Float64(4) + random() THEN Int64(0) ELSE Int64(2) END AS c6 +02)--TableScan: t1 projection=[a, b] +physical_plan +01)ProjectionExec: expr=[a@0 = random() AND b@1 = 0 as c1, a@0 = random() AND b@1 = 1 as c2, a@0 = 2 + random() OR b@1 = 4 as c3, a@0 = 2 + random() OR b@1 = 5 as c4, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 1 END as c5, CASE WHEN a@0 = 4 + random() THEN 0 ELSE 2 END as c6] +02)--MemoryExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 593de07f7d26..df66bffab8e8 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3865,3 +3865,182 @@ logical_plan 01)SubqueryAlias: b 02)--Projection: Int64(1) AS a 03)----EmptyRelation + + +statement ok +set datafusion.execution.target_partitions = 1; + +statement ok +set datafusion.explain.logical_plan_only = false; + +statement ok +set datafusion.execution.batch_size = 3; + +# Right Hash Joins preserve the right ordering +# No nulls on build side: +statement ok +CREATE TABLE left_table_no_nulls(a INT UNSIGNED, b INT UNSIGNED) +AS VALUES +(11, 1), +(12, 3), +(13, 5), +(14, 2), +(15, 4); + +statement ok +CREATE TABLE right_table_no_nulls(a INT UNSIGNED, b INT UNSIGNED) +AS VALUES +(21, 1), +(22, 2), +(23, 3), +(24, 4); + +query IIII +SELECT * FROM ( + SELECT * from left_table_no_nulls +) as lhs RIGHT JOIN ( + SELECT * from right_table_no_nulls + ORDER BY b +) AS rhs ON lhs.b=rhs.b +---- +11 1 21 1 +14 2 22 2 +12 3 23 3 +15 4 24 4 + +query TT +EXPLAIN SELECT * FROM ( + SELECT * from left_table_no_nulls +) as lhs RIGHT JOIN ( + SELECT * from right_table_no_nulls + ORDER BY b +) AS rhs ON lhs.b=rhs.b +---- +logical_plan +01)Right Join: lhs.b = rhs.b +02)--SubqueryAlias: lhs +03)----TableScan: left_table_no_nulls projection=[a, b] +04)--SubqueryAlias: rhs +05)----Sort: right_table_no_nulls.b ASC NULLS LAST +06)------TableScan: right_table_no_nulls projection=[a, b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=3 +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@1, b@1)] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] +05)------MemoryExec: partitions=1, partition_sizes=[1] + + +# Missing probe index in the middle of the batch: +statement ok +CREATE TABLE left_table_missing_probe(a INT UNSIGNED, b INT UNSIGNED) +AS VALUES +(11, 1), +(12, 2), +(13, 3), +(14, 6), +(15, 8); + +statement ok +CREATE TABLE right_table_missing_probe(a INT UNSIGNED, b INT UNSIGNED) +AS VALUES +(21, 1), +(22, 4), +(23, 6), +(24, 7), +(25, 8); + +query IIII +SELECT * FROM ( + SELECT * from left_table_missing_probe +) as lhs RIGHT JOIN ( + SELECT * from right_table_missing_probe + ORDER BY b +) AS rhs ON lhs.b=rhs.b +---- +11 1 21 1 +NULL NULL 22 4 +14 6 23 6 +NULL NULL 24 7 +15 8 25 8 + +query TT +EXPLAIN SELECT * FROM ( + SELECT * from left_table_no_nulls +) as lhs RIGHT JOIN ( + SELECT * from right_table_no_nulls + ORDER BY b +) AS rhs ON lhs.b=rhs.b +---- +logical_plan +01)Right Join: lhs.b = rhs.b +02)--SubqueryAlias: lhs +03)----TableScan: left_table_no_nulls projection=[a, b] +04)--SubqueryAlias: rhs +05)----Sort: right_table_no_nulls.b ASC NULLS LAST +06)------TableScan: right_table_no_nulls projection=[a, b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=3 +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@1, b@1)] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] +05)------MemoryExec: partitions=1, partition_sizes=[1] + + +# Null build indices: +statement ok +CREATE TABLE left_table_append_null_build(a INT UNSIGNED, b INT UNSIGNED) +AS VALUES +(11, 1), +(12, 1), +(13, 5), +(14, 5), +(15, 3); + +statement ok +CREATE TABLE right_table_append_null_build(a INT UNSIGNED, b INT UNSIGNED) +AS VALUES +(21, 4), +(22, 5), +(23, 6), +(24, 7), +(25, 8); + +query IIII +SELECT * FROM ( + SELECT * from left_table_append_null_build +) as lhs RIGHT JOIN ( + SELECT * from right_table_append_null_build + ORDER BY b +) AS rhs ON lhs.b=rhs.b +---- +NULL NULL 21 4 +13 5 22 5 +14 5 22 5 +NULL NULL 23 6 +NULL NULL 24 7 +NULL NULL 25 8 + + +query TT +EXPLAIN SELECT * FROM ( + SELECT * from left_table_no_nulls +) as lhs RIGHT JOIN ( + SELECT * from right_table_no_nulls + ORDER BY b +) AS rhs ON lhs.b=rhs.b +---- +logical_plan +01)Right Join: lhs.b = rhs.b +02)--SubqueryAlias: lhs +03)----TableScan: left_table_no_nulls projection=[a, b] +04)--SubqueryAlias: rhs +05)----Sort: right_table_no_nulls.b ASC NULLS LAST +06)------TableScan: right_table_no_nulls projection=[a, b] +physical_plan +01)CoalesceBatchesExec: target_batch_size=3 +02)--HashJoinExec: mode=CollectLeft, join_type=Right, on=[(b@1, b@1)] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)----SortExec: expr=[b@1 ASC NULLS LAST], preserve_partitioning=[false] +05)------MemoryExec: partitions=1, partition_sizes=[1] + diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index 83a374d7862e..9e7ef9632ad3 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -40,7 +40,7 @@ itertools = { workspace = true } object_store = { workspace = true } pbjson-types = "0.6" prost = "0.12" -substrait = { version = "0.35.0", features = ["serde"] } +substrait = { version = "0.36.0", features = ["serde"] } url = { workspace = true } [dev-dependencies] diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 03692399e1b3..89a6dde51e42 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -27,12 +27,13 @@ use datafusion::common::{ substrait_err, DFSchema, DFSchemaRef, }; use datafusion::execution::FunctionRegistry; -use datafusion::logical_expr::expr::{InSubquery, Sort}; +use datafusion::logical_expr::expr::{Exists, InSubquery, Sort}; use datafusion::logical_expr::{ aggregate_function, expr::find_df_window_func, Aggregate, BinaryExpr, Case, EmptyRelation, Expr, ExprSchemable, LogicalPlan, Operator, Projection, Values, }; +use substrait::proto::expression::subquery::set_predicate::PredicateOp; use url::Url; use crate::variation_const::{ @@ -54,12 +55,12 @@ use datafusion::logical_expr::{ use datafusion::prelude::JoinType; use datafusion::sql::TableReference; use datafusion::{ - error::{DataFusionError, Result}, + error::Result, logical_expr::utils::split_conjunction, prelude::{Column, SessionContext}, scalar::ScalarValue, }; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::str::FromStr; use std::sync::Arc; use substrait::proto::exchange_rel::ExchangeKind; @@ -403,22 +404,33 @@ pub async fn from_substrait_rel( let mut input = LogicalPlanBuilder::from( from_substrait_rel(ctx, input, extensions).await?, ); + let mut names: HashSet = HashSet::new(); let mut exprs: Vec = vec![]; for e in &p.expressions { let x = from_substrait_rex(ctx, e, input.clone().schema(), extensions) .await?; // if the expression is WindowFunction, wrap in a Window relation - // before returning and do not add to list of this Projection's expression list - // otherwise, add expression to the Projection's expression list - match &*x { - Expr::WindowFunction(_) => { - input = input.window(vec![x.as_ref().clone()])?; - exprs.push(x.as_ref().clone()); - } - _ => { - exprs.push(x.as_ref().clone()); - } + if let Expr::WindowFunction(_) = x.as_ref() { + // Adding the same expression here and in the project below + // works because the project's builder uses columnize_expr(..) + // to transform it into a column reference + input = input.window(vec![x.as_ref().clone()])? + } + // Ensure the expression has a unique display name, so that project's + // validate_unique_names doesn't fail + let name = x.display_name()?; + let mut new_name = name.clone(); + let mut i = 0; + while names.contains(&new_name) { + new_name = format!("{}__temp__{}", name, i); + i += 1; + } + names.insert(new_name.clone()); + if new_name != name { + exprs.push(x.as_ref().clone().alias(new_name.clone())); + } else { + exprs.push(x.as_ref().clone()); } } input.project(exprs)?.build() @@ -1142,7 +1154,7 @@ pub async fn from_substrait_rex( Arc::try_unwrap(expr) .unwrap_or_else(|arc: Arc| (*arc).clone()), ), // Avoid cloning if possible - op: op.clone(), + op, right: Box::new(arg), })), None => Arc::new(arg), @@ -1249,10 +1261,7 @@ pub async fn from_substrait_rex( Some(subquery_type) => match subquery_type { SubqueryType::InPredicate(in_predicate) => { if in_predicate.needles.len() != 1 { - Err(DataFusionError::Substrait( - "InPredicate Subquery type must have exactly one Needle expression" - .to_string(), - )) + substrait_err!("InPredicate Subquery type must have exactly one Needle expression") } else { let needle_expr = &in_predicate.needles[0]; let haystack_expr = &in_predicate.haystack; @@ -1297,6 +1306,32 @@ pub async fn from_substrait_rex( outer_ref_columns, }))) } + SubqueryType::SetPredicate(predicate) => { + match predicate.predicate_op() { + // exist + PredicateOp::Exists => { + let relation = &predicate.tuples; + let plan = from_substrait_rel( + ctx, + &relation.clone().unwrap_or_default(), + extensions, + ) + .await?; + let outer_ref_columns = plan.all_out_ref_exprs(); + Ok(Arc::new(Expr::Exists(Exists::new( + Subquery { + subquery: Arc::new(plan), + outer_ref_columns, + }, + false, + )))) + } + other_type => substrait_err!( + "unimplemented type {:?} for set predicate", + other_type + ), + } + } other_type => { substrait_err!("Subquery type {:?} not implemented", other_type) } diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 959542080161..8d039a050249 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -664,12 +664,7 @@ fn to_substrait_join_expr( extension_info, )?; // AND with existing expression - exprs.push(make_binary_op_scalar_func( - &l, - &r, - eq_op.clone(), - extension_info, - )); + exprs.push(make_binary_op_scalar_func(&l, &r, eq_op, extension_info)); } let join_expr: Option = exprs.into_iter().reduce(|acc: Expression, e: Expression| { @@ -1167,12 +1162,7 @@ pub fn to_substrait_rex( let l = to_substrait_rex(ctx, left, schema, col_ref_offset, extension_info)?; let r = to_substrait_rex(ctx, right, schema, col_ref_offset, extension_info)?; - Ok(make_binary_op_scalar_func( - &l, - &r, - op.clone(), - extension_info, - )) + Ok(make_binary_op_scalar_func(&l, &r, *op, extension_info)) } Expr::Case(Case { expr, diff --git a/datafusion/substrait/tests/cases/consumer_integration.rs b/datafusion/substrait/tests/cases/consumer_integration.rs index a8bbeb444a23..5d565c037852 100644 --- a/datafusion/substrait/tests/cases/consumer_integration.rs +++ b/datafusion/substrait/tests/cases/consumer_integration.rs @@ -90,6 +90,40 @@ mod tests { Ok(ctx) } + async fn create_context_tpch4() -> Result { + let ctx = SessionContext::new(); + + let registrations = vec![ + ("FILENAME_PLACEHOLDER_0", "tests/testdata/tpch/orders.csv"), + ("FILENAME_PLACEHOLDER_1", "tests/testdata/tpch/lineitem.csv"), + ]; + + for (table_name, file_path) in registrations { + register_csv(&ctx, table_name, file_path).await?; + } + + Ok(ctx) + } + + async fn create_context_tpch5() -> Result { + let ctx = SessionContext::new(); + + let registrations = vec![ + ("FILENAME_PLACEHOLDER_0", "tests/testdata/tpch/customer.csv"), + ("FILENAME_PLACEHOLDER_1", "tests/testdata/tpch/orders.csv"), + ("FILENAME_PLACEHOLDER_2", "tests/testdata/tpch/lineitem.csv"), + ("FILENAME_PLACEHOLDER_3", "tests/testdata/tpch/supplier.csv"), + ("NATION", "tests/testdata/tpch/nation.csv"), + ("REGION", "tests/testdata/tpch/region.csv"), + ]; + + for (table_name, file_path) in registrations { + register_csv(&ctx, table_name, file_path).await?; + } + + Ok(ctx) + } + #[tokio::test] async fn tpch_test_1() -> Result<()> { let ctx = create_context_tpch1().await?; @@ -180,4 +214,56 @@ mod tests { \n TableScan: FILENAME_PLACEHOLDER_1 projection=[o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment]\n TableScan: FILENAME_PLACEHOLDER_2 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]"); Ok(()) } + + #[tokio::test] + async fn tpch_test_4() -> Result<()> { + let ctx = create_context_tpch4().await?; + let path = "tests/testdata/tpch_substrait_plans/query_4.json"; + let proto = serde_json::from_reader::<_, Plan>(BufReader::new( + File::open(path).expect("file not found"), + )) + .expect("failed to parse json"); + let plan = from_substrait_plan(&ctx, &proto).await?; + let plan_str = format!("{:?}", plan); + assert_eq!(plan_str, "Projection: FILENAME_PLACEHOLDER_0.o_orderpriority AS O_ORDERPRIORITY, count(Int64(1)) AS ORDER_COUNT\ + \n Sort: FILENAME_PLACEHOLDER_0.o_orderpriority ASC NULLS LAST\ + \n Aggregate: groupBy=[[FILENAME_PLACEHOLDER_0.o_orderpriority]], aggr=[[count(Int64(1))]]\ + \n Projection: FILENAME_PLACEHOLDER_0.o_orderpriority\ + \n Filter: FILENAME_PLACEHOLDER_0.o_orderdate >= CAST(Utf8(\"1993-07-01\") AS Date32) AND FILENAME_PLACEHOLDER_0.o_orderdate < CAST(Utf8(\"1993-10-01\") AS Date32) AND EXISTS ()\ + \n Subquery:\ + \n Filter: FILENAME_PLACEHOLDER_1.l_orderkey = FILENAME_PLACEHOLDER_1.l_orderkey AND FILENAME_PLACEHOLDER_1.l_commitdate < FILENAME_PLACEHOLDER_1.l_receiptdate\ + \n TableScan: FILENAME_PLACEHOLDER_1 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]\ + \n TableScan: FILENAME_PLACEHOLDER_0 projection=[o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment]"); + Ok(()) + } + + #[tokio::test] + async fn tpch_test_5() -> Result<()> { + let ctx = create_context_tpch5().await?; + let path = "tests/testdata/tpch_substrait_plans/query_5.json"; + let proto = serde_json::from_reader::<_, Plan>(BufReader::new( + File::open(path).expect("file not found"), + )) + .expect("failed to parse json"); + + let plan = from_substrait_plan(&ctx, &proto).await?; + let plan_str = format!("{:?}", plan); + assert_eq!(plan_str, "Projection: NATION.n_name AS N_NAME, sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) AS REVENUE\ + \n Sort: sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount) DESC NULLS FIRST\ + \n Aggregate: groupBy=[[NATION.n_name]], aggr=[[sum(FILENAME_PLACEHOLDER_2.l_extendedprice * Int32(1) - FILENAME_PLACEHOLDER_2.l_discount)]]\ + \n Projection: NATION.n_name, FILENAME_PLACEHOLDER_2.l_extendedprice * (CAST(Int32(1) AS Decimal128(19, 0)) - FILENAME_PLACEHOLDER_2.l_discount)\ + \n Filter: FILENAME_PLACEHOLDER_0.c_custkey = FILENAME_PLACEHOLDER_1.o_custkey AND FILENAME_PLACEHOLDER_2.l_orderkey = FILENAME_PLACEHOLDER_1.o_orderkey AND FILENAME_PLACEHOLDER_2.l_suppkey = FILENAME_PLACEHOLDER_3.s_suppkey AND FILENAME_PLACEHOLDER_0.c_nationkey = FILENAME_PLACEHOLDER_3.s_nationkey AND FILENAME_PLACEHOLDER_3.s_nationkey = NATION.n_nationkey AND NATION.n_regionkey = REGION.r_regionkey AND REGION.r_name = CAST(Utf8(\"ASIA\") AS Utf8) AND FILENAME_PLACEHOLDER_1.o_orderdate >= CAST(Utf8(\"1994-01-01\") AS Date32) AND FILENAME_PLACEHOLDER_1.o_orderdate < CAST(Utf8(\"1995-01-01\") AS Date32)\ + \n Inner Join: Filter: Boolean(true)\ + \n Inner Join: Filter: Boolean(true)\ + \n Inner Join: Filter: Boolean(true)\ + \n Inner Join: Filter: Boolean(true)\ + \n Inner Join: Filter: Boolean(true)\ + \n TableScan: FILENAME_PLACEHOLDER_0 projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment]\ + \n TableScan: FILENAME_PLACEHOLDER_1 projection=[o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment]\ + \n TableScan: FILENAME_PLACEHOLDER_2 projection=[l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment]\ + \n TableScan: FILENAME_PLACEHOLDER_3 projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment]\ + \n TableScan: NATION projection=[n_nationkey, n_name, n_regionkey, n_comment]\ + \n TableScan: REGION projection=[r_regionkey, r_name, r_comment]"); + Ok(()) + } } diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 52cfa50683a0..2893b1a31a26 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -751,6 +751,22 @@ async fn roundtrip_values_duplicate_column_join() -> Result<()> { .await } +#[tokio::test] +async fn duplicate_column() -> Result<()> { + // Substrait does not keep column names (aliases) in the plan, rather it operates on column indices + // only. DataFusion however, is strict about not having duplicate column names appear in the plan. + // This test confirms that we generate aliases for columns in the plan which would otherwise have + // colliding names. + assert_expected_plan( + "SELECT a + 1 as sum_a, a + 1 as sum_a_2 FROM data", + "Projection: data.a + Int64(1) AS sum_a, data.a + Int64(1) AS data.a + Int64(1)__temp__0 AS sum_a_2\ + \n Projection: data.a + Int64(1)\ + \n TableScan: data projection=[a]", + true, + ) + .await +} + /// Construct a plan that cast columns. Only those SQL types are supported for now. #[tokio::test] async fn new_test_grammar() -> Result<()> { diff --git a/datafusion/substrait/tests/testdata/tpch_substrait_plans/query_4.json b/datafusion/substrait/tests/testdata/tpch_substrait_plans/query_4.json new file mode 100644 index 000000000000..6e946cefdd13 --- /dev/null +++ b/datafusion/substrait/tests/testdata/tpch_substrait_plans/query_4.json @@ -0,0 +1,540 @@ +{ + "extensionUris": [{ + "extensionUriAnchor": 4, + "uri": "/functions_aggregate_generic.yaml" + }, { + "extensionUriAnchor": 1, + "uri": "/functions_boolean.yaml" + }, { + "extensionUriAnchor": 2, + "uri": "/functions_datetime.yaml" + }, { + "extensionUriAnchor": 3, + "uri": "/functions_comparison.yaml" + }], + "extensions": [{ + "extensionFunction": { + "extensionUriReference": 1, + "functionAnchor": 0, + "name": "and:bool" + } + }, { + "extensionFunction": { + "extensionUriReference": 2, + "functionAnchor": 1, + "name": "gte:date_date" + } + }, { + "extensionFunction": { + "extensionUriReference": 2, + "functionAnchor": 2, + "name": "lt:date_date" + } + }, { + "extensionFunction": { + "extensionUriReference": 3, + "functionAnchor": 3, + "name": "equal:any1_any1" + } + }, { + "extensionFunction": { + "extensionUriReference": 4, + "functionAnchor": 4, + "name": "count:opt" + } + }], + "relations": [{ + "root": { + "input": { + "sort": { + "common": { + "direct": { + } + }, + "input": { + "aggregate": { + "common": { + "direct": { + } + }, + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [9] + } + }, + "input": { + "filter": { + "common": { + "direct": { + } + }, + "input": { + "read": { + "common": { + "direct": { + } + }, + "baseSchema": { + "names": ["O_ORDERKEY", "O_CUSTKEY", "O_ORDERSTATUS", "O_TOTALPRICE", "O_ORDERDATE", "O_ORDERPRIORITY", "O_CLERK", "O_SHIPPRIORITY", "O_COMMENT"], + "struct": { + "types": [{ + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, { + "fixedChar": { + "length": 1, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "fixedChar": { + "length": 15, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "fixedChar": { + "length": 15, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "varchar": { + "length": 79, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "local_files": { + "items": [ + { + "uri_file": "file://FILENAME_PLACEHOLDER_0", + "parquet": {} + } + ] + } + } + }, + "condition": { + "scalarFunction": { + "functionReference": 0, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [{ + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [{ + "value": { + "selection": { + "directReference": { + "structField": { + "field": 4 + } + }, + "rootReference": { + } + } + } + }, { + "value": { + "cast": { + "type": { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "input": { + "literal": { + "fixedChar": "1993-07-01", + "nullable": false, + "typeVariationReference": 0 + } + }, + "failureBehavior": "FAILURE_BEHAVIOR_UNSPECIFIED" + } + } + }] + } + } + }, { + "value": { + "scalarFunction": { + "functionReference": 2, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [{ + "value": { + "selection": { + "directReference": { + "structField": { + "field": 4 + } + }, + "rootReference": { + } + } + } + }, { + "value": { + "cast": { + "type": { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "input": { + "literal": { + "fixedChar": "1993-10-01", + "nullable": false, + "typeVariationReference": 0 + } + }, + "failureBehavior": "FAILURE_BEHAVIOR_UNSPECIFIED" + } + } + }] + } + } + }, { + "value": { + "subquery": { + "setPredicate": { + "predicateOp": "PREDICATE_OP_EXISTS", + "tuples": { + "filter": { + "common": { + "direct": { + } + }, + "input": { + "read": { + "common": { + "direct": { + } + }, + "baseSchema": { + "names": ["L_ORDERKEY", "L_PARTKEY", "L_SUPPKEY", "L_LINENUMBER", "L_QUANTITY", "L_EXTENDEDPRICE", "L_DISCOUNT", "L_TAX", "L_RETURNFLAG", "L_LINESTATUS", "L_SHIPDATE", "L_COMMITDATE", "L_RECEIPTDATE", "L_SHIPINSTRUCT", "L_SHIPMODE", "L_COMMENT"], + "struct": { + "types": [{ + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "fixedChar": { + "length": 1, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "fixedChar": { + "length": 1, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "fixedChar": { + "length": 25, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "fixedChar": { + "length": 10, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "varchar": { + "length": 44, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "local_files": { + "items": [ + { + "uri_file": "file://FILENAME_PLACEHOLDER_1", + "parquet": {} + } + ] + } + } + }, + "condition": { + "scalarFunction": { + "functionReference": 0, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [{ + "value": { + "scalarFunction": { + "functionReference": 3, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [{ + "value": { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": { + } + } + } + }, { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "outerReference": { + "stepsOut": 1 + } + } + } + }] + } + } + }, { + "value": { + "scalarFunction": { + "functionReference": 2, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [{ + "value": { + "selection": { + "directReference": { + "structField": { + "field": 11 + } + }, + "rootReference": { + } + } + } + }, { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 12 + } + }, + "rootReference": { + } + } + } + }] + } + } + }] + } + } + } + } + } + } + } + }] + } + } + } + }, + "expressions": [{ + "selection": { + "directReference": { + "structField": { + "field": 5 + } + }, + "rootReference": { + } + } + }] + } + }, + "groupings": [{ + "groupingExpressions": [{ + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": { + } + } + }] + }], + "measures": [{ + "measure": { + "functionReference": 4, + "args": [], + "sorts": [], + "phase": "AGGREGATION_PHASE_INITIAL_TO_RESULT", + "outputType": { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "invocation": "AGGREGATION_INVOCATION_ALL", + "arguments": [] + } + }] + } + }, + "sorts": [{ + "expr": { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": { + } + } + }, + "direction": "SORT_DIRECTION_ASC_NULLS_LAST" + }] + } + }, + "names": ["O_ORDERPRIORITY", "ORDER_COUNT"] + } + }], + "expectedTypeUrls": [] +} diff --git a/datafusion/substrait/tests/testdata/tpch_substrait_plans/query_5.json b/datafusion/substrait/tests/testdata/tpch_substrait_plans/query_5.json new file mode 100644 index 000000000000..75b82a305eb3 --- /dev/null +++ b/datafusion/substrait/tests/testdata/tpch_substrait_plans/query_5.json @@ -0,0 +1,1254 @@ +{ + "extensionUris": [ + { + "extensionUriAnchor": 1, + "uri": "/functions_boolean.yaml" + }, + { + "extensionUriAnchor": 4, + "uri": "/functions_arithmetic_decimal.yaml" + }, + { + "extensionUriAnchor": 3, + "uri": "/functions_datetime.yaml" + }, + { + "extensionUriAnchor": 2, + "uri": "/functions_comparison.yaml" + } + ], + "extensions": [ + { + "extensionFunction": { + "extensionUriReference": 1, + "functionAnchor": 0, + "name": "and:bool" + } + }, + { + "extensionFunction": { + "extensionUriReference": 2, + "functionAnchor": 1, + "name": "equal:any1_any1" + } + }, + { + "extensionFunction": { + "extensionUriReference": 3, + "functionAnchor": 2, + "name": "gte:date_date" + } + }, + { + "extensionFunction": { + "extensionUriReference": 3, + "functionAnchor": 3, + "name": "lt:date_date" + } + }, + { + "extensionFunction": { + "extensionUriReference": 4, + "functionAnchor": 4, + "name": "multiply:opt_decimal_decimal" + } + }, + { + "extensionFunction": { + "extensionUriReference": 4, + "functionAnchor": 5, + "name": "subtract:opt_decimal_decimal" + } + }, + { + "extensionFunction": { + "extensionUriReference": 4, + "functionAnchor": 6, + "name": "sum:opt_decimal" + } + } + ], + "relations": [ + { + "root": { + "input": { + "sort": { + "common": { + "direct": {} + }, + "input": { + "aggregate": { + "common": { + "direct": {} + }, + "input": { + "project": { + "common": { + "emit": { + "outputMapping": [ + 47, + 48 + ] + } + }, + "input": { + "filter": { + "common": { + "direct": {} + }, + "input": { + "join": { + "common": { + "direct": {} + }, + "left": { + "join": { + "common": { + "direct": {} + }, + "left": { + "join": { + "common": { + "direct": {} + }, + "left": { + "join": { + "common": { + "direct": {} + }, + "left": { + "join": { + "common": { + "direct": {} + }, + "left": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "C_CUSTKEY", + "C_NAME", + "C_ADDRESS", + "C_NATIONKEY", + "C_PHONE", + "C_ACCTBAL", + "C_MKTSEGMENT", + "C_COMMENT" + ], + "struct": { + "types": [ + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "varchar": { + "length": 25, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "varchar": { + "length": 40, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "fixedChar": { + "length": 15, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fixedChar": { + "length": 10, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "varchar": { + "length": 117, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "local_files": { + "items": [ + { + "uri_file": "file://FILENAME_PLACEHOLDER_0", + "parquet": {} + } + ] + } + } + }, + "right": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "O_ORDERKEY", + "O_CUSTKEY", + "O_ORDERSTATUS", + "O_TOTALPRICE", + "O_ORDERDATE", + "O_ORDERPRIORITY", + "O_CLERK", + "O_SHIPPRIORITY", + "O_COMMENT" + ], + "struct": { + "types": [ + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "fixedChar": { + "length": 1, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fixedChar": { + "length": 15, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fixedChar": { + "length": 15, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "varchar": { + "length": 79, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "local_files": { + "items": [ + { + "uri_file": "file://FILENAME_PLACEHOLDER_1", + "parquet": {} + } + ] + } + } + }, + "expression": { + "literal": { + "boolean": true, + "nullable": false, + "typeVariationReference": 0 + } + }, + "type": "JOIN_TYPE_INNER" + } + }, + "right": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "L_ORDERKEY", + "L_PARTKEY", + "L_SUPPKEY", + "L_LINENUMBER", + "L_QUANTITY", + "L_EXTENDEDPRICE", + "L_DISCOUNT", + "L_TAX", + "L_RETURNFLAG", + "L_LINESTATUS", + "L_SHIPDATE", + "L_COMMITDATE", + "L_RECEIPTDATE", + "L_SHIPINSTRUCT", + "L_SHIPMODE", + "L_COMMENT" + ], + "struct": { + "types": [ + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "i32": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fixedChar": { + "length": 1, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fixedChar": { + "length": 1, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fixedChar": { + "length": 25, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fixedChar": { + "length": 10, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "varchar": { + "length": 44, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "local_files": { + "items": [ + { + "uri_file": "file://FILENAME_PLACEHOLDER_2", + "parquet": {} + } + ] + } + } + }, + "expression": { + "literal": { + "boolean": true, + "nullable": false, + "typeVariationReference": 0 + } + }, + "type": "JOIN_TYPE_INNER" + } + }, + "right": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "S_SUPPKEY", + "S_NAME", + "S_ADDRESS", + "S_NATIONKEY", + "S_PHONE", + "S_ACCTBAL", + "S_COMMENT" + ], + "struct": { + "types": [ + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "fixedChar": { + "length": 25, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "varchar": { + "length": 40, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "fixedChar": { + "length": 15, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "varchar": { + "length": 101, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "local_files": { + "items": [ + { + "uri_file": "file://FILENAME_PLACEHOLDER_3", + "parquet": {} + } + ] + } + } + }, + "expression": { + "literal": { + "boolean": true, + "nullable": false, + "typeVariationReference": 0 + } + }, + "type": "JOIN_TYPE_INNER" + } + }, + "right": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "N_NATIONKEY", + "N_NAME", + "N_REGIONKEY", + "N_COMMENT" + ], + "struct": { + "types": [ + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "fixedChar": { + "length": 25, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "varchar": { + "length": 152, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "namedTable": { + "names": [ + "NATION" + ] + } + } + }, + "expression": { + "literal": { + "boolean": true, + "nullable": false, + "typeVariationReference": 0 + } + }, + "type": "JOIN_TYPE_INNER" + } + }, + "right": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "R_REGIONKEY", + "R_NAME", + "R_COMMENT" + ], + "struct": { + "types": [ + { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + { + "fixedChar": { + "length": 25, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "varchar": { + "length": 152, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "namedTable": { + "names": [ + "REGION" + ] + } + } + }, + "expression": { + "literal": { + "boolean": true, + "nullable": false, + "typeVariationReference": 0 + } + }, + "type": "JOIN_TYPE_INNER" + } + }, + "condition": { + "scalarFunction": { + "functionReference": 0, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 9 + } + }, + "rootReference": {} + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 17 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 8 + } + }, + "rootReference": {} + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 19 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 33 + } + }, + "rootReference": {} + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 3 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 36 + } + }, + "rootReference": {} + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 36 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 40 + } + }, + "rootReference": {} + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 42 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 44 + } + }, + "rootReference": {} + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 1, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 45 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "cast": { + "type": { + "fixedChar": { + "length": 25, + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "input": { + "literal": { + "fixedChar": "ASIA", + "nullable": false, + "typeVariationReference": 0 + } + }, + "failureBehavior": "FAILURE_BEHAVIOR_UNSPECIFIED" + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 2, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 12 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "cast": { + "type": { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "input": { + "literal": { + "fixedChar": "1994-01-01", + "nullable": false, + "typeVariationReference": 0 + } + }, + "failureBehavior": "FAILURE_BEHAVIOR_UNSPECIFIED" + } + } + } + ] + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 3, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 12 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "cast": { + "type": { + "date": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "input": { + "literal": { + "fixedChar": "1995-01-01", + "nullable": false, + "typeVariationReference": 0 + } + }, + "failureBehavior": "FAILURE_BEHAVIOR_UNSPECIFIED" + } + } + } + ] + } + } + } + ] + } + } + } + }, + "expressions": [ + { + "selection": { + "directReference": { + "structField": { + "field": 41 + } + }, + "rootReference": {} + } + }, + { + "scalarFunction": { + "functionReference": 4, + "args": [], + "outputType": { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 22 + } + }, + "rootReference": {} + } + } + }, + { + "value": { + "scalarFunction": { + "functionReference": 5, + "args": [], + "outputType": { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [ + { + "value": { + "cast": { + "type": { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "input": { + "literal": { + "i32": 1, + "nullable": false, + "typeVariationReference": 0 + } + }, + "failureBehavior": "FAILURE_BEHAVIOR_UNSPECIFIED" + } + } + }, + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 23 + } + }, + "rootReference": {} + } + } + } + ] + } + } + } + ] + } + } + ] + } + }, + "groupings": [ + { + "groupingExpressions": [ + { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": {} + } + } + ] + } + ], + "measures": [ + { + "measure": { + "functionReference": 6, + "args": [], + "sorts": [], + "phase": "AGGREGATION_PHASE_INITIAL_TO_RESULT", + "outputType": { + "decimal": { + "scale": 0, + "precision": 19, + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "invocation": "AGGREGATION_INVOCATION_ALL", + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + }, + "rootReference": {} + } + } + } + ] + } + } + ] + } + }, + "sorts": [ + { + "expr": { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + }, + "rootReference": {} + } + }, + "direction": "SORT_DIRECTION_DESC_NULLS_FIRST" + } + ] + } + }, + "names": [ + "N_NAME", + "REVENUE" + ] + } + } + ], + "expectedTypeUrls": [] +} \ No newline at end of file diff --git a/dev/changelog/40.0.0.md b/dev/changelog/40.0.0.md new file mode 100644 index 000000000000..72143ae48b28 --- /dev/null +++ b/dev/changelog/40.0.0.md @@ -0,0 +1,371 @@ + + +# Apache DataFusion 40.0.0 Changelog + +This release consists of 263 commits from 64 contributors. See credits at the end of this changelog for more information. + +**Breaking changes:** + +- Convert `StringAgg` to UDAF [#10945](https://github.com/apache/datafusion/pull/10945) (lewiszlw) +- Convert `bool_and` & `bool_or` to UDAF [#11009](https://github.com/apache/datafusion/pull/11009) (jcsherin) +- Convert Average to UDAF #10942 [#10964](https://github.com/apache/datafusion/pull/10964) (dharanad) +- fix: remove the Sized requirement on ExecutionPlan::name() [#11047](https://github.com/apache/datafusion/pull/11047) (waynexia) +- Return `&Arc` reference to inner trait object [#11103](https://github.com/apache/datafusion/pull/11103) (linhr) +- Support COPY TO Externally Defined File Formats, add FileType trait [#11060](https://github.com/apache/datafusion/pull/11060) (devinjdangelo) +- expose table name in proto extension codec [#11139](https://github.com/apache/datafusion/pull/11139) (leoyvens) +- fix(typo): unqualifed to unqualified [#11159](https://github.com/apache/datafusion/pull/11159) (waynexia) +- Consolidate `Filter::remove_aliases` into `Expr::unalias_nested` [#11001](https://github.com/apache/datafusion/pull/11001) (alamb) +- Convert `nth_value` to UDAF [#11287](https://github.com/apache/datafusion/pull/11287) (jcsherin) + +**Implemented enhancements:** + +- feat: Add support for Int8 and Int16 data types in data page statistics [#10931](https://github.com/apache/datafusion/pull/10931) (Weijun-H) +- feat: add CliSessionContext trait for cli [#10890](https://github.com/apache/datafusion/pull/10890) (tshauck) +- feat(optimizer): handle partial anchored regex cases and improve doc [#10977](https://github.com/apache/datafusion/pull/10977) (waynexia) +- feat: support uint data page extraction [#11018](https://github.com/apache/datafusion/pull/11018) (tshauck) +- feat: propagate EmptyRelation for more join types [#10963](https://github.com/apache/datafusion/pull/10963) (tshauck) +- feat: Add method to add analyzer rules to SessionContext [#10849](https://github.com/apache/datafusion/pull/10849) (pingsutw) +- feat: Support duplicate column names in Joins in Substrait consumer [#11049](https://github.com/apache/datafusion/pull/11049) (Blizzara) +- feat: Add support for Timestamp data types in data page statistics. [#11123](https://github.com/apache/datafusion/pull/11123) (efredine) +- feat: Add support for `Binary`/`LargeBinary`/`Utf8`/`LargeUtf8` data types in data page statistics [#11136](https://github.com/apache/datafusion/pull/11136) (PsiACE) +- feat: Support Map type in Substrait conversions [#11129](https://github.com/apache/datafusion/pull/11129) (Blizzara) +- feat: Conditionally allow to keep partition_by columns when using PARTITIONED BY enhancement [#11107](https://github.com/apache/datafusion/pull/11107) (hveiga) +- feat: enable "substring" as a UDF in addition to "substr" [#11277](https://github.com/apache/datafusion/pull/11277) (Blizzara) + +**Fixed bugs:** + +- fix: use total ordering in the min & max accumulator for floats [#10627](https://github.com/apache/datafusion/pull/10627) (westonpace) +- fix: Support double quotes in `date_part` [#10833](https://github.com/apache/datafusion/pull/10833) (Weijun-H) +- fix: Ignore nullability of list elements when consuming Substrait [#10874](https://github.com/apache/datafusion/pull/10874) (Blizzara) +- fix: Support `NOT IN ()` via anti join [#10936](https://github.com/apache/datafusion/pull/10936) (akoshchiy) +- fix: CTEs defined in a subquery can escape their scope [#10954](https://github.com/apache/datafusion/pull/10954) (jonahgao) +- fix: Fix the incorrect null joined rows for SMJ outer join with join filter [#10892](https://github.com/apache/datafusion/pull/10892) (viirya) +- fix: gcd returns negative results [#11099](https://github.com/apache/datafusion/pull/11099) (jonahgao) +- fix: LCM panicked due to overflow [#11131](https://github.com/apache/datafusion/pull/11131) (jonahgao) +- fix: Support dictionary type in parquet metadata statistics. [#11169](https://github.com/apache/datafusion/pull/11169) (efredine) +- fix: Ignore nullability in Substrait structs [#11130](https://github.com/apache/datafusion/pull/11130) (Blizzara) +- fix: typo in comment about FinalPhysicalPlan [#11181](https://github.com/apache/datafusion/pull/11181) (c8ef) +- fix: Support Substrait's compound names also for window functions [#11163](https://github.com/apache/datafusion/pull/11163) (Blizzara) +- fix: Incorrect LEFT JOIN evaluation result on OR conditions [#11203](https://github.com/apache/datafusion/pull/11203) (viirya) +- fix: Be more lenient in interpreting input args for builtin window functions [#11199](https://github.com/apache/datafusion/pull/11199) (Blizzara) +- fix: correctly handle Substrait windows with rows bounds (and validate executability of test plans) [#11278](https://github.com/apache/datafusion/pull/11278) (Blizzara) +- fix: When consuming Substrait, temporarily rename clashing duplicate columns [#11329](https://github.com/apache/datafusion/pull/11329) (Blizzara) + +**Documentation updates:** + +- Minor: Clarify `SessionContext::state` docs [#10847](https://github.com/apache/datafusion/pull/10847) (alamb) +- Minor: Update SIGMOD paper reference url [#10860](https://github.com/apache/datafusion/pull/10860) (alamb) +- docs(variance): Correct typos in comments [#10844](https://github.com/apache/datafusion/pull/10844) (pingsutw) +- Add missing code close tick in LiteralGuarantee docs [#10859](https://github.com/apache/datafusion/pull/10859) (adriangb) +- Minor: Add more docs and examples for `Transformed` and `TransformedResult` [#11003](https://github.com/apache/datafusion/pull/11003) (alamb) +- doc: Update links in the documantation [#11044](https://github.com/apache/datafusion/pull/11044) (Weijun-H) +- Minor: Examples cleanup + more docs in pruning example [#11086](https://github.com/apache/datafusion/pull/11086) (alamb) +- Minor: refine documentation pointing to examples [#11110](https://github.com/apache/datafusion/pull/11110) (alamb) +- Fix running in Docker instructions [#11141](https://github.com/apache/datafusion/pull/11141) (findepi) +- docs: add example for custom file format with `COPY TO` [#11174](https://github.com/apache/datafusion/pull/11174) (tshauck) +- Fix docs wordings [#11226](https://github.com/apache/datafusion/pull/11226) (findepi) +- Fix count() docs around including null values [#11293](https://github.com/apache/datafusion/pull/11293) (findepi) + +**Other:** + +- chore: Prepare 39.0.0-rc1 [#10828](https://github.com/apache/datafusion/pull/10828) (andygrove) +- Remove expr_fn::sum and replace them with function stub [#10816](https://github.com/apache/datafusion/pull/10816) (jayzhan211) +- Debug print as many fields as possible for `SessionState` [#10818](https://github.com/apache/datafusion/pull/10818) (lewiszlw) +- Prune Parquet RowGroup in a single call to `PruningPredicate::prune`, update StatisticsExtractor API [#10802](https://github.com/apache/datafusion/pull/10802) (alamb) +- Remove Built-in sum and Rename to lowercase `sum` [#10831](https://github.com/apache/datafusion/pull/10831) (jayzhan211) +- Convert `stddev` and `stddev_pop` to UDAF [#10834](https://github.com/apache/datafusion/pull/10834) (goldmedal) +- Introduce expr builder for aggregate function [#10560](https://github.com/apache/datafusion/pull/10560) (jayzhan211) +- chore: Improve change log generator [#10841](https://github.com/apache/datafusion/pull/10841) (andygrove) +- Support user defined `ParquetAccessPlan` in `ParquetExec`, validation to `ParquetAccessPlan::select` [#10813](https://github.com/apache/datafusion/pull/10813) (alamb) +- Convert `VariancePopulation` to UDAF [#10836](https://github.com/apache/datafusion/pull/10836) (mknaw) +- Convert `approx_median` to UDAF [#10840](https://github.com/apache/datafusion/pull/10840) (goldmedal) +- MINOR: use workspace deps in proto-common (upgrade object store dependency) [#10848](https://github.com/apache/datafusion/pull/10848) (waynexia) +- Minor: add `Window::try_new_with_schema` constructor [#10850](https://github.com/apache/datafusion/pull/10850) (sadboy) +- Add support for reading CSV files with comments [#10467](https://github.com/apache/datafusion/pull/10467) (bbannier) +- Convert approx_distinct to UDAF [#10851](https://github.com/apache/datafusion/pull/10851) (Lordworms) +- minor: add proto-common crate to release instructions [#10858](https://github.com/apache/datafusion/pull/10858) (andygrove) +- Implement TPCH substrait integration teset, support tpch_1 [#10842](https://github.com/apache/datafusion/pull/10842) (Lordworms) +- Remove unecessary passing around of `suffix: &str` in `pruning.rs`'s `RequiredColumns` [#10863](https://github.com/apache/datafusion/pull/10863) (adriangb) +- chore: Make DFSchema::datatype_is_logically_equal function public [#10867](https://github.com/apache/datafusion/pull/10867) (advancedxy) +- Bump braces from 3.0.2 to 3.0.3 in /datafusion/wasmtest/datafusion-wasm-app [#10865](https://github.com/apache/datafusion/pull/10865) (dependabot[bot]) +- Docs: Add `unnest` to SQL Reference [#10839](https://github.com/apache/datafusion/pull/10839) (gloomweaver) +- Support correct output column names and struct field names when consuming/producing Substrait [#10829](https://github.com/apache/datafusion/pull/10829) (Blizzara) +- Make Logical Plans more readable by removing extra aliases [#10832](https://github.com/apache/datafusion/pull/10832) (MohamedAbdeen21) +- Minor: Improve `ListingTable` documentation [#10854](https://github.com/apache/datafusion/pull/10854) (alamb) +- Extending join fuzz tests to support join filtering [#10728](https://github.com/apache/datafusion/pull/10728) (edmondop) +- replace and(_, not(_)) with and_not(\*) [#10885](https://github.com/apache/datafusion/pull/10885) (RTEnzyme) +- Disabling test for semi join with filters [#10887](https://github.com/apache/datafusion/pull/10887) (edmondop) +- Minor: Update `min_statistics` and `max_statistics` to be helpers, update docs [#10866](https://github.com/apache/datafusion/pull/10866) (alamb) +- Remove `Interval` column test // parquet extraction [#10888](https://github.com/apache/datafusion/pull/10888) (marvinlanhenke) +- Minor: SMJ fuzz tests fix for rowcounts [#10891](https://github.com/apache/datafusion/pull/10891) (comphead) +- Move `Count` to `functions-aggregate`, update MSRV to rust 1.75 [#10484](https://github.com/apache/datafusion/pull/10484) (jayzhan211) +- refactor: fetch statistics for a given ParquetMetaData [#10880](https://github.com/apache/datafusion/pull/10880) (NGA-TRAN) +- Move FileSinkExec::metrics to the correct place [#10901](https://github.com/apache/datafusion/pull/10901) (joroKr21) +- Refine ParquetAccessPlan comments and tests [#10896](https://github.com/apache/datafusion/pull/10896) (alamb) +- ci: fix clippy failures on main [#10903](https://github.com/apache/datafusion/pull/10903) (jonahgao) +- Minor: disable flaky fuzz test [#10904](https://github.com/apache/datafusion/pull/10904) (comphead) +- Remove builtin count [#10893](https://github.com/apache/datafusion/pull/10893) (jayzhan211) +- Move Regr\_\* functions to use UDAF [#10898](https://github.com/apache/datafusion/pull/10898) (eejbyfeldt) +- Docs: clarify when the parquet reader will read from object store when using cached metadata [#10909](https://github.com/apache/datafusion/pull/10909) (alamb) +- Minor: Fix `bench.sh tpch data` [#10905](https://github.com/apache/datafusion/pull/10905) (alamb) +- Minor: use venv in benchmark compare [#10894](https://github.com/apache/datafusion/pull/10894) (tmi) +- Support explicit type and name during table creation [#10273](https://github.com/apache/datafusion/pull/10273) (duongcongtoai) +- Simplify Join Partition Rules [#10911](https://github.com/apache/datafusion/pull/10911) (berkaysynnada) +- Move `Literal` to `physical-expr-common` [#10910](https://github.com/apache/datafusion/pull/10910) (lewiszlw) +- chore: update some error messages for clarity [#10916](https://github.com/apache/datafusion/pull/10916) (jeffreyssmith2nd) +- Initial Extract parquet data page statistics API [#10852](https://github.com/apache/datafusion/pull/10852) (marvinlanhenke) +- Add contains function, and support in datafusion substrait consumer [#10879](https://github.com/apache/datafusion/pull/10879) (Lordworms) +- Minor: Improve `arrow_statistics` tests [#10927](https://github.com/apache/datafusion/pull/10927) (alamb) +- Minor: Remove `prefer_hash_join` env variable for clickbench [#10933](https://github.com/apache/datafusion/pull/10933) (jayzhan211) +- Convert ApproxPercentileCont and ApproxPercentileContWithWeight to UDAF [#10917](https://github.com/apache/datafusion/pull/10917) (goldmedal) +- refactor: remove extra default in max rows [#10941](https://github.com/apache/datafusion/pull/10941) (tshauck) +- chore: Improve performance of Parquet statistics conversion [#10932](https://github.com/apache/datafusion/pull/10932) (Weijun-H) +- Add catalog::resolve_table_references [#10876](https://github.com/apache/datafusion/pull/10876) (leoyvens) +- Convert BitAnd, BitOr, BitXor to UDAF [#10930](https://github.com/apache/datafusion/pull/10930) (dharanad) +- refactor: improve PoolType argument handling for CLI [#10940](https://github.com/apache/datafusion/pull/10940) (tshauck) +- Minor: remove potential string copy from Column::from_qualified_name [#10947](https://github.com/apache/datafusion/pull/10947) (alamb) +- Fix: StatisticsConverter `counts` for missing columns [#10946](https://github.com/apache/datafusion/pull/10946) (marvinlanhenke) +- Add initial support for Utf8View and BinaryView types [#10925](https://github.com/apache/datafusion/pull/10925) (XiangpengHao) +- Use shorter aliases in CSE [#10939](https://github.com/apache/datafusion/pull/10939) (peter-toth) +- Substrait support for ParquetExec round trip for simple select [#10949](https://github.com/apache/datafusion/pull/10949) (xinlifoobar) +- Support to unparse `ScalarValue::IntervalMonthDayNano` to String [#10956](https://github.com/apache/datafusion/pull/10956) (goldmedal) +- Minor: Return option from row_group_row_count [#10973](https://github.com/apache/datafusion/pull/10973) (marvinlanhenke) +- Minor: Add routine to debug join fuzz tests [#10970](https://github.com/apache/datafusion/pull/10970) (comphead) +- Support to unparse `ScalarValue::TimestampNanosecond` to String [#10984](https://github.com/apache/datafusion/pull/10984) (goldmedal) +- build(deps-dev): bump ws from 8.14.2 to 8.17.1 in /datafusion/wasmtest/datafusion-wasm-app [#10988](https://github.com/apache/datafusion/pull/10988) (dependabot[bot]) +- Minor: reuse Rows buffer in GroupValuesRows [#10980](https://github.com/apache/datafusion/pull/10980) (alamb) +- Add example for writing SQL analysis using DataFusion structures [#10938](https://github.com/apache/datafusion/pull/10938) (LorrensP-2158466) +- Push down filter for Unnest plan [#10974](https://github.com/apache/datafusion/pull/10974) (jayzhan211) +- Add parquet page stats for float{16, 32, 64} [#10982](https://github.com/apache/datafusion/pull/10982) (tmi) +- Fix `file_stream_provider` example compilation failure on windows [#10975](https://github.com/apache/datafusion/pull/10975) (lewiszlw) +- Stop copying LogicalPlan and Exprs in `CommonSubexprEliminate` (2-3% planning speed improvement) [#10835](https://github.com/apache/datafusion/pull/10835) (alamb) +- chore: Update documentation link in `PhysicalOptimizerRule` comment [#11002](https://github.com/apache/datafusion/pull/11002) (Weijun-H) +- Push down filter plan for unnest on non-unnest column only [#10991](https://github.com/apache/datafusion/pull/10991) (jayzhan211) +- Minor: add test for pushdown past unnest [#11017](https://github.com/apache/datafusion/pull/11017) (alamb) +- Update docs for `protoc` minimum installed version [#11006](https://github.com/apache/datafusion/pull/11006) (jcsherin) +- propagate error instead of panicking on out of bounds in physical-expr/src/analysis.rs [#10992](https://github.com/apache/datafusion/pull/10992) (LorrensP-2158466) +- Add drop_columns to dataframe api [#11010](https://github.com/apache/datafusion/pull/11010) (Omega359) +- Push down filter plan for non-unnest column [#11019](https://github.com/apache/datafusion/pull/11019) (jayzhan211) +- Consider timezones with `UTC` and `+00:00` to be the same [#10960](https://github.com/apache/datafusion/pull/10960) (marvinlanhenke) +- Deprecate `OptimizerRule::try_optimize` [#11022](https://github.com/apache/datafusion/pull/11022) (lewiszlw) +- Relax combine partial final rule [#10913](https://github.com/apache/datafusion/pull/10913) (mustafasrepo) +- Compute gcd with u64 instead of i64 because of overflows [#11036](https://github.com/apache/datafusion/pull/11036) (LorrensP-2158466) +- Add distinct_on to dataframe api [#11012](https://github.com/apache/datafusion/pull/11012) (Omega359) +- chore: add test to show current behavior of `AT TIME ZONE` for string vs. timestamp [#11056](https://github.com/apache/datafusion/pull/11056) (appletreeisyellow) +- Boolean parquet get datapage stat [#11054](https://github.com/apache/datafusion/pull/11054) (LorrensP-2158466) +- Using display_name for Expr::Aggregation [#11020](https://github.com/apache/datafusion/pull/11020) (Lordworms) +- Minor: Convert `Count`'s name to lowercase [#11028](https://github.com/apache/datafusion/pull/11028) (jayzhan211) +- Minor: Move `function::Hint` to `datafusion-expr` crate to avoid physical-expr dependency for `datafusion-function` crate [#11061](https://github.com/apache/datafusion/pull/11061) (jayzhan211) +- Support to unparse ScalarValue::TimestampMillisecond to String [#11046](https://github.com/apache/datafusion/pull/11046) (pingsutw) +- Support to unparse IntervalYearMonth and IntervalDayTime to String [#11065](https://github.com/apache/datafusion/pull/11065) (goldmedal) +- SMJ: fix streaming row concurrency issue for LEFT SEMI filtered join [#11041](https://github.com/apache/datafusion/pull/11041) (comphead) +- Add `advanced_parquet_index.rs` example of index in into parquet files [#10701](https://github.com/apache/datafusion/pull/10701) (alamb) +- Add Expr::column_refs to find column references without copying [#10948](https://github.com/apache/datafusion/pull/10948) (alamb) +- Give `OptimizerRule::try_optimize` default implementation and cleanup duplicated custom implementations [#11059](https://github.com/apache/datafusion/pull/11059) (lewiszlw) +- Fix `FormatOptions::CSV` propagation [#10912](https://github.com/apache/datafusion/pull/10912) (svranesevic) +- Support parsing SQL strings to Exprs [#10995](https://github.com/apache/datafusion/pull/10995) (xinlifoobar) +- Support dictionary data type in array_to_string [#10908](https://github.com/apache/datafusion/pull/10908) (EduardoVega) +- Implement min/max for interval types [#11015](https://github.com/apache/datafusion/pull/11015) (maxburke) +- Improve LIKE performance for Dictionary arrays [#11058](https://github.com/apache/datafusion/pull/11058) (Lordworms) +- handle overflow in gcd and return this as an error [#11057](https://github.com/apache/datafusion/pull/11057) (LorrensP-2158466) +- Convert Correlation to UDAF [#11064](https://github.com/apache/datafusion/pull/11064) (pingsutw) +- Migrate more code from `Expr::to_columns` to `Expr::column_refs` [#11067](https://github.com/apache/datafusion/pull/11067) (alamb) +- decimal support for unparser [#11092](https://github.com/apache/datafusion/pull/11092) (y-f-u) +- Improve `CommonSubexprEliminate` identifier management (10% faster planning) [#10473](https://github.com/apache/datafusion/pull/10473) (peter-toth) +- Change wildcard qualifier type from `String` to `TableReference` [#11073](https://github.com/apache/datafusion/pull/11073) (linhr) +- Allow access to UDTF in `SessionContext` [#11071](https://github.com/apache/datafusion/pull/11071) (linhr) +- Strip table qualifiers from schema in `UNION ALL` for unparser [#11082](https://github.com/apache/datafusion/pull/11082) (phillipleblanc) +- Update ListingTable to use StatisticsConverter [#11068](https://github.com/apache/datafusion/pull/11068) (xinlifoobar) +- to_timestamp functions should preserve timezone [#11038](https://github.com/apache/datafusion/pull/11038) (maxburke) +- Rewrite array operator to function in parser [#11101](https://github.com/apache/datafusion/pull/11101) (jayzhan211) +- Resolve empty relation opt for join types [#11066](https://github.com/apache/datafusion/pull/11066) (LorrensP-2158466) +- Add composed extension codec example [#11095](https://github.com/apache/datafusion/pull/11095) (lewiszlw) +- Minor: Avoid some repetition in to_timestamp [#11116](https://github.com/apache/datafusion/pull/11116) (alamb) +- Minor: fix ScalarValue::new_ten error message (cites one not ten) [#11126](https://github.com/apache/datafusion/pull/11126) (gstvg) +- Deprecate Expr::column_refs [#11115](https://github.com/apache/datafusion/pull/11115) (alamb) +- Overflow in negate operator [#11084](https://github.com/apache/datafusion/pull/11084) (LorrensP-2158466) +- Minor: Add Architectural Goals to the docs [#11109](https://github.com/apache/datafusion/pull/11109) (alamb) +- Fix overflow in pow [#11124](https://github.com/apache/datafusion/pull/11124) (LorrensP-2158466) +- Support to unparse Time scalar value to String [#11121](https://github.com/apache/datafusion/pull/11121) (goldmedal) +- Support to unparse `TimestampSecond` and `TimestampMicrosecond` to String [#11120](https://github.com/apache/datafusion/pull/11120) (goldmedal) +- Add standalone example for `OptimizerRule` [#11087](https://github.com/apache/datafusion/pull/11087) (alamb) +- Fix overflow in factorial [#11134](https://github.com/apache/datafusion/pull/11134) (LorrensP-2158466) +- Temporary Fix: Query error when grouping by case expressions [#11133](https://github.com/apache/datafusion/pull/11133) (jonahgao) +- Fix nullability of return value of array_agg [#11093](https://github.com/apache/datafusion/pull/11093) (eejbyfeldt) +- Support filter for List [#11091](https://github.com/apache/datafusion/pull/11091) (jayzhan211) +- [MINOR]: Fix some minor silent bugs [#11127](https://github.com/apache/datafusion/pull/11127) (mustafasrepo) +- Minor Fix for Logical and Physical Expr Conversions [#11142](https://github.com/apache/datafusion/pull/11142) (berkaysynnada) +- Support Date Parquet Data Page Statistics [#11135](https://github.com/apache/datafusion/pull/11135) (dharanad) +- fix flaky array query slt test [#11140](https://github.com/apache/datafusion/pull/11140) (leoyvens) +- Support Decimal and Decimal256 Parquet Data Page Statistics [#11138](https://github.com/apache/datafusion/pull/11138) (Lordworms) +- Implement comparisons on nested data types such that distinct/except would work [#11117](https://github.com/apache/datafusion/pull/11117) (rtyler) +- Minor: dont panic with bad arguments to round [#10899](https://github.com/apache/datafusion/pull/10899) (tmi) +- Minor: reduce replication for nested comparison [#11149](https://github.com/apache/datafusion/pull/11149) (alamb) +- [Minor]: Remove datafusion-functions-aggregate dependency from physical-expr crate [#11158](https://github.com/apache/datafusion/pull/11158) (mustafasrepo) +- adding config to control Varchar behavior [#11090](https://github.com/apache/datafusion/pull/11090) (Lordworms) +- minor: consolidate `gcd` related tests [#11164](https://github.com/apache/datafusion/pull/11164) (jonahgao) +- Minor: move batch spilling methods to `lib.rs` to make it reusable [#11154](https://github.com/apache/datafusion/pull/11154) (comphead) +- Move schema projection to where it's used in ListingTable [#11167](https://github.com/apache/datafusion/pull/11167) (adriangb) +- Make running in docker instruction be copy-pastable [#11148](https://github.com/apache/datafusion/pull/11148) (findepi) +- Rewrite `array @> array` and `array <@ array` in sql_expr_to_logical_expr [#11155](https://github.com/apache/datafusion/pull/11155) (jayzhan211) +- Minor: make some physical_optimizer rules public [#11171](https://github.com/apache/datafusion/pull/11171) (askalt) +- Remove pr_benchmarks.yml [#11165](https://github.com/apache/datafusion/pull/11165) (alamb) +- Optionally display schema in explain plan [#11177](https://github.com/apache/datafusion/pull/11177) (alamb) +- Minor: Add more support for ScalarValue::Float16 [#11156](https://github.com/apache/datafusion/pull/11156) (Lordworms) +- Minor: fix SQLOptions::with_allow_ddl comments [#11166](https://github.com/apache/datafusion/pull/11166) (alamb) +- Update sqllogictest requirement from 0.20.0 to 0.21.0 [#11189](https://github.com/apache/datafusion/pull/11189) (dependabot[bot]) +- Support Time Parquet Data Page Statistics [#11187](https://github.com/apache/datafusion/pull/11187) (dharanad) +- Adds support for Dictionary data type statistics from parquet data pages. [#11195](https://github.com/apache/datafusion/pull/11195) (efredine) +- [Minor]: Make sort_batch public [#11191](https://github.com/apache/datafusion/pull/11191) (mustafasrepo) +- Introduce user defined SQL planner API [#11180](https://github.com/apache/datafusion/pull/11180) (jayzhan211) +- Covert grouping to udaf [#11147](https://github.com/apache/datafusion/pull/11147) (Rachelint) +- Make statistics_from_parquet_meta a sync function [#11205](https://github.com/apache/datafusion/pull/11205) (adriangb) +- Allow user defined SQL planners to be registered [#11208](https://github.com/apache/datafusion/pull/11208) (samuelcolvin) +- Recursive `unnest` [#11062](https://github.com/apache/datafusion/pull/11062) (duongcongtoai) +- Document how to test examples in user guide, add some more coverage [#11178](https://github.com/apache/datafusion/pull/11178) (alamb) +- Minor: Move MemoryCatalog\*Provider into a module, improve comments [#11183](https://github.com/apache/datafusion/pull/11183) (alamb) +- Add standalone example of using the SQL frontend [#11088](https://github.com/apache/datafusion/pull/11088) (alamb) +- Add Optimizer Sanity Checker, improve sortedness equivalence properties [#11196](https://github.com/apache/datafusion/pull/11196) (mustafasrepo) +- Implement user defined planner for extract [#11215](https://github.com/apache/datafusion/pull/11215) (xinlifoobar) +- Move basic SQL query examples to user guide [#11217](https://github.com/apache/datafusion/pull/11217) (alamb) +- Support FixedSizedBinaryArray Parquet Data Page Statistics [#11200](https://github.com/apache/datafusion/pull/11200) (dharanad) +- Implement ScalarValue::Map [#11224](https://github.com/apache/datafusion/pull/11224) (goldmedal) +- Remove unmaintained python pre-commit configuration [#11255](https://github.com/apache/datafusion/pull/11255) (findepi) +- Enable `clone_on_ref_ptr` clippy lint on execution crate [#11239](https://github.com/apache/datafusion/pull/11239) (lewiszlw) +- Minor: Improve documentation about pushdown join predicates [#11209](https://github.com/apache/datafusion/pull/11209) (alamb) +- Minor: clean up data page statistics tests and fix bugs [#11236](https://github.com/apache/datafusion/pull/11236) (efredine) +- Replacing pattern matching through downcast with trait method [#11257](https://github.com/apache/datafusion/pull/11257) (edmondop) +- Update substrait requirement from 0.34.0 to 0.35.0 [#11206](https://github.com/apache/datafusion/pull/11206) (dependabot[bot]) +- Enhance short circuit handling in `CommonSubexprEliminate` [#11197](https://github.com/apache/datafusion/pull/11197) (peter-toth) +- Add bench for data page statistics parquet extraction [#10950](https://github.com/apache/datafusion/pull/10950) (marvinlanhenke) +- Register SQL planners in `SessionState` constructor [#11253](https://github.com/apache/datafusion/pull/11253) (dharanad) +- Support DuckDB style struct syntax [#11214](https://github.com/apache/datafusion/pull/11214) (jayzhan211) +- Enable `clone_on_ref_ptr` clippy lint on expr crate [#11238](https://github.com/apache/datafusion/pull/11238) (lewiszlw) +- Optimize PushDownFilter to avoid recreating schema columns [#11211](https://github.com/apache/datafusion/pull/11211) (alamb) +- Remove outdated `rewrite_expr.rs` example [#11085](https://github.com/apache/datafusion/pull/11085) (alamb) +- Implement TPCH substrait integration teset, support tpch_2 [#11234](https://github.com/apache/datafusion/pull/11234) (Lordworms) +- Enable `clone_on_ref_ptr` clippy lint on physical-expr crate [#11240](https://github.com/apache/datafusion/pull/11240) (lewiszlw) +- Add standalone `AnalyzerRule` example that implements row level access control [#11089](https://github.com/apache/datafusion/pull/11089) (alamb) +- Replace println! with assert! if possible in DataFusion examples [#11237](https://github.com/apache/datafusion/pull/11237) (Nishi46) +- minor: format `Expr::get_type()` [#11267](https://github.com/apache/datafusion/pull/11267) (jonahgao) +- Fix hash join for nested types [#11232](https://github.com/apache/datafusion/pull/11232) (eejbyfeldt) +- Infer count() aggregation is not null [#11256](https://github.com/apache/datafusion/pull/11256) (findepi) +- Remove unnecessary qualified names [#11292](https://github.com/apache/datafusion/pull/11292) (findepi) +- Fix running examples readme [#11225](https://github.com/apache/datafusion/pull/11225) (findepi) +- Minor: Add `ConstExpr::from` and use in physical optimizer [#11283](https://github.com/apache/datafusion/pull/11283) (alamb) +- Implement TPCH substrait integration teset, support tpch_3 [#11298](https://github.com/apache/datafusion/pull/11298) (Lordworms) +- Implement user defined planner for position [#11243](https://github.com/apache/datafusion/pull/11243) (xinlifoobar) +- Upgrade to arrow 52.1.0 (and fix clippy issues on main) [#11302](https://github.com/apache/datafusion/pull/11302) (alamb) +- AggregateExec: Take grouping sets into account for InputOrderMode [#11301](https://github.com/apache/datafusion/pull/11301) (thinkharderdev) +- Add user_defined_sql_planners(..) to FunctionRegistry [#11296](https://github.com/apache/datafusion/pull/11296) (Omega359) +- use safe cast in propagate_constraints [#11297](https://github.com/apache/datafusion/pull/11297) (Lordworms) +- Minor: Remove clone in optimizer [#11315](https://github.com/apache/datafusion/pull/11315) (jayzhan211) +- minor: Add `PhysicalSortExpr::new` [#11310](https://github.com/apache/datafusion/pull/11310) (andygrove) +- Fix data page statistics when all rows are null in a data page [#11295](https://github.com/apache/datafusion/pull/11295) (efredine) +- Made UserDefinedFunctionPlanner to uniform the usages [#11318](https://github.com/apache/datafusion/pull/11318) (xinlifoobar) +- Implement user defined planner for `create_struct` & `create_named_struct` [#11273](https://github.com/apache/datafusion/pull/11273) (dharanad) +- Improve stats convert performance for Binary/String/Boolean arrays [#11319](https://github.com/apache/datafusion/pull/11319) (Rachelint) +- Fix typos in datafusion-examples/datafusion-cli/docs [#11259](https://github.com/apache/datafusion/pull/11259) (lewiszlw) +- Minor: Fix Failing TPC-DS Test [#11331](https://github.com/apache/datafusion/pull/11331) (berkaysynnada) +- HashJoin can preserve the right ordering when join type is Right [#11276](https://github.com/apache/datafusion/pull/11276) (berkaysynnada) +- Update substrait requirement from 0.35.0 to 0.36.0 [#11328](https://github.com/apache/datafusion/pull/11328) (dependabot[bot]) +- Support to uparse logical plans with timestamp cast to string [#11326](https://github.com/apache/datafusion/pull/11326) (sgrebnov) +- Implement user defined planner for sql_substring_to_expr [#11327](https://github.com/apache/datafusion/pull/11327) (xinlifoobar) +- Improve volatile expression handling in `CommonSubexprEliminate` [#11265](https://github.com/apache/datafusion/pull/11265) (peter-toth) +- Support `IS NULL` and `IS NOT NULL` on Unions [#11321](https://github.com/apache/datafusion/pull/11321) (samuelcolvin) +- Implement TPCH substrait integration test, support tpch_4 and tpch_5 [#11311](https://github.com/apache/datafusion/pull/11311) (Lordworms) +- Enable `clone_on_ref_ptr` clippy lint on physical-plan crate [#11241](https://github.com/apache/datafusion/pull/11241) (lewiszlw) +- Remove any aliases in `Filter::try_new` rather than erroring [#11307](https://github.com/apache/datafusion/pull/11307) (samuelcolvin) +- Improve `DataFrame` Users Guide [#11324](https://github.com/apache/datafusion/pull/11324) (alamb) +- chore: Rename UserDefinedSQLPlanner to ExprPlanner [#11338](https://github.com/apache/datafusion/pull/11338) (andygrove) +- Revert "remove `derive(Copy)` from `Operator` (#11132)" [#11341](https://github.com/apache/datafusion/pull/11341) (alamb) + +## Credits + +Thank you to everyone who contributed to this release. Here is a breakdown of commits (PRs merged) per contributor. + +``` + 41 Andrew Lamb + 17 Jay Zhan + 12 Lordworms + 12 张林伟 + 10 Arttu + 9 Jax Liu + 9 Lorrens Pantelis + 8 Piotr Findeisen + 7 Dharan Aditya + 7 Jonah Gao + 7 Xin Li + 6 Andy Grove + 6 Marvin Lanhenke + 6 Trent Hauck + 5 Alex Huang + 5 Eric Fredine + 5 Mustafa Akur + 5 Oleks V + 5 dependabot[bot] + 4 Adrian Garcia Badaracco + 4 Berkay Şahin + 4 Kevin Su + 4 Peter Toth + 4 Ruihang Xia + 4 Samuel Colvin + 3 Bruce Ritchie + 3 Edmondo Porcu + 3 Emil Ejbyfeldt + 3 Heran Lin + 3 Leonardo Yvens + 3 jcsherin + 3 tmi + 2 Duong Cong Toai + 2 Liang-Chi Hsieh + 2 Max Burke + 2 kamille + 1 Albert Skalt + 1 Andrey Koshchiy + 1 Benjamin Bannier + 1 Bo Lin + 1 Chojan Shang + 1 Chunchun Ye + 1 Dan Harris + 1 Devin D'Angelo + 1 Eduardo Vega + 1 Georgi Krastev + 1 Hector Veiga + 1 Jeffrey Smith II + 1 Kirill Khramkov + 1 Matt Nawara + 1 Mohamed Abdeen + 1 Nga Tran + 1 Nishi + 1 Phillip LeBlanc + 1 R. Tyler Croy + 1 RT_Enzyme + 1 Sava Vranešević + 1 Sergei Grebnov + 1 Weston Pace + 1 Xiangpeng Hao + 1 advancedxy + 1 c8ef + 1 gstvg + 1 yfu +``` + +Thank you also to everyone who contributed in other ways such as filing issues, reviewing PRs, and providing feedback on this release. diff --git a/docs/source/contributor-guide/inviting.md b/docs/source/contributor-guide/inviting.md index be189b95f684..967f417e6e9a 100644 --- a/docs/source/contributor-guide/inviting.md +++ b/docs/source/contributor-guide/inviting.md @@ -294,7 +294,7 @@ Subject: [DISCUSS] $NEW_PMC_MEMBER for PMC I would like to propose adding $NEW_PMC_MEMBER[1] to the DataFusion PMC. -$NEW_PMC_MEMBMER has been a committer since $COMMITER_MONTH [2], has a +$NEW_PMC_MEMBMER has been a committer since $COMMITTER_MONTH [2], has a strong and sustained contribution record for more than a year, and focused on helping the community and the project grow[3]. diff --git a/docs/source/library-user-guide/adding-udfs.md b/docs/source/library-user-guide/adding-udfs.md index f805f0a99292..fe3990b90c3c 100644 --- a/docs/source/library-user-guide/adding-udfs.md +++ b/docs/source/library-user-guide/adding-udfs.md @@ -268,7 +268,7 @@ impl PartitionEvaluator for MyPartitionEvaluator { } } -/// Create a `PartitionEvalutor` to evaluate this function on a new +/// Create a `PartitionEvaluator` to evaluate this function on a new /// partition. fn make_partition_evaluator() -> Result> { Ok(Box::new(MyPartitionEvaluator::new())) @@ -474,7 +474,7 @@ impl Accumulator for GeometricMean { ### registering an Aggregate UDF -To register a Aggreate 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 diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 303caef57700..579088f991ef 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -64,7 +64,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.parquet.statistics_enabled | NULL | Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_statistics_size | NULL | Sets max statistics size for any column. If NULL, uses default parquet writer setting | | datafusion.execution.parquet.max_row_group_size | 1048576 | Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 39.0.0 | Sets "created by" property | +| datafusion.execution.parquet.created_by | datafusion version 40.0.0 | Sets "created by" property | | datafusion.execution.parquet.column_index_truncate_length | NULL | Sets column index truncate length | | datafusion.execution.parquet.data_page_row_count_limit | 18446744073709551615 | Sets best effort maximum number of rows in data page | | datafusion.execution.parquet.encoding | NULL | Sets default encoding for any column Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | diff --git a/docs/source/user-guide/dataframe.md b/docs/source/user-guide/dataframe.md index f011e68fadb2..c3d0b6c2d688 100644 --- a/docs/source/user-guide/dataframe.md +++ b/docs/source/user-guide/dataframe.md @@ -19,17 +19,30 @@ # DataFrame API -A DataFrame represents a logical set of rows with the same named columns, similar to a [Pandas DataFrame](https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html) or -[Spark DataFrame](https://spark.apache.org/docs/latest/sql-programming-guide.html). +A DataFrame represents a logical set of rows with the same named columns, +similar to a [Pandas DataFrame] or [Spark DataFrame]. -DataFrames are typically created by calling a method on -`SessionContext`, such as `read_csv`, and can then be modified -by calling the transformation methods, such as `filter`, `select`, `aggregate`, and `limit` -to build up a query definition. +DataFrames are typically created by calling a method on [`SessionContext`], such +as [`read_csv`], and can then be modified by calling the transformation methods, +such as [`filter`], [`select`], [`aggregate`], and [`limit`] to build up a query +definition. -The query can be executed by calling the `collect` method. +The query can be executed by calling the [`collect`] method. -The DataFrame struct is part of DataFusion's prelude and can be imported with the following statement. +DataFusion DataFrames use lazy evaluation, meaning that each transformation +creates a new plan but does not actually perform any immediate actions. This +approach allows for the overall plan to be optimized before execution. The plan +is evaluated (executed) when an action method is invoked, such as [`collect`]. +See the [Library Users Guide] for more details. + +The DataFrame API is well documented in the [API reference on docs.rs]. +Please refer to the [Expressions Reference] for more information on +building logical expressions (`Expr`) to use with the DataFrame API. + +## Example + +The DataFrame struct is part of DataFusion's `prelude` and can be imported with +the following statement. ```rust use datafusion::prelude::*; @@ -38,73 +51,31 @@ use datafusion::prelude::*; Here is a minimal example showing the execution of a query using the DataFrame API. ```rust -let ctx = SessionContext::new(); -let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; -let df = df.filter(col("a").lt_eq(col("b")))? - .aggregate(vec![col("a")], vec![min(col("b"))])? - .limit(0, Some(100))?; -// Print results -df.show().await?; +use datafusion::prelude::*; +use datafusion::error::Result; + +#[tokio::main] +async fn main() -> Result<()> { + let ctx = SessionContext::new(); + let df = ctx.read_csv("tests/data/example.csv", CsvReadOptions::new()).await?; + let df = df.filter(col("a").lt_eq(col("b")))? + .aggregate(vec![col("a")], vec![min(col("b"))])? + .limit(0, Some(100))?; + // Print results + df.show().await?; + Ok(()) +} ``` -The DataFrame API is well documented in the [API reference on docs.rs](https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html). - -Refer to the [Expressions Reference](expressions) for available functions for building logical expressions for use with the -DataFrame API. - -## DataFrame Transformations - -These methods create a new DataFrame after applying a transformation to the logical plan that the DataFrame represents. - -DataFusion DataFrames use lazy evaluation, meaning that each transformation is just creating a new query plan and -not actually performing any transformations. This approach allows for the overall plan to be optimized before -execution. The plan is evaluated (executed) when an action method is invoked, such as `collect`. - -| Function | Notes | -| ------------------- | ------------------------------------------------------------------------------------------------------------------------------------------ | -| aggregate | Perform an aggregate query with optional grouping expressions. | -| distinct | Filter out duplicate rows. | -| distinct_on | Filter out duplicate rows based on provided expressions. | -| drop_columns | Create a projection with all but the provided column names. | -| except | Calculate the exception of two DataFrames. The two DataFrames must have exactly the same schema | -| filter | Filter a DataFrame to only include rows that match the specified filter expression. | -| intersect | Calculate the intersection of two DataFrames. The two DataFrames must have exactly the same schema | -| join | Join this DataFrame with another DataFrame using the specified columns as join keys. | -| join_on | Join this DataFrame with another DataFrame using arbitrary expressions. | -| limit | Limit the number of rows returned from this DataFrame. | -| repartition | Repartition a DataFrame based on a logical partitioning scheme. | -| sort | Sort the DataFrame by the specified sorting expressions. Any expression can be turned into a sort expression by calling its `sort` method. | -| select | Create a projection based on arbitrary expressions. Example: `df.select(vec![col("c1"), abs(col("c2"))])?` | -| select_columns | Create a projection based on column names. Example: `df.select_columns(&["id", "name"])?`. | -| union | Calculate the union of two DataFrames, preserving duplicate rows. The two DataFrames must have exactly the same schema. | -| union_distinct | Calculate the distinct union of two DataFrames. The two DataFrames must have exactly the same schema. | -| with_column | Add an additional column to the DataFrame. | -| with_column_renamed | Rename one column by applying a new projection. | - -## DataFrame Actions - -These methods execute the logical plan represented by the DataFrame and either collects the results into memory, prints them to stdout, or writes them to disk. - -| Function | Notes | -| -------------------------- | --------------------------------------------------------------------------------------------------------------------------- | -| collect | Executes this DataFrame and collects all results into a vector of RecordBatch. | -| collect_partitioned | Executes this DataFrame and collects all results into a vector of vector of RecordBatch maintaining the input partitioning. | -| count | Executes this DataFrame to get the total number of rows. | -| execute_stream | Executes this DataFrame and returns a stream over a single partition. | -| execute_stream_partitioned | Executes this DataFrame and returns one stream per partition. | -| show | Execute this DataFrame and print the results to stdout. | -| show_limit | Execute this DataFrame and print a subset of results to stdout. | -| write_csv | Execute this DataFrame and write the results to disk in CSV format. | -| write_json | Execute this DataFrame and write the results to disk in JSON format. | -| write_parquet | Execute this DataFrame and write the results to disk in Parquet format. | -| write_table | Execute this DataFrame and write the results via the insert_into method of the registered TableProvider | - -## Other DataFrame Methods - -| Function | Notes | -| ------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------ | -| explain | Return a DataFrame with the explanation of its plan so far. | -| registry | Return a `FunctionRegistry` used to plan udf's calls. | -| schema | Returns the schema describing the output of this DataFrame in terms of columns returned, where each column has a name, data type, and nullability attribute. | -| to_logical_plan | Return the optimized logical plan represented by this DataFrame. | -| to_unoptimized_plan | Return the unoptimized logical plan represented by this DataFrame. | +[pandas dataframe]: https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.html +[spark dataframe]: https://spark.apache.org/docs/latest/sql-programming-guide.html +[`sessioncontext`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionContext.html +[`read_csv`]: https://docs.rs/datafusion/latest/datafusion/execution/context/struct.SessionContext.html#method.read_csv +[`filter`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.filter +[`select`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.select +[`aggregate`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.aggregate +[`limit`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.limit +[`collect`]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html#method.collect +[library users guide]: ../library-user-guide/using-the-dataframe-api.md +[api reference on docs.rs]: https://docs.rs/datafusion/latest/datafusion/dataframe/struct.DataFrame.html +[expressions reference]: expressions diff --git a/docs/source/user-guide/example-usage.md b/docs/source/user-guide/example-usage.md index 62f09a6a9937..7dbd4045e75b 100644 --- a/docs/source/user-guide/example-usage.md +++ b/docs/source/user-guide/example-usage.md @@ -264,8 +264,8 @@ Set environment [variables](https://doc.rust-lang.org/std/backtrace/index.html#e ```bash RUST_BACKTRACE=1 ./target/debug/datafusion-cli DataFusion CLI v31.0.0 -> select row_numer() over (partition by a order by a) from (select 1 a); -Error during planning: Invalid function 'row_numer'. +> select row_number() over (partition by a order by a) from (select 1 a); +Error during planning: Invalid function 'row_number'. Did you mean 'ROW_NUMBER'? backtrace: 0: std::backtrace_rs::backtrace::libunwind::trace @@ -290,7 +290,7 @@ async fn test_get_backtrace_for_failed_code() -> Result<()> { let ctx = SessionContext::new(); let sql = " - select row_numer() over (partition by a order by a) from (select 1 a); + select row_number() over (partition by a order by a) from (select 1 a); "; let _ = ctx.sql(sql).await?.collect().await?;