Skip to content

Commit c9bf3f3

Browse files
authored
Change required input ordering physical plan API to allow any NULLS FIRST / LAST and ASC / DESC (#5772)
* Change required input ordering to format to not absolutely require direction. * remove unnecessary code
1 parent 8e125d2 commit c9bf3f3

File tree

16 files changed

+356
-113
lines changed

16 files changed

+356
-113
lines changed

datafusion/core/src/physical_optimizer/repartition.rs

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -321,6 +321,9 @@ fn init() {
321321
mod tests {
322322
use arrow::compute::SortOptions;
323323
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
324+
use datafusion_physical_expr::{
325+
make_sort_requirements_from_exprs, PhysicalSortRequirement,
326+
};
324327

325328
use super::*;
326329
use crate::datasource::listing::PartitionedFile;
@@ -1131,8 +1134,10 @@ mod tests {
11311134
}
11321135

11331136
// model that it requires the output ordering of its input
1134-
fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
1135-
vec![self.input.output_ordering()]
1137+
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
1138+
vec![self
1139+
.output_ordering()
1140+
.map(make_sort_requirements_from_exprs)]
11361141
}
11371142

11381143
fn with_new_children(

datafusion/core/src/physical_optimizer/sort_enforcement.rs

Lines changed: 15 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,10 @@ use crate::physical_plan::{with_new_children_if_necessary, Distribution, Executi
4747
use arrow::datatypes::SchemaRef;
4848
use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion};
4949
use datafusion_common::{reverse_sort_options, DataFusionError};
50-
use datafusion_physical_expr::utils::{ordering_satisfy, ordering_satisfy_concrete};
50+
use datafusion_physical_expr::utils::{
51+
make_sort_exprs_from_requirements, ordering_satisfy,
52+
ordering_satisfy_requirement_concrete,
53+
};
5154
use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr};
5255
use itertools::{concat, izip};
5356
use std::iter::zip;
@@ -471,17 +474,20 @@ fn ensure_sorting(
471474
let physical_ordering = child.output_ordering();
472475
match (required_ordering, physical_ordering) {
473476
(Some(required_ordering), Some(physical_ordering)) => {
474-
let is_ordering_satisfied = ordering_satisfy_concrete(
477+
if !ordering_satisfy_requirement_concrete(
475478
physical_ordering,
476-
required_ordering,
479+
&required_ordering,
477480
|| child.equivalence_properties(),
478-
);
479-
if !is_ordering_satisfied {
481+
) {
480482
// Make sure we preserve the ordering requirements:
481483
update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?;
482-
let sort_expr = required_ordering.to_vec();
484+
let sort_expr = make_sort_exprs_from_requirements(&required_ordering);
483485
add_sort_above(child, sort_expr)?;
484-
*sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![]));
486+
if is_sort(child) {
487+
*sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![]));
488+
} else {
489+
*sort_onwards = None;
490+
}
485491
}
486492
if let Some(tree) = sort_onwards {
487493
// For window expressions, we can remove some sorts when we can
@@ -497,7 +503,8 @@ fn ensure_sorting(
497503
}
498504
(Some(required), None) => {
499505
// Ordering requirement is not met, we should add a `SortExec` to the plan.
500-
add_sort_above(child, required.to_vec())?;
506+
let sort_expr = make_sort_exprs_from_requirements(&required);
507+
add_sort_above(child, sort_expr)?;
501508
*sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![]));
502509
}
503510
(None, Some(_)) => {
@@ -592,7 +599,6 @@ fn analyze_window_sort_removal(
592599
};
593600

594601
let mut first_should_reverse = None;
595-
let mut physical_ordering_common = vec![];
596602
for sort_any in sort_tree.get_leaves() {
597603
let sort_output_ordering = sort_any.output_ordering();
598604
// Variable `sort_any` will either be a `SortExec` or a
@@ -609,11 +615,6 @@ fn analyze_window_sort_removal(
609615
DataFusionError::Plan("A SortExec should have output ordering".to_string())
610616
})?;
611617
if let Some(physical_ordering) = physical_ordering {
612-
if physical_ordering_common.is_empty()
613-
|| physical_ordering.len() < physical_ordering_common.len()
614-
{
615-
physical_ordering_common = physical_ordering.to_vec();
616-
}
617618
let (can_skip_sorting, should_reverse) = can_skip_sort(
618619
window_expr[0].partition_by(),
619620
required_ordering,
@@ -664,15 +665,13 @@ fn analyze_window_sort_removal(
664665
new_child,
665666
new_schema,
666667
partition_keys.to_vec(),
667-
Some(physical_ordering_common),
668668
)?) as _
669669
} else {
670670
Arc::new(WindowAggExec::try_new(
671671
window_expr,
672672
new_child,
673673
new_schema,
674674
partition_keys.to_vec(),
675-
Some(physical_ordering_common),
676675
)?) as _
677676
};
678677
return Ok(Some(PlanWithCorrespondingSort::new(new_plan)));
@@ -1889,7 +1888,6 @@ mod tests {
18891888
input.clone(),
18901889
input.schema(),
18911890
vec![],
1892-
Some(sort_exprs),
18931891
)
18941892
.unwrap(),
18951893
)

datafusion/core/src/physical_plan/joins/sort_merge_join.rs

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,9 @@ use crate::physical_plan::{
5555
};
5656

5757
use datafusion_common::tree_node::{Transformed, TreeNode};
58+
use datafusion_physical_expr::{
59+
make_sort_requirements_from_exprs, PhysicalSortRequirement,
60+
};
5861

5962
/// join execution plan executes partitions in parallel and combines them into a set of
6063
/// partitions.
@@ -225,8 +228,11 @@ impl ExecutionPlan for SortMergeJoinExec {
225228
]
226229
}
227230

228-
fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
229-
vec![Some(&self.left_sort_exprs), Some(&self.right_sort_exprs)]
231+
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
232+
vec![
233+
Some(make_sort_requirements_from_exprs(&self.left_sort_exprs)),
234+
Some(make_sort_requirements_from_exprs(&self.right_sort_exprs)),
235+
]
230236
}
231237

232238
fn output_partitioning(&self) -> Partitioning {

datafusion/core/src/physical_plan/joins/symmetric_hash_join.rs

Lines changed: 9 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,9 @@ use hashbrown::{raw::RawTable, HashSet};
4646

4747
use datafusion_common::{utils::bisect, ScalarValue};
4848
use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval};
49+
use datafusion_physical_expr::{
50+
make_sort_requirements_from_exprs, PhysicalSortRequirement,
51+
};
4952

5053
use crate::error::{DataFusionError, Result};
5154
use crate::execution::context::TaskContext;
@@ -399,11 +402,12 @@ impl ExecutionPlan for SymmetricHashJoinExec {
399402
self.schema.clone()
400403
}
401404

402-
fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
403-
vec![
404-
Some(&self.left_required_sort_exprs),
405-
Some(&self.right_required_sort_exprs),
406-
]
405+
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
406+
let left_required =
407+
make_sort_requirements_from_exprs(&self.left_required_sort_exprs);
408+
let right_required =
409+
make_sort_requirements_from_exprs(&self.right_required_sort_exprs);
410+
vec![Some(left_required), Some(right_required)]
407411
}
408412

409413
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {

datafusion/core/src/physical_plan/mod.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -142,7 +142,7 @@ pub trait ExecutionPlan: Debug + Send + Sync {
142142
/// NOTE that checking `!is_empty()` does **not** check for a
143143
/// required input ordering. Instead, the correct check is that at
144144
/// least one entry must be `Some`
145-
fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
145+
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
146146
vec![None; self.children().len()]
147147
}
148148

@@ -591,11 +591,11 @@ impl Distribution {
591591

592592
use datafusion_physical_expr::expressions::Column;
593593
pub use datafusion_physical_expr::window::WindowExpr;
594-
use datafusion_physical_expr::EquivalenceProperties;
595594
use datafusion_physical_expr::{
596595
expr_list_eq_strict_order, normalize_expr_with_equivalence_properties,
597596
};
598597
pub use datafusion_physical_expr::{AggregateExpr, PhysicalExpr};
598+
use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement};
599599

600600
/// Applies an optional projection to a [`SchemaRef`], returning the
601601
/// projected schema

datafusion/core/src/physical_plan/planner.rs

Lines changed: 0 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -577,33 +577,6 @@ impl DefaultPhysicalPlanner {
577577

578578
let logical_input_schema = input.schema();
579579

580-
let physical_sort_keys = if sort_keys.is_empty() {
581-
None
582-
} else {
583-
let physical_input_schema = input_exec.schema();
584-
let sort_keys = sort_keys
585-
.iter()
586-
.map(|(e, _)| match e {
587-
Expr::Sort(expr::Sort {
588-
expr,
589-
asc,
590-
nulls_first,
591-
}) => create_physical_sort_expr(
592-
expr,
593-
logical_input_schema,
594-
&physical_input_schema,
595-
SortOptions {
596-
descending: !*asc,
597-
nulls_first: *nulls_first,
598-
},
599-
session_state.execution_props(),
600-
),
601-
_ => unreachable!(),
602-
})
603-
.collect::<Result<Vec<_>>>()?;
604-
Some(sort_keys)
605-
};
606-
607580
let physical_input_schema = input_exec.schema();
608581
let window_expr = window_expr
609582
.iter()
@@ -628,15 +601,13 @@ impl DefaultPhysicalPlanner {
628601
input_exec,
629602
physical_input_schema,
630603
physical_partition_keys,
631-
physical_sort_keys,
632604
)?)
633605
} else {
634606
Arc::new(WindowAggExec::try_new(
635607
window_expr,
636608
input_exec,
637609
physical_input_schema,
638610
physical_partition_keys,
639-
physical_sort_keys,
640611
)?)
641612
})
642613
}

datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,9 @@ use crate::physical_plan::{
4646
Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream,
4747
SendableRecordBatchStream, Statistics,
4848
};
49-
use datafusion_physical_expr::EquivalenceProperties;
49+
use datafusion_physical_expr::{
50+
make_sort_requirements_from_exprs, EquivalenceProperties, PhysicalSortRequirement,
51+
};
5052

5153
/// Sort preserving merge execution plan
5254
///
@@ -125,12 +127,16 @@ impl ExecutionPlan for SortPreservingMergeExec {
125127
vec![Distribution::UnspecifiedDistribution]
126128
}
127129

128-
fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
129-
vec![Some(&self.expr)]
130+
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
131+
vec![Some(make_sort_requirements_from_exprs(&self.expr))]
130132
}
131133

132134
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
133-
Some(&self.expr)
135+
self.input.output_ordering()
136+
}
137+
138+
fn maintains_input_order(&self) -> Vec<bool> {
139+
vec![true]
134140
}
135141

136142
fn equivalence_properties(&self) -> EquivalenceProperties {

datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs

Lines changed: 10 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -50,11 +50,14 @@ use std::pin::Pin;
5050
use std::sync::Arc;
5151
use std::task::{Context, Poll};
5252

53+
use crate::physical_plan::windows::calc_requirements;
5354
use datafusion_physical_expr::window::{
5455
PartitionBatchState, PartitionBatches, PartitionKey, PartitionWindowAggStates,
5556
WindowAggState, WindowState,
5657
};
57-
use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr};
58+
use datafusion_physical_expr::{
59+
EquivalenceProperties, PhysicalExpr, PhysicalSortRequirement,
60+
};
5861
use indexmap::IndexMap;
5962
use log::debug;
6063

@@ -71,8 +74,6 @@ pub struct BoundedWindowAggExec {
7174
input_schema: SchemaRef,
7275
/// Partition Keys
7376
pub partition_keys: Vec<Arc<dyn PhysicalExpr>>,
74-
/// Sort Keys
75-
pub sort_keys: Option<Vec<PhysicalSortExpr>>,
7677
/// Execution metrics
7778
metrics: ExecutionPlanMetricsSet,
7879
}
@@ -84,7 +85,6 @@ impl BoundedWindowAggExec {
8485
input: Arc<dyn ExecutionPlan>,
8586
input_schema: SchemaRef,
8687
partition_keys: Vec<Arc<dyn PhysicalExpr>>,
87-
sort_keys: Option<Vec<PhysicalSortExpr>>,
8888
) -> Result<Self> {
8989
let schema = create_schema(&input_schema, &window_expr)?;
9090
let schema = Arc::new(schema);
@@ -94,7 +94,6 @@ impl BoundedWindowAggExec {
9494
schema,
9595
input_schema,
9696
partition_keys,
97-
sort_keys,
9897
metrics: ExecutionPlanMetricsSet::new(),
9998
})
10099
}
@@ -123,7 +122,7 @@ impl BoundedWindowAggExec {
123122
let mut result = vec![];
124123
// All window exprs have the same partition by, so we just use the first one:
125124
let partition_by = self.window_expr()[0].partition_by();
126-
let sort_keys = self.sort_keys.as_deref().unwrap_or(&[]);
125+
let sort_keys = self.input.output_ordering().unwrap_or(&[]);
127126
for item in partition_by {
128127
if let Some(a) = sort_keys.iter().find(|&e| e.expr.eq(item)) {
129128
result.push(a.clone());
@@ -167,17 +166,18 @@ impl ExecutionPlan for BoundedWindowAggExec {
167166
self.input().output_ordering()
168167
}
169168

170-
fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
171-
let sort_keys = self.sort_keys.as_deref();
172-
vec![sort_keys]
169+
fn required_input_ordering(&self) -> Vec<Option<Vec<PhysicalSortRequirement>>> {
170+
let partition_bys = self.window_expr()[0].partition_by();
171+
let order_keys = self.window_expr()[0].order_by();
172+
let requirements = calc_requirements(partition_bys, order_keys);
173+
vec![requirements]
173174
}
174175

175176
fn required_input_distribution(&self) -> Vec<Distribution> {
176177
if self.partition_keys.is_empty() {
177178
debug!("No partition defined for BoundedWindowAggExec!!!");
178179
vec![Distribution::SinglePartition]
179180
} else {
180-
//TODO support PartitionCollections if there is no common partition columns in the window_expr
181181
vec![Distribution::HashPartitioned(self.partition_keys.clone())]
182182
}
183183
}
@@ -199,7 +199,6 @@ impl ExecutionPlan for BoundedWindowAggExec {
199199
children[0].clone(),
200200
self.input_schema.clone(),
201201
self.partition_keys.clone(),
202-
self.sort_keys.clone(),
203202
)?))
204203
}
205204

0 commit comments

Comments
 (0)