-
Notifications
You must be signed in to change notification settings - Fork 971
/
mod.rs
1352 lines (1175 loc) · 47.8 KB
/
mod.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//! The repartition operator maps N input partitions to M output partitions based on a
//! partitioning scheme (according to flag `preserve_order` ordering can be preserved during
//! repartitioning if its input is ordered).
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use std::{any::Any, vec};
use crate::physical_plan::common::transpose;
use crate::physical_plan::hash_utils::create_hashes;
use crate::physical_plan::metrics::BaselineMetrics;
use crate::physical_plan::repartition::distributor_channels::{
channels, partition_aware_channels,
};
use crate::physical_plan::sorts::streaming_merge;
use crate::physical_plan::{
DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics,
};
use self::distributor_channels::{DistributionReceiver, DistributionSender};
use super::common::{AbortOnDropMany, AbortOnDropSingle, SharedMemoryReservation};
use super::expressions::PhysicalSortExpr;
use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet};
use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream};
use arrow::array::{ArrayRef, UInt64Builder};
use arrow::datatypes::SchemaRef;
use arrow::record_batch::RecordBatch;
use datafusion_common::{DataFusionError, Result};
use datafusion_execution::memory_pool::MemoryConsumer;
use datafusion_execution::TaskContext;
use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalExpr};
use futures::stream::Stream;
use futures::{FutureExt, StreamExt};
use hashbrown::HashMap;
use log::trace;
use parking_lot::Mutex;
use tokio::task::JoinHandle;
mod distributor_channels;
type MaybeBatch = Option<Result<RecordBatch>>;
type InputPartitionsToCurrentPartitionSender = Vec<DistributionSender<MaybeBatch>>;
type InputPartitionsToCurrentPartitionReceiver = Vec<DistributionReceiver<MaybeBatch>>;
/// Inner state of [`RepartitionExec`].
#[derive(Debug)]
struct RepartitionExecState {
/// Channels for sending batches from input partitions to output partitions.
/// Key is the partition number.
channels: HashMap<
usize,
(
InputPartitionsToCurrentPartitionSender,
InputPartitionsToCurrentPartitionReceiver,
SharedMemoryReservation,
),
>,
/// Helper that ensures that that background job is killed once it is no longer needed.
abort_helper: Arc<AbortOnDropMany<()>>,
}
/// A utility that can be used to partition batches based on [`Partitioning`]
pub struct BatchPartitioner {
state: BatchPartitionerState,
timer: metrics::Time,
}
enum BatchPartitionerState {
Hash {
random_state: ahash::RandomState,
exprs: Vec<Arc<dyn PhysicalExpr>>,
num_partitions: usize,
hash_buffer: Vec<u64>,
},
RoundRobin {
num_partitions: usize,
next_idx: usize,
},
}
impl BatchPartitioner {
/// Create a new [`BatchPartitioner`] with the provided [`Partitioning`]
///
/// The time spent repartitioning will be recorded to `timer`
pub fn try_new(partitioning: Partitioning, timer: metrics::Time) -> Result<Self> {
let state = match partitioning {
Partitioning::RoundRobinBatch(num_partitions) => {
BatchPartitionerState::RoundRobin {
num_partitions,
next_idx: 0,
}
}
Partitioning::Hash(exprs, num_partitions) => BatchPartitionerState::Hash {
exprs,
num_partitions,
// Use fixed random hash
random_state: ahash::RandomState::with_seeds(0, 0, 0, 0),
hash_buffer: vec![],
},
other => {
return Err(DataFusionError::NotImplemented(format!(
"Unsupported repartitioning scheme {other:?}"
)))
}
};
Ok(Self { state, timer })
}
/// Partition the provided [`RecordBatch`] into one or more partitioned [`RecordBatch`]
/// based on the [`Partitioning`] specified on construction
///
/// `f` will be called for each partitioned [`RecordBatch`] with the corresponding
/// partition index. Any error returned by `f` will be immediately returned by this
/// function without attempting to publish further [`RecordBatch`]
///
/// The time spent repartitioning, not including time spent in `f` will be recorded
/// to the [`metrics::Time`] provided on construction
pub fn partition<F>(&mut self, batch: RecordBatch, mut f: F) -> Result<()>
where
F: FnMut(usize, RecordBatch) -> Result<()>,
{
self.partition_iter(batch)?.try_for_each(|res| match res {
Ok((partition, batch)) => f(partition, batch),
Err(e) => Err(e),
})
}
/// Actual implementation of [`partition`](Self::partition).
///
/// The reason this was pulled out is that we need to have a variant of `partition` that works w/ sync functions,
/// and one that works w/ async. Using an iterator as an intermediate representation was the best way to achieve
/// this (so we don't need to clone the entire implementation).
fn partition_iter(
&mut self,
batch: RecordBatch,
) -> Result<impl Iterator<Item = Result<(usize, RecordBatch)>> + Send + '_> {
let it: Box<dyn Iterator<Item = Result<(usize, RecordBatch)>> + Send> =
match &mut self.state {
BatchPartitionerState::RoundRobin {
num_partitions,
next_idx,
} => {
let idx = *next_idx;
*next_idx = (*next_idx + 1) % *num_partitions;
Box::new(std::iter::once(Ok((idx, batch))))
}
BatchPartitionerState::Hash {
random_state,
exprs,
num_partitions: partitions,
hash_buffer,
} => {
let timer = self.timer.timer();
let arrays = exprs
.iter()
.map(|expr| {
Ok(expr.evaluate(&batch)?.into_array(batch.num_rows()))
})
.collect::<Result<Vec<_>>>()?;
hash_buffer.clear();
hash_buffer.resize(batch.num_rows(), 0);
create_hashes(&arrays, random_state, hash_buffer)?;
let mut indices: Vec<_> = (0..*partitions)
.map(|_| UInt64Builder::with_capacity(batch.num_rows()))
.collect();
for (index, hash) in hash_buffer.iter().enumerate() {
indices[(*hash % *partitions as u64) as usize]
.append_value(index as u64);
}
let it = indices
.into_iter()
.enumerate()
.filter_map(|(partition, mut indices)| {
let indices = indices.finish();
(!indices.is_empty()).then_some((partition, indices))
})
.map(move |(partition, indices)| {
// Produce batches based on indices
let columns = batch
.columns()
.iter()
.map(|c| {
arrow::compute::take(c.as_ref(), &indices, None)
.map_err(DataFusionError::ArrowError)
})
.collect::<Result<Vec<ArrayRef>>>()?;
let batch =
RecordBatch::try_new(batch.schema(), columns).unwrap();
// bind timer so it drops w/ this iterator
let _ = &timer;
Ok((partition, batch))
});
Box::new(it)
}
};
Ok(it)
}
// return the number of output partitions
fn num_partitions(&self) -> usize {
match self.state {
BatchPartitionerState::RoundRobin { num_partitions, .. } => num_partitions,
BatchPartitionerState::Hash { num_partitions, .. } => num_partitions,
}
}
}
/// The repartition operator maps N input partitions to M output partitions based on a
/// partitioning scheme. No guarantees are made about the order of the resulting partitions.
#[derive(Debug)]
pub struct RepartitionExec {
/// Input execution plan
input: Arc<dyn ExecutionPlan>,
/// Partitioning scheme to use
partitioning: Partitioning,
/// Inner state that is initialized when the first output stream is created.
state: Arc<Mutex<RepartitionExecState>>,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
/// Boolean flag to decide whether to preserve ordering
preserve_order: bool,
}
#[derive(Debug, Clone)]
struct RepartitionMetrics {
/// Time in nanos to execute child operator and fetch batches
fetch_time: metrics::Time,
/// Time in nanos to perform repartitioning
repartition_time: metrics::Time,
/// Time in nanos for sending resulting batches to channels
send_time: metrics::Time,
}
impl RepartitionMetrics {
pub fn new(
output_partition: usize,
input_partition: usize,
metrics: &ExecutionPlanMetricsSet,
) -> Self {
let label = metrics::Label::new("inputPartition", input_partition.to_string());
// Time in nanos to execute child operator and fetch batches
let fetch_time = MetricBuilder::new(metrics)
.with_label(label.clone())
.subset_time("fetch_time", output_partition);
// Time in nanos to perform repartitioning
let repart_time = MetricBuilder::new(metrics)
.with_label(label.clone())
.subset_time("repart_time", output_partition);
// Time in nanos for sending resulting batches to channels
let send_time = MetricBuilder::new(metrics)
.with_label(label)
.subset_time("send_time", output_partition);
Self {
fetch_time,
repartition_time: repart_time,
send_time,
}
}
}
impl RepartitionExec {
/// Input execution plan
pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
&self.input
}
/// Partitioning scheme to use
pub fn partitioning(&self) -> &Partitioning {
&self.partitioning
}
/// Get name of the Executor
pub fn name(&self) -> &str {
if self.preserve_order {
"SortPreservingRepartitionExec"
} else {
"RepartitionExec"
}
}
}
impl DisplayAs for RepartitionExec {
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(
f,
"{}: partitioning={}, input_partitions={}",
self.name(),
self.partitioning,
self.input.output_partitioning().partition_count()
)
}
}
}
}
impl ExecutionPlan for RepartitionExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
/// Get the schema for this execution plan
fn schema(&self) -> SchemaRef {
self.input.schema()
}
fn children(&self) -> Vec<Arc<dyn ExecutionPlan>> {
vec![self.input.clone()]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
let mut repartition =
RepartitionExec::try_new(children[0].clone(), self.partitioning.clone())?;
if self.preserve_order {
repartition = repartition.with_preserve_order();
}
Ok(Arc::new(repartition))
}
/// Specifies whether this plan generates an infinite stream of records.
/// If the plan does not support pipelining, but its input(s) are
/// infinite, returns an error to indicate this.
fn unbounded_output(&self, children: &[bool]) -> Result<bool> {
Ok(children[0])
}
fn output_partitioning(&self) -> Partitioning {
self.partitioning.clone()
}
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
if self.maintains_input_order()[0] {
self.input().output_ordering()
} else {
None
}
}
fn maintains_input_order(&self) -> Vec<bool> {
if self.preserve_order {
vec![true]
} else {
// We preserve ordering when input partitioning is 1
vec![self.input().output_partitioning().partition_count() <= 1]
}
}
fn equivalence_properties(&self) -> EquivalenceProperties {
self.input.equivalence_properties()
}
fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties {
self.input.ordering_equivalence_properties()
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
trace!(
"Start {}::execute for partition: {}",
self.name(),
partition
);
// lock mutexes
let mut state = self.state.lock();
let num_input_partitions = self.input.output_partitioning().partition_count();
let num_output_partitions = self.partitioning.partition_count();
// if this is the first partition to be invoked then we need to set up initial state
if state.channels.is_empty() {
let (txs, rxs) = if self.preserve_order {
let (txs, rxs) =
partition_aware_channels(num_input_partitions, num_output_partitions);
// Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition
let txs = transpose(txs);
let rxs = transpose(rxs);
(txs, rxs)
} else {
// create one channel per *output* partition
// note we use a custom channel that ensures there is always data for each receiver
// but limits the amount of buffering if required.
let (txs, rxs) = channels(num_output_partitions);
// Clone sender for each input partitions
let txs = txs
.into_iter()
.map(|item| vec![item; num_input_partitions])
.collect::<Vec<_>>();
let rxs = rxs.into_iter().map(|item| vec![item]).collect::<Vec<_>>();
(txs, rxs)
};
for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() {
let reservation = Arc::new(Mutex::new(
MemoryConsumer::new(format!("{}[{partition}]", self.name()))
.register(context.memory_pool()),
));
state.channels.insert(partition, (tx, rx, reservation));
}
// launch one async task per *input* partition
let mut join_handles = Vec::with_capacity(num_input_partitions);
for i in 0..num_input_partitions {
let txs: HashMap<_, _> = state
.channels
.iter()
.map(|(partition, (tx, _rx, reservation))| {
(*partition, (tx[i].clone(), Arc::clone(reservation)))
})
.collect();
let r_metrics = RepartitionMetrics::new(i, partition, &self.metrics);
let input_task: JoinHandle<Result<()>> =
tokio::spawn(Self::pull_from_input(
self.input.clone(),
i,
txs.clone(),
self.partitioning.clone(),
r_metrics,
context.clone(),
));
// In a separate task, wait for each input to be done
// (and pass along any errors, including panic!s)
let join_handle = tokio::spawn(Self::wait_for_task(
AbortOnDropSingle::new(input_task),
txs.into_iter()
.map(|(partition, (tx, _reservation))| (partition, tx))
.collect(),
));
join_handles.push(join_handle);
}
state.abort_helper = Arc::new(AbortOnDropMany(join_handles))
}
trace!(
"Before returning stream in {}::execute for partition: {}",
self.name(),
partition
);
// now return stream for the specified *output* partition which will
// read from the channel
let (_tx, mut rx, reservation) = state
.channels
.remove(&partition)
.expect("partition not used yet");
if self.preserve_order {
// Store streams from all the input partitions:
let input_streams = rx
.into_iter()
.map(|receiver| {
Box::pin(PerPartitionStream {
schema: self.schema(),
receiver,
drop_helper: Arc::clone(&state.abort_helper),
reservation: reservation.clone(),
}) as SendableRecordBatchStream
})
.collect::<Vec<_>>();
// Note that receiver size (`rx.len()`) and `num_input_partitions` are same.
// Get existing ordering:
let sort_exprs = self.input.output_ordering().unwrap_or(&[]);
// Merge streams (while preserving ordering) coming from input partitions to this partition:
streaming_merge(
input_streams,
self.schema(),
sort_exprs,
BaselineMetrics::new(&self.metrics, partition),
context.session_config().batch_size(),
None,
)
} else {
Ok(Box::pin(RepartitionStream {
num_input_partitions,
num_input_partitions_processed: 0,
schema: self.input.schema(),
input: rx.swap_remove(0),
drop_helper: Arc::clone(&state.abort_helper),
reservation,
}))
}
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Statistics {
self.input.statistics()
}
}
impl RepartitionExec {
/// Create a new RepartitionExec
pub fn try_new(
input: Arc<dyn ExecutionPlan>,
partitioning: Partitioning,
) -> Result<Self> {
Ok(RepartitionExec {
input,
partitioning,
state: Arc::new(Mutex::new(RepartitionExecState {
channels: HashMap::new(),
abort_helper: Arc::new(AbortOnDropMany::<()>(vec![])),
})),
metrics: ExecutionPlanMetricsSet::new(),
preserve_order: false,
})
}
/// Set Order preserving flag
pub fn with_preserve_order(mut self) -> Self {
self.preserve_order = true;
self
}
/// Pulls data from the specified input plan, feeding it to the
/// output partitions based on the desired partitioning
///
/// txs hold the output sending channels for each output partition
async fn pull_from_input(
input: Arc<dyn ExecutionPlan>,
partition: usize,
mut output_channels: HashMap<
usize,
(DistributionSender<MaybeBatch>, SharedMemoryReservation),
>,
partitioning: Partitioning,
metrics: RepartitionMetrics,
context: Arc<TaskContext>,
) -> Result<()> {
let mut partitioner =
BatchPartitioner::try_new(partitioning, metrics.repartition_time.clone())?;
// execute the child operator
let timer = metrics.fetch_time.timer();
let mut stream = input.execute(partition, context)?;
timer.done();
// While there are still outputs to send to, keep pulling inputs
let mut batches_until_yield = partitioner.num_partitions();
while !output_channels.is_empty() {
// fetch the next batch
let timer = metrics.fetch_time.timer();
let result = stream.next().await;
timer.done();
// Input is done
let batch = match result {
Some(result) => result?,
None => break,
};
for res in partitioner.partition_iter(batch)? {
let (partition, batch) = res?;
let size = batch.get_array_memory_size();
let timer = metrics.send_time.timer();
// if there is still a receiver, send to it
if let Some((tx, reservation)) = output_channels.get_mut(&partition) {
reservation.lock().try_grow(size)?;
if tx.send(Some(Ok(batch))).await.is_err() {
// If the other end has hung up, it was an early shutdown (e.g. LIMIT)
reservation.lock().shrink(size);
output_channels.remove(&partition);
}
}
timer.done();
}
// If the input stream is endless, we may spin forever and
// never yield back to tokio. See
// https://github.com/apache/arrow-datafusion/issues/5278.
//
// However, yielding on every batch causes a bottleneck
// when running with multiple cores. See
// https://github.com/apache/arrow-datafusion/issues/6290
//
// Thus, heuristically yield after producing num_partition
// batches
//
// In round robin this is ideal as each input will get a
// new batch. In hash partitioning it may yield too often
// on uneven distributions even if some partition can not
// make progress, but parallelism is going to be limited
// in that case anyways
if batches_until_yield == 0 {
tokio::task::yield_now().await;
batches_until_yield = partitioner.num_partitions();
} else {
batches_until_yield -= 1;
}
}
Ok(())
}
/// Waits for `input_task` which is consuming one of the inputs to
/// complete. Upon each successful completion, sends a `None` to
/// each of the output tx channels to signal one of the inputs is
/// complete. Upon error, propagates the errors to all output tx
/// channels.
async fn wait_for_task(
input_task: AbortOnDropSingle<Result<()>>,
txs: HashMap<usize, DistributionSender<MaybeBatch>>,
) {
// wait for completion, and propagate error
// note we ignore errors on send (.ok) as that means the receiver has already shutdown.
match input_task.await {
// Error in joining task
Err(e) => {
let e = Arc::new(e);
for (_, tx) in txs {
let err = Err(DataFusionError::Context(
"Join Error".to_string(),
Box::new(DataFusionError::External(Box::new(Arc::clone(&e)))),
));
tx.send(Some(err)).await.ok();
}
}
// Error from running input task
Ok(Err(e)) => {
let e = Arc::new(e);
for (_, tx) in txs {
// wrap it because need to send error to all output partitions
let err = Err(DataFusionError::External(Box::new(e.clone())));
tx.send(Some(err)).await.ok();
}
}
// Input task completed successfully
Ok(Ok(())) => {
// notify each output partition that this input partition has no more data
for (_, tx) in txs {
tx.send(None).await.ok();
}
}
}
}
}
struct RepartitionStream {
/// Number of input partitions that will be sending batches to this output channel
num_input_partitions: usize,
/// Number of input partitions that have finished sending batches to this output channel
num_input_partitions_processed: usize,
/// Schema wrapped by Arc
schema: SchemaRef,
/// channel containing the repartitioned batches
input: DistributionReceiver<MaybeBatch>,
/// Handle to ensure background tasks are killed when no longer needed.
#[allow(dead_code)]
drop_helper: Arc<AbortOnDropMany<()>>,
/// Memory reservation.
reservation: SharedMemoryReservation,
}
impl Stream for RepartitionStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
loop {
match self.input.recv().poll_unpin(cx) {
Poll::Ready(Some(Some(v))) => {
if let Ok(batch) = &v {
self.reservation
.lock()
.shrink(batch.get_array_memory_size());
}
return Poll::Ready(Some(v));
}
Poll::Ready(Some(None)) => {
self.num_input_partitions_processed += 1;
if self.num_input_partitions == self.num_input_partitions_processed {
// all input partitions have finished sending batches
return Poll::Ready(None);
} else {
// other partitions still have data to send
continue;
}
}
Poll::Ready(None) => {
return Poll::Ready(None);
}
Poll::Pending => {
return Poll::Pending;
}
}
}
}
}
impl RecordBatchStream for RepartitionStream {
/// Get the schema
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
/// This struct converts a receiver to a stream.
/// Receiver receives data on an SPSC channel.
struct PerPartitionStream {
/// Schema wrapped by Arc
schema: SchemaRef,
/// channel containing the repartitioned batches
receiver: DistributionReceiver<MaybeBatch>,
/// Handle to ensure background tasks are killed when no longer needed.
#[allow(dead_code)]
drop_helper: Arc<AbortOnDropMany<()>>,
/// Memory reservation.
reservation: SharedMemoryReservation,
}
impl Stream for PerPartitionStream {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
match self.receiver.recv().poll_unpin(cx) {
Poll::Ready(Some(Some(v))) => {
if let Ok(batch) = &v {
self.reservation
.lock()
.shrink(batch.get_array_memory_size());
}
Poll::Ready(Some(v))
}
Poll::Ready(Some(None)) => {
// Input partition has finished sending batches
Poll::Ready(None)
}
Poll::Ready(None) => Poll::Ready(None),
Poll::Pending => Poll::Pending,
}
}
}
impl RecordBatchStream for PerPartitionStream {
/// Get the schema
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::execution::context::SessionConfig;
use crate::prelude::SessionContext;
use crate::test::create_vec_batches;
use crate::{
assert_batches_sorted_eq,
physical_plan::{collect, expressions::col, memory::MemoryExec},
test::{
assert_is_pending,
exec::{
assert_strong_count_converges_to_zero, BarrierExec, BlockingExec,
ErrorExec, MockExec,
},
},
};
use arrow::array::{ArrayRef, StringArray};
use arrow::datatypes::{DataType, Field, Schema};
use arrow::record_batch::RecordBatch;
use datafusion_common::cast::as_string_array;
use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv};
use futures::FutureExt;
use std::collections::HashSet;
use tokio::task::JoinHandle;
#[tokio::test]
async fn one_to_many_round_robin() -> Result<()> {
// define input partitions
let schema = test_schema();
let partition = create_vec_batches(&schema, 50);
let partitions = vec![partition];
// repartition from 1 input to 4 output
let output_partitions =
repartition(&schema, partitions, Partitioning::RoundRobinBatch(4)).await?;
assert_eq!(4, output_partitions.len());
assert_eq!(13, output_partitions[0].len());
assert_eq!(13, output_partitions[1].len());
assert_eq!(12, output_partitions[2].len());
assert_eq!(12, output_partitions[3].len());
Ok(())
}
#[tokio::test]
async fn many_to_one_round_robin() -> Result<()> {
// define input partitions
let schema = test_schema();
let partition = create_vec_batches(&schema, 50);
let partitions = vec![partition.clone(), partition.clone(), partition.clone()];
// repartition from 3 input to 1 output
let output_partitions =
repartition(&schema, partitions, Partitioning::RoundRobinBatch(1)).await?;
assert_eq!(1, output_partitions.len());
assert_eq!(150, output_partitions[0].len());
Ok(())
}
#[tokio::test]
async fn many_to_many_round_robin() -> Result<()> {
// define input partitions
let schema = test_schema();
let partition = create_vec_batches(&schema, 50);
let partitions = vec![partition.clone(), partition.clone(), partition.clone()];
// repartition from 3 input to 5 output
let output_partitions =
repartition(&schema, partitions, Partitioning::RoundRobinBatch(5)).await?;
assert_eq!(5, output_partitions.len());
assert_eq!(30, output_partitions[0].len());
assert_eq!(30, output_partitions[1].len());
assert_eq!(30, output_partitions[2].len());
assert_eq!(30, output_partitions[3].len());
assert_eq!(30, output_partitions[4].len());
Ok(())
}
#[tokio::test]
async fn many_to_many_hash_partition() -> Result<()> {
// define input partitions
let schema = test_schema();
let partition = create_vec_batches(&schema, 50);
let partitions = vec![partition.clone(), partition.clone(), partition.clone()];
let output_partitions = repartition(
&schema,
partitions,
Partitioning::Hash(vec![col("c0", &schema)?], 8),
)
.await?;
let total_rows: usize = output_partitions
.iter()
.map(|x| x.iter().map(|x| x.num_rows()).sum::<usize>())
.sum();
assert_eq!(8, output_partitions.len());
assert_eq!(total_rows, 8 * 50 * 3);
Ok(())
}
fn test_schema() -> Arc<Schema> {
Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)]))
}
async fn repartition(
schema: &SchemaRef,
input_partitions: Vec<Vec<RecordBatch>>,
partitioning: Partitioning,
) -> Result<Vec<Vec<RecordBatch>>> {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
// create physical plan
let exec = MemoryExec::try_new(&input_partitions, schema.clone(), None)?;
let exec = RepartitionExec::try_new(Arc::new(exec), partitioning)?;
// execute and collect results
let mut output_partitions = vec![];
for i in 0..exec.partitioning.partition_count() {
// execute this *output* partition and collect all batches
let mut stream = exec.execute(i, task_ctx.clone())?;
let mut batches = vec![];
while let Some(result) = stream.next().await {
batches.push(result?);
}
output_partitions.push(batches);
}
Ok(output_partitions)
}
#[tokio::test]
async fn many_to_many_round_robin_within_tokio_task() -> Result<()> {
let join_handle: JoinHandle<Result<Vec<Vec<RecordBatch>>>> =
tokio::spawn(async move {
// define input partitions
let schema = test_schema();
let partition = create_vec_batches(&schema, 50);
let partitions =
vec![partition.clone(), partition.clone(), partition.clone()];
// repartition from 3 input to 5 output
repartition(&schema, partitions, Partitioning::RoundRobinBatch(5)).await
});
let output_partitions = join_handle.await.unwrap().unwrap();
assert_eq!(5, output_partitions.len());
assert_eq!(30, output_partitions[0].len());
assert_eq!(30, output_partitions[1].len());
assert_eq!(30, output_partitions[2].len());
assert_eq!(30, output_partitions[3].len());
assert_eq!(30, output_partitions[4].len());
Ok(())
}
#[tokio::test]
async fn unsupported_partitioning() {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
// have to send at least one batch through to provoke error
let batch = RecordBatch::try_from_iter(vec![(
"my_awesome_field",
Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef,
)])
.unwrap();
let schema = batch.schema();
let input = MockExec::new(vec![Ok(batch)], schema);
// This generates an error (partitioning type not supported)
// but only after the plan is executed. The error should be
// returned and no results produced
let partitioning = Partitioning::UnknownPartitioning(1);
let exec = RepartitionExec::try_new(Arc::new(input), partitioning).unwrap();