From 592b92440c93deab341eb65c0f31a2b0f100c2df Mon Sep 17 00:00:00 2001 From: Victor Barua Date: Fri, 1 Nov 2024 10:23:29 -0700 Subject: [PATCH 01/28] feat(substrait): handle emit_kind when consuming Substrait plans (#13127) * feat(substrait): handle emit_kind when consuming Substrait plans * cargo fmt * avoid projection flattening for volatile expressions * simplify application of apply_emit_kind --- .../substrait/src/logical_plan/consumer.rs | 200 +++++++++++++++--- .../substrait/tests/cases/emit_kind_tests.rs | 127 +++++++++++ datafusion/substrait/tests/cases/mod.rs | 1 + .../direct_on_project.substrait.json | 90 ++++++++ .../emit_kind/emit_on_filter.substrait.json | 91 ++++++++ 5 files changed, 483 insertions(+), 26 deletions(-) create mode 100644 datafusion/substrait/tests/cases/emit_kind_tests.rs create mode 100644 datafusion/substrait/tests/testdata/test_plans/emit_kind/direct_on_project.substrait.json create mode 100644 datafusion/substrait/tests/testdata/test_plans/emit_kind/emit_on_filter.substrait.json diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 289aa7b7f448..a12406bd3439 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -56,6 +56,7 @@ use datafusion::arrow::array::{new_empty_array, AsArray}; use datafusion::arrow::temporal_conversions::NANOSECONDS; use datafusion::common::scalar::ScalarStructBuilder; use datafusion::dataframe::DataFrame; +use datafusion::logical_expr::builder::project; use datafusion::logical_expr::expr::InList; use datafusion::logical_expr::{ col, expr, Cast, Extension, GroupingSet, Like, LogicalPlanBuilder, Partitioning, @@ -80,6 +81,7 @@ use substrait::proto::expression::literal::{ use substrait::proto::expression::subquery::SubqueryType; use substrait::proto::expression::{FieldReference, Literal, ScalarFunction}; use substrait::proto::read_rel::local_files::file_or_files::PathType::UriFile; +use substrait::proto::rel_common::{Emit, EmitKind}; use substrait::proto::{ aggregate_function::AggregationInvocation, expression::{ @@ -93,9 +95,9 @@ use substrait::proto::{ join_rel, plan_rel, r#type, read_rel::ReadType, rel::RelType, - set_rel, + rel_common, set_rel, sort_field::{SortDirection, SortKind::*}, - AggregateFunction, Expression, NamedStruct, Plan, Rel, Type, + AggregateFunction, Expression, NamedStruct, Plan, Rel, RelCommon, Type, }; use substrait::proto::{ExtendedExpression, FunctionArgument, SortField}; @@ -562,42 +564,51 @@ pub async fn from_substrait_rel( rel: &Rel, extensions: &Extensions, ) -> Result { - match &rel.rel_type { + let plan: Result = match &rel.rel_type { Some(RelType::Project(p)) => { if let Some(input) = p.input.as_ref() { 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) + let original_schema = input.schema().clone(); + + // Ensure that all expressions have a unique display name, so that + // validate_unique_names does not fail when constructing the project. + let mut name_tracker = NameTracker::new(); + + // By default, a Substrait Project emits all inputs fields followed by all expressions. + // We build the explicit expressions first, and then the input expressions to avoid + // adding aliases to the explicit expressions (as part of ensuring unique names). + // + // This is helpful for plan visualization and tests, because when DataFusion produces + // Substrait Projects it adds an output mapping that excludes all input columns + // leaving only explicit expressions. + + let mut explicit_exprs: Vec = vec![]; + for expr in &p.expressions { + let e = + from_substrait_rex(ctx, expr, input.clone().schema(), extensions) .await?; // if the expression is WindowFunction, wrap in a Window relation - if let Expr::WindowFunction(_) = &x { + if let Expr::WindowFunction(_) = &e { // 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.clone()])? + input = input.window(vec![e.clone()])? } - // Ensure the expression has a unique display name, so that project's - // validate_unique_names doesn't fail - let name = x.schema_name().to_string(); - let mut new_name = name.clone(); - let mut i = 0; - while names.contains(&new_name) { - new_name = format!("{}__temp__{}", name, i); - i += 1; - } - if new_name != name { - exprs.push(x.alias(new_name.clone())); - } else { - exprs.push(x); - } - names.insert(new_name); + explicit_exprs.push(name_tracker.get_uniquely_named_expr(e)?); } - input.project(exprs)?.build() + + let mut final_exprs: Vec = vec![]; + for index in 0..original_schema.fields().len() { + let e = Expr::Column(Column::from( + original_schema.qualified_field(index), + )); + final_exprs.push(name_tracker.get_uniquely_named_expr(e)?); + } + final_exprs.append(&mut explicit_exprs); + + input.project(final_exprs)?.build() } else { not_impl_err!("Projection without an input is not supported") } @@ -1074,6 +1085,143 @@ pub async fn from_substrait_rel( })) } _ => not_impl_err!("Unsupported RelType: {:?}", rel.rel_type), + }; + apply_emit_kind(retrieve_rel_common(rel), plan?) +} + +fn retrieve_rel_common(rel: &Rel) -> Option<&RelCommon> { + match rel.rel_type.as_ref() { + None => None, + Some(rt) => match rt { + RelType::Read(r) => r.common.as_ref(), + RelType::Filter(f) => f.common.as_ref(), + RelType::Fetch(f) => f.common.as_ref(), + RelType::Aggregate(a) => a.common.as_ref(), + RelType::Sort(s) => s.common.as_ref(), + RelType::Join(j) => j.common.as_ref(), + RelType::Project(p) => p.common.as_ref(), + RelType::Set(s) => s.common.as_ref(), + RelType::ExtensionSingle(e) => e.common.as_ref(), + RelType::ExtensionMulti(e) => e.common.as_ref(), + RelType::ExtensionLeaf(e) => e.common.as_ref(), + RelType::Cross(c) => c.common.as_ref(), + RelType::Reference(_) => None, + RelType::Write(w) => w.common.as_ref(), + RelType::Ddl(d) => d.common.as_ref(), + RelType::HashJoin(j) => j.common.as_ref(), + RelType::MergeJoin(j) => j.common.as_ref(), + RelType::NestedLoopJoin(j) => j.common.as_ref(), + RelType::Window(w) => w.common.as_ref(), + RelType::Exchange(e) => e.common.as_ref(), + RelType::Expand(e) => e.common.as_ref(), + }, + } +} + +fn retrieve_emit_kind(rel_common: Option<&RelCommon>) -> EmitKind { + // the default EmitKind is Direct if it is not set explicitly + let default = EmitKind::Direct(rel_common::Direct {}); + rel_common + .and_then(|rc| rc.emit_kind.as_ref()) + .map_or(default, |ek| ek.clone()) +} + +fn contains_volatile_expr(proj: &Projection) -> Result { + for expr in proj.expr.iter() { + if expr.is_volatile()? { + return Ok(true); + } + } + Ok(false) +} + +fn apply_emit_kind( + rel_common: Option<&RelCommon>, + plan: LogicalPlan, +) -> Result { + match retrieve_emit_kind(rel_common) { + EmitKind::Direct(_) => Ok(plan), + EmitKind::Emit(Emit { output_mapping }) => { + // It is valid to reference the same field multiple times in the Emit + // In this case, we need to provide unique names to avoid collisions + let mut name_tracker = NameTracker::new(); + match plan { + // To avoid adding a projection on top of a projection, we apply special case + // handling to flatten Substrait Emits. This is only applicable if none of the + // expressions in the projection are volatile. This is to avoid issues like + // converting a single call of the random() function into multiple calls due to + // duplicate fields in the output_mapping. + LogicalPlan::Projection(proj) if !contains_volatile_expr(&proj)? => { + let mut exprs: Vec = vec![]; + for field in output_mapping { + let expr = proj.expr + .get(field as usize) + .ok_or_else(|| substrait_datafusion_err!( + "Emit output field {} cannot be resolved in input schema {}", + field, proj.input.schema().clone() + ))?; + exprs.push(name_tracker.get_uniquely_named_expr(expr.clone())?); + } + + let input = Arc::unwrap_or_clone(proj.input); + project(input, exprs) + } + // Otherwise we just handle the output_mapping as a projection + _ => { + let input_schema = plan.schema(); + + let mut exprs: Vec = vec![]; + for index in output_mapping.into_iter() { + let column = Expr::Column(Column::from( + input_schema.qualified_field(index as usize), + )); + let expr = name_tracker.get_uniquely_named_expr(column)?; + exprs.push(expr); + } + + project(plan, exprs) + } + } + } + } +} + +struct NameTracker { + seen_names: HashSet, +} + +enum NameTrackerStatus { + NeverSeen, + SeenBefore, +} + +impl NameTracker { + fn new() -> Self { + NameTracker { + seen_names: HashSet::default(), + } + } + fn get_unique_name(&mut self, name: String) -> (String, NameTrackerStatus) { + match self.seen_names.insert(name.clone()) { + true => (name, NameTrackerStatus::NeverSeen), + false => { + let mut counter = 0; + loop { + let candidate_name = format!("{}__temp__{}", name, counter); + if self.seen_names.insert(candidate_name.clone()) { + return (candidate_name, NameTrackerStatus::SeenBefore); + } + counter += 1; + } + } + } + } + + fn get_uniquely_named_expr(&mut self, expr: Expr) -> Result { + match self.get_unique_name(expr.name_for_alias()?) { + (_, NameTrackerStatus::NeverSeen) => Ok(expr), + (name, NameTrackerStatus::SeenBefore) => Ok(expr.alias(name)), + } } } diff --git a/datafusion/substrait/tests/cases/emit_kind_tests.rs b/datafusion/substrait/tests/cases/emit_kind_tests.rs new file mode 100644 index 000000000000..ac66177ed796 --- /dev/null +++ b/datafusion/substrait/tests/cases/emit_kind_tests.rs @@ -0,0 +1,127 @@ +// 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. + +//! Tests for Emit Kind usage + +#[cfg(test)] +mod tests { + use crate::utils::test::{add_plan_schemas_to_ctx, read_json}; + + use datafusion::common::Result; + use datafusion::execution::SessionStateBuilder; + use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext}; + use datafusion_substrait::logical_plan::consumer::from_substrait_plan; + use datafusion_substrait::logical_plan::producer::to_substrait_plan; + + #[tokio::test] + async fn project_respects_direct_emit_kind() -> Result<()> { + let proto_plan = read_json( + "tests/testdata/test_plans/emit_kind/direct_on_project.substrait.json", + ); + let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan)?; + let plan = from_substrait_plan(&ctx, &proto_plan).await?; + + let plan_str = format!("{}", plan); + + assert_eq!( + plan_str, + "Projection: DATA.A AS a, DATA.B AS b, DATA.A + Int64(1) AS add1\ + \n TableScan: DATA" + ); + Ok(()) + } + + #[tokio::test] + async fn handle_emit_as_project() -> Result<()> { + let proto_plan = read_json( + "tests/testdata/test_plans/emit_kind/emit_on_filter.substrait.json", + ); + let ctx = add_plan_schemas_to_ctx(SessionContext::new(), &proto_plan)?; + let plan = from_substrait_plan(&ctx, &proto_plan).await?; + + let plan_str = format!("{}", plan); + + assert_eq!( + plan_str, + // Note that duplicate references in the remap are aliased + "Projection: DATA.B, DATA.A AS A1, DATA.A AS DATA.A__temp__0 AS A2\ + \n Filter: DATA.B = Int64(2)\ + \n TableScan: DATA" + ); + Ok(()) + } + + async fn make_context() -> Result { + let state = SessionStateBuilder::new() + .with_config(SessionConfig::default()) + .with_default_features() + .build(); + let ctx = SessionContext::new_with_state(state); + ctx.register_csv("data", "tests/testdata/data.csv", CsvReadOptions::default()) + .await?; + Ok(ctx) + } + + #[tokio::test] + async fn handle_emit_as_project_with_volatile_expr() -> Result<()> { + let ctx = make_context().await?; + + let df = ctx + .sql("SELECT random() AS c1, a + 1 AS c2 FROM data") + .await?; + + let plan = df.into_unoptimized_plan(); + assert_eq!( + format!("{}", plan), + "Projection: random() AS c1, data.a + Int64(1) AS c2\ + \n TableScan: data" + ); + + let proto = to_substrait_plan(&plan, &ctx)?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; + // note how the Projections are not flattened + assert_eq!( + format!("{}", plan2), + "Projection: random() AS c1, data.a + Int64(1) AS c2\ + \n Projection: data.a, data.b, data.c, data.d, data.e, data.f, random(), data.a + Int64(1)\ + \n TableScan: data" + ); + Ok(()) + } + + #[tokio::test] + async fn handle_emit_as_project_without_volatile_exprs() -> Result<()> { + let ctx = make_context().await?; + let df = ctx.sql("SELECT a + 1, b + 2 FROM data").await?; + + let plan = df.into_unoptimized_plan(); + assert_eq!( + format!("{}", plan), + "Projection: data.a + Int64(1), data.b + Int64(2)\ + \n TableScan: data" + ); + + let proto = to_substrait_plan(&plan, &ctx)?; + let plan2 = from_substrait_plan(&ctx, &proto).await?; + + let plan1str = format!("{plan}"); + let plan2str = format!("{plan2}"); + assert_eq!(plan1str, plan2str); + + Ok(()) + } +} diff --git a/datafusion/substrait/tests/cases/mod.rs b/datafusion/substrait/tests/cases/mod.rs index 42aa23626106..b1f4b95df66f 100644 --- a/datafusion/substrait/tests/cases/mod.rs +++ b/datafusion/substrait/tests/cases/mod.rs @@ -16,6 +16,7 @@ // under the License. mod consumer_integration; +mod emit_kind_tests; mod function_test; mod logical_plans; mod roundtrip_logical_plan; diff --git a/datafusion/substrait/tests/testdata/test_plans/emit_kind/direct_on_project.substrait.json b/datafusion/substrait/tests/testdata/test_plans/emit_kind/direct_on_project.substrait.json new file mode 100644 index 000000000000..63b275e1723f --- /dev/null +++ b/datafusion/substrait/tests/testdata/test_plans/emit_kind/direct_on_project.substrait.json @@ -0,0 +1,90 @@ +{ + "extensionUris": [{ + "extensionUriAnchor": 1, + "uri": "/functions_arithmetic.yaml" + }], + "extensions": [{ + "extensionFunction": { + "extensionUriReference": 1, + "functionAnchor": 0, + "name": "add:i64_i64" + } + }], + "relations": [{ + "root": { + "input": { + "project": { + "common": { + "direct": { + } + }, + "input": { + "read": { + "common": { + "direct": { + } + }, + "baseSchema": { + "names": ["A", "B"], + "struct": { + "types": [{ + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "namedTable": { + "names": ["DATA"] + } + } + }, + "expressions": [{ + "scalarFunction": { + "functionReference": 0, + "args": [], + "outputType": { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [{ + "value": { + "selection": { + "directReference": { + "structField": { + "field": 0 + } + }, + "rootReference": { + } + } + } + }, { + "value": { + "literal": { + "i64": 1, + "nullable": false, + "typeVariationReference": 0 + } + } + }], + "options": [] + } + }] + } + }, + "names": ["a", "b", "add1"] + } + }], + "expectedTypeUrls": [] +} \ No newline at end of file diff --git a/datafusion/substrait/tests/testdata/test_plans/emit_kind/emit_on_filter.substrait.json b/datafusion/substrait/tests/testdata/test_plans/emit_kind/emit_on_filter.substrait.json new file mode 100644 index 000000000000..2fc970155955 --- /dev/null +++ b/datafusion/substrait/tests/testdata/test_plans/emit_kind/emit_on_filter.substrait.json @@ -0,0 +1,91 @@ +{ + "extensionUris": [{ + "extensionUriAnchor": 1, + "uri": "/functions_comparison.yaml" + }], + "extensions": [{ + "extensionFunction": { + "extensionUriReference": 1, + "functionAnchor": 0, + "name": "equal:any_any" + } + }], + "relations": [{ + "root": { + "input": { + "filter": { + "common": { + "emit": { + "outputMapping": [1, 0, 0] + } + }, + "input": { + "read": { + "common": { + "direct": { + } + }, + "baseSchema": { + "names": ["A", "B"], + "struct": { + "types": [{ + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, { + "i64": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }], + "typeVariationReference": 0, + "nullability": "NULLABILITY_REQUIRED" + } + }, + "namedTable": { + "names": ["DATA"] + } + } + }, + "condition": { + "scalarFunction": { + "functionReference": 0, + "args": [], + "outputType": { + "bool": { + "typeVariationReference": 0, + "nullability": "NULLABILITY_NULLABLE" + } + }, + "arguments": [{ + "value": { + "selection": { + "directReference": { + "structField": { + "field": 1 + } + }, + "rootReference": { + } + } + } + }, { + "value": { + "literal": { + "i64": "2", + "nullable": false, + "typeVariationReference": 0 + } + } + }], + "options": [] + } + } + } + }, + "names": ["B", "A1", "A2"] + } + }], + "expectedTypeUrls": [] +} \ No newline at end of file From a2e5330d5cad8516c1524bc86fce32cae324b761 Mon Sep 17 00:00:00 2001 From: Martin Hilton Date: Fri, 1 Nov 2024 17:24:33 +0000 Subject: [PATCH 02/28] fix: date_bin() on timstamps before 1970 (#13204) * fix: date_bin() on timstamps before 1970 The date_bin() function was not working correctly for timestamps before 1970. Specifically if the input timestamp was the exact time of the start of a bin then it would be placed in the previous bin. The % operator has a negative result when the dividend is negative. This causes the date_bin calculation to round up to the next bin. To compensate the size of 1 interval is subtracted from the result if the input is negative. This subtraction is no longer performed if the input is already the exact time of the start of a bin. * fix clippy --------- Co-authored-by: Andrew Lamb --- datafusion/functions/src/datetime/date_bin.rs | 30 ++++++++++++++++++- .../sqllogictest/test_files/timestamps.slt | 17 +++++++++++ 2 files changed, 46 insertions(+), 1 deletion(-) diff --git a/datafusion/functions/src/datetime/date_bin.rs b/datafusion/functions/src/datetime/date_bin.rs index e335c4e097f7..e8d065df8633 100644 --- a/datafusion/functions/src/datetime/date_bin.rs +++ b/datafusion/functions/src/datetime/date_bin.rs @@ -240,7 +240,7 @@ fn date_bin_nanos_interval(stride_nanos: i64, source: i64, origin: i64) -> i64 { fn compute_distance(time_diff: i64, stride: i64) -> i64 { let time_delta = time_diff - (time_diff % stride); - if time_diff < 0 && stride > 1 { + if time_diff < 0 && stride > 1 && time_delta != time_diff { // The origin is later than the source timestamp, round down to the previous bin time_delta - stride } else { @@ -864,4 +864,32 @@ mod tests { assert_eq!(result, expected1, "{source} = {expected}"); }) } + + #[test] + fn test_date_bin_before_epoch() { + let cases = [ + ( + (TimeDelta::try_minutes(15), "1969-12-31T23:44:59.999999999"), + "1969-12-31T23:30:00", + ), + ( + (TimeDelta::try_minutes(15), "1969-12-31T23:45:00"), + "1969-12-31T23:45:00", + ), + ( + (TimeDelta::try_minutes(15), "1969-12-31T23:45:00.000000001"), + "1969-12-31T23:45:00", + ), + ]; + + cases.iter().for_each(|((stride, source), expected)| { + let stride = stride.unwrap(); + let stride1 = stride.num_nanoseconds().unwrap(); + let source1 = string_to_timestamp_nanos(source).unwrap(); + + let expected1 = string_to_timestamp_nanos(expected).unwrap(); + let result = date_bin_nanos_interval(stride1, source1, 0); + assert_eq!(result, expected1, "{source} = {expected}"); + }) + } } diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 38c2a6647273..a09a63a791fc 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -980,6 +980,23 @@ SELECT DATE_BIN('3 years 1 months', '2022-09-01 00:00:00Z'); ---- 2022-06-01T00:00:00 +# Times before the unix epoch +query P +select date_bin('1 hour', column1) +from (values + (timestamp '1969-01-01 00:00:00'), + (timestamp '1969-01-01 00:15:00'), + (timestamp '1969-01-01 00:30:00'), + (timestamp '1969-01-01 00:45:00'), + (timestamp '1969-01-01 01:00:00') +) as sq +---- +1969-01-01T00:00:00 +1969-01-01T00:00:00 +1969-01-01T00:00:00 +1969-01-01T00:00:00 +1969-01-01T01:00:00 + ### ## test date_trunc function ### From b7f4db4b56e945944c0448f84c61b6f3b86728f8 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 1 Nov 2024 18:25:50 +0100 Subject: [PATCH 03/28] Deprecate ScalarUDF::invoke and invoke_no_args for invoke_batch (#13179) `invoke_batch` is the one used now. The others are no longer in use and we should deprecate and remove them. --- datafusion/expr/src/udf.rs | 4 +- datafusion/functions-nested/benches/map.rs | 1 + .../functions/benches/character_length.rs | 28 +++++++-- datafusion/functions/benches/concat.rs | 5 +- datafusion/functions/benches/cot.rs | 10 ++- datafusion/functions/benches/date_bin.rs | 1 + datafusion/functions/benches/encoding.rs | 12 +++- datafusion/functions/benches/isnan.rs | 10 ++- datafusion/functions/benches/iszero.rs | 10 ++- datafusion/functions/benches/lower.rs | 46 ++++++++++---- datafusion/functions/benches/ltrim.rs | 7 ++- datafusion/functions/benches/make_date.rs | 4 ++ datafusion/functions/benches/nullif.rs | 5 +- datafusion/functions/benches/pad.rs | 30 +++++++-- datafusion/functions/benches/repeat.rs | 42 +++++++++++-- datafusion/functions/benches/signum.rs | 10 ++- datafusion/functions/benches/strpos.rs | 28 +++++++-- datafusion/functions/benches/substr.rs | 63 ++++++++++++++++--- datafusion/functions/benches/substr_index.rs | 1 + datafusion/functions/benches/to_char.rs | 3 + datafusion/functions/benches/to_timestamp.rs | 6 ++ datafusion/functions/benches/trunc.rs | 10 ++- datafusion/functions/benches/upper.rs | 5 +- datafusion/functions/src/core/version.rs | 2 +- 24 files changed, 284 insertions(+), 59 deletions(-) diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 83563603f2f3..b5e9a555c2da 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -193,6 +193,7 @@ impl ScalarUDF { /// Invoke the function on `args`, returning the appropriate result. /// /// See [`ScalarUDFImpl::invoke`] for more details. + #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] pub fn invoke(&self, args: &[ColumnarValue]) -> Result { self.inner.invoke(args) } @@ -215,13 +216,14 @@ impl ScalarUDF { /// Invoke the function without `args` but number of rows, returning the appropriate result. /// /// See [`ScalarUDFImpl::invoke_no_args`] for more details. + #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] pub fn invoke_no_args(&self, number_rows: usize) -> Result { self.inner.invoke_no_args(number_rows) } /// Returns a `ScalarFunctionImplementation` that can invoke the function /// during execution - #[deprecated(since = "42.0.0", note = "Use `invoke` or `invoke_no_args` instead")] + #[deprecated(since = "42.0.0", note = "Use `invoke_batch` instead")] pub fn fun(&self) -> ScalarFunctionImplementation { let captured = Arc::clone(&self.inner); Arc::new(move |args| captured.invoke(args)) diff --git a/datafusion/functions-nested/benches/map.rs b/datafusion/functions-nested/benches/map.rs index ca23d8b7ff4c..3c4a09c65992 100644 --- a/datafusion/functions-nested/benches/map.rs +++ b/datafusion/functions-nested/benches/map.rs @@ -96,6 +96,7 @@ fn criterion_benchmark(c: &mut Criterion) { b.iter(|| { black_box( + #[allow(deprecated)] // TODO use invoke_batch map_udf() .invoke(&[keys.clone(), values.clone()]) .expect("map should work on valid values"), diff --git a/datafusion/functions/benches/character_length.rs b/datafusion/functions/benches/character_length.rs index 17c4dd1f8912..9ba16807de01 100644 --- a/datafusion/functions/benches/character_length.rs +++ b/datafusion/functions/benches/character_length.rs @@ -84,28 +84,48 @@ fn criterion_benchmark(c: &mut Criterion) { let args_string_ascii = gen_string_array(n_rows, str_len, 0.1, 0.0, false); c.bench_function( &format!("character_length_StringArray_ascii_str_len_{}", str_len), - |b| b.iter(|| black_box(character_length.invoke(&args_string_ascii))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(character_length.invoke(&args_string_ascii)) + }) + }, ); // StringArray UTF8 let args_string_utf8 = gen_string_array(n_rows, str_len, 0.1, 0.5, false); c.bench_function( &format!("character_length_StringArray_utf8_str_len_{}", str_len), - |b| b.iter(|| black_box(character_length.invoke(&args_string_utf8))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(character_length.invoke(&args_string_utf8)) + }) + }, ); // StringViewArray ASCII only let args_string_view_ascii = gen_string_array(n_rows, str_len, 0.1, 0.0, true); c.bench_function( &format!("character_length_StringViewArray_ascii_str_len_{}", str_len), - |b| b.iter(|| black_box(character_length.invoke(&args_string_view_ascii))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(character_length.invoke(&args_string_view_ascii)) + }) + }, ); // StringViewArray UTF8 let args_string_view_utf8 = gen_string_array(n_rows, str_len, 0.1, 0.5, true); c.bench_function( &format!("character_length_StringViewArray_utf8_str_len_{}", str_len), - |b| b.iter(|| black_box(character_length.invoke(&args_string_view_utf8))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(character_length.invoke(&args_string_view_utf8)) + }) + }, ); } } diff --git a/datafusion/functions/benches/concat.rs b/datafusion/functions/benches/concat.rs index 91c46ac775a8..280819778f93 100644 --- a/datafusion/functions/benches/concat.rs +++ b/datafusion/functions/benches/concat.rs @@ -38,7 +38,10 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args(size, 32); let mut group = c.benchmark_group("concat function"); group.bench_function(BenchmarkId::new("concat", size), |b| { - b.iter(|| criterion::black_box(concat().invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + criterion::black_box(concat().invoke(&args).unwrap()) + }) }); group.finish(); } diff --git a/datafusion/functions/benches/cot.rs b/datafusion/functions/benches/cot.rs index e655d82dec91..a33f00b4b73e 100644 --- a/datafusion/functions/benches/cot.rs +++ b/datafusion/functions/benches/cot.rs @@ -33,12 +33,18 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_array = Arc::new(create_primitive_array::(size, 0.2)); let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("cot f32 array: {}", size), |b| { - b.iter(|| black_box(cot_fn.invoke(&f32_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(cot_fn.invoke(&f32_args).unwrap()) + }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("cot f64 array: {}", size), |b| { - b.iter(|| black_box(cot_fn.invoke(&f64_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(cot_fn.invoke(&f64_args).unwrap()) + }) }); } } diff --git a/datafusion/functions/benches/date_bin.rs b/datafusion/functions/benches/date_bin.rs index c881947354fd..4a8682c42f94 100644 --- a/datafusion/functions/benches/date_bin.rs +++ b/datafusion/functions/benches/date_bin.rs @@ -45,6 +45,7 @@ fn criterion_benchmark(c: &mut Criterion) { let udf = date_bin(); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( udf.invoke(&[interval.clone(), timestamps.clone()]) .expect("date_bin should work on valid values"), diff --git a/datafusion/functions/benches/encoding.rs b/datafusion/functions/benches/encoding.rs index d49235aac938..0615091e90d4 100644 --- a/datafusion/functions/benches/encoding.rs +++ b/datafusion/functions/benches/encoding.rs @@ -29,22 +29,30 @@ fn criterion_benchmark(c: &mut Criterion) { let str_array = Arc::new(create_string_array_with_len::(size, 0.2, 32)); c.bench_function(&format!("base64_decode/{size}"), |b| { let method = ColumnarValue::Scalar("base64".into()); + #[allow(deprecated)] // TODO use invoke_batch let encoded = encoding::encode() .invoke(&[ColumnarValue::Array(str_array.clone()), method.clone()]) .unwrap(); let args = vec![encoded, method]; - b.iter(|| black_box(decode.invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(decode.invoke(&args).unwrap()) + }) }); c.bench_function(&format!("hex_decode/{size}"), |b| { let method = ColumnarValue::Scalar("hex".into()); + #[allow(deprecated)] // TODO use invoke_batch let encoded = encoding::encode() .invoke(&[ColumnarValue::Array(str_array.clone()), method.clone()]) .unwrap(); let args = vec![encoded, method]; - b.iter(|| black_box(decode.invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(decode.invoke(&args).unwrap()) + }) }); } } diff --git a/datafusion/functions/benches/isnan.rs b/datafusion/functions/benches/isnan.rs index 16bbe073daf0..3e50de658b36 100644 --- a/datafusion/functions/benches/isnan.rs +++ b/datafusion/functions/benches/isnan.rs @@ -32,12 +32,18 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_array = Arc::new(create_primitive_array::(size, 0.2)); let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("isnan f32 array: {}", size), |b| { - b.iter(|| black_box(isnan.invoke(&f32_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(isnan.invoke(&f32_args).unwrap()) + }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("isnan f64 array: {}", size), |b| { - b.iter(|| black_box(isnan.invoke(&f64_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(isnan.invoke(&f64_args).unwrap()) + }) }); } } diff --git a/datafusion/functions/benches/iszero.rs b/datafusion/functions/benches/iszero.rs index 3348d172e1f2..3e6ac97063ca 100644 --- a/datafusion/functions/benches/iszero.rs +++ b/datafusion/functions/benches/iszero.rs @@ -32,12 +32,18 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_array = Arc::new(create_primitive_array::(size, 0.2)); let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("iszero f32 array: {}", size), |b| { - b.iter(|| black_box(iszero.invoke(&f32_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(iszero.invoke(&f32_args).unwrap()) + }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("iszero f64 array: {}", size), |b| { - b.iter(|| black_box(iszero.invoke(&f64_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(iszero.invoke(&f64_args).unwrap()) + }) }); } } diff --git a/datafusion/functions/benches/lower.rs b/datafusion/functions/benches/lower.rs index 934c1c6bd189..6cc67791464f 100644 --- a/datafusion/functions/benches/lower.rs +++ b/datafusion/functions/benches/lower.rs @@ -124,19 +124,32 @@ fn criterion_benchmark(c: &mut Criterion) { for size in [1024, 4096, 8192] { let args = create_args1(size, 32); c.bench_function(&format!("lower_all_values_are_ascii: {}", size), |b| { - b.iter(|| black_box(lower.invoke(&args))) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(lower.invoke(&args)) + }) }); let args = create_args2(size); c.bench_function( &format!("lower_the_first_value_is_nonascii: {}", size), - |b| b.iter(|| black_box(lower.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(lower.invoke(&args)) + }) + }, ); let args = create_args3(size); c.bench_function( &format!("lower_the_middle_value_is_nonascii: {}", size), - |b| b.iter(|| black_box(lower.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(lower.invoke(&args)) + }) + }, ); } @@ -151,24 +164,33 @@ fn criterion_benchmark(c: &mut Criterion) { for &size in &sizes { let args = create_args4(size, str_len, *null_density, mixed); c.bench_function( - &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", + &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", size, str_len, null_density, mixed), - |b| b.iter(|| black_box(lower.invoke(&args))), - ); + |b| b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(lower.invoke(&args)) + }), + ); let args = create_args4(size, str_len, *null_density, mixed); c.bench_function( - &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", + &format!("lower_all_values_are_ascii_string_views: size: {}, str_len: {}, null_density: {}, mixed: {}", size, str_len, null_density, mixed), - |b| b.iter(|| black_box(lower.invoke(&args))), - ); + |b| b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(lower.invoke(&args)) + }), + ); let args = create_args5(size, 0.1, *null_density); c.bench_function( - &format!("lower_some_values_are_nonascii_string_views: size: {}, str_len: {}, non_ascii_density: {}, null_density: {}, mixed: {}", + &format!("lower_some_values_are_nonascii_string_views: size: {}, str_len: {}, non_ascii_density: {}, null_density: {}, mixed: {}", size, str_len, 0.1, null_density, mixed), - |b| b.iter(|| black_box(lower.invoke(&args))), - ); + |b| b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(lower.invoke(&args)) + }), + ); } } } diff --git a/datafusion/functions/benches/ltrim.rs b/datafusion/functions/benches/ltrim.rs index b3fa5ef4fdff..4f94729b6fef 100644 --- a/datafusion/functions/benches/ltrim.rs +++ b/datafusion/functions/benches/ltrim.rs @@ -139,7 +139,12 @@ fn run_with_string_type( format!( "{string_type} [size={size}, len_before={len}, len_after={remaining_len}]", ), - |b| b.iter(|| black_box(ltrim.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(ltrim.invoke(&args)) + }) + }, ); } diff --git a/datafusion/functions/benches/make_date.rs b/datafusion/functions/benches/make_date.rs index cb8f1abe6d5d..a9844e4b2541 100644 --- a/datafusion/functions/benches/make_date.rs +++ b/datafusion/functions/benches/make_date.rs @@ -62,6 +62,7 @@ fn criterion_benchmark(c: &mut Criterion) { let days = ColumnarValue::Array(Arc::new(days(&mut rng)) as ArrayRef); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( make_date() .invoke(&[years.clone(), months.clone(), days.clone()]) @@ -77,6 +78,7 @@ fn criterion_benchmark(c: &mut Criterion) { let days = ColumnarValue::Array(Arc::new(days(&mut rng)) as ArrayRef); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( make_date() .invoke(&[year.clone(), months.clone(), days.clone()]) @@ -92,6 +94,7 @@ fn criterion_benchmark(c: &mut Criterion) { let days = ColumnarValue::Array(Arc::new(days(&mut rng)) as ArrayRef); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( make_date() .invoke(&[year.clone(), month.clone(), days.clone()]) @@ -106,6 +109,7 @@ fn criterion_benchmark(c: &mut Criterion) { let day = ColumnarValue::Scalar(ScalarValue::Int32(Some(26))); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( make_date() .invoke(&[year.clone(), month.clone(), day.clone()]) diff --git a/datafusion/functions/benches/nullif.rs b/datafusion/functions/benches/nullif.rs index dfabad335835..6e1154cf182a 100644 --- a/datafusion/functions/benches/nullif.rs +++ b/datafusion/functions/benches/nullif.rs @@ -33,7 +33,10 @@ fn criterion_benchmark(c: &mut Criterion) { ColumnarValue::Array(array), ]; c.bench_function(&format!("nullif scalar array: {}", size), |b| { - b.iter(|| black_box(nullif.invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(nullif.invoke(&args).unwrap()) + }) }); } } diff --git a/datafusion/functions/benches/pad.rs b/datafusion/functions/benches/pad.rs index 71fa68762c1e..4b21ca373047 100644 --- a/datafusion/functions/benches/pad.rs +++ b/datafusion/functions/benches/pad.rs @@ -101,17 +101,26 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("utf8 type", size), |b| { - b.iter(|| criterion::black_box(lpad().invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + criterion::black_box(lpad().invoke(&args).unwrap()) + }) }); let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("largeutf8 type", size), |b| { - b.iter(|| criterion::black_box(lpad().invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + criterion::black_box(lpad().invoke(&args).unwrap()) + }) }); let args = create_args::(size, 32, true); group.bench_function(BenchmarkId::new("stringview type", size), |b| { - b.iter(|| criterion::black_box(lpad().invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + criterion::black_box(lpad().invoke(&args).unwrap()) + }) }); group.finish(); @@ -120,18 +129,27 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("utf8 type", size), |b| { - b.iter(|| criterion::black_box(rpad().invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + criterion::black_box(rpad().invoke(&args).unwrap()) + }) }); let args = create_args::(size, 32, false); group.bench_function(BenchmarkId::new("largeutf8 type", size), |b| { - b.iter(|| criterion::black_box(rpad().invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + criterion::black_box(rpad().invoke(&args).unwrap()) + }) }); // rpad for stringview type let args = create_args::(size, 32, true); group.bench_function(BenchmarkId::new("stringview type", size), |b| { - b.iter(|| criterion::black_box(rpad().invoke(&args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + criterion::black_box(rpad().invoke(&args).unwrap()) + }) }); group.finish(); diff --git a/datafusion/functions/benches/repeat.rs b/datafusion/functions/benches/repeat.rs index 5643ccf07133..6e54c92b9b26 100644 --- a/datafusion/functions/benches/repeat.rs +++ b/datafusion/functions/benches/repeat.rs @@ -71,7 +71,12 @@ fn criterion_benchmark(c: &mut Criterion) { "repeat_string_view [size={}, repeat_times={}]", size, repeat_times ), - |b| b.iter(|| black_box(repeat.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(repeat.invoke(&args)) + }) + }, ); let args = create_args::(size, 32, repeat_times, false); @@ -80,7 +85,12 @@ fn criterion_benchmark(c: &mut Criterion) { "repeat_string [size={}, repeat_times={}]", size, repeat_times ), - |b| b.iter(|| black_box(repeat.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(repeat.invoke(&args)) + }) + }, ); let args = create_args::(size, 32, repeat_times, false); @@ -89,7 +99,12 @@ fn criterion_benchmark(c: &mut Criterion) { "repeat_large_string [size={}, repeat_times={}]", size, repeat_times ), - |b| b.iter(|| black_box(repeat.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(repeat.invoke(&args)) + }) + }, ); group.finish(); @@ -107,7 +122,12 @@ fn criterion_benchmark(c: &mut Criterion) { "repeat_string_view [size={}, repeat_times={}]", size, repeat_times ), - |b| b.iter(|| black_box(repeat.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(repeat.invoke(&args)) + }) + }, ); let args = create_args::(size, 32, repeat_times, false); @@ -116,7 +136,12 @@ fn criterion_benchmark(c: &mut Criterion) { "repeat_string [size={}, repeat_times={}]", size, repeat_times ), - |b| b.iter(|| black_box(repeat.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(repeat.invoke(&args)) + }) + }, ); let args = create_args::(size, 32, repeat_times, false); @@ -125,7 +150,12 @@ fn criterion_benchmark(c: &mut Criterion) { "repeat_large_string [size={}, repeat_times={}]", size, repeat_times ), - |b| b.iter(|| black_box(repeat.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(repeat.invoke(&args)) + }) + }, ); group.finish(); diff --git a/datafusion/functions/benches/signum.rs b/datafusion/functions/benches/signum.rs index 9f8d8258c823..ea1f5433df4e 100644 --- a/datafusion/functions/benches/signum.rs +++ b/datafusion/functions/benches/signum.rs @@ -32,12 +32,18 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_array = Arc::new(create_primitive_array::(size, 0.2)); let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("signum f32 array: {}", size), |b| { - b.iter(|| black_box(signum.invoke(&f32_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(signum.invoke(&f32_args).unwrap()) + }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("signum f64 array: {}", size), |b| { - b.iter(|| black_box(signum.invoke(&f64_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(signum.invoke(&f64_args).unwrap()) + }) }); } } diff --git a/datafusion/functions/benches/strpos.rs b/datafusion/functions/benches/strpos.rs index c78e69826836..31ca61e34c3a 100644 --- a/datafusion/functions/benches/strpos.rs +++ b/datafusion/functions/benches/strpos.rs @@ -112,28 +112,48 @@ fn criterion_benchmark(c: &mut Criterion) { let args_string_ascii = gen_string_array(n_rows, str_len, 0.1, 0.0, false); c.bench_function( &format!("strpos_StringArray_ascii_str_len_{}", str_len), - |b| b.iter(|| black_box(strpos.invoke(&args_string_ascii))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(strpos.invoke(&args_string_ascii)) + }) + }, ); // StringArray UTF8 let args_string_utf8 = gen_string_array(n_rows, str_len, 0.1, 0.5, false); c.bench_function( &format!("strpos_StringArray_utf8_str_len_{}", str_len), - |b| b.iter(|| black_box(strpos.invoke(&args_string_utf8))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(strpos.invoke(&args_string_utf8)) + }) + }, ); // StringViewArray ASCII only let args_string_view_ascii = gen_string_array(n_rows, str_len, 0.1, 0.0, true); c.bench_function( &format!("strpos_StringViewArray_ascii_str_len_{}", str_len), - |b| b.iter(|| black_box(strpos.invoke(&args_string_view_ascii))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(strpos.invoke(&args_string_view_ascii)) + }) + }, ); // StringViewArray UTF8 let args_string_view_utf8 = gen_string_array(n_rows, str_len, 0.1, 0.5, true); c.bench_function( &format!("strpos_StringViewArray_utf8_str_len_{}", str_len), - |b| b.iter(|| black_box(strpos.invoke(&args_string_view_utf8))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(strpos.invoke(&args_string_view_utf8)) + }) + }, ); } } diff --git a/datafusion/functions/benches/substr.rs b/datafusion/functions/benches/substr.rs index 90ba75c1e8a5..21020dad31a4 100644 --- a/datafusion/functions/benches/substr.rs +++ b/datafusion/functions/benches/substr.rs @@ -107,19 +107,34 @@ fn criterion_benchmark(c: &mut Criterion) { let args = create_args_without_count::(size, len, true, true); group.bench_function( format!("substr_string_view [size={}, strlen={}]", size, len), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); let args = create_args_without_count::(size, len, false, false); group.bench_function( format!("substr_string [size={}, strlen={}]", size, len), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); let args = create_args_without_count::(size, len, true, false); group.bench_function( format!("substr_large_string [size={}, strlen={}]", size, len), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); group.finish(); @@ -137,7 +152,12 @@ fn criterion_benchmark(c: &mut Criterion) { "substr_string_view [size={}, count={}, strlen={}]", size, count, len, ), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); let args = create_args_with_count::(size, len, count, false); @@ -146,7 +166,12 @@ fn criterion_benchmark(c: &mut Criterion) { "substr_string [size={}, count={}, strlen={}]", size, count, len, ), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); let args = create_args_with_count::(size, len, count, false); @@ -155,7 +180,12 @@ fn criterion_benchmark(c: &mut Criterion) { "substr_large_string [size={}, count={}, strlen={}]", size, count, len, ), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); group.finish(); @@ -173,7 +203,12 @@ fn criterion_benchmark(c: &mut Criterion) { "substr_string_view [size={}, count={}, strlen={}]", size, count, len, ), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); let args = create_args_with_count::(size, len, count, false); @@ -182,7 +217,12 @@ fn criterion_benchmark(c: &mut Criterion) { "substr_string [size={}, count={}, strlen={}]", size, count, len, ), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); let args = create_args_with_count::(size, len, count, false); @@ -191,7 +231,12 @@ fn criterion_benchmark(c: &mut Criterion) { "substr_large_string [size={}, count={}, strlen={}]", size, count, len, ), - |b| b.iter(|| black_box(substr.invoke(&args))), + |b| { + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(substr.invoke(&args)) + }) + }, ); group.finish(); diff --git a/datafusion/functions/benches/substr_index.rs b/datafusion/functions/benches/substr_index.rs index bb9a5b809eee..1e793cf4db8c 100644 --- a/datafusion/functions/benches/substr_index.rs +++ b/datafusion/functions/benches/substr_index.rs @@ -90,6 +90,7 @@ fn criterion_benchmark(c: &mut Criterion) { let args = [strings, delimiters, counts]; b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( substr_index() .invoke(&args) diff --git a/datafusion/functions/benches/to_char.rs b/datafusion/functions/benches/to_char.rs index d9a153e64abc..09032fdf2de1 100644 --- a/datafusion/functions/benches/to_char.rs +++ b/datafusion/functions/benches/to_char.rs @@ -86,6 +86,7 @@ fn criterion_benchmark(c: &mut Criterion) { let patterns = ColumnarValue::Array(Arc::new(patterns(&mut rng)) as ArrayRef); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_char() .invoke(&[data.clone(), patterns.clone()]) @@ -101,6 +102,7 @@ fn criterion_benchmark(c: &mut Criterion) { ColumnarValue::Scalar(ScalarValue::Utf8(Some("%Y-%m-%d".to_string()))); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_char() .invoke(&[data.clone(), patterns.clone()]) @@ -124,6 +126,7 @@ fn criterion_benchmark(c: &mut Criterion) { ))); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_char() .invoke(&[data.clone(), pattern.clone()]) diff --git a/datafusion/functions/benches/to_timestamp.rs b/datafusion/functions/benches/to_timestamp.rs index 5a87b34caf47..11816fe9c64f 100644 --- a/datafusion/functions/benches/to_timestamp.rs +++ b/datafusion/functions/benches/to_timestamp.rs @@ -113,6 +113,7 @@ fn criterion_benchmark(c: &mut Criterion) { let string_array = ColumnarValue::Array(Arc::new(data()) as ArrayRef); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_timestamp() .invoke(&[string_array.clone()]) @@ -126,6 +127,7 @@ fn criterion_benchmark(c: &mut Criterion) { let string_array = ColumnarValue::Array(Arc::new(data) as ArrayRef); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_timestamp() .invoke(&[string_array.clone()]) @@ -139,6 +141,7 @@ fn criterion_benchmark(c: &mut Criterion) { let string_array = ColumnarValue::Array(Arc::new(data) as ArrayRef); b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_timestamp() .invoke(&[string_array.clone()]) @@ -157,6 +160,7 @@ fn criterion_benchmark(c: &mut Criterion) { ColumnarValue::Array(Arc::new(format3) as ArrayRef), ]; b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_timestamp() .invoke(&args.clone()) @@ -183,6 +187,7 @@ fn criterion_benchmark(c: &mut Criterion) { ), ]; b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_timestamp() .invoke(&args.clone()) @@ -209,6 +214,7 @@ fn criterion_benchmark(c: &mut Criterion) { ), ]; b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch black_box( to_timestamp() .invoke(&args.clone()) diff --git a/datafusion/functions/benches/trunc.rs b/datafusion/functions/benches/trunc.rs index 92a08abf3d32..07ce522eb913 100644 --- a/datafusion/functions/benches/trunc.rs +++ b/datafusion/functions/benches/trunc.rs @@ -33,12 +33,18 @@ fn criterion_benchmark(c: &mut Criterion) { let f32_array = Arc::new(create_primitive_array::(size, 0.2)); let f32_args = vec![ColumnarValue::Array(f32_array)]; c.bench_function(&format!("trunc f32 array: {}", size), |b| { - b.iter(|| black_box(trunc.invoke(&f32_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(trunc.invoke(&f32_args).unwrap()) + }) }); let f64_array = Arc::new(create_primitive_array::(size, 0.2)); let f64_args = vec![ColumnarValue::Array(f64_array)]; c.bench_function(&format!("trunc f64 array: {}", size), |b| { - b.iter(|| black_box(trunc.invoke(&f64_args).unwrap())) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(trunc.invoke(&f64_args).unwrap()) + }) }); } } diff --git a/datafusion/functions/benches/upper.rs b/datafusion/functions/benches/upper.rs index a3e5fbd7a433..ac4ecacff941 100644 --- a/datafusion/functions/benches/upper.rs +++ b/datafusion/functions/benches/upper.rs @@ -37,7 +37,10 @@ fn criterion_benchmark(c: &mut Criterion) { for size in [1024, 4096, 8192] { let args = create_args(size, 32); c.bench_function("upper_all_values_are_ascii", |b| { - b.iter(|| black_box(upper.invoke(&args))) + b.iter(|| { + #[allow(deprecated)] // TODO use invoke_batch + black_box(upper.invoke(&args)) + }) }); } } diff --git a/datafusion/functions/src/core/version.rs b/datafusion/functions/src/core/version.rs index f726122c649a..e7ac749ddddc 100644 --- a/datafusion/functions/src/core/version.rs +++ b/datafusion/functions/src/core/version.rs @@ -118,7 +118,7 @@ mod test { #[tokio::test] async fn test_version_udf() { let version_udf = ScalarUDF::from(VersionFunc::new()); - let version = version_udf.invoke_no_args(0).unwrap(); + let version = version_udf.invoke_batch(&[], 1).unwrap(); if let ColumnarValue::Scalar(ScalarValue::Utf8(Some(version))) = version { assert!(version.starts_with("Apache DataFusion")); From 6b76a35cadb17b33a5140f6f67e0491aabaa409e Mon Sep 17 00:00:00 2001 From: Lordworms <48054792+Lordworms@users.noreply.github.com> Date: Fri, 1 Nov 2024 11:12:22 -0700 Subject: [PATCH 04/28] consider volatile function in simply_expression (#13128) * consider volatile function in simply_expression * refactor and fix bugs * fix clippy * refactor * refactor * format * fix clippy * Resolve logical conflict * simplify more --------- Co-authored-by: Andrew Lamb --- .../simplify_expressions/expr_simplifier.rs | 73 ++++++++++++++++++- .../src/simplify_expressions/utils.rs | 13 +++- 2 files changed, 78 insertions(+), 8 deletions(-) diff --git a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs index ce6734616b80..40be1f85391d 100644 --- a/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs +++ b/datafusion/optimizer/src/simplify_expressions/expr_simplifier.rs @@ -862,8 +862,8 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { right, }) if has_common_conjunction(&left, &right) => { let lhs: IndexSet = iter_conjunction_owned(*left).collect(); - let (common, rhs): (Vec<_>, Vec<_>) = - iter_conjunction_owned(*right).partition(|e| lhs.contains(e)); + let (common, rhs): (Vec<_>, Vec<_>) = iter_conjunction_owned(*right) + .partition(|e| lhs.contains(e) && !e.is_volatile()); let new_rhs = rhs.into_iter().reduce(and); let new_lhs = lhs.into_iter().filter(|e| !common.contains(e)).reduce(and); @@ -1682,8 +1682,8 @@ impl<'a, S: SimplifyInfo> TreeNodeRewriter for Simplifier<'a, S> { } fn has_common_conjunction(lhs: &Expr, rhs: &Expr) -> bool { - let lhs: HashSet<&Expr> = iter_conjunction(lhs).collect(); - iter_conjunction(rhs).any(|e| lhs.contains(&e)) + let lhs_set: HashSet<&Expr> = iter_conjunction(lhs).collect(); + iter_conjunction(rhs).any(|e| lhs_set.contains(&e) && !e.is_volatile()) } // TODO: We might not need this after defer pattern for Box is stabilized. https://github.com/rust-lang/rust/issues/87121 @@ -3978,4 +3978,69 @@ mod tests { unimplemented!("not needed for tests") } } + #[derive(Debug)] + struct VolatileUdf { + signature: Signature, + } + + impl VolatileUdf { + pub fn new() -> Self { + Self { + signature: Signature::exact(vec![], Volatility::Volatile), + } + } + } + impl ScalarUDFImpl for VolatileUdf { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn name(&self) -> &str { + "VolatileUdf" + } + + fn signature(&self) -> &Signature { + &self.signature + } + + fn return_type(&self, _arg_types: &[DataType]) -> Result { + Ok(DataType::Int16) + } + } + #[test] + fn test_optimize_volatile_conditions() { + let fun = Arc::new(ScalarUDF::new_from_impl(VolatileUdf::new())); + let rand = Expr::ScalarFunction(ScalarFunction::new_udf(fun, vec![])); + { + let expr = rand + .clone() + .eq(lit(0)) + .or(col("column1").eq(lit(2)).and(rand.clone().eq(lit(0)))); + + assert_eq!(simplify(expr.clone()), expr); + } + + { + let expr = col("column1") + .eq(lit(2)) + .or(col("column1").eq(lit(2)).and(rand.clone().eq(lit(0)))); + + assert_eq!(simplify(expr), col("column1").eq(lit(2))); + } + + { + let expr = (col("column1").eq(lit(2)).and(rand.clone().eq(lit(0)))).or(col( + "column1", + ) + .eq(lit(2)) + .and(rand.clone().eq(lit(0)))); + + assert_eq!( + simplify(expr), + col("column1") + .eq(lit(2)) + .and((rand.clone().eq(lit(0))).or(rand.clone().eq(lit(0)))) + ); + } + } } diff --git a/datafusion/optimizer/src/simplify_expressions/utils.rs b/datafusion/optimizer/src/simplify_expressions/utils.rs index 38bfc1a93403..c30c3631c193 100644 --- a/datafusion/optimizer/src/simplify_expressions/utils.rs +++ b/datafusion/optimizer/src/simplify_expressions/utils.rs @@ -67,16 +67,21 @@ pub static POWS_OF_TEN: [i128; 38] = [ /// returns true if `needle` is found in a chain of search_op /// expressions. Such as: (A AND B) AND C -pub fn expr_contains(expr: &Expr, needle: &Expr, search_op: Operator) -> bool { +fn expr_contains_inner(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) - || expr_contains(right, needle, search_op) + expr_contains_inner(left, needle, search_op) + || expr_contains_inner(right, needle, search_op) } _ => expr == needle, } } +/// check volatile calls and return if expr contains needle +pub fn expr_contains(expr: &Expr, needle: &Expr, search_op: Operator) -> bool { + expr_contains_inner(expr, needle, search_op) && !needle.is_volatile() +} + /// Deletes all 'needles' or remains one 'needle' that are found in a chain of xor /// expressions. Such as: A ^ (A ^ (B ^ A)) pub fn delete_xor_in_complex_expr(expr: &Expr, needle: &Expr, is_left: bool) -> Expr { @@ -206,7 +211,7 @@ pub fn is_false(expr: &Expr) -> bool { /// returns true if `haystack` looks like (needle OP X) or (X OP needle) pub fn is_op_with(target_op: Operator, haystack: &Expr, needle: &Expr) -> bool { - matches!(haystack, Expr::BinaryExpr(BinaryExpr { left, op, right }) if op == &target_op && (needle == left.as_ref() || needle == right.as_ref())) + matches!(haystack, Expr::BinaryExpr(BinaryExpr { left, op, right }) if op == &target_op && (needle == left.as_ref() || needle == right.as_ref()) && !needle.is_volatile()) } /// returns true if `not_expr` is !`expr` (not) From 9ff08005382d5d71fa5e9b176ed01c087e649c4b Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Fri, 1 Nov 2024 15:05:33 -0400 Subject: [PATCH 05/28] Minor: fix merge conflict (#13219) --- datafusion/substrait/src/logical_plan/consumer.rs | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index a12406bd3439..7ccca8616ba0 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -1126,13 +1126,8 @@ fn retrieve_emit_kind(rel_common: Option<&RelCommon>) -> EmitKind { .map_or(default, |ek| ek.clone()) } -fn contains_volatile_expr(proj: &Projection) -> Result { - for expr in proj.expr.iter() { - if expr.is_volatile()? { - return Ok(true); - } - } - Ok(false) +fn contains_volatile_expr(proj: &Projection) -> bool { + proj.expr.iter().any(|e| e.is_volatile()) } fn apply_emit_kind( @@ -1151,7 +1146,7 @@ fn apply_emit_kind( // expressions in the projection are volatile. This is to avoid issues like // converting a single call of the random() function into multiple calls due to // duplicate fields in the output_mapping. - LogicalPlan::Projection(proj) if !contains_volatile_expr(&proj)? => { + LogicalPlan::Projection(proj) if !contains_volatile_expr(&proj) => { let mut exprs: Vec = vec![]; for field in output_mapping { let expr = proj.expr From 752561a9fc821fce6a657fd1b959996900c44c5f Mon Sep 17 00:00:00 2001 From: ngli-me <107162634+ngli-me@users.noreply.github.com> Date: Fri, 1 Nov 2024 14:39:04 -0500 Subject: [PATCH 06/28] Convert LexOrdering `type` to `struct`. (#13146) * Conversion types for LexOrdering and LexOrderingRef to structs. * Format and fix type errors. Adjusted expected output when using `LexOrdering`. * Updated usage of `FromIterator` and removed `empty()` in favor of `default()`. * Adjusted chained `map` and `flatten` calls to `flat_map`, and swapped `unwrap_or` to `unwrap_or_default`. * Adjusted slt files to include a space after commas, when relating to LexOrdering and LexOrderingRef. * Removed unnecessary path prefixes in `sort_expr`. * Fixed tpch slt files. * Removed LexOrderingRef struct. * Removed dereferences to `LexOrderingRef` left over from the struct removal. * Removed remaining usage of the raw `LexOrderingRef` type. * Formatting. * Apply suggestions from code review, along with formatting. * Merged with main. * Merged with main. --------- Co-authored-by: nglime --- benchmarks/src/sort.rs | 6 +- datafusion/core/benches/physical_plan.rs | 3 +- datafusion/core/benches/sort.rs | 3 +- .../core/src/datasource/listing/table.rs | 37 +-- datafusion/core/src/datasource/mod.rs | 2 +- .../physical_plan/file_scan_config.rs | 5 +- .../core/src/datasource/physical_plan/mod.rs | 7 +- .../datasource/physical_plan/statistics.rs | 19 +- .../enforce_distribution.rs | 181 +++++++------- .../src/physical_optimizer/enforce_sorting.rs | 223 +++++++++--------- .../src/physical_optimizer/join_selection.rs | 8 +- .../physical_optimizer/projection_pushdown.rs | 50 ++-- .../replace_with_order_preserving_variants.rs | 6 +- .../src/physical_optimizer/sort_pushdown.rs | 25 +- .../core/src/physical_optimizer/test_utils.rs | 17 +- .../physical_optimizer/update_aggr_exprs.rs | 6 +- .../core/src/physical_optimizer/utils.rs | 2 +- datafusion/core/src/physical_planner.rs | 12 +- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 10 +- .../aggregation_fuzzer/data_generator.rs | 5 +- .../tests/fuzz_cases/equivalence/ordering.rs | 14 +- .../fuzz_cases/equivalence/projection.rs | 6 +- .../fuzz_cases/equivalence/properties.rs | 4 +- .../tests/fuzz_cases/equivalence/utils.rs | 12 +- .../core/tests/fuzz_cases/merge_fuzz.rs | 5 +- datafusion/core/tests/fuzz_cases/sort_fuzz.rs | 5 +- .../sort_preserving_repartition_fuzz.rs | 3 +- .../core/tests/fuzz_cases/window_fuzz.rs | 21 +- datafusion/core/tests/memory_limit/mod.rs | 40 ++-- .../limited_distinct_aggregation.rs | 5 +- .../tests/physical_optimizer/test_util.rs | 4 +- .../src/accumulator.rs | 7 +- .../functions-aggregate-common/src/utils.rs | 6 +- .../functions-aggregate/benches/count.rs | 3 +- datafusion/functions-aggregate/benches/sum.rs | 3 +- .../functions-aggregate/src/array_agg.rs | 4 +- .../functions-aggregate/src/first_last.rs | 92 +++++--- .../functions-aggregate/src/nth_value.rs | 4 +- datafusion/functions-aggregate/src/stddev.rs | 5 +- .../physical-expr-common/src/sort_expr.rs | 183 +++++++++++--- datafusion/physical-expr-common/src/utils.rs | 4 +- datafusion/physical-expr/src/aggregate.rs | 27 ++- .../physical-expr/src/equivalence/mod.rs | 25 +- .../physical-expr/src/equivalence/ordering.rs | 40 ++-- .../src/equivalence/properties.rs | 130 +++++----- datafusion/physical-expr/src/utils/mod.rs | 6 +- .../physical-expr/src/window/aggregate.rs | 24 +- .../physical-expr/src/window/built_in.rs | 17 +- .../src/window/sliding_aggregate.rs | 22 +- .../physical-expr/src/window/window_expr.rs | 4 +- .../src/topk_aggregation.rs | 3 +- datafusion/physical-plan/benches/spm.rs | 5 +- .../physical-plan/src/aggregates/mod.rs | 26 +- .../physical-plan/src/aggregates/order/mod.rs | 4 +- .../src/aggregates/order/partial.rs | 4 +- .../physical-plan/src/aggregates/row_hash.rs | 16 +- datafusion/physical-plan/src/display.rs | 23 +- .../physical-plan/src/execution_plan.rs | 12 +- .../src/joins/nested_loop_join.rs | 4 +- .../src/joins/sort_merge_join.rs | 14 +- .../src/joins/stream_join_utils.rs | 5 +- .../src/joins/symmetric_hash_join.rs | 112 ++++----- .../physical-plan/src/joins/test_utils.rs | 6 +- datafusion/physical-plan/src/joins/utils.rs | 40 ++-- datafusion/physical-plan/src/memory.rs | 23 +- .../physical-plan/src/repartition/mod.rs | 21 +- .../physical-plan/src/sorts/partial_sort.rs | 52 ++-- datafusion/physical-plan/src/sorts/sort.rs | 71 +++--- .../src/sorts/sort_preserving_merge.rs | 92 ++++---- datafusion/physical-plan/src/sorts/stream.rs | 3 +- .../src/sorts/streaming_merge.rs | 7 +- datafusion/physical-plan/src/topk/mod.rs | 8 +- datafusion/physical-plan/src/union.rs | 5 +- .../src/windows/bounded_window_agg_exec.rs | 33 +-- datafusion/physical-plan/src/windows/mod.rs | 27 ++- .../src/windows/window_agg_exec.rs | 13 +- .../proto/src/physical_plan/from_proto.rs | 8 +- datafusion/proto/src/physical_plan/mod.rs | 19 +- .../proto/src/physical_plan/to_proto.rs | 7 +- .../tests/cases/roundtrip_physical_plan.rs | 31 +-- .../sqllogictest/test_files/aggregate.slt | 2 +- datafusion/sqllogictest/test_files/cte.slt | 2 +- .../test_files/filter_without_sort_exec.slt | 6 +- .../sqllogictest/test_files/group_by.slt | 16 +- datafusion/sqllogictest/test_files/insert.slt | 6 +- .../test_files/insert_to_external.slt | 6 +- .../join_disable_repartition_joins.slt | 2 +- datafusion/sqllogictest/test_files/joins.slt | 12 +- .../test_files/monotonic_projection_test.slt | 10 +- datafusion/sqllogictest/test_files/order.slt | 16 +- .../sqllogictest/test_files/parquet.slt | 10 +- datafusion/sqllogictest/test_files/select.slt | 8 +- .../sqllogictest/test_files/subquery_sort.slt | 10 +- .../sqllogictest/test_files/tpch/q1.slt.part | 4 +- .../sqllogictest/test_files/tpch/q13.slt.part | 4 +- .../sqllogictest/test_files/tpch/q16.slt.part | 4 +- .../sqllogictest/test_files/tpch/q18.slt.part | 4 +- .../sqllogictest/test_files/tpch/q2.slt.part | 4 +- .../sqllogictest/test_files/tpch/q21.slt.part | 4 +- .../sqllogictest/test_files/tpch/q3.slt.part | 4 +- .../sqllogictest/test_files/tpch/q7.slt.part | 4 +- .../sqllogictest/test_files/tpch/q9.slt.part | 4 +- datafusion/sqllogictest/test_files/window.slt | 72 +++--- 103 files changed, 1238 insertions(+), 1022 deletions(-) diff --git a/benchmarks/src/sort.rs b/benchmarks/src/sort.rs index 247727e1b484..b2038c432f77 100644 --- a/benchmarks/src/sort.rs +++ b/benchmarks/src/sort.rs @@ -22,7 +22,7 @@ use crate::util::{AccessLogOpt, BenchmarkRun, CommonOpt}; use arrow::util::pretty; use datafusion::common::Result; -use datafusion::physical_expr::PhysicalSortExpr; +use datafusion::physical_expr::{LexOrdering, LexOrderingRef, PhysicalSortExpr}; use datafusion::physical_plan::collect; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::prelude::{SessionConfig, SessionContext}; @@ -170,13 +170,13 @@ impl RunOpt { async fn exec_sort( ctx: &SessionContext, - expr: &[PhysicalSortExpr], + expr: LexOrderingRef<'_>, test_file: &TestParquetFile, debug: bool, ) -> Result<(usize, std::time::Duration)> { let start = Instant::now(); let scan = test_file.create_scan(ctx, None).await?; - let exec = Arc::new(SortExec::new(expr.to_owned(), scan)); + let exec = Arc::new(SortExec::new(LexOrdering::new(expr.to_owned()), scan)); let task_ctx = ctx.task_ctx(); let result = collect(exec, task_ctx).await?; let elapsed = start.elapsed(); diff --git a/datafusion/core/benches/physical_plan.rs b/datafusion/core/benches/physical_plan.rs index 3ad71be1f447..349c2e438195 100644 --- a/datafusion/core/benches/physical_plan.rs +++ b/datafusion/core/benches/physical_plan.rs @@ -36,6 +36,7 @@ use datafusion::physical_plan::{ memory::MemoryExec, }; use datafusion::prelude::SessionContext; +use datafusion_physical_expr_common::sort_expr::LexOrdering; // Initialise the operator using the provided record batches and the sort key // as inputs. All record batches must have the same schema. @@ -52,7 +53,7 @@ fn sort_preserving_merge_operator( expr: col(name, &schema).unwrap(), options: Default::default(), }) - .collect::>(); + .collect::(); let exec = MemoryExec::try_new( &batches.into_iter().map(|rb| vec![rb]).collect::>(), diff --git a/datafusion/core/benches/sort.rs b/datafusion/core/benches/sort.rs index 99a74b61b3e0..14e80ce364e3 100644 --- a/datafusion/core/benches/sort.rs +++ b/datafusion/core/benches/sort.rs @@ -89,6 +89,7 @@ use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; /// Benchmarks for SortPreservingMerge stream use criterion::{criterion_group, criterion_main, Criterion}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::StreamExt; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -257,7 +258,7 @@ impl BenchCase { } /// Make sort exprs for each column in `schema` -fn make_sort_exprs(schema: &Schema) -> Vec { +fn make_sort_exprs(schema: &Schema) -> LexOrdering { schema .fields() .iter() diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index ea2e098ef14e..15125fe5a090 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -1283,13 +1283,16 @@ mod tests { // ok with one column ( vec![vec![col("string_col").sort(true, false)]], - Ok(vec![vec![PhysicalSortExpr { - expr: physical_col("string_col", &schema).unwrap(), - options: SortOptions { - descending: false, - nulls_first: false, - }, - }]]) + Ok(vec![LexOrdering { + inner: vec![PhysicalSortExpr { + expr: physical_col("string_col", &schema).unwrap(), + options: SortOptions { + descending: false, + nulls_first: false, + }, + }], + } + ]) ), // ok with two columns, different options ( @@ -1297,15 +1300,17 @@ mod tests { col("string_col").sort(true, false), col("int_col").sort(false, true), ]], - Ok(vec![vec![ - PhysicalSortExpr::new_default(physical_col("string_col", &schema).unwrap()) - .asc() - .nulls_last(), - - PhysicalSortExpr::new_default(physical_col("int_col", &schema).unwrap()) - .desc() - .nulls_first() - ]]) + Ok(vec![LexOrdering { + inner: vec![ + PhysicalSortExpr::new_default(physical_col("string_col", &schema).unwrap()) + .asc() + .nulls_last(), + PhysicalSortExpr::new_default(physical_col("int_col", &schema).unwrap()) + .desc() + .nulls_first() + ], + } + ]) ), ]; diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 0ed53418fe32..ad369b75e130 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -62,7 +62,7 @@ fn create_ordering( for exprs in sort_order { // Construct PhysicalSortExpr objects from Expr objects: - let mut sort_exprs = vec![]; + let mut sort_exprs = LexOrdering::default(); for sort in exprs { match &sort.expr { Expr::Column(col) => match expressions::col(&col.name, schema) { diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 96c0e452e29e..74ab0126a557 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -34,7 +34,8 @@ use arrow_array::{ArrayRef, DictionaryArray, RecordBatch, RecordBatchOptions}; use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, ColumnStatistics, DataFusionError, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use log::warn; @@ -307,7 +308,7 @@ impl FileScanConfig { pub fn split_groups_by_statistics( table_schema: &SchemaRef, file_groups: &[Vec], - sort_order: &[PhysicalSortExpr], + sort_order: LexOrderingRef, ) -> Result>> { let flattened_files = file_groups.iter().flatten().collect::>(); // First Fit: diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index 407a3b74f79f..9971e87282a5 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -65,6 +65,7 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::StreamExt; use log::debug; @@ -328,11 +329,11 @@ impl From for FileMeta { fn get_projected_output_ordering( base_config: &FileScanConfig, projected_schema: &SchemaRef, -) -> Vec> { +) -> Vec { let mut all_orderings = vec![]; for output_ordering in &base_config.output_ordering { - let mut new_ordering = vec![]; - for PhysicalSortExpr { expr, options } in output_ordering { + let mut new_ordering = LexOrdering::default(); + for PhysicalSortExpr { expr, options } in output_ordering.iter() { if let Some(col) = expr.as_any().downcast_ref::() { let name = col.name(); if let Some((idx, _)) = projected_schema.column_with_name(name) { diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs index 3ca3ba89f4d9..6af153a731b0 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/statistics.rs @@ -26,6 +26,8 @@ use std::sync::Arc; +use crate::datasource::listing::PartitionedFile; + use arrow::{ compute::SortColumn, row::{Row, Rows}, @@ -34,8 +36,7 @@ use arrow_array::RecordBatch; use arrow_schema::SchemaRef; use datafusion_common::{DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; - -use crate::datasource::listing::PartitionedFile; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; /// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. /// The min/max values are ordered by [`Self::sort_order`]. @@ -43,13 +44,13 @@ use crate::datasource::listing::PartitionedFile; pub(crate) struct MinMaxStatistics { min_by_sort_order: Rows, max_by_sort_order: Rows, - sort_order: Vec, + sort_order: LexOrdering, } impl MinMaxStatistics { /// Sort order used to sort the statistics #[allow(unused)] - pub fn sort_order(&self) -> &[PhysicalSortExpr] { + pub fn sort_order(&self) -> LexOrderingRef { &self.sort_order } @@ -65,8 +66,8 @@ impl MinMaxStatistics { } pub fn new_from_files<'a>( - projected_sort_order: &[PhysicalSortExpr], // Sort order with respect to projected schema - projected_schema: &SchemaRef, // Projected schema + projected_sort_order: LexOrderingRef, // Sort order with respect to projected schema + projected_schema: &SchemaRef, // Projected schema projection: Option<&[usize]>, // Indices of projection in full table schema (None = all columns) files: impl IntoIterator, ) -> Result { @@ -166,7 +167,7 @@ impl MinMaxStatistics { } pub fn new( - sort_order: &[PhysicalSortExpr], + sort_order: LexOrderingRef, schema: &SchemaRef, min_values: RecordBatch, max_values: RecordBatch, @@ -256,7 +257,7 @@ impl MinMaxStatistics { Ok(Self { min_by_sort_order: min.map_err(|e| e.context("build min rows"))?, max_by_sort_order: max.map_err(|e| e.context("build max rows"))?, - sort_order: sort_order.to_vec(), + sort_order: LexOrdering::from_ref(sort_order), }) } @@ -277,7 +278,7 @@ impl MinMaxStatistics { } fn sort_columns_from_physical_sort_exprs( - sort_order: &[PhysicalSortExpr], + sort_order: LexOrderingRef, ) -> Option> { sort_order .iter() diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index ff8f16f4ee9c..6cd902db7244 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -52,11 +52,12 @@ use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, }; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; -use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -935,7 +936,7 @@ fn add_spm_on_top(input: DistributionContext) -> DistributionContext { let new_plan = if should_preserve_ordering { Arc::new(SortPreservingMergeExec::new( - input.plan.output_ordering().unwrap_or(&[]).to_vec(), + LexOrdering::from_ref(input.plan.output_ordering().unwrap_or(&[])), input.plan.clone(), )) as _ } else { @@ -1435,7 +1436,7 @@ pub(crate) mod tests { impl SortRequiredExec { fn new_with_requirement( input: Arc, - requirement: Vec, + requirement: LexOrdering, ) -> Self { let cache = Self::compute_properties(&input); Self { @@ -1461,11 +1462,7 @@ pub(crate) mod tests { _t: DisplayFormatType, f: &mut std::fmt::Formatter, ) -> std::fmt::Result { - write!( - f, - "SortRequiredExec: [{}]", - PhysicalSortExpr::format_list(&self.expr) - ) + write!(f, "SortRequiredExec: [{}]", self.expr) } } @@ -1495,7 +1492,9 @@ pub(crate) mod tests { if self.expr.is_empty() { vec![None] } else { - vec![Some(PhysicalSortRequirement::from_sort_exprs(&self.expr))] + vec![Some(PhysicalSortRequirement::from_sort_exprs( + self.expr.iter(), + ))] } } @@ -1540,7 +1539,7 @@ pub(crate) mod tests { /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec>, + output_ordering: Vec, ) -> Arc { ParquetExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1556,7 +1555,7 @@ pub(crate) mod tests { /// Created a sorted parquet exec with multiple files fn parquet_exec_multiple_sorted( - output_ordering: Vec>, + output_ordering: Vec, ) -> Arc { ParquetExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1573,7 +1572,7 @@ pub(crate) mod tests { csv_exec_with_sort(vec![]) } - fn csv_exec_with_sort(output_ordering: Vec>) -> Arc { + fn csv_exec_with_sort(output_ordering: Vec) -> Arc { Arc::new( CsvExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1596,9 +1595,7 @@ pub(crate) mod tests { } // Created a sorted parquet exec with multiple files - fn csv_exec_multiple_sorted( - output_ordering: Vec>, - ) -> Arc { + fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { Arc::new( CsvExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) @@ -1728,7 +1725,7 @@ pub(crate) mod tests { } fn sort_exec( - sort_exprs: Vec, + sort_exprs: LexOrdering, input: Arc, preserve_partitioning: bool, ) -> Arc { @@ -1738,7 +1735,7 @@ pub(crate) mod tests { } fn sort_preserving_merge_exec( - sort_exprs: Vec, + sort_exprs: LexOrdering, input: Arc, ) -> Arc { Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) @@ -3076,7 +3073,7 @@ pub(crate) mod tests { // Only two RepartitionExecs added let expected = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "SortExec: expr=[b3@1 ASC,a3@0 ASC], preserve_partitioning=[true]", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", @@ -3084,7 +3081,7 @@ pub(crate) mod tests { "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b2@1 ASC,a2@0 ASC], preserve_partitioning=[true]", + "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", @@ -3096,9 +3093,9 @@ pub(crate) mod tests { let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC,a3@0 ASC", + "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b3@1 ASC,a3@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", @@ -3107,9 +3104,9 @@ pub(crate) mod tests { "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC,a2@0 ASC", + "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b2@1 ASC,a2@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", @@ -3127,10 +3124,10 @@ pub(crate) mod tests { fn merge_does_not_need_sort() -> Result<()> { // see https://github.com/apache/datafusion/issues/4331 let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // Scan some sorted parquet files let exec = parquet_exec_multiple_sorted(vec![sort_key.clone()]); @@ -3329,10 +3326,10 @@ pub(crate) mod tests { #[test] fn repartition_sorted_limit() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); let expected = &[ @@ -3351,10 +3348,10 @@ pub(crate) mod tests { #[test] fn repartition_sorted_limit_with_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_required_exec_with_req( filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), sort_key, @@ -3430,10 +3427,10 @@ pub(crate) mod tests { fn repartition_through_sort_preserving_merge() -> Result<()> { // sort preserving merge with non-sorted input let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); // need resort as the data was not sorted correctly @@ -3451,10 +3448,10 @@ pub(crate) mod tests { fn repartition_ignores_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_preserving_merge_exec( sort_key.clone(), parquet_exec_multiple_sorted(vec![sort_key]), @@ -3483,10 +3480,10 @@ pub(crate) mod tests { fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); let plan = sort_preserving_merge_exec(sort_key, input); @@ -3517,10 +3514,10 @@ pub(crate) mod tests { // SortRequired // Parquet(sorted) let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("d", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_required_exec_with_req( filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), sort_key, @@ -3552,10 +3549,10 @@ pub(crate) mod tests { // Parquet(unsorted) let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input1 = sort_required_exec_with_req( parquet_exec_with_sort(vec![sort_key.clone()]), sort_key, @@ -3594,10 +3591,10 @@ pub(crate) mod tests { )]; // non sorted input let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("sum", &proj.schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_preserving_merge_exec(sort_key, proj); let expected = &[ @@ -3627,10 +3624,10 @@ pub(crate) mod tests { #[test] fn repartition_ignores_transitively_with_projection() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let alias = vec![ ("a".to_string(), "a".to_string()), ("b".to_string(), "b".to_string()), @@ -3660,10 +3657,10 @@ pub(crate) mod tests { #[test] fn repartition_transitively_past_sort_with_projection() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let alias = vec![ ("a".to_string(), "a".to_string()), ("b".to_string(), "b".to_string()), @@ -3693,10 +3690,10 @@ pub(crate) mod tests { #[test] fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); let expected = &[ @@ -3727,10 +3724,10 @@ pub(crate) mod tests { #[cfg(feature = "parquet")] fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = sort_exec( sort_key, projection_exec_with_alias( @@ -3797,10 +3794,10 @@ pub(crate) mod tests { #[test] fn parallelization_multiple_files() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); let plan = sort_required_exec_with_req(plan, sort_key); @@ -3961,10 +3958,10 @@ pub(crate) mod tests { #[test] fn parallelization_sorted_limit() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); @@ -3993,10 +3990,10 @@ pub(crate) mod tests { #[test] fn parallelization_limit_with_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let plan_parquet = limit_exec(filter_exec(sort_exec( sort_key.clone(), parquet_exec(), @@ -4116,10 +4113,10 @@ pub(crate) mod tests { #[test] fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // sort preserving merge already sorted input, let plan_parquet = sort_preserving_merge_exec( sort_key.clone(), @@ -4146,10 +4143,10 @@ pub(crate) mod tests { #[test] fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let input_parquet = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); @@ -4180,10 +4177,10 @@ pub(crate) mod tests { #[test] fn parallelization_does_not_benefit() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); // SortRequired // Parquet(sorted) let plan_parquet = sort_required_exec_with_req( @@ -4214,10 +4211,10 @@ pub(crate) mod tests { fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { // sorted input let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); //Projection(a as a2, b as b2) let alias_pairs: Vec<(String, String)> = vec![ @@ -4228,10 +4225,10 @@ pub(crate) mod tests { parquet_exec_with_sort(vec![sort_key]), alias_pairs, ); - let sort_key_after_projection = vec![PhysicalSortExpr { + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_parquet.schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let plan_parquet = sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); let expected = &[ @@ -4255,10 +4252,10 @@ pub(crate) mod tests { fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { // sorted input let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); //Projection(a as a2, b as b2) let alias_pairs: Vec<(String, String)> = vec![ @@ -4268,10 +4265,10 @@ pub(crate) mod tests { let proj_csv = projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); - let sort_key_after_projection = vec![PhysicalSortExpr { + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_csv.schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", @@ -4318,10 +4315,10 @@ pub(crate) mod tests { #[test] fn remove_unnecessary_spm_after_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -4343,10 +4340,10 @@ pub(crate) mod tests { #[test] fn preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("d", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -4366,10 +4363,10 @@ pub(crate) mod tests { #[test] fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -4398,10 +4395,10 @@ pub(crate) mod tests { #[test] fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -4422,16 +4419,16 @@ pub(crate) mod tests { #[test] fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec_multiple_sorted(vec![sort_key]); - let sort_req = vec![PhysicalSortExpr { + let sort_req = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); let expected = &[ @@ -4460,10 +4457,10 @@ pub(crate) mod tests { #[test] fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec_multiple_sorted(vec![sort_key]); let physical_plan = filter_exec(input); @@ -4481,10 +4478,10 @@ pub(crate) mod tests { #[test] fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec(); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); let expected = &[ @@ -4518,10 +4515,10 @@ pub(crate) mod tests { #[test] fn put_sort_when_input_is_valid() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); @@ -4555,10 +4552,10 @@ pub(crate) mod tests { #[test] fn do_not_add_unnecessary_hash() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_with_sort(vec![sort_key]); let physical_plan = aggregate_exec_with_alias(input, alias); @@ -4578,10 +4575,10 @@ pub(crate) mod tests { #[test] fn do_not_add_unnecessary_hash2() -> Result<()> { let schema = schema(); - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_multiple_sorted(vec![sort_key]); let aggregate = aggregate_exec_with_alias(input, alias.clone()); diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index aa28f9d6b6aa..7b111cddc6fd 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -61,13 +61,14 @@ use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::{Partitioning, PhysicalSortExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::{Partitioning, PhysicalSortRequirement}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -231,7 +232,7 @@ fn replace_with_partial_sort( if common_prefix_length > 0 { return Ok(Arc::new( PartialSortExec::new( - sort_plan.expr().to_vec(), + LexOrdering::new(sort_plan.expr().to_vec()), sort_plan.input().clone(), common_prefix_length, ) @@ -275,7 +276,7 @@ fn parallelize_sorts( // Take the initial sort expressions and requirements let (sort_exprs, fetch) = get_sort_exprs(&requirements.plan)?; let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs); - let sort_exprs = sort_exprs.to_vec(); + let sort_exprs = LexOrdering::new(sort_exprs.to_vec()); // If there is a connection between a `CoalescePartitionsExec` and a // global sort that satisfy the requirements (i.e. intermediate @@ -390,15 +391,14 @@ fn analyze_immediate_sort_removal( if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { let sort_input = sort_exec.input(); // If this sort is unnecessary, we should remove it: - if sort_input - .equivalence_properties() - .ordering_satisfy(sort_exec.properties().output_ordering().unwrap_or(&[])) - { + if sort_input.equivalence_properties().ordering_satisfy( + sort_exec.properties().output_ordering().unwrap_or_default(), + ) { node.plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 { // Replace the sort with a sort-preserving merge: - let expr = sort_exec.expr().to_vec(); + let expr = LexOrdering::new(sort_exec.expr().to_vec()); Arc::new(SortPreservingMergeExec::new(expr, sort_input.clone())) as _ } else { // Remove the sort: @@ -619,7 +619,10 @@ fn remove_corresponding_sort_from_sub_plan( // `SortPreservingMergeExec` instead of a `CoalescePartitionsExec`. let plan = node.plan.clone(); let plan = if let Some(ordering) = plan.output_ordering() { - Arc::new(SortPreservingMergeExec::new(ordering.to_vec(), plan)) as _ + Arc::new(SortPreservingMergeExec::new( + LexOrdering::new(ordering.to_vec()), + plan, + )) as _ } else { Arc::new(CoalescePartitionsExec::new(plan)) as _ }; @@ -629,10 +632,10 @@ fn remove_corresponding_sort_from_sub_plan( Ok(node) } -/// Converts an [ExecutionPlan] trait object to a [PhysicalSortExpr] slice when possible. +/// Converts an [ExecutionPlan] trait object to a [LexOrderingRef] when possible. fn get_sort_exprs( sort_any: &Arc, -) -> Result<(&[PhysicalSortExpr], Option)> { +) -> Result<(LexOrderingRef, Option)> { if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { Ok((sort_exec.expr(), sort_exec.fetch())) } else if let Some(spm) = sort_any.as_any().downcast_ref::() @@ -645,7 +648,6 @@ fn get_sort_exprs( #[cfg(test)] mod tests { - use super::*; use crate::physical_optimizer::enforce_distribution::EnforceDistribution; use crate::physical_optimizer::test_utils::{ @@ -936,8 +938,8 @@ mod tests { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", @@ -961,10 +963,10 @@ mod tests { let sort = sort_exec(sort_exprs.clone(), source); let spm = sort_preserving_merge_exec(sort_exprs, sort); - let sort_exprs = vec![ + let sort_exprs = LexOrdering::new(vec![ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; + ]); let repartition_exec = repartition_exec(spm); let sort2 = Arc::new( SortExec::new(sort_exprs.clone(), repartition_exec) @@ -979,8 +981,8 @@ mod tests { // it with a `CoalescePartitionsExec` instead of directly removing it. let expected_input = [ "AggregateExec: mode=Final, gby=[], aggr=[]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[true]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", @@ -1006,7 +1008,7 @@ mod tests { let source2 = repartition_exec(memory_exec(&schema)); let union = union_exec(vec![source1, source2]); - let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort_exprs = LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]); // let sort = sort_exec(sort_exprs.clone(), union); let sort = Arc::new( SortExec::new(sort_exprs.clone(), union).with_preserve_partitioning(true), @@ -1029,7 +1031,7 @@ mod tests { // When removing a `SortPreservingMergeExec`, make sure that partitioning // requirements are not violated. In some cases, we may need to replace // it with a `CoalescePartitionsExec` instead of directly removing it. - let expected_input = ["SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + let expected_input = ["SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " FilterExec: NOT non_nullable_col@1", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", @@ -1039,8 +1041,8 @@ mod tests { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]"]; - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[true]", + let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1085,8 +1087,11 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = Arc::new( - SortExec::new(vec![sort_expr("non_nullable_col", &schema)], source) - .with_fetch(Some(2)), + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + ) + .with_fetch(Some(2)), ); let physical_plan = sort_exec( vec![ @@ -1097,12 +1102,12 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ - "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1115,26 +1120,29 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ sort_expr("non_nullable_col", &schema), sort_expr("nullable_col", &schema), - ], + ]), source, )); let physical_plan = Arc::new( - SortExec::new(vec![sort_expr("non_nullable_col", &schema)], input) - .with_fetch(Some(2)), + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + input, + ) + .with_fetch(Some(2)), ) as Arc; let expected_input = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "GlobalLimitExec: skip=0, fetch=2", - " SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1147,7 +1155,7 @@ mod tests { let schema = create_test_schema()?; let source = memory_exec(&schema); let input = Arc::new(SortExec::new( - vec![sort_expr("non_nullable_col", &schema)], + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), source, )); let limit = Arc::new(LocalLimitExec::new(input, 2)); @@ -1160,14 +1168,14 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " LocalLimitExec: fetch=2", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "LocalLimitExec: fetch=2", - " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC,nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1181,7 +1189,7 @@ mod tests { let source = memory_exec(&schema); // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); let input = Arc::new(SortExec::new( - vec![sort_expr("non_nullable_col", &schema)], + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), source, )); let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; @@ -1253,24 +1261,24 @@ mod tests { let repartition = repartition_exec(union); let physical_plan = sort_preserving_merge_exec(sort_exprs, repartition); - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // We should keep the bottom `SortExec`. - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[true]", + let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1288,12 +1296,12 @@ mod tests { let sort = sort_exec(vec![sort_exprs[0].clone()], source); let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1317,7 +1325,7 @@ mod tests { let expected_input = [ "SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ @@ -1409,17 +1417,17 @@ mod tests { // Input is an invalid plan. In this case rule should add required sorting in appropriate places. // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the // required ordering of SortPreservingMergeExec. - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1450,7 +1458,7 @@ mod tests { // Third input to the union is not Sorted (SortExec is matches required ordering by the SortPreservingMergeExec above). let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", @@ -1490,20 +1498,20 @@ mod tests { // Should modify the plan to ensure that all three inputs to the // `UnionExec` satisfy the ordering, OR add a single sort after // the `UnionExec` (both of which are equally good for this example). - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; - let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1542,9 +1550,9 @@ mod tests { // fine `SortExec`s below with required `SortExec`s that are absolutely necessary. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1588,7 +1596,7 @@ mod tests { " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Should adjust the requirement in the third input of the union so @@ -1625,9 +1633,9 @@ mod tests { // Union has unnecessarily fine ordering below it. We should be able to replace them with absolutely necessary ordering. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", @@ -1676,9 +1684,9 @@ mod tests { // The `UnionExec` doesn't preserve any of the inputs ordering in the // example below. let expected_input = ["UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. @@ -1744,10 +1752,10 @@ mod tests { async fn test_window_multi_path_sort2() -> Result<()> { let schema = create_test_schema()?; - let sort_exprs1 = vec![ + let sort_exprs1 = LexOrdering::new(vec![ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), - ]; + ]); let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; let source1 = parquet_exec_sorted(&schema, sort_exprs2.clone()); let source2 = parquet_exec_sorted(&schema, sort_exprs2.clone()); @@ -1761,11 +1769,11 @@ mod tests { // The `WindowAggExec` can get its required sorting from the leaf nodes directly. // The unnecessary SortExecs should be removed let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortPreservingMergeExec: [nullable_col@0 ASC]", @@ -1810,11 +1818,11 @@ mod tests { // Should not change the unnecessarily fine `SortExec`s because there is `LimitExec` let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", @@ -1822,7 +1830,7 @@ mod tests { " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1867,7 +1875,7 @@ mod tests { let join_plan2 = format!( " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" ); - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", join_plan2.as_str(), " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; @@ -1879,7 +1887,7 @@ mod tests { // can push down the sort requirements and save 1 SortExec vec![ join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", @@ -1888,7 +1896,7 @@ mod tests { _ => { // can not push down the sort requirements vec![ - "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -1938,9 +1946,9 @@ mod tests { ); let spm_plan = match join_type { JoinType::RightAnti => { - "SortPreservingMergeExec: [col_a@0 ASC,col_b@1 ASC]" + "SortPreservingMergeExec: [col_a@0 ASC, col_b@1 ASC]" } - _ => "SortPreservingMergeExec: [col_a@2 ASC,col_b@3 ASC]", + _ => "SortPreservingMergeExec: [col_a@2 ASC, col_b@3 ASC]", }; let join_plan2 = format!( " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" @@ -1956,14 +1964,14 @@ mod tests { join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC,col_b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", ] } _ => { // can not push down the sort requirements for Left and Full join. vec![ - "SortExec: expr=[col_a@2 ASC,col_b@3 ASC], preserve_partitioning=[false]", + "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -2001,13 +2009,13 @@ mod tests { ]; let physical_plan = sort_preserving_merge_exec(sort_exprs1, join.clone()); - let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC,col_a@2 ASC]", + let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec - let expected_optimized = ["SortExec: expr=[col_b@3 ASC,col_a@2 ASC], preserve_partitioning=[false]", + let expected_optimized = ["SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -2023,13 +2031,13 @@ mod tests { ]; let physical_plan = sort_preserving_merge_exec(sort_exprs2, join); - let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC]", + let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec - let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC], preserve_partitioning=[false]", + let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -2069,7 +2077,7 @@ mod tests { let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2124,7 +2132,7 @@ mod tests { let state = session_ctx.state(); let memory_exec = memory_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); let window = bounded_window_exec("nullable_col", sort_exprs.clone(), memory_exec); let repartition = repartition_exec(window); @@ -2174,7 +2182,7 @@ mod tests { let repartition = repartition_exec(source); let coalesce_partitions = Arc::new(CoalescePartitionsExec::new(repartition)); let repartition = repartition_exec(coalesce_partitions); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); // Add local sort let sort = Arc::new( SortExec::new(sort_exprs.clone(), repartition) @@ -2332,11 +2340,11 @@ mod tests { let physical_plan = sort_exec(vec![sort_expr("b", &schema)], spm); let expected_input = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; let expected_optimized = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -2360,12 +2368,12 @@ mod tests { spm, ); - let expected_input = ["SortExec: expr=[a@0 ASC,b@1 ASC,c@2 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", + let expected_input = ["SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; - let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", - " SortExec: expr=[a@0 ASC,b@1 ASC,c@2 ASC], preserve_partitioning=[true]", + let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", + " SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -2387,15 +2395,15 @@ mod tests { let expected_input = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortPreservingMergeExec: [a@0 ASC,b@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC,b@1 ASC", + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC, b@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -2418,11 +2426,11 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[a@0 ASC,c@2 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, c@2 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]" ]; let expected_optimized = [ - "PartialSortExec: expr=[a@0 ASC,c@2 ASC], common_prefix_length=[1]", + "PartialSortExec: expr=[a@0 ASC, c@2 ASC], common_prefix_length=[1]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2445,12 +2453,12 @@ mod tests { ); let expected_input = [ - "SortExec: expr=[a@0 ASC,c@2 ASC,d@3 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]" ]; // let optimized let expected_optimized = [ - "PartialSortExec: expr=[a@0 ASC,c@2 ASC,d@3 ASC], common_prefix_length=[2]", + "PartialSortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], common_prefix_length=[2]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2472,7 +2480,7 @@ mod tests { parquet_input, ); let expected_input = [ - "SortExec: expr=[a@0 ASC,b@1 ASC,c@2 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC]" ]; let expected_no_change = expected_input; @@ -2495,7 +2503,7 @@ mod tests { unbounded_input, ); let expected_input = [ - "SortExec: expr=[a@0 ASC,b@1 ASC,c@2 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" ]; let expected_no_change = expected_input; @@ -2510,8 +2518,8 @@ mod tests { // SortExec: expr=[a] // MemoryExec let schema = create_test_schema3()?; - let sort_exprs_a = vec![sort_expr("a", &schema)]; - let sort_exprs_b = vec![sort_expr("b", &schema)]; + let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); + let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); let plan = memory_exec(&schema); let plan = sort_exec(sort_exprs_a.clone(), plan); let plan = RequirementsTestExec::new(plan) @@ -2540,8 +2548,9 @@ mod tests { // SortExec: expr=[a] // MemoryExec let schema = create_test_schema3()?; - let sort_exprs_a = vec![sort_expr("a", &schema)]; - let sort_exprs_ab = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); + let sort_exprs_ab = + LexOrdering::new(vec![sort_expr("a", &schema), sort_expr("b", &schema)]); let plan = memory_exec(&schema); let plan = sort_exec(sort_exprs_a.clone(), plan); let plan = RequirementsTestExec::new(plan) @@ -2551,7 +2560,7 @@ mod tests { let plan = sort_exec(sort_exprs_ab, plan); let expected_input = [ - "SortExec: expr=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", @@ -2559,7 +2568,7 @@ mod tests { // should able to push shorts let expected = [ "RequiredInputOrderingExec", - " SortExec: expr=[a@0 ASC,b@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected, plan, true); diff --git a/datafusion/core/src/physical_optimizer/join_selection.rs b/datafusion/core/src/physical_optimizer/join_selection.rs index 2bf706f33d60..0312e362afb1 100644 --- a/datafusion/core/src/physical_optimizer/join_selection.rs +++ b/datafusion/core/src/physical_optimizer/join_selection.rs @@ -40,7 +40,8 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{internal_err, JoinSide, JoinType}; use datafusion_expr::sort_properties::SortProperties; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::PhysicalOptimizerRule; /// The [`JoinSelection`] rule tries to modify a given plan so that it can @@ -553,7 +554,7 @@ fn hash_join_convert_symmetric_subrule( // the function concludes that no specific order is required for the SymmetricHashJoinExec. This approach // ensures that the symmetric hash join operation only imposes ordering constraints when necessary, // based on the properties of the child nodes and the filter condition. - let determine_order = |side: JoinSide| -> Option> { + let determine_order = |side: JoinSide| -> Option { hash_join .filter() .map(|filter| { @@ -594,7 +595,7 @@ fn hash_join_convert_symmetric_subrule( JoinSide::Right => hash_join.right().output_ordering(), JoinSide::None => unreachable!(), } - .map(|p| p.to_vec()) + .map(|p| LexOrdering::new(p.to_vec())) }) .flatten() }; @@ -724,7 +725,6 @@ fn apply_subrules( #[cfg(test)] mod tests_statistical { - use super::*; use crate::{ physical_plan::{displayable, ColumnStatistics, Statistics}, diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index b4dd0a995d5f..5aecf036ce18 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -54,7 +54,7 @@ use datafusion_physical_expr::{ use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use itertools::Itertools; @@ -246,7 +246,7 @@ fn try_swapping_with_streaming_table( let mut lex_orderings = vec![]; for lex_ordering in streaming_table.projected_output_ordering().into_iter() { - let mut orderings = vec![]; + let mut orderings = LexOrdering::default(); for order in lex_ordering { let Some(new_ordering) = update_expr(&order.expr, projection.expr(), false)? else { @@ -467,7 +467,7 @@ fn try_swapping_with_sort( return Ok(None); } - let mut updated_exprs = vec![]; + let mut updated_exprs = LexOrdering::default(); for sort in sort.expr() { let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? else { return Ok(None); @@ -497,7 +497,7 @@ fn try_swapping_with_sort_preserving_merge( return Ok(None); } - let mut updated_exprs = vec![]; + let mut updated_exprs = LexOrdering::default(); for sort in spm.expr() { let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? else { @@ -915,8 +915,14 @@ fn try_swapping_with_sym_hash_join( new_filter, sym_join.join_type(), sym_join.null_equals_null(), - sym_join.right().output_ordering().map(|p| p.to_vec()), - sym_join.left().output_ordering().map(|p| p.to_vec()), + sym_join + .right() + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), + sym_join + .left() + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), sym_join.partition_mode(), )?))) } @@ -1863,7 +1869,7 @@ mod tests { }) as _], Some(&vec![0_usize, 2, 4, 3]), vec![ - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("e", 2)), options: SortOptions::default(), @@ -1872,11 +1878,11 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), }, - ], - vec![PhysicalSortExpr { + ]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("d", 3)), options: SortOptions::default(), - }], + }]), ] .into_iter(), true, @@ -1923,7 +1929,7 @@ mod tests { assert_eq!( result.projected_output_ordering().into_iter().collect_vec(), vec![ - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("e", 1)), options: SortOptions::default(), @@ -1932,11 +1938,11 @@ mod tests { expr: Arc::new(Column::new("a", 2)), options: SortOptions::default(), }, - ], - vec![PhysicalSortExpr { + ]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("d", 0)), options: SortOptions::default(), - }], + }]), ] ); assert!(result.is_infinite()); @@ -2553,7 +2559,7 @@ mod tests { fn test_sort_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -2566,7 +2572,7 @@ mod tests { )), options: SortOptions::default(), }, - ], + ]), csv.clone(), )); let projection: Arc = Arc::new(ProjectionExec::try_new( @@ -2581,7 +2587,7 @@ mod tests { let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[b@1 ASC, c@2 + a@0 ASC], preserve_partitioning=[false]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; assert_eq!(initial, expected_initial); @@ -2590,7 +2596,7 @@ mod tests { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "SortExec: expr=[b@2 ASC,c@0 + new_a@1 ASC], preserve_partitioning=[false]", + "SortExec: expr=[b@2 ASC, c@0 + new_a@1 ASC], preserve_partitioning=[false]", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; @@ -2603,7 +2609,7 @@ mod tests { fn test_sort_preserving_after_projection() -> Result<()> { let csv = create_simple_csv_exec(); let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -2616,7 +2622,7 @@ mod tests { )), options: SortOptions::default(), }, - ], + ]), csv.clone(), )); let projection: Arc = Arc::new(ProjectionExec::try_new( @@ -2631,7 +2637,7 @@ mod tests { let initial = get_plan_string(&projection); let expected_initial = [ "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", + " SortPreservingMergeExec: [b@1 ASC, c@2 + a@0 ASC]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; assert_eq!(initial, expected_initial); @@ -2640,7 +2646,7 @@ mod tests { ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "SortPreservingMergeExec: [b@2 ASC,c@0 + new_a@1 ASC]", + "SortPreservingMergeExec: [b@2 ASC, c@0 + new_a@1 ASC]", " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index a989be987d3d..930ce52e6fa2 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -33,6 +33,7 @@ use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; /// For a given `plan`, this object carries the information one needs from its @@ -131,7 +132,8 @@ fn plan_with_order_preserving_variants( if let Some(ordering) = child.output_ordering().map(Vec::from) { // When the input of a `CoalescePartitionsExec` has an ordering, // replace it with a `SortPreservingMergeExec` if appropriate: - let spm = SortPreservingMergeExec::new(ordering, child.clone()); + let spm = + SortPreservingMergeExec::new(LexOrdering::new(ordering), child.clone()); sort_input.plan = Arc::new(spm) as _; sort_input.children[0].data = true; return Ok(sort_input); @@ -255,7 +257,7 @@ pub(crate) fn replace_with_order_preserving_variants( if alternate_plan .plan .equivalence_properties() - .ordering_satisfy(requirements.plan.output_ordering().unwrap_or(&[])) + .ordering_satisfy(requirements.plan.output_ordering().unwrap_or_default()) { for child in alternate_plan.children.iter_mut() { child.data = false; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index fdbda1fe52f7..9eb200f534db 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -36,10 +36,10 @@ use datafusion_common::{plan_err, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{ - LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, +use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexOrderingRef, LexRequirement, }; -use datafusion_physical_expr_common::sort_expr::LexRequirement; use hashbrown::HashSet; @@ -235,7 +235,7 @@ fn pushdown_requirement_to_children( Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, parent_required, - &parent_required_expr, + parent_required_expr.as_ref(), JoinSide::Left, ), Some(JoinSide::Right) => { @@ -248,7 +248,7 @@ fn pushdown_requirement_to_children( try_pushdown_requirements_to_join( smj, parent_required, - &new_right_required_expr, + new_right_required_expr.as_ref(), JoinSide::Right, ) } @@ -277,7 +277,7 @@ fn pushdown_requirement_to_children( spm_eqs = spm_eqs.with_reorder(new_ordering); // Do not push-down through SortPreservingMergeExec when // ordering requirement invalidates requirement of sort preserving merge exec. - if !spm_eqs.ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { + if !spm_eqs.ordering_satisfy(plan.output_ordering().unwrap_or_default()) { Ok(None) } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer @@ -344,10 +344,11 @@ fn determine_children_requirement( RequirementsCompatibility::NonCompatible } } + fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, parent_required: LexRequirementRef, - sort_expr: &[PhysicalSortExpr], + sort_expr: LexOrderingRef, push_side: JoinSide, ) -> Result>>> { let left_eq_properties = smj.left().equivalence_properties(); @@ -355,13 +356,13 @@ fn try_pushdown_requirements_to_join( let mut smj_required_orderings = smj.required_input_ordering(); let right_requirement = smj_required_orderings.swap_remove(1); let left_requirement = smj_required_orderings.swap_remove(0); - let left_ordering = smj.left().output_ordering().unwrap_or(&[]); - let right_ordering = smj.right().output_ordering().unwrap_or(&[]); + let left_ordering = smj.left().output_ordering().unwrap_or_default(); + let right_ordering = smj.right().output_ordering().unwrap_or_default(); let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { let left_eq_properties = left_eq_properties .clone() - .with_reorder(Vec::from(sort_expr)); + .with_reorder(LexOrdering::from_ref(sort_expr)); if left_eq_properties .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) { @@ -374,7 +375,7 @@ fn try_pushdown_requirements_to_join( JoinSide::Right => { let right_eq_properties = right_eq_properties .clone() - .with_reorder(Vec::from(sort_expr)); + .with_reorder(LexOrdering::from_ref(sort_expr)); if right_eq_properties .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) { @@ -418,7 +419,7 @@ fn try_pushdown_requirements_to_join( } fn expr_source_side( - required_exprs: &[PhysicalSortExpr], + required_exprs: LexOrderingRef, join_type: JoinType, left_columns_len: usize, ) -> Option { diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 98f1a7c21a39..bdf16300ea87 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -57,7 +57,7 @@ use datafusion_physical_plan::{ use async_trait::async_trait; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_physical_expr_common::sort_expr::{ - LexRequirement, PhysicalSortRequirement, + LexOrdering, LexRequirement, PhysicalSortRequirement, }; async fn register_current_csv( @@ -243,7 +243,7 @@ pub fn bounded_window_exec( sort_exprs: impl IntoIterator, input: Arc, ) -> Arc { - let sort_exprs: Vec<_> = sort_exprs.into_iter().collect(); + let sort_exprs: LexOrdering = sort_exprs.into_iter().collect(); let schema = input.schema(); Arc::new( @@ -253,7 +253,7 @@ pub fn bounded_window_exec( "count".to_owned(), &[col(col_name, &schema).unwrap()], &[], - &sort_exprs, + sort_exprs.as_ref(), Arc::new(WindowFrame::new(Some(false))), schema.as_ref(), false, @@ -364,7 +364,7 @@ pub fn sort_exec( /// A test [`ExecutionPlan`] whose requirements can be configured. #[derive(Debug)] pub struct RequirementsTestExec { - required_input_ordering: Vec, + required_input_ordering: LexOrdering, maintains_input_order: bool, input: Arc, } @@ -372,7 +372,7 @@ pub struct RequirementsTestExec { impl RequirementsTestExec { pub fn new(input: Arc) -> Self { Self { - required_input_ordering: vec![], + required_input_ordering: LexOrdering::default(), maintains_input_order: true, input, } @@ -381,7 +381,7 @@ impl RequirementsTestExec { /// sets the required input ordering pub fn with_required_input_ordering( mut self, - required_input_ordering: Vec, + required_input_ordering: LexOrdering, ) -> Self { self.required_input_ordering = required_input_ordering; self @@ -419,8 +419,9 @@ impl ExecutionPlan for RequirementsTestExec { } fn required_input_ordering(&self) -> Vec> { - let requirement = - PhysicalSortRequirement::from_sort_exprs(&self.required_input_ordering); + let requirement = PhysicalSortRequirement::from_sort_exprs( + self.required_input_ordering.as_ref().iter(), + ); vec![Some(requirement)] } diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 26cdd65883e4..d85278556cc4 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -138,12 +138,12 @@ fn try_convert_aggregate_if_better( aggr_exprs .into_iter() .map(|aggr_expr| { - let aggr_sort_exprs = aggr_expr.order_bys().unwrap_or(&[]); + let aggr_sort_exprs = &aggr_expr.order_bys().unwrap_or_default(); let reverse_aggr_sort_exprs = reverse_order_bys(aggr_sort_exprs); let aggr_sort_reqs = - PhysicalSortRequirement::from_sort_exprs(aggr_sort_exprs); + PhysicalSortRequirement::from_sort_exprs(aggr_sort_exprs.iter()); let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_sort_exprs); + PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_sort_exprs.inner); // If the aggregate expression benefits from input ordering, and // there is an actual ordering enabling this, try to update the diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 2c0d042281e6..8007d8cc7f00 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -39,7 +39,7 @@ pub fn add_sort_above( fetch: Option, ) -> PlanContext { let mut sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirements); - sort_expr.retain(|sort_expr| { + sort_expr.inner.retain(|sort_expr| { !node .plan .equivalence_properties() diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index c16f3ad104b8..2a96a2ad111f 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1521,7 +1521,7 @@ pub fn create_window_expr_with_name( name, &physical_args, &partition_by, - &order_by, + order_by.as_ref(), window_frame, physical_schema, ignore_nulls, @@ -1550,7 +1550,7 @@ type AggregateExprWithOptionalArgs = ( // The filter clause, if any Option>, // Ordering requirements, if any - Option>, + Option, ); /// Create an aggregate expression with a name from a logical expression @@ -1600,12 +1600,12 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( None => None, }; - let ordering_reqs: Vec = - physical_sort_exprs.clone().unwrap_or(vec![]); + let ordering_reqs: LexOrdering = + physical_sort_exprs.clone().unwrap_or_default(); let agg_expr = AggregateExprBuilder::new(func.to_owned(), physical_args.to_vec()) - .order_by(ordering_reqs.to_vec()) + .order_by(ordering_reqs) .schema(Arc::new(physical_input_schema.to_owned())) .alias(name) .with_ignore_nulls(ignore_nulls) @@ -1674,7 +1674,7 @@ pub fn create_physical_sort_exprs( exprs .iter() .map(|expr| create_physical_sort_expr(expr, input_dfschema, execution_props)) - .collect::>>() + .collect::>() } impl DefaultPhysicalPlanner { diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index d715635c5951..21f604e6c60f 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -39,15 +39,15 @@ use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; +use crate::fuzz_cases::aggregation_fuzzer::{ + AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, +}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::HashMap; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; use tokio::task::JoinSet; -use crate::fuzz_cases::aggregation_fuzzer::{ - AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, -}; - // ======================================================================== // The new aggregation fuzz tests based on [`AggregationFuzzer`] // ======================================================================== @@ -234,7 +234,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); let ctx = SessionContext::new_with_config(session_config); - let mut sort_keys = vec![]; + let mut sort_keys = LexOrdering::default(); for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).unwrap(), diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs index 4fa1b7aa263d..aafa5ed7f66b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/data_generator.rs @@ -25,6 +25,7 @@ use arrow_array::{ArrayRef, RecordBatch}; use arrow_schema::{DataType, Field, Schema}; use datafusion_common::{arrow_datafusion_err, DataFusionError, Result}; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::sorts::sort::sort_batch; use rand::{ rngs::{StdRng, ThreadRng}, @@ -140,8 +141,8 @@ impl DatasetGenerator { let col_expr = col(key, schema)?; Ok(PhysicalSortExpr::new_default(col_expr)) }) - .collect::>>()?; - let sorted_batch = sort_batch(&base_batch, &sort_exprs, None)?; + .collect::>()?; + let sorted_batch = sort_batch(&base_batch, sort_exprs.as_ref(), None)?; let batches = stagger_batch(sorted_batch); let dataset = Dataset::new(batches, sort_keys); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index 94157e11702c..525baadd14a5 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -25,7 +25,7 @@ use datafusion_common::{DFSchema, Result}; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use std::sync::Arc; @@ -62,7 +62,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::>(); + .collect::(); let expected = is_table_same_after_sort( requirement.clone(), table_data_with_properties.clone(), @@ -74,7 +74,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(&requirement), + eq_properties.ordering_satisfy(requirement.as_ref()), expected, "{}", err_msg @@ -135,7 +135,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::>(); + .collect::(); let expected = is_table_same_after_sort( requirement.clone(), table_data_with_properties.clone(), @@ -148,7 +148,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { // experimental result matches. assert_eq!( - eq_properties.ordering_satisfy(&requirement), + eq_properties.ordering_satisfy(requirement.as_ref()), (expected | false), "{}", err_msg @@ -311,7 +311,7 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { expr: Arc::clone(expr), options, }) - .collect::>(); + .collect::(); // Check expected result with experimental result. assert_eq!( @@ -322,7 +322,7 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { expected ); assert_eq!( - eq_properties.ordering_satisfy(&required), + eq_properties.ordering_satisfy(required.as_ref()), expected, "{err_msg}" ); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index c0c8517a612b..3df3e0348e42 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -25,7 +25,7 @@ use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use std::sync::Arc; @@ -173,7 +173,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { expr: Arc::clone(expr), options: SORT_OPTIONS, }) - .collect::>(); + .collect::(); let expected = is_table_same_after_sort( requirement.clone(), projected_batch.clone(), @@ -185,7 +185,7 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - projected_eq.ordering_satisfy(&requirement), + projected_eq.ordering_satisfy(requirement.as_ref()), expected, "{}", err_msg diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index e704fcacc328..82586bd79eda 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -23,7 +23,7 @@ use datafusion_common::{DFSchema, Result}; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use itertools::Itertools; use std::sync::Arc; @@ -76,7 +76,7 @@ fn test_find_longest_permutation_random() -> Result<()> { expr: Arc::clone(&exprs[idx]), options: sort_expr.options, }) - .collect::>(); + .collect::(); assert_eq!( ordering, ordering2, "indices and lexicographical ordering do not match" diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index acc45fe0e591..35da8b596380 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -223,7 +223,7 @@ fn add_equal_conditions_test() -> Result<()> { /// If the table remains the same after sorting with the added unique column, it indicates that the table was /// already sorted according to `required_ordering` to begin with. pub fn is_table_same_after_sort( - mut required_ordering: Vec, + mut required_ordering: LexOrdering, batch: RecordBatch, ) -> Result { // Clone the original schema and columns @@ -444,7 +444,7 @@ pub fn generate_table_for_orderings( assert!(!orderings.is_empty()); // Sort the inner vectors by their lengths (longest first) - orderings.sort_by_key(|v| std::cmp::Reverse(v.len())); + orderings.sort_by_key(|v| std::cmp::Reverse(v.inner.len())); let arrays = schema .fields @@ -459,13 +459,13 @@ pub fn generate_table_for_orderings( let batch = RecordBatch::try_from_iter(arrays)?; // Sort batch according to first ordering expression - let sort_columns = get_sort_columns(&batch, &orderings[0])?; + let sort_columns = get_sort_columns(&batch, orderings[0].as_ref())?; let sort_indices = lexsort_to_indices(&sort_columns, None)?; let mut batch = take_record_batch(&batch, &sort_indices)?; // prune out rows that is invalid according to remaining orderings. for ordering in orderings.iter().skip(1) { - let sort_columns = get_sort_columns(&batch, ordering)?; + let sort_columns = get_sort_columns(&batch, ordering.as_ref())?; // Collect sort options and values into separate vectors. let (sort_options, sort_col_values): (Vec<_>, Vec<_>) = sort_columns @@ -495,7 +495,7 @@ pub fn generate_table_for_orderings( // Convert each tuple to PhysicalSortExpr pub fn convert_to_sort_exprs( in_data: &[(&Arc, SortOptions)], -) -> Vec { +) -> LexOrdering { in_data .iter() .map(|(expr, options)| PhysicalSortExpr { @@ -508,7 +508,7 @@ pub fn convert_to_sort_exprs( // Convert each inner tuple to PhysicalSortExpr pub fn convert_to_orderings( orderings: &[Vec<(&Arc, SortOptions)>], -) -> Vec> { +) -> Vec { orderings .iter() .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 4eb1070e6c85..4e895920dd3d 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -31,6 +31,7 @@ use datafusion::physical_plan::{ sorts::sort_preserving_merge::SortPreservingMergeExec, }; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; #[tokio::test] @@ -107,13 +108,13 @@ async fn run_merge_test(input: Vec>) { .expect("at least one batch"); let schema = first_batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("x", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&input, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 4ba06ef1d2a6..e4acb96f4930 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -30,6 +30,7 @@ use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use rand::Rng; use std::sync::Arc; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; @@ -114,13 +115,13 @@ impl SortTest { .expect("at least one batch"); let schema = first_batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("x", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&input, schema, None).unwrap(); let sort = Arc::new(SortExec::new(sort, Arc::new(exec))); diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 353db8668363..73f4a569954e 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -45,6 +45,7 @@ mod sp_repartition_fuzz_tests { }; use test_utils::add_empty_batches; + use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; @@ -345,7 +346,7 @@ mod sp_repartition_fuzz_tests { let schema = input1[0].schema(); let session_config = SessionConfig::new().with_batch_size(50); let ctx = SessionContext::new_with_config(session_config); - let mut sort_keys = vec![]; + let mut sort_keys = LexOrdering::default(); for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).unwrap(), diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 61b4e32ad6c9..5bfb4d97ed70 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -47,6 +47,7 @@ use test_utils::add_empty_batches; use datafusion::functions_window::row_number::row_number_udwf; use datafusion_functions_window::lead_lag::{lag_udwf, lead_udwf}; use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::HashMap; use rand::distributions::Alphanumeric; use rand::rngs::StdRng; @@ -251,7 +252,7 @@ async fn bounded_window_causal_non_causal() -> Result<()> { ]; let partitionby_exprs = vec![]; - let orderby_exprs = vec![]; + let orderby_exprs = LexOrdering::default(); // Window frame starts with "UNBOUNDED PRECEDING": let start_bound = WindowFrameBound::Preceding(ScalarValue::UInt64(None)); @@ -284,7 +285,7 @@ async fn bounded_window_causal_non_causal() -> Result<()> { fn_name.to_string(), &args, &partitionby_exprs, - &orderby_exprs, + orderby_exprs.as_ref(), Arc::new(window_frame), &extended_schema, false, @@ -599,7 +600,7 @@ async fn run_window_test( let ctx = SessionContext::new_with_config(session_config); let (window_fn, args, fn_name) = get_random_function(&schema, &mut rng, is_linear); let window_frame = get_random_window_frame(&mut rng, is_linear); - let mut orderby_exprs = vec![]; + let mut orderby_exprs = LexOrdering::default(); for column in &orderby_columns { orderby_exprs.push(PhysicalSortExpr { expr: col(column, &schema)?, @@ -607,27 +608,27 @@ async fn run_window_test( }) } if orderby_exprs.len() > 1 && !window_frame.can_accept_multi_orderby() { - orderby_exprs = orderby_exprs[0..1].to_vec(); + orderby_exprs = LexOrdering::new(orderby_exprs[0..1].to_vec()); } let mut partitionby_exprs = vec![]; for column in &partition_by_columns { partitionby_exprs.push(col(column, &schema)?); } - let mut sort_keys = vec![]; + let mut sort_keys = LexOrdering::default(); for partition_by_expr in &partitionby_exprs { sort_keys.push(PhysicalSortExpr { expr: partition_by_expr.clone(), options: SortOptions::default(), }) } - for order_by_expr in &orderby_exprs { + for order_by_expr in &orderby_exprs.inner { if !sort_keys.contains(order_by_expr) { sort_keys.push(order_by_expr.clone()) } } let concat_input_record = concat_batches(&schema, &input1)?; - let source_sort_keys = vec![ + let source_sort_keys = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: Default::default(), @@ -640,7 +641,7 @@ async fn run_window_test( expr: col("c", &schema)?, options: Default::default(), }, - ]; + ]); let mut exec1 = Arc::new( MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None)? .try_with_sort_information(vec![source_sort_keys.clone()])?, @@ -659,7 +660,7 @@ async fn run_window_test( fn_name.clone(), &args, &partitionby_exprs, - &orderby_exprs, + orderby_exprs.as_ref(), Arc::new(window_frame.clone()), &extended_schema, false, @@ -677,7 +678,7 @@ async fn run_window_test( fn_name, &args, &partitionby_exprs, - &orderby_exprs, + orderby_exprs.as_ref(), Arc::new(window_frame.clone()), &extended_schema, false, diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index fc2fb9afb5f9..6817969580da 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -238,15 +238,15 @@ async fn sort_preserving_merge() { // SortPreservingMergeExec (not a Sort which would compete // with the SortPreservingMergeExec for memory) &[ - "+---------------+-----------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+-----------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b ASC NULLS LAST, fetch=10 |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 |", - "| | MemoryExec: partitions=2, partition_sizes=[5, 5], output_ordering=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST |", - "| | |", - "+---------------+-----------------------------------------------------------------------------------------------------------+", + "+---------------+------------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+------------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b ASC NULLS LAST, fetch=10 |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 |", + "| | MemoryExec: partitions=2, partition_sizes=[5, 5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+------------------------------------------------------------------------------------------------------------+", ] ) .run() @@ -281,15 +281,15 @@ async fn sort_spill_reservation() { // also merge, so we can ensure the sort could finish // given enough merging memory &[ - "+---------------+--------------------------------------------------------------------------------------------------------+", - "| plan_type | plan |", - "+---------------+--------------------------------------------------------------------------------------------------------+", - "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", - "| | TableScan: t projection=[a, b] |", - "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC], preserve_partitioning=[false] |", - "| | MemoryExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST |", - "| | |", - "+---------------+--------------------------------------------------------------------------------------------------------+", + "+---------------+---------------------------------------------------------------------------------------------------------+", + "| plan_type | plan |", + "+---------------+---------------------------------------------------------------------------------------------------------+", + "| logical_plan | Sort: t.a ASC NULLS LAST, t.b DESC NULLS FIRST |", + "| | TableScan: t projection=[a, b] |", + "| physical_plan | SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] |", + "| | MemoryExec: partitions=1, partition_sizes=[5], output_ordering=a@0 ASC NULLS LAST, b@1 ASC NULLS LAST |", + "| | |", + "+---------------+---------------------------------------------------------------------------------------------------------+", ] ); @@ -654,7 +654,7 @@ impl Scenario { descending: false, nulls_first: false, }; - let sort_information = vec![vec![ + let sort_information = vec![LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema).unwrap(), options, @@ -663,7 +663,7 @@ impl Scenario { expr: col("b", &schema).unwrap(), options, }, - ]]; + ])]; let table = SortedTableProvider::new(batches, sort_information); Arc::new(table) diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 6859e2f1468c..6910db6285a3 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -37,6 +37,7 @@ use datafusion_physical_expr::{ expressions::{cast, col}, PhysicalExpr, PhysicalSortExpr, }; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::{ limited_distinct_aggregation::LimitedDistinctAggregation, PhysicalOptimizerRule, }; @@ -407,10 +408,10 @@ fn test_has_filter() -> Result<()> { #[test] fn test_has_order_by() -> Result<()> { - let sort_key = vec![PhysicalSortExpr { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema()).unwrap(), options: SortOptions::default(), - }]; + }]); let source = parquet_exec_with_sort(vec![sort_key]); let schema = source.schema(); diff --git a/datafusion/core/tests/physical_optimizer/test_util.rs b/datafusion/core/tests/physical_optimizer/test_util.rs index 131b887c4ec7..12cd08fb3db3 100644 --- a/datafusion/core/tests/physical_optimizer/test_util.rs +++ b/datafusion/core/tests/physical_optimizer/test_util.rs @@ -25,11 +25,11 @@ use datafusion::datasource::{ physical_plan::{FileScanConfig, ParquetExec}, }; use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec>, + output_ordering: Vec, ) -> Arc { ParquetExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index ddf0085b9de4..67ada562800b 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -18,9 +18,8 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; use datafusion_expr_common::accumulator::Accumulator; -use datafusion_physical_expr_common::{ - physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, -}; +use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use std::sync::Arc; /// [`AccumulatorArgs`] contains information about how an aggregate @@ -53,7 +52,7 @@ pub struct AccumulatorArgs<'a> { /// ``` /// /// If no `ORDER BY` is specified, `ordering_req` will be empty. - pub ordering_req: &'a [PhysicalSortExpr], + pub ordering_req: LexOrderingRef<'a>, /// Whether the aggregation is running in reverse order pub is_reversed: bool, diff --git a/datafusion/functions-aggregate-common/src/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs index 4fba772d8ddc..f55e5ec9a41d 100644 --- a/datafusion/functions-aggregate-common/src/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -30,7 +30,7 @@ use arrow::{ }; use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr_common::accumulator::Accumulator; -use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; /// Convert scalar values from an accumulator into arrays. pub fn get_accum_scalar_values_as_arrays( @@ -88,7 +88,7 @@ pub fn adjust_output_array(data_type: &DataType, array: ArrayRef) -> Result Vec { @@ -107,7 +107,7 @@ pub fn ordering_fields( } /// Selects the sort option attribute from all the given `PhysicalSortExpr`s. -pub fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { +pub fn get_sort_options(ordering_req: LexOrderingRef) -> Vec { ordering_req.iter().map(|item| item.options).collect() } diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 65956cb8a1de..1c8266ed5b89 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -23,6 +23,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::count::Count; use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use std::sync::Arc; fn prepare_accumulator() -> Box { @@ -31,7 +32,7 @@ fn prepare_accumulator() -> Box { return_type: &DataType::Int64, schema: &schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: LexOrderingRef::default(), is_reversed: false, name: "COUNT(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index 652d447129dc..1e9493280ed2 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -23,6 +23,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::sum::Sum; use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use std::sync::Arc; fn prepare_accumulator(data_type: &DataType) -> Box { @@ -31,7 +32,7 @@ fn prepare_accumulator(data_type: &DataType) -> Box { return_type: data_type, schema: &schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: LexOrderingRef::default(), is_reversed: false, name: "SUM(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index b3e04c5584ef..7c22c21e38c9 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -135,7 +135,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &data_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + LexOrdering::from_ref(acc_args.ordering_req), acc_args.is_reversed, ) .map(|acc| Box::new(acc) as _) @@ -511,7 +511,7 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { impl OrderSensitiveArrayAggAccumulator { fn evaluate_orderings(&self) -> Result { - let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); + let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); let num_columns = fields.len(); let struct_field = Fields::from(fields.clone()); diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index da3fc62f8c8c..0b05713499a9 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -37,7 +37,7 @@ use datafusion_expr::{ ExprFunctionExt, Signature, SortExpr, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::utils::get_sort_options; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; create_func!(FirstValue, first_value_udaf); @@ -130,7 +130,7 @@ impl AggregateUDFImpl for FirstValue { FirstValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + LexOrdering::from_ref(acc_args.ordering_req), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) @@ -315,7 +315,7 @@ impl Accumulator for FirstValueAccumulator { if compare_rows( &self.orderings, orderings, - &get_sort_options(&self.ordering_req), + &get_sort_options(self.ordering_req.as_ref()), )? .is_gt() { @@ -333,8 +333,10 @@ impl Accumulator for FirstValueAccumulator { let flags = states[is_set_idx].as_boolean(); let filtered_states = filter_states_according_to_is_set(states, flags)?; // 1..is_set_idx range corresponds to ordering section - let sort_cols = - convert_to_sort_cols(&filtered_states[1..is_set_idx], &self.ordering_req); + let sort_cols = convert_to_sort_cols( + &filtered_states[1..is_set_idx], + self.ordering_req.as_ref(), + ); let ordered_states = if sort_cols.is_empty() { // When no ordering is given, use the existing state as is: @@ -347,7 +349,7 @@ impl Accumulator for FirstValueAccumulator { let first_row = get_row_at_idx(&ordered_states, 0)?; // When collecting orderings, we exclude the is_set flag from the state. let first_ordering = &first_row[1..is_set_idx]; - let sort_options = get_sort_options(&self.ordering_req); + let sort_options = get_sort_options(self.ordering_req.as_ref()); // Either there is no existing value, or there is an earlier version in new data. if !self.is_set || compare_rows(&self.orderings, first_ordering, &sort_options)?.is_gt() @@ -453,7 +455,7 @@ impl AggregateUDFImpl for LastValue { LastValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + LexOrdering::from_ref(acc_args.ordering_req), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) @@ -645,7 +647,7 @@ impl Accumulator for LastValueAccumulator { if compare_rows( &self.orderings, orderings, - &get_sort_options(&self.ordering_req), + &get_sort_options(self.ordering_req.as_ref()), )? .is_lt() { @@ -663,8 +665,10 @@ impl Accumulator for LastValueAccumulator { let flags = states[is_set_idx].as_boolean(); let filtered_states = filter_states_according_to_is_set(states, flags)?; // 1..is_set_idx range corresponds to ordering section - let sort_cols = - convert_to_sort_cols(&filtered_states[1..is_set_idx], &self.ordering_req); + let sort_cols = convert_to_sort_cols( + &filtered_states[1..is_set_idx], + self.ordering_req.as_ref(), + ); let ordered_states = if sort_cols.is_empty() { // When no ordering is given, use existing state as is: @@ -679,7 +683,7 @@ impl Accumulator for LastValueAccumulator { let last_row = get_row_at_idx(&ordered_states, last_idx)?; // When collecting orderings, we exclude the is_set flag from the state. let last_ordering = &last_row[1..is_set_idx]; - let sort_options = get_sort_options(&self.ordering_req); + let sort_options = get_sort_options(self.ordering_req.as_ref()); // Either there is no existing value, or there is a newer (latest) // version in the new data: if !self.is_set @@ -721,7 +725,7 @@ fn filter_states_according_to_is_set( /// Combines array refs and their corresponding orderings to construct `SortColumn`s. fn convert_to_sort_cols( arrs: &[ArrayRef], - sort_exprs: &[PhysicalSortExpr], + sort_exprs: LexOrderingRef, ) -> Vec { arrs.iter() .zip(sort_exprs.iter()) @@ -740,10 +744,18 @@ mod tests { #[test] fn test_first_last_value_value() -> Result<()> { - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; - let mut last_accumulator = - LastValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; // first value in the tuple is start of the range (inclusive), // second value in the tuple is end of the range (exclusive) let ranges: Vec<(i64, i64)> = vec![(0, 10), (1, 11), (2, 13)]; @@ -780,14 +792,22 @@ mod tests { .collect::>(); // FirstValueAccumulator - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; first_accumulator.update_batch(&[Arc::clone(&arrs[0])])?; let state1 = first_accumulator.state()?; - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; first_accumulator.update_batch(&[Arc::clone(&arrs[1])])?; let state2 = first_accumulator.state()?; @@ -802,22 +822,34 @@ mod tests { ])?); } - let mut first_accumulator = - FirstValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut first_accumulator = FirstValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; first_accumulator.merge_batch(&states)?; let merged_state = first_accumulator.state()?; assert_eq!(merged_state.len(), state1.len()); // LastValueAccumulator - let mut last_accumulator = - LastValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; last_accumulator.update_batch(&[Arc::clone(&arrs[0])])?; let state1 = last_accumulator.state()?; - let mut last_accumulator = - LastValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; last_accumulator.update_batch(&[Arc::clone(&arrs[1])])?; let state2 = last_accumulator.state()?; @@ -832,8 +864,12 @@ mod tests { ])?); } - let mut last_accumulator = - LastValueAccumulator::try_new(&DataType::Int64, &[], vec![], false)?; + let mut last_accumulator = LastValueAccumulator::try_new( + &DataType::Int64, + &[], + LexOrdering::default(), + false, + )?; last_accumulator.merge_batch(&states)?; let merged_state = last_accumulator.state()?; diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 2a1778d8b232..5f3a8cf2f161 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -133,7 +133,7 @@ impl AggregateUDFImpl for NthValueAgg { n, &data_type, &ordering_dtypes, - acc_args.ordering_req.to_vec(), + LexOrdering::from_ref(acc_args.ordering_req), ) .map(|acc| Box::new(acc) as _) } @@ -403,7 +403,7 @@ impl Accumulator for NthValueAccumulator { impl NthValueAccumulator { fn evaluate_orderings(&self) -> Result { - let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); + let fields = ordering_fields(self.ordering_req.as_ref(), &self.datatypes[1..]); let struct_field = Fields::from(fields.clone()); let mut column_wise_ordering_values = vec![]; diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 355d1d5ad2db..95269ed8217c 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -410,6 +410,7 @@ mod tests { use datafusion_expr::AggregateUDF; use datafusion_functions_aggregate_common::utils::get_accum_scalar_values_as_arrays; use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use std::sync::Arc; #[test] @@ -461,7 +462,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: LexOrderingRef::default(), name: "a", is_distinct: false, is_reversed: false, @@ -472,7 +473,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: &[], + ordering_req: LexOrderingRef::default(), name: "a", is_distinct: false, is_reversed: false, diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index d825bfe7e264..addf2fbfca0c 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -17,12 +17,13 @@ //! Sort expressions +use crate::physical_expr::PhysicalExpr; +use std::fmt; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; -use std::ops::Deref; +use std::ops::{Deref, Index, Range, RangeFrom, RangeTo}; use std::sync::Arc; - -use crate::physical_expr::PhysicalExpr; +use std::vec::IntoIter; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; @@ -143,7 +144,7 @@ impl Hash for PhysicalSortExpr { } impl Display for PhysicalSortExpr { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { write!(f, "{} {}", self.expr, to_str(&self.options)) } } @@ -183,26 +184,6 @@ impl PhysicalSortExpr { .map_or(true, |opts| self.options.descending == opts.descending) } } - - /// Returns a [`Display`]able list of `PhysicalSortExpr`. - pub fn format_list(input: &[PhysicalSortExpr]) -> impl Display + '_ { - struct DisplayableList<'a>(&'a [PhysicalSortExpr]); - impl<'a> Display for DisplayableList<'a> { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - let mut first = true; - for sort_expr in self.0 { - if first { - first = false; - } else { - write!(f, ",")?; - } - write!(f, "{}", sort_expr)?; - } - Ok(()) - } - } - DisplayableList(input) - } } /// Represents sort requirement associated with a plan @@ -260,7 +241,7 @@ impl PartialEq for PhysicalSortRequirement { } impl Display for PhysicalSortRequirement { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { let opts_string = self.options.as_ref().map_or("NA", to_str); write!(f, "{} {}", self.expr, opts_string) } @@ -274,7 +255,7 @@ pub fn format_physical_sort_requirement_list( ) -> impl Display + '_ { struct DisplayWrapper<'a>(&'a [PhysicalSortRequirement]); impl<'a> Display for DisplayWrapper<'a> { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { let mut iter = self.0.iter(); write!(f, "[")?; if let Some(expr) = iter.next() { @@ -345,7 +326,7 @@ impl PhysicalSortRequirement { /// default ordering `ASC, NULLS LAST` if given (see the `PhysicalSortExpr::from`). pub fn to_sort_exprs( requirements: impl IntoIterator, - ) -> Vec { + ) -> LexOrdering { requirements .into_iter() .map(PhysicalSortExpr::from) @@ -364,9 +345,147 @@ fn to_str(options: &SortOptions) -> &str { } } -///`LexOrdering` is an alias for the type `Vec`, which represents +///`LexOrdering` contains a `Vec`, which represents /// a lexicographical ordering. -pub type LexOrdering = Vec; +#[derive(Debug, Default, Clone, PartialEq, Eq, Hash)] +pub struct LexOrdering { + pub inner: Vec, +} + +impl LexOrdering { + // Creates a new [`LexOrdering`] from a vector + pub fn new(inner: Vec) -> Self { + Self { inner } + } + + pub fn as_ref(&self) -> LexOrderingRef { + &self.inner + } + + pub fn capacity(&self) -> usize { + self.inner.capacity() + } + + pub fn clear(&mut self) { + self.inner.clear() + } + + pub fn contains(&self, expr: &PhysicalSortExpr) -> bool { + self.inner.contains(expr) + } + + pub fn extend>(&mut self, iter: I) { + self.inner.extend(iter) + } + + pub fn from_ref(lex_ordering_ref: LexOrderingRef) -> Self { + Self::new(lex_ordering_ref.to_vec()) + } + + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + + pub fn iter(&self) -> impl Iterator { + self.inner.iter() + } + + pub fn len(&self) -> usize { + self.inner.len() + } + + pub fn pop(&mut self) -> Option { + self.inner.pop() + } + + pub fn push(&mut self, physical_sort_expr: PhysicalSortExpr) { + self.inner.push(physical_sort_expr) + } + + pub fn retain(&mut self, f: impl FnMut(&PhysicalSortExpr) -> bool) { + self.inner.retain(f) + } + + pub fn truncate(&mut self, len: usize) { + self.inner.truncate(len) + } +} + +impl Deref for LexOrdering { + type Target = [PhysicalSortExpr]; + + fn deref(&self) -> &Self::Target { + self.inner.as_slice() + } +} + +impl Display for LexOrdering { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + let mut first = true; + for sort_expr in &self.inner { + if first { + first = false; + } else { + write!(f, ", ")?; + } + write!(f, "{}", sort_expr)?; + } + Ok(()) + } +} + +impl FromIterator for LexOrdering { + fn from_iter>(iter: T) -> Self { + let mut lex_ordering = LexOrdering::default(); + + for i in iter { + lex_ordering.push(i); + } + + lex_ordering + } +} + +impl Index for LexOrdering { + type Output = PhysicalSortExpr; + + fn index(&self, index: usize) -> &Self::Output { + &self.inner[index] + } +} + +impl Index> for LexOrdering { + type Output = [PhysicalSortExpr]; + + fn index(&self, range: Range) -> &Self::Output { + &self.inner[range] + } +} + +impl Index> for LexOrdering { + type Output = [PhysicalSortExpr]; + + fn index(&self, range_from: RangeFrom) -> &Self::Output { + &self.inner[range_from] + } +} + +impl Index> for LexOrdering { + type Output = [PhysicalSortExpr]; + + fn index(&self, range_to: RangeTo) -> &Self::Output { + &self.inner[range_to] + } +} + +impl IntoIterator for LexOrdering { + type Item = PhysicalSortExpr; + type IntoIter = IntoIter; + + fn into_iter(self) -> Self::IntoIter { + self.inner.into_iter() + } +} ///`LexOrderingRef` is an alias for the type &`[PhysicalSortExpr]`, which represents /// a reference to a lexicographical ordering. @@ -384,6 +503,10 @@ impl LexRequirement { Self { inner } } + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + pub fn iter(&self) -> impl Iterator { self.inner.iter() } @@ -415,7 +538,7 @@ impl FromIterator for LexRequirement { impl IntoIterator for LexRequirement { type Item = PhysicalSortRequirement; - type IntoIter = std::vec::IntoIter; + type IntoIter = IntoIter; fn into_iter(self) -> Self::IntoIter { self.inner.into_iter() diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index d2c9bf1a2408..26293b1a76a2 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -24,7 +24,7 @@ use datafusion_common::Result; use datafusion_expr_common::sort_properties::ExprProperties; use crate::physical_expr::PhysicalExpr; -use crate::sort_expr::PhysicalSortExpr; +use crate::sort_expr::{LexOrdering, LexOrderingRef, PhysicalSortExpr}; use crate::tree_node::ExprContext; /// Represents a [`PhysicalExpr`] node with associated properties (order and @@ -96,7 +96,7 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { /// Reverses the ORDER BY expression, which is useful during equivalent window /// expression construction. For instance, 'ORDER BY a ASC, NULLS LAST' turns into /// 'ORDER BY a DESC, NULLS FIRST'. -pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec { +pub fn reverse_order_bys(order_bys: LexOrderingRef) -> LexOrdering { order_bys .iter() .map(|e| PhysicalSortExpr::new(e.expr.clone(), !e.options)) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 6330c240241a..e446776affc0 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -45,7 +45,7 @@ use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; use datafusion_physical_expr_common::utils::reverse_order_bys; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; @@ -81,7 +81,7 @@ impl AggregateExprBuilder { args, alias: None, schema: Arc::new(Schema::empty()), - ordering_req: vec![], + ordering_req: LexOrdering::default(), ignore_nulls: false, is_distinct: false, is_reversed: false, @@ -111,7 +111,8 @@ impl AggregateExprBuilder { .map(|e| e.expr.data_type(&schema)) .collect::>>()?; - ordering_fields = utils::ordering_fields(&ordering_req, &ordering_types); + ordering_fields = + utils::ordering_fields(ordering_req.as_ref(), &ordering_types); } let input_exprs_types = args @@ -265,7 +266,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: &self.ordering_req, + ordering_req: self.ordering_req.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -291,13 +292,13 @@ impl AggregateFunctionExpr { /// Order by requirements for the aggregate function /// By default it is `None` (there is no requirement) /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this - pub fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { + pub fn order_bys(&self) -> Option { if self.ordering_req.is_empty() { return None; } if !self.order_sensitivity().is_insensitive() { - return Some(&self.ordering_req); + return Some(self.ordering_req.as_ref()); } None @@ -340,7 +341,7 @@ impl AggregateFunctionExpr { }; AggregateExprBuilder::new(Arc::new(updated_fn), self.args.to_vec()) - .order_by(self.ordering_req.to_vec()) + .order_by(self.ordering_req.clone()) .schema(Arc::new(self.schema.clone())) .alias(self.name().to_string()) .with_ignore_nulls(self.ignore_nulls) @@ -356,7 +357,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: &self.ordering_req, + ordering_req: self.ordering_req.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -425,7 +426,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: &self.ordering_req, + ordering_req: self.ordering_req.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -444,7 +445,7 @@ impl AggregateFunctionExpr { return_type: &self.data_type, schema: &self.schema, ignore_nulls: self.ignore_nulls, - ordering_req: &self.ordering_req, + ordering_req: self.ordering_req.as_ref(), is_distinct: self.is_distinct, name: &self.name, is_reversed: self.is_reversed, @@ -462,7 +463,7 @@ impl AggregateFunctionExpr { ReversedUDAF::NotSupported => None, ReversedUDAF::Identical => Some(self.clone()), ReversedUDAF::Reversed(reverse_udf) => { - let reverse_ordering_req = reverse_order_bys(&self.ordering_req); + let reverse_ordering_req = reverse_order_bys(self.ordering_req.as_ref()); let mut name = self.name().to_string(); // If the function is changed, we need to reverse order_by clause as well // i.e. First(a order by b asc null first) -> Last(a order by b desc null last) @@ -473,7 +474,7 @@ impl AggregateFunctionExpr { replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); AggregateExprBuilder::new(reverse_udf, self.args.to_vec()) - .order_by(reverse_ordering_req.to_vec()) + .order_by(reverse_ordering_req) .schema(Arc::new(self.schema.clone())) .alias(name) .with_ignore_nulls(self.ignore_nulls) @@ -489,7 +490,7 @@ impl AggregateFunctionExpr { /// These expressions are (1)function arguments, (2) order by expressions. pub fn all_expressions(&self) -> AggregatePhysicalExpressions { let args = self.expressions(); - let order_bys = self.order_bys().unwrap_or(&[]); + let order_bys = self.order_bys().unwrap_or_default(); let order_by_exprs = order_bys .iter() .map(|sort_expr| Arc::clone(&sort_expr.expr)) diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index 95bb93d6ca57..902e53a7f236 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -80,6 +80,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use arrow_schema::{SchemaRef, SortOptions}; use datafusion_common::{plan_datafusion_err, Result}; + use datafusion_physical_expr_common::sort_expr::LexOrdering; pub fn output_schema( mapping: &ProjectionMapping, @@ -184,7 +185,7 @@ mod tests { // Convert each tuple to PhysicalSortExpr pub fn convert_to_sort_exprs( in_data: &[(&Arc, SortOptions)], - ) -> Vec { + ) -> LexOrdering { in_data .iter() .map(|(expr, options)| PhysicalSortExpr { @@ -197,7 +198,7 @@ mod tests { // Convert each inner tuple to PhysicalSortExpr pub fn convert_to_orderings( orderings: &[Vec<(&Arc, SortOptions)>], - ) -> Vec> { + ) -> Vec { orderings .iter() .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) @@ -207,20 +208,22 @@ mod tests { // Convert each tuple to PhysicalSortExpr pub fn convert_to_sort_exprs_owned( in_data: &[(Arc, SortOptions)], - ) -> Vec { - in_data - .iter() - .map(|(expr, options)| PhysicalSortExpr { - expr: Arc::clone(expr), - options: *options, - }) - .collect() + ) -> LexOrdering { + LexOrdering::new( + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: Arc::clone(expr), + options: *options, + }) + .collect(), + ) } // Convert each inner tuple to PhysicalSortExpr pub fn convert_to_orderings_owned( orderings: &[Vec<(Arc, SortOptions)>], - ) -> Vec> { + ) -> Vec { orderings .iter() .map(|sort_exprs| convert_to_sort_exprs_owned(sort_exprs)) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index d71f3b037fb1..838c9800f942 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -21,7 +21,7 @@ use std::sync::Arc; use std::vec::IntoIter; use crate::equivalence::add_offset_to_expr; -use crate::{LexOrdering, PhysicalExpr, PhysicalSortExpr}; +use crate::{LexOrdering, PhysicalExpr}; use arrow_schema::SortOptions; /// An `OrderingEquivalenceClass` object keeps track of different alternative @@ -146,7 +146,12 @@ impl OrderingEquivalenceClass { /// Returns the concatenation of all the orderings. This enables merge /// operations to preserve all equivalent orderings simultaneously. pub fn output_ordering(&self) -> Option { - let output_ordering = self.orderings.iter().flatten().cloned().collect(); + let output_ordering = self + .orderings + .iter() + .flat_map(|ordering| ordering.as_ref()) + .cloned() + .collect(); let output_ordering = collapse_lex_ordering(output_ordering); (!output_ordering.is_empty()).then_some(output_ordering) } @@ -169,7 +174,7 @@ impl OrderingEquivalenceClass { for idx in 0..n_ordering { // Calculate cross product index let idx = outer_idx * n_ordering + idx; - self.orderings[idx].extend(ordering.iter().cloned()); + self.orderings[idx].inner.extend(ordering.iter().cloned()); } } self @@ -179,7 +184,7 @@ impl OrderingEquivalenceClass { /// ordering equivalence class. pub fn add_offset(&mut self, offset: usize) { for ordering in self.orderings.iter_mut() { - for sort_expr in ordering { + for sort_expr in ordering.inner.iter_mut() { sort_expr.expr = add_offset_to_expr(Arc::clone(&sort_expr.expr), offset); } } @@ -211,10 +216,10 @@ impl IntoIterator for OrderingEquivalenceClass { /// duplicate entries that have same physical expression inside. For example, /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { - let mut output = Vec::::new(); - for item in input { + let mut output = LexOrdering::default(); + for item in input.iter() { if !output.iter().any(|req| req.expr.eq(&item.expr)) { - output.push(item); + output.push(item.clone()); } } output @@ -239,10 +244,10 @@ impl Display for OrderingEquivalenceClass { write!(f, "[")?; let mut iter = self.orderings.iter(); if let Some(ordering) = iter.next() { - write!(f, "[{}]", PhysicalSortExpr::format_list(ordering))?; + write!(f, "[{}]", ordering)?; } for ordering in iter { - write!(f, ", [{}]", PhysicalSortExpr::format_list(ordering))?; + write!(f, ", [{}]", ordering)?; } write!(f, "]")?; Ok(()) @@ -268,6 +273,7 @@ mod tests { use arrow_schema::SortOptions; use datafusion_common::{DFSchema, Result}; use datafusion_expr::{Operator, ScalarUDF}; + use datafusion_physical_expr_common::sort_expr::LexOrdering; #[test] fn test_ordering_satisfy() -> Result<()> { @@ -275,11 +281,11 @@ mod tests { Field::new("a", DataType::Int64, true), Field::new("b", DataType::Int64, true), ])); - let crude = vec![PhysicalSortExpr { + let crude = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), - }]; - let finer = vec![ + }]); + let finer = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options: SortOptions::default(), @@ -288,18 +294,18 @@ mod tests { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), }, - ]; + ]); // finer ordering satisfies, crude ordering should return true let mut eq_properties_finer = EquivalenceProperties::new(Arc::clone(&input_schema)); eq_properties_finer.oeq_class.push(finer.clone()); - assert!(eq_properties_finer.ordering_satisfy(&crude)); + assert!(eq_properties_finer.ordering_satisfy(crude.as_ref())); // Crude ordering doesn't satisfy finer ordering. should return false let mut eq_properties_crude = EquivalenceProperties::new(Arc::clone(&input_schema)); eq_properties_crude.oeq_class.push(crude); - assert!(!eq_properties_crude.ordering_satisfy(&finer)); + assert!(!eq_properties_crude.ordering_satisfy(finer.as_ref())); Ok(()) } @@ -589,7 +595,7 @@ mod tests { let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(&reqs), + eq_properties.ordering_satisfy(reqs.as_ref()), expected, "{}", err_msg @@ -649,7 +655,7 @@ mod tests { format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - eq_properties.ordering_satisfy(&reqs), + eq_properties.ordering_satisfy(reqs.as_ref()), expected, "{}", err_msg diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 9a16b205ae25..55c99e93d040 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -103,7 +103,7 @@ use itertools::Itertools; /// # use arrow_schema::{Schema, Field, DataType, SchemaRef}; /// # use datafusion_physical_expr::{ConstExpr, EquivalenceProperties}; /// # use datafusion_physical_expr::expressions::col; -/// use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; +/// use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; /// # let schema: SchemaRef = Arc::new(Schema::new(vec![ /// # Field::new("a", DataType::Int32, false), /// # Field::new("b", DataType::Int32, false), @@ -116,12 +116,12 @@ use itertools::Itertools; /// // with a single constant value of b /// let mut eq_properties = EquivalenceProperties::new(schema) /// .with_constants(vec![ConstExpr::from(col_b)]); -/// eq_properties.add_new_ordering(vec![ +/// eq_properties.add_new_ordering(LexOrdering::new(vec![ /// PhysicalSortExpr::new_default(col_a).asc(), /// PhysicalSortExpr::new_default(col_c).desc(), -/// ]); +/// ])); /// -/// assert_eq!(eq_properties.to_string(), "order: [[a@0 ASC,c@2 DESC]], const: [b@1]") +/// assert_eq!(eq_properties.to_string(), "order: [[a@0 ASC, c@2 DESC]], const: [b@1]") /// ``` #[derive(Debug, Clone)] pub struct EquivalenceProperties { @@ -185,6 +185,7 @@ impl EquivalenceProperties { let mut output_ordering = self.oeq_class().output_ordering().unwrap_or_default(); // Prune out constant expressions output_ordering + .inner .retain(|sort_expr| !const_exprs_contains(constants, &sort_expr.expr)); (!output_ordering.is_empty()).then_some(output_ordering) } @@ -196,7 +197,7 @@ impl EquivalenceProperties { OrderingEquivalenceClass::new( self.oeq_class .iter() - .map(|ordering| self.normalize_sort_exprs(ordering)) + .map(|ordering| self.normalize_sort_exprs(ordering.as_ref())) .collect(), ) } @@ -351,7 +352,7 @@ impl EquivalenceProperties { .iter() .filter(|ordering| ordering[0].expr.eq(&normalized_expr)) // First expression after leading ordering - .filter_map(|ordering| Some(ordering).zip(ordering.get(1))) + .filter_map(|ordering| Some(ordering).zip(ordering.inner.get(1))) { let leading_ordering = ordering[0].options; // Currently, we only handle expressions with a single child. @@ -378,7 +379,7 @@ impl EquivalenceProperties { // then we can deduce that ordering `[b ASC]` is also valid. // Hence, ordering `[b ASC]` can be added to the state as valid ordering. // (e.g. existing ordering where leading ordering is removed) - new_orderings.push(ordering[1..].to_vec()); + new_orderings.push(LexOrdering::new(ordering[1..].to_vec())); break; } } @@ -391,7 +392,7 @@ impl EquivalenceProperties { /// Updates the ordering equivalence group within assuming that the table /// is re-sorted according to the argument `sort_exprs`. Note that constants /// and equivalence classes are unchanged as they are unaffected by a re-sort. - pub fn with_reorder(mut self, sort_exprs: Vec) -> Self { + pub fn with_reorder(mut self, sort_exprs: LexOrdering) -> Self { // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. self.oeq_class = OrderingEquivalenceClass::new(vec![sort_exprs]); self @@ -605,8 +606,8 @@ impl EquivalenceProperties { pub fn substitute_ordering_component( &self, mapping: &ProjectionMapping, - sort_expr: &[PhysicalSortExpr], - ) -> Result>> { + sort_expr: LexOrderingRef, + ) -> Result> { let new_orderings = sort_expr .iter() .map(|sort_expr| { @@ -616,7 +617,7 @@ impl EquivalenceProperties { .filter(|source| expr_refers(source, &sort_expr.expr)) .cloned() .collect(); - let mut res = vec![sort_expr.clone()]; + let mut res = LexOrdering::new(vec![sort_expr.clone()]); // TODO: Add one-to-ones analysis for ScalarFunctions. for r_expr in referring_exprs { // we check whether this expression is substitutable or not @@ -639,7 +640,9 @@ impl EquivalenceProperties { // Generate all valid orderings, given substituted expressions. let res = new_orderings .into_iter() + .map(|ordering| ordering.inner) .multi_cartesian_product() + .map(LexOrdering::new) .collect::>(); Ok(res) } @@ -653,7 +656,7 @@ impl EquivalenceProperties { let orderings = &self.oeq_class.orderings; let new_order = orderings .iter() - .map(|order| self.substitute_ordering_component(mapping, order)) + .map(|order| self.substitute_ordering_component(mapping, order.as_ref())) .collect::>>()?; let new_order = new_order.into_iter().flatten().collect(); self.oeq_class = OrderingEquivalenceClass::new(new_order); @@ -836,7 +839,7 @@ impl EquivalenceProperties { if prefixes.is_empty() { // If prefix is empty, there is no dependency. Insert // empty ordering: - prefixes = vec![vec![]]; + prefixes = vec![LexOrdering::default()]; } // Append current ordering on top its dependencies: for ordering in prefixes.iter_mut() { @@ -986,7 +989,8 @@ impl EquivalenceProperties { // Add new ordered section to the state. result.extend(ordered_exprs); } - result.into_iter().unzip() + let (left, right) = result.into_iter().unzip(); + (LexOrdering::new(left), right) } /// This function determines whether the provided expression is constant @@ -1076,6 +1080,7 @@ impl EquivalenceProperties { let mut new_orderings = vec![]; for ordering in self.oeq_class.orderings { let new_ordering = ordering + .inner .into_iter() .map(|mut sort_expr| { sort_expr.expr = with_new_schema(sort_expr.expr, &schema)?; @@ -1313,7 +1318,7 @@ fn construct_prefix_orderings( /// Generates all possible orderings where dependencies are satisfied for the /// current projection expression. /// -/// # Examaple +/// # Example /// If `dependences` is `a + b ASC` and the dependency map holds dependencies /// * `a ASC` --> `[c ASC]` /// * `b ASC` --> `[d DESC]`, @@ -1348,7 +1353,7 @@ fn generate_dependency_orderings( // No dependency, dependent is a leading ordering. if relevant_prefixes.is_empty() { // Return an empty ordering: - return vec![vec![]]; + return vec![LexOrdering::default()]; } relevant_prefixes @@ -1358,7 +1363,12 @@ fn generate_dependency_orderings( prefix_orderings .iter() .permutations(prefix_orderings.len()) - .map(|prefixes| prefixes.into_iter().flatten().cloned().collect()) + .map(|prefixes| { + prefixes + .into_iter() + .flat_map(|ordering| ordering.inner.clone()) + .collect() + }) .collect::>() }) .collect() @@ -1651,7 +1661,7 @@ impl<'a> DependencyEnumerator<'a> { // An empty dependency means the referred_sort_expr represents a global ordering. // Return its projected version, which is the target_expression. if node.dependencies.is_empty() { - return vec![vec![target_sort_expr.clone()]]; + return vec![LexOrdering::new(vec![target_sort_expr.clone()])]; }; node.dependencies @@ -1961,7 +1971,7 @@ impl UnionEquivalentOrderingBuilder { ) -> AddedOrdering { if ordering.is_empty() { AddedOrdering::Yes - } else if constants.is_empty() && properties.ordering_satisfy(&ordering) { + } else if constants.is_empty() && properties.ordering_satisfy(ordering.as_ref()) { // If the ordering satisfies the target properties, no need to // augment it with constants. self.orderings.push(ordering); @@ -2002,7 +2012,7 @@ impl UnionEquivalentOrderingBuilder { &properties.constants, ) { if !augmented_ordering.is_empty() { - assert!(properties.ordering_satisfy(&augmented_ordering)); + assert!(properties.ordering_satisfy(augmented_ordering.as_ref())); self.orderings.push(augmented_ordering); } } @@ -2022,9 +2032,9 @@ impl UnionEquivalentOrderingBuilder { existing_ordering: &LexOrdering, existing_constants: &[ConstExpr], ) -> Option { - let mut augmented_ordering = vec![]; - let mut sort_expr_iter = ordering.iter().peekable(); - let mut existing_sort_expr_iter = existing_ordering.iter().peekable(); + let mut augmented_ordering = LexOrdering::default(); + let mut sort_expr_iter = ordering.inner.iter().peekable(); + let mut existing_sort_expr_iter = existing_ordering.inner.iter().peekable(); // walk in parallel down the two orderings, trying to match them up while sort_expr_iter.peek().is_some() || existing_sort_expr_iter.peek().is_some() @@ -2170,20 +2180,20 @@ mod tests { let mut input_properties = EquivalenceProperties::new(Arc::clone(&input_schema)); // add equivalent ordering [a, b, c, d] - input_properties.add_new_ordering(vec![ + input_properties.add_new_ordering(LexOrdering::new(vec![ parse_sort_expr("a", &input_schema), parse_sort_expr("b", &input_schema), parse_sort_expr("c", &input_schema), parse_sort_expr("d", &input_schema), - ]); + ])); // add equivalent ordering [a, c, b, d] - input_properties.add_new_ordering(vec![ + input_properties.add_new_ordering(LexOrdering::new(vec![ parse_sort_expr("a", &input_schema), parse_sort_expr("c", &input_schema), parse_sort_expr("b", &input_schema), // NB b and c are swapped parse_sort_expr("d", &input_schema), - ]); + ])); // simply project all the columns in order let proj_exprs = vec![ @@ -2197,7 +2207,7 @@ mod tests { assert_eq!( out_properties.to_string(), - "order: [[a@0 ASC,c@2 ASC,b@1 ASC,d@3 ASC], [a@0 ASC,b@1 ASC,c@2 ASC,d@3 ASC]]" + "order: [[a@0 ASC, c@2 ASC, b@1 ASC, d@3 ASC], [a@0 ASC, b@1 ASC, c@2 ASC, d@3 ASC]]" ); Ok(()) @@ -2403,27 +2413,27 @@ mod tests { eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr)?; let others = vec![ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), options: sort_options, - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_c_expr), options: sort_options, - }], + }]), ]; eq_properties.add_new_orderings(others); let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); expected_eqs.add_new_orderings([ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), options: sort_options, - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_c_expr), options: sort_options, - }], + }]), ]); let oeq_class = eq_properties.oeq_class().clone(); @@ -2446,7 +2456,7 @@ mod tests { let col_b = &col("b", &schema)?; let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings([vec![ + eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -2455,12 +2465,12 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]]); + ])]); let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0, 1]); assert_eq!( result, - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_b), options: sort_options_not @@ -2469,7 +2479,7 @@ mod tests { expr: Arc::clone(col_a), options: sort_options } - ] + ]) ); let schema = Schema::new(vec![ @@ -2482,11 +2492,11 @@ mod tests { let required_columns = [Arc::clone(col_b), Arc::clone(col_a)]; let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); eq_properties.add_new_orderings([ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, - }], - vec![ + }]), + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -2495,13 +2505,13 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ], + ]), ]); let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0, 1]); assert_eq!( result, - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_b), options: sort_options_not @@ -2510,7 +2520,7 @@ mod tests { expr: Arc::clone(col_a), options: sort_options } - ] + ]) ); let required_columns = [ @@ -2525,7 +2535,7 @@ mod tests { let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); // not satisfied orders - eq_properties.add_new_orderings([vec![ + eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -2538,7 +2548,7 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]]); + ])]); let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); assert_eq!(idxs, vec![0]); @@ -2567,14 +2577,14 @@ mod tests { eq_properties.add_equal_conditions(col_b, col_a)?; // [b ASC], [d ASC] eq_properties.add_new_orderings(vec![ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(col_b), options: option_asc, - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(col_d), options: option_asc, - }], + }]), ]); let test_cases = vec![ @@ -2605,7 +2615,7 @@ mod tests { let leading_orderings = eq_properties .oeq_class() .iter() - .flat_map(|ordering| ordering.first().cloned()) + .flat_map(|ordering| ordering.inner.first().cloned()) .collect::>(); let expr_props = eq_properties.get_expr_properties(Arc::clone(&expr)); let err_msg = format!( @@ -2649,7 +2659,7 @@ mod tests { nulls_first: true, }; // [d ASC, h DESC] also satisfies schema. - eq_properties.add_new_orderings([vec![ + eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_d), options: option_asc, @@ -2658,7 +2668,7 @@ mod tests { expr: Arc::clone(col_h), options: option_desc, }, - ]]); + ])]); let test_cases = vec![ // TEST CASE 1 (vec![col_a], vec![(col_a, option_asc)]), @@ -2940,7 +2950,7 @@ mod tests { Field::new("c", DataType::Timestamp(TimeUnit::Nanosecond, None), true), ])); let base_properties = EquivalenceProperties::new(Arc::clone(&schema)) - .with_reorder( + .with_reorder(LexOrdering::new( ["a", "b", "c"] .into_iter() .map(|c| { @@ -2953,7 +2963,7 @@ mod tests { }) }) .collect::>>()?, - ); + )); struct TestCase { name: &'static str, @@ -3042,10 +3052,10 @@ mod tests { options: SortOptions::default(), }) }) - .collect::>>()?; + .collect::>()?; assert_eq!( - properties.ordering_satisfy(&sort), + properties.ordering_satisfy(sort.as_ref()), case.should_satisfy_ordering, "failed test '{}'", case.name @@ -3564,7 +3574,7 @@ mod tests { ordering .iter() .map(|name| parse_sort_expr(name, schema)) - .collect::>() + .collect::() }) .collect::>(); diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 4bd022975ac3..c3d1b1425b7f 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -35,6 +35,7 @@ use datafusion_common::tree_node::{ use datafusion_common::Result; use datafusion_expr::Operator; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; @@ -245,10 +246,7 @@ pub fn reassign_predicate_columns( } /// Merge left and right sort expressions, checking for duplicates. -pub fn merge_vectors( - left: &[PhysicalSortExpr], - right: &[PhysicalSortExpr], -) -> Vec { +pub fn merge_vectors(left: LexOrderingRef, right: LexOrderingRef) -> LexOrdering { left.iter() .cloned() .chain(right.iter().cloned()) diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 3fe5d842dfd1..94960c95e4bb 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -25,16 +25,16 @@ use arrow::array::Array; use arrow::record_batch::RecordBatch; use arrow::{array::ArrayRef, datatypes::Field}; -use datafusion_common::ScalarValue; -use datafusion_common::{DataFusionError, Result}; -use datafusion_expr::{Accumulator, WindowFrame}; - use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, SlidingAggregateWindowExpr, WindowExpr, }; -use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; +use crate::{reverse_order_bys, PhysicalExpr}; +use datafusion_common::ScalarValue; +use datafusion_common::{DataFusionError, Result}; +use datafusion_expr::{Accumulator, WindowFrame}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; /// A window expr that takes the form of an aggregate function. /// @@ -43,7 +43,7 @@ use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; pub struct PlainAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: Vec, + order_by: LexOrdering, window_frame: Arc, } @@ -52,13 +52,13 @@ impl PlainAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: LexOrderingRef, window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: order_by.to_vec(), + order_by: LexOrdering::from_ref(order_by), window_frame, } } @@ -124,8 +124,8 @@ impl WindowExpr for PlainAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> &[PhysicalSortExpr] { - &self.order_by + fn order_by(&self) -> LexOrderingRef { + self.order_by.as_ref() } fn get_window_frame(&self) -> &Arc { @@ -139,14 +139,14 @@ impl WindowExpr for PlainAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - &reverse_order_bys(&self.order_by), + reverse_order_bys(self.order_by.as_ref()).as_ref(), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - &reverse_order_bys(&self.order_by), + reverse_order_bys(self.order_by.as_ref()).as_ref(), Arc::new(self.window_frame.reverse()), )) as _ } diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 8ff277db37df..5f6c5e5c2c1b 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -22,7 +22,6 @@ use std::ops::Range; use std::sync::Arc; use super::{BuiltInWindowFunctionExpr, WindowExpr}; -use crate::expressions::PhysicalSortExpr; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; @@ -34,13 +33,14 @@ use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. #[derive(Debug)] pub struct BuiltInWindowExpr { expr: Arc, partition_by: Vec>, - order_by: Vec, + order_by: LexOrdering, window_frame: Arc, } @@ -49,13 +49,13 @@ impl BuiltInWindowExpr { pub fn new( expr: Arc, partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: LexOrderingRef, window_frame: Arc, ) -> Self { Self { expr, partition_by: partition_by.to_vec(), - order_by: order_by.to_vec(), + order_by: LexOrdering::from_ref(order_by), window_frame, } } @@ -76,7 +76,8 @@ impl BuiltInWindowExpr { if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - eq_properties.add_new_orderings([vec![fn_res_ordering]]); + eq_properties + .add_new_orderings([LexOrdering::new(vec![fn_res_ordering])]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -117,8 +118,8 @@ impl WindowExpr for BuiltInWindowExpr { &self.partition_by } - fn order_by(&self) -> &[PhysicalSortExpr] { - &self.order_by + fn order_by(&self) -> LexOrderingRef { + self.order_by.as_ref() } fn evaluate(&self, batch: &RecordBatch) -> Result { @@ -266,7 +267,7 @@ impl WindowExpr for BuiltInWindowExpr { Arc::new(BuiltInWindowExpr::new( reverse_expr, &self.partition_by.clone(), - &reverse_order_bys(&self.order_by), + reverse_order_bys(self.order_by.as_ref()).as_ref(), Arc::new(self.window_frame.reverse()), )) as _ }) diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index b889ec8c5d98..1e46baae7b0a 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -25,15 +25,15 @@ use arrow::array::{Array, ArrayRef}; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; -use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::{Accumulator, WindowFrame}; - use crate::aggregate::AggregateFunctionExpr; use crate::window::window_expr::AggregateWindowExpr; use crate::window::{ PartitionBatches, PartitionWindowAggStates, PlainAggregateWindowExpr, WindowExpr, }; use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::{Accumulator, WindowFrame}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; /// A window expr that takes the form of an aggregate function that /// can be incrementally computed over sliding windows. @@ -43,7 +43,7 @@ use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; pub struct SlidingAggregateWindowExpr { aggregate: Arc, partition_by: Vec>, - order_by: Vec, + order_by: LexOrdering, window_frame: Arc, } @@ -52,13 +52,13 @@ impl SlidingAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: LexOrderingRef, window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: order_by.to_vec(), + order_by: LexOrdering::from_ref(order_by), window_frame, } } @@ -108,8 +108,8 @@ impl WindowExpr for SlidingAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> &[PhysicalSortExpr] { - &self.order_by + fn order_by(&self) -> LexOrderingRef { + self.order_by.as_ref() } fn get_window_frame(&self) -> &Arc { @@ -123,14 +123,14 @@ impl WindowExpr for SlidingAggregateWindowExpr { Arc::new(PlainAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - &reverse_order_bys(&self.order_by), + reverse_order_bys(self.order_by.as_ref()).as_ref(), Arc::new(self.window_frame.reverse()), )) as _ } else { Arc::new(SlidingAggregateWindowExpr::new( Arc::new(reverse_expr), &self.partition_by.clone(), - &reverse_order_bys(&self.order_by), + reverse_order_bys(self.order_by.as_ref()).as_ref(), Arc::new(self.window_frame.reverse()), )) as _ } @@ -157,7 +157,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { expr: new_expr, options: req.options, }) - .collect::>(); + .collect::(); Some(Arc::new(SlidingAggregateWindowExpr { aggregate: self .aggregate diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 46c46fab68c5..0f882def4433 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; -use crate::{LexOrderingRef, PhysicalExpr, PhysicalSortExpr}; +use crate::{LexOrderingRef, PhysicalExpr}; use arrow::array::{new_empty_array, Array, ArrayRef}; use arrow::compute::kernels::sort::SortColumn; @@ -109,7 +109,7 @@ pub trait WindowExpr: Send + Sync + Debug { fn partition_by(&self) -> &[Arc]; /// Expressions that's from the window function's order by clause, empty if absent - fn order_by(&self) -> &[PhysicalSortExpr]; + fn order_by(&self) -> LexOrderingRef; /// Get order by columns, empty if absent fn order_by_columns(&self, batch: &RecordBatch) -> Result> { diff --git a/datafusion/physical-optimizer/src/topk_aggregation.rs b/datafusion/physical-optimizer/src/topk_aggregation.rs index c8a28ed0ec0b..0e5fb82d9e93 100644 --- a/datafusion/physical-optimizer/src/topk_aggregation.rs +++ b/datafusion/physical-optimizer/src/topk_aggregation.rs @@ -25,6 +25,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::Result; use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::execution_plan::CardinalityEffect; use datafusion_physical_plan::projection::ProjectionExec; @@ -126,7 +127,7 @@ impl TopKAggregation { Ok(Transformed::no(plan)) }; let child = Arc::clone(child).transform_down(closure).data().ok()?; - let sort = SortExec::new(sort.expr().to_vec(), child) + let sort = SortExec::new(LexOrdering::new(sort.expr().to_vec()), child) .with_fetch(sort.fetch()) .with_preserve_partitioning(sort.preserve_partitioning()); Some(Arc::new(sort)) diff --git a/datafusion/physical-plan/benches/spm.rs b/datafusion/physical-plan/benches/spm.rs index 9cc703f5f726..fbbd27409173 100644 --- a/datafusion/physical-plan/benches/spm.rs +++ b/datafusion/physical-plan/benches/spm.rs @@ -22,6 +22,7 @@ use arrow_array::{ArrayRef, Int32Array, Int64Array, StringArray}; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::memory::MemoryExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{collect, ExecutionPlan}; @@ -70,7 +71,7 @@ fn generate_spm_for_round_robin_tie_breaker( let partitiones = vec![rbs.clone(); partition_count]; let schema = rb.schema(); - let sort = vec![ + let sort = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), @@ -79,7 +80,7 @@ fn generate_spm_for_round_robin_tie_breaker( expr: col("c", &schema).unwrap(), options: Default::default(), }, - ]; + ]); let exec = MemoryExec::try_new(&partitiones, schema, None).unwrap(); SortPreservingMergeExec::new(sort, Arc::new(exec)) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 48a03af19dbd..4193cc187e10 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -937,10 +937,10 @@ fn get_aggregate_expr_req( // necessary, or the aggregation is performing a "second stage" calculation, // then ignore the ordering requirement. if !aggr_expr.order_sensitivity().hard_requires() || !agg_mode.is_first_stage() { - return vec![]; + return LexOrdering::default(); } - let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + let mut req = LexOrdering::from_ref(aggr_expr.order_bys().unwrap_or_default()); // In non-first stage modes, we accumulate data (using `merge_batch`) from // different partitions (i.e. merge partial results). During this merge, we @@ -983,7 +983,7 @@ fn finer_ordering( agg_mode: &AggregateMode, ) -> Option { let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); - eq_properties.get_finer_ordering(existing_req, &aggr_req) + eq_properties.get_finer_ordering(existing_req.as_ref(), aggr_req.as_ref()) } /// Concatenates the given slices. @@ -1014,12 +1014,12 @@ pub fn get_finer_aggregate_exprs_requirement( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result { - let mut requirement = vec![]; + let mut requirement = LexOrdering::default(); for aggr_expr in aggr_exprs.iter_mut() { if let Some(finer_ordering) = finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) { - if eq_properties.ordering_satisfy(&finer_ordering) { + if eq_properties.ordering_satisfy(finer_ordering.as_ref()) { // Requirement is satisfied by existing ordering requirement = finer_ordering; continue; @@ -1033,7 +1033,7 @@ pub fn get_finer_aggregate_exprs_requirement( eq_properties, agg_mode, ) { - if eq_properties.ordering_satisfy(&finer_ordering) { + if eq_properties.ordering_satisfy(finer_ordering.as_ref()) { // Reverse requirement is satisfied by exiting ordering. // Hence reverse the aggregator requirement = finer_ordering; @@ -1074,7 +1074,9 @@ pub fn get_finer_aggregate_exprs_requirement( ); } - Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) + Ok(PhysicalSortRequirement::from_sort_exprs( + requirement.inner.iter(), + )) } /// Returns physical expressions for arguments to evaluate against a batch. @@ -2088,7 +2090,7 @@ mod tests { let args = [col("b", schema)?]; AggregateExprBuilder::new(first_value_udaf(), args.to_vec()) - .order_by(ordering_req.to_vec()) + .order_by(LexOrdering::new(ordering_req.to_vec())) .schema(Arc::new(schema.clone())) .alias(String::from("first_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2106,7 +2108,7 @@ mod tests { }]; let args = [col("b", schema)?]; AggregateExprBuilder::new(last_value_udaf(), args.to_vec()) - .order_by(ordering_req.to_vec()) + .order_by(LexOrdering::new(ordering_req.to_vec())) .schema(Arc::new(schema.clone())) .alias(String::from("last_value(b) ORDER BY [b ASC NULLS LAST]")) .build() @@ -2272,7 +2274,7 @@ mod tests { ]), ]; - let common_requirement = vec![ + let common_requirement = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_a), options: options1, @@ -2281,14 +2283,14 @@ mod tests { expr: Arc::clone(col_c), options: options1, }, - ]; + ]); let mut aggr_exprs = order_by_exprs .into_iter() .map(|order_by_expr| { let ordering_req = order_by_expr.unwrap_or_default(); AggregateExprBuilder::new(array_agg_udaf(), vec![Arc::clone(col_a)]) .alias("a") - .order_by(ordering_req.to_vec()) + .order_by(LexOrdering::new(ordering_req.to_vec())) .schema(Arc::clone(&test_schema)) .build() .map(Arc::new) diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index accb2fda1131..24846d239591 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -19,7 +19,7 @@ use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; use datafusion_expr::EmitTo; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use std::mem::size_of; mod full; @@ -45,7 +45,7 @@ impl GroupOrdering { pub fn try_new( input_schema: &Schema, mode: &InputOrderMode, - ordering: &[PhysicalSortExpr], + ordering: LexOrderingRef, ) -> Result { match mode { InputOrderMode::Linear => Ok(GroupOrdering::None), diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index 2dd1ea8a5449..5cc55dc0d028 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -21,7 +21,7 @@ use arrow_schema::Schema; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use std::mem::size_of; use std::sync::Arc; @@ -107,7 +107,7 @@ impl GroupOrderingPartial { pub fn try_new( input_schema: &Schema, order_indices: &[usize], - ordering: &[PhysicalSortExpr], + ordering: LexOrderingRef, ) -> Result { assert!(!order_indices.is_empty()); assert!(order_indices.len() <= ordering.len()); diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 7d21cc2f1944..fe05f7375ed3 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -48,14 +48,14 @@ use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; +use super::order::GroupOrdering; +use super::AggregateExec; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::ready; use futures::stream::{Stream, StreamExt}; use log::debug; -use super::order::GroupOrdering; -use super::AggregateExec; - #[derive(Debug, Clone)] /// This object tracks the aggregation phase (input/output) pub(crate) enum ExecutionState { @@ -80,7 +80,7 @@ struct SpillState { // the execution. // ======================================================================== /// Sorting expression for spilling batches - spill_expr: Vec, + spill_expr: LexOrdering, /// Schema for spilling batches spill_schema: SchemaRef, @@ -511,7 +511,7 @@ impl GroupedHashAggregateStream { let group_ordering = GroupOrdering::try_new( &group_schema, &agg.input_order_mode, - ordering.as_slice(), + ordering.as_ref(), )?; let group_values = new_group_values(group_schema)?; @@ -965,7 +965,7 @@ impl GroupedHashAggregateStream { /// Emit all rows, sort them, and store them on disk. fn spill(&mut self) -> Result<()> { let emit = self.emit(EmitTo::All, true)?; - let sorted = sort_batch(&emit, &self.spill_state.spill_expr, None)?; + let sorted = sort_batch(&emit, self.spill_state.spill_expr.as_ref(), None)?; let spillfile = self.runtime.disk_manager.create_tmp_file("HashAggSpill")?; // TODO: slice large `sorted` and write to multiple files in parallel spill_record_batch_by_size( @@ -1030,7 +1030,7 @@ impl GroupedHashAggregateStream { streams.push(Box::pin(RecordBatchStreamAdapter::new( Arc::clone(&schema), futures::stream::once(futures::future::lazy(move |_| { - sort_batch(&batch, &expr, None) + sort_batch(&batch, expr.as_ref(), None) })), ))); for spill in self.spill_state.spills.drain(..) { @@ -1041,7 +1041,7 @@ impl GroupedHashAggregateStream { self.input = StreamingMergeBuilder::new() .with_streams(streams) .with_schema(schema) - .with_expressions(&self.spill_state.spill_expr) + .with_expressions(self.spill_state.spill_expr.as_ref()) .with_metrics(self.baseline_metrics.clone()) .with_batch_size(self.batch_size) .with_reservation(self.reservation.new_empty()) diff --git a/datafusion/physical-plan/src/display.rs b/datafusion/physical-plan/src/display.rs index e79b3c817bd1..9f3a76e28577 100644 --- a/datafusion/physical-plan/src/display.rs +++ b/datafusion/physical-plan/src/display.rs @@ -25,7 +25,7 @@ use arrow_schema::SchemaRef; use datafusion_common::display::{GraphvizBuilder, PlanType, StringifiedPlan}; use datafusion_expr::display_schema; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::LexOrdering; use super::{accept, ExecutionPlan, ExecutionPlanVisitor}; @@ -459,23 +459,6 @@ impl<'a> fmt::Display for ProjectSchemaDisplay<'a> { } } -/// A wrapper to customize output ordering display. -#[derive(Debug)] -pub struct OutputOrderingDisplay<'a>(pub &'a [PhysicalSortExpr]); - -impl<'a> fmt::Display for OutputOrderingDisplay<'a> { - fn fmt(&self, f: &mut Formatter) -> fmt::Result { - write!(f, "[")?; - for (i, e) in self.0.iter().enumerate() { - if i > 0 { - write!(f, ", ")? - } - write!(f, "{e}")?; - } - write!(f, "]") - } -} - pub fn display_orderings(f: &mut Formatter, orderings: &[LexOrdering]) -> fmt::Result { if let Some(ordering) = orderings.first() { if !ordering.is_empty() { @@ -489,8 +472,8 @@ pub fn display_orderings(f: &mut Formatter, orderings: &[LexOrdering]) -> fmt::R orderings.iter().enumerate().filter(|(_, o)| !o.is_empty()) { match idx { - 0 => write!(f, "{}", OutputOrderingDisplay(ordering))?, - _ => write!(f, ", {}", OutputOrderingDisplay(ordering))?, + 0 => write!(f, "[{}]", ordering)?, + _ => write!(f, ", [{}]", ordering)?, } } let end = if orderings.len() == 1 { "" } else { "]" }; diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index e6484452d43e..d65320dbab68 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -37,8 +37,8 @@ pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ expressions, udf, Distribution, Partitioning, PhysicalExpr, }; -use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; +use datafusion_physical_expr_common::sort_expr::{LexOrderingRef, LexRequirement}; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::display::DisplayableExecutionPlan; @@ -443,7 +443,7 @@ pub trait ExecutionPlanProperties { /// For example, `SortExec` (obviously) produces sorted output as does /// `SortPreservingMergeStream`. Less obviously, `Projection` produces sorted /// output if its input is sorted as it does not reorder the input rows. - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]>; + fn output_ordering(&self) -> Option; /// Get the [`EquivalenceProperties`] within the plan. /// @@ -474,7 +474,7 @@ impl ExecutionPlanProperties for Arc { self.properties().execution_mode() } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + fn output_ordering(&self) -> Option { self.properties().output_ordering() } @@ -492,7 +492,7 @@ impl ExecutionPlanProperties for &dyn ExecutionPlan { self.properties().execution_mode() } - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + fn output_ordering(&self) -> Option { self.properties().output_ordering() } @@ -643,7 +643,7 @@ impl PlanProperties { &self.partitioning } - pub fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + pub fn output_ordering(&self) -> Option { self.output_ordering.as_deref() } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 957230f51372..a87743565adf 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -858,7 +858,7 @@ pub(crate) mod tests { use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{Partitioning, PhysicalExpr}; - use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use rstest::rstest; @@ -888,7 +888,7 @@ pub(crate) mod tests { let mut exec = MemoryExec::try_new(&[batches], Arc::clone(&schema), None).unwrap(); if !sorted_column_names.is_empty() { - let mut sort_info = Vec::new(); + let mut sort_info = LexOrdering::default(); for name in sorted_column_names { let index = schema.index_of(name).unwrap(); let sort_expr = PhysicalSortExpr { diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 20fafcc34773..90dc407fcaed 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -52,7 +52,7 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; @@ -88,9 +88,9 @@ pub struct SortMergeJoinExec { /// Execution metrics metrics: ExecutionPlanMetricsSet, /// The left SortExpr - left_sort_exprs: Vec, + left_sort_exprs: LexOrdering, /// The right SortExpr - right_sort_exprs: Vec, + right_sort_exprs: LexOrdering, /// Sort options of join columns used in sorting left and right execution plans pub sort_options: Vec, /// If null_equals_null is true, null == null else null != null @@ -159,8 +159,8 @@ impl SortMergeJoinExec { join_type, schema, metrics: ExecutionPlanMetricsSet::new(), - left_sort_exprs, - right_sort_exprs, + left_sort_exprs: LexOrdering::new(left_sort_exprs), + right_sort_exprs: LexOrdering::new(right_sort_exprs), sort_options, null_equals_null, cache, @@ -299,10 +299,10 @@ impl ExecutionPlan for SortMergeJoinExec { fn required_input_ordering(&self) -> Vec> { vec![ Some(PhysicalSortRequirement::from_sort_exprs( - &self.left_sort_exprs, + self.left_sort_exprs.iter(), )), Some(PhysicalSortRequirement::from_sort_exprs( - &self.right_sort_exprs, + self.right_sort_exprs.iter(), )), ] } diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 02c71dab3df2..5ccdd9b40dee 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -40,6 +40,7 @@ use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use hashbrown::raw::RawTable; use hashbrown::HashSet; @@ -744,8 +745,8 @@ pub fn prepare_sorted_exprs( filter: &JoinFilter, left: &Arc, right: &Arc, - left_sort_exprs: &[PhysicalSortExpr], - right_sort_exprs: &[PhysicalSortExpr], + left_sort_exprs: LexOrderingRef, + right_sort_exprs: LexOrderingRef, ) -> Result<(SortedFilterExpr, SortedFilterExpr, ExprIntervalGraph)> { let err = || { datafusion_common::plan_datafusion_err!("Filter does not include the child order") diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 3e0cd48da2bf..81c13c652513 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -48,7 +48,6 @@ use crate::joins::utils::{ }; use crate::{ execution_mode_from_children, - expressions::PhysicalSortExpr, joins::StreamJoinPartitionMode, metrics::{ExecutionPlanMetricsSet, MetricsSet}, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties, @@ -74,7 +73,9 @@ use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use ahash::RandomState; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexOrderingRef, LexRequirement, +}; use futures::{ready, Stream, StreamExt}; use hashbrown::HashSet; use parking_lot::Mutex; @@ -187,9 +188,9 @@ pub struct SymmetricHashJoinExec { /// If null_equals_null is true, null == null else null != null pub(crate) null_equals_null: bool, /// Left side sort expression(s) - pub(crate) left_sort_exprs: Option>, + pub(crate) left_sort_exprs: Option, /// Right side sort expression(s) - pub(crate) right_sort_exprs: Option>, + pub(crate) right_sort_exprs: Option, /// Partition Mode mode: StreamJoinPartitionMode, /// Cache holding plan properties like equivalences, output partitioning etc. @@ -211,8 +212,8 @@ impl SymmetricHashJoinExec { filter: Option, join_type: &JoinType, null_equals_null: bool, - left_sort_exprs: Option>, - right_sort_exprs: Option>, + left_sort_exprs: Option, + right_sort_exprs: Option, mode: StreamJoinPartitionMode, ) -> Result { let left_schema = left.schema(); @@ -319,12 +320,12 @@ impl SymmetricHashJoinExec { } /// Get left_sort_exprs - pub fn left_sort_exprs(&self) -> Option<&[PhysicalSortExpr]> { + pub fn left_sort_exprs(&self) -> Option { self.left_sort_exprs.as_deref() } /// Get right_sort_exprs - pub fn right_sort_exprs(&self) -> Option<&[PhysicalSortExpr]> { + pub fn right_sort_exprs(&self) -> Option { self.right_sort_exprs.as_deref() } @@ -417,9 +418,11 @@ impl ExecutionPlan for SymmetricHashJoinExec { vec![ self.left_sort_exprs .as_ref() + .map(LexOrdering::iter) .map(PhysicalSortRequirement::from_sort_exprs), self.right_sort_exprs .as_ref() + .map(LexOrdering::iter) .map(PhysicalSortRequirement::from_sort_exprs), ] } @@ -1646,6 +1649,7 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{binary, col, lit, Column}; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use once_cell::sync::Lazy; use rstest::*; @@ -1746,7 +1750,7 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: binary( col("la1", left_schema)?, Operator::Plus, @@ -1754,11 +1758,11 @@ mod tests { left_schema, )?, options: SortOptions::default(), - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -1825,14 +1829,14 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -1968,20 +1972,20 @@ mod tests { let (left_partition, right_partition) = get_or_create_table((11, 21), 8)?; let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1_des", left_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1_des", right_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2026,20 +2030,20 @@ mod tests { let (left_partition, right_partition) = get_or_create_table((10, 11), 8)?; let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_asc_null_first", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_asc_null_first", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2084,20 +2088,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_asc_null_last", left_schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_asc_null_last", right_schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2144,20 +2148,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_desc_null_first", left_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_desc_null_first", right_schema)?, options: SortOptions { descending: true, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2205,15 +2209,15 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }]; + }]); - let right_sorted = vec![PhysicalSortExpr { + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2263,20 +2267,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); let left_sorted = vec![ - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, options: SortOptions::default(), - }], - vec![PhysicalSortExpr { + }]), + LexOrdering::new(vec![PhysicalSortExpr { expr: col("la2", left_schema)?, options: SortOptions::default(), - }], + }]), ]; - let right_sorted = vec![PhysicalSortExpr { + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, @@ -2343,20 +2347,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); let on = vec![(col("lc1", left_schema)?, col("rc1", right_schema)?)]; - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("lt1", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("rt1", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2425,20 +2429,20 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); let on = vec![(col("lc1", left_schema)?, col("rc1", right_schema)?)]; - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("li1", left_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ri1", right_schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, @@ -2500,14 +2504,14 @@ mod tests { let left_schema = &left_partition[0].schema(); let right_schema = &right_partition[0].schema(); - let left_sorted = vec![PhysicalSortExpr { + let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_float", left_schema)?, options: SortOptions::default(), - }]; - let right_sorted = vec![PhysicalSortExpr { + }]); + let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_float", right_schema)?, options: SortOptions::default(), - }]; + }]); let (left, right) = create_memory_table( left_partition, right_partition, diff --git a/datafusion/physical-plan/src/joins/test_utils.rs b/datafusion/physical-plan/src/joins/test_utils.rs index 090d60f0bac3..421fd0da808c 100644 --- a/datafusion/physical-plan/src/joins/test_utils.rs +++ b/datafusion/physical-plan/src/joins/test_utils.rs @@ -101,8 +101,10 @@ pub async fn partitioned_sym_join_with_filter( filter, join_type, null_equals_null, - left.output_ordering().map(|p| p.to_vec()), - right.output_ordering().map(|p| p.to_vec()), + left.output_ordering().map(|p| LexOrdering::new(p.to_vec())), + right + .output_ordering() + .map(|p| LexOrdering::new(p.to_vec())), StreamJoinPartitionMode::Partitioned, )?; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index e7c191f9835e..d3fa37c2ac80 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -449,10 +449,10 @@ pub fn adjust_right_output_partitioning( /// the left column (zeroth index in the tuple) inside `right_ordering`. fn replace_on_columns_of_right_ordering( on_columns: &[(PhysicalExprRef, PhysicalExprRef)], - right_ordering: &mut [PhysicalSortExpr], + right_ordering: &mut LexOrdering, ) -> Result<()> { for (left_col, right_col) in on_columns { - for item in right_ordering.iter_mut() { + for item in right_ordering.inner.iter_mut() { let new_expr = Arc::clone(&item.expr) .transform(|e| { if e.eq(right_col) { @@ -472,7 +472,7 @@ fn offset_ordering( ordering: LexOrderingRef, join_type: &JoinType, offset: usize, -) -> Vec { +) -> LexOrdering { match join_type { // In the case below, right ordering should be offsetted with the left // side length, since we append the right table to the left table. @@ -483,7 +483,7 @@ fn offset_ordering( options: sort_expr.options, }) .collect(), - _ => ordering.to_vec(), + _ => LexOrdering::from_ref(ordering), } } @@ -503,15 +503,16 @@ pub fn calculate_join_output_ordering( if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { replace_on_columns_of_right_ordering( on_columns, - &mut right_ordering.to_vec(), + &mut LexOrdering::from_ref(right_ordering), ) .ok()?; merge_vectors( left_ordering, - &offset_ordering(right_ordering, &join_type, left_columns_len), + offset_ordering(right_ordering, &join_type, left_columns_len) + .as_ref(), ) } else { - left_ordering.to_vec() + LexOrdering::from_ref(left_ordering) } } [false, true] => { @@ -519,11 +520,12 @@ pub fn calculate_join_output_ordering( if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { replace_on_columns_of_right_ordering( on_columns, - &mut right_ordering.to_vec(), + &mut LexOrdering::from_ref(right_ordering), ) .ok()?; merge_vectors( - &offset_ordering(right_ordering, &join_type, left_columns_len), + offset_ordering(right_ordering, &join_type, left_columns_len) + .as_ref(), left_ordering, ) } else { @@ -2600,7 +2602,7 @@ mod tests { #[test] fn test_calculate_join_output_ordering() -> Result<()> { let options = SortOptions::default(); - let left_ordering = vec![ + let left_ordering = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options, @@ -2613,8 +2615,8 @@ mod tests { expr: Arc::new(Column::new("d", 3)), options, }, - ]; - let right_ordering = vec![ + ]); + let right_ordering = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 2)), options, @@ -2623,7 +2625,7 @@ mod tests { expr: Arc::new(Column::new("y", 1)), options, }, - ]; + ]); let join_type = JoinType::Inner; let on_columns = [( Arc::new(Column::new("b", 1)) as _, @@ -2634,7 +2636,7 @@ mod tests { let probe_sides = [Some(JoinSide::Left), Some(JoinSide::Right)]; let expected = [ - Some(vec![ + Some(LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), options, @@ -2655,8 +2657,8 @@ mod tests { expr: Arc::new(Column::new("y", 6)), options, }, - ]), - Some(vec![ + ])), + Some(LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 7)), options, @@ -2677,7 +2679,7 @@ mod tests { expr: Arc::new(Column::new("d", 3)), options, }, - ]), + ])), ]; for (i, (maintains_input_order, probe_side)) in @@ -2685,8 +2687,8 @@ mod tests { { assert_eq!( calculate_join_output_ordering( - &left_ordering, - &right_ordering, + left_ordering.as_ref(), + right_ordering.as_ref(), join_type, &on_columns, left_columns_len, diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index dd4868d1bfcc..56ed144845a0 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -22,7 +22,6 @@ use std::fmt; use std::sync::Arc; use std::task::{Context, Poll}; -use super::expressions::PhysicalSortExpr; use super::{ common, DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan, Partitioning, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -79,10 +78,7 @@ impl DisplayAs for MemoryExec { .sort_information .first() .map(|output_ordering| { - format!( - ", output_ordering={}", - PhysicalSortExpr::format_list(output_ordering) - ) + format!(", output_ordering={}", output_ordering) }) .unwrap_or_default(); @@ -216,7 +212,7 @@ impl MemoryExec { let fields = self.schema.fields(); let ambiguous_column = sort_information .iter() - .flatten() + .flat_map(|ordering| ordering.inner.clone()) .flat_map(|expr| collect_columns(&expr.expr)) .find(|col| { fields @@ -365,6 +361,7 @@ mod tests { use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_physical_expr_common::sort_expr::LexOrdering; #[test] fn test_memory_order_eq() -> datafusion_common::Result<()> { @@ -373,7 +370,7 @@ mod tests { Field::new("b", DataType::Int64, false), Field::new("c", DataType::Int64, false), ])); - let sort1 = vec![ + let sort1 = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), @@ -382,12 +379,12 @@ mod tests { expr: col("b", &schema)?, options: SortOptions::default(), }, - ]; - let sort2 = vec![PhysicalSortExpr { + ]); + let sort2 = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema)?, options: SortOptions::default(), - }]; - let mut expected_output_order = vec![]; + }]); + let mut expected_output_order = LexOrdering::default(); expected_output_order.extend(sort1.clone()); expected_output_order.extend(sort2.clone()); @@ -396,8 +393,8 @@ mod tests { .try_with_sort_information(sort_information)?; assert_eq!( - mem_exec.properties().output_ordering().unwrap(), - expected_output_order + mem_exec.properties().output_ordering().unwrap().to_vec(), + expected_output_order.inner ); let eq_properties = mem_exec.properties().equivalence_properties(); assert!(eq_properties.oeq_class().contains(&sort1)); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 601c1e873152..06144f98c89d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -47,9 +47,10 @@ use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use crate::execution_plan::CardinalityEffect; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use hashbrown::HashMap; @@ -502,11 +503,7 @@ impl DisplayAs for RepartitionExec { } if let Some(sort_exprs) = self.sort_exprs() { - write!( - f, - ", sort_exprs={}", - PhysicalSortExpr::format_list(sort_exprs) - )?; + write!(f, ", sort_exprs={}", LexOrdering::from_ref(sort_exprs))?; } Ok(()) } @@ -1561,10 +1558,10 @@ mod tests { mod test { use arrow_schema::{DataType, Field, Schema, SortOptions}; - use datafusion_physical_expr::expressions::col; - use crate::memory::MemoryExec; use crate::union::UnionExec; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use super::*; @@ -1659,12 +1656,12 @@ mod test { Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) } - fn sort_exprs(schema: &Schema) -> Vec { + fn sort_exprs(schema: &Schema) -> LexOrdering { let options = SortOptions::default(); - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("c0", schema).unwrap(), options, - }] + }]) } fn memory_exec(schema: &SchemaRef) -> Arc { @@ -1673,7 +1670,7 @@ mod test { fn sorted_memory_exec( schema: &SchemaRef, - sort_exprs: Vec, + sort_exprs: LexOrdering, ) -> Arc { Arc::new( MemoryExec::try_new(&[vec![]], Arc::clone(schema), None) diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 649c05d52e8b..8f853464c9bd 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -57,7 +57,6 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::sort::sort_batch; use crate::{ @@ -73,6 +72,7 @@ use datafusion_common::Result; use datafusion_execution::{RecordBatchStream, TaskContext}; use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use futures::{ready, Stream, StreamExt}; use log::trace; @@ -82,7 +82,7 @@ pub struct PartialSortExec { /// Input schema pub(crate) input: Arc, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Length of continuous matching columns of input that satisfy /// the required ordering for the sort common_prefix_length: usize, @@ -100,7 +100,7 @@ pub struct PartialSortExec { impl PartialSortExec { /// Create a new partial sort execution plan pub fn new( - expr: Vec, + expr: LexOrdering, input: Arc, common_prefix_length: usize, ) -> Self { @@ -159,8 +159,8 @@ impl PartialSortExec { } /// Sort expressions - pub fn expr(&self) -> &[PhysicalSortExpr] { - &self.expr + pub fn expr(&self) -> LexOrderingRef { + self.expr.as_ref() } /// If `Some(fetch)`, limits output to only the first "fetch" items @@ -212,13 +212,12 @@ impl DisplayAs for PartialSortExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - let expr = PhysicalSortExpr::format_list(&self.expr); let common_prefix_length = self.common_prefix_length; match self.fetch { Some(fetch) => { - write!(f, "PartialSortExec: TopK(fetch={fetch}), expr=[{expr}], common_prefix_length=[{common_prefix_length}]", ) + write!(f, "PartialSortExec: TopK(fetch={fetch}), expr=[{}], common_prefix_length=[{common_prefix_length}]", self.expr) } - None => write!(f, "PartialSortExec: expr=[{expr}], common_prefix_length=[{common_prefix_length}]"), + None => write!(f, "PartialSortExec: expr=[{}], common_prefix_length=[{common_prefix_length}]", self.expr), } } } @@ -315,7 +314,7 @@ struct PartialSortStream { /// The input plan input: SendableRecordBatchStream, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Length of prefix common to input ordering and required ordering of plan /// should be more than 0 otherwise PartialSort is not applicable common_prefix_length: usize, @@ -394,7 +393,7 @@ impl PartialSortStream { fn sort_in_mem_batches(self: &mut Pin<&mut Self>) -> Result { let input_batch = concat_batches(&self.schema(), &self.in_mem_batches)?; self.in_mem_batches.clear(); - let result = sort_batch(&input_batch, &self.expr, self.fetch)?; + let result = sort_batch(&input_batch, self.expr.as_ref(), self.fetch)?; if let Some(remaining_fetch) = self.fetch { // remaining_fetch - result.num_rows() is always be >= 0 // because result length of sort_batch with limit cannot be @@ -448,6 +447,7 @@ mod tests { use crate::collect; use crate::expressions::col; + use crate::expressions::PhysicalSortExpr; use crate::memory::MemoryExec; use crate::sorts::sort::SortExec; use crate::test; @@ -475,7 +475,7 @@ mod tests { }; let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -488,7 +488,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&source), 2, )) as Arc; @@ -539,7 +539,7 @@ mod tests { for common_prefix_length in [1, 2] { let partial_sort_exec = Arc::new( PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -552,7 +552,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&source), common_prefix_length, ) @@ -611,7 +611,7 @@ mod tests { [(1, &source_tables[0]), (2, &source_tables[1])] { let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -624,7 +624,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(source), common_prefix_length, )); @@ -701,7 +701,7 @@ mod tests { }; let schema = mem_exec.schema(); let partial_sort_executor = PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -714,7 +714,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&mem_exec), 1, ); @@ -762,7 +762,7 @@ mod tests { (Some(250), vec![0, 125, 125]), ] { let partial_sort_executor = PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -775,7 +775,7 @@ mod tests { expr: col("c", &schema)?, options: option_asc, }, - ], + ]), Arc::clone(&mem_exec), 1, ) @@ -834,10 +834,10 @@ mod tests { )?); let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("field_name", &schema)?, options: SortOptions::default(), - }], + }]), input, 1, )); @@ -923,7 +923,7 @@ mod tests { )?; let partial_sort_exec = Arc::new(PartialSortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: option_asc, @@ -936,7 +936,7 @@ mod tests { expr: col("c", &schema)?, options: option_desc, }, - ], + ]), Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None)?), 2, )); @@ -1000,10 +1000,10 @@ mod tests { let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); let sort_exec = Arc::new(PartialSortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }], + }]), blocking_exec, 1, )); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 921678a4ad92..32d6d3e0073c 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -52,7 +52,9 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement; +use datafusion_physical_expr_common::sort_expr::{ + LexOrderingRef, PhysicalSortRequirement, +}; use crate::execution_plan::CardinalityEffect; use futures::{StreamExt, TryStreamExt}; @@ -243,7 +245,7 @@ impl ExternalSorter { pub fn new( partition_id: usize, schema: SchemaRef, - expr: Vec, + expr: LexOrdering, batch_size: usize, fetch: Option, sort_spill_reservation_bytes: usize, @@ -265,7 +267,7 @@ impl ExternalSorter { in_mem_batches: vec![], in_mem_batches_sorted: true, spills: vec![], - expr: expr.into(), + expr: expr.inner.into(), metrics, fetch, reservation, @@ -345,7 +347,7 @@ impl ExternalSorter { StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(&self.expr) + .with_expressions(self.expr.to_vec().as_slice()) .with_metrics(self.metrics.baseline.clone()) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -537,7 +539,7 @@ impl ExternalSorter { StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(&self.expr) + .with_expressions(self.expr.as_ref()) .with_metrics(metrics) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -601,7 +603,7 @@ impl Debug for ExternalSorter { pub fn sort_batch( batch: &RecordBatch, - expressions: &[PhysicalSortExpr], + expressions: LexOrderingRef, fetch: Option, ) -> Result { let sort_columns = expressions @@ -678,7 +680,7 @@ pub struct SortExec { /// Input schema pub(crate) input: Arc, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Containing all metrics set created during sort metrics_set: ExecutionPlanMetricsSet, /// Preserve partitions of input plan. If false, the input partitions @@ -693,7 +695,7 @@ pub struct SortExec { impl SortExec { /// Create a new sort execution plan that produces a single, /// sorted output partition. - pub fn new(expr: Vec, input: Arc) -> Self { + pub fn new(expr: LexOrdering, input: Arc) -> Self { let preserve_partitioning = false; let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); Self { @@ -760,8 +762,8 @@ impl SortExec { } /// Sort expressions - pub fn expr(&self) -> &[PhysicalSortExpr] { - &self.expr + pub fn expr(&self) -> LexOrderingRef { + self.expr.as_ref() } /// If `Some(fetch)`, limits output to only the first "fetch" items @@ -818,13 +820,12 @@ impl DisplayAs for SortExec { fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - let expr = PhysicalSortExpr::format_list(&self.expr); let preserve_partitioning = self.preserve_partitioning; match self.fetch { Some(fetch) => { - write!(f, "SortExec: TopK(fetch={fetch}), expr=[{expr}], preserve_partitioning=[{preserve_partitioning}]",) + write!(f, "SortExec: TopK(fetch={fetch}), expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr) } - None => write!(f, "SortExec: expr=[{expr}], preserve_partitioning=[{preserve_partitioning}]"), + None => write!(f, "SortExec: expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr), } } } @@ -1027,9 +1028,9 @@ mod tests { impl SortedUnboundedExec { fn compute_properties(schema: SchemaRef) -> PlanProperties { let mut eq_properties = EquivalenceProperties::new(schema); - eq_properties.add_new_orderings(vec![vec![PhysicalSortExpr::new_default( - Arc::new(Column::new("c1", 0)), - )]]); + eq_properties.add_new_orderings(vec![LexOrdering::new(vec![ + PhysicalSortExpr::new_default(Arc::new(Column::new("c1", 0))), + ])]); let mode = ExecutionMode::Unbounded; PlanProperties::new(eq_properties, Partitioning::UnknownPartitioning(1), mode) } @@ -1123,10 +1124,10 @@ mod tests { let schema = csv.schema(); let sort_exec = Arc::new(SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }], + }]), Arc::new(CoalescePartitionsExec::new(csv)), )); @@ -1166,10 +1167,10 @@ mod tests { let schema = input.schema(); let sort_exec = Arc::new(SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }], + }]), Arc::new(CoalescePartitionsExec::new(input)), )); @@ -1245,10 +1246,10 @@ mod tests { let sort_exec = Arc::new( SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, options: SortOptions::default(), - }], + }]), Arc::new(CoalescePartitionsExec::new(csv)), ) .with_fetch(fetch), @@ -1294,10 +1295,10 @@ mod tests { ); let sort_exec = Arc::new(SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("field_name", &schema)?, options: SortOptions::default(), - }], + }]), input, )); @@ -1345,7 +1346,7 @@ mod tests { )?; let sort_exec = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -1360,7 +1361,7 @@ mod tests { nulls_first: false, }, }, - ], + ]), Arc::new(MemoryExec::try_new( &[vec![batch]], Arc::clone(&schema), @@ -1435,7 +1436,7 @@ mod tests { )?; let sort_exec = Arc::new(SortExec::new( - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -1450,7 +1451,7 @@ mod tests { nulls_first: false, }, }, - ], + ]), Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None)?), )); @@ -1514,10 +1515,10 @@ mod tests { let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let refs = blocking_exec.refs(); let sort_exec = Arc::new(SortExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }], + }]), blocking_exec, )); @@ -1545,12 +1546,12 @@ mod tests { RecordBatch::try_new_with_options(Arc::clone(&schema), vec![], &options) .unwrap(); - let expressions = vec![PhysicalSortExpr { + let expressions = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Literal::new(ScalarValue::Int64(Some(1)))), options: SortOptions::default(), - }]; + }]); - let result = sort_batch(&batch, &expressions, None).unwrap(); + let result = sort_batch(&batch, expressions.as_ref(), None).unwrap(); assert_eq!(result.num_rows(), 1); } @@ -1564,9 +1565,9 @@ mod tests { cache: SortedUnboundedExec::compute_properties(Arc::new(schema.clone())), }; let mut plan = SortExec::new( - vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new(Column::new( "c1", 0, - )))], + )))]), Arc::new(source), ); plan = plan.with_fetch(Some(9)); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index f17161306c7a..ae39cfe412ba 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -21,7 +21,6 @@ use std::any::Any; use std::sync::Arc; use crate::common::spawn_buffered; -use crate::expressions::PhysicalSortExpr; use crate::limit::LimitStream; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::sorts::streaming_merge::StreamingMergeBuilder; @@ -35,7 +34,9 @@ use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{ + LexOrdering, LexOrderingRef, LexRequirement, +}; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -75,7 +76,7 @@ pub struct SortPreservingMergeExec { /// Input plan input: Arc, /// Sort expressions - expr: Vec, + expr: LexOrdering, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Optional number of rows to fetch. Stops producing rows after this fetch @@ -88,7 +89,7 @@ pub struct SortPreservingMergeExec { impl SortPreservingMergeExec { /// Create a new sort execution plan - pub fn new(expr: Vec, input: Arc) -> Self { + pub fn new(expr: LexOrdering, input: Arc) -> Self { let cache = Self::compute_properties(&input, expr.clone()); Self { input, @@ -121,7 +122,7 @@ impl SortPreservingMergeExec { } /// Sort expressions - pub fn expr(&self) -> &[PhysicalSortExpr] { + pub fn expr(&self) -> LexOrderingRef { &self.expr } @@ -133,7 +134,7 @@ impl SortPreservingMergeExec { /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. fn compute_properties( input: &Arc, - ordering: Vec, + ordering: LexOrdering, ) -> PlanProperties { let mut eq_properties = input.equivalence_properties().clone(); eq_properties.clear_per_partition_constants(); @@ -154,11 +155,7 @@ impl DisplayAs for SortPreservingMergeExec { ) -> std::fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { - write!( - f, - "SortPreservingMergeExec: [{}]", - PhysicalSortExpr::format_list(&self.expr) - )?; + write!(f, "SortPreservingMergeExec: [{}]", self.expr)?; if let Some(fetch) = self.fetch { write!(f, ", fetch={fetch}")?; }; @@ -208,7 +205,9 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn required_input_ordering(&self) -> Vec> { - vec![Some(PhysicalSortRequirement::from_sort_exprs(&self.expr))] + vec![Some(PhysicalSortRequirement::from_sort_exprs( + self.expr.iter(), + ))] } fn maintains_input_order(&self) -> Vec { @@ -290,7 +289,7 @@ impl ExecutionPlan for SortPreservingMergeExec { let result = StreamingMergeBuilder::new() .with_streams(receivers) .with_schema(schema) - .with_expressions(&self.expr) + .with_expressions(self.expr.as_ref()) .with_metrics(BaselineMetrics::new(&self.metrics, partition)) .with_batch_size(context.session_config().batch_size()) .with_fetch(self.fetch) @@ -354,6 +353,7 @@ mod tests { use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use futures::{FutureExt, Stream, StreamExt}; use tokio::time::timeout; @@ -384,7 +384,7 @@ mod tests { let rbs = (0..1024).map(|_| rb.clone()).collect::>(); let schema = rb.schema(); - let sort = vec![ + let sort = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), @@ -393,7 +393,7 @@ mod tests { expr: col("c", &schema).unwrap(), options: Default::default(), }, - ]; + ]); let exec = MemoryExec::try_new(&[rbs], schema, None).unwrap(); let repartition_exec = @@ -486,7 +486,7 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a)]).unwrap(); let schema = batch.schema(); - let sort = vec![]; // no sort expressions + let sort = LexOrdering::default(); // no sort expressions let exec = MemoryExec::try_new(&[vec![batch.clone()], vec![batch]], schema, None) .unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -665,7 +665,7 @@ mod tests { context: Arc, ) { let schema = partitions[0][0].schema(); - let sort = vec![ + let sort = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), @@ -674,7 +674,7 @@ mod tests { expr: col("c", &schema).unwrap(), options: Default::default(), }, - ]; + ]); let exec = MemoryExec::try_new(partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -684,7 +684,7 @@ mod tests { async fn sorted_merge( input: Arc, - sort: Vec, + sort: LexOrdering, context: Arc, ) -> RecordBatch { let merge = Arc::new(SortPreservingMergeExec::new(sort, input)); @@ -695,7 +695,7 @@ mod tests { async fn partition_sort( input: Arc, - sort: Vec, + sort: LexOrdering, context: Arc, ) -> RecordBatch { let sort_exec = @@ -705,7 +705,7 @@ mod tests { async fn basic_sort( src: Arc, - sort: Vec, + sort: LexOrdering, context: Arc, ) -> RecordBatch { let merge = Arc::new(CoalescePartitionsExec::new(src)); @@ -722,13 +722,13 @@ mod tests { let csv = test::scan_partitioned(partitions); let schema = csv.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: SortOptions { descending: true, nulls_first: true, }, - }]; + }]); let basic = basic_sort(Arc::clone(&csv), sort.clone(), Arc::clone(&task_ctx)).await; @@ -773,7 +773,7 @@ mod tests { } async fn sorted_partitioned_input( - sort: Vec, + sort: LexOrdering, sizes: &[usize], context: Arc, ) -> Result> { @@ -792,10 +792,10 @@ mod tests { async fn test_partition_sort_streaming_input() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = make_partition(11).schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: Default::default(), - }]; + }]); let input = sorted_partitioned_input(sort.clone(), &[10, 3, 11], Arc::clone(&task_ctx)) @@ -822,10 +822,10 @@ mod tests { #[tokio::test] async fn test_partition_sort_streaming_input_output() -> Result<()> { let schema = make_partition(11).schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: Default::default(), - }]; + }]); // Test streaming with default batch size let task_ctx = Arc::new(TaskContext::default()); @@ -897,7 +897,7 @@ mod tests { let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap(); let schema = b1.schema(); - let sort = vec![ + let sort = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { @@ -912,7 +912,7 @@ mod tests { nulls_first: false, }, }, - ]; + ]); let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -948,13 +948,13 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new( SortPreservingMergeExec::new(sort, Arc::new(exec)).with_fetch(Some(2)), @@ -984,13 +984,13 @@ mod tests { let batch = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = batch.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&[vec![batch]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -1017,10 +1017,10 @@ mod tests { async fn test_async() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let schema = make_partition(11).schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), options: SortOptions::default(), - }]; + }]); let batches = sorted_partitioned_input(sort.clone(), &[5, 7, 3], Arc::clone(&task_ctx)) @@ -1056,7 +1056,7 @@ mod tests { let merge_stream = StreamingMergeBuilder::new() .with_streams(streams) .with_schema(batches.schema()) - .with_expressions(sort.as_slice()) + .with_expressions(sort.as_ref()) .with_metrics(BaselineMetrics::new(&metrics, 0)) .with_batch_size(task_ctx.session_config().batch_size()) .with_fetch(fetch) @@ -1096,10 +1096,10 @@ mod tests { let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b)]).unwrap(); let schema = b1.schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), options: Default::default(), - }]; + }]); let exec = MemoryExec::try_new(&[vec![b1], vec![b2]], schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -1155,10 +1155,10 @@ mod tests { let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 2)); let refs = blocking_exec.refs(); let sort_preserving_merge_exec = Arc::new(SortPreservingMergeExec::new( - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions::default(), - }], + }]), blocking_exec, )); @@ -1203,13 +1203,13 @@ mod tests { let schema = partitions[0][0].schema(); - let sort = vec![PhysicalSortExpr { + let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("value", &schema).unwrap(), options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let exec = MemoryExec::try_new(&partitions, schema, None).unwrap(); let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec))); @@ -1272,7 +1272,7 @@ mod tests { eq_properties.add_new_orderings(vec![columns .iter() .map(|expr| PhysicalSortExpr::new_default(Arc::clone(expr))) - .collect::>()]); + .collect::()]); let mode = ExecutionMode::Unbounded; PlanProperties::new(eq_properties, Partitioning::Hash(columns, 3), mode) } @@ -1381,9 +1381,9 @@ mod tests { congestion_cleared: Arc::new(Mutex::new(false)), }; let spm = SortPreservingMergeExec::new( - vec![PhysicalSortExpr::new_default(Arc::new(Column::new( + LexOrdering::new(vec![PhysicalSortExpr::new_default(Arc::new(Column::new( "c1", 0, - )))], + )))]), Arc::new(source), ); let spm_task = SpawnedTask::spawn(collect(Arc::new(spm), task_ctx)); diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index c7924edfb1eb..70beb2c4a91b 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -24,6 +24,7 @@ use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use futures::stream::{Fuse, StreamExt}; use std::marker::PhantomData; use std::sync::Arc; @@ -92,7 +93,7 @@ pub struct RowCursorStream { impl RowCursorStream { pub fn try_new( schema: &Schema, - expressions: &[PhysicalSortExpr], + expressions: LexOrderingRef, streams: Vec, reservation: MemoryReservation, ) -> Result { diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index e8330a7cabc0..bd74685eac94 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -23,11 +23,12 @@ use crate::sorts::{ merge::SortPreservingMergeStream, stream::{FieldCursorStream, RowCursorStream}, }; -use crate::{PhysicalSortExpr, SendableRecordBatchStream}; +use crate::SendableRecordBatchStream; use arrow::datatypes::{DataType, SchemaRef}; use arrow_array::*; use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr_common::sort_expr::LexOrderingRef; macro_rules! primitive_merge_helper { ($t:ty, $($v:ident),+) => { @@ -54,7 +55,7 @@ macro_rules! merge_helper { pub struct StreamingMergeBuilder<'a> { streams: Vec, schema: Option, - expressions: &'a [PhysicalSortExpr], + expressions: LexOrderingRef<'a>, metrics: Option, batch_size: Option, fetch: Option, @@ -80,7 +81,7 @@ impl<'a> StreamingMergeBuilder<'a> { self } - pub fn with_expressions(mut self, expressions: &'a [PhysicalSortExpr]) -> Self { + pub fn with_expressions(mut self, expressions: LexOrderingRef<'a>) -> Self { self.expressions = expressions; self } diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 9b46ad2ec7b1..14469ab6c0d9 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -24,6 +24,7 @@ use arrow::{ use std::mem::size_of; use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; +use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; use arrow_array::{Array, ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; use datafusion_common::Result; @@ -32,10 +33,9 @@ use datafusion_execution::{ runtime_env::RuntimeEnv, }; use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::HashMap; -use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; - use super::metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder}; /// Global TopK @@ -101,7 +101,7 @@ impl TopK { pub fn try_new( partition_id: usize, schema: SchemaRef, - expr: Vec, + expr: LexOrdering, k: usize, batch_size: usize, runtime: Arc, @@ -111,7 +111,7 @@ impl TopK { let reservation = MemoryConsumer::new(format!("TopK[{partition_id}]")) .register(&runtime.memory_pool); - let expr: Arc<[PhysicalSortExpr]> = expr.into(); + let expr: Arc<[PhysicalSortExpr]> = expr.inner.into(); let sort_fields: Vec<_> = expr .iter() diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 433dda870def..69cc63f8f65d 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -607,6 +607,7 @@ mod tests { use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; + use datafusion_physical_expr_common::sort_expr::LexOrdering; // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) fn create_test_schema() -> Result { @@ -625,14 +626,14 @@ mod tests { // Convert each tuple to PhysicalSortExpr fn convert_to_sort_exprs( in_data: &[(&Arc, SortOptions)], - ) -> Vec { + ) -> LexOrdering { in_data .iter() .map(|(expr, options)| PhysicalSortExpr { expr: Arc::clone(*expr), options: *options, }) - .collect::>() + .collect::() } #[tokio::test] 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 6495657339fa..2c60be49a480 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -28,7 +28,6 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; -use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -60,7 +59,7 @@ use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::raw::RawTable; @@ -149,7 +148,7 @@ impl BoundedWindowAggExec { // We are sure that partition by columns are always at the beginning of sort_keys // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points - pub fn partition_by_sort_keys(&self) -> Result> { + pub fn partition_by_sort_keys(&self) -> Result { let partition_by = self.window_expr()[0].partition_by(); get_partition_by_sort_exprs( &self.input, @@ -261,7 +260,7 @@ impl ExecutionPlan for BoundedWindowAggExec { .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys)] + vec![calc_requirements(partition_bys, order_keys.iter())] } fn required_input_distribution(&self) -> Vec { @@ -707,7 +706,7 @@ impl LinearSearch { /// when computing partitions. pub struct SortedSearch { /// Stores partition by columns and their ordering information - partition_by_sort_keys: Vec, + partition_by_sort_keys: LexOrdering, /// Input ordering and partition by key ordering need not be the same, so /// this vector stores the mapping between them. For instance, if the input /// is ordered by a, b and the window expression contains a PARTITION BY b, a @@ -1160,6 +1159,7 @@ mod tests { use std::time::Duration; use crate::common::collect; + use crate::expressions::PhysicalSortExpr; use crate::memory::MemoryExec; use crate::projection::ProjectionExec; use crate::streaming::{PartitionStream, StreamingTableExec}; @@ -1184,8 +1184,9 @@ mod tests { use datafusion_physical_expr::window::{ BuiltInWindowExpr, BuiltInWindowFunctionExpr, }; - use datafusion_physical_expr::{LexOrdering, PhysicalExpr, PhysicalSortExpr}; + use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; + use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1286,10 +1287,10 @@ mod tests { Arc::new(Column::new(schema.fields[0].name(), 0)) as Arc; let args = vec![col_expr]; let partitionby_exprs = vec![col(hash, &schema)?]; - let orderby_exprs = vec![PhysicalSortExpr { + let orderby_exprs = LexOrdering::new(vec![PhysicalSortExpr { expr: col(order_by, &schema)?, options: SortOptions::default(), - }]; + }]); let window_frame = WindowFrame::new_bounds( WindowFrameUnits::Range, WindowFrameBound::CurrentRow, @@ -1306,7 +1307,7 @@ mod tests { fn_name, &args, &partitionby_exprs, - &orderby_exprs, + orderby_exprs.as_ref(), Arc::new(window_frame), &input.schema(), false, @@ -1403,13 +1404,13 @@ mod tests { } fn schema_orders(schema: &SchemaRef) -> Result> { - let orderings = vec![vec![PhysicalSortExpr { + let orderings = vec![LexOrdering::new(vec![PhysicalSortExpr { expr: col("sn", schema)?, options: SortOptions { descending: false, nulls_first: false, }, - }]]; + }])]; Ok(orderings) } @@ -1552,7 +1553,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( last_value_func, &[], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1563,7 +1564,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( nth_value_func1, &[], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1574,7 +1575,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( nth_value_func2, &[], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1716,8 +1717,8 @@ mod tests { let plan = projection_exec(window)?; let expected_plan = vec![ - "ProjectionExec: expr=[sn@0 as sn, hash@1 as hash, count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]@2 as col_2]", - " BoundedWindowAggExec: wdw=[count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]: Ok(Field { name: \"count([Column { name: \\\"sn\\\", index: 0 }]) PARTITION BY: [[Column { name: \\\"hash\\\", index: 1 }]], ORDER BY: [[PhysicalSortExpr { expr: Column { name: \\\"sn\\\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }]]\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Linear]", + "ProjectionExec: expr=[sn@0 as sn, hash@1 as hash, count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [LexOrdering { inner: [PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }] }]@2 as col_2]", + " BoundedWindowAggExec: wdw=[count([Column { name: \"sn\", index: 0 }]) PARTITION BY: [[Column { name: \"hash\", index: 1 }]], ORDER BY: [LexOrdering { inner: [PhysicalSortExpr { expr: Column { name: \"sn\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }] }]: Ok(Field { name: \"count([Column { name: \\\"sn\\\", index: 0 }]) PARTITION BY: [[Column { name: \\\"hash\\\", index: 1 }]], ORDER BY: [LexOrdering { inner: [PhysicalSortExpr { expr: Column { name: \\\"sn\\\", index: 0 }, options: SortOptions { descending: false, nulls_first: true } }] }]\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Linear]", " StreamingTableExec: partition_sizes=1, projection=[sn, hash], infinite_source=true, output_ordering=[sn@0 ASC NULLS LAST]", ]; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 7ebb7e71ec57..217823fb6a0a 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -53,7 +53,7 @@ use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, }; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrderingRef, LexRequirement}; pub use window_agg_exec::WindowAggExec; /// Build field from window function and add it into schema @@ -98,7 +98,7 @@ pub fn create_window_expr( name: String, args: &[Arc], partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: LexOrderingRef, window_frame: Arc, input_schema: &Schema, ignore_nulls: bool, @@ -139,7 +139,7 @@ pub fn create_window_expr( /// Creates an appropriate [`WindowExpr`] based on the window frame and fn window_expr_from_aggregate_expr( partition_by: &[Arc], - order_by: &[PhysicalSortExpr], + order_by: LexOrderingRef, window_frame: Arc, aggregate: Arc, ) -> Arc { @@ -497,7 +497,7 @@ pub fn get_best_fitting_window( /// the mode this window operator should work in to accommodate the existing ordering. pub fn get_window_mode( partitionby_exprs: &[Arc], - orderby_keys: &[PhysicalSortExpr], + orderby_keys: LexOrderingRef, input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); @@ -516,9 +516,9 @@ pub fn get_window_mode( // Treat partition by exprs as constant. During analysis of requirements are satisfied. let const_exprs = partitionby_exprs.iter().map(ConstExpr::from); let partition_by_eqs = input_eqs.with_constants(const_exprs); - let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); + let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys.iter()); let reverse_order_by_reqs = - PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); + PhysicalSortRequirement::from_sort_exprs(reverse_order_bys(orderby_keys).iter()); for (should_swap, order_by_reqs) in [(false, order_by_reqs), (true, reverse_order_by_reqs)] { @@ -699,7 +699,7 @@ mod tests { "count".to_owned(), &[col("a", &schema)?], &[], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new(None)), schema.as_ref(), false, @@ -896,7 +896,7 @@ mod tests { partition_by_exprs.push(col(col_name, &test_schema)?); } - let mut order_by_exprs = vec![]; + let mut order_by_exprs = LexOrdering::default(); for col_name in order_by_params { let expr = col(col_name, &test_schema)?; // Give default ordering, this is same with input ordering direction @@ -904,8 +904,11 @@ mod tests { let options = SortOptions::default(); order_by_exprs.push(PhysicalSortExpr { expr, options }); } - let res = - get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded); + let res = get_window_mode( + &partition_by_exprs, + order_by_exprs.as_ref(), + &exec_unbounded, + ); // Since reversibility is not important in this test. Convert Option<(bool, InputOrderMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -1058,7 +1061,7 @@ mod tests { partition_by_exprs.push(col(col_name, &test_schema)?); } - let mut order_by_exprs = vec![]; + let mut order_by_exprs = LexOrdering::default(); for (col_name, descending, nulls_first) in order_by_params { let expr = col(col_name, &test_schema)?; let options = SortOptions { @@ -1069,7 +1072,7 @@ mod tests { } assert_eq!( - get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded), + get_window_mode(&partition_by_exprs, order_by_exprs.as_ref(), &exec_unbounded), *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index afe9700ed08c..1318f36f269e 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -23,7 +23,6 @@ use std::sync::Arc; use std::task::{Context, Poll}; use super::utils::create_schema; -use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, @@ -43,7 +42,7 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{evaluate_partition_ranges, transpose}; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; /// Window execution plan @@ -105,7 +104,7 @@ impl WindowAggExec { // We are sure that partition by columns are always at the beginning of sort_keys // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points - pub fn partition_by_sort_keys(&self) -> Result> { + pub fn partition_by_sort_keys(&self) -> Result { let partition_by = self.window_expr()[0].partition_by(); get_partition_by_sort_exprs( &self.input, @@ -195,13 +194,13 @@ impl ExecutionPlan for WindowAggExec { let partition_bys = self.window_expr()[0].partition_by(); let order_keys = self.window_expr()[0].order_by(); if self.ordered_partition_by_indices.len() < partition_bys.len() { - vec![calc_requirements(partition_bys, order_keys)] + vec![calc_requirements(partition_bys, order_keys.iter())] } else { let partition_bys = self .ordered_partition_by_indices .iter() .map(|idx| &partition_bys[*idx]); - vec![calc_requirements(partition_bys, order_keys)] + vec![calc_requirements(partition_bys, order_keys.iter())] } } @@ -282,7 +281,7 @@ pub struct WindowAggStream { batches: Vec, finished: bool, window_expr: Vec>, - partition_by_sort_keys: Vec, + partition_by_sort_keys: LexOrdering, baseline_metrics: BaselineMetrics, ordered_partition_by_indices: Vec, } @@ -294,7 +293,7 @@ impl WindowAggStream { window_expr: Vec>, input: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, - partition_by_sort_keys: Vec, + partition_by_sort_keys: LexOrdering, ordered_partition_by_indices: Vec, ) -> Result { // In WindowAggExec all partition by columns should be ordered. diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 20ec5eeaeaf8..316166042fc4 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -35,7 +35,7 @@ use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{FileScanConfig, FileSinkConfig}; use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::WindowFunctionDefinition; -use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, Literal, NegativeExpr, NotExpr, TryCastExpr, @@ -99,13 +99,13 @@ pub fn parse_physical_sort_exprs( registry: &dyn FunctionRegistry, input_schema: &Schema, codec: &dyn PhysicalExtensionCodec, -) -> Result> { +) -> Result { proto .iter() .map(|sort_expr| { parse_physical_sort_expr(sort_expr, registry, input_schema, codec) }) - .collect::>>() + .collect::>() } /// Parses a physical window expr from a protobuf. @@ -175,7 +175,7 @@ pub fn parse_physical_window_expr( name, &window_node_expr, &partition_by, - &order_by, + order_by.as_ref(), Arc::new(window_frame), &extended_schema, false, diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 326c7acab392..e84eae2b9082 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -35,7 +35,7 @@ use datafusion::datasource::physical_plan::{AvroExec, CsvExec}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; use datafusion::physical_expr::aggregate::AggregateFunctionExpr; -use datafusion::physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; +use datafusion::physical_expr::{LexOrdering, PhysicalExprRef, PhysicalSortRequirement}; use datafusion::physical_plan::aggregates::AggregateMode; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; @@ -501,8 +501,9 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ExprType::AggregateExpr(agg_node) => { let input_phy_expr: Vec> = agg_node.expr.iter() .map(|e| parse_physical_expr(e, registry, &physical_schema, extension_codec)).collect::>>()?; - let ordering_req: Vec = agg_node.ordering_req.iter() - .map(|e| parse_physical_sort_expr(e, registry, &physical_schema, extension_codec)).collect::>>()?; + let ordering_req: LexOrdering = agg_node.ordering_req.iter() + .map(|e| parse_physical_sort_expr(e, registry, &physical_schema, extension_codec)) + .collect::>()?; agg_node.aggregate_function.as_ref().map(|func| { match func { AggregateFunction::UserDefinedAggrFunction(udaf_name) => { @@ -874,7 +875,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ) } }) - .collect::, _>>()?; + .collect::>()?; let fetch = if sort.fetch < 0 { None } else { @@ -921,7 +922,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { ) } }) - .collect::, _>>()?; + .collect::>()?; let fetch = if sort.fetch < 0 { None } else { @@ -1036,7 +1037,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) + .map(|item| PhysicalSortRequirement::from_sort_exprs(&item.inner)) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1066,7 +1067,7 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) + .map(|item| PhysicalSortRequirement::from_sort_exprs(&item.inner)) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( @@ -1103,7 +1104,9 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { &sink_schema, extension_codec, ) - .map(|item| PhysicalSortRequirement::from_sort_exprs(&item)) + .map(|item| { + PhysicalSortRequirement::from_sort_exprs(&item.inner) + }) }) .transpose()?; Ok(Arc::new(DataSinkExec::new( diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 89a2403922e9..4bf7e353326e 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -21,7 +21,7 @@ use std::sync::Arc; #[cfg(feature = "parquet")] use datafusion::datasource::file_format::parquet::ParquetSink; use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr}; -use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, InListExpr, IsNotNullExpr, IsNullExpr, Literal, NegativeExpr, NotExpr, NthValue, TryCastExpr, @@ -52,7 +52,10 @@ pub fn serialize_physical_aggr_expr( codec: &dyn PhysicalExtensionCodec, ) -> Result { let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; - let ordering_req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); + let ordering_req = match aggr_expr.order_bys() { + Some(order) => LexOrdering::from_ref(order), + None => LexOrdering::default(), + }; let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; let name = aggr_expr.fun().name().to_string(); diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 4a9bf6afb49e..1e078ee410c6 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -52,7 +52,8 @@ use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{ - LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, + LexOrdering, LexOrderingRef, LexRequirement, PhysicalSortRequirement, + ScalarFunctionExpr, }; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -307,7 +308,7 @@ fn roundtrip_window() -> Result<()> { .build() .map(Arc::new)?, &[], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new(None)), )); @@ -327,7 +328,7 @@ fn roundtrip_window() -> Result<()> { let sliding_aggr_window_expr = Arc::new(SlidingAggregateWindowExpr::new( sum_expr, &[], - &[], + LexOrderingRef::default(), Arc::new(window_frame), )); @@ -459,13 +460,13 @@ fn rountrip_aggregate_with_sort() -> Result<()> { let groups: Vec<(Arc, String)> = vec![(col("a", &schema)?, "unused".to_string())]; - let sort_exprs = vec![PhysicalSortExpr { + let sort_exprs = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema)?, options: SortOptions { descending: false, nulls_first: true, }, - }]; + }]); let aggregates = vec![ @@ -585,7 +586,7 @@ fn roundtrip_sort() -> Result<()> { let field_a = Field::new("a", DataType::Boolean, false); let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let sort_exprs = vec![ + let sort_exprs = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -600,7 +601,7 @@ fn roundtrip_sort() -> Result<()> { nulls_first: true, }, }, - ]; + ]); roundtrip_test(Arc::new(SortExec::new( sort_exprs, Arc::new(EmptyExec::new(schema)), @@ -612,7 +613,7 @@ fn roundtrip_sort_preserve_partitioning() -> Result<()> { let field_a = Field::new("a", DataType::Boolean, false); let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let sort_exprs = vec![ + let sort_exprs = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, options: SortOptions { @@ -627,7 +628,7 @@ fn roundtrip_sort_preserve_partitioning() -> Result<()> { nulls_first: true, }, }, - ]; + ]); roundtrip_test(Arc::new(SortExec::new( sort_exprs.clone(), @@ -1013,7 +1014,7 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr.clone(), &[col("author", &schema)?], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new(None)), ))], filter, @@ -1074,7 +1075,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr, &[col("author", &schema)?], - &[], + LexOrderingRef::default(), Arc::new(WindowFrame::new(None)), ))], filter, @@ -1298,17 +1299,17 @@ fn roundtrip_sym_hash_join() -> Result<()> { ] { for left_order in &[ None, - Some(vec![PhysicalSortExpr { + Some(LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("col", schema_left.index_of("col")?)), options: Default::default(), - }]), + }])), ] { for right_order in &[ None, - Some(vec![PhysicalSortExpr { + Some(LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("col", schema_right.index_of("col")?)), options: Default::default(), - }]), + }])), ] { roundtrip_test(Arc::new( datafusion::physical_plan::joins::SymmetricHashJoinExec::try_new( diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index aac315ea0efd..917e037682f2 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -146,7 +146,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] -04)------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST], preserve_partitioning=[true] +04)------SortExec: expr=[c2@1 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[true] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 60569803322c..53ca8d81b9e4 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -158,7 +158,7 @@ logical_plan 07)--------Filter: balances.time < Int64(10) 08)----------TableScan: balances physical_plan -01)SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[time@0 ASC NULLS LAST, name@1 ASC NULLS LAST, account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--RecursiveQueryExec: name=balances, is_distinct=false 03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true 04)----CoalescePartitionsExec diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index 3b4deae3326f..d96044fda8c0 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -37,7 +37,7 @@ logical_plan 02)--Filter: data.ticker = Utf8("A") 03)----TableScan: data projection=[date, ticker, time] physical_plan -01)SortPreservingMergeExec: [date@0 ASC NULLS LAST,time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -105,7 +105,7 @@ logical_plan 02)--Filter: data.ticker = Utf8("A") AND CAST(data.time AS Date32) = data.date 03)----TableScan: data projection=[date, ticker, time] physical_plan -01)SortPreservingMergeExec: [time@2 ASC NULLS LAST,date@0 ASC NULLS LAST] +01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -146,7 +146,7 @@ logical_plan 02)--Filter: data.date = Date32("2006-01-02") 03)----TableScan: data projection=[date, ticker, time] physical_plan -01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST,time@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: date@0 = 2006-01-02 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 61b3ad73cd0a..daf270190870 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2250,7 +2250,7 @@ logical_plan 01)Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -01)PartialSortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +01)PartialSortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] 02)--StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT @@ -2263,7 +2263,7 @@ logical_plan 01)Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST, fetch=50 02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -01)PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +01)PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], common_prefix_length=[2] 02)--StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT @@ -2275,7 +2275,7 @@ logical_plan 01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST 02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d] physical_plan -01)SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST], preserve_partitioning=[false] 02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true query TT @@ -2288,7 +2288,7 @@ logical_plan 02)--TableScan: annotated_data_infinite2 projection=[a, b, d] physical_plan 01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[array_agg(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], ordering_mode=Sorted -02)--PartialSortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@2 ASC NULLS LAST], common_prefix_length=[2] +02)--PartialSortExec: expr=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, d@2 ASC NULLS LAST], common_prefix_length=[2] 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] # as can be seen in the result below d is indeed ordered. @@ -2535,7 +2535,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted -03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST,amount@1 DESC], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2573,7 +2573,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, sum(s.amount)@3 as sum1] 02)--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[array_agg(s.amount) ORDER BY [s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=PartiallySorted([0]) -03)----SortExec: TopK(fetch=10), expr=[country@1 ASC NULLS LAST,amount@2 DESC], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=10), expr=[country@1 ASC NULLS LAST, amount@2 DESC], preserve_partitioning=[false] 04)------MemoryExec: partitions=1, partition_sizes=[1] query TI?R rowsort @@ -2646,7 +2646,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[country@0 as country, array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, sum(s.amount)@2 as sum1] 02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[array_agg(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], sum(s.amount)], ordering_mode=Sorted -03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST,amount@1 DESC], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=10), expr=[country@0 ASC NULLS LAST, amount@1 DESC], preserve_partitioning=[false] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -4328,7 +4328,7 @@ logical_plan 02)--Projection: unbounded_csv_with_timestamps2.name, date_bin(IntervalMonthDayNano("IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }"), unbounded_csv_with_timestamps2.ts) AS time_chunks 03)----TableScan: unbounded_csv_with_timestamps2 projection=[name, ts] physical_plan -01)SortPreservingMergeExec: [name@0 DESC,time_chunks@1 DESC], fetch=5 +01)SortPreservingMergeExec: [name@0 DESC, time_chunks@1 DESC], fetch=5 02)--ProjectionExec: expr=[name@0 as name, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@1) as time_chunks] 03)----RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 04)------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 230ea4d98fc3..804612287246 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -69,7 +69,7 @@ physical_plan 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] 04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -129,7 +129,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -180,7 +180,7 @@ physical_plan 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] 04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as a1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as a2, c1@0 as c1] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index c40f62c3ba80..35decd728eed 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -126,7 +126,7 @@ logical_plan 03)----Values: (Int64(5), Int64(1)), (Int64(4), Int64(2)), (Int64(7), Int64(7)), (Int64(7), Int64(8)), (Int64(7), Int64(9))... physical_plan 01)DataSinkExec: sink=CsvSink(file_groups=[]) -02)--SortExec: expr=[a@0 ASC NULLS LAST,b@1 DESC], preserve_partitioning=[false] +02)--SortExec: expr=[a@0 ASC NULLS LAST, b@1 DESC], preserve_partitioning=[false] 03)----ProjectionExec: expr=[column1@0 as a, column2@1 as b] 04)------ValuesExec @@ -358,7 +358,7 @@ physical_plan 03)----SortPreservingMergeExec: [c1@2 ASC NULLS LAST] 04)------ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, c1@0 as c1] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -419,7 +419,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as field1, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as field2] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@2 ASC NULLS LAST], preserve_partitioning=[true] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index eb47ccdd43ec..44c5ed0d2dab 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -95,7 +95,7 @@ logical_plan 08)--------Filter: annotated_data.d = Int32(3) 09)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan -01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 +01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index bc40f845cc8a..93bb1f1f548e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3243,13 +3243,13 @@ physical_plan 01)SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] 02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@5 ASC NULLS LAST +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 07)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true 09)----CoalesceBatchesExec: target_batch_size=2 -10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 12)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true @@ -3277,11 +3277,11 @@ physical_plan 01)SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] 02)--SortMergeJoin: join_type=Right, on=[(a@1, a@1)] 03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true 07)----CoalesceBatchesExec: target_batch_size=2 -08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@5 ASC NULLS LAST +08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST 09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 11)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] @@ -3315,8 +3315,8 @@ logical_plan 09)--------WindowAggr: windowExpr=[[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] 10)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -01)SortPreservingMergeExec: [a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] -02)--SortExec: expr=[a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST] +02)--SortExec: expr=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@11 ASC NULLS LAST], preserve_partitioning=[true] 03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] 04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=2 diff --git a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt index d41b78dcd3f2..abf48fac5364 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -44,7 +44,7 @@ logical_plan 02)--Projection: CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a_big@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +01)SortPreservingMergeExec: [a_big@0 ASC NULLS LAST, b@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true @@ -60,7 +60,7 @@ logical_plan 02)--Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true @@ -81,7 +81,7 @@ logical_plan 02)--Projection: multiple_ordered_table.a, CAST(multiple_ordered_table.a AS Int64) AS a_big, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a_big@1 ASC NULLS LAST,b@2 ASC NULLS LAST] +01)SortPreservingMergeExec: [a_big@1 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] 03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true @@ -132,8 +132,8 @@ logical_plan 02)--Projection: CAST(multiple_ordered_table.a AS Utf8) AS a_str, multiple_ordered_table.b 03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -01)SortPreservingMergeExec: [a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST] -02)--SortExec: expr=[a_str@0 ASC NULLS LAST,b@1 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST] +02)--SortExec: expr=[a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index dd73447f8b25..d5f0521407c5 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -165,7 +165,7 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] -02)--SortExec: expr=[c2@1 ASC NULLS LAST,c3@2 ASC NULLS LAST], preserve_partitioning=[false] +02)--SortExec: expr=[c2@1 ASC NULLS LAST, c3@2 ASC NULLS LAST], preserve_partitioning=[false] 03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true query II @@ -691,7 +691,7 @@ logical_plan 01)Sort: t1.id DESC NULLS FIRST, t1.name ASC NULLS LAST 02)--TableScan: t1 projection=[id, name] physical_plan -01)SortExec: expr=[id@0 DESC,name@1 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[id@0 DESC, name@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--MemoryExec: partitions=1, partition_sizes=[1] query IT @@ -710,7 +710,7 @@ logical_plan 01)Sort: t1.id ASC NULLS LAST, t1.name ASC NULLS LAST 02)--TableScan: t1 projection=[id, name] physical_plan -01)SortExec: expr=[id@0 ASC NULLS LAST,name@1 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: expr=[id@0 ASC NULLS LAST, name@1 ASC NULLS LAST], preserve_partitioning=[false] 02)--MemoryExec: partitions=1, partition_sizes=[1] @@ -776,8 +776,8 @@ logical_plan 13)------------Projection: column1 AS t 14)--------------Values: (Int64(0)), (Int64(1)) physical_plan -01)SortPreservingMergeExec: [m@0 ASC NULLS LAST,t@1 ASC NULLS LAST] -02)--SortExec: expr=[m@0 ASC NULLS LAST,t@1 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [m@0 ASC NULLS LAST, t@1 ASC NULLS LAST] +02)--SortExec: expr=[m@0 ASC NULLS LAST, t@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----InterleaveExec 04)------ProjectionExec: expr=[0 as m, t@0 as t] 05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] @@ -1237,12 +1237,12 @@ logical_plan 09)----------TableScan: ordered_table projection=[a0, b, c, d] physical_plan 01)ProjectionExec: expr=[b@0 as b, c@1 as c, a@2 as a, a0@3 as a0] -02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], fetch=2 +02)--SortPreservingMergeExec: [d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a@2 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], fetch=2 03)----UnionExec -04)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a@2 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +04)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a@2 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 05)--------ProjectionExec: expr=[b@1 as b, c@2 as c, a@0 as a, NULL as a0, d@3 as d] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[c@2 ASC NULLS LAST], has_header=true -07)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST,c@1 ASC NULLS LAST,a0@3 ASC NULLS LAST,b@0 ASC NULLS LAST], preserve_partitioning=[false] +07)------SortExec: TopK(fetch=2), expr=[d@4 ASC NULLS LAST, c@1 ASC NULLS LAST, a0@3 ASC NULLS LAST, b@0 ASC NULLS LAST], preserve_partitioning=[false] 08)--------ProjectionExec: expr=[b@1 as b, c@2 as c, NULL as a, a0@0 as a0, d@3 as d] 09)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, b, c, d], output_ordering=[c@2 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 656cfcbe076d..ed963466fca6 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -89,8 +89,8 @@ logical_plan 01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, string_col] physical_plan -01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] -02)--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] +02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col] # Tear down test_table: @@ -119,7 +119,7 @@ logical_plan 01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, string_col] physical_plan -01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] 02)--ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet]]}, projection=[int_col, string_col], output_ordering=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] # Add another file to the directory underlying test_table @@ -141,8 +141,8 @@ logical_plan 01)Sort: test_table.string_col ASC NULLS LAST, test_table.int_col ASC NULLS LAST 02)--TableScan: test_table projection=[int_col, string_col] physical_plan -01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST] -02)--SortExec: expr=[string_col@1 ASC NULLS LAST,int_col@0 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST] +02)--SortExec: expr=[string_col@1 ASC NULLS LAST, int_col@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/0.parquet, WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/test_table/2.parquet]]}, projection=[int_col, string_col] diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index d7ff51011b65..c096f6e692af 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1439,7 +1439,7 @@ logical_plan 02)--Filter: annotated_data_finite2.a = Int32(0) 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0)] physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1481,7 +1481,7 @@ logical_plan 02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan -01)SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1502,7 +1502,7 @@ logical_plan 02)--Filter: annotated_data_finite2.a = Int32(0) AND annotated_data_finite2.b = Int32(0) 03)----TableScan: annotated_data_finite2 projection=[a0, a, b, c, d], partial_filters=[annotated_data_finite2.a = Int32(0), annotated_data_finite2.b = Int32(0)] physical_plan -01)SortPreservingMergeExec: [a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1553,7 +1553,7 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(*)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 06)----------ProjectionExec: expr=[c2@0 as c2] -07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST,c2@0 ASC NULLS LAST], preserve_partitioning=[false] +07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], has_header=true # FilterExec can track equality of non-column expressions. diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index e4360a9269ca..a3717dd838d6 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -65,8 +65,8 @@ logical_plan 05)--------TableScan: sink_table projection=[c1, c2, c3, c9] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] -02)--SortExec: expr=[c1@0 ASC NULLS LAST,c3@2 ASC NULLS LAST,c9@3 ASC NULLS LAST], preserve_partitioning=[false] -03)----SortExec: TopK(fetch=2), expr=[c1@0 DESC,c3@2 ASC NULLS LAST], preserve_partitioning=[false] +02)--SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST, c9@3 ASC NULLS LAST], preserve_partitioning=[false] +03)----SortExec: TopK(fetch=2), expr=[c1@0 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[false] 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], has_header=true @@ -98,7 +98,7 @@ logical_plan 07)------------TableScan: sink_table projection=[c1, c3, c9] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, r@1 as r] -02)--SortExec: TopK(fetch=2), expr=[c1@0 ASC NULLS LAST,c3@2 ASC NULLS LAST,c9@3 ASC NULLS LAST], preserve_partitioning=[false] +02)--SortExec: TopK(fetch=2), expr=[c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST, c9@3 ASC NULLS LAST], preserve_partitioning=[false] 03)----ProjectionExec: expr=[c1@0 as c1, rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as r, c3@1 as c3, c9@2 as c9] 04)------BoundedWindowAggExec: wdw=[rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "rank() ORDER BY [sink_table.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Utf8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 05)--------SortExec: expr=[c1@0 DESC], preserve_partitioning=[false] @@ -118,8 +118,8 @@ logical_plan 07)------------TableScan: sink_table projection=[c1, c2, c3, c9] physical_plan 01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2] -02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST,c3@2 DESC,c9@3 ASC NULLS LAST] -03)----SortExec: expr=[c1@0 ASC NULLS LAST,c3@2 DESC,c9@3 ASC NULLS LAST], preserve_partitioning=[true] +02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST, c3@2 DESC, c9@3 ASC NULLS LAST] +03)----SortExec: expr=[c1@0 ASC NULLS LAST, c3@2 DESC, c9@3 ASC NULLS LAST], preserve_partitioning=[true] 04)------ProjectionExec: expr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@1 as c1, first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@2 as c2, first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@3 as c3, first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]@4 as c9] 05)--------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] 06)----------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 8cfd25d26c07..4d4323e93e9e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -47,8 +47,8 @@ logical_plan 05)--------Filter: lineitem.l_shipdate <= Date32("1998-09-02") 06)----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("1998-09-02")] physical_plan -01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] -02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST] +02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST, l_linestatus@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, sum(lineitem.l_quantity)@2 as sum_qty, sum(lineitem.l_extendedprice)@3 as sum_base_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, avg(lineitem.l_quantity)@6 as avg_qty, avg(lineitem.l_extendedprice)@7 as avg_price, avg(lineitem.l_discount)@8 as avg_disc, count(*)@9 as count_order] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus], aggr=[sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax), avg(lineitem.l_quantity), avg(lineitem.l_extendedprice), avg(lineitem.l_discount), count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index bb32fb209700..2a9fb12a31c2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -53,8 +53,8 @@ logical_plan 11)------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") 12)--------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] physical_plan -01)SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC], preserve_partitioning=[true] +01)SortPreservingMergeExec: [custdist@1 DESC, c_count@0 DESC], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[custdist@1 DESC, c_count@0 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[c_count@0 as c_count, count(*)@1 as custdist] 04)------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 8058371764f2..6b2c2f7fdc3e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -65,8 +65,8 @@ logical_plan 13)--------------Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") 14)----------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] physical_plan -01)SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC, p_brand@0 ASC NULLS LAST, p_type@1 ASC NULLS LAST, p_size@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, count(alias1)@3 as supplier_cnt] 04)------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[count(alias1)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index e78b0d87f651..c80352c5d36a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -67,8 +67,8 @@ logical_plan 14)------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[sum(lineitem.l_quantity)]] 15)--------------TableScan: lineitem projection=[l_orderkey, l_quantity] physical_plan -01)SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] -02)--SortExec: expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST] +02)--SortExec: expr=[o_totalprice@4 DESC, o_orderdate@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[sum(lineitem.l_quantity)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 17f3b78a089d..23ffa0d226b8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -99,8 +99,8 @@ logical_plan 34)------------------Filter: region.r_name = Utf8("EUROPE") 35)--------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] physical_plan -01)SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[s_acctbal@0 DESC, n_name@2 ASC NULLS LAST, s_name@1 ASC NULLS LAST, p_partkey@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, min(partsupp.ps_supplycost)@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, n_name@8] diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index 5cf069ec7248..93dcd4c68052 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -90,8 +90,8 @@ logical_plan 30)----------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate 31)------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan -01)SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] -02)--SortExec: expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [numwait@1 DESC, s_name@0 ASC NULLS LAST] +02)--SortExec: expr=[numwait@1 DESC, s_name@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[s_name@0 as s_name, count(*)@1 as numwait] 04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[count(*)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index 16a1c2b6ebb1..289e9c7732bb 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -58,8 +58,8 @@ logical_plan 14)------------Filter: lineitem.l_shipdate > Date32("1995-03-15") 15)--------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("1995-03-15")] physical_plan -01)SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[revenue@1 DESC, o_orderdate@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[l_orderkey@0 as l_orderkey, sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] 04)------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[sum(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 5a6cef5311d4..a16af4710478 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -84,8 +84,8 @@ logical_plan 24)--------------Filter: nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE") 25)----------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY") OR nation.n_name = Utf8("FRANCE")] physical_plan -01)SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST] -02)--SortExec: expr=[supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS LAST,l_year@2 ASC NULLS LAST], preserve_partitioning=[true] +01)SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST] +02)--SortExec: expr=[supp_nation@0 ASC NULLS LAST, cust_nation@1 ASC NULLS LAST, l_year@2 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year, sum(shipping.volume)@3 as revenue] 04)------AggregateExec: mode=FinalPartitioned, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[sum(shipping.volume)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index b3631f07cc8f..c4910beb842b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -75,8 +75,8 @@ logical_plan 21)----------------TableScan: orders projection=[o_orderkey, o_orderdate] 22)------------TableScan: nation projection=[n_nationkey, n_name] physical_plan -01)SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC], fetch=10 -02)--SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST,o_year@1 DESC], preserve_partitioning=[true] +01)SortPreservingMergeExec: [nation@0 ASC NULLS LAST, o_year@1 DESC], fetch=10 +02)--SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST, o_year@1 DESC], preserve_partitioning=[true] 03)----ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, sum(profit.amount)@2 as sum_profit] 04)------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[sum(profit.amount)] 05)--------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 29ff62ab34f9..d593a985c458 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -361,7 +361,7 @@ physical_plan 03)----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[max(d.a), max(d.seq)], ordering_mode=Sorted 04)------ProjectionExec: expr=[row_number() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] 05)--------BoundedWindowAggExec: wdw=[row_number() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortExec: expr=[b@1 ASC NULLS LAST, a@0 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 09)----------------UnionExec @@ -1244,7 +1244,7 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----ProjectionExec: expr=[c9@1 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST], preserve_partitioning=[false] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c8@0 ASC NULLS LAST], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true @@ -1265,7 +1265,7 @@ physical_plan 02)--WindowAggExec: wdw=[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 03)----BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[false] +05)--------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true @@ -1288,9 +1288,9 @@ physical_plan 02)--ProjectionExec: expr=[c2@0 as c2, max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 03)----WindowAggExec: wdw=[sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] 04)------BoundedWindowAggExec: wdw=[max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "max(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST], preserve_partitioning=[false] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 06)----------BoundedWindowAggExec: wdw=[min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "min(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[false] +07)------------SortExec: expr=[c2@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true # test_window_partition_by_order_by @@ -1312,12 +1312,12 @@ logical_plan physical_plan 01)ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -03)----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[true] +03)----SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 04)------CoalesceBatchesExec: target_batch_size=4096 05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 07)------------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[true] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 09)----------------CoalesceBatchesExec: target_batch_size=4096 10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1470,10 +1470,10 @@ physical_plan 01)ProjectionExec: expr=[c9@2 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2] 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -04)------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] +04)------SortExec: expr=[c9@2 ASC NULLS LAST, c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] 05)--------BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c9@2 DESC,c1@0 DESC], preserve_partitioning=[false] +07)------------SortExec: expr=[c9@2 DESC, c1@0 DESC], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true query IIII @@ -1554,17 +1554,17 @@ physical_plan 03)----WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] 04)------ProjectionExec: expr=[c1@0 as c1, c3@2 as c3, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@4 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@6 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@7 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@8 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@9 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@10 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@11 as sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING@12 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@13 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING@14 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING@15 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@16 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@17 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@18 as sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 05)--------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] +06)----------SortExec: expr=[c3@2 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] 07)------------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -08)--------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC], preserve_partitioning=[false] +08)--------------SortExec: expr=[c3@2 ASC NULLS LAST, c1@0 ASC], preserve_partitioning=[false] 09)----------------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST, null_cases.c1 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -10)------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC], preserve_partitioning=[false] +10)------------------SortExec: expr=[c3@2 ASC NULLS LAST, c1@0 DESC], preserve_partitioning=[false] 11)--------------------WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }] 12)----------------------WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] 13)------------------------SortExec: expr=[c3@2 DESC NULLS LAST], preserve_partitioning=[false] 14)--------------------------WindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)), is_causal: false }, sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)), is_causal: false }] 15)----------------------------BoundedWindowAggExec: wdw=[sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST, null_cases.c1 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -16)------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST], preserve_partitioning=[false] +16)------------------------------SortExec: expr=[c3@2 DESC, c1@0 ASC NULLS LAST], preserve_partitioning=[false] 17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII @@ -1639,7 +1639,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[false] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true @@ -1683,7 +1683,7 @@ physical_plan 02)--GlobalLimitExec: skip=0, fetch=5 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[false] +05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[false] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III @@ -1730,8 +1730,8 @@ physical_plan 03)----WindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)), is_causal: false }] 04)------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, c3@2 as c3, c9@3 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 05)--------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -06)----------SortPreservingMergeExec: [__common_expr_1@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -07)------------SortExec: expr=[__common_expr_1@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST], preserve_partitioning=[true] +06)----------SortPreservingMergeExec: [__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST] +07)------------SortExec: expr=[__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------ProjectionExec: expr=[c3@1 + c4@2 as __common_expr_1, c2@0 as c2, c3@1 as c3, c9@3 as c9] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true @@ -1823,13 +1823,13 @@ physical_plan 01)SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[c3@0 as c3, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -04)------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC], preserve_partitioning=[true] +04)------SortExec: expr=[c3@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=4096 06)----------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 08)--------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 09)----------------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -10)------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST], preserve_partitioning=[false] +10)------------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true @@ -1991,7 +1991,7 @@ logical_plan 03)----WindowAggr: windowExpr=[[row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] 04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST, rn1@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[c1@0 as c1, row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] 03)----BoundedWindowAggExec: wdw=[row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] @@ -2023,7 +2023,7 @@ physical_plan 04)------SortPreservingMergeExec: [c9@1 ASC NULLS LAST] 05)--------SortExec: expr=[c9@1 ASC NULLS LAST], preserve_partitioning=[true] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)), is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], preserve_partitioning=[true] +07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------CoalesceBatchesExec: target_batch_size=4096 09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -2112,7 +2112,7 @@ physical_plan 05)--------WindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 07)------------WindowAggExec: wdw=[sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST], preserve_partitioning=[false] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true @@ -2168,7 +2168,7 @@ physical_plan 06)----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] 07)------------BoundedWindowAggExec: wdw=[sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 08)--------------WindowAggExec: wdw=[sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "sum(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(NULL)), is_causal: false }] -09)----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST], preserve_partitioning=[false] +09)----------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST, c9@3 ASC NULLS LAST, c8@2 ASC NULLS LAST], preserve_partitioning=[false] 10)------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] 11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true @@ -2211,7 +2211,7 @@ physical_plan 04)------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(3)), is_causal: true }], mode=[Sorted] 05)--------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING] 06)----------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -07)------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], preserve_partitioning=[false] +07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true query RR @@ -2356,7 +2356,7 @@ logical_plan 03)----WindowAggr: windowExpr=[[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: aggregate_test_100 projection=[c9] physical_plan -01)SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST], preserve_partitioning=[false] +01)SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST, c9@0 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[c9@0 as c9, row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] 03)----BoundedWindowAggExec: wdw=[row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "row_number() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] @@ -3051,15 +3051,15 @@ physical_plan 01)SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST], preserve_partitioning=[false] 02)--ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] 03)----BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)), is_causal: true }], mode=[Sorted] -04)------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +04)------SortExec: expr=[d@4 ASC NULLS LAST, a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 05)--------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -06)----------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +06)----------SortExec: expr=[b@2 ASC NULLS LAST, a@1 ASC NULLS LAST, d@4 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 07)------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -08)--------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +08)--------------SortExec: expr=[b@2 ASC NULLS LAST, a@1 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 09)----------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] -10)------------------SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +10)------------------SortExec: expr=[a@1 ASC NULLS LAST, d@4 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 11)--------------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] -12)----------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST], preserve_partitioning=[false] +12)----------------------SortExec: expr=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, d@4 ASC NULLS LAST, c@3 ASC NULLS LAST], preserve_partitioning=[false] 13)------------------------BoundedWindowAggExec: wdw=[sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)), is_causal: false }, sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "sum(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] 14)--------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 15)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -3144,7 +3144,7 @@ logical_plan 03)----WindowAggr: windowExpr=[[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] 04)------TableScan: aggregate_test_100 projection=[c9] physical_plan -01)SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC], preserve_partitioning=[false] +01)SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST, c9@0 DESC], preserve_partitioning=[false] 02)--ProjectionExec: expr=[c9@0 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] 03)----BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 04)------SortExec: expr=[c9@0 DESC], preserve_partitioning=[false] @@ -3264,17 +3264,17 @@ physical_plan 01)ProjectionExec: expr=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] 02)--BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] 03)----CoalesceBatchesExec: target_batch_size=4096 -04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=__common_expr_1@0 ASC NULLS LAST,a@1 ASC NULLS LAST +04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=__common_expr_1@0 ASC NULLS LAST, a@1 ASC NULLS LAST 05)--------ProjectionExec: expr=[__common_expr_1@0 as __common_expr_1, a@1 as a, d@4 as d, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 06)----------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 07)------------CoalesceBatchesExec: target_batch_size=4096 -08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST +08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 09)----------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] 10)------------------CoalesceBatchesExec: target_batch_size=4096 -11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST +11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 12)----------------------BoundedWindowAggExec: wdw=[sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 13)------------------------CoalesceBatchesExec: target_batch_size=4096 -14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,__common_expr_1@0 ASC NULLS LAST +14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] 16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] @@ -3624,7 +3624,7 @@ physical_plan 02)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] 03)----BoundedWindowAggExec: wdw=[avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] 04)------CoalesceBatchesExec: target_batch_size=4096 -05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 07)------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] From 24d953e19145e5202fd57973a9cd62e7ec96795b Mon Sep 17 00:00:00 2001 From: Jonathan Chen Date: Fri, 1 Nov 2024 20:17:31 -0400 Subject: [PATCH 07/28] fix: array_resize null fix (#13209) * array_resize null fix * comment * clippy * fixes --- datafusion/functions-nested/src/resize.rs | 36 ++++++++++++-- datafusion/sqllogictest/test_files/array.slt | 49 +++++++++++++++++++- 2 files changed, 80 insertions(+), 5 deletions(-) diff --git a/datafusion/functions-nested/src/resize.rs b/datafusion/functions-nested/src/resize.rs index 294076a52b52..b0255e7be2a3 100644 --- a/datafusion/functions-nested/src/resize.rs +++ b/datafusion/functions-nested/src/resize.rs @@ -19,8 +19,10 @@ use crate::utils::make_scalar_function; use arrow::array::{Capacities, MutableArrayData}; -use arrow_array::{ArrayRef, GenericListArray, Int64Array, OffsetSizeTrait}; -use arrow_buffer::{ArrowNativeType, OffsetBuffer}; +use arrow_array::{ + new_null_array, Array, ArrayRef, GenericListArray, Int64Array, OffsetSizeTrait, +}; +use arrow_buffer::{ArrowNativeType, BooleanBufferBuilder, NullBuffer, OffsetBuffer}; use arrow_schema::DataType::{FixedSizeList, LargeList, List}; use arrow_schema::{DataType, FieldRef}; use datafusion_common::cast::{as_int64_array, as_large_list_array, as_list_array}; @@ -134,6 +136,23 @@ pub(crate) fn array_resize_inner(arg: &[ArrayRef]) -> Result { return exec_err!("array_resize needs two or three arguments"); } + let array = &arg[0]; + + // Checks if entire array is null + if array.null_count() == array.len() { + let return_type = match array.data_type() { + List(field) => List(Arc::clone(field)), + LargeList(field) => LargeList(Arc::clone(field)), + _ => { + return exec_err!( + "array_resize does not support type '{:?}'.", + array.data_type() + ) + } + }; + return Ok(new_null_array(&return_type, array.len())); + } + let new_len = as_int64_array(&arg[1])?; let new_element = if arg.len() == 3 { Some(Arc::clone(&arg[2])) @@ -184,7 +203,16 @@ fn general_list_resize>( capacity, ); + let mut null_builder = BooleanBufferBuilder::new(array.len()); + for (row_index, offset_window) in array.offsets().windows(2).enumerate() { + if array.is_null(row_index) { + null_builder.append(false); + offsets.push(offsets[row_index]); + continue; + } + null_builder.append(true); + let count = count_array.value(row_index).to_usize().ok_or_else(|| { internal_datafusion_err!("array_resize: failed to convert size to usize") })?; @@ -211,10 +239,12 @@ fn general_list_resize>( } let data = mutable.freeze(); + let null_bit_buffer: NullBuffer = null_builder.finish().into(); + Ok(Arc::new(GenericListArray::::try_new( Arc::clone(field), OffsetBuffer::::new(offsets.into()), arrow_array::make_array(data), - None, + Some(null_bit_buffer), )?)) } diff --git a/datafusion/sqllogictest/test_files/array.slt b/datafusion/sqllogictest/test_files/array.slt index bfdbfb1bcc5e..1e60699a1f65 100644 --- a/datafusion/sqllogictest/test_files/array.slt +++ b/datafusion/sqllogictest/test_files/array.slt @@ -6985,7 +6985,7 @@ select array_resize(column1, column2, column3) from arrays_values; [11, 12, 13, 14, 15, 16, 17, 18, , 20, 2, 2] [21, 22, 23, , 25, 26, 27, 28, 29, 30, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3] [31, 32, 33, 34, 35, , 37, 38, 39, 40, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4] -[5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5] +NULL [] [51, 52, , 54, 55, 56, 57, 58, 59, 60, , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , ] [61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7] @@ -6997,7 +6997,7 @@ select array_resize(arrow_cast(column1, 'LargeList(Int64)'), column2, column3) f [11, 12, 13, 14, 15, 16, 17, 18, , 20, 2, 2] [21, 22, 23, , 25, 26, 27, 28, 29, 30, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3] [31, 32, 33, 34, 35, , 37, 38, 39, 40, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4, 4] -[5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5, 5] +NULL [] [51, 52, , 54, 55, 56, 57, 58, 59, 60, , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , , ] [61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7, 7] @@ -7013,6 +7013,51 @@ select array_resize(arrow_cast([[1], [2], [3]], 'LargeList(List(Int64))'), 10, [ ---- [[1], [2], [3], [5], [5], [5], [5], [5], [5], [5]] +# array_resize null value +query ? +select array_resize(arrow_cast(NULL, 'List(Int8)'), 1); +---- +NULL + +statement ok +CREATE TABLE array_resize_values +AS VALUES + (make_array(1, NULL, 3, 4, 5, 6, 7, 8, 9, 10), 2, 1), + (make_array(11, 12, NULL, 14, 15, 16, 17, 18, 19, 20), 5, 2), + (make_array(21, 22, 23, 24, NULL, 26, 27, 28, 29, 30), 8, 3), + (make_array(31, 32, 33, 34, 35, 36, NULL, 38, 39, 40), 12, 4), + (NULL, 3, 0), + (make_array(41, 42, 43, 44, 45, 46, 47, 48, 49, 50), NULL, 6), + (make_array(51, 52, 53, 54, 55, NULL, 57, 58, 59, 60), 13, NULL), + (make_array(61, 62, 63, 64, 65, 66, 67, 68, 69, 70), 15, 7) +; + +# array_resize columnar test #1 +query ? +select array_resize(column1, column2, column3) from array_resize_values; +---- +[1, ] +[11, 12, , 14, 15] +[21, 22, 23, 24, , 26, 27, 28] +[31, 32, 33, 34, 35, 36, , 38, 39, 40, 4, 4] +NULL +[] +[51, 52, 53, 54, 55, , 57, 58, 59, 60, , , ] +[61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 7, 7, 7, 7, 7] + +# array_resize columnar test #2 +query ? +select array_resize(arrow_cast(column1, 'LargeList(Int64)'), column2, column3) from array_resize_values; +---- +[1, ] +[11, 12, , 14, 15] +[21, 22, 23, 24, , 26, 27, 28] +[31, 32, 33, 34, 35, 36, , 38, 39, 40, 4, 4] +NULL +[] +[51, 52, 53, 54, 55, , 57, 58, 59, 60, , , ] +[61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 7, 7, 7, 7, 7] + ## array_reverse query ?? select array_reverse(make_array(1, 2, 3)), array_reverse(make_array(1)); From 344f0897d105cdd588d7a7ee8cef0646f844937a Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Sat, 2 Nov 2024 11:22:21 +0100 Subject: [PATCH 08/28] Revert "Improve push down filter of join (#13184)" (#13229) This reverts commit 7ae1ccb4971474d376e5bed1d7116fcaf23f906f. --- datafusion/optimizer/src/push_down_filter.rs | 337 ++-------- datafusion/optimizer/src/utils.rs | 8 - .../join_disable_repartition_joins.slt | 23 +- .../test_files/push_down_filter_join.slt | 612 ------------------ 4 files changed, 71 insertions(+), 909 deletions(-) delete mode 100644 datafusion/sqllogictest/test_files/push_down_filter_join.slt diff --git a/datafusion/optimizer/src/push_down_filter.rs b/datafusion/optimizer/src/push_down_filter.rs index 269ce2910074..acb7ba0fa757 100644 --- a/datafusion/optimizer/src/push_down_filter.rs +++ b/datafusion/optimizer/src/push_down_filter.rs @@ -36,9 +36,7 @@ use datafusion_expr::{ }; use crate::optimizer::ApplyOrder; -use crate::utils::{ - contain_all_columns, has_all_column_refs, is_restrict_null_predicate, -}; +use crate::utils::{has_all_column_refs, is_restrict_null_predicate}; use crate::{OptimizerConfig, OptimizerRule}; /// Optimizer rule for pushing (moving) filter expressions down in a plan so @@ -217,19 +215,19 @@ impl<'a> ColumnChecker<'a> { } /// Return true if the expression references only columns from the left side of the join - fn left_only(&mut self, column_refs: &HashSet<&Column>) -> bool { + fn is_left_only(&mut self, predicate: &Expr) -> bool { if self.left_columns.is_none() { self.left_columns = Some(schema_columns(self.left_schema)); } - contain_all_columns(column_refs, self.left_columns.as_ref().unwrap()) + has_all_column_refs(predicate, self.left_columns.as_ref().unwrap()) } /// Return true if the expression references only columns from the right side of the join - fn right_only(&mut self, column_refs: &HashSet<&Column>) -> bool { + fn is_right_only(&mut self, predicate: &Expr) -> bool { if self.right_columns.is_none() { self.right_columns = Some(schema_columns(self.right_schema)); } - contain_all_columns(column_refs, self.right_columns.as_ref().unwrap()) + has_all_column_refs(predicate, self.right_columns.as_ref().unwrap()) } } @@ -413,13 +411,10 @@ fn extract_or_clause(expr: &Expr, schema_columns: &HashSet) -> Option, - inferred_from_predicates: Vec, + inferred_join_predicates: Vec, mut join: Join, on_filter: Vec, - inferred_from_on_filter: Vec, ) -> Result> { - assert_ne!(join.join_type, JoinType::Full); - let is_inner_join = join.join_type == JoinType::Inner; // Get pushable predicates from current optimizer state let (left_preserved, right_preserved) = lr_is_preserved(join.join_type); @@ -435,24 +430,14 @@ fn push_down_all_join( let mut keep_predicates = vec![]; let mut join_conditions = vec![]; let mut checker = ColumnChecker::new(left_schema, right_schema); - for predicate in predicates { - let columns = predicate.column_refs(); - macro_rules! restrict_null { - () => {{ - let predicate_cloned = predicate.clone(); - let cols = columns.iter().cloned(); - is_restrict_null_predicate(predicate_cloned, cols).unwrap_or(false) - }}; - } - - if checker.left_only(&columns) && (left_preserved || restrict_null!()) { + if left_preserved && checker.is_left_only(&predicate) { left_push.push(predicate); - } else if checker.right_only(&columns) && (right_preserved || restrict_null!()) { + } else if right_preserved && checker.is_right_only(&predicate) { right_push.push(predicate); } else if is_inner_join && can_evaluate_as_join_condition(&predicate)? { - // Here we do not differ it is eq or non-eq predicate, ExtractEquijoinPredicate will - // extract the eq predicate and convert to the join on condition + // Here we do not differ it is eq or non-eq predicate, ExtractEquijoinPredicate will extract the eq predicate + // and convert to the join on condition join_conditions.push(predicate); } else { keep_predicates.push(predicate); @@ -460,13 +445,10 @@ fn push_down_all_join( } // For infer predicates, if they can not push through join, just drop them - // Because we check whether it is_restrict_null in the process of Infer, there is no need to - // check again - for predicate in inferred_from_predicates { - let columns = predicate.column_refs(); - if checker.left_only(&columns) { + for predicate in inferred_join_predicates { + if left_preserved && checker.is_left_only(&predicate) { left_push.push(predicate); - } else if checker.right_only(&columns) { + } else if right_preserved && checker.is_right_only(&predicate) { right_push.push(predicate); } } @@ -474,24 +456,15 @@ fn push_down_all_join( let mut on_filter_join_conditions = vec![]; let (on_left_preserved, on_right_preserved) = on_lr_is_preserved(join.join_type); - for on in on_filter { - let columns = on.column_refs(); - if on_left_preserved && checker.left_only(&columns) { - left_push.push(on) - } else if on_right_preserved && checker.right_only(&columns) { - right_push.push(on) - } else { - on_filter_join_conditions.push(on) - } - } - - // For infer predicates, if they can not push through join, just drop them - for on in inferred_from_on_filter { - let columns = on.column_refs(); - if on_left_preserved && checker.left_only(&columns) { - left_push.push(on) - } else if on_right_preserved && checker.right_only(&columns) { - right_push.push(on) + if !on_filter.is_empty() { + for on in on_filter { + if on_left_preserved && checker.is_left_only(&on) { + left_push.push(on) + } else if on_right_preserved && checker.is_right_only(&on) { + right_push.push(on) + } else { + on_filter_join_conditions.push(on) + } } } @@ -547,17 +520,6 @@ fn push_down_join( join: Join, parent_predicate: Option<&Expr>, ) -> Result> { - if matches!(join.join_type, JoinType::Full) { - let plan = LogicalPlan::Join(join); - return Ok(match parent_predicate { - Some(predicate) => Transformed::yes(LogicalPlan::Filter(Filter::try_new( - predicate.clone(), - Arc::new(plan), - )?)), - None => Transformed::no(plan), - }); - } - // Split the parent predicate into individual conjunctive parts. let predicates = parent_predicate .map_or_else(Vec::new, |pred| split_conjunction_owned(pred.clone())); @@ -569,24 +531,17 @@ fn push_down_join( .map_or_else(Vec::new, |filter| split_conjunction_owned(filter.clone())); // Are there any new join predicates that can be inferred from the filter expressions? - let (inferred_from_predicates, inferred_from_on_filter) = + let inferred_join_predicates = infer_join_predicates(&join, &predicates, &on_filters)?; if on_filters.is_empty() - && inferred_from_on_filter.is_empty() && predicates.is_empty() - && inferred_from_predicates.is_empty() + && inferred_join_predicates.is_empty() { return Ok(Transformed::no(LogicalPlan::Join(join))); } - push_down_all_join( - predicates, - inferred_from_predicates, - join, - on_filters, - inferred_from_on_filter, - ) + push_down_all_join(predicates, inferred_join_predicates, join, on_filters) } /// Extracts any equi-join join predicates from the given filter expressions. @@ -599,13 +554,11 @@ fn push_down_join( /// * `on_filters` filters from the join ON clause that have not already been /// identified as join predicates /// -/// # Return Value -/// A tuple of Expr Vec - (inferred_from_predicates, inferred_from_on_filters). fn infer_join_predicates( join: &Join, predicates: &[Expr], on_filters: &[Expr], -) -> Result<(Vec, Vec)> { +) -> Result> { // Only allow both side key is column. let join_col_keys = join .on @@ -626,7 +579,6 @@ fn infer_join_predicates( predicates, &mut inferred_predicates, )?; - let inferred_from_predicates = inferred_predicates.take_all(); infer_join_predicates_from_on_filters( &join_col_keys, @@ -634,9 +586,8 @@ fn infer_join_predicates( on_filters, &mut inferred_predicates, )?; - let inferred_from_on_filters = inferred_predicates.predicates; - Ok((inferred_from_predicates, inferred_from_on_filters)) + Ok(inferred_predicates.predicates) } /// Inferred predicates collector. @@ -660,12 +611,6 @@ impl InferredPredicates { } } - fn take_all(&mut self) -> Vec { - let mut temp = vec![]; - std::mem::swap(&mut self.predicates, &mut temp); - temp - } - fn try_build_predicate( &mut self, predicate: Expr, @@ -2165,10 +2110,11 @@ mod tests { // filter not duplicated nor pushed down - i.e. noop let expected = "\ - Left Join: Using test.a = test2.a\ - \n TableScan: test, full_filters=[test.a <= Int64(1)]\ - \n Projection: test2.a\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test2.a <= Int64(1)\ + \n Left Join: Using test.a = test2.a\ + \n TableScan: test, full_filters=[test.a <= Int64(1)]\ + \n Projection: test2.a\ + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -2202,10 +2148,11 @@ mod tests { // filter not duplicated nor pushed down - i.e. noop let expected = "\ - Right Join: Using test.a = test2.a\ - \n TableScan: test, full_filters=[test.a <= Int64(1)]\ - \n Projection: test2.a\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test.a <= Int64(1)\ + \n Right Join: Using test.a = test2.a\ + \n TableScan: test\ + \n Projection: test2.a\ + \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; assert_optimized_plan_eq(plan, expected) } @@ -2243,7 +2190,7 @@ mod tests { Left Join: Using test.a = test2.a\ \n TableScan: test, full_filters=[test.a <= Int64(1)]\ \n Projection: test2.a\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -2279,7 +2226,7 @@ mod tests { // filter sent to right side of join, not duplicated to the left let expected = "\ Right Join: Using test.a = test2.a\ - \n TableScan: test, full_filters=[test.a <= Int64(1)]\ + \n TableScan: test\ \n Projection: test2.a\ \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; assert_optimized_plan_eq(plan, expected) @@ -2328,47 +2275,6 @@ mod tests { assert_optimized_plan_eq(plan, expected) } - #[test] - fn join_with_non_restrict_null_predicate() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.b").is_null().and(col("test2.b").is_null()); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Inner, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.b IS NULL AND test2.b IS NULL\ - \n Inner Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = "\ - Inner Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test, full_filters=[test.b IS NULL]\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2, full_filters=[test2.b IS NULL]"; - assert_optimized_plan_eq(plan, expected) - } - /// join filter should be completely removed after pushdown #[test] fn join_filter_removed() -> Result<()> { @@ -2490,49 +2396,7 @@ mod tests { \n Projection: test.a, test.b, test.c\ \n TableScan: test\ \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2, full_filters=[test2.c > UInt32(4), test2.a > UInt32(1)]"; - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn left_join_with_non_restrict_null_predicate() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.b").is_null().and(col("test2.b").is_null()); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Left, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.b IS NULL AND test2.b IS NULL\ - \n Left Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = "\ - Filter: test2.b IS NULL\ - \n Left Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test, full_filters=[test.b IS NULL]\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2"; + \n TableScan: test2, full_filters=[test2.c > UInt32(4)]"; assert_optimized_plan_eq(plan, expected) } @@ -2579,87 +2443,6 @@ mod tests { assert_optimized_plan_eq(plan, expected) } - #[test] - fn right_join_with_non_restrict_null_predicate() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.b").is_null().and(col("test2.b").is_null()); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Right, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.b IS NULL AND test2.b IS NULL\ - \n Right Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = "\ - Filter: test.b IS NULL\ - \n Right Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2, full_filters=[test2.b IS NULL]"; - assert_optimized_plan_eq(plan, expected) - } - - #[test] - fn full_join() -> Result<()> { - let table_scan = test_table_scan()?; - let left = LogicalPlanBuilder::from(table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let right_table_scan = test_table_scan_with_name("test2")?; - let right = LogicalPlanBuilder::from(right_table_scan) - .project(vec![col("a"), col("b"), col("c")])? - .build()?; - let filter = col("test.a") - .gt(lit(1u32)) - .and(col("test.b").lt(col("test2.b"))) - .and(col("test2.c").gt(lit(4u32))); - let plan = LogicalPlanBuilder::from(left) - .join( - right, - JoinType::Full, - (vec![Column::from_name("a")], vec![Column::from_name("a")]), - None, - )? - .filter(filter)? - .build()?; - - // not part of the test, just good to know: - assert_eq!( - format!("{plan}"), - "Filter: test.a > UInt32(1) AND test.b < test2.b AND test2.c > UInt32(4)\ - \n Full Join: test.a = test2.a\ - \n Projection: test.a, test.b, test.c\ - \n TableScan: test\ - \n Projection: test2.a, test2.b, test2.c\ - \n TableScan: test2" - ); - - let expected = &format!("{plan}"); - assert_optimized_plan_eq(plan, expected) - } - /// single table predicate parts of ON condition should not be pushed #[test] fn full_join_on_with_filter() -> Result<()> { @@ -3183,10 +2966,11 @@ Projection: a, b // Inferred the predicate `test1.a <= Int64(1)` and push it down to the left side. let expected = "\ - LeftSemi Join: test1.a = test2.a\ - \n TableScan: test1, full_filters=[test1.a <= Int64(1)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test2.a <= Int64(1)\ + \n LeftSemi Join: test1.a = test2.a\ + \n TableScan: test1, full_filters=[test1.a <= Int64(1)]\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -3263,10 +3047,11 @@ Projection: a, b // Inferred the predicate `test2.a <= Int64(1)` and push it down to the right side. let expected = "\ - RightSemi Join: test1.a = test2.a\ - \n TableScan: test1, full_filters=[test1.a <= Int64(1)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; + Filter: test1.a <= Int64(1)\ + \n RightSemi Join: test1.a = test2.a\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2, full_filters=[test2.a <= Int64(1)]"; assert_optimized_plan_eq(plan, expected) } @@ -3347,11 +3132,12 @@ Projection: a, b // For left anti, filter of the right side filter can be pushed down. let expected = "\ - LeftAnti Join: test1.a = test2.a\ - \n Projection: test1.a, test1.b\ - \n TableScan: test1, full_filters=[test1.a > UInt32(2)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a > UInt32(2)]"; + Filter: test2.a > UInt32(2)\ + \n LeftAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1, full_filters=[test1.a > UInt32(2)]\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2"; assert_optimized_plan_eq(plan, expected) } @@ -3437,11 +3223,12 @@ Projection: a, b // For right anti, filter of the left side can be pushed down. let expected = "\ - RightAnti Join: test1.a = test2.a\ - \n Projection: test1.a, test1.b\ - \n TableScan: test1, full_filters=[test1.a > UInt32(2)]\ - \n Projection: test2.a, test2.b\ - \n TableScan: test2, full_filters=[test2.a > UInt32(2)]"; + Filter: test1.a > UInt32(2)\ + \n RightAnti Join: test1.a = test2.a\ + \n Projection: test1.a, test1.b\ + \n TableScan: test1\ + \n Projection: test2.a, test2.b\ + \n TableScan: test2, full_filters=[test2.a > UInt32(2)]"; assert_optimized_plan_eq(plan, expected) } diff --git a/datafusion/optimizer/src/utils.rs b/datafusion/optimizer/src/utils.rs index 9f8d7b7f97a0..9f325bc01b1d 100644 --- a/datafusion/optimizer/src/utils.rs +++ b/datafusion/optimizer/src/utils.rs @@ -79,14 +79,6 @@ pub fn optimize_children( /// Returns true if `expr` contains all columns in `schema_cols` pub(crate) fn has_all_column_refs(expr: &Expr, schema_cols: &HashSet) -> bool { let column_refs = expr.column_refs(); - contain_all_columns(&column_refs, schema_cols) -} - -/// Returns true if `column_refs` contains all columns in `schema_cols` -pub(crate) fn contain_all_columns( - column_refs: &HashSet<&Column>, - schema_cols: &HashSet, -) -> bool { // note can't use HashSet::intersect because of different types (owned vs References) schema_cols .iter() diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 44c5ed0d2dab..cf897d628da5 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -89,25 +89,20 @@ logical_plan 02)--Projection: t2.a AS a2, t2.b 03)----RightSemi Join: t1.d = t2.d, t1.c = t2.c 04)------SubqueryAlias: t1 -05)--------Filter: annotated_data.d = Int32(3) -06)----------TableScan: annotated_data projection=[c, d], partial_filters=[annotated_data.d = Int32(3)] -07)------SubqueryAlias: t2 -08)--------Filter: annotated_data.d = Int32(3) -09)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] +05)--------TableScan: annotated_data projection=[c, d] +06)------SubqueryAlias: t2 +07)--------Filter: annotated_data.d = Int32(3) +08)----------TableScan: annotated_data projection=[a, b, c, d], partial_filters=[annotated_data.d = Int32(3)] physical_plan 01)SortPreservingMergeExec: [a2@0 ASC NULLS LAST, b@1 ASC NULLS LAST], fetch=10 02)--ProjectionExec: expr=[a@0 as a2, b@1 as b] 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=10 04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] -05)--------CoalescePartitionsExec -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------FilterExec: d@1 = 3 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true -10)--------CoalesceBatchesExec: target_batch_size=8192 -11)----------FilterExec: d@3 = 3 -12)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -13)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true +06)--------CoalesceBatchesExec: target_batch_size=8192 +07)----------FilterExec: d@3 = 3 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/push_down_filter_join.slt b/datafusion/sqllogictest/test_files/push_down_filter_join.slt deleted file mode 100644 index f687c542a683..000000000000 --- a/datafusion/sqllogictest/test_files/push_down_filter_join.slt +++ /dev/null @@ -1,612 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -# Contents -# Inner Join -# `WHERE` clause -# `ON` clause -# Left Outer Join -# `WHERE` clause -# `ON` clause -# Right Outer Join -# `WHERE` clause -# `ON` clause -# Full Outer Join -# `WHERE` clause -# `ON` clause - -# Create table t1 -statement ok -CREATE TABLE t1(t1_id INT, t1_name VARCHAR) AS VALUES -(11, 'a'), -(22, 'b'), -(33, 'c'), -(44, 'd'), -(77, 'e'), -(88, NULL), -(99, NULL) - -# Create table t2 -statement ok -CREATE TABLE t2(t2_id INT, t2_name VARCHAR) AS VALUES -(11, 'z'), -(22, NULL), -(44, 'x'), -(55, 'w'), -(99, 'u') - -# Inner Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -44 d 44 x -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -## `ON` clause - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 INNER JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -# Left Outer Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d 44 x -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -## `ON` clause - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -query ITIT -SELECT * FROM t1 LEFT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL - -# Right Outer Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -44 d 44 x -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- - -## `ON` clause - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 RIGHT JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -# Full Outer Join - -## `WHERE` clause - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d 44 x -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -22 b 22 NULL -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t1.t1_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id WHERE t2.t2_id IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -33 c NULL NULL -77 e NULL NULL -88 NULL NULL NULL - -## `ON` clause - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name = 'a' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name = 'z' ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b NULL NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL 99 u -NULL NULL 11 z -NULL NULL 22 NULL -NULL NULL 44 x -NULL NULL 55 w - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_name IS NULL ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a NULL NULL -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 11 z -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t1.t1_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u - -query ITIT -SELECT * FROM t1 FULL JOIN t2 ON t1.t1_id = t2.t2_id AND t2.t2_id <= 22 ORDER BY t1_id, t1_name, t2_id, t2_name ----- -11 a 11 z -22 b 22 NULL -33 c NULL NULL -44 d NULL NULL -77 e NULL NULL -88 NULL NULL NULL -99 NULL NULL NULL -NULL NULL 44 x -NULL NULL 55 w -NULL NULL 99 u From 89e96b404f07900469fee31740f43edd8a410a10 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sat, 2 Nov 2024 06:24:37 -0400 Subject: [PATCH 09/28] Derive `Clone` for more ExecutionPlans (#13203) * Derive `Clone` for more ExecutionPlans * improve docs --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/coalesce_batches.rs | 2 +- datafusion/physical-plan/src/coalesce_partitions.rs | 2 +- datafusion/physical-plan/src/empty.rs | 2 +- datafusion/physical-plan/src/filter.rs | 2 +- datafusion/physical-plan/src/insert.rs | 1 + datafusion/physical-plan/src/joins/cross_join.rs | 5 +++++ datafusion/physical-plan/src/joins/hash_join.rs | 5 +++++ datafusion/physical-plan/src/joins/nested_loop_join.rs | 4 ++++ datafusion/physical-plan/src/joins/sort_merge_join.rs | 2 +- datafusion/physical-plan/src/joins/symmetric_hash_join.rs | 2 +- datafusion/physical-plan/src/limit.rs | 2 +- datafusion/physical-plan/src/memory.rs | 1 + datafusion/physical-plan/src/placeholder_row.rs | 2 +- datafusion/physical-plan/src/recursive_query.rs | 2 +- datafusion/physical-plan/src/repartition/mod.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- datafusion/physical-plan/src/sorts/sort_preserving_merge.rs | 2 +- datafusion/physical-plan/src/streaming.rs | 1 + datafusion/physical-plan/src/union.rs | 4 ++-- datafusion/physical-plan/src/unnest.rs | 2 +- datafusion/physical-plan/src/values.rs | 2 +- .../physical-plan/src/windows/bounded_window_agg_exec.rs | 2 +- datafusion/physical-plan/src/windows/window_agg_exec.rs | 2 +- 24 files changed, 36 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4193cc187e10..5ffe797c5c26 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -344,7 +344,7 @@ impl From for SendableRecordBatchStream { } /// Hash aggregate execution plan -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct AggregateExec { /// Aggregation mode (full, partial) mode: AggregateMode, diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 61fb3599f013..11678e7a4696 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -48,7 +48,7 @@ use futures::stream::{Stream, StreamExt}; /// reaches the `fetch` value. /// /// See [`BatchCoalescer`] for more information -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct CoalesceBatchesExec { /// The input plan input: Arc, diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index f9d4ec6a1a34..3da101d6092f 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -36,7 +36,7 @@ use datafusion_execution::TaskContext; /// Merge execution plan executes partitions in parallel and combines them into a single /// partition. No guarantees are made about the order of the resulting partition. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct CoalescePartitionsExec { /// Input execution plan input: Arc, diff --git a/datafusion/physical-plan/src/empty.rs b/datafusion/physical-plan/src/empty.rs index f6e0abb94fa8..192619f69f6a 100644 --- a/datafusion/physical-plan/src/empty.rs +++ b/datafusion/physical-plan/src/empty.rs @@ -35,7 +35,7 @@ use datafusion_physical_expr::EquivalenceProperties; use log::trace; /// Execution plan for empty relation with produce_one_row=false -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct EmptyExec { /// The schema for the produced row schema: SchemaRef, diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 30b0af19f43b..97d8159137f4 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -54,7 +54,7 @@ use log::trace; /// FilterExec evaluates a boolean predicate against all input batches to determine which rows to /// include in its output batches. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct FilterExec { /// The expression to filter on. This expression must evaluate to a boolean value. predicate: Arc, diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index 8b3ef5ae01e4..e478cecb7ffc 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -79,6 +79,7 @@ pub type FileSinkExec = DataSinkExec; /// Execution plan for writing record batches to a [`DataSink`] /// /// Returns a single row with the number of values written +#[derive(Clone)] pub struct DataSinkExec { /// Input plan that produces the record batches to be written. input: Arc, diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 8f49885068fd..a67e1df47bc7 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -49,8 +49,13 @@ use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the left side type JoinLeftData = (RecordBatch, MemoryReservation); +#[allow(rustdoc::private_intra_doc_links)] /// executes partitions in parallel and combines them into a set of /// partitions by combining all values from the left with all values on the right +/// +/// Note that the `Clone` trait is not implemented for this struct due to the +/// `left_fut` [`OnceAsync`], which is used to coordinate the loading of the +/// left side with the processing in each output stream. #[derive(Debug)] pub struct CrossJoinExec { /// left (build) side which gets loaded in memory diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index c56c179c17eb..57d8a9ce7b35 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -136,6 +136,7 @@ impl JoinLeftData { } } +#[allow(rustdoc::private_intra_doc_links)] /// Join execution plan: Evaluates eqijoin predicates in parallel on multiple /// partitions using a hash table and an optional filter list to apply post /// join. @@ -293,6 +294,10 @@ impl JoinLeftData { /// │ "dimension" │ │ "fact" │ /// └───────────────┘ └───────────────┘ /// ``` +/// +/// Note that the `Clone` trait is not implemented for this struct due to the +/// `left_fut` [`OnceAsync`], which is used to coordinate the loading of the +/// left side with the processing in each output stream. #[derive(Debug)] pub struct HashJoinExec { /// left (build) side which gets hashed diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index a87743565adf..f36c2395e20f 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -105,6 +105,7 @@ impl JoinLeftData { } } +#[allow(rustdoc::private_intra_doc_links)] /// NestedLoopJoinExec is build-probe join operator, whose main task is to /// perform joins without any equijoin conditions in `ON` clause. /// @@ -140,6 +141,9 @@ impl JoinLeftData { /// "reports" about probe phase completion (which means that "visited" bitmap won't be /// updated anymore), and only the last thread, reporting about completion, will return output. /// +/// Note that the `Clone` trait is not implemented for this struct due to the +/// `left_fut` [`OnceAsync`], which is used to coordinate the loading of the +/// left side with the processing in each output stream. #[derive(Debug)] pub struct NestedLoopJoinExec { /// left side diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 90dc407fcaed..3ad892c880f6 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -71,7 +71,7 @@ use crate::{ /// join execution plan executes partitions in parallel and combines them into a set of /// partitions. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct SortMergeJoinExec { /// Left sorted joining execution plan pub left: Arc, diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 81c13c652513..5b6dc2cd2ae9 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -167,7 +167,7 @@ const HASHMAP_SHRINK_SCALE_FACTOR: usize = 4; /// making the smallest value in 'left_sorted' 1231 and any rows below (since ascending) /// than that can be dropped from the inner buffer. /// ``` -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct SymmetricHashJoinExec { /// Left side stream pub(crate) left: Arc, diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 1fe550a93056..ab1e6cb37bc8 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -39,7 +39,7 @@ use futures::stream::{Stream, StreamExt}; use log::trace; /// Limit execution plan -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct GlobalLimitExec { /// Input execution plan input: Arc, diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 56ed144845a0..c9ada345afc7 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -40,6 +40,7 @@ use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use futures::Stream; /// Execution plan for reading in-memory batches of data +#[derive(Clone)] pub struct MemoryExec { /// The partitions to query partitions: Vec>, diff --git a/datafusion/physical-plan/src/placeholder_row.rs b/datafusion/physical-plan/src/placeholder_row.rs index 5d8ca7e76935..f9437f46f8a6 100644 --- a/datafusion/physical-plan/src/placeholder_row.rs +++ b/datafusion/physical-plan/src/placeholder_row.rs @@ -37,7 +37,7 @@ use datafusion_physical_expr::EquivalenceProperties; use log::trace; /// Execution plan for empty relation with produce_one_row=true -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct PlaceholderRowExec { /// The schema for the produced row schema: SchemaRef, diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index e9ea9d4f5032..cbf22a4b392f 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -53,7 +53,7 @@ use futures::{ready, Stream, StreamExt}; /// Note that there won't be any limit or checks applied to detect /// an infinite recursion, so it is up to the planner to ensure that /// it won't happen. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct RecursiveQueryExec { /// Name of the query handler name: String, diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 06144f98c89d..bc65b251561b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -399,7 +399,7 @@ impl BatchPartitioner { /// Paper](https://w6113.github.io/files/papers/volcanoparallelism-89.pdf) /// which uses the term "Exchange" for the concept of repartitioning /// data across threads. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct RepartitionExec { /// Input execution plan input: Arc, diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 32d6d3e0073c..d90d0f64ceb4 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -675,7 +675,7 @@ pub(crate) fn lexsort_to_indices_multi_columns( /// /// Support sorting datasets that are larger than the memory allotted /// by the memory manager, by spilling to disk. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct SortExec { /// Input schema pub(crate) input: Arc, diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index ae39cfe412ba..9ee0faaa0a44 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -71,7 +71,7 @@ use log::{debug, trace}; /// /// If any of the input partitions return an error, the error is propagated to /// the output and inputs are not polled again. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct SortPreservingMergeExec { /// Input plan input: Arc, diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index cdb94af1fe8a..7ccef3248069 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -55,6 +55,7 @@ pub trait PartitionStream: Debug + Send + Sync { /// /// If your source can be represented as one or more [`PartitionStream`]s, you can /// use this struct to implement [`ExecutionPlan`]. +#[derive(Clone)] pub struct StreamingTableExec { partitions: Vec>, projection: Option>, diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 69cc63f8f65d..bd36753880eb 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -85,7 +85,7 @@ use tokio::macros::support::thread_rng_n; /// │Input 1 │ │Input 2 │ /// └─────────────────┘ └──────────────────┘ /// ``` -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct UnionExec { /// Input execution plan inputs: Vec>, @@ -298,7 +298,7 @@ impl ExecutionPlan for UnionExec { /// | |-----------------+ /// +---------+ /// ``` -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct InterleaveExec { /// Input execution plan inputs: Vec>, diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 3e312b7451be..b7b9f17eb1b6 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -56,7 +56,7 @@ use log::trace; /// Thus the original RecordBatch with dimension (n x m) may have new dimension (n' x m') /// /// See [`UnnestOptions`] for more details and an example. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct UnnestExec { /// Input execution plan input: Arc, diff --git a/datafusion/physical-plan/src/values.rs b/datafusion/physical-plan/src/values.rs index 991146d245a7..edadf98cb10c 100644 --- a/datafusion/physical-plan/src/values.rs +++ b/datafusion/physical-plan/src/values.rs @@ -36,7 +36,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; /// Execution plan for values list based relation (produces constant rows) -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct ValuesExec { /// The schema schema: SchemaRef, 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 2c60be49a480..8c0331f94570 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -67,7 +67,7 @@ use indexmap::IndexMap; use log::debug; /// Window execution plan -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct BoundedWindowAggExec { /// Input plan input: Arc, diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 1318f36f269e..f71a0b9fd095 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -46,7 +46,7 @@ use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; /// Window execution plan -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct WindowAggExec { /// Input plan pub(crate) input: Arc, From b40a298a3a8e7eb0546c06168ef19b44b28acf42 Mon Sep 17 00:00:00 2001 From: Filippo Rossi Date: Sun, 3 Nov 2024 04:23:53 +0100 Subject: [PATCH 10/28] feat(logical-types): add NativeType and LogicalType (#12853) * [logical-types] add NativeType and LogicalType * Add license header * Add NativeField and derivates * Support TypeSignatures * Fix doc * Add documentation * Fix doc tests * Remove dummy test * From NativeField to LogicalField * Add default_cast_for * Add type order with can_cast_types * Rename NativeType Utf8 to String * NativeType from &DataType * Add builtin types * From LazyLock to OnceLock --- datafusion/common/src/lib.rs | 1 + datafusion/common/src/types/builtin.rs | 49 +++ datafusion/common/src/types/field.rs | 114 +++++++ datafusion/common/src/types/logical.rs | 128 ++++++++ datafusion/common/src/types/mod.rs | 26 ++ datafusion/common/src/types/native.rs | 399 +++++++++++++++++++++++++ 6 files changed, 717 insertions(+) create mode 100644 datafusion/common/src/types/builtin.rs create mode 100644 datafusion/common/src/types/field.rs create mode 100644 datafusion/common/src/types/logical.rs create mode 100644 datafusion/common/src/types/mod.rs create mode 100644 datafusion/common/src/types/native.rs diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index e4575038ab98..08431a36e82f 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -44,6 +44,7 @@ pub mod scalar; pub mod stats; pub mod test_util; pub mod tree_node; +pub mod types; pub mod utils; /// Reexport arrow crate diff --git a/datafusion/common/src/types/builtin.rs b/datafusion/common/src/types/builtin.rs new file mode 100644 index 000000000000..c6105d37c3bd --- /dev/null +++ b/datafusion/common/src/types/builtin.rs @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::types::{LogicalTypeRef, NativeType}; +use std::sync::{Arc, OnceLock}; + +macro_rules! singleton { + ($name:ident, $getter:ident, $ty:ident) => { + // TODO: Use LazyLock instead of getter function when MSRV gets bumped + static $name: OnceLock = OnceLock::new(); + + #[doc = "Getter for singleton instance of a logical type representing"] + #[doc = concat!("[`NativeType::", stringify!($ty), "`].")] + pub fn $getter() -> LogicalTypeRef { + Arc::clone($name.get_or_init(|| Arc::new(NativeType::$ty))) + } + }; +} + +singleton!(LOGICAL_NULL, logical_null, Null); +singleton!(LOGICAL_BOOLEAN, logical_boolean, Boolean); +singleton!(LOGICAL_INT8, logical_int8, Int8); +singleton!(LOGICAL_INT16, logical_int16, Int16); +singleton!(LOGICAL_INT32, logical_int32, Int32); +singleton!(LOGICAL_INT64, logical_int64, Int64); +singleton!(LOGICAL_UINT8, logical_uint8, UInt8); +singleton!(LOGICAL_UINT16, logical_uint16, UInt16); +singleton!(LOGICAL_UINT32, logical_uint32, UInt32); +singleton!(LOGICAL_UINT64, logical_uint64, UInt64); +singleton!(LOGICAL_FLOAT16, logical_float16, Float16); +singleton!(LOGICAL_FLOAT32, logical_float32, Float32); +singleton!(LOGICAL_FLOAT64, logical_float64, Float64); +singleton!(LOGICAL_DATE, logical_date, Date); +singleton!(LOGICAL_BINARY, logical_binary, Binary); +singleton!(LOGICAL_STRING, logical_string, String); diff --git a/datafusion/common/src/types/field.rs b/datafusion/common/src/types/field.rs new file mode 100644 index 000000000000..85c7c157272a --- /dev/null +++ b/datafusion/common/src/types/field.rs @@ -0,0 +1,114 @@ +// 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_schema::{Field, Fields, UnionFields}; +use std::hash::{Hash, Hasher}; +use std::{ops::Deref, sync::Arc}; + +use super::{LogicalTypeRef, NativeType}; + +/// A record of a logical type, its name and its nullability. +#[derive(Debug, Clone, Eq, PartialOrd, Ord)] +pub struct LogicalField { + pub name: String, + pub logical_type: LogicalTypeRef, + pub nullable: bool, +} + +impl PartialEq for LogicalField { + fn eq(&self, other: &Self) -> bool { + self.name == other.name + && self.logical_type.eq(&other.logical_type) + && self.nullable == other.nullable + } +} + +impl Hash for LogicalField { + fn hash(&self, state: &mut H) { + self.name.hash(state); + self.logical_type.hash(state); + self.nullable.hash(state); + } +} + +impl From<&Field> for LogicalField { + fn from(value: &Field) -> Self { + Self { + name: value.name().clone(), + logical_type: Arc::new(NativeType::from(value.data_type().clone())), + nullable: value.is_nullable(), + } + } +} + +/// A reference counted [`LogicalField`]. +pub type LogicalFieldRef = Arc; + +/// A cheaply cloneable, owned collection of [`LogicalFieldRef`]. +#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub struct LogicalFields(Arc<[LogicalFieldRef]>); + +impl Deref for LogicalFields { + type Target = [LogicalFieldRef]; + + fn deref(&self) -> &Self::Target { + self.0.as_ref() + } +} + +impl From<&Fields> for LogicalFields { + fn from(value: &Fields) -> Self { + value + .iter() + .map(|field| Arc::new(LogicalField::from(field.as_ref()))) + .collect() + } +} + +impl FromIterator for LogicalFields { + fn from_iter>(iter: T) -> Self { + Self(iter.into_iter().collect()) + } +} + +/// A cheaply cloneable, owned collection of [`LogicalFieldRef`] and their +/// corresponding type ids. +#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub struct LogicalUnionFields(Arc<[(i8, LogicalFieldRef)]>); + +impl Deref for LogicalUnionFields { + type Target = [(i8, LogicalFieldRef)]; + + fn deref(&self) -> &Self::Target { + self.0.as_ref() + } +} + +impl From<&UnionFields> for LogicalUnionFields { + fn from(value: &UnionFields) -> Self { + value + .iter() + .map(|(i, field)| (i, Arc::new(LogicalField::from(field.as_ref())))) + .collect() + } +} + +impl FromIterator<(i8, LogicalFieldRef)> for LogicalUnionFields { + fn from_iter>(iter: T) -> Self { + Self(iter.into_iter().collect()) + } +} diff --git a/datafusion/common/src/types/logical.rs b/datafusion/common/src/types/logical.rs new file mode 100644 index 000000000000..bde393992a0c --- /dev/null +++ b/datafusion/common/src/types/logical.rs @@ -0,0 +1,128 @@ +// 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 super::NativeType; +use crate::error::Result; +use arrow_schema::DataType; +use core::fmt; +use std::{cmp::Ordering, hash::Hash, sync::Arc}; + +/// Signature that uniquely identifies a type among other types. +#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub enum TypeSignature<'a> { + /// Represents a built-in native type. + Native(&'a NativeType), + /// Represents an arrow-compatible extension type. + /// () + /// + /// The `name` should contain the same value as 'ARROW:extension:name'. + Extension { + name: &'a str, + parameters: &'a [TypeParameter<'a>], + }, +} + +#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub enum TypeParameter<'a> { + Type(TypeSignature<'a>), + Number(i128), +} + +/// A reference counted [`LogicalType`]. +pub type LogicalTypeRef = Arc; + +/// Representation of a logical type with its signature and its native backing +/// type. +/// +/// The logical type is meant to be used during the DataFusion logical planning +/// phase in order to reason about logical types without worrying about their +/// underlying physical implementation. +/// +/// ### Extension types +/// +/// [`LogicalType`] is a trait in order to allow the possibility of declaring +/// extension types: +/// +/// ``` +/// use datafusion_common::types::{LogicalType, NativeType, TypeSignature}; +/// +/// struct JSON {} +/// +/// impl LogicalType for JSON { +/// fn native(&self) -> &NativeType { +/// &NativeType::String +/// } +/// +/// fn signature(&self) -> TypeSignature<'_> { +/// TypeSignature::Extension { +/// name: "JSON", +/// parameters: &[], +/// } +/// } +/// } +/// ``` +pub trait LogicalType: Sync + Send { + /// Get the native backing type of this logical type. + fn native(&self) -> &NativeType; + /// Get the unique type signature for this logical type. Logical types with identical + /// signatures are considered equal. + fn signature(&self) -> TypeSignature<'_>; + + /// Get the default physical type to cast `origin` to in order to obtain a physical type + /// that is logically compatible with this logical type. + fn default_cast_for(&self, origin: &DataType) -> Result { + self.native().default_cast_for(origin) + } +} + +impl fmt::Debug for dyn LogicalType { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_tuple("LogicalType") + .field(&self.signature()) + .field(&self.native()) + .finish() + } +} + +impl PartialEq for dyn LogicalType { + fn eq(&self, other: &Self) -> bool { + self.signature().eq(&other.signature()) + } +} + +impl Eq for dyn LogicalType {} + +impl PartialOrd for dyn LogicalType { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +impl Ord for dyn LogicalType { + fn cmp(&self, other: &Self) -> Ordering { + self.signature() + .cmp(&other.signature()) + .then(self.native().cmp(other.native())) + } +} + +impl Hash for dyn LogicalType { + fn hash(&self, state: &mut H) { + self.signature().hash(state); + self.native().hash(state); + } +} diff --git a/datafusion/common/src/types/mod.rs b/datafusion/common/src/types/mod.rs new file mode 100644 index 000000000000..2f9ce4ce0282 --- /dev/null +++ b/datafusion/common/src/types/mod.rs @@ -0,0 +1,26 @@ +// 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. + +mod builtin; +mod field; +mod logical; +mod native; + +pub use builtin::*; +pub use field::*; +pub use logical::*; +pub use native::*; diff --git a/datafusion/common/src/types/native.rs b/datafusion/common/src/types/native.rs new file mode 100644 index 000000000000..bfb546783ea2 --- /dev/null +++ b/datafusion/common/src/types/native.rs @@ -0,0 +1,399 @@ +// 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 super::{ + LogicalField, LogicalFieldRef, LogicalFields, LogicalType, LogicalUnionFields, + TypeSignature, +}; +use crate::error::{Result, _internal_err}; +use arrow::compute::can_cast_types; +use arrow_schema::{ + DataType, Field, FieldRef, Fields, IntervalUnit, TimeUnit, UnionFields, +}; +use std::sync::Arc; + +/// Representation of a type that DataFusion can handle natively. It is a subset +/// of the physical variants in Arrow's native [`DataType`]. +#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub enum NativeType { + /// Null type + Null, + /// A boolean type representing the values `true` and `false`. + Boolean, + /// A signed 8-bit integer. + Int8, + /// A signed 16-bit integer. + Int16, + /// A signed 32-bit integer. + Int32, + /// A signed 64-bit integer. + Int64, + /// An unsigned 8-bit integer. + UInt8, + /// An unsigned 16-bit integer. + UInt16, + /// An unsigned 32-bit integer. + UInt32, + /// An unsigned 64-bit integer. + UInt64, + /// A 16-bit floating point number. + Float16, + /// A 32-bit floating point number. + Float32, + /// A 64-bit floating point number. + Float64, + /// A timestamp with an optional timezone. + /// + /// Time is measured as a Unix epoch, counting the seconds from + /// 00:00:00.000 on 1 January 1970, excluding leap seconds, + /// as a signed 64-bit integer. + /// + /// The time zone is a string indicating the name of a time zone, one of: + /// + /// * As used in the Olson time zone database (the "tz database" or + /// "tzdata"), such as "America/New_York" + /// * An absolute time zone offset of the form +XX:XX or -XX:XX, such as +07:30 + /// + /// Timestamps with a non-empty timezone + /// ------------------------------------ + /// + /// If a Timestamp column has a non-empty timezone value, its epoch is + /// 1970-01-01 00:00:00 (January 1st 1970, midnight) in the *UTC* timezone + /// (the Unix epoch), regardless of the Timestamp's own timezone. + /// + /// Therefore, timestamp values with a non-empty timezone correspond to + /// physical points in time together with some additional information about + /// how the data was obtained and/or how to display it (the timezone). + /// + /// For example, the timestamp value 0 with the timezone string "Europe/Paris" + /// corresponds to "January 1st 1970, 00h00" in the UTC timezone, but the + /// application may prefer to display it as "January 1st 1970, 01h00" in + /// the Europe/Paris timezone (which is the same physical point in time). + /// + /// One consequence is that timestamp values with a non-empty timezone + /// can be compared and ordered directly, since they all share the same + /// well-known point of reference (the Unix epoch). + /// + /// Timestamps with an unset / empty timezone + /// ----------------------------------------- + /// + /// If a Timestamp column has no timezone value, its epoch is + /// 1970-01-01 00:00:00 (January 1st 1970, midnight) in an *unknown* timezone. + /// + /// Therefore, timestamp values without a timezone cannot be meaningfully + /// interpreted as physical points in time, but only as calendar / clock + /// indications ("wall clock time") in an unspecified timezone. + /// + /// For example, the timestamp value 0 with an empty timezone string + /// corresponds to "January 1st 1970, 00h00" in an unknown timezone: there + /// is not enough information to interpret it as a well-defined physical + /// point in time. + /// + /// One consequence is that timestamp values without a timezone cannot + /// be reliably compared or ordered, since they may have different points of + /// reference. In particular, it is *not* possible to interpret an unset + /// or empty timezone as the same as "UTC". + /// + /// Conversion between timezones + /// ---------------------------- + /// + /// If a Timestamp column has a non-empty timezone, changing the timezone + /// to a different non-empty value is a metadata-only operation: + /// the timestamp values need not change as their point of reference remains + /// the same (the Unix epoch). + /// + /// However, if a Timestamp column has no timezone value, changing it to a + /// non-empty value requires to think about the desired semantics. + /// One possibility is to assume that the original timestamp values are + /// relative to the epoch of the timezone being set; timestamp values should + /// then adjusted to the Unix epoch (for example, changing the timezone from + /// empty to "Europe/Paris" would require converting the timestamp values + /// from "Europe/Paris" to "UTC", which seems counter-intuitive but is + /// nevertheless correct). + /// + /// ``` + /// # use arrow_schema::{DataType, TimeUnit}; + /// DataType::Timestamp(TimeUnit::Second, None); + /// DataType::Timestamp(TimeUnit::Second, Some("literal".into())); + /// DataType::Timestamp(TimeUnit::Second, Some("string".to_string().into())); + /// ``` + Timestamp(TimeUnit, Option>), + /// A signed date representing the elapsed time since UNIX epoch (1970-01-01) + /// in days. + Date, + /// A signed time representing the elapsed time since midnight in the unit of `TimeUnit`. + Time(TimeUnit), + /// Measure of elapsed time in either seconds, milliseconds, microseconds or nanoseconds. + Duration(TimeUnit), + /// A "calendar" interval which models types that don't necessarily + /// have a precise duration without the context of a base timestamp (e.g. + /// days can differ in length during day light savings time transitions). + Interval(IntervalUnit), + /// Opaque binary data of variable length. + Binary, + /// Opaque binary data of fixed size. + /// Enum parameter specifies the number of bytes per value. + FixedSizeBinary(i32), + /// A variable-length string in Unicode with UTF-8 encoding. + String, + /// A list of some logical data type with variable length. + List(LogicalFieldRef), + /// A list of some logical data type with fixed length. + FixedSizeList(LogicalFieldRef, i32), + /// A nested type that contains a number of sub-fields. + Struct(LogicalFields), + /// A nested type that can represent slots of differing types. + Union(LogicalUnionFields), + /// Decimal value with precision and scale + /// + /// * precision is the total number of digits + /// * scale is the number of digits past the decimal + /// + /// For example the number 123.45 has precision 5 and scale 2. + /// + /// In certain situations, scale could be negative number. For + /// negative scale, it is the number of padding 0 to the right + /// of the digits. + /// + /// For example the number 12300 could be treated as a decimal + /// has precision 3 and scale -2. + Decimal(u8, i8), + /// A Map is a type that an association between a key and a value. + /// + /// The key and value types are not constrained, but keys should be + /// hashable and unique. + /// + /// In a field with Map type, key type and the second the value type. The names of the + /// child fields may be respectively "entries", "key", and "value", but this is + /// not enforced. + Map(LogicalFieldRef), +} + +impl LogicalType for NativeType { + fn native(&self) -> &NativeType { + self + } + + fn signature(&self) -> TypeSignature<'_> { + TypeSignature::Native(self) + } + + fn default_cast_for(&self, origin: &DataType) -> Result { + use DataType::*; + + fn default_field_cast(to: &LogicalField, from: &Field) -> Result { + Ok(Arc::new(Field::new( + to.name.clone(), + to.logical_type.default_cast_for(from.data_type())?, + to.nullable, + ))) + } + + Ok(match (self, origin) { + (Self::Null, _) => Null, + (Self::Boolean, _) => Boolean, + (Self::Int8, _) => Int8, + (Self::Int16, _) => Int16, + (Self::Int32, _) => Int32, + (Self::Int64, _) => Int64, + (Self::UInt8, _) => UInt8, + (Self::UInt16, _) => UInt16, + (Self::UInt32, _) => UInt32, + (Self::UInt64, _) => UInt64, + (Self::Float16, _) => Float16, + (Self::Float32, _) => Float32, + (Self::Float64, _) => Float64, + (Self::Decimal(p, s), _) if p <= &38 => Decimal128(*p, *s), + (Self::Decimal(p, s), _) => Decimal256(*p, *s), + (Self::Timestamp(tu, tz), _) => Timestamp(*tu, tz.clone()), + (Self::Date, _) => Date32, + (Self::Time(tu), _) => match tu { + TimeUnit::Second | TimeUnit::Millisecond => Time32(*tu), + TimeUnit::Microsecond | TimeUnit::Nanosecond => Time64(*tu), + }, + (Self::Duration(tu), _) => Duration(*tu), + (Self::Interval(iu), _) => Interval(*iu), + (Self::Binary, LargeUtf8) => LargeBinary, + (Self::Binary, Utf8View) => BinaryView, + (Self::Binary, data_type) if can_cast_types(data_type, &BinaryView) => { + BinaryView + } + (Self::Binary, data_type) if can_cast_types(data_type, &LargeBinary) => { + LargeBinary + } + (Self::Binary, data_type) if can_cast_types(data_type, &Binary) => Binary, + (Self::FixedSizeBinary(size), _) => FixedSizeBinary(*size), + (Self::String, LargeBinary) => LargeUtf8, + (Self::String, BinaryView) => Utf8View, + (Self::String, data_type) if can_cast_types(data_type, &Utf8View) => Utf8View, + (Self::String, data_type) if can_cast_types(data_type, &LargeUtf8) => { + LargeUtf8 + } + (Self::String, data_type) if can_cast_types(data_type, &Utf8) => Utf8, + (Self::List(to_field), List(from_field) | FixedSizeList(from_field, _)) => { + List(default_field_cast(to_field, from_field)?) + } + (Self::List(to_field), LargeList(from_field)) => { + LargeList(default_field_cast(to_field, from_field)?) + } + (Self::List(to_field), ListView(from_field)) => { + ListView(default_field_cast(to_field, from_field)?) + } + (Self::List(to_field), LargeListView(from_field)) => { + LargeListView(default_field_cast(to_field, from_field)?) + } + // List array where each element is a len 1 list of the origin type + (Self::List(field), _) => List(Arc::new(Field::new( + field.name.clone(), + field.logical_type.default_cast_for(origin)?, + field.nullable, + ))), + ( + Self::FixedSizeList(to_field, to_size), + FixedSizeList(from_field, from_size), + ) if from_size == to_size => { + FixedSizeList(default_field_cast(to_field, from_field)?, *to_size) + } + ( + Self::FixedSizeList(to_field, size), + List(from_field) + | LargeList(from_field) + | ListView(from_field) + | LargeListView(from_field), + ) => FixedSizeList(default_field_cast(to_field, from_field)?, *size), + // FixedSizeList array where each element is a len 1 list of the origin type + (Self::FixedSizeList(field, size), _) => FixedSizeList( + Arc::new(Field::new( + field.name.clone(), + field.logical_type.default_cast_for(origin)?, + field.nullable, + )), + *size, + ), + // From https://github.com/apache/arrow-rs/blob/56525efbd5f37b89d1b56aa51709cab9f81bc89e/arrow-cast/src/cast/mod.rs#L189-L196 + (Self::Struct(to_fields), Struct(from_fields)) + if from_fields.len() == to_fields.len() => + { + Struct( + from_fields + .iter() + .zip(to_fields.iter()) + .map(|(from, to)| default_field_cast(to, from)) + .collect::>()?, + ) + } + (Self::Struct(to_fields), Null) => Struct( + to_fields + .iter() + .map(|field| { + Ok(Arc::new(Field::new( + field.name.clone(), + field.logical_type.default_cast_for(&Null)?, + field.nullable, + ))) + }) + .collect::>()?, + ), + (Self::Map(to_field), Map(from_field, sorted)) => { + Map(default_field_cast(to_field, from_field)?, *sorted) + } + (Self::Map(field), Null) => Map( + Arc::new(Field::new( + field.name.clone(), + field.logical_type.default_cast_for(&Null)?, + field.nullable, + )), + false, + ), + (Self::Union(to_fields), Union(from_fields, mode)) + if from_fields.len() == to_fields.len() => + { + Union( + from_fields + .iter() + .zip(to_fields.iter()) + .map(|((_, from), (i, to))| { + Ok((*i, default_field_cast(to, from)?)) + }) + .collect::>()?, + *mode, + ) + } + _ => { + return _internal_err!( + "Unavailable default cast for native type {:?} from physical type {:?}", + self, + origin + ) + } + }) + } +} + +// The following From, From, ... implementations are temporary +// mapping solutions to provide backwards compatibility while transitioning from +// the purely physical system to a logical / physical system. + +impl From for NativeType { + fn from(value: DataType) -> Self { + use NativeType::*; + match value { + DataType::Null => Null, + DataType::Boolean => Boolean, + DataType::Int8 => Int8, + DataType::Int16 => Int16, + DataType::Int32 => Int32, + DataType::Int64 => Int64, + DataType::UInt8 => UInt8, + DataType::UInt16 => UInt16, + DataType::UInt32 => UInt32, + DataType::UInt64 => UInt64, + DataType::Float16 => Float16, + DataType::Float32 => Float32, + DataType::Float64 => Float64, + DataType::Timestamp(tu, tz) => Timestamp(tu, tz), + DataType::Date32 | DataType::Date64 => Date, + DataType::Time32(tu) | DataType::Time64(tu) => Time(tu), + DataType::Duration(tu) => Duration(tu), + DataType::Interval(iu) => Interval(iu), + DataType::Binary | DataType::LargeBinary | DataType::BinaryView => Binary, + DataType::FixedSizeBinary(size) => FixedSizeBinary(size), + DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => String, + DataType::List(field) + | DataType::ListView(field) + | DataType::LargeList(field) + | DataType::LargeListView(field) => List(Arc::new(field.as_ref().into())), + DataType::FixedSizeList(field, size) => { + FixedSizeList(Arc::new(field.as_ref().into()), size) + } + DataType::Struct(fields) => Struct(LogicalFields::from(&fields)), + DataType::Union(union_fields, _) => { + Union(LogicalUnionFields::from(&union_fields)) + } + DataType::Decimal128(p, s) | DataType::Decimal256(p, s) => Decimal(p, s), + DataType::Map(field, _) => Map(Arc::new(field.as_ref().into())), + DataType::Dictionary(_, data_type) => data_type.as_ref().clone().into(), + DataType::RunEndEncoded(_, field) => field.data_type().clone().into(), + } + } +} + +impl From<&DataType> for NativeType { + fn from(value: &DataType) -> Self { + value.clone().into() + } +} From a9d4d525df07dd2fc5eb6adc622a821cf54d44ba Mon Sep 17 00:00:00 2001 From: Andrey Koshchiy Date: Sun, 3 Nov 2024 14:48:56 +0300 Subject: [PATCH 11/28] feat(substrait): AggregateRel grouping_expression support (#13173) --- .../substrait/src/logical_plan/consumer.rs | 77 ++++++++++----- .../substrait/src/logical_plan/producer.rs | 58 ++++++++--- .../tests/cases/roundtrip_logical_plan.rs | 13 +++ ...roject_group_expression_ref.substrait.json | 98 +++++++++++++++++++ 4 files changed, 210 insertions(+), 36 deletions(-) create mode 100644 datafusion/substrait/tests/testdata/test_plans/aggregate_no_project_group_expression_ref.substrait.json diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 7ccca8616ba0..890da7361d7c 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -33,6 +33,7 @@ use datafusion::logical_expr::{ expr::find_df_window_func, Aggregate, BinaryExpr, Case, EmptyRelation, Expr, ExprSchemable, LogicalPlan, Operator, Projection, SortExpr, Values, }; +use substrait::proto::aggregate_rel::Grouping; use substrait::proto::expression::subquery::set_predicate::PredicateOp; use substrait::proto::expression_reference::ExprType; use url::Url; @@ -665,39 +666,48 @@ pub async fn from_substrait_rel( let input = LogicalPlanBuilder::from( from_substrait_rel(ctx, input, extensions).await?, ); - let mut group_expr = vec![]; - let mut aggr_expr = vec![]; + let mut ref_group_exprs = vec![]; + + for e in &agg.grouping_expressions { + let x = + from_substrait_rex(ctx, e, input.schema(), extensions).await?; + ref_group_exprs.push(x); + } + + let mut group_exprs = vec![]; + let mut aggr_exprs = vec![]; match agg.groupings.len() { 1 => { - for e in &agg.groupings[0].grouping_expressions { - let x = - from_substrait_rex(ctx, e, input.schema(), extensions) - .await?; - group_expr.push(x); - } + group_exprs.extend_from_slice( + &from_substrait_grouping( + ctx, + &agg.groupings[0], + &ref_group_exprs, + input.schema(), + extensions, + ) + .await?, + ); } _ => { let mut grouping_sets = vec![]; for grouping in &agg.groupings { - let mut grouping_set = vec![]; - for e in &grouping.grouping_expressions { - let x = from_substrait_rex( - ctx, - e, - input.schema(), - extensions, - ) - .await?; - grouping_set.push(x); - } + let grouping_set = from_substrait_grouping( + ctx, + grouping, + &ref_group_exprs, + input.schema(), + extensions, + ) + .await?; grouping_sets.push(grouping_set); } // Single-element grouping expression of type Expr::GroupingSet. // Note that GroupingSet::Rollup would become GroupingSet::GroupingSets, when // parsed by the producer and consumer, since Substrait does not have a type dedicated // to ROLLUP. Only vector of Groupings (grouping sets) is available. - group_expr.push(Expr::GroupingSet(GroupingSet::GroupingSets( + group_exprs.push(Expr::GroupingSet(GroupingSet::GroupingSets( grouping_sets, ))); } @@ -755,9 +765,9 @@ pub async fn from_substrait_rel( "Aggregate without aggregate function is not supported" ), }; - aggr_expr.push(agg_func?.as_ref().clone()); + aggr_exprs.push(agg_func?.as_ref().clone()); } - input.aggregate(group_expr, aggr_expr)?.build() + input.aggregate(group_exprs, aggr_exprs)?.build() } else { not_impl_err!("Aggregate without an input is not valid") } @@ -2762,6 +2772,29 @@ fn from_substrait_null( } } +#[allow(deprecated)] +async fn from_substrait_grouping( + ctx: &SessionContext, + grouping: &Grouping, + expressions: &[Expr], + input_schema: &DFSchemaRef, + extensions: &Extensions, +) -> Result> { + let mut group_exprs = vec![]; + if !grouping.grouping_expressions.is_empty() { + for e in &grouping.grouping_expressions { + let expr = from_substrait_rex(ctx, e, input_schema, extensions).await?; + group_exprs.push(expr); + } + return Ok(group_exprs); + } + for idx in &grouping.expression_references { + let e = &expressions[*idx as usize]; + group_exprs.push(e.clone()); + } + Ok(group_exprs) +} + fn from_substrait_field_reference( field_ref: &FieldReference, input_schema: &DFSchema, diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index c73029f130ad..4d864e4334ce 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -361,7 +361,7 @@ pub fn to_substrait_rel( } LogicalPlan::Aggregate(agg) => { let input = to_substrait_rel(agg.input.as_ref(), ctx, extensions)?; - let groupings = to_substrait_groupings( + let (grouping_expressions, groupings) = to_substrait_groupings( ctx, &agg.group_expr, agg.input.schema(), @@ -377,7 +377,7 @@ pub fn to_substrait_rel( rel_type: Some(RelType::Aggregate(Box::new(AggregateRel { common: None, input: Some(input), - grouping_expressions: vec![], + grouping_expressions, groupings, measures, advanced_extension: None, @@ -774,14 +774,20 @@ pub fn parse_flat_grouping_exprs( exprs: &[Expr], schema: &DFSchemaRef, extensions: &mut Extensions, + ref_group_exprs: &mut Vec, ) -> Result { - let grouping_expressions = exprs - .iter() - .map(|e| to_substrait_rex(ctx, e, schema, 0, extensions)) - .collect::>>()?; + let mut expression_references = vec![]; + let mut grouping_expressions = vec![]; + + for e in exprs { + let rex = to_substrait_rex(ctx, e, schema, 0, extensions)?; + grouping_expressions.push(rex.clone()); + ref_group_exprs.push(rex); + expression_references.push((ref_group_exprs.len() - 1) as u32); + } Ok(Grouping { grouping_expressions, - expression_references: vec![], + expression_references, }) } @@ -790,8 +796,9 @@ pub fn to_substrait_groupings( exprs: &[Expr], schema: &DFSchemaRef, extensions: &mut Extensions, -) -> Result> { - match exprs.len() { +) -> Result<(Vec, Vec)> { + let mut ref_group_exprs = vec![]; + let groupings = match exprs.len() { 1 => match &exprs[0] { Expr::GroupingSet(gs) => match gs { GroupingSet::Cube(_) => Err(DataFusionError::NotImplemented( @@ -799,7 +806,15 @@ pub fn to_substrait_groupings( )), GroupingSet::GroupingSets(sets) => Ok(sets .iter() - .map(|set| parse_flat_grouping_exprs(ctx, set, schema, extensions)) + .map(|set| { + parse_flat_grouping_exprs( + ctx, + set, + schema, + extensions, + &mut ref_group_exprs, + ) + }) .collect::>>()?), GroupingSet::Rollup(set) => { let mut sets: Vec> = vec![vec![]]; @@ -810,19 +825,34 @@ pub fn to_substrait_groupings( .iter() .rev() .map(|set| { - parse_flat_grouping_exprs(ctx, set, schema, extensions) + parse_flat_grouping_exprs( + ctx, + set, + schema, + extensions, + &mut ref_group_exprs, + ) }) .collect::>>()?) } }, _ => Ok(vec![parse_flat_grouping_exprs( - ctx, exprs, schema, extensions, + ctx, + exprs, + schema, + extensions, + &mut ref_group_exprs, )?]), }, _ => Ok(vec![parse_flat_grouping_exprs( - ctx, exprs, schema, extensions, + ctx, + exprs, + schema, + extensions, + &mut ref_group_exprs, )?]), - } + }?; + Ok((ref_group_exprs, groupings)) } #[allow(deprecated)] diff --git a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs index 8fbdefe2852e..5687c9af540a 100644 --- a/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/substrait/tests/cases/roundtrip_logical_plan.rs @@ -665,6 +665,19 @@ async fn aggregate_wo_projection_consume() -> Result<()> { .await } +#[tokio::test] +async fn aggregate_wo_projection_group_expression_ref_consume() -> Result<()> { + let proto_plan = + read_json("tests/testdata/test_plans/aggregate_no_project_group_expression_ref.substrait.json"); + + assert_expected_plan_substrait( + proto_plan, + "Aggregate: groupBy=[[data.a]], aggr=[[count(data.a) AS countA]]\ + \n TableScan: data projection=[a]", + ) + .await +} + #[tokio::test] async fn aggregate_wo_projection_sorted_consume() -> Result<()> { let proto_plan = diff --git a/datafusion/substrait/tests/testdata/test_plans/aggregate_no_project_group_expression_ref.substrait.json b/datafusion/substrait/tests/testdata/test_plans/aggregate_no_project_group_expression_ref.substrait.json new file mode 100644 index 000000000000..b6f14afd6fa9 --- /dev/null +++ b/datafusion/substrait/tests/testdata/test_plans/aggregate_no_project_group_expression_ref.substrait.json @@ -0,0 +1,98 @@ +{ + "extensionUris": [ + { + "uri": "https://github.com/substrait-io/substrait/blob/main/extensions/functions_aggregate_generic.yaml" + } + ], + "extensions": [ + { + "extensionFunction": { + "functionAnchor": 185, + "name": "count:any" + } + } + ], + "relations": [ + { + "root": { + "input": { + "aggregate": { + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "a" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_NULLABLE" + } + }, + "namedTable": { + "names": [ + "data" + ] + } + } + }, + "grouping_expressions": [ + { + "selection": { + "directReference": { + "structField": {} + }, + "rootReference": {} + } + } + ], + "groupings": [ + { + "expression_references": [0] + } + ], + "measures": [ + { + "measure": { + "functionReference": 185, + "phase": "AGGREGATION_PHASE_INITIAL_TO_RESULT", + "outputType": { + "i64": {} + }, + "arguments": [ + { + "value": { + "selection": { + "directReference": { + "structField": {} + }, + "rootReference": {} + } + } + } + ] + } + } + ] + } + }, + "names": [ + "a", + "countA" + ] + } + } + ], + "version": { + "minorNumber": 54, + "producer": "subframe" + } +} \ No newline at end of file From 85f92ef6be1d7364857a2c13fbb026d9e45406ea Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Sun, 3 Nov 2024 07:07:47 -0500 Subject: [PATCH 12/28] Apply projection to `Statistics` in `FilterExec` (#13187) * Apply projection to `Statistics` in `FilterExec` * Use Statistics::project in HashJoin --- datafusion/common/src/stats.rs | 20 ++++++++ datafusion/physical-plan/src/filter.rs | 7 ++- .../physical-plan/src/joins/hash_join.rs | 13 +---- .../sqllogictest/test_files/parquet.slt | 49 +++++++++++++++++++ 4 files changed, 77 insertions(+), 12 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index e669c674f78a..1aa42705e7f8 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -258,6 +258,26 @@ impl Statistics { self } + /// Project the statistics to the given column indices. + /// + /// For example, if we had statistics for columns `{"a", "b", "c"}`, + /// projecting to `vec![2, 1]` would return statistics for columns `{"c", + /// "b"}`. + pub fn project(mut self, projection: Option<&Vec>) -> Self { + let Some(projection) = projection else { + return self; + }; + + // todo: it would be nice to avoid cloning column statistics if + // possible (e.g. if the projection did not contain duplicates) + self.column_statistics = projection + .iter() + .map(|&i| self.column_statistics[i].clone()) + .collect(); + + self + } + /// Calculates the statistics after `fetch` and `skip` operations apply. /// Here, `self` denotes per-partition statistics. Use the `n_partitions` /// parameter to compute global statistics in a multi-partition setting. diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 97d8159137f4..07898e8d22d8 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -371,7 +371,12 @@ impl ExecutionPlan for FilterExec { /// The output statistics of a filtering operation can be estimated if the /// predicate's selectivity value can be determined for the incoming data. fn statistics(&self) -> Result { - Self::statistics_helper(&self.input, self.predicate(), self.default_selectivity) + let stats = Self::statistics_helper( + &self.input, + self.predicate(), + self.default_selectivity, + )?; + Ok(stats.project(self.projection.as_ref())) } fn cardinality_effect(&self) -> CardinalityEffect { diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 57d8a9ce7b35..ae872e13a9f6 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -785,7 +785,7 @@ impl ExecutionPlan for HashJoinExec { // TODO stats: it is not possible in general to know the output size of joins // 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( + let stats = estimate_join_statistics( Arc::clone(&self.left), Arc::clone(&self.right), self.on.clone(), @@ -793,16 +793,7 @@ impl ExecutionPlan for HashJoinExec { &self.join_schema, )?; // Project statistics if there is a projection - if let Some(projection) = &self.projection { - stats.column_statistics = stats - .column_statistics - .into_iter() - .enumerate() - .filter(|(i, _)| projection.contains(i)) - .map(|(_, s)| s) - .collect(); - } - Ok(stats) + Ok(stats.project(self.projection.as_ref())) } } diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index ed963466fca6..253ebb9ea0ac 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -549,3 +549,52 @@ FixedSizeBinary(16) 0166ce1d46129ad104fa4990c6057c91 statement ok DROP TABLE test_non_utf8_binary; + + +## Tests for https://github.com/apache/datafusion/issues/13186 +statement ok +create table cpu (time timestamp, usage_idle float, usage_user float, cpu int); + +statement ok +insert into cpu values ('1970-01-01 00:00:00', 1.0, 2.0, 3); + +# must put it into a parquet file to get statistics +statement ok +copy (select * from cpu) to 'test_files/scratch/parquet/cpu.parquet'; + +# Run queries against parquet files +statement ok +create external table cpu_parquet +stored as parquet +location 'test_files/scratch/parquet/cpu.parquet'; + +# Double filtering +# +# Expect 1 row for both queries +query PI +select time, rn +from ( + select time, row_number() OVER (ORDER BY usage_idle, time) as rn + from cpu + where cpu = 3 +) where rn > 0; +---- +1970-01-01T00:00:00 1 + +query PI +select time, rn +from ( + select time, row_number() OVER (ORDER BY usage_idle, time) as rn + from cpu_parquet + where cpu = 3 +) where rn > 0; +---- +1970-01-01T00:00:00 1 + + +# Clean up +statement ok +drop table cpu; + +statement ok +drop table cpu_parquet; From d19865ee80ee6434694b8dc9cd00c8cae52c6fc7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Burak=20=C5=9Een?= Date: Mon, 4 Nov 2024 14:49:14 +0300 Subject: [PATCH 13/28] [minor] overload from_unixtime func to have optional timezone parameter (#13130) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * overloaded from ts * Update docs/source/user-guide/sql/scalar_functions_new.md Co-authored-by: Bruce Ritchie * fixed return type * added sql example * optional in ∂oc * review --------- Co-authored-by: Bruce Ritchie Co-authored-by: Andrew Lamb --- .../functions/src/datetime/from_unixtime.rs | 132 +++++++++++++++--- .../sqllogictest/test_files/timestamps.slt | 23 +++ .../source/user-guide/sql/scalar_functions.md | 16 ++- 3 files changed, 148 insertions(+), 23 deletions(-) diff --git a/datafusion/functions/src/datetime/from_unixtime.rs b/datafusion/functions/src/datetime/from_unixtime.rs index 84aa9feec654..ed9858106c52 100644 --- a/datafusion/functions/src/datetime/from_unixtime.rs +++ b/datafusion/functions/src/datetime/from_unixtime.rs @@ -15,16 +15,17 @@ // specific language governing permissions and limitations // under the License. -use arrow::datatypes::DataType; -use arrow::datatypes::DataType::{Int64, Timestamp}; -use arrow::datatypes::TimeUnit::Second; use std::any::Any; -use std::sync::OnceLock; +use std::sync::{Arc, OnceLock}; -use datafusion_common::{exec_err, Result}; +use arrow::datatypes::DataType; +use arrow::datatypes::DataType::{Int64, Timestamp, Utf8}; +use arrow::datatypes::TimeUnit::Second; +use datafusion_common::{exec_err, internal_err, ExprSchema, Result, ScalarValue}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; +use datafusion_expr::TypeSignature::Exact; use datafusion_expr::{ - ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, + ColumnarValue, Documentation, Expr, ScalarUDFImpl, Signature, Volatility, }; #[derive(Debug)] @@ -41,7 +42,10 @@ impl Default for FromUnixtimeFunc { impl FromUnixtimeFunc { pub fn new() -> Self { Self { - signature: Signature::uniform(1, vec![Int64], Volatility::Immutable), + signature: Signature::one_of( + vec![Exact(vec![Int64, Utf8]), Exact(vec![Int64])], + Volatility::Immutable, + ), } } } @@ -59,28 +63,63 @@ impl ScalarUDFImpl for FromUnixtimeFunc { &self.signature } + fn return_type_from_exprs( + &self, + args: &[Expr], + _schema: &dyn ExprSchema, + arg_types: &[DataType], + ) -> Result { + match arg_types.len() { + 1 => Ok(Timestamp(Second, None)), + 2 => match &args[1] { + Expr::Literal(ScalarValue::Utf8(Some(tz))) => Ok(Timestamp(Second, Some(Arc::from(tz.to_string())))), + _ => exec_err!( + "Second argument for `from_unixtime` must be non-null utf8, received {:?}", + arg_types[1]), + }, + _ => exec_err!( + "from_unixtime function requires 1 or 2 arguments, got {}", + arg_types.len() + ), + } + } + fn return_type(&self, _arg_types: &[DataType]) -> Result { - Ok(Timestamp(Second, None)) + internal_err!("call return_type_from_exprs instead") } fn invoke(&self, args: &[ColumnarValue]) -> Result { - if args.len() != 1 { + let len = args.len(); + if len != 1 && len != 2 { return exec_err!( - "from_unixtime function requires 1 argument, got {}", + "from_unixtime function requires 1 or 2 argument, got {}", args.len() ); } - match args[0].data_type() { - Int64 => args[0].cast_to(&Timestamp(Second, None), None), - other => { - exec_err!( - "Unsupported data type {:?} for function from_unixtime", - other - ) - } + if args[0].data_type() != Int64 { + return exec_err!( + "Unsupported data type {:?} for function from_unixtime", + args[0].data_type() + ); + } + + match len { + 1 => args[0].cast_to(&Timestamp(Second, None), None), + 2 => match &args[1] { + ColumnarValue::Scalar(ScalarValue::Utf8(Some(tz))) => args[0] + .cast_to(&Timestamp(Second, Some(Arc::from(tz.to_string()))), None), + _ => { + exec_err!( + "Unsupported data type {:?} for function from_unixtime", + args[1].data_type() + ) + } + }, + _ => unreachable!(), } } + fn documentation(&self) -> Option<&Documentation> { Some(get_from_unixtime_doc()) } @@ -93,12 +132,63 @@ fn get_from_unixtime_doc() -> &'static Documentation { Documentation::builder() .with_doc_section(DOC_SECTION_DATETIME) .with_description("Converts an integer to RFC3339 timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp.") - .with_syntax_example("from_unixtime(expression)") + .with_syntax_example("from_unixtime(expression[, timezone])") + .with_standard_argument("expression", None) .with_argument( - "expression", - "Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators." + "timezone", + "Optional timezone to use when converting the integer to a timestamp. If not provided, the default timezone is UTC.", ) + .with_sql_example(r#"```sql +> select from_unixtime(1599572549, 'America/New_York'); ++-----------------------------------------------------------+ +| from_unixtime(Int64(1599572549),Utf8("America/New_York")) | ++-----------------------------------------------------------+ +| 2020-09-08T09:42:29-04:00 | ++-----------------------------------------------------------+ +```"#) .build() .unwrap() }) } + +#[cfg(test)] +mod test { + use crate::datetime::from_unixtime::FromUnixtimeFunc; + use datafusion_common::ScalarValue; + use datafusion_common::ScalarValue::Int64; + use datafusion_expr::{ColumnarValue, ScalarUDFImpl}; + + #[test] + fn test_without_timezone() { + let args = [ColumnarValue::Scalar(Int64(Some(1729900800)))]; + + let result = FromUnixtimeFunc::new().invoke(&args).unwrap(); + + match result { + ColumnarValue::Scalar(ScalarValue::TimestampSecond(Some(sec), None)) => { + assert_eq!(sec, 1729900800); + } + _ => panic!("Expected scalar value"), + } + } + + #[test] + fn test_with_timezone() { + let args = [ + ColumnarValue::Scalar(Int64(Some(1729900800))), + ColumnarValue::Scalar(ScalarValue::Utf8(Some( + "America/New_York".to_string(), + ))), + ]; + + let result = FromUnixtimeFunc::new().invoke(&args).unwrap(); + + match result { + ColumnarValue::Scalar(ScalarValue::TimestampSecond(Some(sec), Some(tz))) => { + assert_eq!(sec, 1729900800); + assert_eq!(tz.to_string(), "America/New_York"); + } + _ => panic!("Expected scalar value"), + } + } +} diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index a09a63a791fc..42abeff674c5 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -308,6 +308,29 @@ SELECT from_unixtime(ts / 1000000000) FROM ts_data LIMIT 3; 2020-09-08T12:42:29 2020-09-08T11:42:29 +# from_unixtime single + +query P +SELECT from_unixtime(1599572549190855123 / 1000000000, 'America/New_York'); +---- +2020-09-08T09:42:29-04:00 + +# from_unixtime with timezone +query P +SELECT from_unixtime(ts / 1000000000, 'Asia/Istanbul') FROM ts_data LIMIT 3; +---- +2020-09-08T16:42:29+03:00 +2020-09-08T15:42:29+03:00 +2020-09-08T14:42:29+03:00 + +# from_unixtime with utc timezone +query P +SELECT from_unixtime(ts / 1000000000, 'UTC') FROM ts_data LIMIT 3; +---- +2020-09-08T13:42:29Z +2020-09-08T12:42:29Z +2020-09-08T11:42:29Z + # to_timestamp query I diff --git a/docs/source/user-guide/sql/scalar_functions.md b/docs/source/user-guide/sql/scalar_functions.md index 98c44cbd981d..b92b815d7c95 100644 --- a/docs/source/user-guide/sql/scalar_functions.md +++ b/docs/source/user-guide/sql/scalar_functions.md @@ -2037,12 +2037,24 @@ _Alias of [date_trunc](#date_trunc)._ Converts an integer to RFC3339 timestamp format (`YYYY-MM-DDT00:00:00.000000000Z`). Integers and unsigned integers are interpreted as nanoseconds since the unix epoch (`1970-01-01T00:00:00Z`) return the corresponding timestamp. ``` -from_unixtime(expression) +from_unixtime(expression[, timezone]) ``` #### Arguments -- **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. +- **expression**: The expression to operate on. Can be a constant, column, or function, and any combination of operators. +- **timezone**: Optional timezone to use when converting the integer to a timestamp. If not provided, the default timezone is UTC. + +#### Example + +```sql +> select from_unixtime(1599572549, 'America/New_York'); ++-----------------------------------------------------------+ +| from_unixtime(Int64(1599572549),Utf8("America/New_York")) | ++-----------------------------------------------------------+ +| 2020-09-08T09:42:29-04:00 | ++-----------------------------------------------------------+ +``` ### `make_date` From 2482ff45d0109c4a576e4cbfdd5769107fd9ede2 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Mon, 4 Nov 2024 08:07:11 -0500 Subject: [PATCH 14/28] Minor: make LeftJoinData into a struct in CrossJoinExec (#13227) --- .../physical-plan/src/joins/cross_join.rs | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index a67e1df47bc7..8c8921eba6a1 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -47,7 +47,15 @@ use async_trait::async_trait; use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the left side -type JoinLeftData = (RecordBatch, MemoryReservation); +#[derive(Debug)] +struct JoinLeftData { + /// Single RecordBatch with all rows from the left side + merged_batch: RecordBatch, + /// Track memory reservation for merged_batch. Relies on drop + /// semantics to release reservation when JoinLeftData is dropped. + #[allow(dead_code)] + reservation: MemoryReservation, +} #[allow(rustdoc::private_intra_doc_links)] /// executes partitions in parallel and combines them into a set of @@ -185,7 +193,10 @@ async fn load_left_input( let merged_batch = concat_batches(&left_schema, &batches)?; - Ok((merged_batch, reservation)) + Ok(JoinLeftData { + merged_batch, + reservation, + }) } impl DisplayAs for CrossJoinExec { @@ -357,7 +368,7 @@ struct CrossJoinStream { join_metrics: BuildProbeJoinMetrics, /// State of the stream state: CrossJoinStreamState, - /// Left data + /// Left data (copy of the entire buffered left side) left_data: RecordBatch, /// Batch transformer batch_transformer: T, @@ -457,16 +468,17 @@ impl CrossJoinStream { cx: &mut std::task::Context<'_>, ) -> Poll>>> { let build_timer = self.join_metrics.build_time.timer(); - let (left_data, _) = match ready!(self.left_fut.get(cx)) { + let left_data = match ready!(self.left_fut.get(cx)) { Ok(left_data) => left_data, Err(e) => return Poll::Ready(Err(e)), }; build_timer.done(); + let left_data = left_data.merged_batch.clone(); let result = if left_data.num_rows() == 0 { StatefulStreamResult::Ready(None) } else { - self.left_data = left_data.clone(); + self.left_data = left_data; self.state = CrossJoinStreamState::FetchProbeBatch; StatefulStreamResult::Continue }; From 274b2229842627f381e7ebeb7bae1fb5c9ac6e71 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 4 Nov 2024 20:16:22 +0100 Subject: [PATCH 15/28] Deprecate invoke and invoke_no_args in favor of invoke_batch (#13174) * Deprecate invoke and invoke_no_args in favor of invoke_batch `invoke_batch` covers all needs, so let's deprecate and eventually remove the redundant variants. * Migrate test_function to invoke_batch * Migrate regexpcount tests to invoke_batch * Migrate log tests to invoke_batch * Migrate tests to use invoke_batch * Migrate ToUnixtimeFunc to implement invoke_batch * Suppress deprecation warnings in tests To be followed-up on. * Migrate random benchmark to invoke_batch * fixup! Suppress deprecation warnings in tests * Fix docstring --- datafusion/expr/src/udf.rs | 31 +++- datafusion/functions/benches/random.rs | 4 +- datafusion/functions/src/datetime/date_bin.rs | 2 + .../functions/src/datetime/date_trunc.rs | 2 + .../functions/src/datetime/from_unixtime.rs | 2 + .../functions/src/datetime/make_date.rs | 8 + datafusion/functions/src/datetime/to_char.rs | 6 + datafusion/functions/src/datetime/to_date.rs | 8 + .../functions/src/datetime/to_local_time.rs | 5 +- .../functions/src/datetime/to_timestamp.rs | 5 +- .../functions/src/datetime/to_unixtime.rs | 15 +- datafusion/functions/src/math/log.rs | 20 +-- datafusion/functions/src/math/power.rs | 4 +- datafusion/functions/src/math/signum.rs | 16 +- datafusion/functions/src/regex/regexpcount.rs | 147 +++++++++++------- datafusion/functions/src/string/concat.rs | 1 + datafusion/functions/src/string/concat_ws.rs | 2 + datafusion/functions/src/string/contains.rs | 1 + datafusion/functions/src/string/lower.rs | 1 + datafusion/functions/src/string/upper.rs | 1 + datafusion/functions/src/utils.rs | 20 +-- 21 files changed, 199 insertions(+), 102 deletions(-) diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index b5e9a555c2da..003a3ed36a60 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -195,6 +195,7 @@ impl ScalarUDF { /// See [`ScalarUDFImpl::invoke`] for more details. #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] pub fn invoke(&self, args: &[ColumnarValue]) -> Result { + #[allow(deprecated)] self.inner.invoke(args) } @@ -218,6 +219,7 @@ impl ScalarUDF { /// See [`ScalarUDFImpl::invoke_no_args`] for more details. #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] pub fn invoke_no_args(&self, number_rows: usize) -> Result { + #[allow(deprecated)] self.inner.invoke_no_args(number_rows) } @@ -226,6 +228,7 @@ impl ScalarUDF { #[deprecated(since = "42.0.0", note = "Use `invoke_batch` instead")] pub fn fun(&self) -> ScalarFunctionImplementation { let captured = Arc::clone(&self.inner); + #[allow(deprecated)] Arc::new(move |args| captured.invoke(args)) } @@ -480,6 +483,7 @@ pub trait ScalarUDFImpl: Debug + Send + Sync { /// to arrays, which will likely be simpler code, but be slower. /// /// [invoke_no_args]: ScalarUDFImpl::invoke_no_args + #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] fn invoke(&self, _args: &[ColumnarValue]) -> Result { not_impl_err!( "Function {} does not implement invoke but called", @@ -489,19 +493,40 @@ pub trait ScalarUDFImpl: Debug + Send + Sync { /// Invoke the function with `args` and the number of rows, /// returning the appropriate result. + /// + /// The function will be invoked with the slice of [`ColumnarValue`] + /// (either scalar or array). + /// + /// # Performance + /// + /// For the best performance, the implementations should handle the common case + /// when one or more of their arguments are constant values (aka + /// [`ColumnarValue::Scalar`]). + /// + /// [`ColumnarValue::values_to_arrays`] can be used to convert the arguments + /// to arrays, which will likely be simpler code, but be slower. fn invoke_batch( &self, args: &[ColumnarValue], number_rows: usize, ) -> Result { match args.is_empty() { - true => self.invoke_no_args(number_rows), - false => self.invoke(args), + true => + { + #[allow(deprecated)] + self.invoke_no_args(number_rows) + } + false => + { + #[allow(deprecated)] + self.invoke(args) + } } } /// Invoke the function without `args`, instead the number of rows are provided, /// returning the appropriate result. + #[deprecated(since = "42.1.0", note = "Use `invoke_batch` instead")] fn invoke_no_args(&self, _number_rows: usize) -> Result { not_impl_err!( "Function {} does not implement invoke_no_args but called", @@ -725,10 +750,12 @@ impl ScalarUDFImpl for AliasedScalarUDFImpl { } fn invoke(&self, args: &[ColumnarValue]) -> Result { + #[allow(deprecated)] self.inner.invoke(args) } fn invoke_no_args(&self, number_rows: usize) -> Result { + #[allow(deprecated)] self.inner.invoke_no_args(number_rows) } diff --git a/datafusion/functions/benches/random.rs b/datafusion/functions/benches/random.rs index a721836bb68c..5df5d9c7dee2 100644 --- a/datafusion/functions/benches/random.rs +++ b/datafusion/functions/benches/random.rs @@ -29,7 +29,7 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("random_1M_rows_batch_8192", |b| { b.iter(|| { for _ in 0..iterations { - black_box(random_func.invoke_no_args(8192).unwrap()); + black_box(random_func.invoke_batch(&[], 8192).unwrap()); } }) }); @@ -39,7 +39,7 @@ fn criterion_benchmark(c: &mut Criterion) { c.bench_function("random_1M_rows_batch_128", |b| { b.iter(|| { for _ in 0..iterations_128 { - black_box(random_func.invoke_no_args(128).unwrap()); + black_box(random_func.invoke_batch(&[], 128).unwrap()); } }) }); diff --git a/datafusion/functions/src/datetime/date_bin.rs b/datafusion/functions/src/datetime/date_bin.rs index e8d065df8633..065201e1caa3 100644 --- a/datafusion/functions/src/datetime/date_bin.rs +++ b/datafusion/functions/src/datetime/date_bin.rs @@ -491,6 +491,7 @@ mod tests { use chrono::TimeDelta; #[test] + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch fn test_date_bin() { let res = DateBinFunc::new().invoke(&[ ColumnarValue::Scalar(ScalarValue::IntervalDayTime(Some(IntervalDayTime { @@ -781,6 +782,7 @@ mod tests { .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::() .with_timezone_opt(tz_opt.clone()); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = DateBinFunc::new() .invoke(&[ ColumnarValue::Scalar(ScalarValue::new_interval_dt(1, 0)), diff --git a/datafusion/functions/src/datetime/date_trunc.rs b/datafusion/functions/src/datetime/date_trunc.rs index 4808f020e0ca..f8abef601f70 100644 --- a/datafusion/functions/src/datetime/date_trunc.rs +++ b/datafusion/functions/src/datetime/date_trunc.rs @@ -724,6 +724,7 @@ mod tests { .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::() .with_timezone_opt(tz_opt.clone()); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = DateTruncFunc::new() .invoke(&[ ColumnarValue::Scalar(ScalarValue::from("day")), @@ -882,6 +883,7 @@ mod tests { .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::() .with_timezone_opt(tz_opt.clone()); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = DateTruncFunc::new() .invoke(&[ ColumnarValue::Scalar(ScalarValue::from("hour")), diff --git a/datafusion/functions/src/datetime/from_unixtime.rs b/datafusion/functions/src/datetime/from_unixtime.rs index ed9858106c52..29b2f29b14c2 100644 --- a/datafusion/functions/src/datetime/from_unixtime.rs +++ b/datafusion/functions/src/datetime/from_unixtime.rs @@ -162,6 +162,7 @@ mod test { fn test_without_timezone() { let args = [ColumnarValue::Scalar(Int64(Some(1729900800)))]; + #[allow(deprecated)] // TODO use invoke_batch let result = FromUnixtimeFunc::new().invoke(&args).unwrap(); match result { @@ -181,6 +182,7 @@ mod test { ))), ]; + #[allow(deprecated)] // TODO use invoke_batch let result = FromUnixtimeFunc::new().invoke(&args).unwrap(); match result { diff --git a/datafusion/functions/src/datetime/make_date.rs b/datafusion/functions/src/datetime/make_date.rs index c8ef349dfbeb..6b246cb088a2 100644 --- a/datafusion/functions/src/datetime/make_date.rs +++ b/datafusion/functions/src/datetime/make_date.rs @@ -234,6 +234,7 @@ mod tests { #[test] fn test_make_date() { + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new() .invoke(&[ ColumnarValue::Scalar(ScalarValue::Int32(Some(2024))), @@ -248,6 +249,7 @@ mod tests { panic!("Expected a scalar value") } + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new() .invoke(&[ ColumnarValue::Scalar(ScalarValue::Int64(Some(2024))), @@ -262,6 +264,7 @@ mod tests { panic!("Expected a scalar value") } + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new() .invoke(&[ ColumnarValue::Scalar(ScalarValue::Utf8(Some("2024".to_string()))), @@ -279,6 +282,7 @@ mod tests { let years = Arc::new((2021..2025).map(Some).collect::()); let months = Arc::new((1..5).map(Some).collect::()); let days = Arc::new((11..15).map(Some).collect::()); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new() .invoke(&[ ColumnarValue::Array(years), @@ -304,6 +308,7 @@ mod tests { // // invalid number of arguments + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new() .invoke(&[ColumnarValue::Scalar(ScalarValue::Int32(Some(1)))]); assert_eq!( @@ -312,6 +317,7 @@ mod tests { ); // invalid type + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new().invoke(&[ ColumnarValue::Scalar(ScalarValue::IntervalYearMonth(Some(1))), ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), @@ -323,6 +329,7 @@ mod tests { ); // overflow of month + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new().invoke(&[ ColumnarValue::Scalar(ScalarValue::Int32(Some(2023))), ColumnarValue::Scalar(ScalarValue::UInt64(Some(u64::MAX))), @@ -334,6 +341,7 @@ mod tests { ); // overflow of day + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let res = MakeDateFunc::new().invoke(&[ ColumnarValue::Scalar(ScalarValue::Int32(Some(2023))), ColumnarValue::Scalar(ScalarValue::Int32(Some(22))), diff --git a/datafusion/functions/src/datetime/to_char.rs b/datafusion/functions/src/datetime/to_char.rs index f0c4a02c1523..ef5d6a4f6990 100644 --- a/datafusion/functions/src/datetime/to_char.rs +++ b/datafusion/functions/src/datetime/to_char.rs @@ -384,6 +384,7 @@ mod tests { ]; for (value, format, expected) in scalar_data { + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ToCharFunc::new() .invoke(&[ColumnarValue::Scalar(value), ColumnarValue::Scalar(format)]) .expect("that to_char parsed values without error"); @@ -458,6 +459,7 @@ mod tests { ]; for (value, format, expected) in scalar_array_data { + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ToCharFunc::new() .invoke(&[ ColumnarValue::Scalar(value), @@ -583,6 +585,7 @@ mod tests { ]; for (value, format, expected) in array_scalar_data { + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ToCharFunc::new() .invoke(&[ ColumnarValue::Array(value as ArrayRef), @@ -599,6 +602,7 @@ mod tests { } for (value, format, expected) in array_array_data { + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ToCharFunc::new() .invoke(&[ ColumnarValue::Array(value), @@ -619,6 +623,7 @@ mod tests { // // invalid number of arguments + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ToCharFunc::new() .invoke(&[ColumnarValue::Scalar(ScalarValue::Int32(Some(1)))]); assert_eq!( @@ -627,6 +632,7 @@ mod tests { ); // invalid type + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ToCharFunc::new().invoke(&[ ColumnarValue::Scalar(ScalarValue::Int32(Some(1))), ColumnarValue::Scalar(ScalarValue::TimestampNanosecond(Some(1), None)), diff --git a/datafusion/functions/src/datetime/to_date.rs b/datafusion/functions/src/datetime/to_date.rs index 82e189698c5e..8f72100416e8 100644 --- a/datafusion/functions/src/datetime/to_date.rs +++ b/datafusion/functions/src/datetime/to_date.rs @@ -213,6 +213,7 @@ mod tests { } fn test_scalar(sv: ScalarValue, tc: &TestCase) { + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ColumnarValue::Scalar(sv)]); match to_date_result { @@ -233,6 +234,7 @@ mod tests { A: From> + Array + 'static, { let date_array = A::from(vec![tc.date_str]); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ColumnarValue::Array(Arc::new(date_array))]); @@ -323,6 +325,7 @@ mod tests { fn test_scalar(sv: ScalarValue, tc: &TestCase) { let format_scalar = ScalarValue::Utf8(Some(tc.format_str.to_string())); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ ColumnarValue::Scalar(sv), ColumnarValue::Scalar(format_scalar), @@ -347,6 +350,7 @@ mod tests { let date_array = A::from(vec![tc.formatted_date]); let format_array = A::from(vec![tc.format_str]); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ ColumnarValue::Array(Arc::new(date_array)), ColumnarValue::Array(Arc::new(format_array)), @@ -382,6 +386,7 @@ mod tests { let format1_scalar = ScalarValue::Utf8(Some("%Y-%m-%d".into())); let format2_scalar = ScalarValue::Utf8(Some("%Y/%m/%d".into())); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ ColumnarValue::Scalar(formatted_date_scalar), ColumnarValue::Scalar(format1_scalar), @@ -410,6 +415,7 @@ mod tests { for date_str in test_cases { let formatted_date_scalar = ScalarValue::Utf8(Some(date_str.into())); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ColumnarValue::Scalar(formatted_date_scalar)]); @@ -428,6 +434,7 @@ mod tests { let date_str = "20241231"; let date_scalar = ScalarValue::Utf8(Some(date_str.into())); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ColumnarValue::Scalar(date_scalar)]); @@ -449,6 +456,7 @@ mod tests { let date_str = "202412311"; let date_scalar = ScalarValue::Utf8(Some(date_str.into())); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let to_date_result = ToDateFunc::new().invoke(&[ColumnarValue::Scalar(date_scalar)]); diff --git a/datafusion/functions/src/datetime/to_local_time.rs b/datafusion/functions/src/datetime/to_local_time.rs index 376cb6f5f2f8..fef1eb9a60c8 100644 --- a/datafusion/functions/src/datetime/to_local_time.rs +++ b/datafusion/functions/src/datetime/to_local_time.rs @@ -558,7 +558,7 @@ mod tests { fn test_to_local_time_helper(input: ScalarValue, expected: ScalarValue) { let res = ToLocalTimeFunc::new() - .invoke(&[ColumnarValue::Scalar(input)]) + .invoke_batch(&[ColumnarValue::Scalar(input)], 1) .unwrap(); match res { ColumnarValue::Scalar(res) => { @@ -616,8 +616,9 @@ mod tests { .iter() .map(|s| Some(string_to_timestamp_nanos(s).unwrap())) .collect::(); + let batch_size = input.len(); let result = ToLocalTimeFunc::new() - .invoke(&[ColumnarValue::Array(Arc::new(input))]) + .invoke_batch(&[ColumnarValue::Array(Arc::new(input))], batch_size) .unwrap(); if let ColumnarValue::Array(result) = result { assert_eq!( diff --git a/datafusion/functions/src/datetime/to_timestamp.rs b/datafusion/functions/src/datetime/to_timestamp.rs index 60482ee3c74a..f15fad701c55 100644 --- a/datafusion/functions/src/datetime/to_timestamp.rs +++ b/datafusion/functions/src/datetime/to_timestamp.rs @@ -636,7 +636,6 @@ mod tests { use arrow::array::{ArrayRef, Int64Array, StringBuilder}; use arrow::datatypes::TimeUnit; use chrono::Utc; - use datafusion_common::{assert_contains, DataFusionError, ScalarValue}; use datafusion_expr::ScalarFunctionImplementation; @@ -1011,7 +1010,7 @@ mod tests { assert!(matches!(rt, Timestamp(_, Some(_)))); let res = udf - .invoke(&[array.clone()]) + .invoke_batch(&[array.clone()], 1) .expect("that to_timestamp parsed values without error"); let array = match res { ColumnarValue::Array(res) => res, @@ -1054,7 +1053,7 @@ mod tests { assert!(matches!(rt, Timestamp(_, None))); let res = udf - .invoke(&[array.clone()]) + .invoke_batch(&[array.clone()], 1) .expect("that to_timestamp parsed values without error"); let array = match res { ColumnarValue::Array(res) => res, diff --git a/datafusion/functions/src/datetime/to_unixtime.rs b/datafusion/functions/src/datetime/to_unixtime.rs index 10f0f87a4ab1..dd90ce6a6c96 100644 --- a/datafusion/functions/src/datetime/to_unixtime.rs +++ b/datafusion/functions/src/datetime/to_unixtime.rs @@ -15,17 +15,16 @@ // specific language governing permissions and limitations // under the License. -use arrow::datatypes::{DataType, TimeUnit}; -use std::any::Any; -use std::sync::OnceLock; - use super::to_timestamp::ToTimestampSecondsFunc; use crate::datetime::common::*; +use arrow::datatypes::{DataType, TimeUnit}; use datafusion_common::{exec_err, Result}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_DATETIME; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; +use std::any::Any; +use std::sync::OnceLock; #[derive(Debug)] pub struct ToUnixtimeFunc { @@ -63,7 +62,11 @@ impl ScalarUDFImpl for ToUnixtimeFunc { Ok(DataType::Int64) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + batch_size: usize, + ) -> Result { if args.is_empty() { return exec_err!("to_unixtime function requires 1 or more arguments, got 0"); } @@ -81,7 +84,7 @@ impl ScalarUDFImpl for ToUnixtimeFunc { .cast_to(&DataType::Timestamp(TimeUnit::Second, None), None)? .cast_to(&DataType::Int64, None), DataType::Utf8 => ToTimestampSecondsFunc::new() - .invoke(args)? + .invoke_batch(args, batch_size)? .cast_to(&DataType::Int64, None), other => { exec_err!("Unsupported data type {:?} for function to_unixtime", other) diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index 9d2e1be3df9d..9110f9f532d8 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -278,7 +278,7 @@ mod tests { ColumnarValue::Array(Arc::new(Int64Array::from(vec![5, 10, 15, 20]))), ]; - let _ = LogFunc::new().invoke(&args); + let _ = LogFunc::new().invoke_batch(&args, 4); } #[test] @@ -287,7 +287,7 @@ mod tests { ColumnarValue::Array(Arc::new(Int64Array::from(vec![10]))), // num ]; - let result = LogFunc::new().invoke(&args); + let result = LogFunc::new().invoke_batch(&args, 1); result.expect_err("expected error"); } @@ -298,7 +298,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 1) .expect("failed to initialize function log"); match result { @@ -322,7 +322,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 1) .expect("failed to initialize function log"); match result { @@ -347,7 +347,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 1) .expect("failed to initialize function log"); match result { @@ -372,7 +372,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 1) .expect("failed to initialize function log"); match result { @@ -398,7 +398,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 4) .expect("failed to initialize function log"); match result { @@ -427,7 +427,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 4) .expect("failed to initialize function log"); match result { @@ -457,7 +457,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 4) .expect("failed to initialize function log"); match result { @@ -487,7 +487,7 @@ mod tests { ]; let result = LogFunc::new() - .invoke(&args) + .invoke_batch(&args, 4) .expect("failed to initialize function log"); match result { diff --git a/datafusion/functions/src/math/power.rs b/datafusion/functions/src/math/power.rs index 9bb6006d55b9..a24c613f5259 100644 --- a/datafusion/functions/src/math/power.rs +++ b/datafusion/functions/src/math/power.rs @@ -207,7 +207,7 @@ mod tests { ]; let result = PowerFunc::new() - .invoke(&args) + .invoke_batch(&args, 4) .expect("failed to initialize function power"); match result { @@ -234,7 +234,7 @@ mod tests { ]; let result = PowerFunc::new() - .invoke(&args) + .invoke_batch(&args, 4) .expect("failed to initialize function power"); match result { diff --git a/datafusion/functions/src/math/signum.rs b/datafusion/functions/src/math/signum.rs index ac881eb42f26..7f21297712c7 100644 --- a/datafusion/functions/src/math/signum.rs +++ b/datafusion/functions/src/math/signum.rs @@ -155,7 +155,7 @@ mod test { #[test] fn test_signum_f32() { - let args = [ColumnarValue::Array(Arc::new(Float32Array::from(vec![ + let array = Arc::new(Float32Array::from(vec![ -1.0, -0.0, 0.0, @@ -165,10 +165,10 @@ mod test { f32::NAN, f32::INFINITY, f32::NEG_INFINITY, - ])))]; - + ])); + let batch_size = array.len(); let result = SignumFunc::new() - .invoke(&args) + .invoke_batch(&[ColumnarValue::Array(array)], batch_size) .expect("failed to initialize function signum"); match result { @@ -195,7 +195,7 @@ mod test { #[test] fn test_signum_f64() { - let args = [ColumnarValue::Array(Arc::new(Float64Array::from(vec![ + let array = Arc::new(Float64Array::from(vec![ -1.0, -0.0, 0.0, @@ -205,10 +205,10 @@ mod test { f64::NAN, f64::INFINITY, f64::NEG_INFINITY, - ])))]; - + ])); + let batch_size = array.len(); let result = SignumFunc::new() - .invoke(&args) + .invoke_batch(&[ColumnarValue::Array(array)], batch_size) .expect("failed to initialize function signum"); match result { diff --git a/datafusion/functions/src/regex/regexpcount.rs b/datafusion/functions/src/regex/regexpcount.rs index 7f7896ecd923..7c4313effffb 100644 --- a/datafusion/functions/src/regex/regexpcount.rs +++ b/datafusion/functions/src/regex/regexpcount.rs @@ -651,8 +651,10 @@ mod tests { let regex_sv = ScalarValue::Utf8(Some(regex.to_string())); let expected = expected.get(pos).cloned(); - let re = RegexpCountFunc::new() - .invoke(&[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)]); + let re = RegexpCountFunc::new().invoke_batch( + &[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -664,8 +666,10 @@ mod tests { let v_sv = ScalarValue::LargeUtf8(Some(v.to_string())); let regex_sv = ScalarValue::LargeUtf8(Some(regex.to_string())); - let re = RegexpCountFunc::new() - .invoke(&[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)]); + let re = RegexpCountFunc::new().invoke_batch( + &[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -677,8 +681,10 @@ mod tests { let v_sv = ScalarValue::Utf8View(Some(v.to_string())); let regex_sv = ScalarValue::Utf8View(Some(regex.to_string())); - let re = RegexpCountFunc::new() - .invoke(&[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)]); + let re = RegexpCountFunc::new().invoke_batch( + &[ColumnarValue::Scalar(v_sv), ColumnarValue::Scalar(regex_sv)], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -701,11 +707,14 @@ mod tests { let start_sv = ScalarValue::Int64(Some(start)); let expected = expected.get(pos).cloned(); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -717,11 +726,14 @@ mod tests { let v_sv = ScalarValue::LargeUtf8(Some(v.to_string())); let regex_sv = ScalarValue::LargeUtf8(Some(regex.to_string())); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -733,11 +745,14 @@ mod tests { let v_sv = ScalarValue::Utf8View(Some(v.to_string())); let regex_sv = ScalarValue::Utf8View(Some(regex.to_string())); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -762,12 +777,15 @@ mod tests { let flags_sv = ScalarValue::Utf8(Some(flags.to_string())); let expected = expected.get(pos).cloned(); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv.clone()), - ColumnarValue::Scalar(flags_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv.clone()), + ColumnarValue::Scalar(flags_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -780,12 +798,15 @@ mod tests { let regex_sv = ScalarValue::LargeUtf8(Some(regex.to_string())); let flags_sv = ScalarValue::LargeUtf8(Some(flags.to_string())); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv.clone()), - ColumnarValue::Scalar(flags_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv.clone()), + ColumnarValue::Scalar(flags_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -798,12 +819,15 @@ mod tests { let regex_sv = ScalarValue::Utf8View(Some(regex.to_string())); let flags_sv = ScalarValue::Utf8View(Some(flags.to_string())); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv), - ColumnarValue::Scalar(flags_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv), + ColumnarValue::Scalar(flags_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -877,12 +901,15 @@ mod tests { let flags_sv = ScalarValue::Utf8(flags.get(pos).map(|f| f.to_string())); let expected = expected.get(pos).cloned(); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv.clone()), - ColumnarValue::Scalar(flags_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv.clone()), + ColumnarValue::Scalar(flags_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -895,12 +922,15 @@ mod tests { let regex_sv = ScalarValue::LargeUtf8(regex.get(pos).map(|s| s.to_string())); let flags_sv = ScalarValue::LargeUtf8(flags.get(pos).map(|f| f.to_string())); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv.clone()), - ColumnarValue::Scalar(flags_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv.clone()), + ColumnarValue::Scalar(flags_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); @@ -913,12 +943,15 @@ mod tests { let regex_sv = ScalarValue::Utf8View(regex.get(pos).map(|s| s.to_string())); let flags_sv = ScalarValue::Utf8View(flags.get(pos).map(|f| f.to_string())); - let re = RegexpCountFunc::new().invoke(&[ - ColumnarValue::Scalar(v_sv), - ColumnarValue::Scalar(regex_sv), - ColumnarValue::Scalar(start_sv), - ColumnarValue::Scalar(flags_sv.clone()), - ]); + let re = RegexpCountFunc::new().invoke_batch( + &[ + ColumnarValue::Scalar(v_sv), + ColumnarValue::Scalar(regex_sv), + ColumnarValue::Scalar(start_sv), + ColumnarValue::Scalar(flags_sv.clone()), + ], + 1, + ); match re { Ok(ColumnarValue::Scalar(ScalarValue::Int64(v))) => { assert_eq!(v, expected, "regexp_count scalar test failed"); diff --git a/datafusion/functions/src/string/concat.rs b/datafusion/functions/src/string/concat.rs index a4218c39e7b2..e3834b291896 100644 --- a/datafusion/functions/src/string/concat.rs +++ b/datafusion/functions/src/string/concat.rs @@ -408,6 +408,7 @@ mod tests { ]))); let args = &[c0, c1, c2]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ConcatFunc::new().invoke(args)?; let expected = Arc::new(StringArray::from(vec!["foo,x", "bar,", "baz,z"])) as ArrayRef; diff --git a/datafusion/functions/src/string/concat_ws.rs b/datafusion/functions/src/string/concat_ws.rs index 8d966f495663..811939c1699b 100644 --- a/datafusion/functions/src/string/concat_ws.rs +++ b/datafusion/functions/src/string/concat_ws.rs @@ -467,6 +467,7 @@ mod tests { ]))); let args = &[c0, c1, c2]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ConcatWsFunc::new().invoke(args)?; let expected = Arc::new(StringArray::from(vec!["foo,x", "bar", "baz,z"])) as ArrayRef; @@ -492,6 +493,7 @@ mod tests { ]))); let args = &[c0, c1, c2]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = ConcatWsFunc::new().invoke(args)?; let expected = Arc::new(StringArray::from(vec![Some("foo,x"), None, Some("baz+z")])) diff --git a/datafusion/functions/src/string/contains.rs b/datafusion/functions/src/string/contains.rs index d0e63bb0f353..0c665a139152 100644 --- a/datafusion/functions/src/string/contains.rs +++ b/datafusion/functions/src/string/contains.rs @@ -145,6 +145,7 @@ mod test { Some("yyy?()"), ]))); let scalar = ColumnarValue::Scalar(ScalarValue::Utf8(Some("x?(".to_string()))); + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let actual = udf.invoke(&[array, scalar]).unwrap(); let expect = ColumnarValue::Array(Arc::new(BooleanArray::from(vec![ Some(true), diff --git a/datafusion/functions/src/string/lower.rs b/datafusion/functions/src/string/lower.rs index b07189a832dc..ef56120c582a 100644 --- a/datafusion/functions/src/string/lower.rs +++ b/datafusion/functions/src/string/lower.rs @@ -105,6 +105,7 @@ mod tests { fn to_lower(input: ArrayRef, expected: ArrayRef) -> Result<()> { let func = LowerFunc::new(); let args = vec![ColumnarValue::Array(input)]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = match func.invoke(&args)? { ColumnarValue::Array(result) => result, _ => unreachable!(), diff --git a/datafusion/functions/src/string/upper.rs b/datafusion/functions/src/string/upper.rs index 042c26b2e3da..68a9d60a1663 100644 --- a/datafusion/functions/src/string/upper.rs +++ b/datafusion/functions/src/string/upper.rs @@ -105,6 +105,7 @@ mod tests { fn to_upper(input: ArrayRef, expected: ArrayRef) -> Result<()> { let func = UpperFunc::new(); let args = vec![ColumnarValue::Array(input)]; + #[allow(deprecated)] // TODO migrate UDF invoke to invoke_batch let result = match func.invoke(&args)? { ColumnarValue::Array(result) => result, _ => unreachable!(), diff --git a/datafusion/functions/src/utils.rs b/datafusion/functions/src/utils.rs index 4d6574d2bd6c..87180cb77de7 100644 --- a/datafusion/functions/src/utils.rs +++ b/datafusion/functions/src/utils.rs @@ -134,6 +134,13 @@ pub mod test { let func = $FUNC; let type_array = $ARGS.iter().map(|arg| arg.data_type()).collect::>(); + let cardinality = $ARGS + .iter() + .fold(Option::::None, |acc, arg| match arg { + ColumnarValue::Scalar(_) => acc, + ColumnarValue::Array(a) => Some(a.len()), + }) + .unwrap_or(1); let return_type = func.return_type(&type_array); match expected { @@ -141,17 +148,10 @@ pub mod test { assert_eq!(return_type.is_ok(), true); assert_eq!(return_type.unwrap(), $EXPECTED_DATA_TYPE); - let result = func.invoke($ARGS); + let result = func.invoke_batch($ARGS, cardinality); assert_eq!(result.is_ok(), true, "function returned an error: {}", result.unwrap_err()); - let len = $ARGS - .iter() - .fold(Option::::None, |acc, arg| match arg { - ColumnarValue::Scalar(_) => acc, - ColumnarValue::Array(a) => Some(a.len()), - }); - let inferred_length = len.unwrap_or(1); - let result = result.unwrap().clone().into_array(inferred_length).expect("Failed to convert to array"); + let result = result.unwrap().clone().into_array(cardinality).expect("Failed to convert to array"); let result = result.as_any().downcast_ref::<$ARRAY_TYPE>().expect("Failed to convert to type"); // value is correct @@ -169,7 +169,7 @@ pub mod test { } else { // invoke is expected error - cannot use .expect_err() due to Debug not being implemented - match func.invoke($ARGS) { + match func.invoke_batch($ARGS, cardinality) { Ok(_) => assert!(false, "expected error"), Err(error) => { assert!(expected_error.strip_backtrace().starts_with(&error.strip_backtrace())); From 49b9351871afe313fa3758c0831eeedfe2582896 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 4 Nov 2024 20:26:45 +0100 Subject: [PATCH 16/28] Support timestamp(n) type (#13231) --- datafusion/sql/src/planner.rs | 16 ++++++--- .../sqllogictest/test_files/timestamps.slt | 35 +++++++++++++++++++ 2 files changed, 47 insertions(+), 4 deletions(-) diff --git a/datafusion/sql/src/planner.rs b/datafusion/sql/src/planner.rs index 072d2320fccf..4d44d5ff2584 100644 --- a/datafusion/sql/src/planner.rs +++ b/datafusion/sql/src/planner.rs @@ -454,7 +454,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLDataType::Char(_) | SQLDataType::Text | SQLDataType::String(_) => Ok(DataType::Utf8), - SQLDataType::Timestamp(None, tz_info) => { + SQLDataType::Timestamp(precision, tz_info) + if precision.is_none() || [0, 3, 6, 9].contains(&precision.unwrap()) => { let tz = if matches!(tz_info, TimezoneInfo::Tz) || matches!(tz_info, TimezoneInfo::WithTimeZone) { @@ -466,7 +467,14 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // Timestamp Without Time zone None }; - Ok(DataType::Timestamp(TimeUnit::Nanosecond, tz.map(Into::into))) + let precision = match precision { + Some(0) => TimeUnit::Second, + Some(3) => TimeUnit::Millisecond, + Some(6) => TimeUnit::Microsecond, + None | Some(9) => TimeUnit::Nanosecond, + _ => unreachable!(), + }; + Ok(DataType::Timestamp(precision, tz.map(Into::into))) } SQLDataType::Date => Ok(DataType::Date32), SQLDataType::Time(None, tz_info) => { @@ -535,8 +543,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { | SQLDataType::CharVarying(_) | SQLDataType::CharacterLargeObject(_) | SQLDataType::CharLargeObject(_) - // Precision is not supported - | SQLDataType::Timestamp(Some(_), _) + // Unsupported precision + | SQLDataType::Timestamp(_, _) // Precision is not supported | SQLDataType::Time(Some(_), _) | SQLDataType::Dec(_) diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index 42abeff674c5..70f7dedeaca0 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -425,6 +425,41 @@ SELECT COUNT(*) FROM ts_data_secs where ts > from_unixtime(1599566400) ---- 2 +query P rowsort +SELECT ts FROM ts_data_nanos; +---- +2020-09-08T11:42:29.190855123 +2020-09-08T12:42:29.190855123 +2020-09-08T13:42:29.190855123 + +query P rowsort +SELECT CAST(ts AS timestamp(0)) FROM ts_data_nanos; +---- +2020-09-08T11:42:29 +2020-09-08T12:42:29 +2020-09-08T13:42:29 + +query P rowsort +SELECT CAST(ts AS timestamp(3)) FROM ts_data_nanos; +---- +2020-09-08T11:42:29.190 +2020-09-08T12:42:29.190 +2020-09-08T13:42:29.190 + +query P rowsort +SELECT CAST(ts AS timestamp(6)) FROM ts_data_nanos; +---- +2020-09-08T11:42:29.190855 +2020-09-08T12:42:29.190855 +2020-09-08T13:42:29.190855 + +query P rowsort +SELECT CAST(ts AS timestamp(9)) FROM ts_data_nanos; +---- +2020-09-08T11:42:29.190855123 +2020-09-08T12:42:29.190855123 +2020-09-08T13:42:29.190855123 + # count_distinct_timestamps query P rowsort From 659d5a33d1249d4c28741afc43aa8d08ce2e8e79 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 4 Nov 2024 20:35:51 +0100 Subject: [PATCH 17/28] Remove elements deprecated since v 38. (#13245) * Remove deprecated transform functions They were deprecated since 38.0.0, which was released 6 months ago. * Remove deprecated and unused FileSinkExec type It was deprecated since 38.0.0, which was released 6 months ago. --- datafusion/common/src/tree_node.rs | 18 ------------------ datafusion/physical-plan/src/insert.rs | 3 --- 2 files changed, 21 deletions(-) diff --git a/datafusion/common/src/tree_node.rs b/datafusion/common/src/tree_node.rs index 563f1fa85614..a0ad1e80be9b 100644 --- a/datafusion/common/src/tree_node.rs +++ b/datafusion/common/src/tree_node.rs @@ -238,15 +238,6 @@ pub trait TreeNode: Sized { transform_down_impl(self, &mut f) } - /// Same as [`Self::transform_down`] but with a mutable closure. - #[deprecated(since = "38.0.0", note = "Use `transform_down` instead")] - fn transform_down_mut Result>>( - self, - f: &mut F, - ) -> Result> { - self.transform_down(f) - } - /// Recursively rewrite the node using `f` in a bottom-up (post-order) /// fashion. /// @@ -271,15 +262,6 @@ pub trait TreeNode: Sized { transform_up_impl(self, &mut f) } - /// Same as [`Self::transform_up`] but with a mutable closure. - #[deprecated(since = "38.0.0", note = "Use `transform_up` instead")] - fn transform_up_mut Result>>( - self, - f: &mut F, - ) -> Result> { - self.transform_up(f) - } - /// Transforms the node using `f_down` while traversing the tree top-down /// (pre-order), and using `f_up` while traversing the tree bottom-up /// (post-order). diff --git a/datafusion/physical-plan/src/insert.rs b/datafusion/physical-plan/src/insert.rs index e478cecb7ffc..ae8a2acce696 100644 --- a/datafusion/physical-plan/src/insert.rs +++ b/datafusion/physical-plan/src/insert.rs @@ -73,9 +73,6 @@ pub trait DataSink: DisplayAs + Debug + Send + Sync { ) -> Result; } -#[deprecated(since = "38.0.0", note = "Use [`DataSinkExec`] instead")] -pub type FileSinkExec = DataSinkExec; - /// Execution plan for writing record batches to a [`DataSink`] /// /// Returns a single row with the number of values written From 0458d30b4bf078251ac4c4dfc4669e8beec8bc3b Mon Sep 17 00:00:00 2001 From: Max Norfolk <66913041+mnorfolk03@users.noreply.github.com> Date: Mon, 4 Nov 2024 15:39:40 -0500 Subject: [PATCH 18/28] fix: CSV Infer Schema now properly supports escaped characters. (#13214) --- .../core/src/datasource/file_format/csv.rs | 56 ++++++++++++++++++- 1 file changed, 55 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/datasource/file_format/csv.rs b/datafusion/core/src/datasource/file_format/csv.rs index 2aaef2cda1c8..0335c8aa3ff6 100644 --- a/datafusion/core/src/datasource/file_format/csv.rs +++ b/datafusion/core/src/datasource/file_format/csv.rs @@ -454,7 +454,12 @@ impl CsvFormat { .has_header .unwrap_or(state.config_options().catalog.has_header), ) - .with_delimiter(self.options.delimiter); + .with_delimiter(self.options.delimiter) + .with_quote(self.options.quote); + + if let Some(escape) = self.options.escape { + format = format.with_escape(escape); + } if let Some(comment) = self.options.comment { format = format.with_comment(comment); @@ -867,6 +872,55 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_infer_schema_escape_chars() -> Result<()> { + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + let variable_object_store = Arc::new(VariableStream::new( + Bytes::from( + r#"c1,c2,c3,c4 +0.3,"Here, is a comma\"",third,3 +0.31,"double quotes are ok, "" quote",third again,9 +0.314,abc,xyz,27"#, + ), + 1, + )); + let object_meta = ObjectMeta { + location: Path::parse("/")?, + last_modified: DateTime::default(), + size: usize::MAX, + e_tag: None, + version: None, + }; + + let num_rows_to_read = 3; + let csv_format = CsvFormat::default() + .with_has_header(true) + .with_schema_infer_max_rec(num_rows_to_read) + .with_quote(b'"') + .with_escape(Some(b'\\')); + + let inferred_schema = csv_format + .infer_schema( + &state, + &(variable_object_store.clone() as Arc), + &[object_meta], + ) + .await?; + + let actual_fields: Vec<_> = inferred_schema + .fields() + .iter() + .map(|f| format!("{}: {:?}", f.name(), f.data_type())) + .collect(); + + assert_eq!( + vec!["c1: Float64", "c2: Utf8", "c3: Utf8", "c4: Int64",], + actual_fields + ); + Ok(()) + } + #[rstest( file_compression_type, case(FileCompressionType::UNCOMPRESSED), From eeb9d58a69f98ef931e23d292f82b44c1c6907e1 Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 5 Nov 2024 00:05:46 -0500 Subject: [PATCH 19/28] Minor: Improve documentation about `OnceAsync` (#13223) * Minor: add documentation about OnceAsync * More refinement * Fix docs CI * Update datafusion/physical-plan/src/joins/hash_join.rs Co-authored-by: Eduard Karacharov --------- Co-authored-by: Eduard Karacharov --- .../physical-plan/src/joins/cross_join.rs | 28 ++++++++++++++----- .../physical-plan/src/joins/hash_join.rs | 13 +++++++-- .../src/joins/nested_loop_join.rs | 19 ++++++++----- datafusion/physical-plan/src/joins/utils.rs | 16 ++++++++--- 4 files changed, 55 insertions(+), 21 deletions(-) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 8c8921eba6a1..7f785006f755 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -46,7 +46,7 @@ use datafusion_physical_expr::equivalence::join_equivalence_properties; use async_trait::async_trait; use futures::{ready, Stream, StreamExt, TryStreamExt}; -/// Data of the left side +/// Data of the left side that is buffered into memory #[derive(Debug)] struct JoinLeftData { /// Single RecordBatch with all rows from the left side @@ -58,12 +58,20 @@ struct JoinLeftData { } #[allow(rustdoc::private_intra_doc_links)] -/// executes partitions in parallel and combines them into a set of -/// partitions by combining all values from the left with all values on the right +/// Cross Join Execution Plan /// -/// Note that the `Clone` trait is not implemented for this struct due to the -/// `left_fut` [`OnceAsync`], which is used to coordinate the loading of the -/// left side with the processing in each output stream. +/// This operator is used when there are no predicates between two tables and +/// returns the Cartesian product of the two tables. +/// +/// Buffers the left input into memory and then streams batches from each +/// partition on the right input combining them with the buffered left input +/// to generate the output. +/// +/// # Clone / Shared State +/// +/// Note this structure includes a [`OnceAsync`] that is used to coordinate the +/// loading of the left side with the processing in each output stream. +/// Therefore it can not be [`Clone`] #[derive(Debug)] pub struct CrossJoinExec { /// left (build) side which gets loaded in memory @@ -72,10 +80,16 @@ pub struct CrossJoinExec { pub right: Arc, /// The schema once the join is applied schema: SchemaRef, - /// Build-side data + /// Buffered copy of left (build) side in memory. + /// + /// This structure is *shared* across all output streams. + /// + /// Each output stream waits on the `OnceAsync` to signal the completion of + /// the left side loading. left_fut: OnceAsync, /// Execution plan metrics metrics: ExecutionPlanMetricsSet, + /// Properties such as schema, equivalence properties, ordering, partitioning, etc. cache: PlanProperties, } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index ae872e13a9f6..32267b118193 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -295,9 +295,11 @@ impl JoinLeftData { /// └───────────────┘ └───────────────┘ /// ``` /// -/// Note that the `Clone` trait is not implemented for this struct due to the -/// `left_fut` [`OnceAsync`], which is used to coordinate the loading of the -/// left side with the processing in each output stream. +/// # Clone / Shared State +/// +/// Note this structure includes a [`OnceAsync`] that is used to coordinate the +/// loading of the left side with the processing in each output stream. +/// Therefore it can not be [`Clone`] #[derive(Debug)] pub struct HashJoinExec { /// left (build) side which gets hashed @@ -314,6 +316,11 @@ pub struct HashJoinExec { /// if there is a projection, the schema isn't the same as the output schema. join_schema: SchemaRef, /// Future that consumes left input and builds the hash table + /// + /// For CollectLeft partition mode, this structure is *shared* across all output streams. + /// + /// Each output stream waits on the `OnceAsync` to signal the completion of + /// the hash table creation. left_fut: OnceAsync, /// Shared the `RandomState` for the hashing algorithm random_state: RandomState, diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index f36c2395e20f..71c617a96300 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -15,9 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! Defines the nested loop join plan, it supports all [`JoinType`]. -//! The nested loop join can execute in parallel by partitions and it is -//! determined by the [`JoinType`]. +//! [`NestedLoopJoinExec`]: joins without equijoin (equality predicates). use std::any::Any; use std::fmt::Formatter; @@ -141,9 +139,11 @@ impl JoinLeftData { /// "reports" about probe phase completion (which means that "visited" bitmap won't be /// updated anymore), and only the last thread, reporting about completion, will return output. /// -/// Note that the `Clone` trait is not implemented for this struct due to the -/// `left_fut` [`OnceAsync`], which is used to coordinate the loading of the -/// left side with the processing in each output stream. +/// # Clone / Shared State +/// +/// Note this structure includes a [`OnceAsync`] that is used to coordinate the +/// loading of the left side with the processing in each output stream. +/// Therefore it can not be [`Clone`] #[derive(Debug)] pub struct NestedLoopJoinExec { /// left side @@ -156,7 +156,12 @@ pub struct NestedLoopJoinExec { pub(crate) join_type: JoinType, /// The schema once the join is applied schema: SchemaRef, - /// Build-side data + /// Future that consumes left input and buffers it in memory + /// + /// This structure is *shared* across all output streams. + /// + /// Each output stream waits on the `OnceAsync` to signal the completion of + /// the hash table creation. inner_table: OnceAsync, /// Information of index and left / right placement of columns column_indices: Vec, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index d3fa37c2ac80..a257119a8b7c 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -700,11 +700,19 @@ pub fn build_join_schema( (fields.finish().with_metadata(metadata), column_indices) } -/// A [`OnceAsync`] can be used to run an async closure once, with subsequent calls -/// to [`OnceAsync::once`] returning a [`OnceFut`] to the same asynchronous computation +/// A [`OnceAsync`] runs an `async` closure once, where multiple calls to +/// [`OnceAsync::once`] return a [`OnceFut`] that resolves to the result of the +/// same computation. /// -/// This is useful for joins where the results of one child are buffered in memory -/// and shared across potentially multiple output partitions +/// This is useful for joins where the results of one child are needed to proceed +/// with multiple output stream +/// +/// +/// For example, in a hash join, one input is buffered and shared across +/// potentially multiple output partitions. Each output partition must wait for +/// the hash table to be built before proceeding. +/// +/// Each output partition waits on the same `OnceAsync` before proceeding. pub(crate) struct OnceAsync { fut: Mutex>>, } From 2e52580b0f10c35b36d02c6f44e13a460de771c9 Mon Sep 17 00:00:00 2001 From: David Rauschenbach Date: Mon, 4 Nov 2024 23:08:12 -0800 Subject: [PATCH 20/28] Introduce HashMap and HashSet type aliases (#13236) * Unite all references to hashbrown::HashMap by using a common type definition * Replace some use of std::collections::HashMap with hashbrown::HashMap * Replace some use of std::collections::HashMap with hashbrown::HashMap * Replace some use of std::collections::HashMap with hashbrown::HashMap * Unite all references to hashbrown::HashSet by using a common type definition * Replace some use of std::collections::HashSet with hashbrown::HashSet --- datafusion/common/src/functional_dependencies.rs | 3 +-- datafusion/common/src/lib.rs | 5 +++++ datafusion/core/src/bin/print_functions_docs.rs | 3 +-- datafusion/core/src/catalog_common/listing_schema.rs | 6 ++++-- datafusion/core/src/datasource/file_format/parquet.rs | 2 +- datafusion/core/src/datasource/listing/helpers.rs | 3 +-- datafusion/core/src/physical_optimizer/sort_pushdown.rs | 4 +--- datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- datafusion/core/tests/fuzz_cases/window_fuzz.rs | 2 +- .../tests/user_defined/user_defined_scalar_functions.rs | 4 ++-- datafusion/execution/src/memory_pool/pool.rs | 2 +- datafusion/expr/src/conditional_expressions.rs | 3 +-- datafusion/expr/src/execution_props.rs | 2 +- datafusion/expr/src/expr.rs | 4 ++-- datafusion/expr/src/registry.rs | 4 ++-- datafusion/expr/src/utils.rs | 4 ++-- datafusion/functions-aggregate/src/median.rs | 3 +-- datafusion/functions-aggregate/src/regr.rs | 7 ++++--- datafusion/functions-nested/src/except.rs | 3 +-- datafusion/functions-nested/src/map.rs | 4 ++-- datafusion/functions/src/core/named_struct.rs | 3 +-- datafusion/functions/src/unicode/translate.rs | 2 +- datafusion/optimizer/src/decorrelate.rs | 4 ++-- datafusion/optimizer/src/optimize_projections/mod.rs | 4 ++-- datafusion/optimizer/src/optimizer.rs | 3 +-- datafusion/optimizer/src/single_distinct_to_groupby.rs | 6 +++--- datafusion/physical-expr-common/src/binary_view_map.rs | 2 +- datafusion/physical-expr/src/expressions/in_list.rs | 2 +- datafusion/physical-expr/src/utils/guarantee.rs | 4 ++-- datafusion/physical-expr/src/utils/mod.rs | 4 +--- datafusion/physical-plan/src/joins/sort_merge_join.rs | 5 ++--- datafusion/physical-plan/src/joins/stream_join_utils.rs | 3 +-- datafusion/physical-plan/src/joins/symmetric_hash_join.rs | 3 +-- datafusion/physical-plan/src/metrics/mod.rs | 2 +- datafusion/physical-plan/src/repartition/mod.rs | 2 +- datafusion/physical-plan/src/topk/mod.rs | 2 +- datafusion/physical-plan/src/unnest.rs | 4 +--- .../physical-plan/src/windows/bounded_window_agg_exec.rs | 6 ++++-- datafusion/sql/src/unparser/rewrite.rs | 7 ++----- datafusion/sql/src/utils.rs | 5 ++--- datafusion/substrait/src/extensions.rs | 3 +-- 41 files changed, 67 insertions(+), 79 deletions(-) diff --git a/datafusion/common/src/functional_dependencies.rs b/datafusion/common/src/functional_dependencies.rs index 31eafc744390..984d8ca267d5 100644 --- a/datafusion/common/src/functional_dependencies.rs +++ b/datafusion/common/src/functional_dependencies.rs @@ -18,13 +18,12 @@ //! FunctionalDependencies keeps track of functional dependencies //! inside DFSchema. -use std::collections::HashSet; use std::fmt::{Display, Formatter}; use std::ops::Deref; use std::vec::IntoIter; use crate::utils::{merge_and_order_indices, set_difference}; -use crate::{DFSchema, JoinType}; +use crate::{DFSchema, HashSet, JoinType}; /// This object defines a constraint on a table. #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash)] diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 08431a36e82f..618e88fb8382 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -66,6 +66,7 @@ pub use functional_dependencies::{ get_target_functional_dependencies, Constraint, Constraints, Dependency, FunctionalDependence, FunctionalDependencies, }; +use hashbrown::hash_map::DefaultHashBuilder; pub use join_type::{JoinConstraint, JoinSide, JoinType}; pub use param_value::ParamValues; pub use scalar::{ScalarType, ScalarValue}; @@ -87,6 +88,10 @@ pub use error::{ _substrait_datafusion_err, }; +// The HashMap and HashSet implementations that should be used as the uniform defaults +pub type HashMap = hashbrown::HashMap; +pub type HashSet = hashbrown::HashSet; + /// Downcast an Arrow Array to a concrete type, return an `DataFusionError::Internal` if the cast is /// not possible. In normal usage of DataFusion the downcast should always succeed. /// diff --git a/datafusion/core/src/bin/print_functions_docs.rs b/datafusion/core/src/bin/print_functions_docs.rs index 3aedcbc2aa63..7f3990c53427 100644 --- a/datafusion/core/src/bin/print_functions_docs.rs +++ b/datafusion/core/src/bin/print_functions_docs.rs @@ -16,12 +16,11 @@ // under the License. use datafusion::execution::SessionStateDefaults; -use datafusion_common::{not_impl_err, Result}; +use datafusion_common::{not_impl_err, HashSet, Result}; use datafusion_expr::{ aggregate_doc_sections, scalar_doc_sections, window_doc_sections, AggregateUDF, DocSection, Documentation, ScalarUDF, WindowUDF, }; -use hashbrown::HashSet; use itertools::Itertools; use std::env::args; use std::fmt::Write as _; diff --git a/datafusion/core/src/catalog_common/listing_schema.rs b/datafusion/core/src/catalog_common/listing_schema.rs index 665ea58c5f75..67952770f41c 100644 --- a/datafusion/core/src/catalog_common/listing_schema.rs +++ b/datafusion/core/src/catalog_common/listing_schema.rs @@ -18,14 +18,16 @@ //! [`ListingSchemaProvider`]: [`SchemaProvider`] that scans ObjectStores for tables automatically use std::any::Any; -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::path::Path; use std::sync::{Arc, Mutex}; use crate::catalog::{SchemaProvider, TableProvider, TableProviderFactory}; use crate::execution::context::SessionState; -use datafusion_common::{Constraints, DFSchema, DataFusionError, TableReference}; +use datafusion_common::{ + Constraints, DFSchema, DataFusionError, HashMap, TableReference, +}; use datafusion_expr::CreateExternalTable; use async_trait::async_trait; diff --git a/datafusion/core/src/datasource/file_format/parquet.rs b/datafusion/core/src/datasource/file_format/parquet.rs index b3f54e0773fd..e27a13b6e735 100644 --- a/datafusion/core/src/datasource/file_format/parquet.rs +++ b/datafusion/core/src/datasource/file_format/parquet.rs @@ -63,7 +63,7 @@ use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; use bytes::Bytes; -use hashbrown::HashMap; +use datafusion_common::HashMap; use log::debug; use object_store::buffered::BufWriter; use parquet::arrow::arrow_writer::{ diff --git a/datafusion/core/src/datasource/listing/helpers.rs b/datafusion/core/src/datasource/listing/helpers.rs index 47012f777ad1..1b3588d9a22d 100644 --- a/datafusion/core/src/datasource/listing/helpers.rs +++ b/datafusion/core/src/datasource/listing/helpers.rs @@ -17,7 +17,6 @@ //! Helper functions for the table implementation -use std::collections::HashMap; use std::mem; use std::sync::Arc; @@ -25,7 +24,7 @@ use super::ListingTableUrl; use super::PartitionedFile; use crate::execution::context::SessionState; use datafusion_common::internal_err; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::{HashMap, Result, ScalarValue}; use datafusion_expr::{BinaryExpr, Operator}; use arrow::{ diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 9eb200f534db..1a53077b1fd5 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -32,7 +32,7 @@ use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, }; -use datafusion_common::{plan_err, JoinSide, Result}; +use datafusion_common::{plan_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; @@ -41,8 +41,6 @@ use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexOrderingRef, LexRequirement, }; -use hashbrown::HashSet; - /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total /// computational cost by pushing down `SortExec`s through some executors. The diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 21f604e6c60f..4cb2b1bfbc5c 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -42,8 +42,8 @@ use test_utils::{add_empty_batches, StringBatchGenerator}; use crate::fuzz_cases::aggregation_fuzzer::{ AggregationFuzzerBuilder, ColumnDescr, DatasetGeneratorConfig, QueryBuilder, }; +use datafusion_common::HashMap; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use hashbrown::HashMap; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; use tokio::task::JoinSet; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 5bfb4d97ed70..e883207f7bfa 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -45,10 +45,10 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; use datafusion::functions_window::row_number::row_number_udwf; +use datafusion_common::HashMap; use datafusion_functions_window::lead_lag::{lag_udwf, lead_udwf}; use datafusion_functions_window::rank::{dense_rank_udwf, rank_udwf}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use hashbrown::HashMap; use rand::distributions::Alphanumeric; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; 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 f1b172862399..8453a360cd25 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -16,7 +16,6 @@ // under the License. use std::any::Any; -use std::collections::HashMap; use std::hash::{DefaultHasher, Hash, Hasher}; use std::sync::Arc; @@ -39,7 +38,8 @@ use datafusion_common::cast::{as_float64_array, as_int32_array}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ assert_batches_eq, assert_batches_sorted_eq, assert_contains, exec_err, internal_err, - not_impl_err, plan_err, DFSchema, DataFusionError, ExprSchema, Result, ScalarValue, + not_impl_err, plan_err, DFSchema, DataFusionError, ExprSchema, HashMap, Result, + ScalarValue, }; use datafusion_expr::simplify::{ExprSimplifyResult, SimplifyInfo}; use datafusion_expr::{ diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index e169c1f319cc..c2ec42d0df1e 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -16,8 +16,8 @@ // under the License. use crate::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; +use datafusion_common::HashMap; use datafusion_common::{resources_datafusion_err, DataFusionError, Result}; -use hashbrown::HashMap; use log::debug; use parking_lot::Mutex; use std::{ diff --git a/datafusion/expr/src/conditional_expressions.rs b/datafusion/expr/src/conditional_expressions.rs index 23cc88f1c0ff..9cb51612d0ca 100644 --- a/datafusion/expr/src/conditional_expressions.rs +++ b/datafusion/expr/src/conditional_expressions.rs @@ -19,8 +19,7 @@ use crate::expr::Case; use crate::{expr_schema::ExprSchemable, Expr}; use arrow::datatypes::DataType; -use datafusion_common::{plan_err, DFSchema, Result}; -use std::collections::HashSet; +use datafusion_common::{plan_err, DFSchema, HashSet, Result}; /// Helper struct for building [Expr::Case] pub struct CaseBuilder { diff --git a/datafusion/expr/src/execution_props.rs b/datafusion/expr/src/execution_props.rs index 3401a94b2736..d672bd1acc46 100644 --- a/datafusion/expr/src/execution_props.rs +++ b/datafusion/expr/src/execution_props.rs @@ -18,7 +18,7 @@ use crate::var_provider::{VarProvider, VarType}; use chrono::{DateTime, TimeZone, Utc}; use datafusion_common::alias::AliasGenerator; -use std::collections::HashMap; +use datafusion_common::HashMap; use std::sync::Arc; /// Holds per-query execution properties and data (such as statement diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index a9c183952fc7..d3a3852a1eaa 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -17,7 +17,7 @@ //! Logical Expressions: [`Expr`] -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::fmt::{self, Display, Formatter, Write}; use std::hash::{Hash, Hasher}; use std::mem; @@ -39,7 +39,7 @@ use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; use datafusion_common::{ - plan_err, Column, DFSchema, Result, ScalarValue, TableReference, + plan_err, Column, DFSchema, HashMap, Result, ScalarValue, TableReference, }; use datafusion_functions_window_common::field::WindowUDFFieldArgs; use sqlparser::ast::{ diff --git a/datafusion/expr/src/registry.rs b/datafusion/expr/src/registry.rs index 6d3457f70d4c..4eb49710bcf8 100644 --- a/datafusion/expr/src/registry.rs +++ b/datafusion/expr/src/registry.rs @@ -20,8 +20,8 @@ use crate::expr_rewriter::FunctionRewrite; use crate::planner::ExprPlanner; use crate::{AggregateUDF, ScalarUDF, UserDefinedLogicalNode, WindowUDF}; -use datafusion_common::{not_impl_err, plan_datafusion_err, Result}; -use std::collections::{HashMap, HashSet}; +use datafusion_common::{not_impl_err, plan_datafusion_err, HashMap, Result}; +use std::collections::HashSet; use std::fmt::Debug; use std::sync::Arc; diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 29c62440abb1..c22ee244fe28 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -18,7 +18,7 @@ //! Expression utilities use std::cmp::Ordering; -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::ops::Deref; use std::sync::Arc; @@ -36,7 +36,7 @@ use datafusion_common::tree_node::{ use datafusion_common::utils::get_at_indices; use datafusion_common::{ internal_err, plan_datafusion_err, plan_err, Column, DFSchema, DFSchemaRef, - DataFusionError, Result, TableReference, + DataFusionError, HashMap, Result, TableReference, }; use indexmap::IndexSet; diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index ff0a930d490b..a7114bb68bfd 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::collections::HashSet; use std::fmt::{Debug, Formatter}; use std::mem::{size_of, size_of_val}; use std::sync::{Arc, OnceLock}; @@ -33,7 +32,7 @@ use arrow::array::Array; use arrow::array::ArrowNativeTypeOp; use arrow::datatypes::ArrowNativeType; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{DataFusionError, HashSet, Result, ScalarValue}; use datafusion_expr::aggregate_doc_sections::DOC_SECTION_GENERAL; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::{ diff --git a/datafusion/functions-aggregate/src/regr.rs b/datafusion/functions-aggregate/src/regr.rs index bf1e81949d23..9dd13634ff2d 100644 --- a/datafusion/functions-aggregate/src/regr.rs +++ b/datafusion/functions-aggregate/src/regr.rs @@ -24,8 +24,10 @@ use arrow::{ datatypes::DataType, datatypes::Field, }; -use datafusion_common::{downcast_value, plan_err, unwrap_or_internal_err, ScalarValue}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{ + downcast_value, plan_err, unwrap_or_internal_err, DataFusionError, HashMap, Result, + ScalarValue, +}; use datafusion_expr::aggregate_doc_sections::DOC_SECTION_STATISTICAL; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; @@ -34,7 +36,6 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, Documentation, Signature, Volatility, }; use std::any::Any; -use std::collections::HashMap; use std::fmt::Debug; use std::mem::size_of_val; use std::sync::OnceLock; diff --git a/datafusion/functions-nested/src/except.rs b/datafusion/functions-nested/src/except.rs index 947d3c018221..100fb587d642 100644 --- a/datafusion/functions-nested/src/except.rs +++ b/datafusion/functions-nested/src/except.rs @@ -23,13 +23,12 @@ use arrow_array::cast::AsArray; use arrow_array::{Array, ArrayRef, GenericListArray, OffsetSizeTrait}; use arrow_buffer::OffsetBuffer; use arrow_schema::{DataType, FieldRef}; -use datafusion_common::{exec_err, internal_err, Result}; +use datafusion_common::{exec_err, internal_err, HashSet, Result}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_ARRAY; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, }; use std::any::Any; -use std::collections::HashSet; use std::sync::{Arc, OnceLock}; make_udf_expr_and_func!( diff --git a/datafusion/functions-nested/src/map.rs b/datafusion/functions-nested/src/map.rs index d7dce3bacbe1..cad193910cee 100644 --- a/datafusion/functions-nested/src/map.rs +++ b/datafusion/functions-nested/src/map.rs @@ -16,7 +16,7 @@ // under the License. use std::any::Any; -use std::collections::{HashSet, VecDeque}; +use std::collections::VecDeque; use std::sync::{Arc, OnceLock}; use arrow::array::ArrayData; @@ -25,7 +25,7 @@ use arrow_buffer::{Buffer, ToByteSlice}; use arrow_schema::{DataType, Field, SchemaBuilder}; use datafusion_common::utils::{fixed_size_list_to_arrays, list_to_arrays}; -use datafusion_common::{exec_err, Result, ScalarValue}; +use datafusion_common::{exec_err, HashSet, Result, ScalarValue}; use datafusion_expr::expr::ScalarFunction; use datafusion_expr::scalar_doc_sections::DOC_SECTION_MAP; use datafusion_expr::{ diff --git a/datafusion/functions/src/core/named_struct.rs b/datafusion/functions/src/core/named_struct.rs index b2c7f06d5868..d53dd2277f84 100644 --- a/datafusion/functions/src/core/named_struct.rs +++ b/datafusion/functions/src/core/named_struct.rs @@ -17,11 +17,10 @@ use arrow::array::StructArray; use arrow::datatypes::{DataType, Field, Fields}; -use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; +use datafusion_common::{exec_err, internal_err, HashSet, Result, ScalarValue}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_STRUCT; use datafusion_expr::{ColumnarValue, Documentation, Expr, ExprSchemable}; use datafusion_expr::{ScalarUDFImpl, Signature, Volatility}; -use hashbrown::HashSet; use std::any::Any; use std::sync::{Arc, OnceLock}; diff --git a/datafusion/functions/src/unicode/translate.rs b/datafusion/functions/src/unicode/translate.rs index fa626b396b3b..845d34c708d4 100644 --- a/datafusion/functions/src/unicode/translate.rs +++ b/datafusion/functions/src/unicode/translate.rs @@ -22,7 +22,7 @@ use arrow::array::{ ArrayAccessor, ArrayIter, ArrayRef, AsArray, GenericStringArray, OffsetSizeTrait, }; use arrow::datatypes::DataType; -use hashbrown::HashMap; +use datafusion_common::HashMap; use unicode_segmentation::UnicodeSegmentation; use crate::utils::{make_scalar_function, utf8_to_str_type}; diff --git a/datafusion/optimizer/src/decorrelate.rs b/datafusion/optimizer/src/decorrelate.rs index 6aa59b77f7f9..b5726d999137 100644 --- a/datafusion/optimizer/src/decorrelate.rs +++ b/datafusion/optimizer/src/decorrelate.rs @@ -17,7 +17,7 @@ //! [`PullUpCorrelatedExpr`] converts correlated subqueries to `Joins` -use std::collections::{BTreeSet, HashMap}; +use std::collections::BTreeSet; use std::ops::Deref; use std::sync::Arc; @@ -27,7 +27,7 @@ use crate::utils::collect_subquery_cols; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter, }; -use datafusion_common::{plan_err, Column, DFSchemaRef, Result, ScalarValue}; +use datafusion_common::{plan_err, Column, DFSchemaRef, HashMap, Result, ScalarValue}; use datafusion_expr::expr::Alias; use datafusion_expr::simplify::SimplifyContext; use datafusion_expr::utils::{conjunction, find_join_exprs, split_conjunction}; diff --git a/datafusion/optimizer/src/optimize_projections/mod.rs b/datafusion/optimizer/src/optimize_projections/mod.rs index ec2225bbc042..67d888abda52 100644 --- a/datafusion/optimizer/src/optimize_projections/mod.rs +++ b/datafusion/optimizer/src/optimize_projections/mod.rs @@ -19,7 +19,7 @@ mod required_indices; -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::sync::Arc; use crate::optimizer::ApplyOrder; @@ -27,7 +27,7 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{ get_required_group_by_exprs_indices, internal_datafusion_err, internal_err, Column, - JoinType, Result, + HashMap, JoinType, Result, }; use datafusion_expr::expr::Alias; use datafusion_expr::Unnest; diff --git a/datafusion/optimizer/src/optimizer.rs b/datafusion/optimizer/src/optimizer.rs index 90a790a0e841..975150cd6122 100644 --- a/datafusion/optimizer/src/optimizer.rs +++ b/datafusion/optimizer/src/optimizer.rs @@ -17,7 +17,6 @@ //! [`Optimizer`] and [`OptimizerRule`] -use std::collections::HashSet; use std::fmt::Debug; use std::sync::Arc; @@ -29,7 +28,7 @@ use datafusion_common::alias::AliasGenerator; use datafusion_common::config::ConfigOptions; use datafusion_common::instant::Instant; use datafusion_common::tree_node::{Transformed, TreeNodeRewriter}; -use datafusion_common::{internal_err, DFSchema, DataFusionError, Result}; +use datafusion_common::{internal_err, DFSchema, DataFusionError, HashSet, Result}; use datafusion_expr::logical_plan::LogicalPlan; use crate::common_subexpr_eliminate::CommonSubexprEliminate; diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index 01875349c922..c8f3a4bc7859 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -22,7 +22,9 @@ use std::sync::Arc; use crate::optimizer::ApplyOrder; use crate::{OptimizerConfig, OptimizerRule}; -use datafusion_common::{internal_err, tree_node::Transformed, DataFusionError, Result}; +use datafusion_common::{ + internal_err, tree_node::Transformed, DataFusionError, HashSet, Result, +}; use datafusion_expr::builder::project; use datafusion_expr::{ col, @@ -31,8 +33,6 @@ use datafusion_expr::{ Expr, }; -use hashbrown::HashSet; - /// single distinct to group by optimizer rule /// ```text /// Before: diff --git a/datafusion/physical-expr-common/src/binary_view_map.rs b/datafusion/physical-expr-common/src/binary_view_map.rs index c6768a19d30e..e131ad8f5085 100644 --- a/datafusion/physical-expr-common/src/binary_view_map.rs +++ b/datafusion/physical-expr-common/src/binary_view_map.rs @@ -393,7 +393,7 @@ where #[cfg(test)] mod tests { use arrow::array::{BinaryViewArray, GenericByteViewArray, StringViewArray}; - use hashbrown::HashMap; + use datafusion_common::HashMap; use super::*; diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index cf57ce3e0e21..1a3cd7600bbb 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -44,8 +44,8 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr_common::datum::compare_with_eq; use ahash::RandomState; +use datafusion_common::HashMap; use hashbrown::hash_map::RawEntryMut; -use hashbrown::HashMap; /// InList pub struct InListExpr { diff --git a/datafusion/physical-expr/src/utils/guarantee.rs b/datafusion/physical-expr/src/utils/guarantee.rs index fbb59cc92fa0..2c37c4d8b394 100644 --- a/datafusion/physical-expr/src/utils/guarantee.rs +++ b/datafusion/physical-expr/src/utils/guarantee.rs @@ -20,9 +20,9 @@ use crate::utils::split_disjunction; use crate::{split_conjunction, PhysicalExpr}; -use datafusion_common::{Column, ScalarValue}; +use datafusion_common::{Column, HashMap, ScalarValue}; use datafusion_expr::Operator; -use std::collections::{HashMap, HashSet}; +use std::collections::HashSet; use std::fmt::{self, Display, Formatter}; use std::sync::Arc; diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index c3d1b1425b7f..73d744b4b614 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -17,10 +17,8 @@ mod guarantee; pub use guarantee::{Guarantee, LiteralGuarantee}; -use hashbrown::HashSet; use std::borrow::Borrow; -use std::collections::HashMap; use std::sync::Arc; use crate::expressions::{BinaryExpr, Column}; @@ -32,7 +30,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, }; -use datafusion_common::Result; +use datafusion_common::{HashMap, HashSet, Result}; use datafusion_expr::Operator; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 3ad892c880f6..2f6dc5fa0bc1 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -43,8 +43,8 @@ use arrow::error::ArrowError; use arrow::ipc::reader::FileReader; use arrow_array::types::UInt64Type; use datafusion_common::{ - exec_err, internal_err, not_impl_err, plan_err, DataFusionError, JoinSide, JoinType, - Result, + exec_err, internal_err, not_impl_err, plan_err, DataFusionError, HashSet, JoinSide, + JoinType, Result, }; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -54,7 +54,6 @@ use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{Stream, StreamExt}; -use hashbrown::HashSet; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 5ccdd9b40dee..f08ce0ea2f0f 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -32,7 +32,7 @@ use arrow_buffer::{ArrowNativeType, BooleanBufferBuilder}; use arrow_schema::{Schema, SchemaRef}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{ - arrow_datafusion_err, DataFusionError, JoinSide, Result, ScalarValue, + arrow_datafusion_err, DataFusionError, HashSet, JoinSide, Result, ScalarValue, }; use datafusion_expr::interval_arithmetic::Interval; use datafusion_physical_expr::expressions::Column; @@ -42,7 +42,6 @@ use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use hashbrown::raw::RawTable; -use hashbrown::HashSet; /// Implementation of `JoinHashMapType` for `PruningJoinHashMap`. impl JoinHashMapType for PruningJoinHashMap { diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 5b6dc2cd2ae9..f082bdbdd3f9 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -64,7 +64,7 @@ use arrow::record_batch::RecordBatch; use arrow_buffer::ArrowNativeType; use datafusion_common::hash_utils::create_hashes; use datafusion_common::utils::bisect; -use datafusion_common::{internal_err, plan_err, JoinSide, JoinType, Result}; +use datafusion_common::{internal_err, plan_err, HashSet, JoinSide, JoinType, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_expr::interval_arithmetic::Interval; @@ -77,7 +77,6 @@ use datafusion_physical_expr_common::sort_expr::{ LexOrdering, LexOrderingRef, LexRequirement, }; use futures::{ready, Stream, StreamExt}; -use hashbrown::HashSet; use parking_lot::Mutex; const HASHMAP_SHRINK_SCALE_FACTOR: usize = 4; diff --git a/datafusion/physical-plan/src/metrics/mod.rs b/datafusion/physical-plan/src/metrics/mod.rs index ead0ca336938..4712729bdaf5 100644 --- a/datafusion/physical-plan/src/metrics/mod.rs +++ b/datafusion/physical-plan/src/metrics/mod.rs @@ -28,7 +28,7 @@ use std::{ sync::Arc, }; -use hashbrown::HashMap; +use datafusion_common::HashMap; // public exports pub use baseline::{BaselineMetrics, RecordOutput}; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index bc65b251561b..4d0dbc75d40a 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -50,10 +50,10 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use crate::execution_plan::CardinalityEffect; +use datafusion_common::HashMap; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; -use hashbrown::HashMap; use log::trace; use parking_lot::Mutex; diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 14469ab6c0d9..27bb3b2b36b9 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -27,6 +27,7 @@ use std::{cmp::Ordering, collections::BinaryHeap, sync::Arc}; use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; use arrow_array::{Array, ArrayRef, RecordBatch}; use arrow_schema::SchemaRef; +use datafusion_common::HashMap; use datafusion_common::Result; use datafusion_execution::{ memory_pool::{MemoryConsumer, MemoryReservation}, @@ -34,7 +35,6 @@ use datafusion_execution::{ }; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use hashbrown::HashMap; use super::metrics::{BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder}; diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index b7b9f17eb1b6..06288a1f7041 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -18,7 +18,6 @@ //! Define a plan for unnesting values in columns that contain a list type. use std::cmp::{self, Ordering}; -use std::collections::HashMap; use std::{any::Any, sync::Arc}; use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; @@ -40,14 +39,13 @@ use arrow::record_batch::RecordBatch; use arrow_array::{Int64Array, Scalar, StructArray}; use arrow_ord::cmp::lt; use datafusion_common::{ - exec_datafusion_err, exec_err, internal_err, Result, UnnestOptions, + exec_datafusion_err, exec_err, internal_err, HashMap, HashSet, Result, UnnestOptions, }; use datafusion_execution::TaskContext; use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use futures::{Stream, StreamExt}; -use hashbrown::HashSet; use log::trace; /// Unnest the given columns (either with type struct or list) 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 8c0331f94570..c3e0a4e3897c 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -22,7 +22,7 @@ use std::any::Any; use std::cmp::{min, Ordering}; -use std::collections::{HashMap, VecDeque}; +use std::collections::VecDeque; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -51,7 +51,9 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::{ evaluate_partition_ranges, get_at_indices, get_row_at_idx, }; -use datafusion_common::{arrow_datafusion_err, exec_err, DataFusionError, Result}; +use datafusion_common::{ + arrow_datafusion_err, exec_err, DataFusionError, HashMap, Result, +}; use datafusion_execution::TaskContext; use datafusion_expr::window_state::{PartitionBatchState, WindowAggState}; use datafusion_expr::ColumnarValue; diff --git a/datafusion/sql/src/unparser/rewrite.rs b/datafusion/sql/src/unparser/rewrite.rs index 57d700f86955..6b3b999ba04b 100644 --- a/datafusion/sql/src/unparser/rewrite.rs +++ b/datafusion/sql/src/unparser/rewrite.rs @@ -15,15 +15,12 @@ // specific language governing permissions and limitations // under the License. -use std::{ - collections::{HashMap, HashSet}, - sync::Arc, -}; +use std::{collections::HashSet, sync::Arc}; use arrow_schema::Schema; use datafusion_common::{ tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, - Column, Result, TableReference, + Column, HashMap, Result, TableReference, }; use datafusion_expr::{expr::Alias, tree_node::transform_sort_vec}; use datafusion_expr::{Expr, LogicalPlan, Projection, Sort, SortExpr}; diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index 14436de01843..e479bdbacd83 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -17,7 +17,6 @@ //! SQL Utility Functions -use std::collections::HashMap; use std::vec; use arrow_schema::{ @@ -27,8 +26,8 @@ use datafusion_common::tree_node::{ Transformed, TransformedResult, TreeNode, TreeNodeRecursion, TreeNodeRewriter, }; use datafusion_common::{ - exec_err, internal_err, plan_err, Column, DFSchemaRef, DataFusionError, Result, - ScalarValue, + exec_err, internal_err, plan_err, Column, DFSchemaRef, DataFusionError, HashMap, + Result, ScalarValue, }; use datafusion_expr::builder::get_struct_unnested_columns; use datafusion_expr::expr::{Alias, GroupingSet, Unnest, WindowFunction}; diff --git a/datafusion/substrait/src/extensions.rs b/datafusion/substrait/src/extensions.rs index 459d0e0c5ae5..c74061f2c9f3 100644 --- a/datafusion/substrait/src/extensions.rs +++ b/datafusion/substrait/src/extensions.rs @@ -15,8 +15,7 @@ // specific language governing permissions and limitations // under the License. -use datafusion::common::{plan_err, DataFusionError}; -use std::collections::HashMap; +use datafusion::common::{plan_err, DataFusionError, HashMap}; use substrait::proto::extensions::simple_extension_declaration::{ ExtensionFunction, ExtensionType, ExtensionTypeVariation, MappingType, }; From 19779d806b88c2aed2b58f2f85831254da6c6563 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Tue, 5 Nov 2024 15:43:50 +0100 Subject: [PATCH 21/28] Remove `Expr` clones from `SortExpr`s (#13258) * Remove `Expr` clones from `SortExpr`s * Update datafusion/expr/src/expr.rs Co-authored-by: Andrew Lamb --------- Co-authored-by: Andrew Lamb --- .../tests/user_defined/user_defined_plan.rs | 3 +- datafusion/expr/src/expr.rs | 9 ++++++ datafusion/expr/src/logical_plan/plan.rs | 7 +++-- datafusion/expr/src/tree_node.rs | 28 +++---------------- .../optimizer/src/common_subexpr_eliminate.rs | 19 ++++++++++--- 5 files changed, 34 insertions(+), 32 deletions(-) diff --git a/datafusion/core/tests/user_defined/user_defined_plan.rs b/datafusion/core/tests/user_defined/user_defined_plan.rs index c96256784402..520a91aeb4d6 100644 --- a/datafusion/core/tests/user_defined/user_defined_plan.rs +++ b/datafusion/core/tests/user_defined/user_defined_plan.rs @@ -97,7 +97,6 @@ use datafusion::{ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; -use datafusion_expr::tree_node::replace_sort_expression; use datafusion_expr::{FetchType, Projection, SortExpr}; use datafusion_optimizer::optimizer::ApplyOrder; use datafusion_optimizer::AnalyzerRule; @@ -440,7 +439,7 @@ impl UserDefinedLogicalNodeCore for TopKPlanNode { Ok(Self { k: self.k, input: inputs.swap_remove(0), - expr: replace_sort_expression(self.expr.clone(), exprs.swap_remove(0)), + expr: self.expr.with_expr(exprs.swap_remove(0)), }) } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index d3a3852a1eaa..025a48731d48 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -629,6 +629,15 @@ impl Sort { nulls_first: !self.nulls_first, } } + + /// Replaces the Sort expressions with `expr` + pub fn with_expr(&self, expr: Expr) -> Self { + Self { + expr, + asc: self.asc, + nulls_first: self.nulls_first, + } + } } impl Display for Sort { diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 191a42e38e3a..ea8fca3ec9d6 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -56,7 +56,6 @@ use indexmap::IndexSet; // backwards compatibility use crate::display::PgJsonVisitor; -use crate::tree_node::replace_sort_expressions; pub use datafusion_common::display::{PlanType, StringifiedPlan, ToStringifiedPlan}; pub use datafusion_common::{JoinConstraint, JoinType}; @@ -866,7 +865,11 @@ impl LogicalPlan { }) => { let input = self.only_input(inputs)?; Ok(LogicalPlan::Sort(Sort { - expr: replace_sort_expressions(sort_expr.clone(), expr), + expr: expr + .into_iter() + .zip(sort_expr.iter()) + .map(|(expr, sort)| sort.with_expr(expr)) + .collect(), input: Arc::new(input), fetch: *fetch, })) diff --git a/datafusion/expr/src/tree_node.rs b/datafusion/expr/src/tree_node.rs index 90afe5722abb..e964091aae66 100644 --- a/datafusion/expr/src/tree_node.rs +++ b/datafusion/expr/src/tree_node.rs @@ -408,29 +408,9 @@ pub fn transform_sort_option_vec Result>>( /// Transforms an vector of sort expressions by applying the provided closure `f`. pub fn transform_sort_vec Result>>( sorts: Vec, - mut f: &mut F, + f: &mut F, ) -> Result>> { - Ok(sorts - .iter() - .map(|sort| sort.expr.clone()) - .map_until_stop_and_collect(&mut f)? - .update_data(|transformed_exprs| { - replace_sort_expressions(sorts, transformed_exprs) - })) -} - -pub fn replace_sort_expressions(sorts: Vec, new_expr: Vec) -> Vec { - assert_eq!(sorts.len(), new_expr.len()); - sorts - .into_iter() - .zip(new_expr) - .map(|(sort, expr)| replace_sort_expression(sort, expr)) - .collect() -} - -pub fn replace_sort_expression(sort: Sort, new_expr: Expr) -> Sort { - Sort { - expr: new_expr, - ..sort - } + sorts.into_iter().map_until_stop_and_collect(|s| { + Ok(f(s.expr)?.update_data(|e| Sort { expr: e, ..s })) + }) } diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index 4fe22d252744..53a0453d8001 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -34,8 +34,7 @@ use datafusion_expr::expr::{Alias, ScalarFunction}; use datafusion_expr::logical_plan::{ Aggregate, Filter, LogicalPlan, Projection, Sort, Window, }; -use datafusion_expr::tree_node::replace_sort_expressions; -use datafusion_expr::{col, BinaryExpr, Case, Expr, Operator}; +use datafusion_expr::{col, BinaryExpr, Case, Expr, Operator, SortExpr}; const CSE_PREFIX: &str = "__common_expr"; @@ -91,6 +90,7 @@ impl CommonSubexprEliminate { .map(LogicalPlan::Projection) }) } + fn try_optimize_sort( &self, sort: Sort, @@ -98,12 +98,23 @@ impl CommonSubexprEliminate { ) -> Result> { let Sort { expr, input, fetch } = sort; let input = Arc::unwrap_or_clone(input); - let sort_expressions = expr.iter().map(|sort| sort.expr.clone()).collect(); + let (sort_expressions, sort_params): (Vec<_>, Vec<(_, _)>) = expr + .into_iter() + .map(|sort| (sort.expr, (sort.asc, sort.nulls_first))) + .unzip(); let new_sort = self .try_unary_plan(sort_expressions, input, config)? .update_data(|(new_expr, new_input)| { LogicalPlan::Sort(Sort { - expr: replace_sort_expressions(expr, new_expr), + expr: new_expr + .into_iter() + .zip(sort_params) + .map(|(expr, (asc, nulls_first))| SortExpr { + expr, + asc, + nulls_first, + }) + .collect(), input: Arc::new(new_input), fetch, }) From 4f169ecca4652c7923c435b90bad49c72d1fad66 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Nov 2024 15:51:12 +0100 Subject: [PATCH 22/28] Update substrait requirement from 0.45 to 0.46 (#13257) Updates the requirements on [substrait](https://github.com/substrait-io/substrait-rs) to permit the latest version. - [Release notes](https://github.com/substrait-io/substrait-rs/releases) - [Changelog](https://github.com/substrait-io/substrait-rs/blob/main/CHANGELOG.md) - [Commits](https://github.com/substrait-io/substrait-rs/compare/v0.45.0...v0.46.1) --- updated-dependencies: - dependency-name: substrait dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- datafusion/substrait/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/substrait/Cargo.toml b/datafusion/substrait/Cargo.toml index b0aa6acf3c7c..9432c798e6d6 100644 --- a/datafusion/substrait/Cargo.toml +++ b/datafusion/substrait/Cargo.toml @@ -41,7 +41,7 @@ object_store = { workspace = true } pbjson-types = "0.7" # TODO use workspace version prost = "0.13" -substrait = { version = "0.45", features = ["serde"] } +substrait = { version = "0.46", features = ["serde"] } url = { workspace = true } [dev-dependencies] From 9005585fa6f4eb6a4d0cc515b6ad76794c33c626 Mon Sep 17 00:00:00 2001 From: Jagdish Parihar Date: Tue, 5 Nov 2024 20:21:36 +0530 Subject: [PATCH 23/28] Deprecate `LexOrderingRef` and `LexRequirementRef` (#13233) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * converted LexOrderingRef to &LexOrdering * using LexOrdering::from_ref fn instead of directly cloning it * using as_ref instread of & * using as_ref * removed commented code * updated cargo lock * updated LexRequirementRef to &LexRequirement * fixed clippy issues * fixed taplo error for cargo.toml in physical-expr-common * removed commented code * fixed clippy errors * fixed clippy error * fixes * removed LexOrdering::from_ref instead using clone and created LexOrdering::empty() fn * Update mod.rs --------- Co-authored-by: Berkay Şahin <124376117+berkaysynnada@users.noreply.github.com> Co-authored-by: berkaysynnada --- benchmarks/src/sort.rs | 39 ++++--- datafusion-cli/Cargo.lock | 33 +++--- .../physical_plan/file_scan_config.rs | 27 ++--- .../datasource/physical_plan/statistics.rs | 34 +++--- .../enforce_distribution.rs | 8 +- .../src/physical_optimizer/enforce_sorting.rs | 17 +-- .../replace_with_order_preserving_variants.rs | 15 ++- .../src/physical_optimizer/sort_pushdown.rs | 72 +++++++------ .../physical_optimizer/update_aggr_exprs.rs | 19 ++-- .../tests/fuzz_cases/equivalence/utils.rs | 6 +- .../src/accumulator.rs | 4 +- .../functions-aggregate-common/src/utils.rs | 6 +- .../functions-aggregate/benches/count.rs | 4 +- datafusion/functions-aggregate/benches/sum.rs | 4 +- .../functions-aggregate/src/array_agg.rs | 2 +- .../functions-aggregate/src/first_last.rs | 11 +- .../functions-aggregate/src/nth_value.rs | 2 +- datafusion/functions-aggregate/src/stddev.rs | 6 +- datafusion/physical-expr-common/Cargo.toml | 1 + .../physical-expr-common/src/sort_expr.rs | 102 ++++++++++++++---- datafusion/physical-expr-common/src/utils.rs | 4 +- datafusion/physical-expr/src/aggregate.rs | 9 +- .../physical-expr/src/equivalence/class.rs | 9 +- .../physical-expr/src/equivalence/ordering.rs | 7 +- .../src/equivalence/properties.rs | 34 +++--- datafusion/physical-expr/src/lib.rs | 3 +- datafusion/physical-expr/src/utils/mod.rs | 4 +- .../physical-expr/src/window/aggregate.rs | 8 +- .../physical-expr/src/window/built_in.rs | 8 +- .../src/window/sliding_aggregate.rs | 8 +- .../physical-expr/src/window/window_expr.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 4 +- .../physical-plan/src/aggregates/order/mod.rs | 4 +- .../src/aggregates/order/partial.rs | 4 +- .../physical-plan/src/execution_plan.rs | 12 +-- .../src/joins/stream_join_utils.rs | 6 +- .../src/joins/symmetric_hash_join.rs | 12 +-- datafusion/physical-plan/src/joins/utils.rs | 16 +-- .../physical-plan/src/repartition/mod.rs | 10 +- .../physical-plan/src/sorts/partial_sort.rs | 3 +- datafusion/physical-plan/src/sorts/sort.rs | 36 +++---- .../src/sorts/sort_preserving_merge.rs | 8 +- datafusion/physical-plan/src/sorts/stream.rs | 4 +- .../src/sorts/streaming_merge.rs | 22 +++- .../src/windows/bounded_window_agg_exec.rs | 7 +- datafusion/physical-plan/src/windows/mod.rs | 10 +- .../proto/src/physical_plan/to_proto.rs | 2 +- .../tests/cases/roundtrip_physical_plan.rs | 27 ++--- 48 files changed, 396 insertions(+), 303 deletions(-) diff --git a/benchmarks/src/sort.rs b/benchmarks/src/sort.rs index b2038c432f77..f4b707611cfb 100644 --- a/benchmarks/src/sort.rs +++ b/benchmarks/src/sort.rs @@ -22,7 +22,7 @@ use crate::util::{AccessLogOpt, BenchmarkRun, CommonOpt}; use arrow::util::pretty; use datafusion::common::Result; -use datafusion::physical_expr::{LexOrdering, LexOrderingRef, PhysicalSortExpr}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::collect; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::prelude::{SessionConfig, SessionContext}; @@ -70,31 +70,28 @@ impl RunOpt { let sort_cases = vec![ ( "sort utf8", - vec![PhysicalSortExpr { + LexOrdering::new(vec![PhysicalSortExpr { expr: col("request_method", &schema)?, options: Default::default(), - }], + }]), ), ( "sort int", - vec![PhysicalSortExpr { - expr: col("request_bytes", &schema)?, + LexOrdering::new(vec![PhysicalSortExpr { + expr: col("response_bytes", &schema)?, options: Default::default(), - }], + }]), ), ( "sort decimal", - vec![ - // sort decimal - PhysicalSortExpr { - expr: col("decimal_price", &schema)?, - options: Default::default(), - }, - ], + LexOrdering::new(vec![PhysicalSortExpr { + expr: col("decimal_price", &schema)?, + options: Default::default(), + }]), ), ( "sort integer tuple", - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("request_bytes", &schema)?, options: Default::default(), @@ -103,11 +100,11 @@ impl RunOpt { expr: col("response_bytes", &schema)?, options: Default::default(), }, - ], + ]), ), ( "sort utf8 tuple", - vec![ + LexOrdering::new(vec![ // sort utf8 tuple PhysicalSortExpr { expr: col("service", &schema)?, @@ -125,11 +122,11 @@ impl RunOpt { expr: col("image", &schema)?, options: Default::default(), }, - ], + ]), ), ( "sort mixed tuple", - vec![ + LexOrdering::new(vec![ PhysicalSortExpr { expr: col("service", &schema)?, options: Default::default(), @@ -142,7 +139,7 @@ impl RunOpt { expr: col("decimal_price", &schema)?, options: Default::default(), }, - ], + ]), ), ]; for (title, expr) in sort_cases { @@ -170,13 +167,13 @@ impl RunOpt { async fn exec_sort( ctx: &SessionContext, - expr: LexOrderingRef<'_>, + expr: &LexOrdering, test_file: &TestParquetFile, debug: bool, ) -> Result<(usize, std::time::Duration)> { let start = Instant::now(); let scan = test_file.create_scan(ctx, None).await?; - let exec = Arc::new(SortExec::new(LexOrdering::new(expr.to_owned()), scan)); + let exec = Arc::new(SortExec::new(expr.clone(), scan)); let task_ctx = ctx.task_ctx(); let result = collect(exec, task_ctx).await?; let elapsed = start.elapsed(); diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 541d464d381f..3348adb10386 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -99,9 +99,9 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.9" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8365de52b16c035ff4fcafe0092ba9390540e3e352870ac09933bebcaa2c8c56" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anstyle-parse" @@ -523,9 +523,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.47.0" +version = "1.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8776850becacbd3a82a4737a9375ddb5c6832a51379f24443a98e61513f852c" +checksum = "ded855583fa1d22e88fe39fd6062b062376e50a8211989e07cf5e38d52eb3453" dependencies = [ "aws-credential-types", "aws-runtime", @@ -545,9 +545,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.48.0" +version = "1.49.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0007b5b8004547133319b6c4e87193eee2a0bcb3e4c18c75d09febe9dab7b383" +checksum = "9177ea1192e6601ae16c7273385690d88a7ed386a00b74a6bc894d12103cd933" dependencies = [ "aws-credential-types", "aws-runtime", @@ -567,9 +567,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.47.0" +version = "1.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fffaa356e7f1c725908b75136d53207fa714e348f365671df14e95a60530ad3" +checksum = "823ef553cf36713c97453e2ddff1eb8f62be7f4523544e2a5db64caf80100f0a" dependencies = [ "aws-credential-types", "aws-runtime", @@ -917,9 +917,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.1.31" +version = "1.1.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2e7962b54006dcfcc61cb72735f4d89bb97061dd6a7ed882ec6b8ee53714c6f" +checksum = "67b9470d453346108f93a59222a9a1a5724db32d0a4727b7ab7ace4b4d822dc9" dependencies = [ "jobserver", "libc", @@ -1520,6 +1520,7 @@ dependencies = [ "datafusion-common", "datafusion-expr-common", "hashbrown 0.14.5", + "itertools", "rand", ] @@ -3614,9 +3615,9 @@ checksum = "13c2bddecc57b384dee18652358fb23172facb8a2c51ccc10d74c157bdea3292" [[package]] name = "syn" -version = "2.0.85" +version = "2.0.87" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5023162dfcd14ef8f32034d8bcd4cc5ddc61ef7a247c024a33e24e1f24d21b56" +checksum = "25aa4ce346d03a6dcd68dd8b4010bcb74e54e62c90c573f394c46eae99aba32d" dependencies = [ "proc-macro2", "quote", @@ -3653,18 +3654,18 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" [[package]] name = "thiserror" -version = "1.0.65" +version = "1.0.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d11abd9594d9b38965ef50805c5e469ca9cc6f197f883f717e0269a3057b3d5" +checksum = "5d171f59dbaa811dbbb1aee1e73db92ec2b122911a48e1390dfe327a821ddede" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.65" +version = "1.0.66" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae71770322cbd277e69d762a16c444af02aa0575ac0d174f0b9562d3b37f8602" +checksum = "b08be0f17bd307950653ce45db00cd31200d82b624b36e181337d9c7d92765b5" dependencies = [ "proc-macro2", "quote", diff --git a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs index 74ab0126a557..6a162c97b666 100644 --- a/datafusion/core/src/datasource/physical_plan/file_scan_config.rs +++ b/datafusion/core/src/datasource/physical_plan/file_scan_config.rs @@ -35,7 +35,6 @@ use arrow_schema::{DataType, Field, Schema, SchemaRef}; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, ColumnStatistics, DataFusionError, Statistics}; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use log::warn; @@ -308,7 +307,7 @@ impl FileScanConfig { pub fn split_groups_by_statistics( table_schema: &SchemaRef, file_groups: &[Vec], - sort_order: LexOrderingRef, + sort_order: &LexOrdering, ) -> Result>> { let flattened_files = file_groups.iter().flatten().collect::>(); // First Fit: @@ -1113,17 +1112,19 @@ mod tests { )))) .collect::>(), )); - let sort_order = case - .sort - .into_iter() - .map(|expr| { - crate::physical_planner::create_physical_sort_expr( - &expr, - &DFSchema::try_from(table_schema.as_ref().clone())?, - &ExecutionProps::default(), - ) - }) - .collect::>>()?; + let sort_order = LexOrdering { + inner: case + .sort + .into_iter() + .map(|expr| { + crate::physical_planner::create_physical_sort_expr( + &expr, + &DFSchema::try_from(table_schema.as_ref().clone())?, + &ExecutionProps::default(), + ) + }) + .collect::>>()?, + }; let partitioned_files = case.files.into_iter().map(From::from).collect::>(); diff --git a/datafusion/core/src/datasource/physical_plan/statistics.rs b/datafusion/core/src/datasource/physical_plan/statistics.rs index 6af153a731b0..488098e7861c 100644 --- a/datafusion/core/src/datasource/physical_plan/statistics.rs +++ b/datafusion/core/src/datasource/physical_plan/statistics.rs @@ -36,7 +36,7 @@ use arrow_array::RecordBatch; use arrow_schema::SchemaRef; use datafusion_common::{DataFusionError, Result}; use datafusion_physical_expr::{expressions::Column, PhysicalSortExpr}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// A normalized representation of file min/max statistics that allows for efficient sorting & comparison. /// The min/max values are ordered by [`Self::sort_order`]. @@ -50,7 +50,7 @@ pub(crate) struct MinMaxStatistics { impl MinMaxStatistics { /// Sort order used to sort the statistics #[allow(unused)] - pub fn sort_order(&self) -> LexOrderingRef { + pub fn sort_order(&self) -> &LexOrdering { &self.sort_order } @@ -66,8 +66,8 @@ impl MinMaxStatistics { } pub fn new_from_files<'a>( - projected_sort_order: LexOrderingRef, // Sort order with respect to projected schema - projected_schema: &SchemaRef, // Projected schema + projected_sort_order: &LexOrdering, // Sort order with respect to projected schema + projected_schema: &SchemaRef, // Projected schema projection: Option<&[usize]>, // Indices of projection in full table schema (None = all columns) files: impl IntoIterator, ) -> Result { @@ -119,15 +119,17 @@ impl MinMaxStatistics { projected_schema .project(&(sort_columns.iter().map(|c| c.index()).collect::>()))?, ); - let min_max_sort_order = sort_columns - .iter() - .zip(projected_sort_order.iter()) - .enumerate() - .map(|(i, (col, sort))| PhysicalSortExpr { - expr: Arc::new(Column::new(col.name(), i)), - options: sort.options, - }) - .collect::>(); + let min_max_sort_order = LexOrdering { + inner: sort_columns + .iter() + .zip(projected_sort_order.iter()) + .enumerate() + .map(|(i, (col, sort))| PhysicalSortExpr { + expr: Arc::new(Column::new(col.name(), i)), + options: sort.options, + }) + .collect::>(), + }; let (min_values, max_values): (Vec<_>, Vec<_>) = sort_columns .iter() @@ -167,7 +169,7 @@ impl MinMaxStatistics { } pub fn new( - sort_order: LexOrderingRef, + sort_order: &LexOrdering, schema: &SchemaRef, min_values: RecordBatch, max_values: RecordBatch, @@ -257,7 +259,7 @@ impl MinMaxStatistics { Ok(Self { min_by_sort_order: min.map_err(|e| e.context("build min rows"))?, max_by_sort_order: max.map_err(|e| e.context("build max rows"))?, - sort_order: LexOrdering::from_ref(sort_order), + sort_order: sort_order.clone(), }) } @@ -278,7 +280,7 @@ impl MinMaxStatistics { } fn sort_columns_from_physical_sort_exprs( - sort_order: LexOrderingRef, + sort_order: &LexOrdering, ) -> Option> { sort_order .iter() diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 6cd902db7244..6863978610db 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -52,12 +52,12 @@ use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, }; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; /// The `EnforceDistribution` rule ensures that distribution requirements are @@ -936,7 +936,11 @@ fn add_spm_on_top(input: DistributionContext) -> DistributionContext { let new_plan = if should_preserve_ordering { Arc::new(SortPreservingMergeExec::new( - LexOrdering::from_ref(input.plan.output_ordering().unwrap_or(&[])), + input + .plan + .output_ordering() + .unwrap_or(&LexOrdering::default()) + .clone(), input.plan.clone(), )) as _ } else { diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 7b111cddc6fd..adc3d7cac10c 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -62,7 +62,7 @@ use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode}; use datafusion_common::plan_err; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_physical_expr::{Partitioning, PhysicalSortRequirement}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; @@ -224,9 +224,9 @@ fn replace_with_partial_sort( let sort_req = PhysicalSortRequirement::from_sort_exprs(sort_plan.expr()); let mut common_prefix_length = 0; - while child_eq_properties - .ordering_satisfy_requirement(&sort_req[0..common_prefix_length + 1]) - { + while child_eq_properties.ordering_satisfy_requirement(&LexRequirement { + inner: sort_req[0..common_prefix_length + 1].to_vec(), + }) { common_prefix_length += 1; } if common_prefix_length > 0 { @@ -392,7 +392,10 @@ fn analyze_immediate_sort_removal( let sort_input = sort_exec.input(); // If this sort is unnecessary, we should remove it: if sort_input.equivalence_properties().ordering_satisfy( - sort_exec.properties().output_ordering().unwrap_or_default(), + sort_exec + .properties() + .output_ordering() + .unwrap_or(LexOrdering::empty()), ) { node.plan = if !sort_exec.preserve_partitioning() && sort_input.output_partitioning().partition_count() > 1 @@ -632,10 +635,10 @@ fn remove_corresponding_sort_from_sub_plan( Ok(node) } -/// Converts an [ExecutionPlan] trait object to a [LexOrderingRef] when possible. +/// Converts an [ExecutionPlan] trait object to a [LexOrdering] reference when possible. fn get_sort_exprs( sort_any: &Arc, -) -> Result<(LexOrderingRef, Option)> { +) -> Result<(&LexOrdering, Option)> { if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { Ok((sort_exec.expr(), sort_exec.fetch())) } else if let Some(spm) = sort_any.as_any().downcast_ref::() diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 930ce52e6fa2..c80aea411f57 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -129,11 +129,13 @@ fn plan_with_order_preserving_variants( return Ok(sort_input); } else if is_coalesce_partitions(&sort_input.plan) && is_spm_better { let child = &sort_input.children[0].plan; - if let Some(ordering) = child.output_ordering().map(Vec::from) { + if let Some(ordering) = child.output_ordering() { // When the input of a `CoalescePartitionsExec` has an ordering, // replace it with a `SortPreservingMergeExec` if appropriate: - let spm = - SortPreservingMergeExec::new(LexOrdering::new(ordering), child.clone()); + let spm = SortPreservingMergeExec::new( + LexOrdering::new(ordering.inner.clone()), + child.clone(), + ); sort_input.plan = Arc::new(spm) as _; sort_input.children[0].data = true; return Ok(sort_input); @@ -257,7 +259,12 @@ pub(crate) fn replace_with_order_preserving_variants( if alternate_plan .plan .equivalence_properties() - .ordering_satisfy(requirements.plan.output_ordering().unwrap_or_default()) + .ordering_satisfy( + requirements + .plan + .output_ordering() + .unwrap_or(LexOrdering::empty()), + ) { for child in alternate_plan.children.iter_mut() { child.data = false; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 1a53077b1fd5..e231e719b25f 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -36,10 +36,8 @@ use datafusion_common::{plan_err, HashSet, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; -use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexOrderingRef, LexRequirement, -}; +use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total @@ -87,11 +85,12 @@ fn pushdown_sorts_helper( let parent_reqs = requirements .data .ordering_requirement - .as_deref() - .unwrap_or(&[]); + .clone() + .unwrap_or_default(); let satisfy_parent = plan .equivalence_properties() - .ordering_satisfy_requirement(parent_reqs); + .ordering_satisfy_requirement(&parent_reqs); + if is_sort(plan) { let required_ordering = plan .output_ordering() @@ -139,7 +138,7 @@ fn pushdown_sorts_helper( for (child, order) in requirements.children.iter_mut().zip(reqs) { child.data.ordering_requirement = order; } - } else if let Some(adjusted) = pushdown_requirement_to_children(plan, parent_reqs)? { + } else if let Some(adjusted) = pushdown_requirement_to_children(plan, &parent_reqs)? { // Can not satisfy the parent requirements, check whether we can push // requirements down: for (child, order) in requirements.children.iter_mut().zip(adjusted) { @@ -162,14 +161,16 @@ fn pushdown_sorts_helper( fn pushdown_requirement_to_children( plan: &Arc, - parent_required: LexRequirementRef, + parent_required: &LexRequirement, ) -> Result>>> { let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); - let request_child = required_input_ordering[0].as_deref().unwrap_or(&[]); + let request_child = required_input_ordering[0].clone().unwrap_or_default(); let child_plan = plan.children().swap_remove(0); - match determine_children_requirement(parent_required, request_child, child_plan) { + + match determine_children_requirement(parent_required, &request_child, child_plan) + { RequirementsCompatibility::Satisfy => { let req = (!request_child.is_empty()) .then(|| LexRequirement::new(request_child.to_vec())); @@ -180,7 +181,10 @@ fn pushdown_requirement_to_children( } } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_req = PhysicalSortRequirement::from_sort_exprs( - sort_exec.properties().output_ordering().unwrap_or(&[]), + sort_exec + .properties() + .output_ordering() + .unwrap_or(&LexOrdering::default()), ); if sort_exec .properties() @@ -202,7 +206,9 @@ fn pushdown_requirement_to_children( .all(|maintain| *maintain) { let output_req = PhysicalSortRequirement::from_sort_exprs( - plan.properties().output_ordering().unwrap_or(&[]), + plan.properties() + .output_ordering() + .unwrap_or(&LexOrdering::default()), ); // Push down through operator with fetch when: // - requirement is aligned with output ordering @@ -229,7 +235,11 @@ fn pushdown_requirement_to_children( let left_columns_len = smj.left().schema().fields().len(); let parent_required_expr = PhysicalSortRequirement::to_sort_exprs(parent_required.iter().cloned()); - match expr_source_side(&parent_required_expr, smj.join_type(), left_columns_len) { + match expr_source_side( + parent_required_expr.as_ref(), + smj.join_type(), + left_columns_len, + ) { Some(JoinSide::Left) => try_pushdown_requirements_to_join( smj, parent_required, @@ -275,7 +285,8 @@ fn pushdown_requirement_to_children( spm_eqs = spm_eqs.with_reorder(new_ordering); // Do not push-down through SortPreservingMergeExec when // ordering requirement invalidates requirement of sort preserving merge exec. - if !spm_eqs.ordering_satisfy(plan.output_ordering().unwrap_or_default()) { + if !spm_eqs.ordering_satisfy(&plan.output_ordering().cloned().unwrap_or_default()) + { Ok(None) } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer @@ -293,7 +304,7 @@ fn pushdown_requirement_to_children( /// Return true if pushing the sort requirements through a node would violate /// the input sorting requirements for the plan fn pushdown_would_violate_requirements( - parent_required: LexRequirementRef, + parent_required: &LexRequirement, child: &dyn ExecutionPlan, ) -> bool { child @@ -319,8 +330,8 @@ fn pushdown_would_violate_requirements( /// - If parent requirements are more specific, push down parent requirements. /// - If they are not compatible, need to add a sort. fn determine_children_requirement( - parent_required: LexRequirementRef, - request_child: LexRequirementRef, + parent_required: &LexRequirement, + request_child: &LexRequirement, child_plan: &Arc, ) -> RequirementsCompatibility { if child_plan @@ -345,8 +356,8 @@ fn determine_children_requirement( fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, - parent_required: LexRequirementRef, - sort_expr: LexOrderingRef, + parent_required: &LexRequirement, + sort_expr: &LexOrdering, push_side: JoinSide, ) -> Result>>> { let left_eq_properties = smj.left().equivalence_properties(); @@ -354,13 +365,13 @@ fn try_pushdown_requirements_to_join( let mut smj_required_orderings = smj.required_input_ordering(); let right_requirement = smj_required_orderings.swap_remove(1); let left_requirement = smj_required_orderings.swap_remove(0); - let left_ordering = smj.left().output_ordering().unwrap_or_default(); - let right_ordering = smj.right().output_ordering().unwrap_or_default(); + let left_ordering = &smj.left().output_ordering().cloned().unwrap_or_default(); + let right_ordering = &smj.right().output_ordering().cloned().unwrap_or_default(); + let (new_left_ordering, new_right_ordering) = match push_side { JoinSide::Left => { - let left_eq_properties = left_eq_properties - .clone() - .with_reorder(LexOrdering::from_ref(sort_expr)); + let left_eq_properties = + left_eq_properties.clone().with_reorder(sort_expr.clone()); if left_eq_properties .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) { @@ -371,9 +382,8 @@ fn try_pushdown_requirements_to_join( } } JoinSide::Right => { - let right_eq_properties = right_eq_properties - .clone() - .with_reorder(LexOrdering::from_ref(sort_expr)); + let right_eq_properties = + right_eq_properties.clone().with_reorder(sort_expr.clone()); if right_eq_properties .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) { @@ -417,7 +427,7 @@ fn try_pushdown_requirements_to_join( } fn expr_source_side( - required_exprs: LexOrderingRef, + required_exprs: &LexOrdering, join_type: JoinType, left_columns_len: usize, ) -> Option { @@ -469,7 +479,7 @@ fn expr_source_side( } fn shift_right_required( - parent_required: LexRequirementRef, + parent_required: &LexRequirement, left_columns_len: usize, ) -> Result { let new_right_required = parent_required @@ -507,7 +517,7 @@ fn shift_right_required( /// pushed down, `Ok(None)` if not. On error, returns a `Result::Err`. fn handle_custom_pushdown( plan: &Arc, - parent_required: LexRequirementRef, + parent_required: &LexRequirement, maintains_input_order: Vec, ) -> Result>>> { // If there's no requirement from the parent or the plan has no children, return early diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index d85278556cc4..c9363b00e18f 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -27,6 +27,7 @@ use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::{ reverse_order_bys, EquivalenceProperties, PhysicalSortRequirement, }; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::concat_slices; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; @@ -138,7 +139,7 @@ fn try_convert_aggregate_if_better( aggr_exprs .into_iter() .map(|aggr_expr| { - let aggr_sort_exprs = &aggr_expr.order_bys().unwrap_or_default(); + let aggr_sort_exprs = &aggr_expr.order_bys().cloned().unwrap_or_default(); let reverse_aggr_sort_exprs = reverse_order_bys(aggr_sort_exprs); let aggr_sort_reqs = PhysicalSortRequirement::from_sort_exprs(aggr_sort_exprs.iter()); @@ -151,14 +152,20 @@ fn try_convert_aggregate_if_better( // Otherwise, leave it as is. if aggr_expr.order_sensitivity().is_beneficial() && !aggr_sort_reqs.is_empty() { - let reqs = concat_slices(prefix_requirement, &aggr_sort_reqs); + let reqs = LexRequirement { + inner: concat_slices(prefix_requirement, &aggr_sort_reqs), + }; + + let prefix_requirement = LexRequirement { + inner: prefix_requirement.to_vec(), + }; + if eq_properties.ordering_satisfy_requirement(&reqs) { // Existing ordering satisfies the aggregator requirements: aggr_expr.with_beneficial_ordering(true)?.map(Arc::new) - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { + } else if eq_properties.ordering_satisfy_requirement(&LexRequirement { + inner: concat_slices(&prefix_requirement, &reverse_aggr_req), + }) { // Converting to reverse enables more efficient execution // given the existing ordering (if possible): aggr_expr diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index 35da8b596380..921332bca539 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -32,7 +32,7 @@ use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ColumnarValue, ScalarUDFImpl, Signature, Volatility}; use datafusion_physical_expr::equivalence::{EquivalenceClass, ProjectionMapping}; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; use rand::prelude::*; @@ -465,7 +465,7 @@ pub fn generate_table_for_orderings( // prune out rows that is invalid according to remaining orderings. for ordering in orderings.iter().skip(1) { - let sort_columns = get_sort_columns(&batch, ordering.as_ref())?; + let sort_columns = get_sort_columns(&batch, ordering)?; // Collect sort options and values into separate vectors. let (sort_options, sort_col_values): (Vec<_>, Vec<_>) = sort_columns @@ -530,7 +530,7 @@ fn generate_random_f64_array( // Helper function to get sort columns from a batch fn get_sort_columns( batch: &RecordBatch, - ordering: LexOrderingRef, + ordering: &LexOrdering, ) -> Result> { ordering .iter() diff --git a/datafusion/functions-aggregate-common/src/accumulator.rs b/datafusion/functions-aggregate-common/src/accumulator.rs index 67ada562800b..a230bb028909 100644 --- a/datafusion/functions-aggregate-common/src/accumulator.rs +++ b/datafusion/functions-aggregate-common/src/accumulator.rs @@ -19,7 +19,7 @@ use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; use datafusion_expr_common::accumulator::Accumulator; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; /// [`AccumulatorArgs`] contains information about how an aggregate @@ -52,7 +52,7 @@ pub struct AccumulatorArgs<'a> { /// ``` /// /// If no `ORDER BY` is specified, `ordering_req` will be empty. - pub ordering_req: LexOrderingRef<'a>, + pub ordering_req: &'a LexOrdering, /// Whether the aggregation is running in reverse order pub is_reversed: bool, diff --git a/datafusion/functions-aggregate-common/src/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs index f55e5ec9a41d..e440abe2de69 100644 --- a/datafusion/functions-aggregate-common/src/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -30,7 +30,7 @@ use arrow::{ }; use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr_common::accumulator::Accumulator; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// Convert scalar values from an accumulator into arrays. pub fn get_accum_scalar_values_as_arrays( @@ -88,7 +88,7 @@ pub fn adjust_output_array(data_type: &DataType, array: ArrayRef) -> Result Vec { @@ -107,7 +107,7 @@ pub fn ordering_fields( } /// Selects the sort option attribute from all the given `PhysicalSortExpr`s. -pub fn get_sort_options(ordering_req: LexOrderingRef) -> Vec { +pub fn get_sort_options(ordering_req: &LexOrdering) -> Vec { ordering_req.iter().map(|item| item.options).collect() } diff --git a/datafusion/functions-aggregate/benches/count.rs b/datafusion/functions-aggregate/benches/count.rs index 1c8266ed5b89..e6b62e6e1856 100644 --- a/datafusion/functions-aggregate/benches/count.rs +++ b/datafusion/functions-aggregate/benches/count.rs @@ -23,7 +23,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::count::Count; use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; fn prepare_accumulator() -> Box { @@ -32,7 +32,7 @@ fn prepare_accumulator() -> Box { return_type: &DataType::Int64, schema: &schema, ignore_nulls: false, - ordering_req: LexOrderingRef::default(), + ordering_req: &LexOrdering::default(), is_reversed: false, name: "COUNT(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/benches/sum.rs b/datafusion/functions-aggregate/benches/sum.rs index 1e9493280ed2..1c180126a313 100644 --- a/datafusion/functions-aggregate/benches/sum.rs +++ b/datafusion/functions-aggregate/benches/sum.rs @@ -23,7 +23,7 @@ use criterion::{black_box, criterion_group, criterion_main, Criterion}; use datafusion_expr::{function::AccumulatorArgs, AggregateUDFImpl, GroupsAccumulator}; use datafusion_functions_aggregate::sum::Sum; use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; fn prepare_accumulator(data_type: &DataType) -> Box { @@ -32,7 +32,7 @@ fn prepare_accumulator(data_type: &DataType) -> Box { return_type: data_type, schema: &schema, ignore_nulls: false, - ordering_req: LexOrderingRef::default(), + ordering_req: &LexOrdering::default(), is_reversed: false, name: "SUM(f)", is_distinct: false, diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 7c22c21e38c9..252a07cb11d8 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -135,7 +135,7 @@ impl AggregateUDFImpl for ArrayAgg { OrderSensitiveArrayAggAccumulator::try_new( &data_type, &ordering_dtypes, - LexOrdering::from_ref(acc_args.ordering_req), + acc_args.ordering_req.clone(), acc_args.is_reversed, ) .map(|acc| Box::new(acc) as _) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 0b05713499a9..3ca1422668e0 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -37,7 +37,7 @@ use datafusion_expr::{ ExprFunctionExt, Signature, SortExpr, TypeSignature, Volatility, }; use datafusion_functions_aggregate_common::utils::get_sort_options; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; create_func!(FirstValue, first_value_udaf); @@ -130,7 +130,7 @@ impl AggregateUDFImpl for FirstValue { FirstValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - LexOrdering::from_ref(acc_args.ordering_req), + acc_args.ordering_req.clone(), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) @@ -455,7 +455,7 @@ impl AggregateUDFImpl for LastValue { LastValueAccumulator::try_new( acc_args.return_type, &ordering_dtypes, - LexOrdering::from_ref(acc_args.ordering_req), + acc_args.ordering_req.clone(), acc_args.ignore_nulls, ) .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) @@ -723,10 +723,7 @@ fn filter_states_according_to_is_set( } /// Combines array refs and their corresponding orderings to construct `SortColumn`s. -fn convert_to_sort_cols( - arrs: &[ArrayRef], - sort_exprs: LexOrderingRef, -) -> Vec { +fn convert_to_sort_cols(arrs: &[ArrayRef], sort_exprs: &LexOrdering) -> Vec { arrs.iter() .zip(sort_exprs.iter()) .map(|(item, sort_expr)| SortColumn { diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index 5f3a8cf2f161..f3e892fa73d8 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -133,7 +133,7 @@ impl AggregateUDFImpl for NthValueAgg { n, &data_type, &ordering_dtypes, - LexOrdering::from_ref(acc_args.ordering_req), + acc_args.ordering_req.clone(), ) .map(|acc| Box::new(acc) as _) } diff --git a/datafusion/functions-aggregate/src/stddev.rs b/datafusion/functions-aggregate/src/stddev.rs index 95269ed8217c..b785d8e9859e 100644 --- a/datafusion/functions-aggregate/src/stddev.rs +++ b/datafusion/functions-aggregate/src/stddev.rs @@ -410,7 +410,7 @@ mod tests { use datafusion_expr::AggregateUDF; use datafusion_functions_aggregate_common::utils::get_accum_scalar_values_as_arrays; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr_common::sort_expr::LexOrderingRef; + use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::sync::Arc; #[test] @@ -462,7 +462,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: LexOrderingRef::default(), + ordering_req: &LexOrdering::default(), name: "a", is_distinct: false, is_reversed: false, @@ -473,7 +473,7 @@ mod tests { return_type: &DataType::Float64, schema, ignore_nulls: false, - ordering_req: LexOrderingRef::default(), + ordering_req: &LexOrdering::default(), name: "a", is_distinct: false, is_reversed: false, diff --git a/datafusion/physical-expr-common/Cargo.toml b/datafusion/physical-expr-common/Cargo.toml index 45ccb08e52e9..ad27c9d49cf7 100644 --- a/datafusion/physical-expr-common/Cargo.toml +++ b/datafusion/physical-expr-common/Cargo.toml @@ -41,4 +41,5 @@ arrow = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr-common = { workspace = true } hashbrown = { workspace = true } +itertools = { workspace = true } rand = { workspace = true } diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index addf2fbfca0c..f91d583215b3 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -22,7 +22,7 @@ use std::fmt; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; use std::ops::{Deref, Index, Range, RangeFrom, RangeTo}; -use std::sync::Arc; +use std::sync::{Arc, OnceLock}; use std::vec::IntoIter; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; @@ -30,6 +30,7 @@ use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_expr_common::columnar_value::ColumnarValue; +use itertools::Itertools; /// Represents Sort operation for a column in a RecordBatch /// @@ -218,7 +219,7 @@ impl From for PhysicalSortExpr { /// If options is `None`, the default sort options `ASC, NULLS LAST` is used. /// /// The default is picked to be consistent with - /// PostgreSQL: + /// PostgreSQL: fn from(value: PhysicalSortRequirement) -> Self { let options = value.options.unwrap_or(SortOptions { descending: false, @@ -309,13 +310,8 @@ impl PhysicalSortRequirement { pub fn from_sort_exprs<'a>( ordering: impl IntoIterator, ) -> LexRequirement { - LexRequirement::new( - ordering - .into_iter() - .cloned() - .map(PhysicalSortRequirement::from) - .collect(), - ) + let ordering = ordering.into_iter().cloned().collect(); + LexRequirement::from_lex_ordering(ordering) } /// Converts an iterator of [`PhysicalSortRequirement`] into a Vec @@ -327,10 +323,8 @@ impl PhysicalSortRequirement { pub fn to_sort_exprs( requirements: impl IntoIterator, ) -> LexOrdering { - requirements - .into_iter() - .map(PhysicalSortExpr::from) - .collect() + let requirements = requirements.into_iter().collect(); + LexOrdering::from_lex_requirement(requirements) } } @@ -352,14 +346,23 @@ pub struct LexOrdering { pub inner: Vec, } +impl AsRef for LexOrdering { + fn as_ref(&self) -> &LexOrdering { + self + } +} + +static EMPTY_ORDER: OnceLock = OnceLock::new(); + impl LexOrdering { // Creates a new [`LexOrdering`] from a vector pub fn new(inner: Vec) -> Self { Self { inner } } - pub fn as_ref(&self) -> LexOrderingRef { - &self.inner + /// Return an empty LexOrdering (no expressions) + pub fn empty() -> &'static LexOrdering { + EMPTY_ORDER.get_or_init(LexOrdering::default) } pub fn capacity(&self) -> usize { @@ -378,10 +381,6 @@ impl LexOrdering { self.inner.extend(iter) } - pub fn from_ref(lex_ordering_ref: LexOrderingRef) -> Self { - Self::new(lex_ordering_ref.to_vec()) - } - pub fn is_empty(&self) -> bool { self.inner.is_empty() } @@ -409,6 +408,36 @@ impl LexOrdering { pub fn truncate(&mut self, len: usize) { self.inner.truncate(len) } + + /// Merge the contents of `other` into `self`, removing duplicates. + pub fn merge(mut self, other: LexOrdering) -> Self { + self.inner = self.inner.into_iter().chain(other).unique().collect(); + self + } + + /// Converts a `LexRequirement` into a `LexOrdering`. + /// + /// This function converts `PhysicalSortRequirement` to `PhysicalSortExpr` + /// for each entry in the input. If required ordering is None for an entry + /// default ordering `ASC, NULLS LAST` if given (see the `PhysicalSortExpr::from`). + pub fn from_lex_requirement(requirements: LexRequirement) -> LexOrdering { + requirements + .into_iter() + .map(PhysicalSortExpr::from) + .collect() + } +} + +impl From> for LexOrdering { + fn from(value: Vec) -> Self { + Self::new(value) + } +} + +impl From for LexOrdering { + fn from(value: LexRequirement) -> Self { + Self::from_lex_requirement(value) + } } impl Deref for LexOrdering { @@ -489,6 +518,7 @@ impl IntoIterator for LexOrdering { ///`LexOrderingRef` is an alias for the type &`[PhysicalSortExpr]`, which represents /// a reference to a lexicographical ordering. +#[deprecated(since = "43.0.0", note = "use &LexOrdering instead")] pub type LexOrderingRef<'a> = &'a [PhysicalSortExpr]; ///`LexRequirement` is an struct containing a `Vec`, which @@ -514,6 +544,30 @@ impl LexRequirement { pub fn push(&mut self, physical_sort_requirement: PhysicalSortRequirement) { self.inner.push(physical_sort_requirement) } + + /// Create a new [`LexRequirement`] from a vector of [`PhysicalSortExpr`]s. + /// + /// Returns [`PhysicalSortRequirement`] that requires the exact + /// sort of the [`PhysicalSortExpr`]s in `ordering` + /// + /// This method takes `&'a PhysicalSortExpr` to make it easy to + /// use implementing [`ExecutionPlan::required_input_ordering`]. + /// + /// [`ExecutionPlan::required_input_ordering`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/trait.ExecutionPlan.html#method.required_input_ordering + pub fn from_lex_ordering(ordering: LexOrdering) -> Self { + Self::new( + ordering + .into_iter() + .map(PhysicalSortRequirement::from) + .collect(), + ) + } +} + +impl From for LexRequirement { + fn from(value: LexOrdering) -> Self { + Self::from_lex_ordering(value) + } } impl Deref for LexRequirement { @@ -545,6 +599,16 @@ impl IntoIterator for LexRequirement { } } +impl<'a> IntoIterator for &'a LexOrdering { + type Item = &'a PhysicalSortExpr; + type IntoIter = std::slice::Iter<'a, PhysicalSortExpr>; + + fn into_iter(self) -> Self::IntoIter { + self.inner.iter() + } +} + ///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which /// represents a reference to a lexicographical ordering requirement. +/// #[deprecated(since = "43.0.0", note = "use &LexRequirement instead")] pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 26293b1a76a2..ffdab6c6d385 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -24,7 +24,7 @@ use datafusion_common::Result; use datafusion_expr_common::sort_properties::ExprProperties; use crate::physical_expr::PhysicalExpr; -use crate::sort_expr::{LexOrdering, LexOrderingRef, PhysicalSortExpr}; +use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; use crate::tree_node::ExprContext; /// Represents a [`PhysicalExpr`] node with associated properties (order and @@ -96,7 +96,7 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { /// Reverses the ORDER BY expression, which is useful during equivalent window /// expression construction. For instance, 'ORDER BY a ASC, NULLS LAST' turns into /// 'ORDER BY a DESC, NULLS FIRST'. -pub fn reverse_order_bys(order_bys: LexOrderingRef) -> LexOrdering { +pub fn reverse_order_bys(order_bys: &LexOrdering) -> LexOrdering { order_bys .iter() .map(|e| PhysicalSortExpr::new(e.expr.clone(), !e.options)) diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index e446776affc0..5dc138933430 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -45,7 +45,7 @@ use datafusion_functions_aggregate_common::accumulator::AccumulatorArgs; use datafusion_functions_aggregate_common::accumulator::StateFieldsArgs; use datafusion_functions_aggregate_common::order::AggregateOrderSensitivity; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_expr_common::utils::reverse_order_bys; use datafusion_expr_common::groups_accumulator::GroupsAccumulator; @@ -292,7 +292,7 @@ impl AggregateFunctionExpr { /// Order by requirements for the aggregate function /// By default it is `None` (there is no requirement) /// Order-sensitive aggregators, such as `FIRST_VALUE(x ORDER BY y)` should implement this - pub fn order_bys(&self) -> Option { + pub fn order_bys(&self) -> Option<&LexOrdering> { if self.ordering_req.is_empty() { return None; } @@ -490,7 +490,10 @@ impl AggregateFunctionExpr { /// These expressions are (1)function arguments, (2) order by expressions. pub fn all_expressions(&self) -> AggregatePhysicalExpressions { let args = self.expressions(); - let order_bys = self.order_bys().unwrap_or_default(); + let order_bys = self + .order_bys() + .cloned() + .unwrap_or_else(LexOrdering::default); let order_by_exprs = order_bys .iter() .map(|sort_expr| Arc::clone(&sort_expr.expr)) diff --git a/datafusion/physical-expr/src/equivalence/class.rs b/datafusion/physical-expr/src/equivalence/class.rs index 7305bc1b0a2b..35ff6f685b53 100644 --- a/datafusion/physical-expr/src/equivalence/class.rs +++ b/datafusion/physical-expr/src/equivalence/class.rs @@ -21,9 +21,8 @@ use std::sync::Arc; use super::{add_offset_to_expr, collapse_lex_req, ProjectionMapping}; use crate::{ expressions::Column, physical_expr::deduplicate_physical_exprs, - physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexOrderingRef, - LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, - PhysicalSortRequirement, + physical_exprs_bag_equal, physical_exprs_contains, LexOrdering, LexRequirement, + PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement, }; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; @@ -475,7 +474,7 @@ impl EquivalenceGroup { /// This function applies the `normalize_sort_expr` function for all sort /// expressions in `sort_exprs` and returns the corresponding normalized /// sort expressions. - pub fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + pub fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { // Convert sort expressions to sort requirements: let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); // Normalize the requirements: @@ -489,7 +488,7 @@ impl EquivalenceGroup { /// sort requirements. pub fn normalize_sort_requirements( &self, - sort_reqs: LexRequirementRef, + sort_reqs: &LexRequirement, ) -> LexRequirement { collapse_lex_req(LexRequirement::new( sort_reqs diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 838c9800f942..06f85b657e09 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -146,12 +146,7 @@ impl OrderingEquivalenceClass { /// Returns the concatenation of all the orderings. This enables merge /// operations to preserve all equivalent orderings simultaneously. pub fn output_ordering(&self) -> Option { - let output_ordering = self - .orderings - .iter() - .flat_map(|ordering| ordering.as_ref()) - .cloned() - .collect(); + let output_ordering = self.orderings.iter().flatten().cloned().collect(); let output_ordering = collapse_lex_ordering(output_ordering); (!output_ordering.is_empty()).then_some(output_ordering) } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 55c99e93d040..061e77222f29 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -30,9 +30,8 @@ use crate::equivalence::{ }; use crate::expressions::{with_new_schema, CastExpr, Column, Literal}; use crate::{ - physical_exprs_contains, ConstExpr, LexOrdering, LexOrderingRef, LexRequirement, - LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, - PhysicalSortRequirement, + physical_exprs_contains, ConstExpr, LexOrdering, LexRequirement, PhysicalExpr, + PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow_schema::{SchemaRef, SortOptions}; @@ -197,7 +196,7 @@ impl EquivalenceProperties { OrderingEquivalenceClass::new( self.oeq_class .iter() - .map(|ordering| self.normalize_sort_exprs(ordering.as_ref())) + .map(|ordering| self.normalize_sort_exprs(ordering)) .collect(), ) } @@ -408,7 +407,7 @@ impl EquivalenceProperties { /// function would return `vec![a ASC, c ASC]`. Internally, it would first /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result /// after deduplication. - fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + fn normalize_sort_exprs(&self, sort_exprs: &LexOrdering) -> LexOrdering { // Convert sort expressions to sort requirements: let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); // Normalize the requirements: @@ -430,10 +429,7 @@ impl EquivalenceProperties { /// function would return `vec![a ASC, c ASC]`. Internally, it would first /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result /// after deduplication. - fn normalize_sort_requirements( - &self, - sort_reqs: LexRequirementRef, - ) -> LexRequirement { + fn normalize_sort_requirements(&self, sort_reqs: &LexRequirement) -> LexRequirement { let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); let mut constant_exprs = vec![]; constant_exprs.extend( @@ -456,7 +452,7 @@ impl EquivalenceProperties { /// Checks whether the given ordering is satisfied by any of the existing /// orderings. - pub fn ordering_satisfy(&self, given: LexOrderingRef) -> bool { + pub fn ordering_satisfy(&self, given: &LexOrdering) -> bool { // Convert the given sort expressions to sort requirements: let sort_requirements = PhysicalSortRequirement::from_sort_exprs(given.iter()); self.ordering_satisfy_requirement(&sort_requirements) @@ -464,7 +460,7 @@ impl EquivalenceProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. - pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + pub fn ordering_satisfy_requirement(&self, reqs: &LexRequirement) -> bool { let mut eq_properties = self.clone(); // First, standardize the given requirement: let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); @@ -525,8 +521,8 @@ impl EquivalenceProperties { /// than the `reference` sort requirements. pub fn requirements_compatible( &self, - given: LexRequirementRef, - reference: LexRequirementRef, + given: &LexRequirement, + reference: &LexRequirement, ) -> bool { let normalized_given = self.normalize_sort_requirements(given); let normalized_reference = self.normalize_sort_requirements(reference); @@ -548,8 +544,8 @@ impl EquivalenceProperties { /// the latter. pub fn get_finer_ordering( &self, - lhs: LexOrderingRef, - rhs: LexOrderingRef, + lhs: &LexOrdering, + rhs: &LexOrdering, ) -> Option { // Convert the given sort expressions to sort requirements: let lhs = PhysicalSortRequirement::from_sort_exprs(lhs); @@ -569,8 +565,8 @@ impl EquivalenceProperties { /// is the latter. pub fn get_finer_requirement( &self, - req1: LexRequirementRef, - req2: LexRequirementRef, + req1: &LexRequirement, + req2: &LexRequirement, ) -> Option { let mut lhs = self.normalize_sort_requirements(req1); let mut rhs = self.normalize_sort_requirements(req2); @@ -606,7 +602,7 @@ impl EquivalenceProperties { pub fn substitute_ordering_component( &self, mapping: &ProjectionMapping, - sort_expr: LexOrderingRef, + sort_expr: &LexOrdering, ) -> Result> { let new_orderings = sort_expr .iter() @@ -656,7 +652,7 @@ impl EquivalenceProperties { let orderings = &self.oeq_class.orderings; let new_order = orderings .iter() - .map(|order| self.substitute_ordering_component(mapping, order.as_ref())) + .map(|order| self.substitute_ordering_component(mapping, order)) .collect::>>()?; let new_order = new_order.into_iter().flatten().collect(); self.oeq_class = OrderingEquivalenceClass::new(new_order); diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e7c2b4119c5a..405b6bbd69f4 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -54,8 +54,7 @@ pub use physical_expr::{ pub use datafusion_physical_expr_common::physical_expr::PhysicalExpr; pub use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalSortExpr, - PhysicalSortRequirement, + LexOrdering, LexRequirement, PhysicalSortExpr, PhysicalSortRequirement, }; pub use planner::{create_physical_expr, create_physical_exprs}; diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 73d744b4b614..30cfecf0e235 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -33,7 +33,7 @@ use datafusion_common::tree_node::{ use datafusion_common::{HashMap, HashSet, Result}; use datafusion_expr::Operator; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; @@ -244,7 +244,7 @@ pub fn reassign_predicate_columns( } /// Merge left and right sort expressions, checking for duplicates. -pub fn merge_vectors(left: LexOrderingRef, right: LexOrderingRef) -> LexOrdering { +pub fn merge_vectors(left: &LexOrdering, right: &LexOrdering) -> LexOrdering { left.iter() .cloned() .chain(right.iter().cloned()) diff --git a/datafusion/physical-expr/src/window/aggregate.rs b/datafusion/physical-expr/src/window/aggregate.rs index 94960c95e4bb..0c56bdc92985 100644 --- a/datafusion/physical-expr/src/window/aggregate.rs +++ b/datafusion/physical-expr/src/window/aggregate.rs @@ -34,7 +34,7 @@ use crate::{reverse_order_bys, PhysicalExpr}; use datafusion_common::ScalarValue; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::{Accumulator, WindowFrame}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// A window expr that takes the form of an aggregate function. /// @@ -52,13 +52,13 @@ impl PlainAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: LexOrderingRef, + order_by: &LexOrdering, window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: LexOrdering::from_ref(order_by), + order_by: order_by.clone(), window_frame, } } @@ -124,7 +124,7 @@ impl WindowExpr for PlainAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> LexOrderingRef { + fn order_by(&self) -> &LexOrdering { self.order_by.as_ref() } diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 5f6c5e5c2c1b..0f6c3f921892 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -33,7 +33,7 @@ use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. #[derive(Debug)] @@ -49,13 +49,13 @@ impl BuiltInWindowExpr { pub fn new( expr: Arc, partition_by: &[Arc], - order_by: LexOrderingRef, + order_by: &LexOrdering, window_frame: Arc, ) -> Self { Self { expr, partition_by: partition_by.to_vec(), - order_by: LexOrdering::from_ref(order_by), + order_by: order_by.clone(), window_frame, } } @@ -118,7 +118,7 @@ impl WindowExpr for BuiltInWindowExpr { &self.partition_by } - fn order_by(&self) -> LexOrderingRef { + fn order_by(&self) -> &LexOrdering { self.order_by.as_ref() } diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 1e46baae7b0a..572eb8866a44 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -33,7 +33,7 @@ use crate::window::{ use crate::{expressions::PhysicalSortExpr, reverse_order_bys, PhysicalExpr}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::{Accumulator, WindowFrame}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexOrderingRef}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; /// A window expr that takes the form of an aggregate function that /// can be incrementally computed over sliding windows. @@ -52,13 +52,13 @@ impl SlidingAggregateWindowExpr { pub fn new( aggregate: Arc, partition_by: &[Arc], - order_by: LexOrderingRef, + order_by: &LexOrdering, window_frame: Arc, ) -> Self { Self { aggregate, partition_by: partition_by.to_vec(), - order_by: LexOrdering::from_ref(order_by), + order_by: order_by.clone(), window_frame, } } @@ -108,7 +108,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { &self.partition_by } - fn order_by(&self) -> LexOrderingRef { + fn order_by(&self) -> &LexOrdering { self.order_by.as_ref() } diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 0f882def4433..828e5ad20625 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -20,7 +20,7 @@ use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; -use crate::{LexOrderingRef, PhysicalExpr}; +use crate::{LexOrdering, PhysicalExpr}; use arrow::array::{new_empty_array, Array, ArrayRef}; use arrow::compute::kernels::sort::SortColumn; @@ -109,7 +109,7 @@ pub trait WindowExpr: Send + Sync + Debug { fn partition_by(&self) -> &[Arc]; /// Expressions that's from the window function's order by clause, empty if absent - fn order_by(&self) -> LexOrderingRef; + fn order_by(&self) -> &LexOrdering; /// Get order by columns, empty if absent fn order_by_columns(&self, batch: &RecordBatch) -> Result> { @@ -344,7 +344,7 @@ pub(crate) fn is_end_bound_safe( window_frame_ctx: &WindowFrameContext, order_bys: &[ArrayRef], most_recent_order_bys: Option<&[ArrayRef]>, - sort_exprs: LexOrderingRef, + sort_exprs: &LexOrdering, idx: usize, ) -> Result { if sort_exprs.is_empty() { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 5ffe797c5c26..a71039b5733b 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -940,7 +940,7 @@ fn get_aggregate_expr_req( return LexOrdering::default(); } - let mut req = LexOrdering::from_ref(aggr_expr.order_bys().unwrap_or_default()); + let mut req = aggr_expr.order_bys().cloned().unwrap_or_default(); // In non-first stage modes, we accumulate data (using `merge_batch`) from // different partitions (i.e. merge partial results). During this merge, we @@ -983,7 +983,7 @@ fn finer_ordering( agg_mode: &AggregateMode, ) -> Option { let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); - eq_properties.get_finer_ordering(existing_req.as_ref(), aggr_req.as_ref()) + eq_properties.get_finer_ordering(existing_req, aggr_req.as_ref()) } /// Concatenates the given slices. diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 24846d239591..7d9a50e20ae0 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -19,7 +19,7 @@ use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; use datafusion_expr::EmitTo; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::mem::size_of; mod full; @@ -45,7 +45,7 @@ impl GroupOrdering { pub fn try_new( input_schema: &Schema, mode: &InputOrderMode, - ordering: LexOrderingRef, + ordering: &LexOrdering, ) -> Result { match mode { InputOrderMode::Linear => Ok(GroupOrdering::None), diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index 5cc55dc0d028..5a05b88798ef 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -21,7 +21,7 @@ use arrow_schema::Schema; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use std::mem::size_of; use std::sync::Arc; @@ -107,7 +107,7 @@ impl GroupOrderingPartial { pub fn try_new( input_schema: &Schema, order_indices: &[usize], - ordering: LexOrderingRef, + ordering: &LexOrdering, ) -> Result { assert!(!order_indices.is_empty()); assert!(order_indices.len() <= ordering.len()); diff --git a/datafusion/physical-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index d65320dbab68..7220e7594ea6 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -38,7 +38,7 @@ pub use datafusion_physical_expr::{ expressions, udf, Distribution, Partitioning, PhysicalExpr, }; use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; -use datafusion_physical_expr_common::sort_expr::{LexOrderingRef, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::display::DisplayableExecutionPlan; @@ -443,7 +443,7 @@ pub trait ExecutionPlanProperties { /// For example, `SortExec` (obviously) produces sorted output as does /// `SortPreservingMergeStream`. Less obviously, `Projection` produces sorted /// output if its input is sorted as it does not reorder the input rows. - fn output_ordering(&self) -> Option; + fn output_ordering(&self) -> Option<&LexOrdering>; /// Get the [`EquivalenceProperties`] within the plan. /// @@ -474,7 +474,7 @@ impl ExecutionPlanProperties for Arc { self.properties().execution_mode() } - fn output_ordering(&self) -> Option { + fn output_ordering(&self) -> Option<&LexOrdering> { self.properties().output_ordering() } @@ -492,7 +492,7 @@ impl ExecutionPlanProperties for &dyn ExecutionPlan { self.properties().execution_mode() } - fn output_ordering(&self) -> Option { + fn output_ordering(&self) -> Option<&LexOrdering> { self.properties().output_ordering() } @@ -643,8 +643,8 @@ impl PlanProperties { &self.partitioning } - pub fn output_ordering(&self) -> Option { - self.output_ordering.as_deref() + pub fn output_ordering(&self) -> Option<&LexOrdering> { + self.output_ordering.as_ref() } pub fn execution_mode(&self) -> ExecutionMode { diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index f08ce0ea2f0f..cea04ccad3fc 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -40,7 +40,7 @@ use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use hashbrown::raw::RawTable; /// Implementation of `JoinHashMapType` for `PruningJoinHashMap`. @@ -744,8 +744,8 @@ pub fn prepare_sorted_exprs( filter: &JoinFilter, left: &Arc, right: &Arc, - left_sort_exprs: LexOrderingRef, - right_sort_exprs: LexOrderingRef, + left_sort_exprs: &LexOrdering, + right_sort_exprs: &LexOrdering, ) -> Result<(SortedFilterExpr, SortedFilterExpr, ExprIntervalGraph)> { let err = || { datafusion_common::plan_datafusion_err!("Filter does not include the child order") diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index f082bdbdd3f9..84c9f03b07be 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -73,9 +73,7 @@ use datafusion_physical_expr::intervals::cp_solver::ExprIntervalGraph; use datafusion_physical_expr::{PhysicalExprRef, PhysicalSortRequirement}; use ahash::RandomState; -use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexOrderingRef, LexRequirement, -}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use futures::{ready, Stream, StreamExt}; use parking_lot::Mutex; @@ -319,13 +317,13 @@ impl SymmetricHashJoinExec { } /// Get left_sort_exprs - pub fn left_sort_exprs(&self) -> Option { - self.left_sort_exprs.as_deref() + pub fn left_sort_exprs(&self) -> Option<&LexOrdering> { + self.left_sort_exprs.as_ref() } /// Get right_sort_exprs - pub fn right_sort_exprs(&self) -> Option { - self.right_sort_exprs.as_deref() + pub fn right_sort_exprs(&self) -> Option<&LexOrdering> { + self.right_sort_exprs.as_ref() } /// Check if order information covers every column in the filter expression. diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index a257119a8b7c..0366c9fa5e46 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -51,7 +51,7 @@ use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::{collect_columns, merge_vectors}; use datafusion_physical_expr::{ - LexOrdering, LexOrderingRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, + LexOrdering, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, }; use futures::future::{BoxFuture, Shared}; @@ -469,7 +469,7 @@ fn replace_on_columns_of_right_ordering( } fn offset_ordering( - ordering: LexOrderingRef, + ordering: &LexOrdering, join_type: &JoinType, offset: usize, ) -> LexOrdering { @@ -483,14 +483,14 @@ fn offset_ordering( options: sort_expr.options, }) .collect(), - _ => LexOrdering::from_ref(ordering), + _ => ordering.clone(), } } /// Calculate the output ordering of a given join operation. pub fn calculate_join_output_ordering( - left_ordering: LexOrderingRef, - right_ordering: LexOrderingRef, + left_ordering: &LexOrdering, + right_ordering: &LexOrdering, join_type: JoinType, on_columns: &[(PhysicalExprRef, PhysicalExprRef)], left_columns_len: usize, @@ -503,7 +503,7 @@ pub fn calculate_join_output_ordering( if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { replace_on_columns_of_right_ordering( on_columns, - &mut LexOrdering::from_ref(right_ordering), + &mut right_ordering.clone(), ) .ok()?; merge_vectors( @@ -512,7 +512,7 @@ pub fn calculate_join_output_ordering( .as_ref(), ) } else { - LexOrdering::from_ref(left_ordering) + left_ordering.clone() } } [false, true] => { @@ -520,7 +520,7 @@ pub fn calculate_join_output_ordering( if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { replace_on_columns_of_right_ordering( on_columns, - &mut LexOrdering::from_ref(right_ordering), + &mut right_ordering.clone(), ) .ok()?; merge_vectors( diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 4d0dbc75d40a..1730c7d8dc61 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -29,6 +29,7 @@ use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use super::{ DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream, }; +use crate::execution_plan::CardinalityEffect; use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; use crate::repartition::distributor_channels::{ @@ -48,10 +49,9 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; -use crate::execution_plan::CardinalityEffect; use datafusion_common::HashMap; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; @@ -503,7 +503,7 @@ impl DisplayAs for RepartitionExec { } if let Some(sort_exprs) = self.sort_exprs() { - write!(f, ", sort_exprs={}", LexOrdering::from_ref(sort_exprs))?; + write!(f, ", sort_exprs={}", sort_exprs.clone())?; } Ok(()) } @@ -572,7 +572,7 @@ impl ExecutionPlan for RepartitionExec { let schema_captured = Arc::clone(&schema); // Get existing ordering to use for merging - let sort_exprs = self.sort_exprs().unwrap_or(&[]).to_owned(); + let sort_exprs = self.sort_exprs().cloned().unwrap_or_default(); let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); @@ -756,7 +756,7 @@ impl RepartitionExec { } /// Return the sort expressions that are used to merge - fn sort_exprs(&self) -> Option<&[PhysicalSortExpr]> { + fn sort_exprs(&self) -> Option<&LexOrdering> { if self.preserve_order { self.input.output_ordering() } else { diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index 8f853464c9bd..e69989c1be91 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -72,7 +72,6 @@ use datafusion_common::Result; use datafusion_execution::{RecordBatchStream, TaskContext}; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use futures::{ready, Stream, StreamExt}; use log::trace; @@ -159,7 +158,7 @@ impl PartialSortExec { } /// Sort expressions - pub fn expr(&self) -> LexOrderingRef { + pub fn expr(&self) -> &LexOrdering { self.expr.as_ref() } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index d90d0f64ceb4..ce7efce41577 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -52,9 +52,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; -use datafusion_physical_expr_common::sort_expr::{ - LexOrderingRef, PhysicalSortRequirement, -}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; use crate::execution_plan::CardinalityEffect; use futures::{StreamExt, TryStreamExt}; @@ -344,10 +342,12 @@ impl ExternalSorter { streams.push(stream); } + let expressions: LexOrdering = self.expr.iter().cloned().collect(); + StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(self.expr.to_vec().as_slice()) + .with_expressions(expressions.as_ref()) .with_metrics(self.metrics.baseline.clone()) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -536,10 +536,12 @@ impl ExternalSorter { }) .collect::>()?; + let expressions: LexOrdering = self.expr.iter().cloned().collect(); + StreamingMergeBuilder::new() .with_streams(streams) .with_schema(Arc::clone(&self.schema)) - .with_expressions(self.expr.as_ref()) + .with_expressions(expressions.as_ref()) .with_metrics(metrics) .with_batch_size(self.batch_size) .with_fetch(self.fetch) @@ -561,7 +563,7 @@ impl ExternalSorter { let schema = batch.schema(); let fetch = self.fetch; - let expressions = Arc::clone(&self.expr); + let expressions: LexOrdering = self.expr.iter().cloned().collect(); let stream = futures::stream::once(futures::future::lazy(move |_| { let timer = metrics.elapsed_compute().timer(); let sorted = sort_batch(&batch, &expressions, fetch)?; @@ -603,7 +605,7 @@ impl Debug for ExternalSorter { pub fn sort_batch( batch: &RecordBatch, - expressions: LexOrderingRef, + expressions: &LexOrdering, fetch: Option, ) -> Result { let sort_columns = expressions @@ -762,8 +764,8 @@ impl SortExec { } /// Sort expressions - pub fn expr(&self) -> LexOrderingRef { - self.expr.as_ref() + pub fn expr(&self) -> &LexOrdering { + &self.expr } /// If `Some(fetch)`, limits output to only the first "fetch" items @@ -790,11 +792,10 @@ impl SortExec { preserve_partitioning: bool, ) -> PlanProperties { // Determine execution mode: - let sort_satisfied = input.equivalence_properties().ordering_satisfy_requirement( - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()) - .inner - .as_slice(), - ); + let requirement = LexRequirement::from(sort_exprs); + let sort_satisfied = input + .equivalence_properties() + .ordering_satisfy_requirement(&requirement); let mode = match input.execution_mode() { ExecutionMode::Unbounded if sort_satisfied => ExecutionMode::Unbounded, ExecutionMode::Bounded => ExecutionMode::Bounded, @@ -803,6 +804,7 @@ impl SortExec { // Calculate equivalence properties; i.e. reset the ordering equivalence // class with the new ordering: + let sort_exprs = LexOrdering::from(requirement); let eq_properties = input .equivalence_properties() .clone() @@ -890,11 +892,7 @@ impl ExecutionPlan for SortExec { let sort_satisfied = self .input .equivalence_properties() - .ordering_satisfy_requirement( - PhysicalSortRequirement::from_sort_exprs(self.expr.iter()) - .inner - .as_slice(), - ); + .ordering_satisfy_requirement(&LexRequirement::from(self.expr.clone())); match (sort_satisfied, self.fetch.as_ref()) { (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 9ee0faaa0a44..5c80322afe0c 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -34,9 +34,7 @@ use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; -use datafusion_physical_expr_common::sort_expr::{ - LexOrdering, LexOrderingRef, LexRequirement, -}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -122,8 +120,8 @@ impl SortPreservingMergeExec { } /// Sort expressions - pub fn expr(&self) -> LexOrderingRef { - &self.expr + pub fn expr(&self) -> &LexOrdering { + self.expr.as_ref() } /// Fetch diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index 70beb2c4a91b..7c57fdf9baae 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -24,7 +24,7 @@ use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::{Fuse, StreamExt}; use std::marker::PhantomData; use std::sync::Arc; @@ -93,7 +93,7 @@ pub struct RowCursorStream { impl RowCursorStream { pub fn try_new( schema: &Schema, - expressions: LexOrderingRef, + expressions: &LexOrdering, streams: Vec, reservation: MemoryReservation, ) -> Result { diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index bd74685eac94..4350235ef47d 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -28,7 +28,7 @@ use arrow::datatypes::{DataType, SchemaRef}; use arrow_array::*; use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryReservation; -use datafusion_physical_expr_common::sort_expr::LexOrderingRef; +use datafusion_physical_expr_common::sort_expr::LexOrdering; macro_rules! primitive_merge_helper { ($t:ty, $($v:ident),+) => { @@ -51,11 +51,10 @@ macro_rules! merge_helper { }}; } -#[derive(Default)] pub struct StreamingMergeBuilder<'a> { streams: Vec, schema: Option, - expressions: LexOrderingRef<'a>, + expressions: &'a LexOrdering, metrics: Option, batch_size: Option, fetch: Option, @@ -63,6 +62,21 @@ pub struct StreamingMergeBuilder<'a> { enable_round_robin_tie_breaker: bool, } +impl<'a> Default for StreamingMergeBuilder<'a> { + fn default() -> Self { + Self { + streams: vec![], + schema: None, + expressions: LexOrdering::empty(), + metrics: None, + batch_size: None, + fetch: None, + reservation: None, + enable_round_robin_tie_breaker: false, + } + } +} + impl<'a> StreamingMergeBuilder<'a> { pub fn new() -> Self { Self { @@ -81,7 +95,7 @@ impl<'a> StreamingMergeBuilder<'a> { self } - pub fn with_expressions(mut self, expressions: LexOrderingRef<'a>) -> Self { + pub fn with_expressions(mut self, expressions: &'a LexOrdering) -> Self { self.expressions = expressions; self } 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 c3e0a4e3897c..602efa54f8da 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1188,7 +1188,6 @@ mod tests { }; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; - use datafusion_physical_expr_common::sort_expr::LexOrderingRef; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1555,7 +1554,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( last_value_func, &[], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1566,7 +1565,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( nth_value_func1, &[], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), @@ -1577,7 +1576,7 @@ mod tests { Arc::new(BuiltInWindowExpr::new( nth_value_func2, &[], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(WindowFrame::new_bounds( WindowFrameUnits::Rows, WindowFrameBound::Preceding(ScalarValue::UInt64(None)), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 217823fb6a0a..da7f6d79e578 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -53,7 +53,7 @@ use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, }; -use datafusion_physical_expr_common::sort_expr::{LexOrderingRef, LexRequirement}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; pub use window_agg_exec::WindowAggExec; /// Build field from window function and add it into schema @@ -98,7 +98,7 @@ pub fn create_window_expr( name: String, args: &[Arc], partition_by: &[Arc], - order_by: LexOrderingRef, + order_by: &LexOrdering, window_frame: Arc, input_schema: &Schema, ignore_nulls: bool, @@ -139,7 +139,7 @@ pub fn create_window_expr( /// Creates an appropriate [`WindowExpr`] based on the window frame and fn window_expr_from_aggregate_expr( partition_by: &[Arc], - order_by: LexOrderingRef, + order_by: &LexOrdering, window_frame: Arc, aggregate: Arc, ) -> Arc { @@ -497,7 +497,7 @@ pub fn get_best_fitting_window( /// the mode this window operator should work in to accommodate the existing ordering. pub fn get_window_mode( partitionby_exprs: &[Arc], - orderby_keys: LexOrderingRef, + orderby_keys: &LexOrdering, input: &Arc, ) -> Option<(bool, InputOrderMode)> { let input_eqs = input.equivalence_properties().clone(); @@ -699,7 +699,7 @@ mod tests { "count".to_owned(), &[col("a", &schema)?], &[], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(WindowFrame::new(None)), schema.as_ref(), false, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 4bf7e353326e..dc94ad075c53 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -53,7 +53,7 @@ pub fn serialize_physical_aggr_expr( ) -> Result { let expressions = serialize_physical_exprs(&aggr_expr.expressions(), codec)?; let ordering_req = match aggr_expr.order_bys() { - Some(order) => LexOrdering::from_ref(order), + Some(order) => order.clone(), None => LexOrdering::default(), }; let ordering_req = serialize_physical_sort_exprs(ordering_req, codec)?; diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 1e078ee410c6..8c8dcccee376 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -52,8 +52,7 @@ use datafusion::logical_expr::{create_udf, JoinType, Operator, Volatility}; use datafusion::physical_expr::expressions::Literal; use datafusion::physical_expr::window::SlidingAggregateWindowExpr; use datafusion::physical_expr::{ - LexOrdering, LexOrderingRef, LexRequirement, PhysicalSortRequirement, - ScalarFunctionExpr, + LexOrdering, LexRequirement, PhysicalSortRequirement, ScalarFunctionExpr, }; use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -288,13 +287,15 @@ fn roundtrip_window() -> Result<()> { false, )), &[col("b", &schema)?], - &[PhysicalSortExpr { - expr: col("a", &schema)?, - options: SortOptions { - descending: false, - nulls_first: false, - }, - }], + &LexOrdering{ + inner: vec![PhysicalSortExpr { + expr: col("a", &schema)?, + options: SortOptions { + descending: false, + nulls_first: false, + }, + }] + }, Arc::new(window_frame), )); @@ -308,7 +309,7 @@ fn roundtrip_window() -> Result<()> { .build() .map(Arc::new)?, &[], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(WindowFrame::new(None)), )); @@ -328,7 +329,7 @@ fn roundtrip_window() -> Result<()> { let sliding_aggr_window_expr = Arc::new(SlidingAggregateWindowExpr::new( sum_expr, &[], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(window_frame), )); @@ -1014,7 +1015,7 @@ fn roundtrip_scalar_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr.clone(), &[col("author", &schema)?], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(WindowFrame::new(None)), ))], filter, @@ -1075,7 +1076,7 @@ fn roundtrip_aggregate_udf_extension_codec() -> Result<()> { vec![Arc::new(PlainAggregateWindowExpr::new( aggr_expr, &[col("author", &schema)?], - LexOrderingRef::default(), + &LexOrdering::default(), Arc::new(WindowFrame::new(None)), ))], filter, From 003813ae307aa24a0d8369dba9977355431b799d Mon Sep 17 00:00:00 2001 From: Andrew Lamb Date: Tue, 5 Nov 2024 10:54:33 -0500 Subject: [PATCH 24/28] Optimize Statistics::projection (#13225) --- datafusion/common/src/stats.rs | 78 +++++++++++++++++++++++++++++++--- 1 file changed, 73 insertions(+), 5 deletions(-) diff --git a/datafusion/common/src/stats.rs b/datafusion/common/src/stats.rs index 1aa42705e7f8..d2ce965c5c49 100644 --- a/datafusion/common/src/stats.rs +++ b/datafusion/common/src/stats.rs @@ -268,13 +268,35 @@ impl Statistics { return self; }; - // todo: it would be nice to avoid cloning column statistics if - // possible (e.g. if the projection did not contain duplicates) - self.column_statistics = projection - .iter() - .map(|&i| self.column_statistics[i].clone()) + enum Slot { + /// The column is taken and put into the specified statistics location + Taken(usize), + /// The original columns is present + Present(ColumnStatistics), + } + + // Convert to Vec so we can avoid copying the statistics + let mut columns: Vec<_> = std::mem::take(&mut self.column_statistics) + .into_iter() + .map(Slot::Present) .collect(); + for idx in projection { + let next_idx = self.column_statistics.len(); + let slot = std::mem::replace( + columns.get_mut(*idx).expect("projection out of bounds"), + Slot::Taken(next_idx), + ); + match slot { + // The column was there, so just move it + Slot::Present(col) => self.column_statistics.push(col), + // The column was taken, so copy from the previous location + Slot::Taken(prev_idx) => self + .column_statistics + .push(self.column_statistics[prev_idx].clone()), + } + } + self } @@ -581,4 +603,50 @@ mod tests { let p2 = precision.clone(); assert_eq!(precision, p2); } + + #[test] + fn test_project_none() { + let projection = None; + let stats = make_stats(vec![10, 20, 30]).project(projection.as_ref()); + assert_eq!(stats, make_stats(vec![10, 20, 30])); + } + + #[test] + fn test_project_empty() { + let projection = Some(vec![]); + let stats = make_stats(vec![10, 20, 30]).project(projection.as_ref()); + assert_eq!(stats, make_stats(vec![])); + } + + #[test] + fn test_project_swap() { + let projection = Some(vec![2, 1]); + let stats = make_stats(vec![10, 20, 30]).project(projection.as_ref()); + assert_eq!(stats, make_stats(vec![30, 20])); + } + + #[test] + fn test_project_repeated() { + let projection = Some(vec![1, 2, 1, 1, 0, 2]); + let stats = make_stats(vec![10, 20, 30]).project(projection.as_ref()); + assert_eq!(stats, make_stats(vec![20, 30, 20, 20, 10, 30])); + } + + // Make a Statistics structure with the specified null counts for each column + fn make_stats(counts: impl IntoIterator) -> Statistics { + Statistics { + num_rows: Precision::Exact(42), + total_byte_size: Precision::Exact(500), + column_statistics: counts.into_iter().map(col_stats_i64).collect(), + } + } + + fn col_stats_i64(null_count: usize) -> ColumnStatistics { + ColumnStatistics { + null_count: Precision::Exact(null_count), + max_value: Precision::Exact(ScalarValue::Int64(Some(42))), + min_value: Precision::Exact(ScalarValue::Int64(Some(64))), + distinct_count: Precision::Exact(100), + } + } } From 48b8c13396e241bb327e7fc9193528d46d1c7ca6 Mon Sep 17 00:00:00 2001 From: Emil Ejbyfeldt Date: Tue, 5 Nov 2024 17:01:58 +0100 Subject: [PATCH 25/28] fix: Fix stddev indeterministically producing NAN (#13248) In the VarianceGroupAccumulator we were missing a `count == 0` check that is present in the normal Accumulator. This mostly does not matter except for the case where the first state to be merged has `count == 0` then the `merge` function will incorrectly calculate a new m2 of NAN which will propagate to the final result. This fixes the bug bu adding the missing `count == 0` check. --- .../functions-aggregate/src/variance.rs | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index 810247a2884a..009088610901 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -316,6 +316,7 @@ fn merge( mean2: f64, m22: f64, ) -> (u64, f64, f64) { + debug_assert!(count != 0 || count2 != 0, "Cannot merge two empty states"); let new_count = count + count2; let new_mean = mean * count as f64 / new_count as f64 + mean2 * count2 as f64 / new_count as f64; @@ -573,6 +574,9 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { partial_m2s, opt_filter, |group_index, partial_count, partial_mean, partial_m2| { + if partial_count == 0 { + return; + } let (new_count, new_mean, new_m2) = merge( self.counts[group_index], self.means[group_index], @@ -612,3 +616,32 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { + self.counts.capacity() * size_of::() } } + +#[cfg(test)] +mod tests { + use datafusion_expr::EmitTo; + + use super::*; + + #[test] + fn test_groups_accumulator_merge_empty_states() -> Result<()> { + let state_1 = vec![ + Arc::new(UInt64Array::from(vec![0])) as ArrayRef, + Arc::new(Float64Array::from(vec![0.0])), + Arc::new(Float64Array::from(vec![0.0])), + ]; + let state_2 = vec![ + Arc::new(UInt64Array::from(vec![2])) as ArrayRef, + Arc::new(Float64Array::from(vec![1.0])), + Arc::new(Float64Array::from(vec![1.0])), + ]; + let mut acc = VarianceGroupsAccumulator::new(StatsType::Sample); + acc.merge_batch(&state_1, &[0], None, 1)?; + acc.merge_batch(&state_2, &[0], None, 1)?; + let result = acc.evaluate(EmitTo::All)?; + let result = result.as_any().downcast_ref::().unwrap(); + assert_eq!(result.len(), 1); + assert_eq!(result.value(0), 1.0); + Ok(()) + } +} From e43466a79cee7272e3279dbbda24c39c21631e3f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 5 Nov 2024 17:03:21 +0100 Subject: [PATCH 26/28] Migrate invoke_no_args to invoke_batch (#13246) Migrate scalar UDFs from deprecated invoke_no_args to invoke_batch. --- .../user_defined/user_defined_scalar_functions.rs | 10 +++++----- datafusion/expr/src/udf.rs | 14 ++++++-------- datafusion/functions-nested/src/make_array.rs | 10 +++++----- datafusion/functions/src/core/version.rs | 15 +++++++++------ datafusion/functions/src/math/pi.rs | 11 +++++++++-- datafusion/functions/src/math/random.rs | 15 +++++++++------ datafusion/functions/src/string/uuid.rs | 15 +++++++++------ 7 files changed, 52 insertions(+), 38 deletions(-) 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 8453a360cd25..cf403e5d640f 100644 --- a/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs +++ b/datafusion/core/tests/user_defined/user_defined_scalar_functions.rs @@ -209,11 +209,11 @@ impl ScalarUDFImpl for Simple0ArgsScalarUDF { Ok(self.return_type.clone()) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { - not_impl_err!("{} function does not accept arguments", self.name()) - } - - fn invoke_no_args(&self, _number_rows: usize) -> Result { + fn invoke_batch( + &self, + _args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some(100)))) } } diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 003a3ed36a60..9ee9f9253847 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -749,14 +749,12 @@ impl ScalarUDFImpl for AliasedScalarUDFImpl { self.inner.return_type_from_exprs(args, schema, arg_types) } - fn invoke(&self, args: &[ColumnarValue]) -> Result { - #[allow(deprecated)] - self.inner.invoke(args) - } - - fn invoke_no_args(&self, number_rows: usize) -> Result { - #[allow(deprecated)] - self.inner.invoke_no_args(number_rows) + fn invoke_batch( + &self, + args: &[ColumnarValue], + number_rows: usize, + ) -> Result { + self.inner.invoke_batch(args, number_rows) } fn simplify( diff --git a/datafusion/functions-nested/src/make_array.rs b/datafusion/functions-nested/src/make_array.rs index c2c6f24948b8..7aa3445f6858 100644 --- a/datafusion/functions-nested/src/make_array.rs +++ b/datafusion/functions-nested/src/make_array.rs @@ -98,14 +98,14 @@ impl ScalarUDFImpl for MakeArray { } } - fn invoke(&self, args: &[ColumnarValue]) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { make_scalar_function(make_array_inner)(args) } - fn invoke_no_args(&self, _number_rows: usize) -> Result { - make_scalar_function(make_array_inner)(&[]) - } - fn aliases(&self) -> &[String] { &self.aliases } diff --git a/datafusion/functions/src/core/version.rs b/datafusion/functions/src/core/version.rs index e7ac749ddddc..36cf07e9e5da 100644 --- a/datafusion/functions/src/core/version.rs +++ b/datafusion/functions/src/core/version.rs @@ -18,7 +18,7 @@ //! [`VersionFunc`]: Implementation of the `version` function. use arrow::datatypes::DataType; -use datafusion_common::{not_impl_err, plan_err, Result, ScalarValue}; +use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_OTHER; use datafusion_expr::{ ColumnarValue, Documentation, ScalarUDFImpl, Signature, Volatility, @@ -66,11 +66,14 @@ impl ScalarUDFImpl for VersionFunc { } } - fn invoke(&self, _: &[ColumnarValue]) -> Result { - not_impl_err!("version does not take any arguments") - } - - fn invoke_no_args(&self, _: usize) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { + if !args.is_empty() { + return internal_err!("{} function does not accept arguments", self.name()); + } // TODO it would be great to add rust version and arrow version, // but that requires a `build.rs` script and/or adding a version const to arrow-rs let version = format!( diff --git a/datafusion/functions/src/math/pi.rs b/datafusion/functions/src/math/pi.rs index ea0f33161772..502429d0ca5d 100644 --- a/datafusion/functions/src/math/pi.rs +++ b/datafusion/functions/src/math/pi.rs @@ -20,7 +20,7 @@ use std::sync::OnceLock; use arrow::datatypes::DataType; use arrow::datatypes::DataType::Float64; -use datafusion_common::{not_impl_err, Result, ScalarValue}; +use datafusion_common::{internal_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_MATH; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_expr::{ @@ -67,7 +67,14 @@ impl ScalarUDFImpl for PiFunc { not_impl_err!("{} function does not accept arguments", self.name()) } - fn invoke_no_args(&self, _number_rows: usize) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + _number_rows: usize, + ) -> Result { + if !args.is_empty() { + return internal_err!("{} function does not accept arguments", self.name()); + } Ok(ColumnarValue::Scalar(ScalarValue::Float64(Some( std::f64::consts::PI, )))) diff --git a/datafusion/functions/src/math/random.rs b/datafusion/functions/src/math/random.rs index cf564e5328a5..cd92798d67dd 100644 --- a/datafusion/functions/src/math/random.rs +++ b/datafusion/functions/src/math/random.rs @@ -23,7 +23,7 @@ use arrow::datatypes::DataType; use arrow::datatypes::DataType::Float64; use rand::{thread_rng, Rng}; -use datafusion_common::{not_impl_err, Result}; +use datafusion_common::{internal_err, Result}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_MATH; use datafusion_expr::ColumnarValue; use datafusion_expr::{Documentation, ScalarUDFImpl, Signature, Volatility}; @@ -64,11 +64,14 @@ impl ScalarUDFImpl for RandomFunc { Ok(Float64) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { - not_impl_err!("{} function does not accept arguments", self.name()) - } - - fn invoke_no_args(&self, num_rows: usize) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + num_rows: usize, + ) -> Result { + if !args.is_empty() { + return internal_err!("{} function does not accept arguments", self.name()); + } let mut rng = thread_rng(); let mut values = vec![0.0; num_rows]; // Equivalent to set each element with rng.gen_range(0.0..1.0), but more efficient diff --git a/datafusion/functions/src/string/uuid.rs b/datafusion/functions/src/string/uuid.rs index 0fbdce16ccd1..2282670dac79 100644 --- a/datafusion/functions/src/string/uuid.rs +++ b/datafusion/functions/src/string/uuid.rs @@ -23,7 +23,7 @@ use arrow::datatypes::DataType; use arrow::datatypes::DataType::Utf8; use uuid::Uuid; -use datafusion_common::{not_impl_err, Result}; +use datafusion_common::{internal_err, Result}; use datafusion_expr::scalar_doc_sections::DOC_SECTION_STRING; use datafusion_expr::{ColumnarValue, Documentation, Volatility}; use datafusion_expr::{ScalarUDFImpl, Signature}; @@ -64,13 +64,16 @@ impl ScalarUDFImpl for UuidFunc { Ok(Utf8) } - fn invoke(&self, _args: &[ColumnarValue]) -> Result { - not_impl_err!("{} function does not accept arguments", self.name()) - } - /// Prints random (v4) uuid values per row /// uuid() = 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11' - fn invoke_no_args(&self, num_rows: usize) -> Result { + fn invoke_batch( + &self, + args: &[ColumnarValue], + num_rows: usize, + ) -> Result { + if !args.is_empty() { + return internal_err!("{} function does not accept arguments", self.name()); + } let values = std::iter::repeat_with(|| Uuid::new_v4().to_string()).take(num_rows); let array = GenericStringArray::::from_iter_values(values); Ok(ColumnarValue::Array(Arc::new(array))) From e8520ab7199c341c7767bdf7375f65d5ae9ab3fa Mon Sep 17 00:00:00 2001 From: Lordworms <48054792+Lordworms@users.noreply.github.com> Date: Tue, 5 Nov 2024 08:08:52 -0800 Subject: [PATCH 27/28] fix bugs explain with non-correlated query (#13210) * fix bugs explain with non-correlated query * Use explicit enum for physical errors * fix comments / fmt * strip_backtrace to passs ci --------- Co-authored-by: Andrew Lamb --- datafusion/common/src/display/mod.rs | 7 +++++- .../core/src/datasource/listing/table.rs | 12 ++++----- datafusion/core/src/physical_planner.rs | 8 ++++-- datafusion/physical-plan/src/explain.rs | 1 - datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 13 ++++++++++ datafusion/proto/src/generated/prost.rs | 4 ++- .../proto/src/logical_plan/from_proto.rs | 3 ++- datafusion/proto/src/logical_plan/to_proto.rs | 4 +++ .../sqllogictest/test_files/explain.slt | 25 +++++++++++++++++++ .../sqllogictest/test_files/group_by.slt | 1 + datafusion/sqllogictest/test_files/joins.slt | 4 ++- .../sqllogictest/test_files/prepare.slt | 2 ++ datafusion/sqllogictest/test_files/update.slt | 9 +++++-- 14 files changed, 79 insertions(+), 15 deletions(-) diff --git a/datafusion/common/src/display/mod.rs b/datafusion/common/src/display/mod.rs index c12e7419e4b6..bad51c45f8ee 100644 --- a/datafusion/common/src/display/mod.rs +++ b/datafusion/common/src/display/mod.rs @@ -62,6 +62,8 @@ pub enum PlanType { FinalPhysicalPlanWithStats, /// The final with schema, fully optimized physical plan which would be executed FinalPhysicalPlanWithSchema, + /// An error creating the physical plan + PhysicalPlanError, } impl Display for PlanType { @@ -91,6 +93,7 @@ impl Display for PlanType { PlanType::FinalPhysicalPlanWithSchema => { write!(f, "physical_plan_with_schema") } + PlanType::PhysicalPlanError => write!(f, "physical_plan_error"), } } } @@ -118,7 +121,9 @@ impl StringifiedPlan { /// `verbose_mode = true` will display all available plans pub fn should_display(&self, verbose_mode: bool) -> bool { match self.plan_type { - PlanType::FinalLogicalPlan | PlanType::FinalPhysicalPlan => true, + PlanType::FinalLogicalPlan + | PlanType::FinalPhysicalPlan + | PlanType::PhysicalPlanError => true, _ => verbose_mode, } } diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 15125fe5a090..b937a28e9332 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -880,18 +880,18 @@ impl TableProvider for ListingTable { None => {} // no ordering required }; - let filters = conjunction(filters.to_vec()) - .map(|expr| -> Result<_> { - // NOTE: Use the table schema (NOT file schema) here because `expr` may contain references to partition columns. + let filters = match conjunction(filters.to_vec()) { + Some(expr) => { let table_df_schema = self.table_schema.as_ref().clone().to_dfschema()?; let filters = create_physical_expr( &expr, &table_df_schema, state.execution_props(), )?; - Ok(Some(filters)) - }) - .unwrap_or(Ok(None))?; + Some(filters) + } + None => None, + }; let Some(object_store_url) = self.table_paths.first().map(ListingTableUrl::object_store) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 2a96a2ad111f..7d475ad2e2a1 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1797,8 +1797,12 @@ impl DefaultPhysicalPlanner { Err(e) => return Err(e), } } - Err(e) => stringified_plans - .push(StringifiedPlan::new(InitialPhysicalPlan, e.to_string())), + Err(err) => { + stringified_plans.push(StringifiedPlan::new( + PhysicalPlanError, + err.strip_backtrace(), + )); + } } } diff --git a/datafusion/physical-plan/src/explain.rs b/datafusion/physical-plan/src/explain.rs index 96f55a1446b0..cc42e0587151 100644 --- a/datafusion/physical-plan/src/explain.rs +++ b/datafusion/physical-plan/src/explain.rs @@ -132,7 +132,6 @@ impl ExecutionPlan for ExplainExec { if 0 != partition { return internal_err!("ExplainExec invalid partition {partition}"); } - let mut type_builder = StringBuilder::with_capacity(self.stringified_plans.len(), 1024); let mut plan_builder = diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index b68c47c57eb9..d6fa129edc3f 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -655,6 +655,7 @@ message PlanType { datafusion_common.EmptyMessage FinalPhysicalPlan = 6; datafusion_common.EmptyMessage FinalPhysicalPlanWithStats = 10; datafusion_common.EmptyMessage FinalPhysicalPlanWithSchema = 12; + datafusion_common.EmptyMessage PhysicalPlanError = 13; } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index e54edb718808..16f14d9ddf61 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -16683,6 +16683,9 @@ impl serde::Serialize for PlanType { plan_type::PlanTypeEnum::FinalPhysicalPlanWithSchema(v) => { struct_ser.serialize_field("FinalPhysicalPlanWithSchema", v)?; } + plan_type::PlanTypeEnum::PhysicalPlanError(v) => { + struct_ser.serialize_field("PhysicalPlanError", v)?; + } } } struct_ser.end() @@ -16707,6 +16710,7 @@ impl<'de> serde::Deserialize<'de> for PlanType { "FinalPhysicalPlan", "FinalPhysicalPlanWithStats", "FinalPhysicalPlanWithSchema", + "PhysicalPlanError", ]; #[allow(clippy::enum_variant_names)] @@ -16723,6 +16727,7 @@ impl<'de> serde::Deserialize<'de> for PlanType { FinalPhysicalPlan, FinalPhysicalPlanWithStats, FinalPhysicalPlanWithSchema, + PhysicalPlanError, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -16756,6 +16761,7 @@ impl<'de> serde::Deserialize<'de> for PlanType { "FinalPhysicalPlan" => Ok(GeneratedField::FinalPhysicalPlan), "FinalPhysicalPlanWithStats" => Ok(GeneratedField::FinalPhysicalPlanWithStats), "FinalPhysicalPlanWithSchema" => Ok(GeneratedField::FinalPhysicalPlanWithSchema), + "PhysicalPlanError" => Ok(GeneratedField::PhysicalPlanError), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -16860,6 +16866,13 @@ impl<'de> serde::Deserialize<'de> for PlanType { return Err(serde::de::Error::duplicate_field("FinalPhysicalPlanWithSchema")); } plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::FinalPhysicalPlanWithSchema) +; + } + GeneratedField::PhysicalPlanError => { + if plan_type_enum__.is_some() { + return Err(serde::de::Error::duplicate_field("PhysicalPlanError")); + } + plan_type_enum__ = map_.next_value::<::std::option::Option<_>>()?.map(plan_type::PlanTypeEnum::PhysicalPlanError) ; } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index dfc30e809108..59a90eb31ade 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -888,7 +888,7 @@ pub struct OptimizedPhysicalPlanType { pub struct PlanType { #[prost( oneof = "plan_type::PlanTypeEnum", - tags = "1, 7, 8, 2, 3, 4, 9, 11, 5, 6, 10, 12" + tags = "1, 7, 8, 2, 3, 4, 9, 11, 5, 6, 10, 12, 13" )] pub plan_type_enum: ::core::option::Option, } @@ -920,6 +920,8 @@ pub mod plan_type { FinalPhysicalPlanWithStats(super::super::datafusion_common::EmptyMessage), #[prost(message, tag = "12")] FinalPhysicalPlanWithSchema(super::super::datafusion_common::EmptyMessage), + #[prost(message, tag = "13")] + PhysicalPlanError(super::super::datafusion_common::EmptyMessage), } } #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index f25fb0bf2561..33b718558827 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -44,7 +44,7 @@ use crate::protobuf::{ AnalyzedLogicalPlan, FinalAnalyzedLogicalPlan, FinalLogicalPlan, FinalPhysicalPlan, FinalPhysicalPlanWithStats, InitialLogicalPlan, InitialPhysicalPlan, InitialPhysicalPlanWithStats, OptimizedLogicalPlan, - OptimizedPhysicalPlan, + OptimizedPhysicalPlan, PhysicalPlanError, }, AnalyzedLogicalPlanType, CubeNode, GroupingSetNode, OptimizedLogicalPlanType, OptimizedPhysicalPlanType, PlaceholderNode, RollupNode, @@ -141,6 +141,7 @@ impl From<&protobuf::StringifiedPlan> for StringifiedPlan { FinalPhysicalPlan(_) => PlanType::FinalPhysicalPlan, FinalPhysicalPlanWithStats(_) => PlanType::FinalPhysicalPlanWithStats, FinalPhysicalPlanWithSchema(_) => PlanType::FinalPhysicalPlanWithSchema, + PhysicalPlanError(_) => PlanType::PhysicalPlanError, }, plan: Arc::new(stringified_plan.plan.clone()), } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 8af7b19d9091..a5497b2c15e1 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -38,6 +38,7 @@ use crate::protobuf::{ FinalPhysicalPlan, FinalPhysicalPlanWithSchema, FinalPhysicalPlanWithStats, InitialLogicalPlan, InitialPhysicalPlan, InitialPhysicalPlanWithSchema, InitialPhysicalPlanWithStats, OptimizedLogicalPlan, OptimizedPhysicalPlan, + PhysicalPlanError, }, AnalyzedLogicalPlanType, CubeNode, EmptyMessage, GroupingSetNode, LogicalExprList, OptimizedLogicalPlanType, OptimizedPhysicalPlanType, PlaceholderNode, RollupNode, @@ -115,6 +116,9 @@ impl From<&StringifiedPlan> for protobuf::StringifiedPlan { PlanType::FinalPhysicalPlanWithSchema => Some(protobuf::PlanType { plan_type_enum: Some(FinalPhysicalPlanWithSchema(EmptyMessage {})), }), + PlanType::PhysicalPlanError => Some(protobuf::PlanType { + plan_type_enum: Some(PhysicalPlanError(EmptyMessage {})), + }), }, plan: stringified_plan.plan.to_string(), } diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 54658f36ca14..f3fee4f1fca6 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -411,3 +411,28 @@ logical_plan physical_plan 01)ProjectionExec: expr=[{c0:1,c1:2.3,c2:abc} as struct(Int64(1),Float64(2.3),Utf8("abc"))] 02)--PlaceholderRowExec + + +statement ok +create table t1(a int); + +statement ok +create table t2(b int); + +query TT +explain select a from t1 where exists (select count(*) from t2); +---- +logical_plan +01)Filter: EXISTS () +02)--Subquery: +03)----Projection: count(*) +04)------Aggregate: groupBy=[[]], aggr=[[count(Int64(1)) AS count(*)]] +05)--------TableScan: t2 +06)--TableScan: t1 projection=[a] +physical_plan_error This feature is not implemented: Physical plan does not support logical expression Exists(Exists { subquery: , negated: false }) + +statement ok +drop table t1; + +statement ok +drop table t2; diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index daf270190870..4b90ddf2ea5f 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4082,6 +4082,7 @@ logical_plan 02)--Projection: multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b, sum(multiple_ordered_table_with_pk.d) AS sum1 03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b]], aggr=[[sum(CAST(multiple_ordered_table_with_pk.d AS Int64))]] 04)------TableScan: multiple_ordered_table_with_pk projection=[a, b, c, d] +physical_plan_error This feature is not implemented: Physical plan does not support DistributeBy partitioning # union with aggregate query TT diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 93bb1f1f548e..d45dbc7ee1ae 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -4053,7 +4053,7 @@ query TT explain select t1_id, t1_name, i from join_t1 t1 cross join lateral (select * from unnest(generate_series(1, t1_int))) as series(i); ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--SubqueryAlias: t1 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)--SubqueryAlias: series @@ -4062,6 +4062,7 @@ logical_plan 07)--------Unnest: lists[unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int)))|depth=1] structs[] 08)----------Projection: generate_series(Int64(1), CAST(outer_ref(t1.t1_int) AS Int64)) AS unnest_placeholder(generate_series(Int64(1),outer_ref(t1.t1_int))) 09)------------EmptyRelation +physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(UInt32, Column { relation: Some(Bare { table: "t1" }), name: "t1_int" }) # Test CROSS JOIN LATERAL syntax (execution) @@ -4084,6 +4085,7 @@ logical_plan 07)--------Unnest: lists[unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int)))|depth=1] structs[] 08)----------Projection: generate_series(Int64(1), CAST(outer_ref(t2.t1_int) AS Int64)) AS unnest_placeholder(generate_series(Int64(1),outer_ref(t2.t1_int))) 09)------------EmptyRelation +physical_plan_error This feature is not implemented: Physical plan does not support logical expression OuterReferenceColumn(UInt32, Column { relation: Some(Bare { table: "t2" }), name: "t1_int" }) # Test INNER JOIN LATERAL syntax (execution) diff --git a/datafusion/sqllogictest/test_files/prepare.slt b/datafusion/sqllogictest/test_files/prepare.slt index e306ec7767c7..91b925efa26c 100644 --- a/datafusion/sqllogictest/test_files/prepare.slt +++ b/datafusion/sqllogictest/test_files/prepare.slt @@ -86,11 +86,13 @@ query TT EXPLAIN EXECUTE my_plan; ---- logical_plan Execute: my_plan params=[] +physical_plan_error This feature is not implemented: Unsupported logical plan: Execute query TT EXPLAIN EXECUTE my_plan(10*2 + 1, 'Foo'); ---- logical_plan Execute: my_plan params=[Int64(21), Utf8("Foo")] +physical_plan_error This feature is not implemented: Unsupported logical plan: Execute query error DataFusion error: Schema error: No field named a\. EXPLAIN EXECUTE my_plan(a); diff --git a/datafusion/sqllogictest/test_files/update.slt b/datafusion/sqllogictest/test_files/update.slt index aaba6998ee63..0f9582b04c58 100644 --- a/datafusion/sqllogictest/test_files/update.slt +++ b/datafusion/sqllogictest/test_files/update.slt @@ -33,6 +33,7 @@ logical_plan 01)Dml: op=[Update] table=[t1] 02)--Projection: CAST(Int64(1) AS Int32) AS a, CAST(Int64(2) AS Utf8) AS b, Float64(3) AS c, CAST(NULL AS Int32) AS d 03)----TableScan: t1 +physical_plan_error This feature is not implemented: Unsupported logical plan: Dml(Update) query TT explain update t1 set a=c+1, b=a, c=c+1.0, d=b; @@ -41,6 +42,7 @@ logical_plan 01)Dml: op=[Update] table=[t1] 02)--Projection: CAST(t1.c + CAST(Int64(1) AS Float64) AS Int32) AS a, CAST(t1.a AS Utf8) AS b, t1.c + Float64(1) AS c, CAST(t1.b AS Int32) AS d 03)----TableScan: t1 +physical_plan_error This feature is not implemented: Unsupported logical plan: Dml(Update) statement ok create table t2(a int, b varchar, c double, d int); @@ -58,6 +60,7 @@ logical_plan 06)----------Filter: outer_ref(t1.a) = t2.a 07)------------TableScan: t2 08)----TableScan: t1 +physical_plan_error This feature is not implemented: Physical plan does not support logical expression ScalarSubquery() # set from other table query TT @@ -67,9 +70,10 @@ logical_plan 01)Dml: op=[Update] table=[t1] 02)--Projection: t1.a AS a, t2.b AS b, CAST(t2.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d 03)----Filter: t1.a = t2.a AND t1.b > Utf8("foo") AND t2.c > Float64(1) -04)------Cross Join: +04)------Cross Join: 05)--------TableScan: t1 06)--------TableScan: t2 +physical_plan_error This feature is not implemented: Unsupported logical plan: Dml(Update) statement ok create table t3(a int, b varchar, c double, d int); @@ -86,7 +90,8 @@ logical_plan 01)Dml: op=[Update] table=[t1] 02)--Projection: t.a AS a, t2.b AS b, CAST(t.a AS Float64) AS c, CAST(Int64(1) AS Int32) AS d 03)----Filter: t.a = t2.a AND t.b > Utf8("foo") AND t2.c > Float64(1) -04)------Cross Join: +04)------Cross Join: 05)--------SubqueryAlias: t 06)----------TableScan: t1 07)--------TableScan: t2 +physical_plan_error This feature is not implemented: Unsupported logical plan: Dml(Update) From 7c6f891b4b5a007e29fb3890ed5315ef916ae1d3 Mon Sep 17 00:00:00 2001 From: Jax Liu Date: Wed, 6 Nov 2024 00:45:21 +0800 Subject: [PATCH 28/28] Introduce `full_qualified_col` option for the unparser dialect (#13241) * introduce `full_qualified_col` for unparser dialect * fix test and clippy --- datafusion/sql/src/unparser/dialect.rs | 23 ++++++++++++++ datafusion/sql/src/unparser/expr.rs | 10 ++++-- datafusion/sql/tests/cases/plan_to_sql.rs | 38 +++++++++++++++++++---- 3 files changed, 62 insertions(+), 9 deletions(-) diff --git a/datafusion/sql/src/unparser/dialect.rs b/datafusion/sql/src/unparser/dialect.rs index 88159ab6df15..87ed1b8f4140 100644 --- a/datafusion/sql/src/unparser/dialect.rs +++ b/datafusion/sql/src/unparser/dialect.rs @@ -137,6 +137,14 @@ pub trait Dialect: Send + Sync { ) -> Result> { Ok(None) } + + /// Allow to unparse a qualified column with a full qualified name + /// (e.g. catalog_name.schema_name.table_name.column_name) + /// Otherwise, the column will be unparsed with only the table name and colum name + /// (e.g. table_name.column_name) + fn full_qualified_col(&self) -> bool { + false + } } /// `IntervalStyle` to use for unparsing @@ -373,6 +381,7 @@ pub struct CustomDialect { date32_cast_dtype: ast::DataType, supports_column_alias_in_table_alias: bool, requires_derived_table_alias: bool, + full_qualified_col: bool, } impl Default for CustomDialect { @@ -396,6 +405,7 @@ impl Default for CustomDialect { date32_cast_dtype: ast::DataType::Date, supports_column_alias_in_table_alias: true, requires_derived_table_alias: false, + full_qualified_col: false, } } } @@ -488,6 +498,10 @@ impl Dialect for CustomDialect { fn requires_derived_table_alias(&self) -> bool { self.requires_derived_table_alias } + + fn full_qualified_col(&self) -> bool { + self.full_qualified_col + } } /// `CustomDialectBuilder` to build `CustomDialect` using builder pattern @@ -520,6 +534,7 @@ pub struct CustomDialectBuilder { date32_cast_dtype: ast::DataType, supports_column_alias_in_table_alias: bool, requires_derived_table_alias: bool, + full_qualified_col: bool, } impl Default for CustomDialectBuilder { @@ -549,6 +564,7 @@ impl CustomDialectBuilder { date32_cast_dtype: ast::DataType::Date, supports_column_alias_in_table_alias: true, requires_derived_table_alias: false, + full_qualified_col: false, } } @@ -570,6 +586,7 @@ impl CustomDialectBuilder { supports_column_alias_in_table_alias: self .supports_column_alias_in_table_alias, requires_derived_table_alias: self.requires_derived_table_alias, + full_qualified_col: self.full_qualified_col, } } @@ -677,4 +694,10 @@ impl CustomDialectBuilder { self.requires_derived_table_alias = requires_derived_table_alias; self } + + /// Customize the dialect to allow full qualified column names + pub fn with_full_qualified_col(mut self, full_qualified_col: bool) -> Self { + self.full_qualified_col = full_qualified_col; + self + } } diff --git a/datafusion/sql/src/unparser/expr.rs b/datafusion/sql/src/unparser/expr.rs index b41b0a54b86f..0678e7d0306c 100644 --- a/datafusion/sql/src/unparser/expr.rs +++ b/datafusion/sql/src/unparser/expr.rs @@ -527,7 +527,11 @@ impl Unparser<'_> { fn col_to_sql(&self, col: &Column) -> Result { if let Some(table_ref) = &col.relation { - let mut id = table_ref.to_vec(); + let mut id = if self.dialect.full_qualified_col() { + table_ref.to_vec() + } else { + vec![table_ref.table().to_string()] + }; id.push(col.name.to_string()); return Ok(ast::Expr::CompoundIdentifier( id.iter() @@ -1545,7 +1549,7 @@ mod tests { name: "c".to_string(), }) .gt(lit(4)), - r#"(a.b.c > 4)"#, + r#"(b.c > 4)"#, ), ( case(col("a")) @@ -1882,7 +1886,7 @@ mod tests { name: "array_col".to_string(), })), }), - r#"UNNEST("schema"."table".array_col)"#, + r#"UNNEST("table".array_col)"#, ), ]; diff --git a/datafusion/sql/tests/cases/plan_to_sql.rs b/datafusion/sql/tests/cases/plan_to_sql.rs index ea0ccb8e4b43..669f9f06f035 100644 --- a/datafusion/sql/tests/cases/plan_to_sql.rs +++ b/datafusion/sql/tests/cases/plan_to_sql.rs @@ -28,8 +28,8 @@ use datafusion_functions_nested::make_array::make_array_udf; use datafusion_functions_window::rank::rank_udwf; use datafusion_sql::planner::{ContextProvider, PlannerContext, SqlToRel}; use datafusion_sql::unparser::dialect::{ - DefaultDialect as UnparserDefaultDialect, Dialect as UnparserDialect, - MySqlDialect as UnparserMySqlDialect, SqliteDialect, + CustomDialectBuilder, DefaultDialect as UnparserDefaultDialect, DefaultDialect, + Dialect as UnparserDialect, MySqlDialect as UnparserMySqlDialect, SqliteDialect, }; use datafusion_sql::unparser::{expr_to_sql, plan_to_sql, Unparser}; @@ -565,7 +565,7 @@ Projection: unnest_placeholder(unnest_table.struct_col).field1, unnest_placehold #[test] fn test_table_references_in_plan_to_sql() { - fn test(table_name: &str, expected_sql: &str) { + fn test(table_name: &str, expected_sql: &str, dialect: &impl UnparserDialect) { let schema = Schema::new(vec![ Field::new("id", DataType::Utf8, false), Field::new("value", DataType::Utf8, false), @@ -576,22 +576,48 @@ fn test_table_references_in_plan_to_sql() { .unwrap() .build() .unwrap(); - let sql = plan_to_sql(&plan).unwrap(); + + let unparser = Unparser::new(dialect); + let sql = unparser.plan_to_sql(&plan).unwrap(); assert_eq!(sql.to_string(), expected_sql) } test( "catalog.schema.table", - r#"SELECT "catalog"."schema"."table".id, "catalog"."schema"."table"."value" FROM "catalog"."schema"."table""#, + r#"SELECT "table".id, "table"."value" FROM "catalog"."schema"."table""#, + &DefaultDialect {}, ); test( "schema.table", - r#"SELECT "schema"."table".id, "schema"."table"."value" FROM "schema"."table""#, + r#"SELECT "table".id, "table"."value" FROM "schema"."table""#, + &DefaultDialect {}, ); test( "table", r#"SELECT "table".id, "table"."value" FROM "table""#, + &DefaultDialect {}, + ); + + let custom_dialect = CustomDialectBuilder::default() + .with_full_qualified_col(true) + .with_identifier_quote_style('"') + .build(); + + test( + "catalog.schema.table", + r#"SELECT "catalog"."schema"."table"."id", "catalog"."schema"."table"."value" FROM "catalog"."schema"."table""#, + &custom_dialect, + ); + test( + "schema.table", + r#"SELECT "schema"."table"."id", "schema"."table"."value" FROM "schema"."table""#, + &custom_dialect, + ); + test( + "table", + r#"SELECT "table"."id", "table"."value" FROM "table""#, + &custom_dialect, ); }