@@ -97,8 +97,8 @@ type PartitionChannels = (Vec<Sender<usize>>, Vec<Receiver<usize>>);
97
97
/// └─────────────────┘ Distribute data to the output partitions
98
98
///
99
99
/// ```
100
- type OnDemandDistributionSender = tokio:: sync:: mpsc:: Sender < MaybeBatch > ;
101
- type OnDemandDistributionReceiver = tokio:: sync:: mpsc:: Receiver < MaybeBatch > ;
100
+ type OnDemandDistributionSender = tokio:: sync:: mpsc:: UnboundedSender < MaybeBatch > ;
101
+ type OnDemandDistributionReceiver = tokio:: sync:: mpsc:: UnboundedReceiver < MaybeBatch > ;
102
102
103
103
type OnDemandInputPartitionsToCurrentPartitionSender = Vec < OnDemandDistributionSender > ;
104
104
type OnDemandInputPartitionsToCurrentPartitionReceiver =
@@ -631,10 +631,8 @@ impl OnDemandRepartitionExec {
631
631
metrics : OnDemandRepartitionMetrics ,
632
632
context : Arc < TaskContext > ,
633
633
) -> Result < ( ) > {
634
- let num_output_partition = partitioning. partition_count ( ) ;
635
634
// initialize buffer channel so that we can pre-fetch from input
636
- let ( buffer_tx, mut buffer_rx) =
637
- tokio:: sync:: mpsc:: channel ( num_output_partition * 2 ) ;
635
+ let ( buffer_tx, mut buffer_rx) = tokio:: sync:: mpsc:: channel ( 2 ) ;
638
636
// execute the child operator in a separate task
639
637
// that pushes batches into buffer channel with limited capacity
640
638
let processing_task = SpawnedTask :: spawn ( Self :: process_input (
@@ -683,7 +681,7 @@ impl OnDemandRepartitionExec {
683
681
if let Some ( ( tx, reservation) ) = output_channels. get_mut ( & partition) {
684
682
reservation. lock ( ) . try_grow ( size) ?;
685
683
686
- if tx. send ( Some ( Ok ( batch) ) ) . await . is_err ( ) {
684
+ if tx. send ( Some ( Ok ( batch) ) ) . is_err ( ) {
687
685
// If the other end has hung up, it was an early shutdown (e.g. LIMIT)
688
686
reservation. lock ( ) . shrink ( size) ;
689
687
output_channels. remove ( & partition) ;
@@ -737,7 +735,7 @@ impl OnDemandRepartitionExec {
737
735
"Join Error" . to_string ( ) ,
738
736
Box :: new ( DataFusionError :: External ( Box :: new ( Arc :: clone ( & e) ) ) ) ,
739
737
) ) ;
740
- tx. send ( Some ( err) ) . await . ok ( ) ;
738
+ tx. send ( Some ( err) ) . ok ( ) ;
741
739
}
742
740
}
743
741
// Error from running input task
@@ -748,14 +746,14 @@ impl OnDemandRepartitionExec {
748
746
for ( _, tx) in txs {
749
747
// wrap it because need to send error to all output partitions
750
748
let err = Err ( DataFusionError :: from ( & e) ) ;
751
- tx. send ( Some ( err) ) . await . ok ( ) ;
749
+ tx. send ( Some ( err) ) . ok ( ) ;
752
750
}
753
751
}
754
752
// Input task completed successfully
755
753
Ok ( Ok ( ( ) ) ) => {
756
754
// notify each output partition that this input partition has no more data
757
755
for ( _, tx) in txs {
758
- tx. send ( None ) . await . ok ( ) ;
756
+ tx. send ( None ) . ok ( ) ;
759
757
}
760
758
}
761
759
}
0 commit comments