diff --git a/datafusion/catalog-listing/src/helpers.rs b/datafusion/catalog-listing/src/helpers.rs index 9ac8423042d3..a6837f252820 100644 --- a/datafusion/catalog-listing/src/helpers.rs +++ b/datafusion/catalog-listing/src/helpers.rs @@ -544,6 +544,7 @@ mod tests { // use futures::StreamExt; use super::*; + use datafusion_expr::registry::MemoryExtensionTypeRegistry; use datafusion_expr::{ case, col, lit, AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF, }; @@ -1060,6 +1061,10 @@ mod tests { unimplemented!() } + fn extension_types(&self) -> &MemoryExtensionTypeRegistry { + unimplemented!() + } + fn runtime_env(&self) -> &Arc { unimplemented!() } diff --git a/datafusion/catalog/src/session.rs b/datafusion/catalog/src/session.rs index 9dd870e43568..fb6842f3bbde 100644 --- a/datafusion/catalog/src/session.rs +++ b/datafusion/catalog/src/session.rs @@ -22,6 +22,7 @@ use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; +use datafusion_expr::registry::MemoryExtensionTypeRegistry; use datafusion_expr::{AggregateUDF, Expr, LogicalPlan, ScalarUDF, WindowUDF}; use datafusion_physical_plan::{ExecutionPlan, PhysicalExpr}; use parking_lot::{Mutex, RwLock}; @@ -113,6 +114,9 @@ pub trait Session: Send + Sync { /// Return reference to window functions fn window_functions(&self) -> &HashMap>; + /// Return reference to extension types + fn extension_types(&self) -> &MemoryExtensionTypeRegistry; + /// Return the runtime env fn runtime_env(&self) -> &Arc; @@ -145,6 +149,7 @@ impl From<&dyn Session> for TaskContext { state.scalar_functions().clone(), state.aggregate_functions().clone(), state.window_functions().clone(), + state.extension_types().clone(), state.runtime_env().clone(), ) } diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index d5b7c22a546c..acb9c14c0b24 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -48,6 +48,7 @@ pub mod instant; pub mod parsers; pub mod rounding; pub mod scalar; +pub mod sort; pub mod spans; pub mod stats; pub mod test_util; diff --git a/datafusion/common/src/sort.rs b/datafusion/common/src/sort.rs new file mode 100644 index 000000000000..d9c7ce6a5e9c --- /dev/null +++ b/datafusion/common/src/sort.rs @@ -0,0 +1,277 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::error::{_exec_err, _internal_err}; +use crate::types::SortOrdering; +use crate::Result; +use arrow::array::{ArrayRef, DynComparator, UInt32Array}; +use arrow::compute::{partial_sort, SortColumn, SortOptions}; +use arrow::datatypes::DataType; +use arrow::row::{RowConverter, SortField}; +use std::cmp::Ordering; + +/// An advanced version of arrow's [SortOptions] that allows for the following features: +/// - Custom sort ordering +#[derive(Clone, Debug, Default, Hash, PartialEq, Eq)] +pub struct AdvSortOptions { + /// Specifies the ordering that is used for sorting. This enables implementing user-defined + /// sorting. + pub ordering: SortOrdering, + /// Whether to sort in descending order + pub descending: bool, + /// Whether to sort nulls first + pub nulls_first: bool, +} + +impl AdvSortOptions { + /// Creates a new [AdvSortOptions]. + pub fn new(ordering: SortOrdering, descending: bool, nulls_first: bool) -> Self { + Self { + ordering, + descending, + nulls_first, + } + } + + /// Creates a new [AdvSortOptions] with a default ordering from the arrow [SortOption]. + pub fn with_default_ordering(options: SortOptions) -> Self { + Self::new( + SortOrdering::Default, + options.descending, + options.nulls_first, + ) + } + + /// Tries to create an [SortOptions] with the same `descending` and `nulls_first`. + /// + /// # Errors + /// + /// This method will return an error if a custom [SortOrdering] is used. + pub fn to_arrow(&self) -> Result { + match self.ordering { + SortOrdering::Default => Ok(SortOptions { + descending: self.descending, + nulls_first: self.nulls_first, + }), + SortOrdering::Custom(_) => { + _internal_err!("Cannot create arrow SortOptions with custom ordering") + } + } + } + + /// Returns a [AdvSortOptions] with a flipped descending. + /// + /// This does not change the order of nulls. + pub fn with_reversed_order(mut self) -> Self { + self.descending = !self.descending; + self + } + + /// Returns a [AdvSortOptions] with the given `value` for `descending`. + pub fn with_descending(mut self, value: bool) -> Self { + self.descending = value; + self + } + + /// Returns a [AdvSortOptions] with the given `value` for `nulls_first`. + pub fn with_nulls_first(mut self, value: bool) -> Self { + self.nulls_first = value; + self + } +} + +/// An advanced version of arrow's [SortColumn] that uses an [AdvSortOptions]. +#[derive(Clone, Debug)] +pub struct AdvSortColumn { + pub values: ArrayRef, + pub options: Option, +} + +impl AdvSortColumn { + /// Creates a [DynComparator] from this sort column. The comparison implementation is decided + /// by [SortOrdering::dyn_comparator]. + pub fn dyn_compartor(&self) -> Result { + let ordering = self + .options + .as_ref() + .map(|opt| opt.ordering.clone()) + .unwrap_or_default(); + let options = self + .options + .as_ref() + .and_then(|opt| opt.to_arrow().ok()) + .unwrap_or_default(); + ordering.dyn_comparator(self.values.clone(), options) + } + + /// Tries to convert this sort column into an arrow-native [SortColumn]. + /// + /// # Errors + /// + /// This method returns an error if a custom ordering is specified. This is because this + /// ordering cannot be encoded in an arrow native version. + pub fn to_arrow(&self) -> Result { + let has_custom_sort = self + .options + .as_ref() + .map(|opt| opt.ordering != SortOrdering::Default) + .unwrap_or(false); + match has_custom_sort { + true => _internal_err!("Cannot create arrow SortColumn with custom sort"), + false => Ok(SortColumn { + values: self.values.clone(), + options: self.options.as_ref().map(|o| o.to_arrow().unwrap()), + }), + } + } +} + +/// A lexicographical comparator that wraps given array data (columns) and can lexicographically +/// compare data at given two indices. The lifetime is the same at the data wrapped. +/// +/// We require a separate version of this arrow data structure as we must construct it directly +/// from a `Vec`. +pub struct LexicographicalComparator { + /// Comparators for the lexicographical ordering. + compare_items: Vec, +} + +impl LexicographicalComparator { + /// lexicographically compare values at the wrapped columns with given indices. + pub fn compare(&self, a_idx: usize, b_idx: usize) -> Ordering { + for comparator in &self.compare_items { + match comparator(a_idx, b_idx) { + Ordering::Equal => continue, + r => return r, + } + } + Ordering::Equal + } + + /// Create a new lex comparator that will wrap the given sort columns and give comparison + /// results with two indices. + pub fn new(compare_items: Vec) -> LexicographicalComparator { + LexicographicalComparator { compare_items } + } +} + +/// Sort elements lexicographically from a list of `ArrayRef` into an unsigned integer +/// (`UInt32Array`) of indices. +pub fn lexsort_to_indices( + columns: &[AdvSortColumn], + fetch: Option, +) -> Result { + if columns.is_empty() { + return _exec_err!("Sort requires at least one column"); + } + + let all_columns_default_ordering = columns + .iter() + .map(|c| c.to_arrow()) + .collect::>>(); + if let Ok(sort_columns) = all_columns_default_ordering { + if is_multi_column_with_lists(&sort_columns) { + // lex_sort_to_indices doesn't support List with more than one column + // https://github.com/apache/arrow-rs/issues/5454 + lexsort_to_indices_multi_columns(sort_columns, fetch)? + } else { + arrow::compute::lexsort_to_indices(&sort_columns, fetch)? + }; + } + + let row_count = columns[0].values.len(); + if columns.iter().any(|item| item.values.len() != row_count) { + return _exec_err!("lexical sort columns have different row counts"); + }; + + let mut value_indices = (0..row_count).collect::>(); + let mut len = value_indices.len(); + + if let Some(limit) = fetch { + len = limit.min(len); + } + + let compare_items = columns + .iter() + .map(|c| c.dyn_compartor()) + .collect::>>()?; + + let lexicographical_comparator = LexicographicalComparator::new(compare_items); + // uint32 can be sorted unstably + sort_unstable_by(&mut value_indices, len, |a, b| { + lexicographical_comparator.compare(*a, *b) + }); + + Ok(UInt32Array::from_iter_values( + value_indices.iter().take(len).map(|i| *i as u32), + )) +} + +#[inline] +fn is_multi_column_with_lists(sort_columns: &[SortColumn]) -> bool { + sort_columns.iter().any(|c| { + matches!( + c.values.data_type(), + DataType::List(_) | DataType::LargeList(_) | DataType::FixedSizeList(_, _) + ) + }) +} + +pub(crate) fn lexsort_to_indices_multi_columns( + sort_columns: Vec, + limit: Option, +) -> Result { + let (fields, columns) = sort_columns.into_iter().fold( + (vec![], vec![]), + |(mut fields, mut columns), sort_column| { + fields.push(SortField::new_with_options( + sort_column.values.data_type().clone(), + sort_column.options.unwrap_or_default(), + )); + columns.push(sort_column.values); + (fields, columns) + }, + ); + + // TODO reuse converter and rows, refer to TopK. + let converter = RowConverter::new(fields)?; + let rows = converter.convert_columns(&columns)?; + let mut sort: Vec<_> = rows.iter().enumerate().collect(); + sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); + + let mut len = rows.num_rows(); + if let Some(limit) = limit { + len = limit.min(len); + } + let indices = + UInt32Array::from_iter_values(sort.iter().take(len).map(|(i, _)| *i as u32)); + + Ok(indices) +} + +/// we can only do this if the T is primitive +#[inline] +fn sort_unstable_by(array: &mut [T], limit: usize, cmp: F) +where + F: FnMut(&T, &T) -> Ordering, +{ + if array.len() == limit { + array.sort_unstable_by(cmp); + } else { + partial_sort(array, limit, cmp); + } +} diff --git a/datafusion/common/src/types/field.rs b/datafusion/common/src/types/field.rs index 5a880ba10a41..0150d4d51ad7 100644 --- a/datafusion/common/src/types/field.rs +++ b/datafusion/common/src/types/field.rs @@ -29,6 +29,21 @@ pub struct LogicalField { pub nullable: bool, } +impl LogicalField { + /// Creates a new [LogicalField]. + pub fn new( + name: impl Into, + logical_type: LogicalTypeRef, + nullable: bool, + ) -> Self { + Self { + name: name.into(), + logical_type, + nullable, + } + } +} + impl PartialEq for LogicalField { fn eq(&self, other: &Self) -> bool { self.name == other.name diff --git a/datafusion/common/src/types/logical.rs b/datafusion/common/src/types/logical.rs index 884ce20fd9e2..cbd37c547bab 100644 --- a/datafusion/common/src/types/logical.rs +++ b/datafusion/common/src/types/logical.rs @@ -17,8 +17,12 @@ use super::NativeType; use crate::error::Result; +use arrow::array::{make_comparator, ArrayRef, DynComparator}; +use arrow::compute::SortOptions; use arrow::datatypes::DataType; use core::fmt; +use std::fmt::Debug; +use std::hash::Hasher; use std::{cmp::Ordering, hash::Hash, sync::Arc}; /// Signature that uniquely identifies a type among other types. @@ -78,6 +82,7 @@ pub type LogicalTypeRef = Arc; pub trait LogicalType: Sync + Send { /// Get the native backing type of this logical type. fn native(&self) -> &NativeType; + /// Get the unique type signature for this logical type. Logical types with identical /// signatures are considered equal. fn signature(&self) -> TypeSignature<'_>; @@ -87,13 +92,21 @@ pub trait LogicalType: Sync + Send { fn default_cast_for(&self, origin: &DataType) -> Result { self.native().default_cast_for(origin) } + + /// Returns a [LogicalTypePlanningInformation] for this logical type. + /// + /// The default implementation returns the planning information of the underlying [NativeType]. + fn planning_information(&self) -> LogicalTypePlanningInformation { + self.native().planning_information() + } } -impl fmt::Debug for dyn LogicalType { +impl Debug for dyn LogicalType { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_tuple("LogicalType") .field(&self.signature()) .field(&self.native()) + .field(&self.planning_information()) .finish() } } @@ -127,8 +140,83 @@ impl Ord for dyn LogicalType { } impl Hash for dyn LogicalType { - fn hash(&self, state: &mut H) { + fn hash(&self, state: &mut H) { self.signature().hash(state); self.native().hash(state); } } + +/// Encapsulates information on how planning should be done in the presence of a logical type. +#[derive(Clone, Debug)] +pub struct LogicalTypePlanningInformation { + /// Specifies an ordering on elements of this logical type. + pub ordering: SortOrdering, +} + +/// Specifies how a logical type should be sorted. +#[derive(Clone, Debug, Default)] +pub enum SortOrdering { + /// Use the default arrow comparison. + #[default] + Default, + /// Use a custom comparison. + /// + /// Using a custom sorting allows users to override the default order of elements or implement + /// ordering for values that do not have a natural order (e.g., unions). It is expected that + /// the custom ordering handles all native types for the [LogicalType]. + Custom(Arc), +} + +impl SortOrdering { + pub fn dyn_comparator( + &self, + array: ArrayRef, + options: SortOptions, + ) -> Result { + Ok(match self { + SortOrdering::Default => { + make_comparator(array.as_ref(), array.as_ref(), options)? + } + SortOrdering::Custom(c) => c.dyn_comparator(array, options)?, + }) + } +} + +impl PartialEq for SortOrdering { + fn eq(&self, other: &Self) -> bool { + match (self, other) { + (SortOrdering::Default, SortOrdering::Default) => true, + (SortOrdering::Custom(c1), SortOrdering::Custom(c2)) => { + c1.ordering_id() == c2.ordering_id() + } + _ => false, + } + } +} + +impl Eq for SortOrdering {} + +impl Hash for SortOrdering { + fn hash(&self, state: &mut H) { + match self { + SortOrdering::Default => state.write_u8(1), + SortOrdering::Custom(ordering) => ordering.ordering_id().hash(state), + } + } +} + +/// A [CustomOrdering] can implement non-standard comparisons between values. This ability can be +/// used to customize algorithms that must compare elements. The most prominent example is sorting. +pub trait CustomOrdering: Debug + Send + Sync { + /// Returns the ordering id. + /// + /// The ordering id is used to establish equality between instances of [CustomOrdering]. + fn ordering_id(&self) -> &str; + + /// Returns a [DynComparator] over the given `array` adhering to the given `options`. + fn dyn_comparator( + &self, + array: ArrayRef, + options: SortOptions, + ) -> Result; +} diff --git a/datafusion/common/src/types/native.rs b/datafusion/common/src/types/native.rs index 39c79b4b9974..0355f9868b1a 100644 --- a/datafusion/common/src/types/native.rs +++ b/datafusion/common/src/types/native.rs @@ -16,8 +16,8 @@ // under the License. use super::{ - LogicalField, LogicalFieldRef, LogicalFields, LogicalType, LogicalUnionFields, - TypeSignature, + LogicalField, LogicalFieldRef, LogicalFields, LogicalType, + LogicalTypePlanningInformation, LogicalUnionFields, SortOrdering, TypeSignature, }; use crate::error::{Result, _internal_err}; use arrow::compute::can_cast_types; @@ -359,6 +359,12 @@ impl LogicalType for NativeType { } }) } + + fn planning_information(&self) -> LogicalTypePlanningInformation { + LogicalTypePlanningInformation { + ordering: SortOrdering::Default, + } + } } // The following From, From, ... implementations are temporary diff --git a/datafusion/core/src/datasource/listing/table.rs b/datafusion/core/src/datasource/listing/table.rs index 4d7762784d78..24b9c45fd51a 100644 --- a/datafusion/core/src/datasource/listing/table.rs +++ b/datafusion/core/src/datasource/listing/table.rs @@ -55,6 +55,7 @@ use datafusion_physical_expr::{ use async_trait::async_trait; use datafusion_catalog::Session; +use datafusion_expr::registry::EmptyExtensionTypeRegistry; use datafusion_physical_expr_common::sort_expr::LexRequirement; use futures::{future, stream, StreamExt, TryStreamExt}; use itertools::Itertools; @@ -816,7 +817,11 @@ impl ListingTable { /// If file_sort_order is specified, creates the appropriate physical expressions fn try_create_output_ordering(&self) -> Result> { - create_ordering(&self.table_schema, &self.options.file_sort_order) + create_ordering( + &EmptyExtensionTypeRegistry::new(), + &self.table_schema, + &self.options.file_sort_order, + ) } } @@ -1194,7 +1199,6 @@ mod tests { test::{columns, object_store::register_test_store}, }; - use arrow::compute::SortOptions; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; use datafusion_common::{assert_contains, ScalarValue}; @@ -1204,6 +1208,8 @@ mod tests { use datafusion_physical_plan::ExecutionPlanProperties; use crate::test::object_store::{ensure_head_concurrency, make_test_store_and_state}; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use tempfile::TempDir; use url::Url; @@ -1314,7 +1320,8 @@ mod tests { Ok(vec![LexOrdering::new( vec![PhysicalSortExpr { expr: physical_col("string_col", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::default(), descending: false, nulls_first: false, }, diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index d96944fa7a69..a7dcd7b16430 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -242,11 +242,7 @@ impl TableProvider for MemTable { let file_sort_order = sort_order .iter() .map(|sort_exprs| { - create_physical_sort_exprs( - sort_exprs, - &df_schema, - state.execution_props(), - ) + create_physical_sort_exprs(state, sort_exprs, &df_schema) }) .collect::>>()?; source = source.try_with_sort_information(file_sort_order)?; diff --git a/datafusion/core/src/datasource/mod.rs b/datafusion/core/src/datasource/mod.rs index 18a1318dd40d..e4ae8049ed2e 100644 --- a/datafusion/core/src/datasource/mod.rs +++ b/datafusion/core/src/datasource/mod.rs @@ -47,9 +47,10 @@ pub use crate::logical_expr::TableType; pub use datafusion_execution::object_store; pub use statistics::get_statistics_with_limit; -use arrow::compute::SortOptions; use arrow::datatypes::Schema; +use datafusion_common::sort::AdvSortOptions; use datafusion_common::{plan_err, Result}; +use datafusion_expr::registry::ExtensionTypeRegistry; use datafusion_expr::{Expr, SortExpr}; use datafusion_physical_expr::{expressions, LexOrdering, PhysicalSortExpr}; @@ -93,6 +94,7 @@ use datafusion_physical_expr::{expressions, LexOrdering, PhysicalSortExpr}; /// let result = create_ordering(&schema, &sort_exprs).unwrap(); /// ``` pub fn create_ordering( + extension_types: &impl ExtensionTypeRegistry, schema: &Schema, sort_order: &[Vec], ) -> Result> { @@ -105,9 +107,16 @@ pub fn create_ordering( match &sort.expr { Expr::Column(col) => match expressions::col(&col.name, schema) { Ok(expr) => { + let ordering = schema + .field_with_name(&col.name)? + .extension_type_name() + .and_then(|ext| extension_types.get_extension_type(ext).ok()) + .map(|ext| ext.planning_information().ordering.clone()) + .unwrap_or_default(); sort_exprs.push(PhysicalSortExpr { expr, - options: SortOptions { + options: AdvSortOptions { + ordering, descending: !sort.asc, nulls_first: sort.nulls_first, }, diff --git a/datafusion/core/src/datasource/stream.rs b/datafusion/core/src/datasource/stream.rs index d5fe070be82c..80c9418f7424 100644 --- a/datafusion/core/src/datasource/stream.rs +++ b/datafusion/core/src/datasource/stream.rs @@ -323,17 +323,18 @@ impl TableProvider for StreamTable { async fn scan( &self, - _state: &dyn Session, + state: &dyn Session, projection: Option<&Vec>, _filters: &[Expr], limit: Option, ) -> Result> { + let task_ctx = TaskContext::from(state); let projected_schema = match projection { Some(p) => { let projected = self.0.source.schema().project(p)?; - create_ordering(&projected, &self.0.order)? + create_ordering(&task_ctx, &projected, &self.0.order)? } - None => create_ordering(self.0.source.schema(), &self.0.order)?, + None => create_ordering(&task_ctx, self.0.source.schema(), &self.0.order)?, }; Ok(Arc::new(StreamingTableExec::try_new( @@ -348,14 +349,15 @@ impl TableProvider for StreamTable { async fn insert_into( &self, - _state: &dyn Session, + state: &dyn Session, input: Arc, _insert_op: InsertOp, ) -> Result> { + let task_ctx = TaskContext::from(state); let ordering = match self.0.order.first() { Some(x) => { let schema = self.0.source.schema(); - let orders = create_ordering(schema, std::slice::from_ref(x))?; + let orders = create_ordering(&task_ctx, schema, std::slice::from_ref(x))?; let ordering = orders.into_iter().next().unwrap(); Some(ordering.into_iter().map(Into::into).collect()) } diff --git a/datafusion/core/src/execution/context/mod.rs b/datafusion/core/src/execution/context/mod.rs index ad0993ed43ca..17d76c255324 100644 --- a/datafusion/core/src/execution/context/mod.rs +++ b/datafusion/core/src/execution/context/mod.rs @@ -75,9 +75,11 @@ use chrono::{DateTime, Utc}; use datafusion_catalog::{ DynamicFileCatalog, SessionStore, TableFunction, TableFunctionImpl, UrlTableFactory, }; +use datafusion_common::types::LogicalTypeRef; pub use datafusion_execution::config::SessionConfig; pub use datafusion_execution::TaskContext; pub use datafusion_expr::execution_props::ExecutionProps; +use datafusion_expr::registry::ExtensionTypeRegistry; use datafusion_optimizer::{AnalyzerRule, OptimizerRule}; use object_store::ObjectStore; use parking_lot::RwLock; @@ -1643,6 +1645,26 @@ impl FunctionRegistry for SessionContext { } } +impl ExtensionTypeRegistry for SessionContext { + fn get_extension_type(&self, name: &str) -> Result { + self.state.read().get_extension_type(name) + } + + fn register_extension_type( + &mut self, + logical_type: LogicalTypeRef, + ) -> Result> { + self.state.write().register_extension_type(logical_type) + } + + fn deregister_extension_type( + &mut self, + name: &str, + ) -> Result> { + self.state.write().deregister_extension_type(name) + } +} + /// Create a new task context instance from SessionContext impl From<&SessionContext> for TaskContext { fn from(session: &SessionContext) -> Self { diff --git a/datafusion/core/src/execution/session_state.rs b/datafusion/core/src/execution/session_state.rs index 0e83156ab53f..41195c6af446 100644 --- a/datafusion/core/src/execution/session_state.rs +++ b/datafusion/core/src/execution/session_state.rs @@ -52,7 +52,10 @@ use datafusion_execution::TaskContext; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::expr_rewriter::FunctionRewrite; use datafusion_expr::planner::{ExprPlanner, TypePlanner}; -use datafusion_expr::registry::{FunctionRegistry, SerializerRegistry}; +use datafusion_expr::registry::{ + ExtensionTypeRegistry, FunctionRegistry, MemoryExtensionTypeRegistry, + SerializerRegistry, +}; use datafusion_expr::simplify::SimplifyInfo; use datafusion_expr::var_provider::{is_system_variables, VarType}; use datafusion_expr::{ @@ -73,6 +76,7 @@ use datafusion_sql::planner::{ContextProvider, ParserOptions, PlannerContext, Sq use async_trait::async_trait; use chrono::{DateTime, Utc}; +use datafusion_common::types::LogicalTypeRef; use itertools::Itertools; use log::{debug, info}; use object_store::ObjectStore; @@ -148,6 +152,8 @@ pub struct SessionState { aggregate_functions: HashMap>, /// Window functions registered in the context window_functions: HashMap>, + /// Extension types registered in the context + extension_types: MemoryExtensionTypeRegistry, /// Deserializer registry for extensions. serializer_registry: Arc, /// Holds registered external FileFormat implementations @@ -247,6 +253,10 @@ impl Session for SessionState { &self.window_functions } + fn extension_types(&self) -> &MemoryExtensionTypeRegistry { + &self.extension_types + } + fn runtime_env(&self) -> &Arc { self.runtime_env() } @@ -1357,6 +1367,7 @@ impl SessionStateBuilder { scalar_functions: HashMap::new(), aggregate_functions: HashMap::new(), window_functions: HashMap::new(), + extension_types: MemoryExtensionTypeRegistry::new(), serializer_registry: serializer_registry .unwrap_or(Arc::new(EmptySerializerRegistry)), file_formats: HashMap::new(), @@ -1852,6 +1863,29 @@ impl FunctionRegistry for SessionState { } } +impl ExtensionTypeRegistry for SessionState { + fn get_extension_type( + &self, + name: &str, + ) -> datafusion_common::Result { + self.extension_types.get_extension_type(name) + } + + fn register_extension_type( + &mut self, + logical_type: LogicalTypeRef, + ) -> datafusion_common::Result> { + self.extension_types.register_extension_type(logical_type) + } + + fn deregister_extension_type( + &mut self, + name: &str, + ) -> datafusion_common::Result> { + self.extension_types.deregister_extension_type(name) + } +} + impl OptimizerConfig for SessionState { fn query_execution_start_time(&self) -> DateTime { self.execution_props.query_execution_start_time @@ -1881,6 +1915,7 @@ impl From<&SessionState> for TaskContext { state.scalar_functions.clone(), state.aggregate_functions.clone(), state.window_functions.clone(), + state.extension_types.clone(), Arc::clone(&state.runtime_env), ) } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index eef3d1b79694..9f0668afee14 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -27,7 +27,7 @@ use crate::datasource::listing::ListingTableUrl; use crate::datasource::physical_plan::FileSinkConfig; use crate::datasource::{source_as_provider, DefaultTableSource}; use crate::error::{DataFusionError, Result}; -use crate::execution::context::{ExecutionProps, SessionState}; +use crate::execution::context::SessionState; use crate::logical_expr::utils::generate_sort_key; use crate::logical_expr::{ Aggregate, EmptyRelation, Join, Projection, Sort, TableScan, Unnest, Values, Window, @@ -61,8 +61,8 @@ use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::recursive_query::RecursiveQueryExec; use arrow::array::{builder::StringBuilder, RecordBatch}; -use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; +use arrow_schema::SortOptions; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; use datafusion_common::{ @@ -92,6 +92,10 @@ use datafusion_physical_plan::DisplayFormatType; use crate::schema_equivalence::schema_satisfied_by; use async_trait::async_trait; +use datafusion_catalog::Session; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; +use datafusion_expr::registry::ExtensionTypeRegistry; use futures::{StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; use log::{debug, trace}; @@ -609,13 +613,7 @@ impl DefaultPhysicalPlanner { let logical_schema = node.schema(); let window_expr = window_expr .iter() - .map(|e| { - create_window_expr( - e, - logical_schema, - session_state.execution_props(), - ) - }) + .map(|e| create_window_expr(session_state, e, logical_schema)) .collect::>>()?; let can_repartition = session_state.config().target_partitions() > 1 @@ -707,10 +705,10 @@ impl DefaultPhysicalPlanner { .iter() .map(|e| { create_aggregate_expr_and_maybe_filter( + session_state, e, logical_input_schema, &physical_input_schema, - session_state.execution_props(), ) }) .collect::>>()?; @@ -820,11 +818,8 @@ impl DefaultPhysicalPlanner { }) => { let physical_input = children.one()?; let input_dfschema = input.as_ref().schema(); - let sort_expr = create_physical_sort_exprs( - expr, - input_dfschema, - session_state.execution_props(), - )?; + let sort_expr = + create_physical_sort_exprs(session_state, expr, input_dfschema)?; let new_sort = SortExec::new(sort_expr, physical_input).with_fetch(*fetch); Arc::new(new_sort) @@ -1509,10 +1504,10 @@ pub fn is_window_frame_bound_valid(window_frame: &WindowFrame) -> bool { /// Create a window expression with a name from a logical expression pub fn create_window_expr_with_name( + session_state: &SessionState, e: &Expr, name: impl Into, logical_schema: &DFSchema, - execution_props: &ExecutionProps, ) -> Result> { let name = name.into(); let physical_schema: &Schema = &logical_schema.into(); @@ -1528,12 +1523,13 @@ pub fn create_window_expr_with_name( null_treatment, }, }) => { + let execution_props = session_state.execution_props(); let physical_args = create_physical_exprs(args, logical_schema, execution_props)?; let partition_by = create_physical_exprs(partition_by, logical_schema, execution_props)?; let order_by = - create_physical_sort_exprs(order_by, logical_schema, execution_props)?; + create_physical_sort_exprs(session_state, order_by, logical_schema)?; if !is_window_frame_bound_valid(window_frame) { return plan_err!( @@ -1562,16 +1558,16 @@ pub fn create_window_expr_with_name( /// Create a window expression from a logical expression or an alias pub fn create_window_expr( + session_state: &SessionState, e: &Expr, logical_schema: &DFSchema, - execution_props: &ExecutionProps, ) -> Result> { // unpack aliased logical expressions, e.g. "sum(col) over () as total" let (name, e) = match e { Expr::Alias(Alias { expr, name, .. }) => (name.clone(), expr.as_ref()), _ => (e.schema_name().to_string(), e), }; - create_window_expr_with_name(e, name, logical_schema, execution_props) + create_window_expr_with_name(session_state, e, name, logical_schema) } type AggregateExprWithOptionalArgs = ( @@ -1584,11 +1580,11 @@ type AggregateExprWithOptionalArgs = ( /// Create an aggregate expression with a name from a logical expression pub fn create_aggregate_expr_with_name_and_maybe_filter( + session: &dyn Session, e: &Expr, name: Option, logical_input_schema: &DFSchema, physical_input_schema: &Schema, - execution_props: &ExecutionProps, ) -> Result { match e { Expr::AggregateFunction(AggregateFunction { @@ -1608,6 +1604,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( physical_name(e)? }; + let execution_props = session.execution_props(); let physical_args = create_physical_exprs(args, logical_input_schema, execution_props)?; let filter = match filter { @@ -1625,9 +1622,9 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( let (agg_expr, filter, order_by) = { let physical_sort_exprs = match order_by { Some(exprs) => Some(create_physical_sort_exprs( + session, exprs, logical_input_schema, - execution_props, )?), None => None, }; @@ -1656,10 +1653,10 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( /// Create an aggregate expression from a logical expression or an alias pub fn create_aggregate_expr_and_maybe_filter( + session: &dyn Session, e: &Expr, logical_input_schema: &DFSchema, physical_input_schema: &Schema, - execution_props: &ExecutionProps, ) -> Result { // unpack (nested) aliased logical expressions, e.g. "sum(col) as total" let (name, e) = match e { @@ -1669,28 +1666,42 @@ pub fn create_aggregate_expr_and_maybe_filter( }; create_aggregate_expr_with_name_and_maybe_filter( + session, e, name, logical_input_schema, physical_input_schema, - execution_props, ) } /// Create a physical sort expression from a logical expression pub fn create_physical_sort_expr( + session: &dyn Session, e: &SortExpr, input_dfschema: &DFSchema, - execution_props: &ExecutionProps, ) -> Result { + // TODO this is not a nice solution. Somewhat related to the discussion in #14247 as we would + // a field method for PhysicalExpr. + let extension_types = session.extension_types(); + let ordering = match &e.expr { + Expr::Column(name) => input_dfschema + .field_from_column(name)? + .extension_type_name() + .and_then(|ext| extension_types.get_extension_type(ext).ok()) + .map(|ext| ext.planning_information().ordering.clone()) + .unwrap_or_default(), + _ => SortOrdering::Default, + }; + let SortExpr { expr, asc, nulls_first, } = e; Ok(PhysicalSortExpr { - expr: create_physical_expr(expr, input_dfschema, execution_props)?, - options: SortOptions { + expr: create_physical_expr(expr, input_dfschema, session.execution_props())?, + options: AdvSortOptions { + ordering, descending: !asc, nulls_first: *nulls_first, }, @@ -1699,13 +1710,13 @@ pub fn create_physical_sort_expr( /// Create vector of physical sort expression from a vector of logical expression pub fn create_physical_sort_exprs( + session: &dyn Session, exprs: &[SortExpr], input_dfschema: &DFSchema, - execution_props: &ExecutionProps, ) -> Result { exprs .iter() - .map(|expr| create_physical_sort_expr(expr, input_dfschema, execution_props)) + .map(|expr| create_physical_sort_expr(session, expr, input_dfschema)) .collect::>() } diff --git a/datafusion/core/tests/dataframe/mod.rs b/datafusion/core/tests/dataframe/mod.rs index 43428d6846a1..68c59bffdf92 100644 --- a/datafusion/core/tests/dataframe/mod.rs +++ b/datafusion/core/tests/dataframe/mod.rs @@ -18,26 +18,30 @@ // Include tests in dataframe_functions mod dataframe_functions; mod describe; +mod test_types; use arrow::array::{ - record_batch, Array, ArrayRef, BooleanArray, DictionaryArray, FixedSizeListArray, - FixedSizeListBuilder, Float32Array, Float64Array, Int32Array, Int32Builder, - Int8Array, LargeListArray, ListArray, ListBuilder, RecordBatch, StringArray, - StringBuilder, StructBuilder, UInt32Array, UInt32Builder, UnionArray, + as_union_array, record_batch, Array, ArrayRef, AsArray, BooleanArray, + DictionaryArray, FixedSizeListArray, FixedSizeListBuilder, Float32Array, + Float64Array, Int32Array, Int32Builder, Int8Array, LargeListArray, ListArray, + ListBuilder, RecordBatch, StringArray, StringBuilder, StructBuilder, UInt32Array, + UInt32Builder, UnionArray, UnionBuilder, }; use arrow::buffer::ScalarBuffer; use arrow::datatypes::{ - DataType, Field, Float32Type, Int32Type, Schema, SchemaRef, UInt64Type, UnionFields, - UnionMode, + DataType, Field, Float32Type, Float64Type, Int32Type, Int64Type, Schema, SchemaRef, + UInt64Type, UnionFields, UnionMode, }; use arrow::error::ArrowError; use arrow::util::pretty::pretty_format_batches; +use arrow_schema::extension::EXTENSION_TYPE_NAME_KEY; use datafusion_functions_aggregate::count::{count_all, count_all_window}; use datafusion_functions_aggregate::expr_fn::{ array_agg, avg, count, count_distinct, max, median, min, sum, }; use datafusion_functions_nested::make_array::make_array_udf; use datafusion_functions_window::expr_fn::{first_value, row_number}; +use futures::StreamExt; use object_store::local::LocalFileSystem; use sqlparser::ast::NullTreatment; use std::collections::HashMap; @@ -46,6 +50,7 @@ use std::sync::Arc; use tempfile::TempDir; use url::Url; +use crate::dataframe::test_types::IntOrFloatType; use datafusion::dataframe::{DataFrame, DataFrameWriteOptions}; use datafusion::datasource::MemTable; use datafusion::error::Result; @@ -69,6 +74,7 @@ use datafusion_common_runtime::SpawnedTask; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_expr::expr::{GroupingSet, Sort, WindowFunction}; +use datafusion_expr::registry::ExtensionTypeRegistry; use datafusion_expr::var_provider::{VarProvider, VarType}; use datafusion_expr::{ cast, col, create_udf, exists, in_subquery, lit, out_ref_col, placeholder, @@ -3080,6 +3086,61 @@ async fn sort_on_ambiguous_column() -> Result<()> { Ok(()) } +#[tokio::test] +async fn sort_on_union_with_logical_type() -> Result<()> { + let mut builder = UnionBuilder::new_dense(); + builder.append::("integer", 10)?; + builder.append::("float", 6.0)?; + builder.append::("integer", -1)?; + builder.append::("float", 3.0)?; + let union = builder.build()?; + + let my_extension_type = Arc::new(IntOrFloatType::new()); + let field = Field::new("my_union", union.data_type().clone(), false).with_metadata( + HashMap::from([( + EXTENSION_TYPE_NAME_KEY.into(), + IntOrFloatType::name().into(), + )]), + ); + let schema = Arc::new(Schema::new(vec![field])); + + let mut ctx = SessionContext::new(); + ctx.register_extension_type(my_extension_type)?; + ctx.register_table( + "test_table", + Arc::new(MemTable::try_new( + schema.clone(), + vec![vec![RecordBatch::try_new( + schema.clone(), + vec![Arc::new(union)], + )?]], + )?), + )?; + + let record_batch = ctx + .table("test_table") + .await? + .sort_by(vec![col("my_union")])? + .execute_stream() + .await? + .next() + .await + .unwrap()?; + + let result = as_union_array(record_batch.column_by_name("my_union").unwrap()); + assert_eq!(result.type_ids(), &[0, 1, 1, 0]); + assert_eq!( + result.child(0).as_primitive::().values(), + &[-1, 10] + ); + assert_eq!( + result.child(1).as_primitive::().values(), + &[3.0, 6.0] + ); + + Ok(()) +} + #[tokio::test] async fn group_by_ambiguous_column() -> Result<()> { let err = create_test_table("t1") diff --git a/datafusion/core/tests/dataframe/test_types.rs b/datafusion/core/tests/dataframe/test_types.rs new file mode 100644 index 000000000000..269031f0aca2 --- /dev/null +++ b/datafusion/core/tests/dataframe/test_types.rs @@ -0,0 +1,148 @@ +use arrow::array::{Array, ArrayRef, AsArray, DynComparator, UnionArray}; +use arrow::datatypes::{Float64Type, Int64Type}; +use arrow_schema::{DataType, SortOptions}; +use datafusion_common::cast::as_union_array; +use datafusion_common::types::{ + logical_float64, logical_int32, CustomOrdering, LogicalField, LogicalType, + LogicalTypePlanningInformation, NativeType, SortOrdering, TypeSignature, +}; +use std::cmp::Ordering; +use std::sync::Arc; + +/// Represents a type that is either an integer or a float. +pub struct IntOrFloatType { + native_type: NativeType, +} + +impl IntOrFloatType { + pub fn name() -> &'static str { + "int_or_float" + } + + pub fn new() -> IntOrFloatType { + let fields = [ + ( + 0, + Arc::new(LogicalField::new("integer", logical_int32(), false)), + ), + ( + 1, + Arc::new(LogicalField::new("float", logical_float64(), false)), + ), + ] + .into_iter() + .collect(); + Self { + native_type: NativeType::Union(fields), + } + } +} + +impl LogicalType for IntOrFloatType { + fn native(&self) -> &NativeType { + &self.native_type + } + + fn signature(&self) -> TypeSignature<'_> { + TypeSignature::Extension { + name: Self::name(), + parameters: &[], + } + } + + fn default_cast_for( + &self, + _origin: &DataType, + ) -> datafusion_common::Result { + unimplemented!() + } + + fn planning_information(&self) -> LogicalTypePlanningInformation { + LogicalTypePlanningInformation { + ordering: SortOrdering::Custom(Arc::new(IntOrFloatTypeOrdering {})), + } + } +} + +/// The order of the IntOrFloat is computed by converting both values to an `f64` and comparing +/// the resulting value. +#[derive(Debug)] +struct IntOrFloatTypeOrdering {} + +impl CustomOrdering for IntOrFloatTypeOrdering { + fn ordering_id(&self) -> &str { + "order_int_or_float" + } + + fn dyn_comparator( + &self, + array: ArrayRef, + options: SortOptions, + ) -> datafusion_common::Result { + // TODO check data type + + Ok(Box::new(move |lhs, rhs| { + let array = as_union_array(array.as_ref()).expect("should be union"); + + match (array.is_null(lhs), array.is_null(rhs)) { + (true, true) => Ordering::Equal, + (true, false) => { + if options.nulls_first { + Ordering::Less + } else { + Ordering::Greater + } + } + (false, true) => { + if options.nulls_first { + Ordering::Greater + } else { + Ordering::Less + } + } + (false, false) => { + let result = compare_impl(array, lhs, rhs); + match options.descending { + true => result.reverse(), + false => result, + } + } + } + })) + } +} + +/// Default comparison between two (`lhs` & `rhs`) non-null [IntOrFloat] elements. +fn compare_impl(array: &UnionArray, lhs: usize, rhs: usize) -> Ordering { + let type_lhs = array.type_ids()[lhs]; + let type_rhs = array.type_ids()[rhs]; + + let offset_lhs = array.value_offset(lhs); + let offset_rhs = array.value_offset(rhs); + + let lhs = match type_lhs { + 0 => { + let array = array.child(type_lhs).as_primitive::(); + array.value(offset_lhs) as f64 + } + 1 => { + let array = array.child(type_lhs).as_primitive::(); + array.value(offset_lhs) + } + _ => unreachable!("Union only has two variants"), + }; + + let rhs = match type_rhs { + 0 => { + let array = array.child(type_rhs).as_primitive::(); + array.value(offset_rhs) as f64 + } + 1 => { + let array = array.child(type_rhs).as_primitive::(); + array.value(offset_rhs) + } + _ => unreachable!("Union only has two variants"), + }; + + lhs.total_cmp(&rhs) +} diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 1025a49ea1e3..1d0953338953 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -23,7 +23,7 @@ use crate::fuzz_cases::aggregation_fuzzer::{ }; use arrow::array::{types::Int64Type, Array, ArrayRef, AsArray, Int64Array, RecordBatch}; -use arrow::compute::{concat_batches, SortOptions}; +use arrow::compute::concat_batches; use arrow::datatypes::{ DataType, IntervalUnit, TimeUnit, DECIMAL128_MAX_PRECISION, DECIMAL128_MAX_SCALE, DECIMAL256_MAX_PRECISION, DECIMAL256_MAX_SCALE, @@ -48,6 +48,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::InputOrderMode; use test_utils::{add_empty_batches, StringBatchGenerator}; +use datafusion_common::sort::AdvSortOptions; use rand::rngs::StdRng; use rand::{thread_rng, Rng, SeedableRng}; use tokio::task::JoinSet; @@ -315,7 +316,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }) } diff --git a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs index 769deef1187d..8a46a4371080 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/ordering.rs @@ -21,6 +21,8 @@ use crate::fuzz_cases::equivalence::utils::{ is_table_same_after_sort, TestScalarUDF, }; use arrow::compute::SortOptions; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use datafusion_common::Result; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::expressions::{col, BinaryExpr}; @@ -35,7 +37,8 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { const N_RANDOM_SCHEMA: usize = 5; const N_ELEMENTS: usize = 125; const N_DISTINCT: usize = 5; - const SORT_OPTIONS: SortOptions = SortOptions { + const SORT_OPTIONS: AdvSortOptions = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -61,7 +64,7 @@ fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { .into_iter() .map(|expr| PhysicalSortExpr { expr: Arc::clone(expr), - options: SORT_OPTIONS, + options: SORT_OPTIONS.clone(), }) .collect::(); let expected = is_table_same_after_sort( @@ -92,7 +95,8 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { const N_RANDOM_SCHEMA: usize = 100; const N_ELEMENTS: usize = 125; const N_DISTINCT: usize = 5; - const SORT_OPTIONS: SortOptions = SortOptions { + const SORT_OPTIONS: AdvSortOptions = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -133,7 +137,7 @@ fn test_ordering_satisfy_with_equivalence_complex_random() -> Result<()> { .into_iter() .map(|expr| PhysicalSortExpr { expr: Arc::clone(expr), - options: SORT_OPTIONS, + options: SORT_OPTIONS.clone(), }) .collect::(); let expected = is_table_same_after_sort( @@ -309,7 +313,11 @@ fn test_ordering_satisfy_with_equivalence() -> Result<()> { .into_iter() .map(|(expr, options)| PhysicalSortExpr { expr: Arc::clone(expr), - options, + options: AdvSortOptions::new( + SortOrdering::Default, + options.descending, + options.nulls_first, + ), }) .collect::(); diff --git a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs index a3fa1157b38f..a8bd8301bf1c 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/projection.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/projection.rs @@ -19,7 +19,8 @@ use crate::fuzz_cases::equivalence::utils::{ apply_projection, create_random_schema, generate_table_for_eq_properties, is_table_same_after_sort, TestScalarUDF, }; -use arrow::compute::SortOptions; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use datafusion_common::Result; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr::equivalence::ProjectionMapping; @@ -108,7 +109,8 @@ fn ordering_satisfy_after_projection_random() -> Result<()> { const N_RANDOM_SCHEMA: usize = 20; const N_ELEMENTS: usize = 125; const N_DISTINCT: usize = 5; - const SORT_OPTIONS: SortOptions = SortOptions { + const SORT_OPTIONS: AdvSortOptions = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; diff --git a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs index 593e1c6c2dca..60356ceb6d4d 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/properties.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/properties.rs @@ -75,7 +75,7 @@ fn test_find_longest_permutation_random() -> Result<()> { .zip(ordering.iter()) .map(|(&idx, sort_expr)| PhysicalSortExpr { expr: Arc::clone(&exprs[idx]), - options: sort_expr.options, + options: sort_expr.options.clone(), }) .collect::(); assert_eq!( diff --git a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs index d4b41b686631..fd6788b26be7 100644 --- a/datafusion/core/tests/fuzz_cases/equivalence/utils.rs +++ b/datafusion/core/tests/fuzz_cases/equivalence/utils.rs @@ -23,8 +23,8 @@ use std::cmp::Ordering; use std::sync::Arc; use arrow::array::{ArrayRef, Float32Array, Float64Array, RecordBatch, UInt32Array}; -use arrow::compute::SortOptions; -use arrow::compute::{lexsort_to_indices, take_record_batch, SortColumn}; +use arrow::compute::{lexsort_to_indices, take_record_batch}; +use arrow::compute::{SortColumn, SortOptions}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, plan_datafusion_err, DataFusionError, Result}; @@ -36,6 +36,8 @@ use datafusion_physical_expr::equivalence::{EquivalenceClass, ProjectionMapping} use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use itertools::izip; use rand::prelude::*; @@ -108,7 +110,8 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti let mut rng = StdRng::seed_from_u64(seed); let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted - let options_asc = SortOptions { + let options_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -121,7 +124,7 @@ pub fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperti .drain(0..n_sort_expr) .map(|expr| PhysicalSortExpr { expr: Arc::clone(expr), - options: options_asc, + options: options_asc.clone(), }) .collect(); @@ -267,7 +270,7 @@ pub fn is_table_same_after_sort( let values = expr_result.into_array(new_batch.num_rows())?; Ok(SortColumn { values, - options: Some(order_expr.options), + options: Some(order_expr.options.to_arrow().unwrap()), }) }) .collect::>>()?; @@ -394,7 +397,7 @@ pub fn generate_table_for_eq_properties( ( SortColumn { values: arr, - options: Some(*options), + options: Some(options.to_arrow().unwrap()), }, idx, ) @@ -502,7 +505,7 @@ pub fn convert_to_sort_exprs( .iter() .map(|(expr, options)| PhysicalSortExpr { expr: Arc::clone(*expr), - options: *options, + options: AdvSortOptions::with_default_ordering(*options), }) .collect() } @@ -580,7 +583,7 @@ impl ScalarUDFImpl for TestScalarUDF { } fn output_ordering(&self, input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { diff --git a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs index 92f375525066..6592f57adfa0 100644 --- a/datafusion/core/tests/fuzz_cases/merge_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/merge_fuzz.rs @@ -21,7 +21,6 @@ use std::sync::Arc; use arrow::{ array::{ArrayRef, Int32Array}, - compute::SortOptions, record_batch::RecordBatch, }; use datafusion::datasource::memory::MemorySourceConfig; @@ -31,6 +30,8 @@ use datafusion::physical_plan::{ sorts::sort_preserving_merge::SortPreservingMergeExec, }; use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use datafusion_physical_expr_common::sort_expr::LexOrdering; use test_utils::{batches_to_vec, partitions_to_sorted_vec, stagger_batch_with_seed}; @@ -111,7 +112,8 @@ async fn run_merge_test(input: Vec>) { let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("x", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, diff --git a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs index 0b0f0aa2f105..98c415b98cf2 100644 --- a/datafusion/core/tests/fuzz_cases/sort_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_fuzz.rs @@ -21,7 +21,6 @@ use std::sync::Arc; use arrow::{ array::{as_string_array, ArrayRef, Int32Array, StringArray}, - compute::SortOptions, record_batch::RecordBatch, }; use datafusion::datasource::memory::MemorySourceConfig; @@ -35,6 +34,8 @@ use datafusion_execution::memory_pool::GreedyMemoryPool; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use rand::Rng; use test_utils::{batches_to_vec, partitions_to_sorted_vec}; @@ -237,7 +238,8 @@ impl SortTest { .iter() .map(|c| PhysicalSortExpr { expr: col(c, &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 06b93d41af36..201586f2e155 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -20,9 +20,8 @@ mod sp_repartition_fuzz_tests { use std::sync::Arc; use arrow::array::{ArrayRef, Int64Array, RecordBatch, UInt64Array}; - use arrow::compute::{concat_batches, lexsort, SortColumn, SortOptions}; + use arrow::compute::{concat_batches, lexsort, SortColumn}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion::physical_plan::{ collect, metrics::{BaselineMetrics, ExecutionPlanMetricsSet}, @@ -46,6 +45,8 @@ mod sp_repartition_fuzz_tests { use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; use rand::{rngs::StdRng, seq::SliceRandom, Rng, SeedableRng}; @@ -88,7 +89,8 @@ mod sp_repartition_fuzz_tests { let mut rng = StdRng::seed_from_u64(seed); let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted - let options_asc = SortOptions { + let options_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -101,7 +103,7 @@ mod sp_repartition_fuzz_tests { .drain(0..n_sort_expr) .map(|expr| PhysicalSortExpr { expr: expr.clone(), - options: options_asc, + options: options_asc.clone(), }) .collect(); @@ -169,7 +171,7 @@ mod sp_repartition_fuzz_tests { ( SortColumn { values: arr, - options: Some(*options), + options: Some(options.to_arrow().unwrap()), }, idx, ) @@ -351,7 +353,7 @@ mod sp_repartition_fuzz_tests { for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { expr: col(ordering_col, &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }) } diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index a7f9e38c9ae3..b5b6c9446c80 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use arrow::array::{ArrayRef, Int32Array, StringArray}; -use arrow::compute::{concat_batches, SortOptions}; +use arrow::compute::concat_batches; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; @@ -51,6 +51,7 @@ use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_common::sort::AdvSortOptions; use rand::distributions::Alphanumeric; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; @@ -598,7 +599,7 @@ async fn run_window_test( for column in &orderby_columns { orderby_exprs.push(PhysicalSortExpr { expr: col(column, &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }) } if orderby_exprs.len() > 1 && !window_frame.can_accept_multi_orderby() { @@ -612,7 +613,7 @@ async fn run_window_test( for partition_by_expr in &partitionby_exprs { sort_keys.push(PhysicalSortExpr { expr: partition_by_expr.clone(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }) } for order_by_expr in &orderby_exprs { diff --git a/datafusion/core/tests/memory_limit/mod.rs b/datafusion/core/tests/memory_limit/mod.rs index 8f690edc5426..162a56d37824 100644 --- a/datafusion/core/tests/memory_limit/mod.rs +++ b/datafusion/core/tests/memory_limit/mod.rs @@ -24,7 +24,6 @@ use std::sync::{Arc, LazyLock}; #[cfg(feature = "extended_tests")] mod memory_limit_validation; use arrow::array::{ArrayRef, DictionaryArray, Int32Array, RecordBatch, StringViewArray}; -use arrow::compute::SortOptions; use arrow::datatypes::{Int32Type, SchemaRef}; use arrow_schema::{DataType, Field, Schema}; use datafusion::assert_batches_eq; @@ -54,6 +53,8 @@ use rand::Rng; use test_utils::AccessLogGenerator; use async_trait::async_trait; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use futures::StreamExt; use tokio::fs::File; @@ -749,14 +750,15 @@ impl Scenario { .collect(); let schema = batches[0][0].schema(); - let options = SortOptions { + let options = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; let sort_information = vec![LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options, + options: options.clone(), }, PhysicalSortExpr { expr: col("b", &schema).unwrap(), diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 85d826109f89..ae37c64c5b8f 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -25,7 +25,6 @@ use crate::physical_optimizer::test_utils::{ sort_merge_join_exec, sort_preserving_merge_exec, }; -use arrow::compute::SortOptions; use datafusion::config::ConfigOptions; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; @@ -33,6 +32,7 @@ use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSource}; use datafusion::datasource::source::DataSourceExec; use datafusion_common::error::Result; +use datafusion_common::sort::AdvSortOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; use datafusion_expr::{JoinType, Operator}; @@ -1670,7 +1670,7 @@ fn merge_does_not_need_sort() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); // Scan some sorted parquet files @@ -1872,7 +1872,7 @@ fn repartition_sorted_limit() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); @@ -1894,7 +1894,7 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = sort_required_exec_with_req( filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), @@ -1973,7 +1973,7 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); @@ -1994,7 +1994,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = sort_preserving_merge_exec( sort_key.clone(), @@ -2026,7 +2026,7 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); let plan = sort_preserving_merge_exec(sort_key, input); @@ -2060,7 +2060,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("d", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = sort_required_exec_with_req( filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), @@ -2095,7 +2095,7 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input1 = sort_required_exec_with_req( parquet_exec_with_sort(vec![sort_key.clone()]), @@ -2137,7 +2137,7 @@ fn repartition_transitively_with_projection() -> Result<()> { let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("sum", &proj.schema()).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = sort_preserving_merge_exec(sort_key, proj); @@ -2170,7 +2170,7 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let alias = vec![ ("a".to_string(), "a".to_string()), @@ -2203,7 +2203,7 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let alias = vec![ ("a".to_string(), "a".to_string()), @@ -2236,7 +2236,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); @@ -2270,7 +2270,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = sort_exec( sort_key, @@ -2340,7 +2340,7 @@ fn parallelization_multiple_files() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); @@ -2493,7 +2493,7 @@ fn parallelization_sorted_limit() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); @@ -2525,7 +2525,7 @@ fn parallelization_limit_with_filter() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan_parquet = limit_exec(filter_exec(sort_exec( sort_key.clone(), @@ -2646,7 +2646,7 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); // sort preserving merge already sorted input, let plan_parquet = sort_preserving_merge_exec( @@ -2674,7 +2674,7 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) let input_parquet = @@ -2708,7 +2708,7 @@ fn parallelization_does_not_benefit() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); // SortRequired // Parquet(sorted) @@ -2740,7 +2740,7 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); //Projection(a as a2, b as b2) @@ -2752,7 +2752,7 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> projection_exec_with_alias(parquet_exec_with_sort(vec![sort_key]), alias_pairs); let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_parquet.schema()).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan_parquet = sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); @@ -2779,7 +2779,7 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); //Projection(a as a2, b as b2) @@ -2792,7 +2792,7 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_csv.schema()).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); let expected = &[ @@ -2842,7 +2842,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -2867,7 +2867,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("d", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -2890,7 +2890,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -2922,7 +2922,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); @@ -2946,13 +2946,13 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec_multiple_sorted(vec![sort_key]); let sort_req = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); @@ -2984,7 +2984,7 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec_multiple_sorted(vec![sort_key]); let physical_plan = filter_exec(input); @@ -3005,7 +3005,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec(); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); @@ -3042,7 +3042,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); @@ -3079,7 +3079,7 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_with_sort(vec![sort_key]); @@ -3102,7 +3102,7 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { let schema = schema(); let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let alias = vec![("a".to_string(), "a".to_string())]; let input = parquet_exec_multiple_sorted(vec![sort_key]); diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index dd2c1960a658..4d5cf93fc0dd 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -17,10 +17,10 @@ use std::sync::Arc; -use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; +use datafusion_common::sort::AdvSortOptions; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; @@ -317,7 +317,7 @@ fn pushes_global_limit_into_multiple_fetch_plans() -> Result<()> { let sort = sort_exec( vec![PhysicalSortExpr { expr: col("c1", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }], repartition, ); diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index f9810eab8f59..1cd43c6085e0 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -25,8 +25,9 @@ use crate::physical_optimizer::test_utils::{ }; use arrow::datatypes::DataType; -use arrow::{compute::SortOptions, util::pretty::pretty_format_batches}; +use arrow::util::pretty::pretty_format_batches; use datafusion::prelude::SessionContext; +use datafusion_common::sort::AdvSortOptions; use datafusion_common::Result; use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; @@ -238,7 +239,7 @@ async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { fn test_has_order_by() -> Result<()> { let sort_key = LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema()).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let source = parquet_exec_with_sort(vec![sort_key]); let schema = source.schema(); diff --git a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs index b0b5f731063f..b1d7e7994c48 100644 --- a/datafusion/core/tests/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/projection_pushdown.rs @@ -18,7 +18,6 @@ use std::any::Any; use std::sync::Arc; -use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::memory::MemorySourceConfig; @@ -57,6 +56,7 @@ use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; +use datafusion_common::sort::AdvSortOptions; use itertools::Itertools; /// Mocked UDF @@ -510,16 +510,16 @@ fn test_streaming_table_after_projection() -> Result<()> { LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("e", 2)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, ]), LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("d", 3)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), ] .into_iter(), @@ -570,16 +570,16 @@ fn test_streaming_table_after_projection() -> Result<()> { LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("e", 1)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, PhysicalSortExpr { expr: Arc::new(Column::new("a", 2)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, ]), LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("d", 0)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), ] ); @@ -644,7 +644,7 @@ fn test_output_req_after_projection() -> Result<()> { Some(LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 1)), - options: Some(SortOptions::default()), + options: Some(AdvSortOptions::default()), }, PhysicalSortRequirement { expr: Arc::new(BinaryExpr::new( @@ -652,7 +652,7 @@ fn test_output_req_after_projection() -> Result<()> { Operator::Plus, Arc::new(Column::new("a", 0)), )), - options: Some(SortOptions::default()), + options: Some(AdvSortOptions::default()), }, ])), Distribution::HashPartitioned(vec![ @@ -690,7 +690,7 @@ fn test_output_req_after_projection() -> Result<()> { let expected_reqs = LexRequirement::new(vec![ PhysicalSortRequirement { expr: Arc::new(Column::new("b", 2)), - options: Some(SortOptions::default()), + options: Some(AdvSortOptions::default()), }, PhysicalSortRequirement { expr: Arc::new(BinaryExpr::new( @@ -698,7 +698,7 @@ fn test_output_req_after_projection() -> Result<()> { Operator::Plus, Arc::new(Column::new("new_a", 1)), )), - options: Some(SortOptions::default()), + options: Some(AdvSortOptions::default()), }, ]); assert_eq!( @@ -1249,7 +1249,7 @@ fn test_sort_after_projection() -> Result<()> { LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, PhysicalSortExpr { expr: Arc::new(BinaryExpr::new( @@ -1257,7 +1257,7 @@ fn test_sort_after_projection() -> Result<()> { Operator::Plus, Arc::new(Column::new("a", 0)), )), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, ]), csv.clone(), @@ -1299,7 +1299,7 @@ fn test_sort_preserving_after_projection() -> Result<()> { LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, PhysicalSortExpr { expr: Arc::new(BinaryExpr::new( @@ -1307,7 +1307,7 @@ fn test_sort_preserving_after_projection() -> Result<()> { Operator::Plus, Arc::new(Column::new("a", 0)), )), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, ]), csv.clone(), diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 58eb866c590c..58154b7070e8 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -47,6 +47,7 @@ use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; use datafusion_common::config::ConfigOptions; +use datafusion_common::sort::AdvSortOptions; use object_store::memory::InMemory; use object_store::ObjectStore; use rstest::rstest; @@ -1141,7 +1142,7 @@ fn sort_expr_options( ) -> PhysicalSortExpr { PhysicalSortExpr { expr: col(name, schema).unwrap(), - options, + options: AdvSortOptions::with_default_ordering(options), } } diff --git a/datafusion/core/tests/physical_optimizer/test_utils.rs b/datafusion/core/tests/physical_optimizer/test_utils.rs index 1b8e754ee357..9fc0f697233a 100644 --- a/datafusion/core/tests/physical_optimizer/test_utils.rs +++ b/datafusion/core/tests/physical_optimizer/test_utils.rs @@ -30,6 +30,7 @@ use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::physical_plan::ParquetSource; use datafusion::datasource::source::DataSourceExec; use datafusion_common::config::ConfigOptions; +use datafusion_common::sort::AdvSortOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; @@ -160,7 +161,7 @@ pub fn sort_expr_options( ) -> PhysicalSortExpr { PhysicalSortExpr { expr: col(name, schema).unwrap(), - options, + options: AdvSortOptions::with_default_ordering(options), } } diff --git a/datafusion/datasource/src/file_scan_config.rs b/datafusion/datasource/src/file_scan_config.rs index 91b5f0157739..b3d3549179dd 100644 --- a/datafusion/datasource/src/file_scan_config.rs +++ b/datafusion/datasource/src/file_scan_config.rs @@ -1004,7 +1004,7 @@ fn get_projected_output_ordering( // Compute the new sort expression (with correct index) after projection: new_ordering.push(PhysicalSortExpr { expr: Arc::new(Column::new(name, idx)), - options: *options, + options: options.clone(), }); continue; } @@ -1082,12 +1082,11 @@ mod tests { use crate::{test_util::MockSource, tests::aggr_test_schema}; use super::*; - use arrow::{ - array::{Int32Array, RecordBatch}, - compute::SortOptions, - }; + use arrow::array::{Int32Array, RecordBatch}; + use datafusion_common::sort::AdvSortOptions; use datafusion_common::stats::Precision; + use datafusion_common::types::SortOrdering; use datafusion_common::{assert_batches_eq, DFSchema}; use datafusion_expr::{execution_props::ExecutionProps, SortExpr}; use datafusion_physical_expr::create_physical_expr; @@ -1105,7 +1104,8 @@ mod tests { } = e; Ok(PhysicalSortExpr { expr: create_physical_expr(expr, input_dfschema, execution_props)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: !asc, nulls_first: *nulls_first, }, diff --git a/datafusion/datasource/src/memory.rs b/datafusion/datasource/src/memory.rs index 64fd56971b29..2f01010f245d 100644 --- a/datafusion/datasource/src/memory.rs +++ b/datafusion/datasource/src/memory.rs @@ -728,8 +728,8 @@ mod memory_source_tests { use crate::source::DataSourceExec; use datafusion_physical_plan::ExecutionPlan; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::sort::AdvSortOptions; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -744,16 +744,16 @@ mod memory_source_tests { let sort1 = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }, PhysicalSortExpr { expr: col("b", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }, ]); let sort2 = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let mut expected_output_order = LexOrdering::default(); expected_output_order.extend(sort1.clone()); diff --git a/datafusion/datasource/src/statistics.rs b/datafusion/datasource/src/statistics.rs index 9df5aa993d43..4ff6f59b3822 100644 --- a/datafusion/datasource/src/statistics.rs +++ b/datafusion/datasource/src/statistics.rs @@ -122,7 +122,7 @@ impl MinMaxStatistics { .enumerate() .map(|(i, (col, sort))| PhysicalSortExpr { expr: Arc::new(Column::new(col.name(), i)), - options: sort.options, + options: sort.options.clone(), }) .collect::>(), ); @@ -175,9 +175,12 @@ impl MinMaxStatistics { let sort_fields = sort_order .iter() .map(|expr| { - expr.expr - .data_type(schema) - .map(|data_type| SortField::new_with_options(data_type, expr.options)) + expr.expr.data_type(schema).and_then(|data_type| { + Ok(SortField::new_with_options( + data_type, + expr.options.to_arrow()?, + )) + }) }) .collect::>>() .map_err(|e| e.context("create sort fields"))?; @@ -233,7 +236,7 @@ impl MinMaxStatistics { Ok(SortColumn { values: Arc::clone(values.column(idx)), - options: Some(sort_expr.options), + options: Some(sort_expr.options.to_arrow()?), }) }) .collect::>>() diff --git a/datafusion/execution/src/task.rs b/datafusion/execution/src/task.rs index b11596c4a30f..46fcb0b98bb6 100644 --- a/datafusion/execution/src/task.rs +++ b/datafusion/execution/src/task.rs @@ -19,8 +19,10 @@ use crate::{ config::SessionConfig, memory_pool::MemoryPool, registry::FunctionRegistry, runtime_env::RuntimeEnv, }; +use datafusion_common::types::LogicalTypeRef; use datafusion_common::{plan_datafusion_err, DataFusionError, Result}; use datafusion_expr::planner::ExprPlanner; +use datafusion_expr::registry::{ExtensionTypeRegistry, MemoryExtensionTypeRegistry}; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; use std::collections::HashSet; use std::{collections::HashMap, sync::Arc}; @@ -46,6 +48,8 @@ pub struct TaskContext { aggregate_functions: HashMap>, /// Window functions associated with this task context window_functions: HashMap>, + /// Extension types associated with this task context + extension_types: MemoryExtensionTypeRegistry, /// Runtime environment associated with this task context runtime: Arc, } @@ -62,6 +66,7 @@ impl Default for TaskContext { scalar_functions: HashMap::new(), aggregate_functions: HashMap::new(), window_functions: HashMap::new(), + extension_types: MemoryExtensionTypeRegistry::new(), runtime, } } @@ -80,6 +85,7 @@ impl TaskContext { scalar_functions: HashMap>, aggregate_functions: HashMap>, window_functions: HashMap>, + extension_types: MemoryExtensionTypeRegistry, runtime: Arc, ) -> Self { Self { @@ -89,6 +95,7 @@ impl TaskContext { scalar_functions, aggregate_functions, window_functions, + extension_types, runtime, } } @@ -203,6 +210,26 @@ impl FunctionRegistry for TaskContext { } } +impl ExtensionTypeRegistry for TaskContext { + fn get_extension_type(&self, name: &str) -> Result { + self.extension_types.get_extension_type(name) + } + + fn register_extension_type( + &mut self, + logical_type: LogicalTypeRef, + ) -> Result> { + self.extension_types.register_extension_type(logical_type) + } + + fn deregister_extension_type( + &mut self, + name: &str, + ) -> Result> { + self.extension_types.deregister_extension_type(name) + } +} + #[cfg(test)] mod tests { use super::*; @@ -240,6 +267,7 @@ mod tests { HashMap::default(), HashMap::default(), HashMap::default(), + MemoryExtensionTypeRegistry::new(), runtime, ); @@ -272,6 +300,7 @@ mod tests { HashMap::default(), HashMap::default(), HashMap::default(), + MemoryExtensionTypeRegistry::new(), runtime, ); diff --git a/datafusion/expr-common/src/sort_properties.rs b/datafusion/expr-common/src/sort_properties.rs index 5d17a34a96fb..1a5ee26f5005 100644 --- a/datafusion/expr-common/src/sort_properties.rs +++ b/datafusion/expr-common/src/sort_properties.rs @@ -19,10 +19,10 @@ use std::ops::Neg; use crate::interval_arithmetic::Interval; -use arrow::compute::SortOptions; use arrow::datatypes::DataType; +use datafusion_common::sort::AdvSortOptions; -/// To propagate [`SortOptions`] across the `PhysicalExpr`, it is insufficient +/// To propagate [`AdvSortOptions`] across the `PhysicalExpr`, it is insufficient /// to simply use `Option`: There must be a differentiation between /// unordered columns and literal values, since literals may not break the ordering /// when they are used as a child of some binary expression when the other child has @@ -33,29 +33,29 @@ use arrow::datatypes::DataType; /// sorted data; however the ((a_ordered + 999) + c_ordered) expression can. Therefore, /// we need two different variants for literals and unordered columns as literals are /// often more ordering-friendly under most mathematical operations. -#[derive(PartialEq, Debug, Clone, Copy, Default)] +#[derive(PartialEq, Debug, Clone, Default)] pub enum SortProperties { - /// Use the ordinary [`SortOptions`] struct to represent ordered data: - Ordered(SortOptions), - // This alternative represents unordered data: + /// Use the ordinary [`AdvSortOptions`] struct to represent ordered data + Ordered(AdvSortOptions), + /// This alternative represents unordered data: #[default] Unordered, - // Singleton is used for single-valued literal numbers: + /// Singleton is used for single-valued literal numbers: Singleton, } impl SortProperties { pub fn add(&self, rhs: &Self) -> Self { match (self, rhs) { - (Self::Singleton, _) => *rhs, - (_, Self::Singleton) => *self, + (Self::Singleton, _) => rhs.clone(), + (_, Self::Singleton) => self.clone(), (Self::Ordered(lhs), Self::Ordered(rhs)) - if lhs.descending == rhs.descending => + if lhs.ordering == rhs.ordering && lhs.descending == rhs.descending => { - Self::Ordered(SortOptions { - descending: lhs.descending, - nulls_first: lhs.nulls_first || rhs.nulls_first, - }) + Self::Ordered( + lhs.clone() + .with_nulls_first(lhs.nulls_first || rhs.nulls_first), + ) } _ => Self::Unordered, } @@ -64,18 +64,17 @@ impl SortProperties { pub fn sub(&self, rhs: &Self) -> Self { match (self, rhs) { (Self::Singleton, Self::Singleton) => Self::Singleton, - (Self::Singleton, Self::Ordered(rhs)) => Self::Ordered(SortOptions { - descending: !rhs.descending, - nulls_first: rhs.nulls_first, - }), - (_, Self::Singleton) => *self, + (Self::Singleton, Self::Ordered(rhs)) => { + Self::Ordered(rhs.clone().with_descending(!rhs.descending)) + } + (_, Self::Singleton) => self.clone(), (Self::Ordered(lhs), Self::Ordered(rhs)) - if lhs.descending != rhs.descending => + if lhs.ordering == rhs.ordering && lhs.descending != rhs.descending => { - Self::Ordered(SortOptions { - descending: lhs.descending, - nulls_first: lhs.nulls_first || rhs.nulls_first, - }) + Self::Ordered( + lhs.clone() + .with_nulls_first(lhs.nulls_first || rhs.nulls_first), + ) } _ => Self::Unordered, } @@ -83,15 +82,14 @@ impl SortProperties { pub fn gt_or_gteq(&self, rhs: &Self) -> Self { match (self, rhs) { - (Self::Singleton, Self::Ordered(rhs)) => Self::Ordered(SortOptions { - descending: !rhs.descending, - nulls_first: rhs.nulls_first, - }), - (_, Self::Singleton) => *self, + (Self::Singleton, Self::Ordered(rhs)) => { + Self::Ordered(rhs.clone().with_descending(!rhs.descending)) + } + (_, Self::Singleton) => self.clone(), (Self::Ordered(lhs), Self::Ordered(rhs)) - if lhs.descending != rhs.descending => + if lhs.ordering == rhs.ordering && lhs.descending != rhs.descending => { - *self + self.clone() } _ => Self::Unordered, } @@ -100,18 +98,15 @@ impl SortProperties { pub fn and_or(&self, rhs: &Self) -> Self { match (self, rhs) { (Self::Ordered(lhs), Self::Ordered(rhs)) - if lhs.descending == rhs.descending => + if lhs.ordering == rhs.ordering && lhs.descending == rhs.descending => { - Self::Ordered(SortOptions { - descending: lhs.descending, - nulls_first: lhs.nulls_first || rhs.nulls_first, - }) + Self::Ordered( + lhs.clone() + .with_nulls_first(lhs.nulls_first || rhs.nulls_first), + ) } (Self::Ordered(opt), Self::Singleton) - | (Self::Singleton, Self::Ordered(opt)) => Self::Ordered(SortOptions { - descending: opt.descending, - nulls_first: opt.nulls_first, - }), + | (Self::Singleton, Self::Ordered(opt)) => Self::Ordered(opt.clone()), (Self::Singleton, Self::Singleton) => Self::Singleton, _ => Self::Unordered, } @@ -121,11 +116,14 @@ impl SortProperties { impl Neg for SortProperties { type Output = Self; - fn neg(mut self) -> Self::Output { - if let SortProperties::Ordered(SortOptions { descending, .. }) = &mut self { - *descending = !*descending; + fn neg(self) -> Self::Output { + match self { + SortProperties::Ordered(sort_definition) => { + SortProperties::Ordered(sort_definition.with_reversed_order()) + } + SortProperties::Unordered => self, + SortProperties::Singleton => self, } - self } } diff --git a/datafusion/expr/src/registry.rs b/datafusion/expr/src/registry.rs index 4eb49710bcf8..ca23b59b7189 100644 --- a/datafusion/expr/src/registry.rs +++ b/datafusion/expr/src/registry.rs @@ -15,13 +15,16 @@ // specific language governing permissions and limitations // under the License. -//! FunctionRegistry trait +//! [FunctionRegistry] and [ExtensionTypeRegistry] traits use crate::expr_rewriter::FunctionRewrite; use crate::planner::ExprPlanner; use crate::{AggregateUDF, ScalarUDF, UserDefinedLogicalNode, WindowUDF}; -use datafusion_common::{not_impl_err, plan_datafusion_err, HashMap, Result}; -use std::collections::HashSet; +use datafusion_common::types::{LogicalTypeRef, TypeSignature}; +use datafusion_common::{ + internal_err, not_impl_err, plan_datafusion_err, plan_err, Result, +}; +use std::collections::{HashMap, HashSet}; use std::fmt::Debug; use std::sync::Arc; @@ -201,3 +204,105 @@ impl FunctionRegistry for MemoryFunctionRegistry { vec![] } } + +pub trait ExtensionTypeRegistry { + /// Returns a reference to the logical type named `name`. + fn get_extension_type(&self, name: &str) -> Result; + + /// Registers a new [LogicalTypeRef], returning any previously registered implementation. + /// + /// Returns an error if the type cannot be registered, for example if the registry is read only. + fn register_extension_type( + &mut self, + logical_type: LogicalTypeRef, + ) -> Result>; + + /// Deregisters a logical type with the name `name`, returning the implementation that was + /// deregistered. + /// + /// Returns an error if the type cannot be deregistered, for example if the registry is read + /// only. + fn deregister_extension_type(&mut self, name: &str) + -> Result>; +} + +/// An [`ExtensionTypeRegistry`] that uses in memory [`HashMap`]s. +#[derive(Clone, Default, Debug)] +pub struct MemoryExtensionTypeRegistry { + /// Holds a mapping between the name of an extension type and its logical type. + extension_types: HashMap, +} + +impl MemoryExtensionTypeRegistry { + /// Creates an empty [MemoryExtensionTypeRegistry]. + pub fn new() -> Self { + Self::default() + } +} + +impl ExtensionTypeRegistry for MemoryExtensionTypeRegistry { + fn get_extension_type(&self, name: &str) -> Result { + self.extension_types + .get(name) + .ok_or_else(|| plan_datafusion_err!("Extension type not found.")) + .cloned() + } + + fn register_extension_type( + &mut self, + logical_type: LogicalTypeRef, + ) -> Result> { + let signature = match logical_type.signature() { + TypeSignature::Native(_) => { + return internal_err!("Cannot register a native type") + } + TypeSignature::Extension { name, .. } => name, + }; + Ok(self.extension_types.insert(signature.into(), logical_type)) + } + + fn deregister_extension_type( + &mut self, + name: &str, + ) -> Result> { + Ok(self.extension_types.remove(name)) + } +} + +impl From> for MemoryExtensionTypeRegistry { + fn from(value: HashMap) -> Self { + Self { + extension_types: value, + } + } +} + +/// Represents an [ExtensionTypeRegistry] with no registered extension types. +pub struct EmptyExtensionTypeRegistry; + +impl EmptyExtensionTypeRegistry { + /// Creates a new [EmptyExtensionTypeRegistry]. + pub fn new() -> Self { + Self {} + } +} + +impl ExtensionTypeRegistry for EmptyExtensionTypeRegistry { + fn get_extension_type(&self, _name: &str) -> Result { + plan_err!("Extension type not found.") + } + + fn register_extension_type( + &mut self, + _logical_type: LogicalTypeRef, + ) -> Result> { + plan_err!("Cannot register type.") + } + + fn deregister_extension_type( + &mut self, + _name: &str, + ) -> Result> { + plan_err!("Cannot deregister type.") + } +} diff --git a/datafusion/expr/src/udf.rs b/datafusion/expr/src/udf.rs index 8215b671a379..35aef98ea8c7 100644 --- a/datafusion/expr/src/udf.rs +++ b/datafusion/expr/src/udf.rs @@ -767,7 +767,7 @@ pub trait ScalarUDFImpl: Debug + Send + Sync { .skip(1) .all(|input| &input.sort_properties == first_order) { - Ok(*first_order) + Ok(first_order.clone()) } else { Ok(SortProperties::Unordered) } diff --git a/datafusion/expr/src/udwf.rs b/datafusion/expr/src/udwf.rs index 4da63d7955f5..e62b1c827908 100644 --- a/datafusion/expr/src/udwf.rs +++ b/datafusion/expr/src/udwf.rs @@ -18,6 +18,7 @@ //! [`WindowUDF`]: User Defined Window Functions use arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field}; use std::cmp::Ordering; use std::hash::{DefaultHasher, Hash, Hasher}; use std::{ @@ -26,8 +27,6 @@ use std::{ sync::Arc, }; -use arrow::datatypes::{DataType, Field}; - use crate::expr::WindowFunction; use crate::{ function::WindowFunctionSimplification, Expr, PartitionEvaluator, Signature, diff --git a/datafusion/functions-aggregate-common/src/utils.rs b/datafusion/functions-aggregate-common/src/utils.rs index 083dac615b5d..0a5043bcf899 100644 --- a/datafusion/functions-aggregate-common/src/utils.rs +++ b/datafusion/functions-aggregate-common/src/utils.rs @@ -108,8 +108,11 @@ pub fn ordering_fields( } /// Selects the sort option attribute from all the given `PhysicalSortExpr`s. -pub fn get_sort_options(ordering_req: &LexOrdering) -> Vec { - ordering_req.iter().map(|item| item.options).collect() +pub fn get_sort_options(ordering_req: &LexOrdering) -> Result> { + ordering_req + .iter() + .map(|item| item.options.to_arrow()) + .collect() } /// A wrapper around a type to provide hash for floats diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 0f12ac34bfd2..6930eeea0c23 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -504,8 +504,8 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { let sort_options = self .ordering_req .iter() - .map(|sort_expr| sort_expr.options) - .collect::>(); + .map(|sort_expr| sort_expr.options.to_arrow()) + .collect::>>()?; (self.values, self.ordering_values) = merge_ordered_arrays( &mut partition_values, diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 6df8ede4fc77..a5f89528ee2a 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -254,11 +254,13 @@ impl FirstValueAccumulator { let sort_columns = ordering_values .iter() .zip(self.ordering_req.iter()) - .map(|(values, req)| SortColumn { - values: Arc::clone(values), - options: Some(req.options), + .map(|(values, req)| { + Ok(SortColumn { + values: Arc::clone(values), + options: Some(req.options.to_arrow()?), + }) }) - .collect::>(); + .collect::>>()?; let comparator = LexicographicalComparator::try_new(&sort_columns)?; @@ -295,7 +297,7 @@ impl Accumulator for FirstValueAccumulator { if compare_rows( &self.orderings, orderings, - &get_sort_options(self.ordering_req.as_ref()), + &get_sort_options(self.ordering_req.as_ref())?, )? .is_gt() { @@ -317,7 +319,7 @@ impl Accumulator for FirstValueAccumulator { let sort_columns = convert_to_sort_cols( &filtered_states[1..is_set_idx], self.ordering_req.as_ref(), - ); + )?; let comparator = LexicographicalComparator::try_new(&sort_columns)?; let min = (0..filtered_states[0].len()).min_by(|&a, &b| comparator.compare(a, b)); @@ -326,7 +328,7 @@ impl Accumulator for FirstValueAccumulator { let first_row = get_row_at_idx(&filtered_states, first_idx)?; // When collecting orderings, we exclude the is_set flag from the state. let first_ordering = &first_row[1..is_set_idx]; - let sort_options = get_sort_options(self.ordering_req.as_ref()); + let sort_options = get_sort_options(self.ordering_req.as_ref())?; // Either there is no existing value, or there is an earlier version in new data. if !self.is_set || compare_rows(&self.orderings, first_ordering, &sort_options)?.is_gt() @@ -552,14 +554,8 @@ impl LastValueAccumulator { return Ok((!value.is_empty()).then_some(value.len() - 1)); } } - let sort_columns = ordering_values - .iter() - .zip(self.ordering_req.iter()) - .map(|(values, req)| SortColumn { - values: Arc::clone(values), - options: Some(req.options), - }) - .collect::>(); + let sort_columns = + convert_to_sort_cols(&ordering_values, self.ordering_req.as_ref())?; let comparator = LexicographicalComparator::try_new(&sort_columns)?; let max_ind = if self.ignore_nulls { @@ -600,7 +596,7 @@ impl Accumulator for LastValueAccumulator { if compare_rows( &self.orderings, orderings, - &get_sort_options(self.ordering_req.as_ref()), + &get_sort_options(self.ordering_req.as_ref())?, )? .is_lt() { @@ -622,7 +618,7 @@ impl Accumulator for LastValueAccumulator { let sort_columns = convert_to_sort_cols( &filtered_states[1..is_set_idx], self.ordering_req.as_ref(), - ); + )?; let comparator = LexicographicalComparator::try_new(&sort_columns)?; let max = (0..filtered_states[0].len()).max_by(|&a, &b| comparator.compare(a, b)); @@ -631,7 +627,7 @@ impl Accumulator for LastValueAccumulator { let last_row = get_row_at_idx(&filtered_states, last_idx)?; // When collecting orderings, we exclude the is_set flag from the state. let last_ordering = &last_row[1..is_set_idx]; - let sort_options = get_sort_options(self.ordering_req.as_ref()); + let sort_options = get_sort_options(self.ordering_req.as_ref())?; // Either there is no existing value, or there is a newer (latest) // version in the new data: if !self.is_set @@ -672,14 +668,18 @@ 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: &LexOrdering) -> Vec { - arrs.iter() - .zip(sort_exprs.iter()) - .map(|(item, sort_expr)| SortColumn { +fn convert_to_sort_cols( + arrs: &[ArrayRef], + sort_exprs: &LexOrdering, +) -> Result> { + Ok(arrs + .iter() + .zip(get_sort_options(sort_exprs)?) + .map(|(item, options)| SortColumn { values: Arc::clone(item), - options: Some(sort_expr.options), + options: Some(options), }) - .collect::>() + .collect::>()) } #[cfg(test)] diff --git a/datafusion/functions-aggregate/src/nth_value.rs b/datafusion/functions-aggregate/src/nth_value.rs index d84bd02a6baf..e4f0a9db5525 100644 --- a/datafusion/functions-aggregate/src/nth_value.rs +++ b/datafusion/functions-aggregate/src/nth_value.rs @@ -35,7 +35,7 @@ use datafusion_expr::{ Signature, SortExpr, Volatility, }; use datafusion_functions_aggregate_common::merge_arrays::merge_ordered_arrays; -use datafusion_functions_aggregate_common::utils::ordering_fields; +use datafusion_functions_aggregate_common::utils::{get_sort_options, ordering_fields}; use datafusion_macros::user_doc; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -330,11 +330,7 @@ impl Accumulator for NthValueAccumulator { partition_ordering_values.push(ordering_values.into()); } - let sort_options = self - .ordering_req - .iter() - .map(|sort_expr| sort_expr.options) - .collect::>(); + let sort_options = get_sort_options(&self.ordering_req)?; let (new_values, new_orderings) = merge_ordered_arrays( &mut partition_values, &mut partition_ordering_values, diff --git a/datafusion/functions/src/datetime/date_bin.rs b/datafusion/functions/src/datetime/date_bin.rs index 5ffae46dde48..3bcd13936a2e 100644 --- a/datafusion/functions/src/datetime/date_bin.rs +++ b/datafusion/functions/src/datetime/date_bin.rs @@ -217,7 +217,7 @@ impl ScalarUDFImpl for DateBinFunc { .map(|r| r.sort_properties.eq(&SortProperties::Singleton)) .unwrap_or(true) { - Ok(date_value.sort_properties) + Ok(date_value.sort_properties.clone()) } else { Ok(SortProperties::Unordered) } diff --git a/datafusion/functions/src/datetime/date_trunc.rs b/datafusion/functions/src/datetime/date_trunc.rs index ed3eb228bf03..26a2c0b60422 100644 --- a/datafusion/functions/src/datetime/date_trunc.rs +++ b/datafusion/functions/src/datetime/date_trunc.rs @@ -273,7 +273,7 @@ impl ScalarUDFImpl for DateTruncFunc { let date_value = &input[1]; if precision.sort_properties.eq(&SortProperties::Singleton) { - Ok(date_value.sort_properties) + Ok(date_value.sort_properties.clone()) } else { Ok(SortProperties::Unordered) } diff --git a/datafusion/functions/src/math/abs.rs b/datafusion/functions/src/math/abs.rs index 0c686a59016a..4c18986dde7a 100644 --- a/datafusion/functions/src/math/abs.rs +++ b/datafusion/functions/src/math/abs.rs @@ -186,9 +186,9 @@ impl ScalarUDFImpl for AbsFunc { let zero_point = Interval::make_zero(&range.lower().data_type())?; if range.gt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else if range.lt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(-arg.sort_properties) + Ok(-arg.sort_properties.clone()) } else { Ok(SortProperties::Unordered) } diff --git a/datafusion/functions/src/math/log.rs b/datafusion/functions/src/math/log.rs index fd135f4c5ec0..7a7187b3dc2f 100644 --- a/datafusion/functions/src/math/log.rs +++ b/datafusion/functions/src/math/log.rs @@ -95,23 +95,27 @@ impl ScalarUDFImpl for LogFunc { fn output_ordering(&self, input: &[ExprProperties]) -> Result { let (base_sort_properties, num_sort_properties) = if input.len() == 1 { // log(x) defaults to log(10, x) - (SortProperties::Singleton, input[0].sort_properties) + (SortProperties::Singleton, input[0].sort_properties.clone()) } else { - (input[0].sort_properties, input[1].sort_properties) + ( + input[0].sort_properties.clone(), + input[1].sort_properties.clone(), + ) }; - match (num_sort_properties, base_sort_properties) { + match (&num_sort_properties, &base_sort_properties) { (first @ SortProperties::Ordered(num), SortProperties::Ordered(base)) - if num.descending != base.descending + if num.ordering == base.ordering + && num.descending != base.descending && num.nulls_first == base.nulls_first => { - Ok(first) + Ok(first.clone()) } ( first @ (SortProperties::Ordered(_) | SortProperties::Singleton), SortProperties::Singleton, - ) => Ok(first), + ) => Ok(first.clone()), (SortProperties::Singleton, second @ SortProperties::Ordered(_)) => { - Ok(-second) + Ok(-second.clone()) } _ => Ok(SortProperties::Unordered), } @@ -255,8 +259,9 @@ mod tests { use super::*; use arrow::array::{Float32Array, Float64Array, Int64Array}; - use arrow::compute::SortOptions; use datafusion_common::cast::{as_float32_array, as_float64_array}; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use datafusion_common::DFSchema; use datafusion_expr::execution_props::ExecutionProps; use datafusion_expr::simplify::SimplifyContext; @@ -581,13 +586,15 @@ mod tests { let orders = vec![ ExprProperties::new_unknown(), ExprProperties::new_unknown().with_order(SortProperties::Ordered( - SortOptions { + AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, )), ExprProperties::new_unknown().with_order(SortProperties::Ordered( - SortOptions { + AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }, @@ -624,38 +631,44 @@ mod tests { // base: Ascending, num: Ascending SortProperties::Unordered, // base: Ascending, num: Descending - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }), // base: Ascending, num: Literal - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }), // base: Descending, num: Unordered SortProperties::Unordered, // base: Descending, num: Ascending - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }), // base: Descending, num: Descending SortProperties::Unordered, // base: Descending, num: Literal - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }), // base: Literal, num: Unordered SortProperties::Unordered, // base: Literal, num: Ascending - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }), // base: Literal, num: Descending - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }), @@ -666,13 +679,15 @@ mod tests { // Test with different `nulls_first` let base_order = ExprProperties::new_unknown().with_order( - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }), ); let num_order = ExprProperties::new_unknown().with_order( - SortProperties::Ordered(SortOptions { + SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), diff --git a/datafusion/functions/src/math/monotonicity.rs b/datafusion/functions/src/math/monotonicity.rs index baa3147f6258..ecbd56fe7a3a 100644 --- a/datafusion/functions/src/math/monotonicity.rs +++ b/datafusion/functions/src/math/monotonicity.rs @@ -32,7 +32,7 @@ pub fn acos_order(input: &[ExprProperties]) -> Result { Interval::make_symmetric_unit_interval(&range.lower().data_type())?; if valid_domain.contains(range)? == Interval::CERTAINLY_TRUE { - Ok(-arg.sort_properties) + Ok(-arg.sort_properties.clone()) } else { exec_err!("Input range of ACOS contains out-of-domain values") } @@ -63,7 +63,7 @@ pub fn acosh_order(input: &[ExprProperties]) -> Result { )?; if valid_domain.contains(range)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else { exec_err!("Input range of ACOSH contains out-of-domain values") } @@ -92,7 +92,7 @@ pub fn asin_order(input: &[ExprProperties]) -> Result { Interval::make_symmetric_unit_interval(&range.lower().data_type())?; if valid_domain.contains(range)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else { exec_err!("Input range of ASIN contains out-of-domain values") } @@ -114,7 +114,7 @@ pub fn get_asin_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn asinh_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_ASINH: LazyLock = LazyLock::new(|| { @@ -133,7 +133,7 @@ pub fn get_asinh_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn atan_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_ATAN: LazyLock = LazyLock::new(|| { @@ -159,7 +159,7 @@ pub fn atanh_order(input: &[ExprProperties]) -> Result { Interval::make_symmetric_unit_interval(&range.lower().data_type())?; if valid_domain.contains(range)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else { exec_err!("Input range of ATANH contains out-of-domain values") } @@ -210,7 +210,7 @@ pub fn get_atan2_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn cbrt_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_CBRT: LazyLock = LazyLock::new(|| { @@ -229,7 +229,7 @@ pub fn get_cbrt_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn ceil_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_CEIL: LazyLock = LazyLock::new(|| { @@ -275,9 +275,9 @@ pub fn cosh_order(input: &[ExprProperties]) -> Result { let zero_point = Interval::make_zero(&range.lower().data_type())?; if range.gt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else if range.lt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(-arg.sort_properties) + Ok(-arg.sort_properties.clone()) } else { Ok(SortProperties::Unordered) } @@ -299,7 +299,7 @@ pub fn get_cosh_doc() -> &'static Documentation { /// Non-decreasing function that converts radians to degrees. pub fn degrees_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_DEGREES: LazyLock = LazyLock::new(|| { @@ -318,7 +318,7 @@ pub fn get_degrees_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn exp_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_EXP: LazyLock = LazyLock::new(|| { @@ -337,7 +337,7 @@ pub fn get_exp_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn floor_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_FLOOR: LazyLock = LazyLock::new(|| { @@ -362,7 +362,7 @@ pub fn ln_order(input: &[ExprProperties]) -> Result { let zero_point = Interval::make_zero(&range.lower().data_type())?; if range.gt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else { exec_err!("Input range of LN contains out-of-domain values") } @@ -390,7 +390,7 @@ pub fn log2_order(input: &[ExprProperties]) -> Result { let zero_point = Interval::make_zero(&range.lower().data_type())?; if range.gt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else { exec_err!("Input range of LOG2 contains out-of-domain values") } @@ -418,7 +418,7 @@ pub fn log10_order(input: &[ExprProperties]) -> Result { let zero_point = Interval::make_zero(&range.lower().data_type())?; if range.gt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else { exec_err!("Input range of LOG10 contains out-of-domain values") } @@ -440,7 +440,7 @@ pub fn get_log10_doc() -> &'static Documentation { /// Non-decreasing for all real numbers x. pub fn radians_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_RADIONS: LazyLock = LazyLock::new(|| { @@ -480,7 +480,7 @@ pub fn get_sin_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn sinh_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_SINH: LazyLock = LazyLock::new(|| { @@ -505,7 +505,7 @@ pub fn sqrt_order(input: &[ExprProperties]) -> Result { let zero_point = Interval::make_zero(&range.lower().data_type())?; if range.gt_eq(&zero_point)? == Interval::CERTAINLY_TRUE { - Ok(arg.sort_properties) + Ok(arg.sort_properties.clone()) } else { exec_err!("Input range of SQRT contains out-of-domain values") } @@ -548,7 +548,7 @@ pub fn get_tan_doc() -> &'static Documentation { /// Non-decreasing for all real numbers. pub fn tanh_order(input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } static DOCUMENTATION_TANH: LazyLock = LazyLock::new(|| { @@ -567,10 +567,10 @@ pub fn get_tanh_doc() -> &'static Documentation { #[cfg(test)] mod tests { - use arrow::compute::SortOptions; - use datafusion_common::Result; - use super::*; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; + use datafusion_common::Result; #[derive(Debug)] struct MonotonicityTestCase { @@ -602,11 +602,13 @@ mod tests { func: acos_order, lower: -0.5, upper: 0.5, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: false, })), @@ -616,7 +618,8 @@ mod tests { func: acos_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), @@ -627,11 +630,13 @@ mod tests { func: acosh_order, lower: 2.0, upper: 100.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, })), @@ -641,7 +646,8 @@ mod tests { func: acosh_order, lower: 0.5, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: false, }), @@ -652,11 +658,13 @@ mod tests { func: asin_order, lower: -0.5, upper: 0.5, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -666,7 +674,8 @@ mod tests { func: asin_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), @@ -677,11 +686,13 @@ mod tests { func: asinh_order, lower: -1.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -691,11 +702,13 @@ mod tests { func: asinh_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -705,11 +718,13 @@ mod tests { func: atan_order, lower: -1.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -719,11 +734,13 @@ mod tests { func: atan_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -733,11 +750,13 @@ mod tests { func: atanh_order, lower: -0.6, upper: 0.6, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -747,7 +766,8 @@ mod tests { func: atanh_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), @@ -758,11 +778,13 @@ mod tests { func: cbrt_order, lower: -1.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -772,11 +794,13 @@ mod tests { func: cbrt_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -786,11 +810,13 @@ mod tests { func: ceil_order, lower: -1.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -800,11 +826,13 @@ mod tests { func: ceil_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -814,7 +842,8 @@ mod tests { func: cos_order, lower: 0.0, upper: 2.0 * std::f64::consts::PI, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), @@ -825,7 +854,8 @@ mod tests { func: cos_order, lower: -2.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), @@ -836,11 +866,13 @@ mod tests { func: cosh_order, lower: 5.0, upper: 100.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -850,11 +882,13 @@ mod tests { func: cosh_order, lower: -100.0, upper: -5.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: false, })), @@ -864,7 +898,8 @@ mod tests { func: cosh_order, lower: -1.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), @@ -875,11 +910,13 @@ mod tests { func: degrees_order, lower: -1.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, })), @@ -889,11 +926,13 @@ mod tests { func: exp_order, lower: -1000.0, upper: 1000.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -903,11 +942,13 @@ mod tests { func: floor_order, lower: -1.0, upper: 1.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, })), @@ -917,11 +958,13 @@ mod tests { func: ln_order, lower: 1.0, upper: 2.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), - expected: Ok(SortProperties::Ordered(SortOptions { + expected: Ok(SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, })), @@ -931,7 +974,8 @@ mod tests { func: ln_order, lower: -5.0, upper: -4.0, - input_sort: SortProperties::Ordered(SortOptions { + input_sort: SortProperties::Ordered(AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }), diff --git a/datafusion/functions/src/math/round.rs b/datafusion/functions/src/math/round.rs index fc87b7e63a62..9a5c36692915 100644 --- a/datafusion/functions/src/math/round.rs +++ b/datafusion/functions/src/math/round.rs @@ -104,7 +104,7 @@ impl ScalarUDFImpl for RoundFunc { .map(|r| r.sort_properties.eq(&SortProperties::Singleton)) .unwrap_or(true) { - Ok(value.sort_properties) + Ok(value.sort_properties.clone()) } else { Ok(SortProperties::Unordered) } diff --git a/datafusion/functions/src/math/signum.rs b/datafusion/functions/src/math/signum.rs index ba5422afa768..19780758082d 100644 --- a/datafusion/functions/src/math/signum.rs +++ b/datafusion/functions/src/math/signum.rs @@ -86,7 +86,7 @@ impl ScalarUDFImpl for SignumFunc { fn output_ordering(&self, input: &[ExprProperties]) -> Result { // Non-decreasing for all real numbers x. - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { diff --git a/datafusion/functions/src/math/trunc.rs b/datafusion/functions/src/math/trunc.rs index 2ac291204a0b..6f5ddf4cd41f 100644 --- a/datafusion/functions/src/math/trunc.rs +++ b/datafusion/functions/src/math/trunc.rs @@ -113,7 +113,7 @@ impl ScalarUDFImpl for TruncFunc { .map(|r| r.sort_properties.eq(&SortProperties::Singleton)) .unwrap_or(true) { - Ok(value.sort_properties) + Ok(value.sort_properties.clone()) } else { Ok(SortProperties::Unordered) } diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 38b820edc544..3d95a9cf3306 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -18,6 +18,13 @@ //! Sort expressions use crate::physical_expr::PhysicalExpr; +use arrow::array::RecordBatch; +use arrow::compute::SortColumn; +use arrow::datatypes::Schema; +use datafusion_common::sort::{AdvSortColumn, AdvSortOptions}; +use datafusion_common::Result; +use datafusion_expr_common::columnar_value::ColumnarValue; +use itertools::Itertools; use std::fmt; use std::fmt::{Display, Formatter}; use std::hash::{Hash, Hasher}; @@ -25,13 +32,6 @@ use std::ops::{Deref, Index, Range, RangeFrom, RangeTo}; use std::sync::{Arc, LazyLock}; use std::vec::IntoIter; -use arrow::compute::kernels::sort::{SortColumn, SortOptions}; -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 /// /// Example: @@ -79,41 +79,41 @@ pub struct PhysicalSortExpr { /// Physical expression representing the column to sort pub expr: Arc, /// Option to specify how the given column should be sorted - pub options: SortOptions, + pub options: AdvSortOptions, } impl PhysicalSortExpr { /// Create a new PhysicalSortExpr - pub fn new(expr: Arc, options: SortOptions) -> Self { + pub fn new(expr: Arc, options: AdvSortOptions) -> Self { Self { expr, options } } - /// Create a new PhysicalSortExpr with default [`SortOptions`] + /// Create a new PhysicalSortExpr with default [`AdvSortOptions`] pub fn new_default(expr: Arc) -> Self { - Self::new(expr, SortOptions::default()) + Self::new(expr, AdvSortOptions::default()) } /// Set the sort sort options to ASC pub fn asc(mut self) -> Self { - self.options.descending = false; + self.options = self.options.with_descending(false); self } /// Set the sort sort options to DESC pub fn desc(mut self) -> Self { - self.options.descending = true; + self.options = self.options.with_descending(true); self } /// Set the sort sort options to NULLS FIRST pub fn nulls_first(mut self) -> Self { - self.options.nulls_first = true; + self.options = self.options.with_nulls_first(true); self } /// Set the sort sort options to NULLS LAST pub fn nulls_last(mut self) -> Self { - self.options.nulls_first = false; + self.options = self.options.with_nulls_first(false); self } } @@ -149,14 +149,23 @@ impl Display for PhysicalSortExpr { impl PhysicalSortExpr { /// evaluate the sort expression into SortColumn that can be passed into arrow sort kernel pub fn evaluate_to_sort_column(&self, batch: &RecordBatch) -> Result { + self.evaluate_to_adv_sort_column(batch)?.to_arrow() + } + + /// Evaluate the sort expression into an [AdvSortColumn] that can be passed to the DataFusion + /// sorting algorithms that supporting user-defined sorting. + pub fn evaluate_to_adv_sort_column( + &self, + batch: &RecordBatch, + ) -> Result { let value_to_sort = self.expr.evaluate(batch)?; let array_to_sort = match value_to_sort { ColumnarValue::Array(array) => array, ColumnarValue::Scalar(scalar) => scalar.to_array_of_size(batch.num_rows())?, }; - Ok(SortColumn { + Ok(AdvSortColumn { values: array_to_sort, - options: Some(self.options), + options: Some(self.options.clone()), }) } @@ -172,21 +181,25 @@ impl PhysicalSortExpr { let nullable = self.expr.nullable(schema).unwrap_or(true); self.expr.eq(&requirement.expr) && if nullable { - requirement.options.is_none_or(|opts| self.options == opts) - } else { requirement .options - .is_none_or(|opts| self.options.descending == opts.descending) + .as_ref() + .is_none_or(|opts| &self.options == opts) + } else { + requirement.options.as_ref().is_none_or(|opts| { + self.options.ordering == opts.ordering + && self.options.descending == opts.descending + }) } } } /// Represents sort requirement associated with a plan /// -/// If the requirement includes [`SortOptions`] then both the +/// If the requirement includes [`AdvSortOptions`] then both the /// expression *and* the sort options must match. /// -/// If the requirement does not include [`SortOptions`]) then only the +/// If the requirement does not include [`AdvSortOptions`]) then only the /// expressions must match. /// /// # Examples @@ -206,7 +219,7 @@ pub struct PhysicalSortRequirement { pub expr: Arc, /// Option to specify how the given column should be sorted. /// If unspecified, there are no constraints on sort options. - pub options: Option, + pub options: Option, } impl From for PhysicalSortExpr { @@ -215,17 +228,14 @@ impl From for PhysicalSortExpr { /// The default is picked to be consistent with /// PostgreSQL: fn from(value: PhysicalSortRequirement) -> Self { - let options = value.options.unwrap_or(SortOptions { - descending: false, - nulls_first: false, - }); - PhysicalSortExpr::new(value.expr, options) + let sort_definition = value.options.unwrap_or_default(); + PhysicalSortExpr::new(value.expr, sort_definition) } } impl From for PhysicalSortRequirement { fn from(value: PhysicalSortExpr) -> Self { - PhysicalSortRequirement::new(value.expr, Some(value.options)) + PhysicalSortRequirement::new(value.expr, Some(value.options.clone())) } } @@ -276,7 +286,7 @@ impl PhysicalSortRequirement { /// which must match only `expr`. /// /// See [`PhysicalSortRequirement`] for examples. - pub fn new(expr: Arc, options: Option) -> Self { + pub fn new(expr: Arc, options: Option) -> Self { Self { expr, options } } @@ -291,7 +301,8 @@ impl PhysicalSortRequirement { self.expr.eq(&other.expr) && other .options - .is_none_or(|other_opts| self.options == Some(other_opts)) + .as_ref() + .is_none_or(|other_opts| self.options.as_ref() == Some(other_opts)) } #[deprecated(since = "43.0.0", note = "use LexRequirement::from_lex_ordering")] @@ -310,9 +321,9 @@ impl PhysicalSortRequirement { } } -/// Returns the SQL string representation of the given [SortOptions] object. +/// Returns the SQL string representation of the given [AdvSortOptions] object. #[inline] -fn to_str(options: &SortOptions) -> &str { +fn to_str(options: &AdvSortOptions) -> &str { match (options.descending, options.nulls_first) { (true, true) => "DESC", (true, false) => "DESC NULLS LAST", @@ -438,7 +449,7 @@ impl LexOrdering { /// Collapse a `LexOrdering` into a new duplicate-free `LexOrdering` based on expression. /// /// This function filters duplicate entries that have same physical - /// expression inside, ignoring [`SortOptions`]. For example: + /// expression inside, ignoring [`AdvSortOptions`]. For example: /// /// `vec![a ASC, a DESC]` collapses to `vec![a ASC]`. pub fn collapse(self) -> Self { diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 114007bfa6af..04a93aa65e6b 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -99,7 +99,12 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { pub fn reverse_order_bys(order_bys: &LexOrdering) -> LexOrdering { order_bys .iter() - .map(|e| PhysicalSortExpr::new(Arc::clone(&e.expr), !e.options)) + .map(|e| { + PhysicalSortExpr::new( + Arc::clone(&e.expr), + e.options.clone().with_reversed_order(), + ) + }) .collect() } diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 34c4e52d517e..e4d327fc2473 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -40,8 +40,9 @@ use std::sync::Arc; use crate::expressions::Column; -use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use datafusion_common::{internal_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::{AggregateUDF, ReversedUDAF, SetMonotonicity}; use datafusion_expr_common::accumulator::Accumulator; @@ -562,8 +563,11 @@ impl AggregateFunctionExpr { return None; } let expr = Arc::new(Column::new(self.name(), aggr_func_idx)); - let options = - SortOptions::new(monotonicity == SetMonotonicity::Decreasing, false); + let options = AdvSortOptions::new( + SortOrdering::Default, + monotonicity == SetMonotonicity::Decreasing, + false, + ); Some(PhysicalSortExpr { expr, options }) } } diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index e94d2bad5712..51296de2bf39 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -67,13 +67,13 @@ pub fn add_offset_to_expr( #[cfg(test)] mod tests { - use super::*; use crate::expressions::col; use crate::PhysicalSortExpr; - use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use datafusion_common::sort::AdvSortOptions; use datafusion_common::{plan_datafusion_err, Result}; use datafusion_physical_expr_common::sort_expr::{ LexOrdering, PhysicalSortRequirement, @@ -89,7 +89,7 @@ mod tests { let name = parts.next().expect("empty sort expression"); let mut sort_expr = PhysicalSortExpr::new( col(name, schema).expect("invalid column name"), - SortOptions::default(), + AdvSortOptions::default(), ); if let Some(options) = parts.next() { @@ -206,7 +206,10 @@ mod tests { in_data .iter() .map(|(expr, options)| { - PhysicalSortRequirement::new(Arc::clone(*expr), *options) + PhysicalSortRequirement::new( + Arc::clone(*expr), + options.map(|opt| AdvSortOptions::with_default_ordering(opt)), + ) }) .collect() } @@ -219,7 +222,7 @@ mod tests { .iter() .map(|(expr, options)| PhysicalSortExpr { expr: Arc::clone(*expr), - options: *options, + options: AdvSortOptions::with_default_ordering(*options), }) .collect() } @@ -243,7 +246,7 @@ mod tests { .iter() .map(|(expr, options)| PhysicalSortExpr { expr: Arc::clone(expr), - options: *options, + options: AdvSortOptions::with_default_ordering(*options), }) .collect(), ) diff --git a/datafusion/physical-expr/src/equivalence/ordering.rs b/datafusion/physical-expr/src/equivalence/ordering.rs index 0efd46ad912e..837abfd5ae6c 100644 --- a/datafusion/physical-expr/src/equivalence/ordering.rs +++ b/datafusion/physical-expr/src/equivalence/ordering.rs @@ -23,7 +23,7 @@ use std::vec::IntoIter; use crate::equivalence::add_offset_to_expr; use crate::{LexOrdering, PhysicalExpr}; -use arrow::compute::SortOptions; +use datafusion_common::sort::AdvSortOptions; use datafusion_common::HashSet; /// An `OrderingEquivalenceClass` object keeps track of different alternative @@ -227,11 +227,11 @@ impl OrderingEquivalenceClass { /// Gets sort options associated with this expression if it is a leading /// ordering expression. Otherwise, returns `None`. - pub fn get_options(&self, expr: &Arc) -> Option { + pub fn get_options(&self, expr: &Arc) -> Option<&AdvSortOptions> { for ordering in self.iter() { let leading_ordering = &ordering[0]; if leading_ordering.expr.eq(expr) { - return Some(leading_ordering.options); + return Some(&leading_ordering.options); } } None @@ -359,6 +359,7 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::sort::AdvSortOptions; use datafusion_common::Result; use datafusion_expr::{Operator, ScalarUDF}; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -371,16 +372,16 @@ mod tests { ])); let crude = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let finer = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, ]); // finer ordering satisfies, crude ordering should return true diff --git a/datafusion/physical-expr/src/equivalence/properties/dependency.rs b/datafusion/physical-expr/src/equivalence/properties/dependency.rs index 9eba295e562e..5a018c62a2a0 100644 --- a/datafusion/physical-expr/src/equivalence/properties/dependency.rs +++ b/datafusion/physical-expr/src/equivalence/properties/dependency.rs @@ -424,7 +424,6 @@ pub fn generate_dependency_orderings( #[cfg(test)] mod tests { - use std::ops::Not; use std::sync::Arc; use super::*; @@ -438,6 +437,8 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use datafusion_common::{Constraint, Constraints, Result}; use datafusion_expr::sort_properties::SortProperties; use datafusion_expr::Operator; @@ -541,7 +542,7 @@ mod tests { #[test] fn test_normalize_ordering_equivalence_classes() -> Result<()> { - let sort_options = SortOptions::default(); + let sort_options = AdvSortOptions::default(); let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), @@ -557,11 +558,11 @@ mod tests { let others = vec![ LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), - options: sort_options, + options: sort_options.clone(), }]), LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_c_expr), - options: sort_options, + options: sort_options.clone(), }]), ]; eq_properties.add_new_orderings(others); @@ -570,11 +571,11 @@ mod tests { expected_eqs.add_new_orderings([ LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_b_expr), - options: sort_options, + options: sort_options.clone(), }]), LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_c_expr), - options: sort_options, + options: sort_options.clone(), }]), ]); @@ -587,8 +588,8 @@ mod tests { #[test] fn test_get_indices_of_matching_sort_exprs_with_order_eq() -> Result<()> { - let sort_options = SortOptions::default(); - let sort_options_not = SortOptions::default().not(); + let sort_options = AdvSortOptions::default(); + let sort_options_not = AdvSortOptions::default().with_reversed_order(); let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), @@ -601,11 +602,11 @@ mod tests { eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, + options: sort_options_not.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options: sort_options, + options: sort_options.clone(), }, ])]); let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); @@ -615,11 +616,11 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_b), - options: sort_options_not + options: sort_options_not.clone() }, PhysicalSortExpr { expr: Arc::clone(col_a), - options: sort_options + options: sort_options.clone() } ]) ); @@ -636,16 +637,16 @@ mod tests { eq_properties.add_new_orderings([ LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), - options: sort_options, + options: sort_options.clone(), }]), LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, + options: sort_options_not.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options: sort_options, + options: sort_options.clone(), }, ]), ]); @@ -656,11 +657,11 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_b), - options: sort_options_not + options: sort_options_not.clone() }, PhysicalSortExpr { expr: Arc::clone(col_a), - options: sort_options + options: sort_options.clone() } ]) ); @@ -680,15 +681,15 @@ mod tests { eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), - options: sort_options_not, + options: sort_options_not.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), - options: sort_options, + options: sort_options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options: sort_options, + options: sort_options.clone(), }, ])]); let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); @@ -711,7 +712,8 @@ mod tests { let col_b = &col("b", &schema)?; let col_c = &col("c", &schema)?; let col_d = &col("d", &schema)?; - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -721,11 +723,11 @@ mod tests { eq_properties.add_new_orderings(vec![ LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(col_b), - options: option_asc, + options: option_asc.clone(), }]), LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(col_d), - options: option_asc, + options: option_asc.clone(), }]), ]); @@ -737,12 +739,18 @@ mod tests { Operator::Plus, Arc::clone(col_b), )) as Arc, - SortProperties::Ordered(option_asc), + SortProperties::Ordered(option_asc.clone()), ), // b - (Arc::clone(col_b), SortProperties::Ordered(option_asc)), + ( + Arc::clone(col_b), + SortProperties::Ordered(option_asc.clone()), + ), // a - (Arc::clone(col_a), SortProperties::Ordered(option_asc)), + ( + Arc::clone(col_a), + SortProperties::Ordered(option_asc.clone()), + ), // a + c ( Arc::new(BinaryExpr::new( @@ -804,11 +812,11 @@ mod tests { eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_d), - options: option_asc, + options: AdvSortOptions::with_default_ordering(option_asc), }, PhysicalSortExpr { expr: Arc::clone(col_h), - options: option_desc, + options: AdvSortOptions::with_default_ordering(option_desc), }, ])]); let test_cases = vec![ @@ -1098,7 +1106,8 @@ mod tests { .map(|c| { col(c, schema.as_ref()).map(|expr| PhysicalSortExpr { expr, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, @@ -1191,7 +1200,7 @@ mod tests { .map(|&name| { col(name, &schema).map(|col| PhysicalSortExpr { expr: col, - options: SortOptions::default(), + options: AdvSortOptions::default(), }) }) .collect::>()?; @@ -1362,11 +1371,11 @@ mod tests { let sort_exprs = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(&col_a), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, PhysicalSortExpr { expr: Arc::clone(&col_b), - options: SortOptions::default(), + options: AdvSortOptions::default(), }, ]); @@ -1390,8 +1399,9 @@ mod tests { let col_b = col("b", &schema)?; let col_c = col("c", &schema)?; - let asc = SortOptions::default(); - let desc = SortOptions { + let asc = AdvSortOptions::default(); + let desc = AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }; @@ -1400,22 +1410,22 @@ mod tests { eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(&col_a), - options: asc, + options: asc.clone(), }, PhysicalSortExpr { expr: Arc::clone(&col_b), - options: desc, + options: desc.clone(), }, PhysicalSortExpr { expr: Arc::clone(&col_c), - options: asc, + options: asc.clone(), }, ])]); // New ordering: [a ASC] let new_order = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_a), - options: asc, + options: asc.clone(), }]); let result = eq_properties.with_reorder(new_order); @@ -1446,24 +1456,24 @@ mod tests { // Make a and b equivalent eq_properties.add_equal_conditions(&col_a, &col_b)?; - let asc = SortOptions::default(); + let asc = AdvSortOptions::default(); // Initial ordering: [a ASC, c ASC] eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(&col_a), - options: asc, + options: asc.clone(), }, PhysicalSortExpr { expr: Arc::clone(&col_c), - options: asc, + options: asc.clone(), }, ])]); // New ordering: [b ASC] let new_order = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_b), - options: asc, + options: asc.clone(), }]); let result = eq_properties.with_reorder(new_order); @@ -1490,8 +1500,9 @@ mod tests { let col_a = col("a", &schema)?; let col_b = col("b", &schema)?; - let asc = SortOptions::default(); - let desc = SortOptions { + let asc = AdvSortOptions::default(); + let desc = AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }; @@ -1500,18 +1511,18 @@ mod tests { eq_properties.add_new_orderings([LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(&col_a), - options: asc, + options: asc.clone(), }, PhysicalSortExpr { expr: Arc::clone(&col_b), - options: desc, + options: desc.clone(), }, ])]); // New ordering: [a DESC] let new_order = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_a), - options: desc, + options: desc.clone(), }]); let result = eq_properties.with_reorder(new_order.clone()); @@ -1535,7 +1546,7 @@ mod tests { let col_d = col("d", &schema)?; let col_e = col("e", &schema)?; - let asc = SortOptions::default(); + let asc = AdvSortOptions::default(); // Constants: c is constant eq_properties = eq_properties.with_constants([ConstExpr::from(&col_c)]); @@ -1548,16 +1559,16 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(&col_d), - options: asc, + options: asc.clone(), }, PhysicalSortExpr { expr: Arc::clone(&col_a), - options: asc, + options: asc.clone(), }, ]), LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::clone(&col_e), - options: asc, + options: asc.clone(), }]), ]); @@ -1565,11 +1576,11 @@ mod tests { let new_order = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(&col_b), - options: asc, + options: asc.clone(), }, PhysicalSortExpr { expr: Arc::clone(&col_c), - options: asc, + options: asc.clone(), }, ]); @@ -1697,7 +1708,7 @@ mod tests { .iter() .map(|col_name| PhysicalSortExpr { expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }) .collect(), ); @@ -1710,7 +1721,7 @@ mod tests { cols.iter() .map(|col_name| PhysicalSortExpr { expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }) .collect(), ) @@ -1724,7 +1735,7 @@ mod tests { cols.iter() .map(|col_name| PhysicalSortExpr { expr: col(col_name, schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }) .collect(), ) diff --git a/datafusion/physical-expr/src/equivalence/properties/mod.rs b/datafusion/physical-expr/src/equivalence/properties/mod.rs index 080587c0e231..32e40d9d41eb 100644 --- a/datafusion/physical-expr/src/equivalence/properties/mod.rs +++ b/datafusion/physical-expr/src/equivalence/properties/mod.rs @@ -41,7 +41,6 @@ use crate::{ PhysicalSortExpr, PhysicalSortRequirement, }; -use arrow::compute::SortOptions; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_err, Constraint, Constraints, HashMap, Result}; @@ -49,6 +48,7 @@ use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_physical_expr_common::utils::ExprPropertiesNode; +use datafusion_common::sort::AdvSortOptions; use indexmap::IndexSet; use itertools::Itertools; @@ -378,7 +378,7 @@ impl EquivalenceProperties { continue; } - let leading_ordering_options = ordering[0].options; + let leading_ordering_options = ordering[0].options.clone(); for equivalent_expr in &eq_class { let children = equivalent_expr.children(); @@ -398,7 +398,9 @@ impl EquivalenceProperties { break; } child_properties.push(ExprProperties { - sort_properties: SortProperties::Ordered(next.options), + sort_properties: SortProperties::Ordered( + next.options.clone(), + ), range: Interval::make_unbounded( &child.data_type(&self.schema)?, )?, @@ -415,9 +417,14 @@ impl EquivalenceProperties { if let Ok(expr_properties) = equivalent_expr.get_properties(&child_properties) { + let SortProperties::Ordered(expr_ordering_options) = + &expr_properties.sort_properties + else { + break; + }; + if expr_properties.preserves_lex_ordering - && SortProperties::Ordered(leading_ordering_options) - == expr_properties.sort_properties + && &leading_ordering_options == expr_ordering_options { // Assume existing ordering is [c ASC, a ASC, b ASC] // When equality c = f(a,b) is given, if we know that given ordering `[a ASC, b ASC]`, @@ -647,7 +654,8 @@ impl EquivalenceProperties { req.expr.eq(&existing.expr) && req .options - .is_none_or(|req_opts| req_opts == existing.options) + .as_ref() + .is_none_or(|req_opts| *req_opts == existing.options) }, ) }) @@ -741,14 +749,14 @@ impl EquivalenceProperties { .zip(rhs.inner.iter_mut()) .all(|(lhs, rhs)| { lhs.expr.eq(&rhs.expr) - && match (lhs.options, rhs.options) { + && match (&lhs.options, &rhs.options) { (Some(lhs_opt), Some(rhs_opt)) => lhs_opt == rhs_opt, (Some(options), None) => { - rhs.options = Some(options); + rhs.options = Some(options.clone()); true } (None, Some(options)) => { - lhs.options = Some(options); + lhs.options = Some(options.clone()); true } (None, None) => true, @@ -791,7 +799,7 @@ impl EquivalenceProperties { { res.push(PhysicalSortExpr { expr: Arc::clone(&r_expr), - options: sort_expr.options, + options: sort_expr.options.clone(), }); } } @@ -881,7 +889,7 @@ impl EquivalenceProperties { self.project_expr(&sort_expr.expr, mapping).map(|expr| { PhysicalSortExpr { expr, - options: sort_expr.options, + options: sort_expr.options.clone(), } }); let is_projected = target_sort_expr.is_some(); @@ -1157,7 +1165,7 @@ impl EquivalenceProperties { )), SortProperties::Singleton => { // Assign default ordering to constant expressions - let options = SortOptions::default(); + let options = AdvSortOptions::default(); Some(( PhysicalSortExpr { expr: Arc::clone(&exprs[idx]), @@ -1477,7 +1485,7 @@ fn update_properties( { node.data.sort_properties = SortProperties::Singleton; } else if let Some(options) = oeq_class.get_options(&normalized_expr) { - node.data.sort_properties = SortProperties::Ordered(options); + node.data.sort_properties = SortProperties::Ordered(options.clone()); } Ok(Transformed::yes(node)) } @@ -1554,7 +1562,7 @@ fn get_expr_properties( if let Some(column_order) = dependencies.iter().find(|&order| expr.eq(&order.expr)) { // If exact match is found, return its ordering. Ok(ExprProperties { - sort_properties: SortProperties::Ordered(column_order.options), + sort_properties: SortProperties::Ordered(column_order.options.clone()), range: Interval::make_unbounded(&expr.data_type(schema)?)?, preserves_lex_ordering: false, }) diff --git a/datafusion/physical-expr/src/equivalence/properties/union.rs b/datafusion/physical-expr/src/equivalence/properties/union.rs index 64ef9278e248..df0721fa51a5 100644 --- a/datafusion/physical-expr/src/equivalence/properties/union.rs +++ b/datafusion/physical-expr/src/equivalence/properties/union.rs @@ -313,7 +313,8 @@ fn advance_if_matches_constant( ) -> Option { let expr = iter.peek()?; let const_expr = constants.iter().find(|c| c.eq_expr(expr))?; - let found_expr = PhysicalSortExpr::new(Arc::clone(const_expr.expr()), expr.options); + let found_expr = + PhysicalSortExpr::new(Arc::clone(const_expr.expr()), expr.options.clone()); iter.next(); Some(found_expr) } diff --git a/datafusion/physical-expr/src/expressions/binary.rs b/datafusion/physical-expr/src/expressions/binary.rs index 1f16c5471ed7..47b4d30ba974 100644 --- a/datafusion/physical-expr/src/expressions/binary.rs +++ b/datafusion/physical-expr/src/expressions/binary.rs @@ -525,8 +525,8 @@ impl PhysicalExpr for BinaryExpr { /// For each operator, [`BinaryExpr`] has distinct rules. /// TODO: There may be rules specific to some data types and expression ranges. fn get_properties(&self, children: &[ExprProperties]) -> Result { - let (l_order, l_range) = (children[0].sort_properties, &children[0].range); - let (r_order, r_range) = (children[1].sort_properties, &children[1].range); + let (l_order, l_range) = (&children[0].sort_properties, &children[0].range); + let (r_order, r_range) = (&children[1].sort_properties, &children[1].range); match self.op() { Operator::Plus => Ok(ExprProperties { sort_properties: l_order.add(&r_order), diff --git a/datafusion/physical-expr/src/expressions/negative.rs b/datafusion/physical-expr/src/expressions/negative.rs index 8795545274a2..4d8e83172b28 100644 --- a/datafusion/physical-expr/src/expressions/negative.rs +++ b/datafusion/physical-expr/src/expressions/negative.rs @@ -162,7 +162,7 @@ impl PhysicalExpr for NegativeExpr { /// The ordering of a [`NegativeExpr`] is simply the reverse of its child. fn get_properties(&self, children: &[ExprProperties]) -> Result { Ok(ExprProperties { - sort_properties: -children[0].sort_properties, + sort_properties: -children[0].sort_properties.clone(), range: children[0].range.clone().arithmetic_negate()?, preserves_lex_ordering: false, }) diff --git a/datafusion/physical-expr/src/utils/mod.rs b/datafusion/physical-expr/src/utils/mod.rs index 7e4c7f0e10ba..a527c7fc6f96 100644 --- a/datafusion/physical-expr/src/utils/mod.rs +++ b/datafusion/physical-expr/src/utils/mod.rs @@ -308,7 +308,7 @@ pub(crate) mod tests { } fn output_ordering(&self, input: &[ExprProperties]) -> Result { - Ok(input[0].sort_properties) + Ok(input[0].sort_properties.clone()) } fn invoke_with_args(&self, args: ScalarFunctionArgs) -> Result { diff --git a/datafusion/physical-expr/src/window/sliding_aggregate.rs b/datafusion/physical-expr/src/window/sliding_aggregate.rs index 23967e78f07a..4055b70bc783 100644 --- a/datafusion/physical-expr/src/window/sliding_aggregate.rs +++ b/datafusion/physical-expr/src/window/sliding_aggregate.rs @@ -155,7 +155,7 @@ impl WindowExpr for SlidingAggregateWindowExpr { .zip(order_by_exprs) .map(|(req, new_expr)| PhysicalSortExpr { expr: new_expr, - options: req.options, + options: req.options.clone(), }) .collect::(); Some(Arc::new(SlidingAggregateWindowExpr { diff --git a/datafusion/physical-expr/src/window/standard.rs b/datafusion/physical-expr/src/window/standard.rs index 22e8aea83fe7..d3b34efa4232 100644 --- a/datafusion/physical-expr/src/window/standard.rs +++ b/datafusion/physical-expr/src/window/standard.rs @@ -29,8 +29,9 @@ use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; +use datafusion_common::types::SortOrdering; use datafusion_common::utils::evaluate_partition_ranges; -use datafusion_common::{Result, ScalarValue}; +use datafusion_common::{internal_err, Result, ScalarValue}; use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -46,18 +47,25 @@ pub struct StandardWindowExpr { impl StandardWindowExpr { /// create a new standard window function expression - pub fn new( + pub fn try_new( expr: Arc, partition_by: &[Arc], order_by: &LexOrdering, window_frame: Arc, - ) -> Self { - Self { + ) -> Result { + let has_unsupported_ordering = order_by + .iter() + .any(|expr| expr.options.ordering != SortOrdering::Default); + if has_unsupported_ordering { + return internal_err!("Unsupported ordering for window expressions."); + } + + Ok(Self { expr, partition_by: partition_by.to_vec(), order_by: order_by.clone(), window_frame, - } + }) } /// Get StandardWindowFunction expr of StandardWindowExpr @@ -112,8 +120,11 @@ impl WindowExpr for StandardWindowExpr { let mut evaluator = self.expr.create_evaluator()?; let num_rows = batch.num_rows(); if evaluator.uses_window_frame() { - let sort_options: Vec = - self.order_by.iter().map(|o| o.options).collect(); + let sort_options: Vec = self + .order_by + .iter() + .map(|o| o.options.to_arrow().expect("Checked in try_new")) + .collect(); let mut row_wise_results = vec![]; let mut values = self.evaluate_args(batch)?; @@ -157,7 +168,11 @@ impl WindowExpr for StandardWindowExpr { ) -> Result<()> { let field = self.expr.field()?; let out_type = field.data_type(); - let sort_options = self.order_by.iter().map(|o| o.options).collect::>(); + let sort_options = self + .order_by + .iter() + .map(|o| o.options.to_arrow().expect("Checked in try_new")) + .collect::>(); for (partition_row, partition_batch_state) in partition_batches.iter() { let window_state = if let Some(window_state) = window_agg_state.get_mut(partition_row) { @@ -250,12 +265,15 @@ impl WindowExpr for StandardWindowExpr { fn get_reverse_expr(&self) -> Option> { self.expr.reverse_expr().map(|reverse_expr| { - Arc::new(StandardWindowExpr::new( - reverse_expr, - &self.partition_by.clone(), - reverse_order_bys(self.order_by.as_ref()).as_ref(), - Arc::new(self.window_frame.reverse()), - )) as _ + Arc::new( + StandardWindowExpr::try_new( + reverse_expr, + &self.partition_by.clone(), + reverse_order_bys(self.order_by.as_ref()).as_ref(), + Arc::new(self.window_frame.reverse()), + ) + .expect("self has no custom sorts"), + ) as _ }) } diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 793f2e5ee586..1c181031bc09 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -190,8 +190,11 @@ pub trait AggregateWindowExpr: WindowExpr { fn aggregate_evaluate(&self, batch: &RecordBatch) -> Result { let mut accumulator = self.get_accumulator()?; let mut last_range = Range { start: 0, end: 0 }; - let sort_options: Vec = - self.order_by().iter().map(|o| o.options).collect(); + let sort_options: Vec = self + .order_by() + .iter() + .map(|o| o.options.to_arrow()) + .collect::>>()?; let mut window_frame_ctx = WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options); self.get_result_column( @@ -238,9 +241,12 @@ pub trait AggregateWindowExpr: WindowExpr { let most_recent_row = partition_batch_state.most_recent_row.as_ref(); // If there is no window state context, initialize it. + let sort_options: Vec = self + .order_by() + .iter() + .map(|o| o.options.to_arrow()) + .collect::>>()?; let window_frame_ctx = state.window_frame_ctx.get_or_insert_with(|| { - let sort_options: Vec = - self.order_by().iter().map(|o| o.options).collect(); WindowFrameContext::new(Arc::clone(self.get_window_frame()), sort_options) }); let out_col = self.get_result_column( @@ -360,7 +366,7 @@ pub(crate) fn is_end_bound_safe( &window_frame.end_bound, &order_bys[0], most_recent_order_bys.map(|items| &items[0]), - &sort_exprs[0].options, + &sort_exprs[0].options.to_arrow()?, idx, ), WindowFrameContext::Groups { @@ -371,7 +377,7 @@ pub(crate) fn is_end_bound_safe( state, &order_bys[0], most_recent_order_bys.map(|items| &items[0]), - &sort_exprs[0].options, + &sort_exprs[0].options.to_arrow()?, ), } } diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 5e76edad1f56..7c838da945c3 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -21,17 +21,16 @@ //! according to the configuration), this rule increases partition counts in //! the physical plan. -use std::fmt::Debug; -use std::sync::Arc; - use crate::optimizer::PhysicalOptimizerRule; use crate::output_requirements::OutputRequirementExec; use crate::utils::{ add_sort_above_with_check, is_coalesce_partitions, is_repartition, is_sort_preserving_merge, }; - use arrow::compute::SortOptions; +use std::fmt::Debug; +use std::sync::Arc; + use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; use datafusion_common::stats::Precision; @@ -459,7 +458,7 @@ where if !positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); let new_sort_options = (0..sort_options.len()) - .map(|idx| sort_options[positions[idx]]) + .map(|idx| sort_options[positions[idx]].clone()) .collect(); join_plan.plan = join_constructor((new_join_on, new_sort_options))?; } @@ -673,7 +672,7 @@ pub fn reorder_join_keys_to_inputs( } = join_keys; let new_join_on = new_join_conditions(&left_keys, &right_keys); let new_sort_options = (0..sort_options.len()) - .map(|idx| sort_options[positions[idx]]) + .map(|idx| sort_options[positions[idx]].clone()) .collect(); return SortMergeJoinExec::try_new( Arc::clone(left), diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 17acb6272938..9a869211404e 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -613,7 +613,10 @@ fn handle_custom_pushdown( } })? .data; - Ok(PhysicalSortRequirement::new(updated_columns, req.options)) + Ok(PhysicalSortRequirement::new( + updated_columns, + req.options.clone(), + )) }) .collect::>>()?; @@ -691,7 +694,10 @@ fn handle_hash_join( } })? .data; - Ok(PhysicalSortRequirement::new(updated_columns, req.options)) + Ok(PhysicalSortRequirement::new( + updated_columns, + req.options.clone(), + )) }) .collect::>>()?; diff --git a/datafusion/physical-optimizer/src/output_requirements.rs b/datafusion/physical-optimizer/src/output_requirements.rs index 3ca0547aa11d..c086a0a22f50 100644 --- a/datafusion/physical-optimizer/src/output_requirements.rs +++ b/datafusion/physical-optimizer/src/output_requirements.rs @@ -222,7 +222,7 @@ impl ExecutionPlan for OutputRequirementExec { }; updated_sort_reqs.push(PhysicalSortRequirement { expr: new_expr, - options: req.options, + options: req.options.clone(), }); } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7d4837d04774..d6ecb079f4f8 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1380,7 +1380,7 @@ mod tests { DictionaryArray, Float32Array, Float64Array, Int32Array, StructArray, UInt32Array, UInt64Array, }; - use arrow::compute::{concat_batches, SortOptions}; + use arrow::compute::concat_batches; use arrow::datatypes::{DataType, Int32Type}; use datafusion_common::{ assert_batches_eq, assert_batches_sorted_eq, internal_err, DataFusionError, @@ -1401,6 +1401,8 @@ mod tests { use datafusion_physical_expr::Partitioning; use datafusion_physical_expr::PhysicalSortExpr; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -2152,7 +2154,7 @@ mod tests { // FIRST_VALUE(b ORDER BY b ) fn test_first_value_agg_expr( schema: &Schema, - sort_options: SortOptions, + sort_options: AdvSortOptions, ) -> Result> { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, @@ -2171,7 +2173,7 @@ mod tests { // LAST_VALUE(b ORDER BY b ) fn test_last_value_agg_expr( schema: &Schema, - sort_options: SortOptions, + sort_options: AdvSortOptions, ) -> Result> { let ordering_req = [PhysicalSortExpr { expr: col("b", schema)?, @@ -2224,7 +2226,8 @@ mod tests { let groups = PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); - let sort_options = SortOptions { + let sort_options = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -2301,7 +2304,8 @@ mod tests { // Assume column a and b are aliases // Assume also that a ASC and c DESC describe the same global ordering for the table. (Since they are ordering equivalent). - let options1 = SortOptions { + let options1 = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -2317,30 +2321,30 @@ mod tests { None, Some(vec![PhysicalSortExpr { expr: Arc::clone(col_a), - options: options1, + options: options1.clone(), }]), Some(vec![ PhysicalSortExpr { expr: Arc::clone(col_a), - options: options1, + options: options1.clone(), }, PhysicalSortExpr { expr: Arc::clone(col_b), - options: options1, + options: options1.clone(), }, PhysicalSortExpr { expr: Arc::clone(col_c), - options: options1, + options: options1.clone(), }, ]), Some(vec![ PhysicalSortExpr { expr: Arc::clone(col_a), - options: options1, + options: options1.clone(), }, PhysicalSortExpr { expr: Arc::clone(col_b), - options: options1, + options: options1.clone(), }, ]), ]; @@ -2348,11 +2352,11 @@ mod tests { let common_requirement = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::clone(col_a), - options: options1, + options: options1.clone(), }, PhysicalSortExpr { expr: Arc::clone(col_c), - options: options1, + options: options1.clone(), }, ]); let mut aggr_exprs = order_by_exprs @@ -2388,15 +2392,16 @@ mod tests { ])); let col_a = col("a", &schema)?; - let option_desc = SortOptions { + let option_desc = AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }; let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); let aggregates: Vec> = vec![ - test_first_value_agg_expr(&schema, option_desc)?, - test_last_value_agg_expr(&schema, option_desc)?, + test_first_value_agg_expr(&schema, option_desc.clone())?, + test_last_value_agg_expr(&schema, option_desc.clone())?, ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let aggregate_exec = Arc::new(AggregateExec::try_new( diff --git a/datafusion/physical-plan/src/aggregates/order/partial.rs b/datafusion/physical-plan/src/aggregates/order/partial.rs index aff69277a4ce..4c7caf26c3cf 100644 --- a/datafusion/physical-plan/src/aggregates/order/partial.rs +++ b/datafusion/physical-plan/src/aggregates/order/partial.rs @@ -278,6 +278,7 @@ impl GroupOrderingPartial { mod tests { use arrow::array::Int32Array; use arrow_schema::{DataType, Field}; + use datafusion_common::sort::AdvSortOptions; use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; use super::*; @@ -294,7 +295,7 @@ mod tests { let ordering = LexOrdering::new(vec![PhysicalSortExpr::new( col("a", &schema)?, - SortOptions::default(), + AdvSortOptions::default(), )]); let mut group_ordering = diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 05122d5a5403..ecd2e0ca91e1 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -36,7 +36,6 @@ use crate::{aggregates, metrics, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; -use arrow::compute::SortOptions; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; @@ -50,6 +49,7 @@ use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; use super::order::GroupOrdering; use super::AggregateExec; +use datafusion_common::sort::AdvSortOptions; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::ready; @@ -521,7 +521,7 @@ impl GroupedHashAggregateStream { .enumerate() .map(|(idx, field)| PhysicalSortExpr { expr: Arc::new(Column::new(field.name().as_str(), idx)) as _, - options: SortOptions::default(), + options: AdvSortOptions::default(), }) .collect(); diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index f680de6738e5..124d007db4ed 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -1040,7 +1040,6 @@ pub(crate) mod tests { }; use arrow::array::Int32Array; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field}; use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_execution::runtime_env::RuntimeEnvBuilder; @@ -1049,6 +1048,8 @@ pub(crate) mod tests { use datafusion_physical_expr::{Partitioning, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use rstest::rstest; fn build_table( @@ -1082,7 +1083,8 @@ pub(crate) mod tests { let index = schema.index_of(name).unwrap(); let sort_expr = PhysicalSortExpr { expr: Arc::new(Column::new(name, index)), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }, diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 20c0d76a9cdb..f462bc4cba60 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -72,6 +72,8 @@ use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::PhysicalExprRef; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use futures::{Stream, StreamExt}; /// Join execution plan that executes equi-join predicates on multiple partitions using Sort-Merge @@ -188,11 +190,19 @@ impl SortMergeJoinExec { .map(|((l, r), sort_op)| { let left = PhysicalSortExpr { expr: Arc::clone(l), - options: *sort_op, + options: AdvSortOptions::new( + SortOrdering::Default, + sort_op.descending, + sort_op.nulls_first, + ), }; let right = PhysicalSortExpr { expr: Arc::clone(r), - options: *sort_op, + options: AdvSortOptions::new( + SortOrdering::Default, + sort_op.descending, + sort_op.nulls_first, + ), }; (left, right) }) diff --git a/datafusion/physical-plan/src/joins/stream_join_utils.rs b/datafusion/physical-plan/src/joins/stream_join_utils.rs index 677601a12845..34e6d69b5a3e 100644 --- a/datafusion/physical-plan/src/joins/stream_join_utils.rs +++ b/datafusion/physical-plan/src/joins/stream_join_utils.rs @@ -799,8 +799,8 @@ pub mod tests { use super::*; use crate::{joins::test_utils::complicated_filter, joins::utils::ColumnIndex}; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field}; + use datafusion_common::sort::AdvSortOptions; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::{binary, cast, col}; @@ -811,7 +811,7 @@ pub mod tests { // Sorting information for the left side: let left_child_sort_expr = PhysicalSortExpr { expr: col("left_1", &left_child_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }; let right_child_schema = Schema::new(vec![ @@ -826,7 +826,7 @@ pub mod tests { col("right_2", &right_child_schema)?, &right_child_schema, )?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }; let intermediate_schema = Schema::new(vec![ @@ -1002,7 +1002,7 @@ pub mod tests { &left_schema, &PhysicalSortExpr { expr: col("la1", left_schema.as_ref())?, - options: SortOptions::default(), + options: AdvSortOptions::default(), } )? .is_some()); @@ -1012,7 +1012,7 @@ pub mod tests { &left_schema, &PhysicalSortExpr { expr: col("lt1", left_schema.as_ref())?, - options: SortOptions::default(), + options: AdvSortOptions::default(), } )? .is_none()); @@ -1022,7 +1022,7 @@ pub mod tests { &right_schema, &PhysicalSortExpr { expr: col("ra1", right_schema.as_ref())?, - options: SortOptions::default(), + options: AdvSortOptions::default(), } )? .is_some()); @@ -1032,7 +1032,7 @@ pub mod tests { &right_schema, &PhysicalSortExpr { expr: col("rb1", right_schema.as_ref())?, - options: SortOptions::default(), + options: AdvSortOptions::default(), } )? .is_none()); @@ -1078,7 +1078,7 @@ pub mod tests { col("b", &schema)?, &schema, )?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }; let res = convert_sort_expr_with_filter_schema( diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 03a68831438a..2c3afaca4047 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -734,7 +734,11 @@ fn determine_prune_length( }; // Perform binary search on the array to determine the length of the record batch to be pruned - bisect::(&[batch_arr], &[target], &[origin_sorted_expr.options]) + bisect::( + &[batch_arr], + &[target], + &[origin_sorted_expr.options.to_arrow()?], + ) } /// This method determines if the result of the join should be produced in the final step or not. @@ -1724,7 +1728,6 @@ mod tests { partitioned_sym_join_with_filter, split_record_batches, }; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, IntervalUnit, TimeUnit}; use datafusion_common::ScalarValue; use datafusion_execution::config::SessionConfig; @@ -1732,6 +1735,8 @@ mod tests { use datafusion_physical_expr::expressions::{binary, col, lit, Column}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use rstest::*; const TABLE_SIZE: i32 = 30; @@ -1837,11 +1842,11 @@ mod tests { col("la2", left_schema)?, left_schema, )?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let (left, right) = create_memory_table( left_partition, @@ -1912,11 +1917,11 @@ mod tests { let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let (left, right) = create_memory_table( left_partition, @@ -2057,14 +2062,16 @@ mod tests { let right_schema = &right_partition[0].schema(); let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1_des", left_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }, }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1_des", right_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }, @@ -2116,14 +2123,16 @@ mod tests { let right_schema = &right_partition[0].schema(); let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_asc_null_first", left_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_asc_null_first", right_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, @@ -2175,14 +2184,16 @@ mod tests { let right_schema = &right_partition[0].schema(); let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_asc_null_last", left_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }, }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_asc_null_last", right_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }, @@ -2236,14 +2247,16 @@ mod tests { let right_schema = &right_partition[0].schema(); let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_desc_null_first", left_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }, }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_desc_null_first", right_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }, @@ -2298,12 +2311,12 @@ mod tests { let right_schema = &right_partition[0].schema(); let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let (left, right) = create_memory_table( left_partition, @@ -2357,17 +2370,17 @@ mod tests { let left_sorted = vec![ LexOrdering::new(vec![PhysicalSortExpr { expr: col("la1", left_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), LexOrdering::new(vec![PhysicalSortExpr { expr: col("la2", left_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), ]; let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ra1", right_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let (left, right) = create_memory_table( @@ -2438,14 +2451,16 @@ mod tests { let on = vec![(col("lc1", left_schema)?, col("rc1", right_schema)?)]; let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("lt1", left_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("rt1", right_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, @@ -2521,14 +2536,16 @@ mod tests { let on = vec![(col("lc1", left_schema)?, col("rc1", right_schema)?)]; let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("li1", left_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("ri1", right_schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, @@ -2597,11 +2614,11 @@ mod tests { let right_schema = &right_partition[0].schema(); let left_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("l_float", left_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let right_sorted = LexOrdering::new(vec![PhysicalSortExpr { expr: col("r_float", right_schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let (left, right) = create_memory_table( left_partition, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index cffc4b4bff8e..f39d0e682d16 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -489,7 +489,7 @@ fn offset_ordering( .iter() .map(|sort_expr| PhysicalSortExpr { expr: add_offset_to_expr(Arc::clone(&sort_expr.expr), offset), - options: sort_expr.options, + options: sort_expr.options.clone(), }) .collect(), _ => ordering.clone(), @@ -1827,12 +1827,12 @@ mod tests { use std::pin::Pin; use arrow::array::Int32Array; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Fields}; use arrow::error::{ArrowError, Result as ArrowResult}; use datafusion_common::stats::Precision::{Absent, Exact, Inexact}; use datafusion_common::{arrow_datafusion_err, arrow_err, ScalarValue}; + use datafusion_common::sort::AdvSortOptions; use rstest::rstest; fn check( @@ -2642,29 +2642,29 @@ mod tests { #[test] fn test_calculate_join_output_ordering() -> Result<()> { - let options = SortOptions::default(); + let options = AdvSortOptions::default(); let left_ordering = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("d", 3)), - options, + options: options.clone(), }, ]); let right_ordering = LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 2)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("y", 1)), - options, + options: options.clone(), }, ]); let join_type = JoinType::Inner; @@ -2680,45 +2680,45 @@ mod tests { Some(LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("d", 3)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("z", 7)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("y", 6)), - options, + options: options.clone(), }, ])), Some(LexOrdering::new(vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 7)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("y", 6)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("a", 0)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), - options, + options: options.clone(), }, PhysicalSortExpr { expr: Arc::new(Column::new("d", 3)), - options, + options: options.clone(), }, ])), ]; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index e9a360c2ece3..17c6882a8eda 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1599,12 +1599,11 @@ mod tests { #[cfg(test)] mod test { - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema}; - use super::*; use crate::test::TestMemoryExec; use crate::union::UnionExec; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::sort::AdvSortOptions; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -1701,7 +1700,7 @@ mod test { } fn sort_exprs(schema: &Schema) -> LexOrdering { - let options = SortOptions::default(); + let options = AdvSortOptions::default(); LexOrdering::new(vec![PhysicalSortExpr { expr: col("c0", schema).unwrap(), options, diff --git a/datafusion/physical-plan/src/sorts/partial_sort.rs b/datafusion/physical-plan/src/sorts/partial_sort.rs index bd0e6268de52..371a93c3802c 100644 --- a/datafusion/physical-plan/src/sorts/partial_sort.rs +++ b/datafusion/physical-plan/src/sorts/partial_sort.rs @@ -460,13 +460,10 @@ mod tests { use std::collections::HashMap; use arrow::array::*; - use arrow::compute::SortOptions; use arrow::datatypes::*; use futures::FutureExt; use itertools::Itertools; - use datafusion_common::assert_batches_eq; - use crate::collect; use crate::expressions::col; use crate::expressions::PhysicalSortExpr; @@ -475,6 +472,9 @@ mod tests { use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::TestMemoryExec; + use datafusion_common::assert_batches_eq; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use super::*; @@ -491,7 +491,8 @@ mod tests { Field::new("b", DataType::Int32, false), Field::new("c", DataType::Int32, false), ]); - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -500,15 +501,15 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("b", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("c", &schema)?, - options: option_asc, + options: option_asc.clone(), }, ]), Arc::clone(&source), @@ -553,7 +554,8 @@ mod tests { Field::new("b", DataType::Int32, false), Field::new("c", DataType::Int32, false), ]); - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -564,15 +566,15 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("b", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("c", &schema)?, - options: option_asc, + options: option_asc.clone(), }, ]), Arc::clone(&source), @@ -625,7 +627,8 @@ mod tests { Field::new("b", DataType::Int32, false), Field::new("c", DataType::Int32, false), ]); - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -636,15 +639,15 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("b", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("c", &schema)?, - options: option_asc, + options: option_asc.clone(), }, ]), Arc::clone(source), @@ -712,11 +715,13 @@ mod tests { async fn test_partitioned_input_partial_sort() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); let mem_exec = prepare_partitioned_input(); - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; - let option_desc = SortOptions { + let option_desc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -725,15 +730,15 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("b", &schema)?, - options: option_desc, + options: option_desc.clone(), }, PhysicalSortExpr { expr: col("c", &schema)?, - options: option_asc, + options: option_asc.clone(), }, ]), Arc::clone(&mem_exec), @@ -768,11 +773,13 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let mem_exec = prepare_partitioned_input(); let schema = mem_exec.schema(); - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; - let option_desc = SortOptions { + let option_desc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -786,15 +793,15 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("b", &schema)?, - options: option_desc, + options: option_desc.clone(), }, PhysicalSortExpr { expr: col("c", &schema)?, - options: option_asc, + options: option_asc.clone(), }, ]), Arc::clone(&mem_exec), @@ -832,7 +839,8 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let mem_exec = prepare_partitioned_input(); let schema = mem_exec.schema(); - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }; @@ -841,11 +849,11 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("c", &schema)?, - options: option_asc, + options: option_asc.clone(), }, ]), Arc::clone(&mem_exec), @@ -890,7 +898,7 @@ mod tests { let partial_sort_exec = Arc::new(PartialSortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("field_name", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), input, 1, @@ -930,11 +938,13 @@ mod tests { Field::new("b", DataType::Float64, true), Field::new("c", DataType::Float64, true), ])); - let option_asc = SortOptions { + let option_asc = AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }; - let option_desc = SortOptions { + let option_desc = AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }; @@ -980,15 +990,15 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("b", &schema)?, - options: option_asc, + options: option_asc.clone(), }, PhysicalSortExpr { expr: col("c", &schema)?, - options: option_desc, + options: option_desc.clone(), }, ]), TestMemoryExec::try_new_exec(&[vec![batch]], schema, None)?, @@ -1056,7 +1066,7 @@ mod tests { let sort_exec = Arc::new(PartialSortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), blocking_exec, 1, diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 55ba77096a7d..decc4ecf4741 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -44,12 +44,9 @@ use crate::{ Statistics, }; -use arrow::array::{ - Array, RecordBatch, RecordBatchOptions, StringViewArray, UInt32Array, -}; -use arrow::compute::{concat_batches, lexsort_to_indices, take_arrays, SortColumn}; -use arrow::datatypes::{DataType, SchemaRef}; -use arrow::row::{RowConverter, SortField}; +use arrow::array::{Array, RecordBatch, RecordBatchOptions, StringViewArray}; +use arrow::compute::{concat_batches, take_arrays}; +use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; @@ -58,6 +55,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::LexOrdering; use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_common::sort::lexsort_to_indices; use futures::{StreamExt, TryStreamExt}; use log::{debug, trace}; @@ -748,17 +746,10 @@ pub fn sort_batch( ) -> Result { let sort_columns = expressions .iter() - .map(|expr| expr.evaluate_to_sort_column(batch)) + .map(|expr| expr.evaluate_to_adv_sort_column(batch)) .collect::>>()?; - let indices = if is_multi_column_with_lists(&sort_columns) { - // lex_sort_to_indices doesn't support List with more than one column - // https://github.com/apache/arrow-rs/issues/5454 - lexsort_to_indices_multi_columns(sort_columns, fetch)? - } else { - lexsort_to_indices(&sort_columns, fetch)? - }; - + let indices = lexsort_to_indices(sort_columns.as_slice(), fetch)?; let mut columns = take_arrays(batch.columns(), &indices, None)?; // The columns may be larger than the unsorted columns in `batch` especially for variable length @@ -777,48 +768,6 @@ pub fn sort_batch( )?) } -#[inline] -fn is_multi_column_with_lists(sort_columns: &[SortColumn]) -> bool { - sort_columns.iter().any(|c| { - matches!( - c.values.data_type(), - DataType::List(_) | DataType::LargeList(_) | DataType::FixedSizeList(_, _) - ) - }) -} - -pub(crate) fn lexsort_to_indices_multi_columns( - sort_columns: Vec, - limit: Option, -) -> Result { - let (fields, columns) = sort_columns.into_iter().fold( - (vec![], vec![]), - |(mut fields, mut columns), sort_column| { - fields.push(SortField::new_with_options( - sort_column.values.data_type().clone(), - sort_column.options.unwrap_or_default(), - )); - columns.push(sort_column.values); - (fields, columns) - }, - ); - - // TODO reuse converter and rows, refer to TopK. - let converter = RowConverter::new(fields)?; - let rows = converter.convert_columns(&columns)?; - let mut sort: Vec<_> = rows.iter().enumerate().collect(); - sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); - - let mut len = rows.num_rows(); - if let Some(limit) = limit { - len = limit.min(len); - } - let indices = - UInt32Array::from_iter_values(sort.iter().take(len).map(|(i, _)| *i as u32)); - - Ok(indices) -} - /// Sort execution plan. /// /// Support sorting datasets that are larger than the memory allotted @@ -1174,7 +1123,7 @@ impl ExecutionPlan for SortExec { }; updated_exprs.push(PhysicalSortExpr { expr: new_expr, - options: sort.options, + options: sort.options.clone(), }); } @@ -1203,7 +1152,6 @@ mod tests { use crate::test::TestMemoryExec; use arrow::array::*; - use arrow::compute::SortOptions; use arrow::datatypes::*; use datafusion_common::cast::as_primitive_array; use datafusion_common::{assert_batches_eq, Result, ScalarValue}; @@ -1213,6 +1161,8 @@ mod tests { use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::EquivalenceProperties; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use futures::{FutureExt, Stream}; #[derive(Debug, Clone)] @@ -1344,7 +1294,7 @@ mod tests { let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), Arc::new(CoalescePartitionsExec::new(csv)), )); @@ -1390,7 +1340,7 @@ mod tests { let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), Arc::new(CoalescePartitionsExec::new(input)), )); @@ -1458,7 +1408,7 @@ mod tests { let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), Arc::new(CoalescePartitionsExec::new(input)), )); @@ -1551,7 +1501,7 @@ mod tests { SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), Arc::new(CoalescePartitionsExec::new(csv)), ) @@ -1600,7 +1550,7 @@ mod tests { let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("field_name", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), input, )); @@ -1652,14 +1602,16 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, }, PhysicalSortExpr { expr: col("b", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: false, }, @@ -1738,14 +1690,16 @@ mod tests { LexOrdering::new(vec![ PhysicalSortExpr { expr: col("a", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }, }, PhysicalSortExpr { expr: col("b", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }, @@ -1816,7 +1770,7 @@ mod tests { let sort_exec = Arc::new(SortExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), blocking_exec, )); @@ -1847,7 +1801,7 @@ mod tests { let expressions = LexOrdering::new(vec![PhysicalSortExpr { expr: Arc::new(Literal::new(ScalarValue::Int64(Some(1)))), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let result = sort_batch(&batch, expressions.as_ref(), None).unwrap(); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 00fa78ce5229..ac39a2c4fd37 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -361,7 +361,7 @@ impl ExecutionPlan for SortPreservingMergeExec { }; updated_exprs.push(PhysicalSortExpr { expr: updated_expr, - options: sort.options, + options: sort.options.clone(), }); } @@ -401,7 +401,6 @@ mod tests { ArrayRef, Int32Array, Int64Array, RecordBatch, StringArray, TimestampNanosecondArray, }; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{assert_batches_eq, assert_contains, DataFusionError}; use datafusion_common_runtime::SpawnedTask; @@ -412,6 +411,8 @@ mod tests { use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr; use futures::{FutureExt, Stream, StreamExt}; use tokio::time::timeout; @@ -789,7 +790,8 @@ mod tests { let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: true, nulls_first: true, }, @@ -963,14 +965,16 @@ mod tests { let sort = LexOrdering::new(vec![ PhysicalSortExpr { expr: col("b", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, }, PhysicalSortExpr { expr: col("c", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }, @@ -1014,7 +1018,8 @@ mod tests { let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, @@ -1049,7 +1054,8 @@ mod tests { let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("b", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, @@ -1082,7 +1088,7 @@ mod tests { let schema = make_partition(11).schema(); let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("i", &schema).unwrap(), - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let batches = @@ -1221,7 +1227,7 @@ mod tests { let sort_preserving_merge_exec = Arc::new(SortPreservingMergeExec::new( LexOrdering::new(vec![PhysicalSortExpr { expr: col("a", &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]), blocking_exec, )); @@ -1269,7 +1275,8 @@ mod tests { let sort = LexOrdering::new(vec![PhysicalSortExpr { expr: col("value", &schema).unwrap(), - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: true, }, diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index e029c60b285b..d15e1b34aae9 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -22,7 +22,7 @@ use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; -use datafusion_common::Result; +use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::{Fuse, StreamExt}; @@ -101,7 +101,12 @@ impl RowCursorStream { .iter() .map(|expr| { let data_type = expr.expr.data_type(schema)?; - Ok(SortField::new_with_options(data_type, expr.options)) + match expr.options.to_arrow() { + Ok(options) => Ok(SortField::new_with_options(data_type, options)), + Err(_) => internal_err!( + "Custom orderings not supported in RowCursorStream." + ), + } }) .collect::>>()?; @@ -195,7 +200,7 @@ impl FieldCursorStream { let mut array_reservation = self.reservation.new_empty(); array_reservation.try_grow(size_in_mem)?; Ok(ArrayValues::new( - self.sort.options, + self.sort.options.to_arrow()?, array, array_reservation, )) diff --git a/datafusion/physical-plan/src/topk/mod.rs b/datafusion/physical-plan/src/topk/mod.rs index 85de1eefce2e..a36d3c314fae 100644 --- a/datafusion/physical-plan/src/topk/mod.rs +++ b/datafusion/physical-plan/src/topk/mod.rs @@ -29,8 +29,8 @@ use crate::spill::get_record_batch_memory_size; use crate::{stream::RecordBatchStreamAdapter, SendableRecordBatchStream}; use arrow::array::{Array, ArrayRef, RecordBatch}; use arrow::datatypes::SchemaRef; -use datafusion_common::HashMap; use datafusion_common::Result; +use datafusion_common::{internal_datafusion_err, HashMap}; use datafusion_execution::{ memory_pool::{MemoryConsumer, MemoryReservation}, runtime_env::RuntimeEnv, @@ -115,7 +115,9 @@ impl TopK { .map(|e| { Ok(SortField::new_with_options( e.expr.data_type(&schema)?, - e.options, + e.options.to_arrow().map_err(|_| { + internal_datafusion_err!("Custom sorts not supported in TopK") + })?, )) }) .collect::>()?; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 791370917523..9741b724adf8 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -653,6 +653,7 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::DataType; + use datafusion_common::sort::AdvSortOptions; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; @@ -680,7 +681,7 @@ mod tests { .iter() .map(|(expr, options)| PhysicalSortExpr { expr: Arc::clone(*expr), - options: *options, + options: AdvSortOptions::with_default_ordering(*options), }) .collect::() } 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 7e37156a133e..f7ffa0b4d0e2 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -1213,7 +1213,6 @@ mod tests { builder::{Int64Builder, UInt64Builder}, RecordBatch, }; - use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{ assert_batches_eq, exec_datafusion_err, Result, ScalarValue, @@ -1232,6 +1231,8 @@ mod tests { use datafusion_physical_expr::window::StandardWindowExpr; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; + use datafusion_common::sort::AdvSortOptions; + use datafusion_common::types::SortOrdering; use futures::future::Shared; use futures::{pin_mut, ready, FutureExt, Stream, StreamExt}; use itertools::Itertools; @@ -1334,7 +1335,7 @@ mod tests { let partitionby_exprs = vec![col(hash, &schema)?]; let orderby_exprs = LexOrdering::new(vec![PhysicalSortExpr { expr: col(order_by, &schema)?, - options: SortOptions::default(), + options: AdvSortOptions::default(), }]); let window_frame = WindowFrame::new_bounds( WindowFrameUnits::Range, @@ -1451,7 +1452,8 @@ mod tests { fn schema_orders(schema: &SchemaRef) -> Result> { let orderings = vec![LexOrdering::new(vec![PhysicalSortExpr { expr: col("sn", schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: false, nulls_first: false, }, @@ -1603,7 +1605,7 @@ mod tests { let window_exprs = vec![ // LAST_VALUE(a) - Arc::new(StandardWindowExpr::new( + Arc::new(StandardWindowExpr::try_new( last_value_func, &[], &LexOrdering::default(), @@ -1612,9 +1614,9 @@ mod tests { WindowFrameBound::Preceding(ScalarValue::UInt64(None)), WindowFrameBound::CurrentRow, )), - )) as _, + )?) as _, // NTH_VALUE(a, -1) - Arc::new(StandardWindowExpr::new( + Arc::new(StandardWindowExpr::try_new( nth_value_func1, &[], &LexOrdering::default(), @@ -1623,9 +1625,9 @@ mod tests { WindowFrameBound::Preceding(ScalarValue::UInt64(None)), WindowFrameBound::CurrentRow, )), - )) as _, + )?) as _, // NTH_VALUE(a, -2) - Arc::new(StandardWindowExpr::new( + Arc::new(StandardWindowExpr::try_new( nth_value_func2, &[], &LexOrdering::default(), @@ -1634,7 +1636,7 @@ mod tests { WindowFrameBound::Preceding(ScalarValue::UInt64(None)), WindowFrameBound::CurrentRow, )), - )) as _, + )?) as _, ]; let physical_plan = BoundedWindowAggExec::try_new( window_exprs, diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index d38bf2a186a8..83f7726f97a6 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -31,7 +31,6 @@ use crate::{ }; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use arrow_schema::SortOptions; use datafusion_common::{exec_err, Result}; use datafusion_expr::{ PartitionEvaluator, ReversedUDWF, SetMonotonicity, WindowFrame, @@ -53,6 +52,8 @@ use itertools::Itertools; // Public interface: pub use bounded_window_agg_exec::BoundedWindowAggExec; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; pub use datafusion_physical_expr::window::{ PlainAggregateWindowExpr, StandardWindowExpr, WindowExpr, }; @@ -120,12 +121,14 @@ pub fn create_window_expr( aggregate, ) } - WindowFunctionDefinition::WindowUDF(fun) => Arc::new(StandardWindowExpr::new( - create_udwf_window_expr(fun, args, input_schema, name, ignore_nulls)?, - partition_by, - order_by, - window_frame, - )), + WindowFunctionDefinition::WindowUDF(fun) => { + Arc::new(StandardWindowExpr::try_new( + create_udwf_window_expr(fun, args, input_schema, name, ignore_nulls)?, + partition_by, + order_by, + window_frame, + )?) + } }) } @@ -268,7 +271,10 @@ impl StandardWindowFunctionExpr for WindowUDFExpr { .zip(schema.column_with_name(self.name())) .map(|(options, (idx, field))| { let expr = Arc::new(Column::new(field.name(), idx)); - PhysicalSortExpr { expr, options } + PhysicalSortExpr { + expr, + options: AdvSortOptions::with_default_ordering(options), + } }) } } @@ -293,7 +299,7 @@ pub(crate) fn calc_requirements< if !sort_reqs.iter().any(|e| e.expr.eq(expr)) { sort_reqs.push(PhysicalSortRequirement::new( Arc::clone(expr), - Some(*options), + Some(options.clone()), )); } } @@ -428,7 +434,11 @@ pub(crate) fn window_equivalence_properties( let new_ordering = vec![LexOrdering::new(vec![PhysicalSortExpr::new( Arc::new(window_col), - SortOptions::new(increasing, true), + AdvSortOptions::new( + SortOrdering::Default, + increasing, + true, + ), )])]; window_eq_properties.add_new_orderings(new_ordering); } else { @@ -437,7 +447,11 @@ pub(crate) fn window_equivalence_properties( let mut existing = lex.take_exprs(); existing.push(PhysicalSortExpr::new( Arc::new(window_col.clone()), - SortOptions::new(increasing, true), + AdvSortOptions::new( + SortOrdering::Default, + increasing, + true, + ), )); window_eq_properties .add_new_ordering(LexOrdering::new(existing)); @@ -472,14 +486,22 @@ pub(crate) fn window_equivalence_properties( let new_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( Arc::new(window_col), - SortOptions::new(false, false), + AdvSortOptions::new( + SortOrdering::Default, + false, + false, + ), )]); window_eq_properties.add_new_ordering(new_ordering); } else if !increasing && (!asc || no_partitioning) { let new_ordering = LexOrdering::new(vec![PhysicalSortExpr::new( Arc::new(window_col), - SortOptions::new(true, false), + AdvSortOptions::new( + SortOrdering::Default, + true, + false, + ), )]); window_eq_properties.add_new_ordering(new_ordering); }; @@ -626,8 +648,14 @@ pub fn get_window_mode( fn sort_options_resolving_constant(expr: Arc) -> Vec { vec![ - PhysicalSortExpr::new(Arc::clone(&expr), SortOptions::new(false, false)), - PhysicalSortExpr::new(expr, SortOptions::new(true, true)), + PhysicalSortExpr::new( + Arc::clone(&expr), + AdvSortOptions::new(SortOrdering::Default, false, false), + ), + PhysicalSortExpr::new( + expr, + AdvSortOptions::new(SortOrdering::Default, true, true), + ), ] } @@ -689,7 +717,7 @@ mod tests { ) -> PhysicalSortExpr { PhysicalSortExpr { expr: col(name, schema).unwrap(), - options, + options: AdvSortOptions::with_default_ordering(options), } } @@ -749,7 +777,8 @@ mod tests { let mut orderbys = vec![]; for (col_name, descending, nulls_first) in ob_params { let expr = col(col_name, &schema)?; - let options = SortOptions { + let options = AdvSortOptions { + ordering: SortOrdering::Default, descending, nulls_first, }; @@ -758,7 +787,8 @@ mod tests { let mut expected: Option = None; for (col_name, reqs) in expected_params { - let options = reqs.map(|(descending, nulls_first)| SortOptions { + let options = reqs.map(|(descending, nulls_first)| AdvSortOptions { + ordering: SortOrdering::Default, descending, nulls_first, }); @@ -828,14 +858,16 @@ mod tests { { let physical_ordering = PhysicalSortExpr { expr: col("nullable_col", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: physical_desc, nulls_first: physical_nulls_first, }, }; let required_ordering = PhysicalSortExpr { expr: col("nullable_col", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: req_desc, nulls_first: req_nulls_first, }, @@ -868,14 +900,16 @@ mod tests { { let physical_ordering = PhysicalSortExpr { expr: col("non_nullable_col", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: physical_desc, nulls_first: physical_nulls_first, }, }; let required_ordering = PhysicalSortExpr { expr: col("non_nullable_col", &schema)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: req_desc, nulls_first: req_nulls_first, }, @@ -991,7 +1025,7 @@ mod tests { let expr = col(col_name, &test_schema)?; // Give default ordering, this is same with input ordering direction // In this test we do check for reversibility. - let options = SortOptions::default(); + let options = AdvSortOptions::default(); order_by_exprs.push(PhysicalSortExpr { expr, options }); } let res = get_window_mode( @@ -1154,7 +1188,8 @@ mod tests { let mut order_by_exprs = LexOrdering::default(); for (col_name, descending, nulls_first) in order_by_params { let expr = col(col_name, &test_schema)?; - let options = SortOptions { + let options = AdvSortOptions { + ordering: SortOrdering::Default, descending: *descending, nulls_first: *nulls_first, }; diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 6331b7fb3114..3f5c24673119 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -19,7 +19,6 @@ use std::sync::Arc; -use arrow::compute::SortOptions; use chrono::{TimeZone, Utc}; use datafusion_expr::dml::InsertOp; use object_store::path::Path; @@ -42,6 +41,8 @@ use datafusion::physical_plan::expressions::{ }; use datafusion::physical_plan::windows::{create_window_expr, schema_add_window_field}; use datafusion::physical_plan::{Partitioning, PhysicalExpr, WindowExpr}; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_proto_common::common::proto_error; @@ -75,7 +76,8 @@ pub fn parse_physical_sort_expr( ) -> Result { if let Some(expr) = &proto.expr { let expr = parse_physical_expr(expr.as_ref(), registry, input_schema, codec)?; - let options = SortOptions { + let options = AdvSortOptions { + ordering: SortOrdering::Default, // TODO this should come from a registry descending: !proto.asc, nulls_first: proto.nulls_first, }; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 60972ac54ba7..453c2686c163 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -22,7 +22,6 @@ use datafusion::physical_expr::aggregate::AggregateExprBuilder; use prost::bytes::BufMut; use prost::Message; -use datafusion::arrow::compute::SortOptions; use datafusion::arrow::datatypes::SchemaRef; use datafusion::datasource::file_format::csv::CsvSink; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; @@ -67,6 +66,8 @@ use datafusion::physical_plan::{ ExecutionPlan, InputOrderMode, PhysicalExpr, WindowExpr, }; use datafusion_common::config::TableParquetOptions; +use datafusion_common::sort::AdvSortOptions; +use datafusion_common::types::SortOrdering; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::{AggregateUDF, ScalarUDF, WindowUDF}; @@ -893,7 +894,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .as_ref(); Ok(PhysicalSortExpr { expr: parse_physical_expr(expr, registry, input.schema().as_ref(), extension_codec)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: !sort_expr.asc, nulls_first: sort_expr.nulls_first, }, @@ -940,7 +942,8 @@ impl AsExecutionPlan for protobuf::PhysicalPlanNode { .as_ref(); Ok(PhysicalSortExpr { expr: parse_physical_expr(expr, registry, input.schema().as_ref(), extension_codec)?, - options: SortOptions { + options: AdvSortOptions { + ordering: SortOrdering::Default, descending: !sort_expr.asc, nulls_first: sort_expr.nulls_first, }, diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index b5bfef99a6f3..81519f37cf9f 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -307,7 +307,7 @@ fn roundtrip_udwf() -> Result<()> { let field_b = Field::new("b", DataType::Int64, false); let schema = Arc::new(Schema::new(vec![field_a, field_b])); - let udwf_expr = Arc::new(StandardWindowExpr::new( + let udwf_expr = Arc::new(StandardWindowExpr::try_new( create_udwf_window_expr( &row_number_udwf(), &[], @@ -354,7 +354,7 @@ fn roundtrip_window() -> Result<()> { "NTH_VALUE(a, 2) PARTITION BY [b] ORDER BY [a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW".to_string(), false, )?; - let udwf_expr = Arc::new(StandardWindowExpr::new( + let udwf_expr = Arc::new(StandardWindowExpr::try_new( nth_value_window, &[col("b", &schema)?], &LexOrdering::new(vec![PhysicalSortExpr { @@ -1142,7 +1142,7 @@ fn roundtrip_udwf_extension_codec() -> Result<()> { WindowFrameBound::CurrentRow, ); - let udwf_expr = Arc::new(StandardWindowExpr::new( + let udwf_expr = Arc::new(StandardWindowExpr::try_new( udwf, &[col("b", &schema)?], &LexOrdering::new(vec![PhysicalSortExpr {