@@ -22,18 +22,17 @@ use std::sync::Arc;
2222
2323use arrow:: datatypes:: SchemaRef ;
2424use log:: { debug, trace} ;
25- use tokio:: sync:: mpsc;
2625
2726use crate :: error:: { DataFusionError , Result } ;
2827use crate :: execution:: context:: TaskContext ;
28+ use crate :: physical_plan:: common:: spawn_buffered;
2929use crate :: physical_plan:: metrics:: {
3030 ExecutionPlanMetricsSet , MemTrackingMetrics , MetricsSet ,
3131} ;
3232use crate :: physical_plan:: sorts:: streaming_merge;
33- use crate :: physical_plan:: stream:: RecordBatchReceiverStream ;
3433use crate :: physical_plan:: {
35- common :: spawn_execution , expressions:: PhysicalSortExpr , DisplayFormatType ,
36- Distribution , ExecutionPlan , Partitioning , SendableRecordBatchStream , Statistics ,
34+ expressions:: PhysicalSortExpr , DisplayFormatType , Distribution , ExecutionPlan ,
35+ Partitioning , SendableRecordBatchStream , Statistics ,
3736} ;
3837use datafusion_physical_expr:: { EquivalenceProperties , PhysicalSortRequirement } ;
3938
@@ -181,29 +180,12 @@ impl ExecutionPlan for SortPreservingMergeExec {
181180 result
182181 }
183182 _ => {
184- // Use tokio only if running from a tokio context (#2201)
185- let receivers = match tokio:: runtime:: Handle :: try_current ( ) {
186- Ok ( _) => ( 0 ..input_partitions)
187- . map ( |part_i| {
188- let ( sender, receiver) = mpsc:: channel ( 1 ) ;
189- let join_handle = spawn_execution (
190- self . input . clone ( ) ,
191- sender,
192- part_i,
193- context. clone ( ) ,
194- ) ;
195-
196- RecordBatchReceiverStream :: create (
197- & schema,
198- receiver,
199- join_handle,
200- )
201- } )
202- . collect ( ) ,
203- Err ( _) => ( 0 ..input_partitions)
204- . map ( |partition| self . input . execute ( partition, context. clone ( ) ) )
205- . collect :: < Result < _ > > ( ) ?,
206- } ;
183+ let receivers = ( 0 ..input_partitions)
184+ . map ( |partition| {
185+ let stream = self . input . execute ( partition, context. clone ( ) ) ?;
186+ Ok ( spawn_buffered ( stream, 1 ) )
187+ } )
188+ . collect :: < Result < _ > > ( ) ?;
207189
208190 debug ! ( "Done setting up sender-receiver for SortPreservingMergeExec::execute" ) ;
209191
@@ -262,6 +244,7 @@ mod tests {
262244 use crate :: physical_plan:: memory:: MemoryExec ;
263245 use crate :: physical_plan:: metrics:: MetricValue ;
264246 use crate :: physical_plan:: sorts:: sort:: SortExec ;
247+ use crate :: physical_plan:: stream:: RecordBatchReceiverStream ;
265248 use crate :: physical_plan:: { collect, common} ;
266249 use crate :: prelude:: { SessionConfig , SessionContext } ;
267250 use crate :: test:: exec:: { assert_strong_count_converges_to_zero, BlockingExec } ;
@@ -812,7 +795,7 @@ mod tests {
812795 let mut streams = Vec :: with_capacity ( partition_count) ;
813796
814797 for partition in 0 ..partition_count {
815- let ( sender, receiver) = mpsc:: channel ( 1 ) ;
798+ let ( sender, receiver) = tokio :: sync :: mpsc:: channel ( 1 ) ;
816799 let mut stream = batches. execute ( partition, task_ctx. clone ( ) ) . unwrap ( ) ;
817800 let join_handle = tokio:: spawn ( async move {
818801 while let Some ( batch) = stream. next ( ) . await {
0 commit comments