@@ -29,7 +29,6 @@ use crate::physical_plan::{
29
29
} ;
30
30
use arrow:: array:: ArrayRef ;
31
31
use arrow:: datatypes:: { Field , Schema , SchemaRef } ;
32
- use async_trait:: async_trait;
33
32
use datafusion_common:: Result ;
34
33
use datafusion_expr:: Accumulator ;
35
34
use datafusion_physical_expr:: expressions:: Column ;
@@ -145,7 +144,6 @@ impl AggregateExec {
145
144
}
146
145
}
147
146
148
- #[ async_trait]
149
147
impl ExecutionPlan for AggregateExec {
150
148
/// Return a reference to Any that can be used for down-casting
151
149
fn as_any ( & self ) -> & dyn Any {
@@ -196,12 +194,12 @@ impl ExecutionPlan for AggregateExec {
196
194
) ?) )
197
195
}
198
196
199
- async fn execute (
197
+ fn execute (
200
198
& self ,
201
199
partition : usize ,
202
200
context : Arc < TaskContext > ,
203
201
) -> Result < SendableRecordBatchStream > {
204
- let input = self . input . execute ( partition, context) . await ?;
202
+ let input = self . input . execute ( partition, context) ?;
205
203
let group_expr = self . group_expr . iter ( ) . map ( |x| x. 0 . clone ( ) ) . collect ( ) ;
206
204
207
205
let baseline_metrics = BaselineMetrics :: new ( & self . metrics , partition) ;
@@ -417,7 +415,6 @@ mod tests {
417
415
use arrow:: datatypes:: { DataType , Field , Schema , SchemaRef } ;
418
416
use arrow:: error:: Result as ArrowResult ;
419
417
use arrow:: record_batch:: RecordBatch ;
420
- use async_trait:: async_trait;
421
418
use datafusion_common:: { DataFusionError , Result } ;
422
419
use datafusion_physical_expr:: { AggregateExpr , PhysicalExpr , PhysicalSortExpr } ;
423
420
use futures:: { FutureExt , Stream } ;
@@ -489,8 +486,7 @@ mod tests {
489
486
) ?) ;
490
487
491
488
let result =
492
- common:: collect ( partial_aggregate. execute ( 0 , task_ctx. clone ( ) ) . await ?)
493
- . await ?;
489
+ common:: collect ( partial_aggregate. execute ( 0 , task_ctx. clone ( ) ) ?) . await ?;
494
490
495
491
let expected = vec ! [
496
492
"+---+---------------+-------------+" ,
@@ -522,7 +518,7 @@ mod tests {
522
518
) ?) ;
523
519
524
520
let result =
525
- common:: collect ( merged_aggregate. execute ( 0 , task_ctx. clone ( ) ) . await ?) . await ?;
521
+ common:: collect ( merged_aggregate. execute ( 0 , task_ctx. clone ( ) ) ?) . await ?;
526
522
assert_eq ! ( result. len( ) , 1 ) ;
527
523
528
524
let batch = & result[ 0 ] ;
@@ -556,7 +552,6 @@ mod tests {
556
552
pub yield_first : bool ,
557
553
}
558
554
559
- #[ async_trait]
560
555
impl ExecutionPlan for TestYieldingExec {
561
556
fn as_any ( & self ) -> & dyn Any {
562
557
self
@@ -587,7 +582,7 @@ mod tests {
587
582
) ) )
588
583
}
589
584
590
- async fn execute (
585
+ fn execute (
591
586
& self ,
592
587
_partition : usize ,
593
588
_context : Arc < TaskContext > ,
0 commit comments