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 01/11] 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 02/11] 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 03/11] 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 04/11] 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 05/11] 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 06/11] 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 07/11] 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 08/11] 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 09/11] 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 10/11] 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 11/11] 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, ); }