-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
sort.rs
1592 lines (1418 loc) · 56.2 KB
/
sort.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.
//! Sort that deals with an arbitrary size of the input.
//! It will do in-memory sorting if it has enough memory budget
//! but spills to disk if needed.
use std::any::Any;
use std::fmt;
use std::fmt::{Debug, Formatter};
use std::sync::Arc;
use crate::common::spawn_buffered;
use crate::expressions::PhysicalSortExpr;
use crate::limit::LimitStream;
use crate::metrics::{
BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet,
};
use crate::sorts::streaming_merge::StreamingMergeBuilder;
use crate::spill::{read_spill_as_stream, spill_record_batches};
use crate::stream::RecordBatchStreamAdapter;
use crate::topk::TopK;
use crate::{
DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionMode,
ExecutionPlan, ExecutionPlanProperties, Partitioning, PlanProperties,
SendableRecordBatchStream, Statistics,
};
use arrow::compute::{concat_batches, lexsort_to_indices, take_arrays, SortColumn};
use arrow::datatypes::SchemaRef;
use arrow::record_batch::RecordBatch;
use arrow::row::{RowConverter, SortField};
use arrow_array::{Array, RecordBatchOptions, UInt32Array};
use arrow_schema::DataType;
use datafusion_common::{internal_err, Result};
use datafusion_execution::disk_manager::RefCountedTempFile;
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion_execution::runtime_env::RuntimeEnv;
use datafusion_execution::TaskContext;
use datafusion_physical_expr::LexOrdering;
use datafusion_physical_expr_common::sort_expr::LexRequirement;
use crate::execution_plan::CardinalityEffect;
use futures::{StreamExt, TryStreamExt};
use log::{debug, trace};
struct ExternalSorterMetrics {
/// metrics
baseline: BaselineMetrics,
/// count of spills during the execution of the operator
spill_count: Count,
/// total spilled bytes during the execution of the operator
spilled_bytes: Count,
/// total spilled rows during the execution of the operator
spilled_rows: Count,
}
impl ExternalSorterMetrics {
fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self {
Self {
baseline: BaselineMetrics::new(metrics, partition),
spill_count: MetricBuilder::new(metrics).spill_count(partition),
spilled_bytes: MetricBuilder::new(metrics).spilled_bytes(partition),
spilled_rows: MetricBuilder::new(metrics).spilled_rows(partition),
}
}
}
/// Sorts an arbitrary sized, unsorted, stream of [`RecordBatch`]es to
/// a total order. Depending on the input size and memory manager
/// configuration, writes intermediate results to disk ("spills")
/// using Arrow IPC format.
///
/// # Algorithm
///
/// 1. get a non-empty new batch from input
///
/// 2. check with the memory manager there is sufficient space to
/// buffer the batch in memory 2.1 if memory sufficient, buffer
/// batch in memory, go to 1.
///
/// 2.2 if no more memory is available, sort all buffered batches and
/// spill to file. buffer the next batch in memory, go to 1.
///
/// 3. when input is exhausted, merge all in memory batches and spills
/// to get a total order.
///
/// # When data fits in available memory
///
/// If there is sufficient memory, data is sorted in memory to produce the output
///
/// ```text
/// ┌─────┐
/// │ 2 │
/// │ 3 │
/// │ 1 │─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
/// │ 4 │
/// │ 2 │ │
/// └─────┘ ▼
/// ┌─────┐
/// │ 1 │ In memory
/// │ 4 │─ ─ ─ ─ ─ ─▶ sort/merge ─ ─ ─ ─ ─▶ total sorted output
/// │ 1 │
/// └─────┘ ▲
/// ... │
///
/// ┌─────┐ │
/// │ 4 │
/// │ 3 │─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
/// └─────┘
///
/// in_mem_batches
///
/// ```
///
/// # When data does not fit in available memory
///
/// When memory is exhausted, data is first sorted and written to one
/// or more spill files on disk:
///
/// ```text
/// ┌─────┐ .─────────────────.
/// │ 2 │ ( )
/// │ 3 │ │`─────────────────'│
/// │ 1 │─ ─ ─ ─ ─ ─ ─ │ ┌────┐ │
/// │ 4 │ │ │ │ 1 │░ │
/// │ 2 │ │ │... │░ │
/// └─────┘ ▼ │ │ 4 │░ ┌ ─ ─ │
/// ┌─────┐ │ └────┘░ 1 │░ │
/// │ 1 │ In memory │ ░░░░░░ │ ░░ │
/// │ 4 │─ ─ ▶ sort/merge ─ ─ ─ ─ ┼ ─ ─ ─ ─ ─▶ ... │░ │
/// │ 1 │ and write to file │ │ ░░ │
/// └─────┘ │ 4 │░ │
/// ... ▲ │ └░─░─░░ │
/// │ │ ░░░░░░ │
/// ┌─────┐ │.─────────────────.│
/// │ 4 │ │ ( )
/// │ 3 │─ ─ ─ ─ ─ ─ ─ `─────────────────'
/// └─────┘
///
/// in_mem_batches spills
/// (file on disk in Arrow
/// IPC format)
/// ```
///
/// Once the input is completely read, the spill files are read and
/// merged with any in memory batches to produce a single total sorted
/// output:
///
/// ```text
/// .─────────────────.
/// ( )
/// │`─────────────────'│
/// │ ┌────┐ │
/// │ │ 1 │░ │
/// │ │... │─ ─ ─ ─ ─ ─│─ ─ ─ ─ ─ ─
/// │ │ 4 │░ ┌────┐ │ │
/// │ └────┘░ │ 1 │░ │ ▼
/// │ ░░░░░░ │ │░ │
/// │ │... │─ ─│─ ─ ─ ▶ merge ─ ─ ─▶ total sorted output
/// │ │ │░ │
/// │ │ 4 │░ │ ▲
/// │ └────┘░ │ │
/// │ ░░░░░░ │
/// │.─────────────────.│ │
/// ( )
/// `─────────────────' │
/// spills
/// │
///
/// │
///
/// ┌─────┐ │
/// │ 1 │
/// │ 4 │─ ─ ─ ─ │
/// └─────┘ │
/// ... In memory
/// └ ─ ─ ─▶ sort/merge
/// ┌─────┐
/// │ 4 │ ▲
/// │ 3 │─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘
/// └─────┘
///
/// in_mem_batches
/// ```
struct ExternalSorter {
/// schema of the output (and the input)
schema: SchemaRef,
/// Potentially unsorted in memory buffer
in_mem_batches: Vec<RecordBatch>,
/// if `Self::in_mem_batches` are sorted
in_mem_batches_sorted: bool,
/// If data has previously been spilled, the locations of the
/// spill files (in Arrow IPC format)
spills: Vec<RefCountedTempFile>,
/// Sort expressions
expr: Arc<[PhysicalSortExpr]>,
/// Runtime metrics
metrics: ExternalSorterMetrics,
/// If Some, the maximum number of output rows that will be
/// produced.
fetch: Option<usize>,
/// Reservation for in_mem_batches
reservation: MemoryReservation,
/// Reservation for the merging of in-memory batches. If the sort
/// might spill, `sort_spill_reservation_bytes` will be
/// pre-reserved to ensure there is some space for this sort/merge.
merge_reservation: MemoryReservation,
/// A handle to the runtime to get spill files
runtime: Arc<RuntimeEnv>,
/// The target number of rows for output batches
batch_size: usize,
/// How much memory to reserve for performing in-memory sort/merges
/// prior to spilling.
sort_spill_reservation_bytes: usize,
/// If the in size of buffered memory batches is below this size,
/// the data will be concatenated and sorted in place rather than
/// sort/merged.
sort_in_place_threshold_bytes: usize,
}
impl ExternalSorter {
// TODO: make a builder or some other nicer API to avoid the
// clippy warning
#[allow(clippy::too_many_arguments)]
pub fn new(
partition_id: usize,
schema: SchemaRef,
expr: LexOrdering,
batch_size: usize,
fetch: Option<usize>,
sort_spill_reservation_bytes: usize,
sort_in_place_threshold_bytes: usize,
metrics: &ExecutionPlanMetricsSet,
runtime: Arc<RuntimeEnv>,
) -> Self {
let metrics = ExternalSorterMetrics::new(metrics, partition_id);
let reservation = MemoryConsumer::new(format!("ExternalSorter[{partition_id}]"))
.with_can_spill(true)
.register(&runtime.memory_pool);
let merge_reservation =
MemoryConsumer::new(format!("ExternalSorterMerge[{partition_id}]"))
.register(&runtime.memory_pool);
Self {
schema,
in_mem_batches: vec![],
in_mem_batches_sorted: true,
spills: vec![],
expr: expr.inner.into(),
metrics,
fetch,
reservation,
merge_reservation,
runtime,
batch_size,
sort_spill_reservation_bytes,
sort_in_place_threshold_bytes,
}
}
/// Appends an unsorted [`RecordBatch`] to `in_mem_batches`
///
/// Updates memory usage metrics, and possibly triggers spilling to disk
async fn insert_batch(&mut self, input: RecordBatch) -> Result<()> {
if input.num_rows() == 0 {
return Ok(());
}
self.reserve_memory_for_merge()?;
let size = input.get_array_memory_size();
if self.reservation.try_grow(size).is_err() {
let before = self.reservation.size();
self.in_mem_sort().await?;
// Sorting may have freed memory, especially if fetch is `Some`
//
// As such we check again, and if the memory usage has dropped by
// a factor of 2, and we can allocate the necessary capacity,
// we don't spill
//
// The factor of 2 aims to avoid a degenerate case where the
// memory required for `fetch` is just under the memory available,
// causing repeated re-sorting of data
if self.reservation.size() > before / 2
|| self.reservation.try_grow(size).is_err()
{
self.spill().await?;
self.reservation.try_grow(size)?
}
}
self.in_mem_batches.push(input);
self.in_mem_batches_sorted = false;
Ok(())
}
fn spilled_before(&self) -> bool {
!self.spills.is_empty()
}
/// Returns the final sorted output of all batches inserted via
/// [`Self::insert_batch`] as a stream of [`RecordBatch`]es.
///
/// This process could either be:
///
/// 1. An in-memory sort/merge (if the input fit in memory)
///
/// 2. A combined streaming merge incorporating both in-memory
/// batches and data from spill files on disk.
fn sort(&mut self) -> Result<SendableRecordBatchStream> {
if self.spilled_before() {
let mut streams = vec![];
if !self.in_mem_batches.is_empty() {
let in_mem_stream =
self.in_mem_sort_stream(self.metrics.baseline.intermediate())?;
streams.push(in_mem_stream);
}
for spill in self.spills.drain(..) {
if !spill.path().exists() {
return internal_err!("Spill file {:?} does not exist", spill.path());
}
let stream = read_spill_as_stream(spill, Arc::clone(&self.schema), 2)?;
streams.push(stream);
}
let expressions: LexOrdering = self.expr.iter().cloned().collect();
StreamingMergeBuilder::new()
.with_streams(streams)
.with_schema(Arc::clone(&self.schema))
.with_expressions(expressions.as_ref())
.with_metrics(self.metrics.baseline.clone())
.with_batch_size(self.batch_size)
.with_fetch(self.fetch)
.with_reservation(self.reservation.new_empty())
.build()
} else {
self.in_mem_sort_stream(self.metrics.baseline.clone())
}
}
/// How much memory is buffered in this `ExternalSorter`?
fn used(&self) -> usize {
self.reservation.size()
}
/// How many bytes have been spilled to disk?
fn spilled_bytes(&self) -> usize {
self.metrics.spilled_bytes.value()
}
/// How many rows have been spilled to disk?
fn spilled_rows(&self) -> usize {
self.metrics.spilled_rows.value()
}
/// How many spill files have been created?
fn spill_count(&self) -> usize {
self.metrics.spill_count.value()
}
/// Writes any `in_memory_batches` to a spill file and clears
/// the batches. The contents of the spill file are sorted.
///
/// Returns the amount of memory freed.
async fn spill(&mut self) -> Result<usize> {
// we could always get a chance to free some memory as long as we are holding some
if self.in_mem_batches.is_empty() {
return Ok(0);
}
debug!("Spilling sort data of ExternalSorter to disk whilst inserting");
self.in_mem_sort().await?;
let spill_file = self.runtime.disk_manager.create_tmp_file("Sorting")?;
let batches = std::mem::take(&mut self.in_mem_batches);
let spilled_rows = spill_record_batches(
batches,
spill_file.path().into(),
Arc::clone(&self.schema),
)?;
let used = self.reservation.free();
self.metrics.spill_count.add(1);
self.metrics.spilled_bytes.add(used);
self.metrics.spilled_rows.add(spilled_rows);
self.spills.push(spill_file);
Ok(used)
}
/// Sorts the in_mem_batches in place
async fn in_mem_sort(&mut self) -> Result<()> {
if self.in_mem_batches_sorted {
return Ok(());
}
// Release the memory reserved for merge back to the pool so
// there is some left when `in_memo_sort_stream` requests an
// allocation.
self.merge_reservation.free();
self.in_mem_batches = self
.in_mem_sort_stream(self.metrics.baseline.intermediate())?
.try_collect()
.await?;
let size: usize = self
.in_mem_batches
.iter()
.map(|x| x.get_array_memory_size())
.sum();
// Reserve headroom for next sort/merge
self.reserve_memory_for_merge()?;
self.reservation.try_resize(size)?;
self.in_mem_batches_sorted = true;
Ok(())
}
/// Consumes in_mem_batches returning a sorted stream of
/// batches. This proceeds in one of two ways:
///
/// # Small Datasets
///
/// For "smaller" datasets, the data is first concatenated into a
/// single batch and then sorted. This is often faster than
/// sorting and then merging.
///
/// ```text
/// ┌─────┐
/// │ 2 │
/// │ 3 │
/// │ 1 │─ ─ ─ ─ ┐ ┌─────┐
/// │ 4 │ │ 2 │
/// │ 2 │ │ │ 3 │
/// └─────┘ │ 1 │ sorted output
/// ┌─────┐ ▼ │ 4 │ stream
/// │ 1 │ │ 2 │
/// │ 4 │─ ─▶ concat ─ ─ ─ ─ ▶│ 1 │─ ─ ▶ sort ─ ─ ─ ─ ─▶
/// │ 1 │ │ 4 │
/// └─────┘ ▲ │ 1 │
/// ... │ │ ... │
/// │ 4 │
/// ┌─────┐ │ │ 3 │
/// │ 4 │ └─────┘
/// │ 3 │─ ─ ─ ─ ┘
/// └─────┘
/// in_mem_batches
/// ```
///
/// # Larger datasets
///
/// For larger datasets, the batches are first sorted individually
/// and then merged together.
///
/// ```text
/// ┌─────┐ ┌─────┐
/// │ 2 │ │ 1 │
/// │ 3 │ │ 2 │
/// │ 1 │─ ─▶ sort ─ ─▶│ 2 │─ ─ ─ ─ ─ ┐
/// │ 4 │ │ 3 │
/// │ 2 │ │ 4 │ │
/// └─────┘ └─────┘ sorted output
/// ┌─────┐ ┌─────┐ ▼ stream
/// │ 1 │ │ 1 │
/// │ 4 │─ ▶ sort ─ ─ ▶│ 1 ├ ─ ─ ▶ merge ─ ─ ─ ─▶
/// │ 1 │ │ 4 │
/// └─────┘ └─────┘ ▲
/// ... ... ... │
///
/// ┌─────┐ ┌─────┐ │
/// │ 4 │ │ 3 │
/// │ 3 │─ ▶ sort ─ ─ ▶│ 4 │─ ─ ─ ─ ─ ┘
/// └─────┘ └─────┘
///
/// in_mem_batches
/// ```
fn in_mem_sort_stream(
&mut self,
metrics: BaselineMetrics,
) -> Result<SendableRecordBatchStream> {
if self.in_mem_batches.is_empty() {
return Ok(Box::pin(EmptyRecordBatchStream::new(Arc::clone(
&self.schema,
))));
}
// The elapsed compute timer is updated when the value is dropped.
// There is no need for an explicit call to drop.
let elapsed_compute = metrics.elapsed_compute().clone();
let _timer = elapsed_compute.timer();
if self.in_mem_batches.len() == 1 {
let batch = self.in_mem_batches.swap_remove(0);
let reservation = self.reservation.take();
return self.sort_batch_stream(batch, metrics, reservation);
}
// If less than sort_in_place_threshold_bytes, concatenate and sort in place
if self.reservation.size() < self.sort_in_place_threshold_bytes {
// Concatenate memory batches together and sort
let batch = concat_batches(&self.schema, &self.in_mem_batches)?;
self.in_mem_batches.clear();
self.reservation.try_resize(batch.get_array_memory_size())?;
let reservation = self.reservation.take();
return self.sort_batch_stream(batch, metrics, reservation);
}
let streams = std::mem::take(&mut self.in_mem_batches)
.into_iter()
.map(|batch| {
let metrics = self.metrics.baseline.intermediate();
let reservation = self.reservation.split(batch.get_array_memory_size());
let input = self.sort_batch_stream(batch, metrics, reservation)?;
Ok(spawn_buffered(input, 1))
})
.collect::<Result<_>>()?;
let expressions: LexOrdering = self.expr.iter().cloned().collect();
StreamingMergeBuilder::new()
.with_streams(streams)
.with_schema(Arc::clone(&self.schema))
.with_expressions(expressions.as_ref())
.with_metrics(metrics)
.with_batch_size(self.batch_size)
.with_fetch(self.fetch)
.with_reservation(self.merge_reservation.new_empty())
.build()
}
/// Sorts a single `RecordBatch` into a single stream.
///
/// `reservation` accounts for the memory used by this batch and
/// is released when the sort is complete
fn sort_batch_stream(
&self,
batch: RecordBatch,
metrics: BaselineMetrics,
reservation: MemoryReservation,
) -> Result<SendableRecordBatchStream> {
assert_eq!(batch.get_array_memory_size(), reservation.size());
let schema = batch.schema();
let fetch = self.fetch;
let expressions: LexOrdering = self.expr.iter().cloned().collect();
let stream = futures::stream::once(futures::future::lazy(move |_| {
let timer = metrics.elapsed_compute().timer();
let sorted = sort_batch(&batch, &expressions, fetch)?;
timer.done();
metrics.record_output(sorted.num_rows());
drop(batch);
drop(reservation);
Ok(sorted)
}));
Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
}
/// If this sort may spill, pre-allocates
/// `sort_spill_reservation_bytes` of memory to gurarantee memory
/// left for the in memory sort/merge.
fn reserve_memory_for_merge(&mut self) -> Result<()> {
// Reserve headroom for next merge sort
if self.runtime.disk_manager.tmp_files_enabled() {
let size = self.sort_spill_reservation_bytes;
if self.merge_reservation.size() != size {
self.merge_reservation.try_resize(size)?;
}
}
Ok(())
}
}
impl Debug for ExternalSorter {
fn fmt(&self, f: &mut Formatter) -> fmt::Result {
f.debug_struct("ExternalSorter")
.field("memory_used", &self.used())
.field("spilled_bytes", &self.spilled_bytes())
.field("spilled_rows", &self.spilled_rows())
.field("spill_count", &self.spill_count())
.finish()
}
}
pub fn sort_batch(
batch: &RecordBatch,
expressions: &LexOrdering,
fetch: Option<usize>,
) -> Result<RecordBatch> {
let sort_columns = expressions
.iter()
.map(|expr| expr.evaluate_to_sort_column(batch))
.collect::<Result<Vec<_>>>()?;
let indices = if is_multi_column_with_lists(&sort_columns) {
// lex_sort_to_indices doesn't support List with more than one column
// https://github.com/apache/arrow-rs/issues/5454
lexsort_to_indices_multi_columns(sort_columns, fetch)?
} else {
lexsort_to_indices(&sort_columns, fetch)?
};
let columns = take_arrays(batch.columns(), &indices, None)?;
let options = RecordBatchOptions::new().with_row_count(Some(indices.len()));
Ok(RecordBatch::try_new_with_options(
batch.schema(),
columns,
&options,
)?)
}
#[inline]
fn is_multi_column_with_lists(sort_columns: &[SortColumn]) -> bool {
sort_columns.iter().any(|c| {
matches!(
c.values.data_type(),
DataType::List(_) | DataType::LargeList(_) | DataType::FixedSizeList(_, _)
)
})
}
pub(crate) fn lexsort_to_indices_multi_columns(
sort_columns: Vec<SortColumn>,
limit: Option<usize>,
) -> Result<UInt32Array> {
let (fields, columns) = sort_columns.into_iter().fold(
(vec![], vec![]),
|(mut fields, mut columns), sort_column| {
fields.push(SortField::new_with_options(
sort_column.values.data_type().clone(),
sort_column.options.unwrap_or_default(),
));
columns.push(sort_column.values);
(fields, columns)
},
);
// TODO reuse converter and rows, refer to TopK.
let converter = RowConverter::new(fields)?;
let rows = converter.convert_columns(&columns)?;
let mut sort: Vec<_> = rows.iter().enumerate().collect();
sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b));
let mut len = rows.num_rows();
if let Some(limit) = limit {
len = limit.min(len);
}
let indices =
UInt32Array::from_iter_values(sort.iter().take(len).map(|(i, _)| *i as u32));
Ok(indices)
}
/// Sort execution plan.
///
/// Support sorting datasets that are larger than the memory allotted
/// by the memory manager, by spilling to disk.
#[derive(Debug, Clone)]
pub struct SortExec {
/// Input schema
pub(crate) input: Arc<dyn ExecutionPlan>,
/// Sort expressions
expr: LexOrdering,
/// Containing all metrics set created during sort
metrics_set: ExecutionPlanMetricsSet,
/// Preserve partitions of input plan. If false, the input partitions
/// will be sorted and merged into a single output partition.
preserve_partitioning: bool,
/// Fetch highest/lowest n results
fetch: Option<usize>,
/// Cache holding plan properties like equivalences, output partitioning etc.
cache: PlanProperties,
}
impl SortExec {
/// Create a new sort execution plan that produces a single,
/// sorted output partition.
pub fn new(expr: LexOrdering, input: Arc<dyn ExecutionPlan>) -> Self {
let preserve_partitioning = false;
let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning);
Self {
expr,
input,
metrics_set: ExecutionPlanMetricsSet::new(),
preserve_partitioning,
fetch: None,
cache,
}
}
/// Whether this `SortExec` preserves partitioning of the children
pub fn preserve_partitioning(&self) -> bool {
self.preserve_partitioning
}
/// Specify the partitioning behavior of this sort exec
///
/// If `preserve_partitioning` is true, sorts each partition
/// individually, producing one sorted stream for each input partition.
///
/// If `preserve_partitioning` is false, sorts and merges all
/// input partitions producing a single, sorted partition.
pub fn with_preserve_partitioning(mut self, preserve_partitioning: bool) -> Self {
self.preserve_partitioning = preserve_partitioning;
self.cache = self
.cache
.with_partitioning(Self::output_partitioning_helper(
&self.input,
self.preserve_partitioning,
));
self
}
/// Modify how many rows to include in the result
///
/// If None, then all rows will be returned, in sorted order.
/// If Some, then only the top `fetch` rows will be returned.
/// This can reduce the memory pressure required by the sort
/// operation since rows that are not going to be included
/// can be dropped.
pub fn with_fetch(&self, fetch: Option<usize>) -> Self {
let mut cache = self.cache.clone();
if fetch.is_some() && self.cache.execution_mode == ExecutionMode::Unbounded {
// When a theoretically unnecessary sort becomes a top-K (which
// sometimes arises as an intermediate state before full removal),
// its execution mode should become `Bounded`.
cache.execution_mode = ExecutionMode::Bounded;
}
SortExec {
input: Arc::clone(&self.input),
expr: self.expr.clone(),
metrics_set: self.metrics_set.clone(),
preserve_partitioning: self.preserve_partitioning,
fetch,
cache,
}
}
/// Input schema
pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
&self.input
}
/// Sort expressions
pub fn expr(&self) -> &LexOrdering {
&self.expr
}
/// If `Some(fetch)`, limits output to only the first "fetch" items
pub fn fetch(&self) -> Option<usize> {
self.fetch
}
fn output_partitioning_helper(
input: &Arc<dyn ExecutionPlan>,
preserve_partitioning: bool,
) -> Partitioning {
// Get output partitioning:
if preserve_partitioning {
input.output_partitioning().clone()
} else {
Partitioning::UnknownPartitioning(1)
}
}
/// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
fn compute_properties(
input: &Arc<dyn ExecutionPlan>,
sort_exprs: LexOrdering,
preserve_partitioning: bool,
) -> PlanProperties {
// Determine execution mode:
let requirement = LexRequirement::from(sort_exprs);
let sort_satisfied = input
.equivalence_properties()
.ordering_satisfy_requirement(&requirement);
let mode = match input.execution_mode() {
ExecutionMode::Unbounded if sort_satisfied => ExecutionMode::Unbounded,
ExecutionMode::Bounded => ExecutionMode::Bounded,
_ => ExecutionMode::PipelineBreaking,
};
// Calculate equivalence properties; i.e. reset the ordering equivalence
// class with the new ordering:
let sort_exprs = LexOrdering::from(requirement);
let eq_properties = input
.equivalence_properties()
.clone()
.with_reorder(sort_exprs);
// Get output partitioning:
let output_partitioning =
Self::output_partitioning_helper(input, preserve_partitioning);
PlanProperties::new(eq_properties, output_partitioning, mode)
}
}
impl DisplayAs for SortExec {
fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
let preserve_partitioning = self.preserve_partitioning;
match self.fetch {
Some(fetch) => {
write!(f, "SortExec: TopK(fetch={fetch}), expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr)
}
None => write!(f, "SortExec: expr=[{}], preserve_partitioning=[{preserve_partitioning}]", self.expr),
}
}
}
}
}
impl ExecutionPlan for SortExec {
fn name(&self) -> &'static str {
"SortExec"
}
fn as_any(&self) -> &dyn Any {
self
}
fn properties(&self) -> &PlanProperties {
&self.cache
}
fn required_input_distribution(&self) -> Vec<Distribution> {
if self.preserve_partitioning {
vec![Distribution::UnspecifiedDistribution]
} else {
// global sort
// TODO support RangePartition and OrderedDistribution
vec![Distribution::SinglePartition]
}
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![&self.input]
}
fn benefits_from_input_partitioning(&self) -> Vec<bool> {
vec![false]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
let new_sort = SortExec::new(self.expr.clone(), Arc::clone(&children[0]))
.with_fetch(self.fetch)
.with_preserve_partitioning(self.preserve_partitioning);
Ok(Arc::new(new_sort))
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
trace!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id());
let mut input = self.input.execute(partition, Arc::clone(&context))?;
let execution_options = &context.session_config().options().execution;
trace!("End SortExec's input.execute for partition: {}", partition);
let sort_satisfied = self
.input
.equivalence_properties()
.ordering_satisfy_requirement(&LexRequirement::from(self.expr.clone()));
match (sort_satisfied, self.fetch.as_ref()) {
(true, Some(fetch)) => Ok(Box::pin(LimitStream::new(
input,
0,
Some(*fetch),
BaselineMetrics::new(&self.metrics_set, partition),
))),
(true, None) => Ok(input),
(false, Some(fetch)) => {
let mut topk = TopK::try_new(
partition,
input.schema(),
self.expr.clone(),
*fetch,
context.session_config().batch_size(),
context.runtime_env(),
&self.metrics_set,
partition,
)?;
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
futures::stream::once(async move {
while let Some(batch) = input.next().await {
let batch = batch?;
topk.insert_batch(batch)?;
}
topk.emit()
})
.try_flatten(),
)))
}
(false, None) => {
let mut sorter = ExternalSorter::new(
partition,
input.schema(),
self.expr.clone(),
context.session_config().batch_size(),
self.fetch,
execution_options.sort_spill_reservation_bytes,
execution_options.sort_in_place_threshold_bytes,
&self.metrics_set,
context.runtime_env(),
);
Ok(Box::pin(RecordBatchStreamAdapter::new(
self.schema(),
futures::stream::once(async move {
while let Some(batch) = input.next().await {
let batch = batch?;
sorter.insert_batch(batch).await?;
}
sorter.sort()
})
.try_flatten(),
)))
}
}
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics_set.clone_inner())
}
fn statistics(&self) -> Result<Statistics> {
Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1)
}
fn with_fetch(&self, limit: Option<usize>) -> Option<Arc<dyn ExecutionPlan>> {
Some(Arc::new(SortExec::with_fetch(self, limit)))
}
fn fetch(&self) -> Option<usize> {
self.fetch
}
fn cardinality_effect(&self) -> CardinalityEffect {
if self.fetch.is_none() {
CardinalityEffect::Equal
} else {
CardinalityEffect::LowerEqual
}
}
}
#[cfg(test)]
mod tests {
use std::collections::HashMap;
use std::pin::Pin;
use std::task::{Context, Poll};
use super::*;
use crate::coalesce_partitions::CoalescePartitionsExec;
use crate::collect;
use crate::expressions::col;
use crate::memory::MemoryExec;
use crate::test;
use crate::test::assert_is_pending;
use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec};
use arrow::array::*;
use arrow::compute::SortOptions;
use arrow::datatypes::*;
use datafusion_common::cast::as_primitive_array;
use datafusion_common::{assert_batches_eq, Result, ScalarValue};
use datafusion_execution::config::SessionConfig;