Skip to content

Commit 2c2e0e7

Browse files
authored
Simplifications (#12656)
1 parent 84ac4f9 commit 2c2e0e7

File tree

1 file changed

+5
-11
lines changed

1 file changed

+5
-11
lines changed

datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs

Lines changed: 5 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -257,17 +257,11 @@ impl ExecutionPlan for BoundedWindowAggExec {
257257
fn required_input_ordering(&self) -> Vec<Option<LexRequirement>> {
258258
let partition_bys = self.window_expr()[0].partition_by();
259259
let order_keys = self.window_expr()[0].order_by();
260-
if self.input_order_mode != InputOrderMode::Sorted
261-
|| self.ordered_partition_by_indices.len() >= partition_bys.len()
262-
{
263-
let partition_bys = self
264-
.ordered_partition_by_indices
265-
.iter()
266-
.map(|idx| &partition_bys[*idx]);
267-
vec![calc_requirements(partition_bys, order_keys)]
268-
} else {
269-
vec![calc_requirements(partition_bys, order_keys)]
270-
}
260+
let partition_bys = self
261+
.ordered_partition_by_indices
262+
.iter()
263+
.map(|idx| &partition_bys[*idx]);
264+
vec![calc_requirements(partition_bys, order_keys)]
271265
}
272266

273267
fn required_input_distribution(&self) -> Vec<Distribution> {

0 commit comments

Comments
 (0)