@@ -30,11 +30,11 @@ use ballista_core::{
30
30
} ;
31
31
use datafusion:: execution:: context:: { ExecutionConfig , ExecutionContext } ;
32
32
use datafusion:: physical_optimizer:: coalesce_batches:: CoalesceBatches ;
33
- use datafusion:: physical_optimizer:: merge_exec:: AddMergeExec ;
33
+ use datafusion:: physical_optimizer:: merge_exec:: AddCoalescePartitionsExec ;
34
34
use datafusion:: physical_optimizer:: optimizer:: PhysicalOptimizerRule ;
35
+ use datafusion:: physical_plan:: coalesce_partitions:: CoalescePartitionsExec ;
35
36
use datafusion:: physical_plan:: hash_aggregate:: { AggregateMode , HashAggregateExec } ;
36
37
use datafusion:: physical_plan:: hash_join:: HashJoinExec ;
37
- use datafusion:: physical_plan:: merge:: MergeExec ;
38
38
use datafusion:: physical_plan:: windows:: WindowAggExec ;
39
39
use datafusion:: physical_plan:: ExecutionPlan ;
40
40
use log:: info;
@@ -101,12 +101,15 @@ impl DistributedPlanner {
101
101
// remove Repartition rule because that isn't supported yet
102
102
let rules: Vec < Arc < dyn PhysicalOptimizerRule + Send + Sync > > = vec ! [
103
103
Arc :: new( CoalesceBatches :: new( ) ) ,
104
- Arc :: new( AddMergeExec :: new( ) ) ,
104
+ Arc :: new( AddCoalescePartitionsExec :: new( ) ) ,
105
105
] ;
106
106
let config = ExecutionConfig :: new ( ) . with_physical_optimizer_rules ( rules) ;
107
107
let ctx = ExecutionContext :: with_config ( config) ;
108
108
Ok ( ( ctx. create_physical_plan ( & adapter. logical_plan ) ?, stages) )
109
- } else if let Some ( merge) = execution_plan. as_any ( ) . downcast_ref :: < MergeExec > ( ) {
109
+ } else if let Some ( merge) = execution_plan
110
+ . as_any ( )
111
+ . downcast_ref :: < CoalescePartitionsExec > ( )
112
+ {
110
113
let query_stage = create_query_stage (
111
114
job_id,
112
115
self . next_stage_id ( ) ,
@@ -244,8 +247,10 @@ mod test {
244
247
use ballista_core:: serde:: protobuf;
245
248
use datafusion:: physical_plan:: hash_aggregate:: HashAggregateExec ;
246
249
use datafusion:: physical_plan:: sort:: SortExec ;
250
+ use datafusion:: physical_plan:: {
251
+ coalesce_partitions:: CoalescePartitionsExec , projection:: ProjectionExec ,
252
+ } ;
247
253
use datafusion:: physical_plan:: { displayable, ExecutionPlan } ;
248
- use datafusion:: physical_plan:: { merge:: MergeExec , projection:: ProjectionExec } ;
249
254
use std:: convert:: TryInto ;
250
255
use std:: sync:: Arc ;
251
256
use uuid:: Uuid ;
@@ -284,7 +289,7 @@ mod test {
284
289
HashAggregateExec: groupBy=["l_returnflag"], aggrExpr=["SUM(l_extendedprice Multiply Int64(1)) [\"l_extendedprice * CAST(1 AS Float64)\"]"]
285
290
CsvExec: testdata/lineitem; partitions=2
286
291
QueryStageExec: job=f011432e-e424-4016-915d-e3d8b84f6dbd, stage=2
287
- MergeExec
292
+ CoalescePartitionsExec
288
293
UnresolvedShuffleExec: stages=[1]
289
294
QueryStageExec: job=f011432e-e424-4016-915d-e3d8b84f6dbd, stage=3
290
295
SortExec { input: ProjectionExec { expr: [(Column { name: "l_returnflag" }, "l_returnflag"), (Column { name: "SUM(l_ext
@@ -309,7 +314,7 @@ mod test {
309
314
assert_eq ! ( unresolved_shuffle. query_stage_ids, vec![ 2 ] ) ;
310
315
311
316
let merge_exec = stages[ 1 ] . children ( ) [ 0 ] . clone ( ) ;
312
- let merge_exec = downcast_exec ! ( merge_exec, MergeExec ) ;
317
+ let merge_exec = downcast_exec ! ( merge_exec, CoalescePartitionsExec ) ;
313
318
314
319
let unresolved_shuffle = merge_exec. children ( ) [ 0 ] . clone ( ) ;
315
320
let unresolved_shuffle =
0 commit comments