forked from apache/datafusion-ballista
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathplanner.rs
More file actions
1154 lines (1041 loc) · 45.1 KB
/
planner.rs
File metadata and controls
1154 lines (1041 loc) · 45.1 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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.
//! Distributed query execution
use std::collections::HashMap;
use std::sync::Arc;
use ballista_core::config::BallistaConfig;
use ballista_core::error::{BallistaError, Result};
use ballista_core::execution_plans::ShuffleWriter;
use ballista_core::execution_plans::sort_shuffle::SortShuffleConfig;
use ballista_core::{
execution_plans::{
ShuffleReaderExec, ShuffleWriterExec, SortShuffleWriterExec,
UnresolvedShuffleExec,
},
serde::scheduler::PartitionLocation,
};
use datafusion::common::JoinType;
use datafusion::common::tree_node::{Transformed, TreeNode};
use datafusion::config::ConfigOptions;
use datafusion::physical_optimizer::PhysicalOptimizerRule;
use datafusion::physical_optimizer::enforce_sorting::EnforceSorting;
use datafusion::physical_plan::coalesce_partitions::CoalescePartitionsExec;
use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode};
use datafusion::physical_plan::repartition::RepartitionExec;
use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec;
use datafusion::physical_plan::{
ExecutionPlan, Partitioning, with_new_children_if_necessary,
};
use log::{debug, info};
type PartialQueryStageResult = (Arc<dyn ExecutionPlan>, Vec<Arc<dyn ShuffleWriter>>);
/// Trait for breaking an execution plan into distributed query stages.
///
/// The planner creates a DAG of stages where each stage can be executed
/// independently once its input stages are complete.
pub trait DistributedPlanner {
/// Returns a vector of ExecutionPlans, where the root node is a [`ShuffleWriter`].
///
/// Plans that depend on the input of other plans will have leaf nodes of type
/// [`UnresolvedShuffleExec`]. A shuffle writer is created whenever the
/// partitioning changes.
fn plan_query_stages<'a>(
&'a mut self,
job_id: &'a str,
execution_plan: Arc<dyn ExecutionPlan>,
config: &ConfigOptions,
) -> Result<Vec<Arc<dyn ShuffleWriter>>>;
}
/// Default implementation of [`DistributedPlanner`].
///
/// Breaks execution plans into stages at shuffle boundaries (repartition, coalesce).
pub struct DefaultDistributedPlanner {
/// Counter for generating unique stage IDs.
next_stage_id: usize,
/// Optimizer rule for enforcing sort requirements after stage splitting.
optimizer_enforce_sorting: EnforceSorting,
}
impl DefaultDistributedPlanner {
/// Creates a new `DefaultDistributedPlanner`.
pub fn new() -> Self {
Self {
next_stage_id: 0,
// when plan is broken into stages some sorting information may get lost in the process
// thus stage re-optimisation is needed to adjust sort information
optimizer_enforce_sorting:
datafusion::physical_optimizer::enforce_sorting::EnforceSorting::default(),
}
}
}
impl Default for DefaultDistributedPlanner {
fn default() -> Self {
Self::new()
}
}
impl DistributedPlanner for DefaultDistributedPlanner {
/// Returns a vector of ExecutionPlans, where the root node is a shuffle writer.
/// Plans that depend on the input of other plans will have leaf nodes of type [UnresolvedShuffleExec].
/// A shuffle writer is created whenever the partitioning changes.
fn plan_query_stages<'a>(
&'a mut self,
job_id: &'a str,
execution_plan: Arc<dyn ExecutionPlan>,
config: &ConfigOptions,
) -> Result<Vec<Arc<dyn ShuffleWriter>>> {
info!("planning query stages for job {job_id}");
// Safety: revert CollectLeft to Partitioned for non-INNER joins.
// CollectLeft broadcasts the build side to all partitions, but in
// distributed mode each executor independently emits "unmatched"
// build rows for outer/semi/anti joins, producing duplicates.
// See https://github.com/apache/datafusion-ballista/issues/1055
let execution_plan = revert_non_inner_collect_left(execution_plan)?;
let (new_plan, mut stages) =
self.plan_query_stages_internal(job_id, execution_plan, config)?;
stages.push(create_shuffle_writer_with_config(
job_id,
self.next_stage_id(),
new_plan,
None,
config,
)?);
Ok(stages)
}
}
impl DefaultDistributedPlanner {
/// Returns a potentially modified version of the input execution_plan along with the resulting query stages.
/// This function is needed because the input execution_plan might need to be modified, but it might not hold a
/// complete query stage (its parent might also belong to the same stage)
fn plan_query_stages_internal<'a>(
&'a mut self,
job_id: &'a str,
execution_plan: Arc<dyn ExecutionPlan>,
config: &ConfigOptions,
) -> Result<PartialQueryStageResult> {
// recurse down and replace children
if execution_plan.children().is_empty() {
return Ok((execution_plan, vec![]));
}
let mut stages = vec![];
let mut children = vec![];
for child in execution_plan.children() {
let (new_child, mut child_stages) =
self.plan_query_stages_internal(job_id, child.clone(), config)?;
children.push(new_child);
stages.append(&mut child_stages);
}
if let Some(_coalesce) = execution_plan
.as_any()
.downcast_ref::<CoalescePartitionsExec>()
{
let input = children[0].clone();
let input = self.optimizer_enforce_sorting.optimize(input, config)?;
let shuffle_writer = create_shuffle_writer_with_config(
job_id,
self.next_stage_id(),
input,
None,
config,
)?;
let unresolved_shuffle = create_unresolved_shuffle(shuffle_writer.as_ref());
stages.push(shuffle_writer);
Ok((
with_new_children_if_necessary(execution_plan, vec![unresolved_shuffle])?,
stages,
))
} else if let Some(sort_preserving_merge) = execution_plan
.as_any()
.downcast_ref::<SortPreservingMergeExec>(
) {
// For TopK queries (SortPreservingMergeExec with a small fetch/limit),
// skip the stage break and keep the merge in the same stage as its children.
// This avoids the overhead of shuffle write/read for a small number of rows,
// which dominates execution time for TopK queries in distributed mode.
//
// Note on parallelism: because SortPreservingMergeExec has an output
// partitioning of 1, the entire stage becomes a single task assigned to
// one executor (ShuffleWriterExec::input_partition_count() == 1).
// This does sacrifice cluster-level parallelism (no cross-executor
// distribution). However, within that executor the child partitions
// still execute as parallel async streams, so intra-executor parallelism
// is preserved. For small fetch values this trade-off is worthwhile as
// the shuffle coordination overhead far exceeds the merge cost.
const TOPK_FETCH_THRESHOLD: usize = 1000;
if sort_preserving_merge
.fetch()
.is_some_and(|f| f <= TOPK_FETCH_THRESHOLD)
{
Ok((
with_new_children_if_necessary(execution_plan, children)?,
stages,
))
} else {
let shuffle_writer = create_shuffle_writer_with_config(
job_id,
self.next_stage_id(),
children[0].clone(),
None,
config,
)?;
let unresolved_shuffle =
create_unresolved_shuffle(shuffle_writer.as_ref());
stages.push(shuffle_writer);
Ok((
with_new_children_if_necessary(
execution_plan,
vec![unresolved_shuffle],
)?,
stages,
))
}
} else if let Some(repart) =
execution_plan.as_any().downcast_ref::<RepartitionExec>()
{
match repart.properties().output_partitioning() {
Partitioning::Hash(_, _) => {
let input = children[0].clone();
let input = self.optimizer_enforce_sorting.optimize(input, config)?;
let shuffle_writer = create_shuffle_writer_with_config(
job_id,
self.next_stage_id(),
input,
Some(repart.partitioning().to_owned()),
config,
)?;
let unresolved_shuffle =
create_unresolved_shuffle(shuffle_writer.as_ref());
stages.push(shuffle_writer);
Ok((unresolved_shuffle, stages))
}
_ => {
// remove any non-hash repartition from the distributed plan
Ok((children[0].clone(), stages))
}
}
} else {
Ok((
with_new_children_if_necessary(execution_plan, children)?,
stages,
))
}
}
/// Generate a new stage ID
fn next_stage_id(&mut self) -> usize {
self.next_stage_id += 1;
self.next_stage_id
}
}
fn create_unresolved_shuffle(
shuffle_writer: &dyn ShuffleWriter,
) -> Arc<UnresolvedShuffleExec> {
Arc::new(UnresolvedShuffleExec::new(
shuffle_writer.stage_id(),
shuffle_writer.schema(),
shuffle_writer.properties().output_partitioning().clone(),
))
}
/// Revert CollectLeft hash joins to Partitioned for non-INNER join types.
///
/// In distributed execution, CollectLeft broadcasts the build side to every
/// partition/task. For INNER joins this is correct — each task independently
/// produces matching rows. For LEFT/RIGHT/FULL OUTER and SEMI/ANTI joins,
/// each task would independently emit "unmatched" build rows, producing
/// duplicate or incorrect results across executors.
///
/// This replaces non-INNER CollectLeft joins with Partitioned joins and
/// inserts the necessary RepartitionExec(Hash) nodes on both sides so the
/// distributed planner creates proper shuffle stages.
fn revert_non_inner_collect_left(
plan: Arc<dyn ExecutionPlan>,
) -> Result<Arc<dyn ExecutionPlan>> {
plan.transform_up(|node| {
if let Some(hash_join) = node.as_any().downcast_ref::<HashJoinExec>()
&& *hash_join.partition_mode() == PartitionMode::CollectLeft
&& hash_join.join_type() != &JoinType::Inner
{
info!(
"Reverting CollectLeft to Partitioned for {:?} join (bug #1055 workaround)",
hash_join.join_type()
);
let left = hash_join.left().clone();
let right = hash_join.right().clone();
let on = hash_join.on();
let partition_count = right
.properties()
.output_partitioning()
.partition_count();
// Build hash expressions for both sides from join keys
let left_exprs: Vec<Arc<dyn datafusion::physical_expr::PhysicalExpr>> =
on.iter().map(|(l, _)| l.clone()).collect();
let right_exprs: Vec<Arc<dyn datafusion::physical_expr::PhysicalExpr>> =
on.iter().map(|(_, r)| r.clone()).collect();
let left_repart = Arc::new(RepartitionExec::try_new(
left,
Partitioning::Hash(left_exprs, partition_count),
)?) as Arc<dyn ExecutionPlan>;
let right_repart = Arc::new(RepartitionExec::try_new(
right,
Partitioning::Hash(right_exprs, partition_count),
)?) as Arc<dyn ExecutionPlan>;
let new_join = Arc::new(HashJoinExec::try_new(
left_repart,
right_repart,
on.to_vec(),
hash_join.filter().cloned(),
hash_join.join_type(),
hash_join.projection.clone(),
PartitionMode::Partitioned,
hash_join.null_equality(),
)?);
return Ok(Transformed::yes(new_join as Arc<dyn ExecutionPlan>));
}
Ok(Transformed::no(node))
})
.map(|t| t.data)
.map_err(|e| BallistaError::DataFusionError(Box::new(e)))
}
/// Returns all unresolved shuffle nodes in the execution plan.
///
/// Used to identify which input stages a plan depends on.
pub fn find_unresolved_shuffles(
plan: &Arc<dyn ExecutionPlan>,
) -> Result<Vec<UnresolvedShuffleExec>> {
if let Some(unresolved_shuffle) =
plan.as_any().downcast_ref::<UnresolvedShuffleExec>()
{
Ok(vec![unresolved_shuffle.clone()])
} else {
Ok(plan
.children()
.into_iter()
.map(find_unresolved_shuffles)
.collect::<Result<Vec<_>>>()?
.into_iter()
.flatten()
.collect())
}
}
/// Replaces [`UnresolvedShuffleExec`] nodes with [`ShuffleReaderExec`] nodes.
///
/// Called after input stages complete to connect stages with their actual partition locations.
pub fn remove_unresolved_shuffles(
stage: Arc<dyn ExecutionPlan>,
partition_locations: &HashMap<usize, HashMap<usize, Vec<PartitionLocation>>>,
) -> Result<Arc<dyn ExecutionPlan>> {
let mut new_children: Vec<Arc<dyn ExecutionPlan>> = vec![];
for child in stage.children() {
if let Some(unresolved_shuffle) =
child.as_any().downcast_ref::<UnresolvedShuffleExec>()
{
let mut relevant_locations = vec![];
let p = partition_locations
.get(&unresolved_shuffle.stage_id)
.ok_or_else(|| {
BallistaError::General(
"Missing partition location. Could not remove unresolved shuffles"
.to_owned(),
)
})?
.clone();
for i in 0..unresolved_shuffle.output_partition_count {
if let Some(x) = p.get(&i) {
relevant_locations.push(x.to_owned());
} else {
relevant_locations.push(vec![]);
}
}
debug!(
"Creating shuffle reader: {}",
relevant_locations
.iter()
.map(|c| c
.iter()
.filter(|l| !l.path.is_empty())
.map(|l| l.path.clone())
.collect::<Vec<_>>()
.join(", "))
.collect::<Vec<_>>()
.join("\n")
);
new_children.push(Arc::new(ShuffleReaderExec::try_new(
unresolved_shuffle.stage_id,
relevant_locations,
unresolved_shuffle.schema().clone(),
unresolved_shuffle
.properties()
.output_partitioning()
.clone(),
)?))
} else {
new_children.push(remove_unresolved_shuffles(
child.clone(),
partition_locations,
)?);
}
}
Ok(with_new_children_if_necessary(stage, new_children)?)
}
/// Rollback the ShuffleReaderExec to UnresolvedShuffleExec.
/// Used when the input stages are finished but some partitions are missing due to executor lost.
/// The entire stage need to be rolled back and rescheduled.
pub fn rollback_resolved_shuffles(
stage: Arc<dyn ExecutionPlan>,
) -> Result<Arc<dyn ExecutionPlan>> {
let mut new_children: Vec<Arc<dyn ExecutionPlan>> = vec![];
for child in stage.children() {
if let Some(shuffle_reader) = child.as_any().downcast_ref::<ShuffleReaderExec>() {
let stage_id = shuffle_reader.stage_id;
let unresolved_shuffle = Arc::new(UnresolvedShuffleExec::new(
stage_id,
shuffle_reader.schema(),
shuffle_reader.properties().partitioning.clone(),
));
new_children.push(unresolved_shuffle);
} else {
new_children.push(rollback_resolved_shuffles(child.clone())?);
}
}
Ok(with_new_children_if_necessary(stage, new_children)?)
}
fn create_shuffle_writer_with_config(
job_id: &str,
stage_id: usize,
plan: Arc<dyn ExecutionPlan>,
partitioning: Option<Partitioning>,
config: &ConfigOptions,
) -> Result<Arc<dyn ShuffleWriter>> {
// Check if sort-based shuffle is enabled
let ballista_config = config
.extensions
.get::<BallistaConfig>()
.cloned()
.unwrap_or_default();
if ballista_config.shuffle_sort_based_enabled() {
// Sort shuffle requires hash partitioning
if let Some(Partitioning::Hash(exprs, partition_count)) = partitioning {
let sort_config = SortShuffleConfig::new(
true,
ballista_config.shuffle_sort_based_buffer_size(),
ballista_config.shuffle_sort_based_memory_limit(),
ballista_config.shuffle_sort_based_spill_threshold(),
datafusion::arrow::ipc::CompressionType::LZ4_FRAME,
ballista_config.shuffle_sort_based_batch_size(),
);
return Ok(Arc::new(SortShuffleWriterExec::try_new(
job_id.to_owned(),
stage_id,
plan,
"".to_owned(),
Partitioning::Hash(exprs, partition_count),
sort_config,
)?));
}
}
// Fall back to standard shuffle writer
Ok(Arc::new(ShuffleWriterExec::try_new(
job_id.to_owned(),
stage_id,
plan,
"".to_owned(),
partitioning,
)?))
}
#[cfg(test)]
mod test {
use crate::planner::{
DefaultDistributedPlanner, DistributedPlanner, revert_non_inner_collect_left,
};
use crate::test_utils::datafusion_test_context;
use ballista_core::error::BallistaError;
use ballista_core::execution_plans::{ShuffleWriterExec, UnresolvedShuffleExec};
use ballista_core::serde::BallistaCodec;
use datafusion::arrow::compute::SortOptions;
use datafusion::arrow::datatypes::{DataType, Field, Schema};
use datafusion::common::{JoinType, NullEquality};
use datafusion::execution::TaskContext;
use datafusion::physical_expr::expressions::Column;
use datafusion::physical_plan::aggregates::{AggregateExec, AggregateMode};
use datafusion::physical_plan::empty::EmptyExec;
use datafusion::physical_plan::filter::FilterExec;
use datafusion::physical_plan::joins::{HashJoinExec, PartitionMode};
use datafusion::physical_plan::projection::ProjectionExec;
use datafusion::physical_plan::repartition::RepartitionExec;
use datafusion::physical_plan::sorts::sort::SortExec;
use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec;
use datafusion::physical_plan::windows::BoundedWindowAggExec;
use datafusion::physical_plan::{ExecutionPlan, displayable};
use datafusion::physical_plan::{InputOrderMode, Partitioning};
use datafusion_proto::physical_plan::AsExecutionPlan;
use datafusion_proto::protobuf::LogicalPlanNode;
use datafusion_proto::protobuf::PhysicalPlanNode;
use std::sync::Arc;
use uuid::Uuid;
macro_rules! downcast_exec {
($exec: expr, $ty: ty) => {
$exec.as_any().downcast_ref::<$ty>().expect(&format!(
"Downcast to {} failed. Got {:?}",
stringify!($ty),
$exec
))
};
}
#[tokio::test]
async fn distributed_aggregate_plan() -> Result<(), BallistaError> {
let ctx = datafusion_test_context("testdata").await?;
let session_state = ctx.state();
// simplified form of TPC-H query 1
let df = ctx
.sql(
"select l_returnflag, sum(l_extendedprice * 1) as sum_disc_price
from lineitem
group by l_returnflag
order by l_returnflag",
)
.await?;
let plan = df.into_optimized_plan()?;
let plan = session_state.optimize(&plan)?;
let plan = session_state.create_physical_plan(&plan).await?;
let mut planner = DefaultDistributedPlanner::new();
let job_uuid = Uuid::new_v4();
let stages = planner.plan_query_stages(
&job_uuid.to_string(),
plan,
ctx.state().config().options(),
)?;
for (i, stage) in stages.iter().enumerate() {
println!("Stage {i}:\n{}", displayable(stage.as_ref()).indent(false));
}
/* Expected result:
ShuffleWriterExec: partitioning: Hash([l_returnflag@0], 2)
AggregateExec: mode=Partial, gby=[l_returnflag@1 as l_returnflag], aggr=[sum(lineitem.l_extendedprice * Int64(1))]
DataSourceExec: file_groups={2 groups: [[ballista/scheduler/testdata/lineitem/partition0.tbl], [ballista/scheduler/testdata/lineitem/partition1.tbl]]}, projection=[l_extendedprice, l_returnflag], file_type=csv, has_header=false
ShuffleWriterExec: partitioning: None
SortExec: expr=[l_returnflag@0 ASC NULLS LAST], preserve_partitioning=[true]
ProjectionExec: expr=[l_returnflag@0 as l_returnflag, sum(lineitem.l_extendedprice * Int64(1))@1 as sum_disc_price]
AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag], aggr=[sum(lineitem.l_extendedprice * Int64(1))]
UnresolvedShuffleExec: partitioning: Hash([l_returnflag@0], 2)
ShuffleWriterExec: partitioning: None
SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST]
UnresolvedShuffleExec: partitioning: Hash([l_returnflag@0], 2)
*/
assert_eq!(3, stages.len());
// verify stage 0
let stage0 = stages[0].children()[0].clone();
let partial_hash = downcast_exec!(stage0, AggregateExec);
assert!(*partial_hash.mode() == AggregateMode::Partial);
// verify stage 1
let stage1 = stages[1].children()[0].clone();
let sort = downcast_exec!(stage1, SortExec);
let projection = sort.children()[0].clone();
let projection = downcast_exec!(projection, ProjectionExec);
let final_hash = projection.children()[0].clone();
let final_hash = downcast_exec!(final_hash, AggregateExec);
assert!(*final_hash.mode() == AggregateMode::FinalPartitioned);
let unresolved_shuffle = final_hash.children()[0].clone();
let unresolved_shuffle =
downcast_exec!(unresolved_shuffle, UnresolvedShuffleExec);
assert_eq!(unresolved_shuffle.stage_id, 1);
assert_eq!(unresolved_shuffle.output_partition_count, 2);
assert_eq!(
unresolved_shuffle.properties().partitioning,
Partitioning::Hash(vec![Arc::new(Column::new("l_returnflag", 0))], 2)
);
// verify stage 2
let stage2 = stages[2].children()[0].clone();
let merge = downcast_exec!(stage2, SortPreservingMergeExec);
let unresolved_shuffle = merge.children()[0].clone();
let unresolved_shuffle =
downcast_exec!(unresolved_shuffle, UnresolvedShuffleExec);
assert_eq!(unresolved_shuffle.stage_id, 2);
assert_eq!(unresolved_shuffle.output_partition_count, 2);
assert_eq!(
unresolved_shuffle.properties().partitioning,
Partitioning::Hash(vec![Arc::new(Column::new("l_returnflag", 0))], 2)
);
Ok(())
}
#[tokio::test]
async fn distributed_join_plan() -> Result<(), BallistaError> {
let ctx = datafusion_test_context("testdata").await?;
let session_state = ctx.state();
// simplified form of TPC-H query 12
let df = ctx
.sql(
"select
l_shipmode,
sum(case
when o_orderpriority = '1-URGENT'
or o_orderpriority = '2-HIGH'
then 1
else 0
end) as high_line_count,
sum(case
when o_orderpriority <> '1-URGENT'
and o_orderpriority <> '2-HIGH'
then 1
else 0
end) as low_line_count
from
lineitem
join
orders
on
l_orderkey = o_orderkey
where
l_shipmode in ('MAIL', 'SHIP')
and l_commitdate < l_receiptdate
and l_shipdate < l_commitdate
and l_receiptdate >= date '1994-01-01'
and l_receiptdate < date '1995-01-01'
group by
l_shipmode
order by
l_shipmode;
",
)
.await?;
let plan = df.into_optimized_plan()?;
let plan = session_state.optimize(&plan)?;
let plan = session_state.create_physical_plan(&plan).await?;
let mut planner = DefaultDistributedPlanner::new();
let job_uuid = Uuid::new_v4();
let stages = planner.plan_query_stages(
&job_uuid.to_string(),
plan,
ctx.state().config().options(),
)?;
for (i, stage) in stages.iter().enumerate() {
println!("Stage {i}:\n{}", displayable(stage.as_ref()).indent(false));
}
/* Expected result:
ShuffleWriterExec: partitioning: Hash([l_orderkey@0], 2)
FilterExec: (l_shipmode@4 = MAIL OR l_shipmode@4 = SHIP) AND l_receiptdate@3 > l_commitdate@2 AND l_shipdate@1 < l_commitdate@2 AND l_receiptdate@3 >= 1994-01-01 AND l_receiptdate@3 < 1995-01-01, projection=[l_orderkey@0, l_shipmode@4]
DataSourceExec: file_groups={2 groups: [[ballista/scheduler/testdata/lineitem/partition0.tbl], [ballista/scheduler/testdata/lineitem/partition1.tbl]]}, projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], file_type=csv, has_header=false
ShuffleWriterExec: partitioning: Hash([o_orderkey@0], 2)
DataSourceExec: file_groups={1 group: [[ballista/scheduler/testdata/orders/orders.tbl]]}, projection=[o_orderkey, o_orderpriority], file_type=csv, has_header=false
ShuffleWriterExec: partitioning: Hash([l_shipmode@0], 2)
AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]
HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3]
UnresolvedShuffleExec: partitioning: Hash([l_orderkey@0], 2)
UnresolvedShuffleExec: partitioning: Hash([o_orderkey@0], 2)
ShuffleWriterExec: partitioning: None
SortExec: expr=[l_shipmode@0 ASC NULLS LAST], preserve_partitioning=[true]
ProjectionExec: expr=[l_shipmode@0 as l_shipmode, sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count]
AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[sum(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), sum(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]
UnresolvedShuffleExec: partitioning: Hash([l_shipmode@0], 2)
ShuffleWriterExec: partitioning: None
SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST]
UnresolvedShuffleExec: partitioning: Hash([l_shipmode@0], 2)
*/
assert_eq!(5, stages.len());
// verify partitioning for each stage
// csv "lineitem" (2 files)
assert_eq!(
2,
stages[0].children()[0]
.properties()
.output_partitioning()
.partition_count()
);
assert_eq!(
2,
stages[0]
.shuffle_output_partitioning()
.expect("stage 0")
.partition_count()
);
// csv "orders" (1 file)
assert_eq!(
1,
stages[1].children()[0]
.properties()
.output_partitioning()
.partition_count()
);
assert_eq!(
2,
stages[1]
.shuffle_output_partitioning()
.expect("stage 1")
.partition_count()
);
// join and partial hash aggregate
let input = stages[2].children()[0].clone();
assert_eq!(
2,
input.properties().output_partitioning().partition_count()
);
assert_eq!(
2,
stages[2]
.shuffle_output_partitioning()
.expect("stage 2")
.partition_count()
);
let hash_agg = downcast_exec!(input, AggregateExec);
let join = hash_agg.children()[0].clone();
let join = downcast_exec!(join, HashJoinExec);
assert!(join.contains_projection());
let join_input_1 = join.children()[0].clone();
let unresolved_shuffle_reader_1 =
downcast_exec!(join_input_1, UnresolvedShuffleExec);
assert_eq!(unresolved_shuffle_reader_1.output_partition_count, 2);
assert_eq!(
unresolved_shuffle_reader_1.properties().partitioning,
Partitioning::Hash(vec![Arc::new(Column::new("l_orderkey", 0))], 2)
);
let join_input_2 = join.children()[1].clone();
let unresolved_shuffle_reader_2 =
downcast_exec!(join_input_2, UnresolvedShuffleExec);
assert_eq!(unresolved_shuffle_reader_2.output_partition_count, 2);
assert_eq!(
unresolved_shuffle_reader_2.properties().partitioning,
Partitioning::Hash(vec![Arc::new(Column::new("o_orderkey", 0))], 2)
);
// final partitioned hash aggregate
assert_eq!(
2,
stages[3].children()[0]
.properties()
.output_partitioning()
.partition_count()
);
assert!(stages[3].shuffle_output_partitioning().is_none());
// coalesce partitions and sort
assert_eq!(
1,
stages[4].children()[0]
.properties()
.output_partitioning()
.partition_count()
);
assert!(stages[4].shuffle_output_partitioning().is_none());
Ok(())
}
#[tokio::test]
async fn distributed_window_plan() -> Result<(), BallistaError> {
let ctx = datafusion_test_context("testdata").await?;
let session_state = ctx.state();
// simplified form of TPC-DS query 67
let df = ctx
.sql(
"
select * from (
select
l_shipmode,
l_shipdate,
rank() over (partition by l_shipmode order by l_shipdate desc) rk
from lineitem
) alias1
where rk <= 100 order by l_shipdate, rk;
",
)
.await?;
let plan = df.into_optimized_plan()?;
let plan = session_state.optimize(&plan)?;
let plan = session_state.create_physical_plan(&plan).await?;
let mut planner = DefaultDistributedPlanner::new();
let job_uuid = Uuid::new_v4();
let stages = planner.plan_query_stages(
&job_uuid.to_string(),
plan,
ctx.state().config().options(),
)?;
for (i, stage) in stages.iter().enumerate() {
println!("Stage {i}:\n{}", displayable(stage.as_ref()).indent(false));
}
/*
expected result:
Stage 0:
ShuffleWriterExec: partitioning: Hash([l_shipmode@1], 2)
DataSourceExec: file_groups={2 groups: [[ballista/scheduler/testdata/lineitem/partition0.tbl], [ballista/scheduler/testdata/lineitem/partition1.tbl]]}, projection=[l_shipdate, l_shipmode], file_type=csv, has_header=false
Stage 1:
ShuffleWriterExec: partitioning: None
SortExec: expr=[l_shipdate@1 ASC NULLS LAST, rk@2 ASC NULLS LAST], preserve_partitioning=[true]
ProjectionExec: expr=[l_shipmode@1 as l_shipmode, l_shipdate@0 as l_shipdate, rank() PARTITION BY [lineitem.l_shipmode] ORDER BY [lineitem.l_shipdate DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rk]
FilterExec: rank() PARTITION BY [lineitem.l_shipmode] ORDER BY [lineitem.l_shipdate DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 <= 100
BoundedWindowAggExec: wdw=[rank() PARTITION BY [lineitem.l_shipmode] ORDER BY [lineitem.l_shipdate DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Field { "rank() PARTITION BY [lineitem.l_shipmode] ORDER BY [lineitem.l_shipdate DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW": UInt64 }, frame: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], mode=[Sorted]
SortExec: expr=[l_shipmode@1 ASC NULLS LAST, l_shipdate@0 DESC], preserve_partitioning=[true]
UnresolvedShuffleExec: partitioning: Hash([l_shipmode@1], 2)
Stage 2:
ShuffleWriterExec: partitioning: None
SortPreservingMergeExec: [l_shipdate@1 ASC NULLS LAST, rk@2 ASC NULLS LAST]
UnresolvedShuffleExec: partitioning: Hash([l_shipmode@0], 2)
*/
assert_eq!(3, stages.len());
// stage0
let stage0 = stages[0].clone();
let shuffle_write = downcast_exec!(stage0, ShuffleWriterExec);
let partitioning = shuffle_write.shuffle_output_partitioning().expect("stage0");
assert_eq!(2, partitioning.partition_count());
let partition_col = match partitioning {
Partitioning::Hash(exprs, 2) => match exprs.as_slice() {
[col] => col.as_any().downcast_ref::<Column>(),
_ => None,
},
_ => None,
};
assert_eq!(Some(&Column::new("l_shipmode", 1)), partition_col);
// stage1
let sort = downcast_exec!(stages[1].children()[0], SortExec);
let projection = downcast_exec!(sort.children()[0], ProjectionExec);
let filter = downcast_exec!(projection.children()[0], FilterExec);
let window = downcast_exec!(filter.children()[0], BoundedWindowAggExec);
let partition_by = window.partition_keys();
let partition_by = match partition_by[..] {
[ref col] => col.as_any().downcast_ref::<Column>(),
_ => None,
};
assert_eq!(Some(&Column::new("l_shipmode", 1)), partition_by);
assert_eq!(InputOrderMode::Sorted, window.input_order_mode);
let sort = downcast_exec!(window.children()[0], SortExec);
match &sort.expr().iter().collect::<Vec<_>>()[..] {
[expr1, expr2] => {
assert_eq!(
SortOptions {
descending: false,
nulls_first: false
},
expr1.options
);
assert_eq!(
Some(&Column::new("l_shipmode", 1)),
expr1.expr.as_any().downcast_ref()
);
assert_eq!(
SortOptions {
descending: true,
nulls_first: true
},
expr2.options
);
assert_eq!(
Some(&Column::new("l_shipdate", 0)),
expr2.expr.as_any().downcast_ref()
);
}
_ => panic!("invalid sort {sort:?}"),
};
Ok(())
}
#[tokio::test]
async fn roundtrip_serde_aggregate() -> Result<(), BallistaError> {
let ctx = datafusion_test_context("testdata").await?;
let session_state = ctx.state();
// simplified form of TPC-H query 1
let df = ctx
.sql(
"select l_returnflag, sum(l_extendedprice * 1) as sum_disc_price
from lineitem
group by l_returnflag
order by l_returnflag",
)
.await?;
let plan = df.into_optimized_plan()?;
let plan = session_state.optimize(&plan)?;
let plan = session_state.create_physical_plan(&plan).await?;
let mut planner = DefaultDistributedPlanner::new();
let job_uuid = Uuid::new_v4();
let stages = planner.plan_query_stages(
&job_uuid.to_string(),
plan,
ctx.state().config().options(),
)?;
let partial_hash = stages[0].children()[0].clone();
let partial_hash_serde =
roundtrip_operator(&ctx.task_ctx(), partial_hash.clone())?;
let partial_hash = downcast_exec!(partial_hash, AggregateExec);
let partial_hash_serde = downcast_exec!(partial_hash_serde, AggregateExec);
assert_eq!(
format!("{partial_hash:?}"),
format!("{partial_hash_serde:?}")
);
Ok(())
}
fn roundtrip_operator(
ctx: &TaskContext,
plan: Arc<dyn ExecutionPlan>,
) -> Result<Arc<dyn ExecutionPlan>, BallistaError> {
let codec: BallistaCodec<LogicalPlanNode, PhysicalPlanNode> =
BallistaCodec::default();
let proto: datafusion_proto::protobuf::PhysicalPlanNode =
datafusion_proto::protobuf::PhysicalPlanNode::try_from_physical_plan(
plan.clone(),
codec.physical_extension_codec(),
)?;
let result_exec_plan: Arc<dyn ExecutionPlan> =
(proto).try_into_physical_plan(ctx, codec.physical_extension_codec())?;
Ok(result_exec_plan)
}
/// Verifies that TopK queries (ORDER BY ... LIMIT N, where N is small)
/// do NOT create a stage break at SortPreservingMergeExec, avoiding
/// shuffle overhead for small result sets.
#[tokio::test]
async fn test_topk_avoids_stage_break() -> Result<(), BallistaError> {
use datafusion::prelude::{CsvReadOptions, SessionConfig, SessionContext};
use std::io::Write;
let tmp_dir = tempfile::tempdir().unwrap();
let schema = "id,value\n";
for i in 0..4 {
let path = tmp_dir.path().join(format!("part{i:02}.csv"));
let mut f = std::fs::File::create(&path).unwrap();
write!(f, "{schema}").unwrap();
for j in 0..10 {
writeln!(f, "{},{}", i * 10 + j, (i * 10 + j) * 100).unwrap();
}
}
let config = SessionConfig::new().with_target_partitions(4);
let ctx = SessionContext::new_with_config(config);
ctx.register_csv(
"test_table",
tmp_dir.path().to_str().unwrap(),
CsvReadOptions::new(),
)
.await?;
// TopK query with small LIMIT — should produce a single stage
let df = ctx