forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
/
sort_preserving_merge.rs
1407 lines (1243 loc) · 50.6 KB
/
sort_preserving_merge.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.
//! Defines the sort preserving merge plan
use std::any::Any;
use std::cmp::Reverse;
use std::collections::{BinaryHeap, VecDeque};
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use arrow::error::ArrowError;
use arrow::row::{RowConverter, SortField};
use arrow::{
array::{make_array as make_arrow_array, MutableArrayData},
datatypes::SchemaRef,
error::Result as ArrowResult,
record_batch::RecordBatch,
};
use futures::stream::{Fuse, FusedStream};
use futures::{Stream, StreamExt};
use log::debug;
use tokio::sync::mpsc;
use crate::error::{DataFusionError, Result};
use crate::execution::context::TaskContext;
use crate::physical_plan::metrics::{
ExecutionPlanMetricsSet, MemTrackingMetrics, MetricsSet,
};
use crate::physical_plan::sorts::{RowIndex, SortKeyCursor, SortedStream};
use crate::physical_plan::stream::RecordBatchReceiverStream;
use crate::physical_plan::{
common::spawn_execution, expressions::PhysicalSortExpr, DisplayFormatType,
Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream,
SendableRecordBatchStream, Statistics,
};
use datafusion_physical_expr::EquivalenceProperties;
/// Sort preserving merge execution plan
///
/// This takes an input execution plan and a list of sort expressions, and
/// provided each partition of the input plan is sorted with respect to
/// these sort expressions, this operator will yield a single partition
/// that is also sorted with respect to them
///
/// ```text
/// ┌─────────────────────────┐
/// │ ┌───┬───┬───┬───┐ │
/// │ │ A │ B │ C │ D │ ... │──┐
/// │ └───┴───┴───┴───┘ │ │
/// └─────────────────────────┘ │ ┌───────────────────┐ ┌───────────────────────────────┐
/// Stream 1 │ │ │ │ ┌───┬───╦═══╦───┬───╦═══╗ │
/// ├─▶│SortPreservingMerge│───▶│ │ A │ B ║ B ║ C │ D ║ E ║ ... │
/// │ │ │ │ └───┴─▲─╩═══╩───┴───╩═══╝ │
/// ┌─────────────────────────┐ │ └───────────────────┘ └─┬─────┴───────────────────────┘
/// │ ╔═══╦═══╗ │ │
/// │ ║ B ║ E ║ ... │──┘ │
/// │ ╚═══╩═══╝ │ Note Stable Sort: the merged stream
/// └─────────────────────────┘ places equal rows from stream 1
/// Stream 2
///
///
/// Input Streams Output stream
/// (sorted) (sorted)
/// ```
#[derive(Debug)]
pub struct SortPreservingMergeExec {
/// Input plan
input: Arc<dyn ExecutionPlan>,
/// Sort expressions
expr: Vec<PhysicalSortExpr>,
/// Execution metrics
metrics: ExecutionPlanMetricsSet,
}
impl SortPreservingMergeExec {
/// Create a new sort execution plan
pub fn new(expr: Vec<PhysicalSortExpr>, input: Arc<dyn ExecutionPlan>) -> Self {
Self {
input,
expr,
metrics: ExecutionPlanMetricsSet::new(),
}
}
/// Input schema
pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
&self.input
}
/// Sort expressions
pub fn expr(&self) -> &[PhysicalSortExpr] {
&self.expr
}
}
impl ExecutionPlan for SortPreservingMergeExec {
/// Return a reference to Any that can be used for downcasting
fn as_any(&self) -> &dyn Any {
self
}
fn schema(&self) -> SchemaRef {
self.input.schema()
}
/// Get the output partitioning of this plan
fn output_partitioning(&self) -> Partitioning {
Partitioning::UnknownPartitioning(1)
}
fn required_input_distribution(&self) -> Vec<Distribution> {
vec![Distribution::UnspecifiedDistribution]
}
fn required_input_ordering(&self) -> Vec<Option<&[PhysicalSortExpr]>> {
vec![Some(&self.expr)]
}
fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> {
Some(&self.expr)
}
fn equivalence_properties(&self) -> EquivalenceProperties {
self.input.equivalence_properties()
}
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>> {
Ok(Arc::new(SortPreservingMergeExec::new(
self.expr.clone(),
children[0].clone(),
)))
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
debug!(
"Start SortPreservingMergeExec::execute for partition: {}",
partition
);
if 0 != partition {
return Err(DataFusionError::Internal(format!(
"SortPreservingMergeExec invalid partition {}",
partition
)));
}
let tracking_metrics = MemTrackingMetrics::new(&self.metrics, partition);
let input_partitions = self.input.output_partitioning().partition_count();
debug!(
"Number of input partitions of SortPreservingMergeExec::execute: {}",
input_partitions
);
let schema = self.schema();
match input_partitions {
0 => Err(DataFusionError::Internal(
"SortPreservingMergeExec requires at least one input partition"
.to_owned(),
)),
1 => {
// bypass if there is only one partition to merge (no metrics in this case either)
let result = self.input.execute(0, context);
debug!("Done getting stream for SortPreservingMergeExec::execute with 1 input");
result
}
_ => {
// Use tokio only if running from a tokio context (#2201)
let receivers = match tokio::runtime::Handle::try_current() {
Ok(_) => (0..input_partitions)
.into_iter()
.map(|part_i| {
let (sender, receiver) = mpsc::channel(1);
let join_handle = spawn_execution(
self.input.clone(),
sender,
part_i,
context.clone(),
);
SortedStream::new(
RecordBatchReceiverStream::create(
&schema,
receiver,
join_handle,
),
0,
)
})
.collect(),
Err(_) => (0..input_partitions)
.map(|partition| {
let stream =
self.input.execute(partition, context.clone())?;
Ok(SortedStream::new(stream, 0))
})
.collect::<Result<_>>()?,
};
debug!("Done setting up sender-receiver for SortPreservingMergeExec::execute");
let result = Box::pin(SortPreservingMergeStream::new_from_streams(
receivers,
schema,
&self.expr,
tracking_metrics,
context.session_config().batch_size(),
)?);
debug!("Got stream result from SortPreservingMergeStream::new_from_receivers");
Ok(result)
}
}
}
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default => {
let expr: Vec<String> = self.expr.iter().map(|e| e.to_string()).collect();
write!(f, "SortPreservingMergeExec: [{}]", expr.join(","))
}
}
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Statistics {
self.input.statistics()
}
}
struct MergingStreams {
/// The sorted input streams to merge together
streams: Vec<Fuse<SendableRecordBatchStream>>,
/// number of streams
num_streams: usize,
}
impl std::fmt::Debug for MergingStreams {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("MergingStreams")
.field("num_streams", &self.num_streams)
.finish()
}
}
impl MergingStreams {
fn new(input_streams: Vec<Fuse<SendableRecordBatchStream>>) -> Self {
Self {
num_streams: input_streams.len(),
streams: input_streams,
}
}
fn num_streams(&self) -> usize {
self.num_streams
}
}
#[derive(Debug)]
pub(crate) struct SortPreservingMergeStream {
/// The schema of the RecordBatches yielded by this stream
schema: SchemaRef,
/// The sorted input streams to merge together
streams: MergingStreams,
/// For each input stream maintain a dequeue of RecordBatches
///
/// Exhausted batches will be popped off the front once all
/// their rows have been yielded to the output
batches: Vec<VecDeque<RecordBatch>>,
/// Maintain a flag for each stream denoting if the current cursor
/// has finished and needs to poll from the stream
cursor_finished: Vec<bool>,
/// The accumulated row indexes for the next record batch
in_progress: Vec<RowIndex>,
/// The physical expressions to sort by
column_expressions: Vec<Arc<dyn PhysicalExpr>>,
/// used to record execution metrics
tracking_metrics: MemTrackingMetrics,
/// If the stream has encountered an error
aborted: bool,
/// An id to uniquely identify the input stream batch
next_batch_id: usize,
/// Heap that yields [`SortKeyCursor`] in increasing order
heap: BinaryHeap<Reverse<SortKeyCursor>>,
/// target batch size
batch_size: usize,
/// row converter
row_converter: RowConverter,
}
impl SortPreservingMergeStream {
pub(crate) fn new_from_streams(
streams: Vec<SortedStream>,
schema: SchemaRef,
expressions: &[PhysicalSortExpr],
tracking_metrics: MemTrackingMetrics,
batch_size: usize,
) -> Result<Self> {
let stream_count = streams.len();
let batches = (0..stream_count)
.into_iter()
.map(|_| VecDeque::new())
.collect();
tracking_metrics.init_mem_used(streams.iter().map(|s| s.mem_used).sum());
let wrappers = streams.into_iter().map(|s| s.stream.fuse()).collect();
let sort_fields = expressions
.iter()
.map(|expr| {
let data_type = expr.expr.data_type(&schema)?;
Ok(SortField::new_with_options(data_type, expr.options))
})
.collect::<Result<Vec<_>>>()?;
let row_converter = RowConverter::new(sort_fields)?;
Ok(Self {
schema,
batches,
cursor_finished: vec![true; stream_count],
streams: MergingStreams::new(wrappers),
column_expressions: expressions.iter().map(|x| x.expr.clone()).collect(),
tracking_metrics,
aborted: false,
in_progress: vec![],
next_batch_id: 0,
heap: BinaryHeap::with_capacity(stream_count),
batch_size,
row_converter,
})
}
/// If the stream at the given index is not exhausted, and the last cursor for the
/// stream is finished, poll the stream for the next RecordBatch and create a new
/// cursor for the stream from the returned result
fn maybe_poll_stream(
&mut self,
cx: &mut Context<'_>,
idx: usize,
) -> Poll<ArrowResult<()>> {
if !self.cursor_finished[idx] {
// Cursor is not finished - don't need a new RecordBatch yet
return Poll::Ready(Ok(()));
}
let mut empty_batch = false;
{
let stream = &mut self.streams.streams[idx];
if stream.is_terminated() {
return Poll::Ready(Ok(()));
}
// Fetch a new input record and create a cursor from it
match futures::ready!(stream.poll_next_unpin(cx)) {
None => return Poll::Ready(Ok(())),
Some(Err(e)) => {
return Poll::Ready(Err(e));
}
Some(Ok(batch)) => {
if batch.num_rows() > 0 {
let cols = self
.column_expressions
.iter()
.map(|expr| {
Ok(expr.evaluate(&batch)?.into_array(batch.num_rows()))
})
.collect::<Result<Vec<_>>>()?;
let rows = match self.row_converter.convert_columns(&cols) {
Ok(rows) => rows,
Err(e) => {
return Poll::Ready(Err(ArrowError::ExternalError(
Box::new(e),
)));
}
};
let cursor = SortKeyCursor::new(
idx,
self.next_batch_id, // assign this batch an ID
rows,
);
self.next_batch_id += 1;
self.heap.push(Reverse(cursor));
self.cursor_finished[idx] = false;
self.batches[idx].push_back(batch)
} else {
empty_batch = true;
}
}
}
}
if empty_batch {
self.maybe_poll_stream(cx, idx)
} else {
Poll::Ready(Ok(()))
}
}
/// Drains the in_progress row indexes, and builds a new RecordBatch from them
///
/// Will then drop any batches for which all rows have been yielded to the output
fn build_record_batch(&mut self) -> ArrowResult<RecordBatch> {
// Mapping from stream index to the index of the first buffer from that stream
let mut buffer_idx = 0;
let mut stream_to_buffer_idx = Vec::with_capacity(self.batches.len());
for batches in &self.batches {
stream_to_buffer_idx.push(buffer_idx);
buffer_idx += batches.len();
}
let columns = self
.schema
.fields()
.iter()
.enumerate()
.map(|(column_idx, field)| {
let arrays = self
.batches
.iter()
.flat_map(|batch| {
batch.iter().map(|batch| batch.column(column_idx).data())
})
.collect();
let mut array_data = MutableArrayData::new(
arrays,
field.is_nullable(),
self.in_progress.len(),
);
if self.in_progress.is_empty() {
return make_arrow_array(array_data.freeze());
}
let first = &self.in_progress[0];
let mut buffer_idx =
stream_to_buffer_idx[first.stream_idx] + first.batch_idx;
let mut start_row_idx = first.row_idx;
let mut end_row_idx = start_row_idx + 1;
for row_index in self.in_progress.iter().skip(1) {
let next_buffer_idx =
stream_to_buffer_idx[row_index.stream_idx] + row_index.batch_idx;
if next_buffer_idx == buffer_idx && row_index.row_idx == end_row_idx {
// subsequent row in same batch
end_row_idx += 1;
continue;
}
// emit current batch of rows for current buffer
array_data.extend(buffer_idx, start_row_idx, end_row_idx);
// start new batch of rows
buffer_idx = next_buffer_idx;
start_row_idx = row_index.row_idx;
end_row_idx = start_row_idx + 1;
}
// emit final batch of rows
array_data.extend(buffer_idx, start_row_idx, end_row_idx);
make_arrow_array(array_data.freeze())
})
.collect();
self.in_progress.clear();
// New cursors are only created once the previous cursor for the stream
// is finished. This means all remaining rows from all but the last batch
// for each stream have been yielded to the newly created record batch
//
// Additionally as `in_progress` has been drained, there are no longer
// any RowIndex's reliant on the batch indexes
//
// We can therefore drop all but the last batch for each stream
for batches in &mut self.batches {
if batches.len() > 1 {
// Drain all but the last batch
batches.drain(0..(batches.len() - 1));
}
}
RecordBatch::try_new(self.schema.clone(), columns)
}
}
impl Stream for SortPreservingMergeStream {
type Item = ArrowResult<RecordBatch>;
fn poll_next(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
let poll = self.poll_next_inner(cx);
self.tracking_metrics.record_poll(poll)
}
}
impl SortPreservingMergeStream {
#[inline]
fn poll_next_inner(
self: &mut Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<ArrowResult<RecordBatch>>> {
if self.aborted {
return Poll::Ready(None);
}
// Ensure all non-exhausted streams have a cursor from which
// rows can be pulled
for i in 0..self.streams.num_streams() {
match futures::ready!(self.maybe_poll_stream(cx, i)) {
Ok(_) => {}
Err(e) => {
self.aborted = true;
return Poll::Ready(Some(Err(e)));
}
}
}
// NB timer records time taken on drop, so there are no
// calls to `timer.done()` below.
let elapsed_compute = self.tracking_metrics.elapsed_compute().clone();
let _timer = elapsed_compute.timer();
loop {
match self.heap.pop() {
Some(Reverse(mut cursor)) => {
let stream_idx = cursor.stream_idx();
let batch_idx = self.batches[stream_idx].len() - 1;
let row_idx = cursor.advance();
let mut cursor_finished = false;
// insert the cursor back to heap if the record batch is not exhausted
if !cursor.is_finished() {
self.heap.push(Reverse(cursor));
} else {
cursor_finished = true;
self.cursor_finished[stream_idx] = true;
}
self.in_progress.push(RowIndex {
stream_idx,
batch_idx,
row_idx,
});
if self.in_progress.len() == self.batch_size {
return Poll::Ready(Some(self.build_record_batch()));
}
// If removed the last row from the cursor, need to fetch a new record
// batch if possible, before looping round again
if cursor_finished {
match futures::ready!(self.maybe_poll_stream(cx, stream_idx)) {
Ok(_) => {}
Err(e) => {
self.aborted = true;
return Poll::Ready(Some(Err(e)));
}
}
}
}
None if self.in_progress.is_empty() => return Poll::Ready(None),
None => return Poll::Ready(Some(self.build_record_batch())),
}
}
}
}
impl RecordBatchStream for SortPreservingMergeStream {
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
#[cfg(test)]
mod tests {
use std::iter::FromIterator;
use arrow::array::ArrayRef;
use arrow::compute::SortOptions;
use arrow::datatypes::{DataType, Field, Schema};
use futures::FutureExt;
use tokio_stream::StreamExt;
use crate::arrow::array::{Int32Array, StringArray, TimestampNanosecondArray};
use crate::from_slice::FromSlice;
use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec;
use crate::physical_plan::expressions::col;
use crate::physical_plan::memory::MemoryExec;
use crate::physical_plan::metrics::MetricValue;
use crate::physical_plan::sorts::sort::SortExec;
use crate::physical_plan::{collect, common};
use crate::prelude::{SessionConfig, SessionContext};
use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec};
use crate::test::{self, assert_is_pending};
use crate::{assert_batches_eq, test_util};
use super::*;
#[tokio::test]
async fn test_merge_interleave() {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let a: ArrayRef = Arc::new(Int32Array::from_slice([1, 2, 7, 9, 3]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("a"),
Some("c"),
Some("e"),
Some("g"),
Some("j"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![8, 7, 6, 5, 8]));
let b1 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
let a: ArrayRef = Arc::new(Int32Array::from_slice([10, 20, 70, 90, 30]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("b"),
Some("d"),
Some("f"),
Some("h"),
Some("j"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![4, 6, 2, 2, 6]));
let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
_test_merge(
&[vec![b1], vec![b2]],
&[
"+----+---+-------------------------------+",
"| a | b | c |",
"+----+---+-------------------------------+",
"| 1 | a | 1970-01-01T00:00:00.000000008 |",
"| 10 | b | 1970-01-01T00:00:00.000000004 |",
"| 2 | c | 1970-01-01T00:00:00.000000007 |",
"| 20 | d | 1970-01-01T00:00:00.000000006 |",
"| 7 | e | 1970-01-01T00:00:00.000000006 |",
"| 70 | f | 1970-01-01T00:00:00.000000002 |",
"| 9 | g | 1970-01-01T00:00:00.000000005 |",
"| 90 | h | 1970-01-01T00:00:00.000000002 |",
"| 30 | j | 1970-01-01T00:00:00.000000006 |", // input b2 before b1
"| 3 | j | 1970-01-01T00:00:00.000000008 |",
"+----+---+-------------------------------+",
],
task_ctx,
)
.await;
}
#[tokio::test]
async fn test_merge_some_overlap() {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let a: ArrayRef = Arc::new(Int32Array::from_slice([1, 2, 7, 9, 3]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("a"),
Some("b"),
Some("c"),
Some("d"),
Some("e"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![8, 7, 6, 5, 8]));
let b1 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
let a: ArrayRef = Arc::new(Int32Array::from_slice([70, 90, 30, 100, 110]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("c"),
Some("d"),
Some("e"),
Some("f"),
Some("g"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![4, 6, 2, 2, 6]));
let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
_test_merge(
&[vec![b1], vec![b2]],
&[
"+-----+---+-------------------------------+",
"| a | b | c |",
"+-----+---+-------------------------------+",
"| 1 | a | 1970-01-01T00:00:00.000000008 |",
"| 2 | b | 1970-01-01T00:00:00.000000007 |",
"| 70 | c | 1970-01-01T00:00:00.000000004 |",
"| 7 | c | 1970-01-01T00:00:00.000000006 |",
"| 9 | d | 1970-01-01T00:00:00.000000005 |",
"| 90 | d | 1970-01-01T00:00:00.000000006 |",
"| 30 | e | 1970-01-01T00:00:00.000000002 |",
"| 3 | e | 1970-01-01T00:00:00.000000008 |",
"| 100 | f | 1970-01-01T00:00:00.000000002 |",
"| 110 | g | 1970-01-01T00:00:00.000000006 |",
"+-----+---+-------------------------------+",
],
task_ctx,
)
.await;
}
#[tokio::test]
async fn test_merge_no_overlap() {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let a: ArrayRef = Arc::new(Int32Array::from_slice([1, 2, 7, 9, 3]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("a"),
Some("b"),
Some("c"),
Some("d"),
Some("e"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![8, 7, 6, 5, 8]));
let b1 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
let a: ArrayRef = Arc::new(Int32Array::from_slice([10, 20, 70, 90, 30]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("f"),
Some("g"),
Some("h"),
Some("i"),
Some("j"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![4, 6, 2, 2, 6]));
let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
_test_merge(
&[vec![b1], vec![b2]],
&[
"+----+---+-------------------------------+",
"| a | b | c |",
"+----+---+-------------------------------+",
"| 1 | a | 1970-01-01T00:00:00.000000008 |",
"| 2 | b | 1970-01-01T00:00:00.000000007 |",
"| 7 | c | 1970-01-01T00:00:00.000000006 |",
"| 9 | d | 1970-01-01T00:00:00.000000005 |",
"| 3 | e | 1970-01-01T00:00:00.000000008 |",
"| 10 | f | 1970-01-01T00:00:00.000000004 |",
"| 20 | g | 1970-01-01T00:00:00.000000006 |",
"| 70 | h | 1970-01-01T00:00:00.000000002 |",
"| 90 | i | 1970-01-01T00:00:00.000000002 |",
"| 30 | j | 1970-01-01T00:00:00.000000006 |",
"+----+---+-------------------------------+",
],
task_ctx,
)
.await;
}
#[tokio::test]
async fn test_merge_three_partitions() {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let a: ArrayRef = Arc::new(Int32Array::from_slice([1, 2, 7, 9, 3]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("a"),
Some("b"),
Some("c"),
Some("d"),
Some("f"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![8, 7, 6, 5, 8]));
let b1 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
let a: ArrayRef = Arc::new(Int32Array::from_slice([10, 20, 70, 90, 30]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("e"),
Some("g"),
Some("h"),
Some("i"),
Some("j"),
]));
let c: ArrayRef =
Arc::new(TimestampNanosecondArray::from(vec![40, 60, 20, 20, 60]));
let b2 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
let a: ArrayRef = Arc::new(Int32Array::from_slice([100, 200, 700, 900, 300]));
let b: ArrayRef = Arc::new(StringArray::from_iter(vec![
Some("f"),
Some("g"),
Some("h"),
Some("i"),
Some("j"),
]));
let c: ArrayRef = Arc::new(TimestampNanosecondArray::from(vec![4, 6, 2, 2, 6]));
let b3 = RecordBatch::try_from_iter(vec![("a", a), ("b", b), ("c", c)]).unwrap();
_test_merge(
&[vec![b1], vec![b2], vec![b3]],
&[
"+-----+---+-------------------------------+",
"| a | b | c |",
"+-----+---+-------------------------------+",
"| 1 | a | 1970-01-01T00:00:00.000000008 |",
"| 2 | b | 1970-01-01T00:00:00.000000007 |",
"| 7 | c | 1970-01-01T00:00:00.000000006 |",
"| 9 | d | 1970-01-01T00:00:00.000000005 |",
"| 10 | e | 1970-01-01T00:00:00.000000040 |",
"| 100 | f | 1970-01-01T00:00:00.000000004 |",
"| 3 | f | 1970-01-01T00:00:00.000000008 |",
"| 200 | g | 1970-01-01T00:00:00.000000006 |",
"| 20 | g | 1970-01-01T00:00:00.000000060 |",
"| 700 | h | 1970-01-01T00:00:00.000000002 |",
"| 70 | h | 1970-01-01T00:00:00.000000020 |",
"| 900 | i | 1970-01-01T00:00:00.000000002 |",
"| 90 | i | 1970-01-01T00:00:00.000000020 |",
"| 300 | j | 1970-01-01T00:00:00.000000006 |",
"| 30 | j | 1970-01-01T00:00:00.000000060 |",
"+-----+---+-------------------------------+",
],
task_ctx,
)
.await;
}
async fn _test_merge(
partitions: &[Vec<RecordBatch>],
exp: &[&str],
context: Arc<TaskContext>,
) {
let schema = partitions[0][0].schema();
let sort = vec![
PhysicalSortExpr {
expr: col("b", &schema).unwrap(),
options: Default::default(),
},
PhysicalSortExpr {
expr: col("c", &schema).unwrap(),
options: Default::default(),
},
];
let exec = MemoryExec::try_new(partitions, schema, None).unwrap();
let merge = Arc::new(SortPreservingMergeExec::new(sort, Arc::new(exec)));
let collected = collect(merge, context).await.unwrap();
assert_batches_eq!(exp, collected.as_slice());
}
async fn sorted_merge(
input: Arc<dyn ExecutionPlan>,
sort: Vec<PhysicalSortExpr>,
context: Arc<TaskContext>,
) -> RecordBatch {
let merge = Arc::new(SortPreservingMergeExec::new(sort, input));
let mut result = collect(merge, context).await.unwrap();
assert_eq!(result.len(), 1);
result.remove(0)
}
async fn partition_sort(
input: Arc<dyn ExecutionPlan>,
sort: Vec<PhysicalSortExpr>,
context: Arc<TaskContext>,
) -> RecordBatch {
let sort_exec = Arc::new(SortExec::new_with_partitioning(
sort.clone(),
input,
true,
None,
));
sorted_merge(sort_exec, sort, context).await
}
async fn basic_sort(
src: Arc<dyn ExecutionPlan>,
sort: Vec<PhysicalSortExpr>,
context: Arc<TaskContext>,
) -> RecordBatch {
let merge = Arc::new(CoalescePartitionsExec::new(src));
let sort_exec = Arc::new(SortExec::try_new(sort, merge, None).unwrap());
let mut result = collect(sort_exec, context).await.unwrap();
assert_eq!(result.len(), 1);
result.remove(0)
}
#[tokio::test]
async fn test_partition_sort() {
let session_ctx = SessionContext::new();
let task_ctx = session_ctx.task_ctx();
let partitions = 4;
let csv = test::scan_partitioned_csv(partitions).unwrap();
let schema = csv.schema();
let sort = vec![
PhysicalSortExpr {
expr: col("c1", &schema).unwrap(),
options: SortOptions {
descending: true,
nulls_first: true,
},
},
PhysicalSortExpr {
expr: col("c2", &schema).unwrap(),
options: Default::default(),
},
PhysicalSortExpr {
expr: col("c7", &schema).unwrap(),
options: SortOptions::default(),
},
PhysicalSortExpr {
expr: col("c12", &schema).unwrap(),
options: SortOptions::default(),
},
];
let basic = basic_sort(csv.clone(), sort.clone(), task_ctx.clone()).await;
let partition = partition_sort(csv, sort, task_ctx.clone()).await;
let basic = arrow::util::pretty::pretty_format_batches(&[basic])
.unwrap()
.to_string();
let partition = arrow::util::pretty::pretty_format_batches(&[partition])
.unwrap()
.to_string();
assert_eq!(
basic, partition,
"basic:\n\n{}\n\npartition:\n\n{}\n\n",
basic, partition
);
}
// Split the provided record batch into multiple batch_size record batches
fn split_batch(sorted: &RecordBatch, batch_size: usize) -> Vec<RecordBatch> {
let batches = (sorted.num_rows() + batch_size - 1) / batch_size;
// Split the sorted RecordBatch into multiple
(0..batches)
.into_iter()
.map(|batch_idx| {
let columns = (0..sorted.num_columns())
.map(|column_idx| {
let length =
batch_size.min(sorted.num_rows() - batch_idx * batch_size);
sorted
.column(column_idx)
.slice(batch_idx * batch_size, length)
})
.collect();
RecordBatch::try_new(sorted.schema(), columns).unwrap()
})
.collect()
}
async fn sorted_partitioned_input(
sort: Vec<PhysicalSortExpr>,
sizes: &[usize],
context: Arc<TaskContext>,
) -> Arc<dyn ExecutionPlan> {
let partitions = 4;
let csv = test::scan_partitioned_csv(partitions).unwrap();
let sorted = basic_sort(csv, sort, context).await;