@@ -5,6 +5,7 @@ use datafusion::common::tree_node::TreeNodeRecursion;
55use datafusion:: datasource:: source:: DataSourceExec ;
66use datafusion:: error:: DataFusionError ;
77use datafusion:: physical_expr:: Partitioning ;
8+ use datafusion:: physical_plan:: ExecutionPlanProperties ;
89use datafusion:: physical_plan:: coalesce_partitions:: CoalescePartitionsExec ;
910use datafusion:: physical_plan:: joins:: { HashJoinExec , PartitionMode } ;
1011use datafusion:: physical_plan:: sorts:: sort_preserving_merge:: SortPreservingMergeExec ;
@@ -119,8 +120,14 @@ impl PhysicalOptimizerRule for DistributedPhysicalOptimizerRule {
119120impl DistributedPhysicalOptimizerRule {
120121 fn apply_network_boundaries (
121122 & self ,
122- plan : Arc < dyn ExecutionPlan > ,
123+ mut plan : Arc < dyn ExecutionPlan > ,
123124 ) -> Result < Arc < dyn ExecutionPlan > , DataFusionError > {
125+ if plan. output_partitioning ( ) . partition_count ( ) > 1 {
126+ // Coalescing partitions here will allow us to put a NetworkCoalesceExec on top
127+ // of the plan, executing it in parallel.
128+ plan = Arc :: new ( CoalescePartitionsExec :: new ( plan) )
129+ }
130+
124131 let result = plan. transform_up ( |plan| {
125132 // If this node is a DataSourceExec, we need to wrap it with PartitionIsolatorExec so
126133 // that not all tasks have access to all partitions of the underlying DataSource.
@@ -205,7 +212,7 @@ impl DistributedPhysicalOptimizerRule {
205212 ) -> Result < StageExec , DataFusionError > {
206213 let mut inputs = vec ! [ ] ;
207214
208- let distributed = plan. clone ( ) . transform_down ( |plan| {
215+ let mut distributed = plan. clone ( ) . transform_down ( |plan| {
209216 // We cannot break down CollectLeft hash joins into more than 1 task, as these need
210217 // a full materialized build size with all the data in it.
211218 //
@@ -268,6 +275,13 @@ impl DistributedPhysicalOptimizerRule {
268275 Ok ( Transformed :: new ( node, true , TreeNodeRecursion :: Jump ) )
269276 } ) ?;
270277
278+ // The head stage is executable, and upon execution, it will lazily assign worker URLs to
279+ // all tasks. This must only be done once, so the executable StageExec must only be called
280+ // once on 1 partition.
281+ if depth == 0 && distributed. data . output_partitioning ( ) . partition_count ( ) > 1 {
282+ distributed. data = Arc :: new ( CoalescePartitionsExec :: new ( distributed. data ) ) ;
283+ }
284+
271285 let inputs = inputs. into_iter ( ) . map ( Arc :: new) . collect ( ) ;
272286 let mut stage = StageExec :: new ( query_id, * num, distributed. data , inputs, n_tasks) ;
273287 * num += 1 ;
@@ -396,8 +410,9 @@ mod tests {
396410 let query = r#"SELECT * FROM weather"# ;
397411 let plan = sql_to_explain ( query, 1 ) . await . unwrap ( ) ;
398412 assert_snapshot ! ( plan, @r"
399- ┌───── Stage 1 Tasks: t0:[p0,p1,p2]
400- │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, MaxTemp, Rainfall, Evaporation, Sunshine, WindGustDir, WindGustSpeed, WindDir9am, WindDir3pm, WindSpeed9am, WindSpeed3pm, Humidity9am, Humidity3pm, Pressure9am, Pressure3pm, Cloud9am, Cloud3pm, Temp9am, Temp3pm, RainToday, RISK_MM, RainTomorrow], file_type=parquet
413+ ┌───── Stage 1 Tasks: t0:[p0]
414+ │ CoalescePartitionsExec
415+ │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, MaxTemp, Rainfall, Evaporation, Sunshine, WindGustDir, WindGustSpeed, WindDir9am, WindDir3pm, WindSpeed9am, WindSpeed3pm, Humidity9am, Humidity3pm, Pressure9am, Pressure3pm, Cloud9am, Cloud3pm, Temp9am, Temp3pm, RainToday, RISK_MM, RainTomorrow], file_type=parquet
401416 └──────────────────────────────────────────────────
402417 " ) ;
403418 }
@@ -463,12 +478,13 @@ mod tests {
463478 let query = r#"SELECT a."MinTemp", b."MaxTemp" FROM weather a LEFT JOIN weather b ON a."RainToday" = b."RainToday" "# ;
464479 let plan = sql_to_explain ( query, 2 ) . await . unwrap ( ) ;
465480 assert_snapshot ! ( plan, @r"
466- ┌───── Stage 1 Tasks: t0:[p0,p1,p2]
467- │ CoalesceBatchesExec: target_batch_size=8192
468- │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2]
469- │ CoalescePartitionsExec
470- │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet
471- │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet
481+ ┌───── Stage 1 Tasks: t0:[p0]
482+ │ CoalescePartitionsExec
483+ │ CoalesceBatchesExec: target_batch_size=8192
484+ │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainToday@1, RainToday@1)], projection=[MinTemp@0, MaxTemp@2]
485+ │ CoalescePartitionsExec
486+ │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MinTemp, RainToday], file_type=parquet
487+ │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[MaxTemp, RainToday], file_type=parquet
472488 └──────────────────────────────────────────────────
473489 " ) ;
474490 }
@@ -501,15 +517,16 @@ mod tests {
501517 "# ;
502518 let plan = sql_to_explain ( query, 2 ) . await . unwrap ( ) ;
503519 assert_snapshot ! ( plan, @r"
504- ┌───── Stage 4 Tasks: t0:[p0,p1,p2,p3]
505- │ CoalesceBatchesExec: target_batch_size=8192
506- │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainTomorrow@1, RainTomorrow@1)], projection=[MinTemp@0, MaxTemp@2]
507- │ CoalescePartitionsExec
508- │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=2
509- │ ProjectionExec: expr=[avg(weather.MaxTemp)@1 as MaxTemp, RainTomorrow@0 as RainTomorrow]
510- │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MaxTemp)]
511- │ CoalesceBatchesExec: target_batch_size=8192
512- │ NetworkShuffleExec read_from=Stage 3, output_partitions=4, n_tasks=1, input_tasks=2
520+ ┌───── Stage 4 Tasks: t0:[p0]
521+ │ CoalescePartitionsExec
522+ │ CoalesceBatchesExec: target_batch_size=8192
523+ │ HashJoinExec: mode=CollectLeft, join_type=Left, on=[(RainTomorrow@1, RainTomorrow@1)], projection=[MinTemp@0, MaxTemp@2]
524+ │ CoalescePartitionsExec
525+ │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=2
526+ │ ProjectionExec: expr=[avg(weather.MaxTemp)@1 as MaxTemp, RainTomorrow@0 as RainTomorrow]
527+ │ AggregateExec: mode=FinalPartitioned, gby=[RainTomorrow@0 as RainTomorrow], aggr=[avg(weather.MaxTemp)]
528+ │ CoalesceBatchesExec: target_batch_size=8192
529+ │ NetworkShuffleExec read_from=Stage 3, output_partitions=4, n_tasks=1, input_tasks=2
513530 └──────────────────────────────────────────────────
514531 ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3]
515532 │ ProjectionExec: expr=[avg(weather.MinTemp)@1 as MinTemp, RainTomorrow@0 as RainTomorrow]
@@ -560,18 +577,22 @@ mod tests {
560577 let query = r#"SELECT DISTINCT "RainToday", "WindGustDir" FROM weather"# ;
561578 let plan = sql_to_explain ( query, 2 ) . await . unwrap ( ) ;
562579 assert_snapshot ! ( plan, @r"
563- ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3]
564- │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[]
565- │ CoalesceBatchesExec: target_batch_size=8192
566- │ NetworkShuffleExec read_from=Stage 1, output_partitions=4, n_tasks=1, input_tasks=2
580+ ┌───── Stage 3 Tasks: t0:[p0]
581+ │ CoalescePartitionsExec
582+ │ NetworkCoalesceExec read_from=Stage 2, output_partitions=8, input_tasks=2
567583 └──────────────────────────────────────────────────
568- ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3]
569- │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 4), input_partitions=4
570- │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
571- │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[]
572- │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
573- │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet
584+ ┌───── Stage 2 Tasks: t0:[p0,p1,p2,p3] t1:[p0,p1,p2,p3]
585+ │ AggregateExec: mode=FinalPartitioned, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[]
586+ │ CoalesceBatchesExec: target_batch_size=8192
587+ │ NetworkShuffleExec read_from=Stage 1, output_partitions=4, n_tasks=2, input_tasks=2
574588 └──────────────────────────────────────────────────
589+ ┌───── Stage 1 Tasks: t0:[p0,p1,p2,p3,p4,p5,p6,p7] t1:[p0,p1,p2,p3,p4,p5,p6,p7]
590+ │ RepartitionExec: partitioning=Hash([RainToday@0, WindGustDir@1], 8), input_partitions=4
591+ │ RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2
592+ │ AggregateExec: mode=Partial, gby=[RainToday@0 as RainToday, WindGustDir@1 as WindGustDir], aggr=[]
593+ │ PartitionIsolatorExec Tasks: t0:[p0,p1,__] t1:[__,__,p0]
594+ │ DataSourceExec: file_groups={3 groups: [[/testdata/weather/result-000000.parquet], [/testdata/weather/result-000001.parquet], [/testdata/weather/result-000002.parquet]]}, projection=[RainToday, WindGustDir], file_type=parquet
595+ └──────────────────────────────────────────────────
575596 " ) ;
576597 }
577598
0 commit comments