@@ -43,7 +43,6 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat
43
43
use arrow:: array:: { PrimitiveArray , RecordBatch , RecordBatchOptions } ;
44
44
use arrow:: compute:: take_arrays;
45
45
use arrow:: datatypes:: { SchemaRef , UInt32Type } ;
46
- use async_channel:: Receiver ;
47
46
use datafusion_common:: utils:: transpose;
48
47
use datafusion_common:: HashMap ;
49
48
use datafusion_common:: { not_impl_err, DataFusionError , Result } ;
@@ -56,7 +55,6 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering;
56
55
use futures:: stream:: Stream ;
57
56
use futures:: { ready, FutureExt , StreamExt , TryStreamExt } ;
58
57
use log:: trace;
59
- use on_demand_repartition:: { OnDemandRepartitionExec , OnDemandRepartitionMetrics } ;
60
58
use parking_lot:: Mutex ;
61
59
62
60
mod distributor_channels;
@@ -66,68 +64,6 @@ type MaybeBatch = Option<Result<RecordBatch>>;
66
64
type InputPartitionsToCurrentPartitionSender = Vec < DistributionSender < MaybeBatch > > ;
67
65
type InputPartitionsToCurrentPartitionReceiver = Vec < DistributionReceiver < MaybeBatch > > ;
68
66
69
- struct RepartitionExecStateBuilder {
70
- /// Whether to enable pull based execution.
71
- enable_pull_based : bool ,
72
- partition_receivers : Option < Vec < Receiver < usize > > > ,
73
- }
74
-
75
- impl RepartitionExecStateBuilder {
76
- fn new ( ) -> Self {
77
- Self {
78
- enable_pull_based : false ,
79
- partition_receivers : None ,
80
- }
81
- }
82
- fn enable_pull_based ( mut self , enable_pull_based : bool ) -> Self {
83
- self . enable_pull_based = enable_pull_based;
84
- self
85
- }
86
- fn partition_receivers ( mut self , partition_receivers : Vec < Receiver < usize > > ) -> Self {
87
- self . partition_receivers = Some ( partition_receivers) ;
88
- self
89
- }
90
-
91
- fn build (
92
- & self ,
93
- input : Arc < dyn ExecutionPlan > ,
94
- partitioning : Partitioning ,
95
- metrics : ExecutionPlanMetricsSet ,
96
- preserve_order : bool ,
97
- name : String ,
98
- context : Arc < TaskContext > ,
99
- ) -> RepartitionExecState {
100
- RepartitionExecState :: new (
101
- input,
102
- partitioning,
103
- metrics,
104
- preserve_order,
105
- name,
106
- context,
107
- self . enable_pull_based ,
108
- self . partition_receivers . clone ( ) ,
109
- )
110
- }
111
- }
112
-
113
- /// Inner state of [`RepartitionExec`].
114
- #[ derive( Debug ) ]
115
- struct RepartitionExecState {
116
- /// Channels for sending batches from input partitions to output partitions.
117
- /// Key is the partition number.
118
- channels : HashMap <
119
- usize ,
120
- (
121
- InputPartitionsToCurrentPartitionSender ,
122
- InputPartitionsToCurrentPartitionReceiver ,
123
- SharedMemoryReservation ,
124
- ) ,
125
- > ,
126
-
127
- /// Helper that ensures that that background job is killed once it is no longer needed.
128
- abort_helper : Arc < Vec < SpawnedTask < ( ) > > > ,
129
- }
130
-
131
67
/// create channels for sending batches from input partitions to output partitions.
132
68
fn create_repartition_channels (
133
69
preserve_order : bool ,
@@ -185,17 +121,33 @@ fn create_partition_channels_hashmap(
185
121
186
122
channels
187
123
}
124
+
125
+ /// Inner state of [`RepartitionExec`].
126
+ #[ derive( Debug ) ]
127
+ struct RepartitionExecState {
128
+ /// Channels for sending batches from input partitions to output partitions.
129
+ /// Key is the partition number.
130
+ channels : HashMap <
131
+ usize ,
132
+ (
133
+ InputPartitionsToCurrentPartitionSender ,
134
+ InputPartitionsToCurrentPartitionReceiver ,
135
+ SharedMemoryReservation ,
136
+ ) ,
137
+ > ,
138
+
139
+ /// Helper that ensures that that background job is killed once it is no longer needed.
140
+ abort_helper : Arc < Vec < SpawnedTask < ( ) > > > ,
141
+ }
142
+
188
143
impl RepartitionExecState {
189
- #[ allow( clippy:: too_many_arguments) ]
190
144
fn new (
191
145
input : Arc < dyn ExecutionPlan > ,
192
146
partitioning : Partitioning ,
193
147
metrics : ExecutionPlanMetricsSet ,
194
148
preserve_order : bool ,
195
149
name : String ,
196
150
context : Arc < TaskContext > ,
197
- enable_pull_based : bool ,
198
- partition_receivers : Option < Vec < Receiver < usize > > > ,
199
151
) -> Self {
200
152
let num_input_partitions = input. output_partitioning ( ) . partition_count ( ) ;
201
153
let num_output_partitions = partitioning. partition_count ( ) ;
@@ -219,42 +171,16 @@ impl RepartitionExecState {
219
171
} )
220
172
. collect ( ) ;
221
173
222
- let input_task = if enable_pull_based {
223
- let partition_rx = if preserve_order {
224
- partition_receivers. clone ( ) . expect (
225
- "partition_receivers must be provided when preserve_order is enabled" ,
226
- ) [ i]
227
- . clone ( )
228
- } else {
229
- partition_receivers. clone ( ) . expect (
230
- "partition_receivers must be provided when preserve_order is disabled" ,
231
- ) [ 0 ] . clone ( )
232
- } ;
233
- let r_metrics =
234
- OnDemandRepartitionMetrics :: new ( i, num_output_partitions, & metrics) ;
235
-
236
- SpawnedTask :: spawn ( OnDemandRepartitionExec :: pull_from_input (
237
- Arc :: clone ( & input) ,
238
- i,
239
- txs. clone ( ) ,
240
- partitioning. clone ( ) ,
241
- partition_rx,
242
- r_metrics,
243
- Arc :: clone ( & context) ,
244
- ) )
245
- } else {
246
- let r_metrics =
247
- RepartitionMetrics :: new ( i, num_output_partitions, & metrics) ;
248
-
249
- SpawnedTask :: spawn ( RepartitionExec :: pull_from_input (
250
- Arc :: clone ( & input) ,
251
- i,
252
- txs. clone ( ) ,
253
- partitioning. clone ( ) ,
254
- r_metrics,
255
- Arc :: clone ( & context) ,
256
- ) )
257
- } ;
174
+ let r_metrics = RepartitionMetrics :: new ( i, num_output_partitions, & metrics) ;
175
+
176
+ let input_task = SpawnedTask :: spawn ( RepartitionExec :: pull_from_input (
177
+ Arc :: clone ( & input) ,
178
+ i,
179
+ txs. clone ( ) ,
180
+ partitioning. clone ( ) ,
181
+ r_metrics,
182
+ Arc :: clone ( & context) ,
183
+ ) ) ;
258
184
259
185
// In a separate task, wait for each input to be done
260
186
// (and pass along any errors, including panic!s)
@@ -268,7 +194,6 @@ impl RepartitionExecState {
268
194
269
195
spawned_tasks. push ( wait_for_task) ;
270
196
}
271
-
272
197
Self {
273
198
channels,
274
199
abort_helper : Arc :: new ( spawned_tasks) ,
@@ -467,8 +392,6 @@ pub struct RepartitionExecBase {
467
392
preserve_order : bool ,
468
393
/// Cache holding plan properties like equivalences, output partitioning etc.
469
394
cache : PlanProperties ,
470
- /// Inner state that is initialized when the first output stream is created.
471
- state : LazyState ,
472
395
}
473
396
474
397
impl RepartitionExecBase {
@@ -611,6 +534,8 @@ impl RepartitionExecBase {
611
534
pub struct RepartitionExec {
612
535
/// Common fields for all repartitioning executors
613
536
base : RepartitionExecBase ,
537
+ /// Inner state that is initialized when the first output stream is created.
538
+ state : LazyState ,
614
539
}
615
540
616
541
#[ derive( Debug , Clone ) ]
@@ -776,7 +701,7 @@ impl ExecutionPlan for RepartitionExec {
776
701
partition
777
702
) ;
778
703
779
- let lazy_state = Arc :: clone ( & self . base . state ) ;
704
+ let lazy_state = Arc :: clone ( & self . state ) ;
780
705
let input = Arc :: clone ( & self . base . input ) ;
781
706
let partitioning = self . partitioning ( ) . clone ( ) ;
782
707
let metrics = self . base . metrics . clone ( ) ;
@@ -797,7 +722,7 @@ impl ExecutionPlan for RepartitionExec {
797
722
let context_captured = Arc :: clone ( & context) ;
798
723
let state = lazy_state
799
724
. get_or_init ( || async move {
800
- Mutex :: new ( RepartitionExecStateBuilder :: new ( ) . build (
725
+ Mutex :: new ( RepartitionExecState :: new (
801
726
input_captured,
802
727
partitioning. clone ( ) ,
803
728
metrics_captured,
@@ -945,11 +870,11 @@ impl RepartitionExec {
945
870
Ok ( RepartitionExec {
946
871
base : RepartitionExecBase {
947
872
input,
948
- state : Default :: default ( ) ,
949
873
metrics : ExecutionPlanMetricsSet :: new ( ) ,
950
874
preserve_order,
951
875
cache,
952
876
} ,
877
+ state : Default :: default ( ) ,
953
878
} )
954
879
}
955
880
0 commit comments