Skip to content

Commit 3550758

Browse files
authored
chore: remove partition_keys from (Bounded)WindowAggExec (#14526)
* chore: remove partition_keys from (Bounded)WindowAggExec * support bounded_window_agg_exec * fix * fix * fix
1 parent 94d2baf commit 3550758

File tree

10 files changed

+79
-64
lines changed

10 files changed

+79
-64
lines changed

datafusion/core/src/physical_planner.rs

Lines changed: 6 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -86,7 +86,6 @@ use datafusion_physical_plan::execution_plan::InvariantLevel;
8686
use datafusion_physical_plan::memory::MemorySourceConfig;
8787
use datafusion_physical_plan::placeholder_row::PlaceholderRowExec;
8888
use datafusion_physical_plan::unnest::ListUnnest;
89-
use datafusion_sql::utils::window_expr_common_partition_keys;
9089

9190
use crate::schema_equivalence::schema_satisfied_by;
9291
use async_trait::async_trait;
@@ -557,34 +556,13 @@ impl DefaultPhysicalPlanner {
557556
return exec_err!("Table '{table_name}' does not exist");
558557
}
559558
}
560-
LogicalPlan::Window(Window {
561-
input, window_expr, ..
562-
}) => {
559+
LogicalPlan::Window(Window { window_expr, .. }) => {
563560
if window_expr.is_empty() {
564561
return internal_err!("Impossibly got empty window expression");
565562
}
566563

567564
let input_exec = children.one()?;
568565

569-
// at this moment we are guaranteed by the logical planner
570-
// to have all the window_expr to have equal sort key
571-
let partition_keys = window_expr_common_partition_keys(window_expr)?;
572-
573-
let can_repartition = !partition_keys.is_empty()
574-
&& session_state.config().target_partitions() > 1
575-
&& session_state.config().repartition_window_functions();
576-
577-
let physical_partition_keys = if can_repartition {
578-
partition_keys
579-
.iter()
580-
.map(|e| {
581-
self.create_physical_expr(e, input.schema(), session_state)
582-
})
583-
.collect::<Result<Vec<Arc<dyn PhysicalExpr>>>>()?
584-
} else {
585-
vec![]
586-
};
587-
588566
let get_sort_keys = |expr: &Expr| match expr {
589567
Expr::WindowFunction(WindowFunction {
590568
ref partition_by,
@@ -626,6 +604,9 @@ impl DefaultPhysicalPlanner {
626604
})
627605
.collect::<Result<Vec<_>>>()?;
628606

607+
let can_repartition = session_state.config().target_partitions() > 1
608+
&& session_state.config().repartition_window_functions();
609+
629610
let uses_bounded_memory =
630611
window_expr.iter().all(|e| e.uses_bounded_memory());
631612
// If all window expressions can run with bounded memory,
@@ -634,14 +615,14 @@ impl DefaultPhysicalPlanner {
634615
Arc::new(BoundedWindowAggExec::try_new(
635616
window_expr,
636617
input_exec,
637-
physical_partition_keys,
638618
InputOrderMode::Sorted,
619+
can_repartition,
639620
)?)
640621
} else {
641622
Arc::new(WindowAggExec::try_new(
642623
window_expr,
643624
input_exec,
644-
physical_partition_keys,
625+
can_repartition,
645626
)?)
646627
}
647628
}

datafusion/core/tests/fuzz_cases/window_fuzz.rs

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -293,8 +293,8 @@ async fn bounded_window_causal_non_causal() -> Result<()> {
293293
let running_window_exec = Arc::new(BoundedWindowAggExec::try_new(
294294
vec![window_expr],
295295
memory_exec.clone(),
296-
vec![],
297296
Linear,
297+
false,
298298
)?);
299299
let task_ctx = ctx.task_ctx();
300300
let collected_results = collect(running_window_exec, task_ctx).await?;
@@ -660,7 +660,7 @@ async fn run_window_test(
660660
false,
661661
)?],
662662
exec1,
663-
vec![],
663+
false,
664664
)?) as _;
665665
let exec2 = Arc::new(DataSourceExec::new(Arc::new(
666666
MemorySourceConfig::try_new(&[input1.clone()], schema.clone(), None)?
@@ -678,8 +678,8 @@ async fn run_window_test(
678678
false,
679679
)?],
680680
exec2,
681-
vec![],
682681
search_mode.clone(),
682+
false,
683683
)?) as _;
684684
let task_ctx = ctx.task_ctx();
685685
let collected_usual = collect(usual_window_exec, task_ctx.clone()).await?;

datafusion/core/tests/physical_optimizer/test_utils.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -237,8 +237,8 @@ pub fn bounded_window_exec_with_partition(
237237
BoundedWindowAggExec::try_new(
238238
vec![window_expr],
239239
Arc::clone(&input),
240-
vec![],
241240
InputOrderMode::Sorted,
241+
false,
242242
)
243243
.unwrap(),
244244
)
@@ -266,8 +266,8 @@ pub fn bounded_window_exec_non_set_monotonic(
266266
)
267267
.unwrap()],
268268
Arc::clone(&input),
269-
vec![],
270269
InputOrderMode::Sorted,
270+
false,
271271
)
272272
.unwrap(),
273273
)

datafusion/physical-optimizer/src/enforce_distribution.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1196,15 +1196,15 @@ pub fn ensure_distribution(
11961196
if let Some(updated_window) = get_best_fitting_window(
11971197
exec.window_expr(),
11981198
exec.input(),
1199-
&exec.partition_keys,
1199+
&exec.partition_keys(),
12001200
)? {
12011201
plan = updated_window;
12021202
}
12031203
} else if let Some(exec) = plan.as_any().downcast_ref::<BoundedWindowAggExec>() {
12041204
if let Some(updated_window) = get_best_fitting_window(
12051205
exec.window_expr(),
12061206
exec.input(),
1207-
&exec.partition_keys,
1207+
&exec.partition_keys(),
12081208
)? {
12091209
plan = updated_window;
12101210
}

datafusion/physical-optimizer/src/enforce_sorting/mod.rs

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -460,12 +460,12 @@ fn adjust_window_sort_removal(
460460
if let Some(exec) = plan.downcast_ref::<WindowAggExec>() {
461461
let window_expr = exec.window_expr();
462462
let new_window =
463-
get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?;
463+
get_best_fitting_window(window_expr, child_plan, &exec.partition_keys())?;
464464
(window_expr, new_window)
465465
} else if let Some(exec) = plan.downcast_ref::<BoundedWindowAggExec>() {
466466
let window_expr = exec.window_expr();
467467
let new_window =
468-
get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?;
468+
get_best_fitting_window(window_expr, child_plan, &exec.partition_keys())?;
469469
(window_expr, new_window)
470470
} else {
471471
return plan_err!("Expected WindowAggExec or BoundedWindowAggExec");
@@ -493,14 +493,14 @@ fn adjust_window_sort_removal(
493493
Arc::new(BoundedWindowAggExec::try_new(
494494
window_expr.to_vec(),
495495
child_plan,
496-
window_expr[0].partition_by().to_vec(),
497496
InputOrderMode::Sorted,
497+
!window_expr[0].partition_by().is_empty(),
498498
)?) as _
499499
} else {
500500
Arc::new(WindowAggExec::try_new(
501501
window_expr.to_vec(),
502502
child_plan,
503-
window_expr[0].partition_by().to_vec(),
503+
!window_expr[0].partition_by().is_empty(),
504504
)?) as _
505505
}
506506
};

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

Lines changed: 26 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -78,8 +78,6 @@ pub struct BoundedWindowAggExec {
7878
window_expr: Vec<Arc<dyn WindowExpr>>,
7979
/// Schema after the window is run
8080
schema: SchemaRef,
81-
/// Partition Keys
82-
pub partition_keys: Vec<Arc<dyn PhysicalExpr>>,
8381
/// Execution metrics
8482
metrics: ExecutionPlanMetricsSet,
8583
/// Describes how the input is ordered relative to the partition keys
@@ -93,15 +91,17 @@ pub struct BoundedWindowAggExec {
9391
ordered_partition_by_indices: Vec<usize>,
9492
/// Cache holding plan properties like equivalences, output partitioning etc.
9593
cache: PlanProperties,
94+
/// If `can_rerepartition` is false, partition_keys is always empty.
95+
can_repartition: bool,
9696
}
9797

9898
impl BoundedWindowAggExec {
9999
/// Create a new execution plan for window aggregates
100100
pub fn try_new(
101101
window_expr: Vec<Arc<dyn WindowExpr>>,
102102
input: Arc<dyn ExecutionPlan>,
103-
partition_keys: Vec<Arc<dyn PhysicalExpr>>,
104103
input_order_mode: InputOrderMode,
104+
can_repartition: bool,
105105
) -> Result<Self> {
106106
let schema = create_schema(&input.schema(), &window_expr)?;
107107
let schema = Arc::new(schema);
@@ -128,11 +128,11 @@ impl BoundedWindowAggExec {
128128
input,
129129
window_expr,
130130
schema,
131-
partition_keys,
132131
metrics: ExecutionPlanMetricsSet::new(),
133132
input_order_mode,
134133
ordered_partition_by_indices,
135134
cache,
135+
can_repartition,
136136
})
137137
}
138138

@@ -209,6 +209,23 @@ impl BoundedWindowAggExec {
209209
input.boundedness(),
210210
)
211211
}
212+
213+
pub fn partition_keys(&self) -> Vec<Arc<dyn PhysicalExpr>> {
214+
if !self.can_repartition {
215+
vec![]
216+
} else {
217+
let all_partition_keys = self
218+
.window_expr()
219+
.iter()
220+
.map(|expr| expr.partition_by().to_vec())
221+
.collect::<Vec<_>>();
222+
223+
all_partition_keys
224+
.into_iter()
225+
.min_by_key(|s| s.len())
226+
.unwrap_or_else(Vec::new)
227+
}
228+
}
212229
}
213230

214231
impl DisplayAs for BoundedWindowAggExec {
@@ -269,11 +286,11 @@ impl ExecutionPlan for BoundedWindowAggExec {
269286
}
270287

271288
fn required_input_distribution(&self) -> Vec<Distribution> {
272-
if self.partition_keys.is_empty() {
289+
if self.partition_keys().is_empty() {
273290
debug!("No partition defined for BoundedWindowAggExec!!!");
274291
vec![Distribution::SinglePartition]
275292
} else {
276-
vec![Distribution::HashPartitioned(self.partition_keys.clone())]
293+
vec![Distribution::HashPartitioned(self.partition_keys().clone())]
277294
}
278295
}
279296

@@ -288,8 +305,8 @@ impl ExecutionPlan for BoundedWindowAggExec {
288305
Ok(Arc::new(BoundedWindowAggExec::try_new(
289306
self.window_expr.clone(),
290307
Arc::clone(&children[0]),
291-
self.partition_keys.clone(),
292308
self.input_order_mode.clone(),
309+
self.can_repartition,
293310
)?))
294311
}
295312

@@ -1329,8 +1346,8 @@ mod tests {
13291346
false,
13301347
)?],
13311348
input,
1332-
partitionby_exprs,
13331349
input_order_mode,
1350+
true,
13341351
)?))
13351352
}
13361353

@@ -1610,8 +1627,8 @@ mod tests {
16101627
let physical_plan = BoundedWindowAggExec::try_new(
16111628
window_exprs,
16121629
memory_exec,
1613-
vec![],
16141630
InputOrderMode::Sorted,
1631+
true,
16151632
)
16161633
.map(|e| Arc::new(e) as Arc<dyn ExecutionPlan>)?;
16171634

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -425,8 +425,8 @@ pub fn get_best_fitting_window(
425425
Ok(Some(Arc::new(BoundedWindowAggExec::try_new(
426426
window_expr,
427427
Arc::clone(input),
428-
physical_partition_keys.to_vec(),
429428
input_order_mode,
429+
!physical_partition_keys.is_empty(),
430430
)?) as _))
431431
} else if input_order_mode != InputOrderMode::Sorted {
432432
// For `WindowAggExec` to work correctly PARTITION BY columns should be sorted.
@@ -438,7 +438,7 @@ pub fn get_best_fitting_window(
438438
Ok(Some(Arc::new(WindowAggExec::try_new(
439439
window_expr,
440440
Arc::clone(input),
441-
physical_partition_keys.to_vec(),
441+
!physical_partition_keys.is_empty(),
442442
)?) as _))
443443
}
444444
}
@@ -663,7 +663,7 @@ mod tests {
663663
false,
664664
)?],
665665
blocking_exec,
666-
vec![],
666+
false,
667667
)?);
668668

669669
let fut = collect(window_agg_exec, task_ctx);

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

Lines changed: 24 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -57,23 +57,23 @@ pub struct WindowAggExec {
5757
window_expr: Vec<Arc<dyn WindowExpr>>,
5858
/// Schema after the window is run
5959
schema: SchemaRef,
60-
/// Partition Keys
61-
pub partition_keys: Vec<Arc<dyn PhysicalExpr>>,
6260
/// Execution metrics
6361
metrics: ExecutionPlanMetricsSet,
6462
/// Partition by indices that defines preset for existing ordering
6563
// see `get_ordered_partition_by_indices` for more details.
6664
ordered_partition_by_indices: Vec<usize>,
6765
/// Cache holding plan properties like equivalences, output partitioning etc.
6866
cache: PlanProperties,
67+
/// If `can_partition` is false, partition_keys is always empty.
68+
can_repartition: bool,
6969
}
7070

7171
impl WindowAggExec {
7272
/// Create a new execution plan for window aggregates
7373
pub fn try_new(
7474
window_expr: Vec<Arc<dyn WindowExpr>>,
7575
input: Arc<dyn ExecutionPlan>,
76-
partition_keys: Vec<Arc<dyn PhysicalExpr>>,
76+
can_repartition: bool,
7777
) -> Result<Self> {
7878
let schema = create_schema(&input.schema(), &window_expr)?;
7979
let schema = Arc::new(schema);
@@ -85,10 +85,10 @@ impl WindowAggExec {
8585
input,
8686
window_expr,
8787
schema,
88-
partition_keys,
8988
metrics: ExecutionPlanMetricsSet::new(),
9089
ordered_partition_by_indices,
9190
cache,
91+
can_repartition,
9292
})
9393
}
9494

@@ -139,6 +139,23 @@ impl WindowAggExec {
139139
input.boundedness(),
140140
)
141141
}
142+
143+
pub fn partition_keys(&self) -> Vec<Arc<dyn PhysicalExpr>> {
144+
if !self.can_repartition {
145+
vec![]
146+
} else {
147+
let all_partition_keys = self
148+
.window_expr()
149+
.iter()
150+
.map(|expr| expr.partition_by().to_vec())
151+
.collect::<Vec<_>>();
152+
153+
all_partition_keys
154+
.into_iter()
155+
.min_by_key(|s| s.len())
156+
.unwrap_or_else(Vec::new)
157+
}
158+
}
142159
}
143160

144161
impl DisplayAs for WindowAggExec {
@@ -206,10 +223,10 @@ impl ExecutionPlan for WindowAggExec {
206223
}
207224

208225
fn required_input_distribution(&self) -> Vec<Distribution> {
209-
if self.partition_keys.is_empty() {
226+
if self.partition_keys().is_empty() {
210227
vec![Distribution::SinglePartition]
211228
} else {
212-
vec![Distribution::HashPartitioned(self.partition_keys.clone())]
229+
vec![Distribution::HashPartitioned(self.partition_keys())]
213230
}
214231
}
215232

@@ -220,7 +237,7 @@ impl ExecutionPlan for WindowAggExec {
220237
Ok(Arc::new(WindowAggExec::try_new(
221238
self.window_expr.clone(),
222239
Arc::clone(&children[0]),
223-
self.partition_keys.clone(),
240+
true,
224241
)?))
225242
}
226243

0 commit comments

Comments
 (0)