/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/sorts/sort.rs
Line | Count | Source (jump to first uncovered line) |
1 | | // Licensed to the Apache Software Foundation (ASF) under one |
2 | | // or more contributor license agreements. See the NOTICE file |
3 | | // distributed with this work for additional information |
4 | | // regarding copyright ownership. The ASF licenses this file |
5 | | // to you under the Apache License, Version 2.0 (the |
6 | | // "License"); you may not use this file except in compliance |
7 | | // with the License. You may obtain a copy of the License at |
8 | | // |
9 | | // http://www.apache.org/licenses/LICENSE-2.0 |
10 | | // |
11 | | // Unless required by applicable law or agreed to in writing, |
12 | | // software distributed under the License is distributed on an |
13 | | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
14 | | // KIND, either express or implied. See the License for the |
15 | | // specific language governing permissions and limitations |
16 | | // under the License. |
17 | | |
18 | | //! Sort that deals with an arbitrary size of the input. |
19 | | //! It will do in-memory sorting if it has enough memory budget |
20 | | //! but spills to disk if needed. |
21 | | |
22 | | use std::any::Any; |
23 | | use std::fmt; |
24 | | use std::fmt::{Debug, Formatter}; |
25 | | use std::sync::Arc; |
26 | | |
27 | | use crate::common::spawn_buffered; |
28 | | use crate::expressions::PhysicalSortExpr; |
29 | | use crate::limit::LimitStream; |
30 | | use crate::metrics::{ |
31 | | BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, |
32 | | }; |
33 | | use crate::sorts::streaming_merge::StreamingMergeBuilder; |
34 | | use crate::spill::{read_spill_as_stream, spill_record_batches}; |
35 | | use crate::stream::RecordBatchStreamAdapter; |
36 | | use crate::topk::TopK; |
37 | | use crate::{ |
38 | | DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionMode, |
39 | | ExecutionPlan, ExecutionPlanProperties, Partitioning, PlanProperties, |
40 | | SendableRecordBatchStream, Statistics, |
41 | | }; |
42 | | |
43 | | use arrow::compute::{concat_batches, lexsort_to_indices, SortColumn}; |
44 | | use arrow::datatypes::SchemaRef; |
45 | | use arrow::record_batch::RecordBatch; |
46 | | use arrow::row::{RowConverter, SortField}; |
47 | | use arrow_array::{Array, RecordBatchOptions, UInt32Array}; |
48 | | use arrow_schema::DataType; |
49 | | use datafusion_common::utils::take_arrays; |
50 | | use datafusion_common::{internal_err, Result}; |
51 | | use datafusion_execution::disk_manager::RefCountedTempFile; |
52 | | use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; |
53 | | use datafusion_execution::runtime_env::RuntimeEnv; |
54 | | use datafusion_execution::TaskContext; |
55 | | use datafusion_physical_expr::LexOrdering; |
56 | | use datafusion_physical_expr_common::sort_expr::PhysicalSortRequirement; |
57 | | |
58 | | use futures::{StreamExt, TryStreamExt}; |
59 | | use log::{debug, trace}; |
60 | | |
61 | | struct ExternalSorterMetrics { |
62 | | /// metrics |
63 | | baseline: BaselineMetrics, |
64 | | |
65 | | /// count of spills during the execution of the operator |
66 | | spill_count: Count, |
67 | | |
68 | | /// total spilled bytes during the execution of the operator |
69 | | spilled_bytes: Count, |
70 | | |
71 | | /// total spilled rows during the execution of the operator |
72 | | spilled_rows: Count, |
73 | | } |
74 | | |
75 | | impl ExternalSorterMetrics { |
76 | 19 | fn new(metrics: &ExecutionPlanMetricsSet, partition: usize) -> Self { |
77 | 19 | Self { |
78 | 19 | baseline: BaselineMetrics::new(metrics, partition), |
79 | 19 | spill_count: MetricBuilder::new(metrics).spill_count(partition), |
80 | 19 | spilled_bytes: MetricBuilder::new(metrics).spilled_bytes(partition), |
81 | 19 | spilled_rows: MetricBuilder::new(metrics).spilled_rows(partition), |
82 | 19 | } |
83 | 19 | } |
84 | | } |
85 | | |
86 | | /// Sorts an arbitrary sized, unsorted, stream of [`RecordBatch`]es to |
87 | | /// a total order. Depending on the input size and memory manager |
88 | | /// configuration, writes intermediate results to disk ("spills") |
89 | | /// using Arrow IPC format. |
90 | | /// |
91 | | /// # Algorithm |
92 | | /// |
93 | | /// 1. get a non-empty new batch from input |
94 | | /// |
95 | | /// 2. check with the memory manager there is sufficient space to |
96 | | /// buffer the batch in memory 2.1 if memory sufficient, buffer |
97 | | /// batch in memory, go to 1. |
98 | | /// |
99 | | /// 2.2 if no more memory is available, sort all buffered batches and |
100 | | /// spill to file. buffer the next batch in memory, go to 1. |
101 | | /// |
102 | | /// 3. when input is exhausted, merge all in memory batches and spills |
103 | | /// to get a total order. |
104 | | /// |
105 | | /// # When data fits in available memory |
106 | | /// |
107 | | /// If there is sufficient memory, data is sorted in memory to produce the output |
108 | | /// |
109 | | /// ```text |
110 | | /// ┌─────┐ |
111 | | /// │ 2 │ |
112 | | /// │ 3 │ |
113 | | /// │ 1 │─ ─ ─ ─ ─ ─ ─ ─ ─ ┐ |
114 | | /// │ 4 │ |
115 | | /// │ 2 │ │ |
116 | | /// └─────┘ ▼ |
117 | | /// ┌─────┐ |
118 | | /// │ 1 │ In memory |
119 | | /// │ 4 │─ ─ ─ ─ ─ ─▶ sort/merge ─ ─ ─ ─ ─▶ total sorted output |
120 | | /// │ 1 │ |
121 | | /// └─────┘ ▲ |
122 | | /// ... │ |
123 | | /// |
124 | | /// ┌─────┐ │ |
125 | | /// │ 4 │ |
126 | | /// │ 3 │─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ |
127 | | /// └─────┘ |
128 | | /// |
129 | | /// in_mem_batches |
130 | | /// |
131 | | /// ``` |
132 | | /// |
133 | | /// # When data does not fit in available memory |
134 | | /// |
135 | | /// When memory is exhausted, data is first sorted and written to one |
136 | | /// or more spill files on disk: |
137 | | /// |
138 | | /// ```text |
139 | | /// ┌─────┐ .─────────────────. |
140 | | /// │ 2 │ ( ) |
141 | | /// │ 3 │ │`─────────────────'│ |
142 | | /// │ 1 │─ ─ ─ ─ ─ ─ ─ │ ┌────┐ │ |
143 | | /// │ 4 │ │ │ │ 1 │░ │ |
144 | | /// │ 2 │ │ │... │░ │ |
145 | | /// └─────┘ ▼ │ │ 4 │░ ┌ ─ ─ │ |
146 | | /// ┌─────┐ │ └────┘░ 1 │░ │ |
147 | | /// │ 1 │ In memory │ ░░░░░░ │ ░░ │ |
148 | | /// │ 4 │─ ─ ▶ sort/merge ─ ─ ─ ─ ┼ ─ ─ ─ ─ ─▶ ... │░ │ |
149 | | /// │ 1 │ and write to file │ │ ░░ │ |
150 | | /// └─────┘ │ 4 │░ │ |
151 | | /// ... ▲ │ └░─░─░░ │ |
152 | | /// │ │ ░░░░░░ │ |
153 | | /// ┌─────┐ │.─────────────────.│ |
154 | | /// │ 4 │ │ ( ) |
155 | | /// │ 3 │─ ─ ─ ─ ─ ─ ─ `─────────────────' |
156 | | /// └─────┘ |
157 | | /// |
158 | | /// in_mem_batches spills |
159 | | /// (file on disk in Arrow |
160 | | /// IPC format) |
161 | | /// ``` |
162 | | /// |
163 | | /// Once the input is completely read, the spill files are read and |
164 | | /// merged with any in memory batches to produce a single total sorted |
165 | | /// output: |
166 | | /// |
167 | | /// ```text |
168 | | /// .─────────────────. |
169 | | /// ( ) |
170 | | /// │`─────────────────'│ |
171 | | /// │ ┌────┐ │ |
172 | | /// │ │ 1 │░ │ |
173 | | /// │ │... │─ ─ ─ ─ ─ ─│─ ─ ─ ─ ─ ─ |
174 | | /// │ │ 4 │░ ┌────┐ │ │ |
175 | | /// │ └────┘░ │ 1 │░ │ ▼ |
176 | | /// │ ░░░░░░ │ │░ │ |
177 | | /// │ │... │─ ─│─ ─ ─ ▶ merge ─ ─ ─▶ total sorted output |
178 | | /// │ │ │░ │ |
179 | | /// │ │ 4 │░ │ ▲ |
180 | | /// │ └────┘░ │ │ |
181 | | /// │ ░░░░░░ │ |
182 | | /// │.─────────────────.│ │ |
183 | | /// ( ) |
184 | | /// `─────────────────' │ |
185 | | /// spills |
186 | | /// │ |
187 | | /// |
188 | | /// │ |
189 | | /// |
190 | | /// ┌─────┐ │ |
191 | | /// │ 1 │ |
192 | | /// │ 4 │─ ─ ─ ─ │ |
193 | | /// └─────┘ │ |
194 | | /// ... In memory |
195 | | /// └ ─ ─ ─▶ sort/merge |
196 | | /// ┌─────┐ |
197 | | /// │ 4 │ ▲ |
198 | | /// │ 3 │─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┘ |
199 | | /// └─────┘ |
200 | | /// |
201 | | /// in_mem_batches |
202 | | /// ``` |
203 | | struct ExternalSorter { |
204 | | /// schema of the output (and the input) |
205 | | schema: SchemaRef, |
206 | | /// Potentially unsorted in memory buffer |
207 | | in_mem_batches: Vec<RecordBatch>, |
208 | | /// if `Self::in_mem_batches` are sorted |
209 | | in_mem_batches_sorted: bool, |
210 | | /// If data has previously been spilled, the locations of the |
211 | | /// spill files (in Arrow IPC format) |
212 | | spills: Vec<RefCountedTempFile>, |
213 | | /// Sort expressions |
214 | | expr: Arc<[PhysicalSortExpr]>, |
215 | | /// Runtime metrics |
216 | | metrics: ExternalSorterMetrics, |
217 | | /// If Some, the maximum number of output rows that will be |
218 | | /// produced. |
219 | | fetch: Option<usize>, |
220 | | /// Reservation for in_mem_batches |
221 | | reservation: MemoryReservation, |
222 | | /// Reservation for the merging of in-memory batches. If the sort |
223 | | /// might spill, `sort_spill_reservation_bytes` will be |
224 | | /// pre-reserved to ensure there is some space for this sort/merge. |
225 | | merge_reservation: MemoryReservation, |
226 | | /// A handle to the runtime to get spill files |
227 | | runtime: Arc<RuntimeEnv>, |
228 | | /// The target number of rows for output batches |
229 | | batch_size: usize, |
230 | | /// How much memory to reserve for performing in-memory sort/merges |
231 | | /// prior to spilling. |
232 | | sort_spill_reservation_bytes: usize, |
233 | | /// If the in size of buffered memory batches is below this size, |
234 | | /// the data will be concatenated and sorted in place rather than |
235 | | /// sort/merged. |
236 | | sort_in_place_threshold_bytes: usize, |
237 | | } |
238 | | |
239 | | impl ExternalSorter { |
240 | | // TODO: make a builder or some other nicer API to avoid the |
241 | | // clippy warning |
242 | | #[allow(clippy::too_many_arguments)] |
243 | 19 | pub fn new( |
244 | 19 | partition_id: usize, |
245 | 19 | schema: SchemaRef, |
246 | 19 | expr: Vec<PhysicalSortExpr>, |
247 | 19 | batch_size: usize, |
248 | 19 | fetch: Option<usize>, |
249 | 19 | sort_spill_reservation_bytes: usize, |
250 | 19 | sort_in_place_threshold_bytes: usize, |
251 | 19 | metrics: &ExecutionPlanMetricsSet, |
252 | 19 | runtime: Arc<RuntimeEnv>, |
253 | 19 | ) -> Self { |
254 | 19 | let metrics = ExternalSorterMetrics::new(metrics, partition_id); |
255 | 19 | let reservation = MemoryConsumer::new(format!("ExternalSorter[{partition_id}]")) |
256 | 19 | .with_can_spill(true) |
257 | 19 | .register(&runtime.memory_pool); |
258 | 19 | |
259 | 19 | let merge_reservation = |
260 | 19 | MemoryConsumer::new(format!("ExternalSorterMerge[{partition_id}]")) |
261 | 19 | .register(&runtime.memory_pool); |
262 | 19 | |
263 | 19 | Self { |
264 | 19 | schema, |
265 | 19 | in_mem_batches: vec![], |
266 | 19 | in_mem_batches_sorted: true, |
267 | 19 | spills: vec![], |
268 | 19 | expr: expr.into(), |
269 | 19 | metrics, |
270 | 19 | fetch, |
271 | 19 | reservation, |
272 | 19 | merge_reservation, |
273 | 19 | runtime, |
274 | 19 | batch_size, |
275 | 19 | sort_spill_reservation_bytes, |
276 | 19 | sort_in_place_threshold_bytes, |
277 | 19 | } |
278 | 19 | } |
279 | | |
280 | | /// Appends an unsorted [`RecordBatch`] to `in_mem_batches` |
281 | | /// |
282 | | /// Updates memory usage metrics, and possibly triggers spilling to disk |
283 | 769 | async fn insert_batch(&mut self, input: RecordBatch) -> Result<()> { |
284 | 769 | if input.num_rows() == 0 { |
285 | 0 | return Ok(()); |
286 | 769 | } |
287 | 769 | self.reserve_memory_for_merge()?0 ; |
288 | | |
289 | 769 | let size = input.get_array_memory_size(); |
290 | 769 | if self.reservation.try_grow(size).is_err() { |
291 | 5 | let before = self.reservation.size(); |
292 | 5 | self.in_mem_sort().await0 ?0 ; |
293 | | // Sorting may have freed memory, especially if fetch is `Some` |
294 | | // |
295 | | // As such we check again, and if the memory usage has dropped by |
296 | | // a factor of 2, and we can allocate the necessary capacity, |
297 | | // we don't spill |
298 | | // |
299 | | // The factor of 2 aims to avoid a degenerate case where the |
300 | | // memory required for `fetch` is just under the memory available, |
301 | | // causing repeated re-sorting of data |
302 | 5 | if self.reservation.size() > before / 2 |
303 | 0 | || self.reservation.try_grow(size).is_err() |
304 | | { |
305 | 5 | self.spill().await0 ?0 ; |
306 | 5 | self.reservation.try_grow(size)?0 |
307 | 0 | } |
308 | 764 | } |
309 | | |
310 | 769 | self.in_mem_batches.push(input); |
311 | 769 | self.in_mem_batches_sorted = false; |
312 | 769 | Ok(()) |
313 | 769 | } |
314 | | |
315 | 18 | fn spilled_before(&self) -> bool { |
316 | 18 | !self.spills.is_empty() |
317 | 18 | } |
318 | | |
319 | | /// Returns the final sorted output of all batches inserted via |
320 | | /// [`Self::insert_batch`] as a stream of [`RecordBatch`]es. |
321 | | /// |
322 | | /// This process could either be: |
323 | | /// |
324 | | /// 1. An in-memory sort/merge (if the input fit in memory) |
325 | | /// |
326 | | /// 2. A combined streaming merge incorporating both in-memory |
327 | | /// batches and data from spill files on disk. |
328 | 18 | fn sort(&mut self) -> Result<SendableRecordBatchStream> { |
329 | 18 | if self.spilled_before() { |
330 | 2 | let mut streams = vec![]; |
331 | 2 | if !self.in_mem_batches.is_empty() { |
332 | 2 | let in_mem_stream = |
333 | 2 | self.in_mem_sort_stream(self.metrics.baseline.intermediate())?0 ; |
334 | 2 | streams.push(in_mem_stream); |
335 | 0 | } |
336 | | |
337 | 5 | for spill in self.spills.drain(..)2 { |
338 | 5 | if !spill.path().exists() { |
339 | 0 | return internal_err!("Spill file {:?} does not exist", spill.path()); |
340 | 5 | } |
341 | 5 | let stream = read_spill_as_stream(spill, Arc::clone(&self.schema), 2)?0 ; |
342 | 5 | streams.push(stream); |
343 | | } |
344 | | |
345 | 2 | StreamingMergeBuilder::new() |
346 | 2 | .with_streams(streams) |
347 | 2 | .with_schema(Arc::clone(&self.schema)) |
348 | 2 | .with_expressions(&self.expr) |
349 | 2 | .with_metrics(self.metrics.baseline.clone()) |
350 | 2 | .with_batch_size(self.batch_size) |
351 | 2 | .with_fetch(self.fetch) |
352 | 2 | .with_reservation(self.reservation.new_empty()) |
353 | 2 | .build() |
354 | | } else { |
355 | 16 | self.in_mem_sort_stream(self.metrics.baseline.clone()) |
356 | | } |
357 | 18 | } |
358 | | |
359 | | /// How much memory is buffered in this `ExternalSorter`? |
360 | 0 | fn used(&self) -> usize { |
361 | 0 | self.reservation.size() |
362 | 0 | } |
363 | | |
364 | | /// How many bytes have been spilled to disk? |
365 | 0 | fn spilled_bytes(&self) -> usize { |
366 | 0 | self.metrics.spilled_bytes.value() |
367 | 0 | } |
368 | | |
369 | | /// How many rows have been spilled to disk? |
370 | 0 | fn spilled_rows(&self) -> usize { |
371 | 0 | self.metrics.spilled_rows.value() |
372 | 0 | } |
373 | | |
374 | | /// How many spill files have been created? |
375 | 0 | fn spill_count(&self) -> usize { |
376 | 0 | self.metrics.spill_count.value() |
377 | 0 | } |
378 | | |
379 | | /// Writes any `in_memory_batches` to a spill file and clears |
380 | | /// the batches. The contents of the spill file are sorted. |
381 | | /// |
382 | | /// Returns the amount of memory freed. |
383 | 5 | async fn spill(&mut self) -> Result<usize> { |
384 | 5 | // we could always get a chance to free some memory as long as we are holding some |
385 | 5 | if self.in_mem_batches.is_empty() { |
386 | 0 | return Ok(0); |
387 | 5 | } |
388 | 5 | |
389 | 5 | debug!("Spilling sort data of ExternalSorter to disk whilst inserting"0 ); |
390 | | |
391 | 5 | self.in_mem_sort().await0 ?0 ; |
392 | | |
393 | 5 | let spill_file = self.runtime.disk_manager.create_tmp_file("Sorting")?0 ; |
394 | 5 | let batches = std::mem::take(&mut self.in_mem_batches); |
395 | 5 | let spilled_rows = spill_record_batches( |
396 | 5 | batches, |
397 | 5 | spill_file.path().into(), |
398 | 5 | Arc::clone(&self.schema), |
399 | 5 | )?0 ; |
400 | 5 | let used = self.reservation.free(); |
401 | 5 | self.metrics.spill_count.add(1); |
402 | 5 | self.metrics.spilled_bytes.add(used); |
403 | 5 | self.metrics.spilled_rows.add(spilled_rows); |
404 | 5 | self.spills.push(spill_file); |
405 | 5 | Ok(used) |
406 | 5 | } |
407 | | |
408 | | /// Sorts the in_mem_batches in place |
409 | 10 | async fn in_mem_sort(&mut self) -> Result<()> { |
410 | 10 | if self.in_mem_batches_sorted { |
411 | 5 | return Ok(()); |
412 | 5 | } |
413 | 5 | |
414 | 5 | // Release the memory reserved for merge back to the pool so |
415 | 5 | // there is some left when `in_memo_sort_stream` requests an |
416 | 5 | // allocation. |
417 | 5 | self.merge_reservation.free(); |
418 | | |
419 | 5 | self.in_mem_batches = self |
420 | 5 | .in_mem_sort_stream(self.metrics.baseline.intermediate())?0 |
421 | 5 | .try_collect() |
422 | 0 | .await?; |
423 | | |
424 | 5 | let size: usize = self |
425 | 5 | .in_mem_batches |
426 | 5 | .iter() |
427 | 5 | .map(|x| x.get_array_memory_size()) |
428 | 5 | .sum(); |
429 | 5 | |
430 | 5 | // Reserve headroom for next sort/merge |
431 | 5 | self.reserve_memory_for_merge()?0 ; |
432 | | |
433 | 5 | self.reservation.try_resize(size)?0 ; |
434 | 5 | self.in_mem_batches_sorted = true; |
435 | 5 | Ok(()) |
436 | 10 | } |
437 | | |
438 | | /// Consumes in_mem_batches returning a sorted stream of |
439 | | /// batches. This proceeds in one of two ways: |
440 | | /// |
441 | | /// # Small Datasets |
442 | | /// |
443 | | /// For "smaller" datasets, the data is first concatenated into a |
444 | | /// single batch and then sorted. This is often faster than |
445 | | /// sorting and then merging. |
446 | | /// |
447 | | /// ```text |
448 | | /// ┌─────┐ |
449 | | /// │ 2 │ |
450 | | /// │ 3 │ |
451 | | /// │ 1 │─ ─ ─ ─ ┐ ┌─────┐ |
452 | | /// │ 4 │ │ 2 │ |
453 | | /// │ 2 │ │ │ 3 │ |
454 | | /// └─────┘ │ 1 │ sorted output |
455 | | /// ┌─────┐ ▼ │ 4 │ stream |
456 | | /// │ 1 │ │ 2 │ |
457 | | /// │ 4 │─ ─▶ concat ─ ─ ─ ─ ▶│ 1 │─ ─ ▶ sort ─ ─ ─ ─ ─▶ |
458 | | /// │ 1 │ │ 4 │ |
459 | | /// └─────┘ ▲ │ 1 │ |
460 | | /// ... │ │ ... │ |
461 | | /// │ 4 │ |
462 | | /// ┌─────┐ │ │ 3 │ |
463 | | /// │ 4 │ └─────┘ |
464 | | /// │ 3 │─ ─ ─ ─ ┘ |
465 | | /// └─────┘ |
466 | | /// in_mem_batches |
467 | | /// ``` |
468 | | /// |
469 | | /// # Larger datasets |
470 | | /// |
471 | | /// For larger datasets, the batches are first sorted individually |
472 | | /// and then merged together. |
473 | | /// |
474 | | /// ```text |
475 | | /// ┌─────┐ ┌─────┐ |
476 | | /// │ 2 │ │ 1 │ |
477 | | /// │ 3 │ │ 2 │ |
478 | | /// │ 1 │─ ─▶ sort ─ ─▶│ 2 │─ ─ ─ ─ ─ ┐ |
479 | | /// │ 4 │ │ 3 │ |
480 | | /// │ 2 │ │ 4 │ │ |
481 | | /// └─────┘ └─────┘ sorted output |
482 | | /// ┌─────┐ ┌─────┐ ▼ stream |
483 | | /// │ 1 │ │ 1 │ |
484 | | /// │ 4 │─ ▶ sort ─ ─ ▶│ 1 ├ ─ ─ ▶ merge ─ ─ ─ ─▶ |
485 | | /// │ 1 │ │ 4 │ |
486 | | /// └─────┘ └─────┘ ▲ |
487 | | /// ... ... ... │ |
488 | | /// |
489 | | /// ┌─────┐ ┌─────┐ │ |
490 | | /// │ 4 │ │ 3 │ |
491 | | /// │ 3 │─ ▶ sort ─ ─ ▶│ 4 │─ ─ ─ ─ ─ ┘ |
492 | | /// └─────┘ └─────┘ |
493 | | /// |
494 | | /// in_mem_batches |
495 | | /// ``` |
496 | 23 | fn in_mem_sort_stream( |
497 | 23 | &mut self, |
498 | 23 | metrics: BaselineMetrics, |
499 | 23 | ) -> Result<SendableRecordBatchStream> { |
500 | 23 | if self.in_mem_batches.is_empty() { |
501 | 0 | return Ok(Box::pin(EmptyRecordBatchStream::new(Arc::clone( |
502 | 0 | &self.schema, |
503 | 0 | )))); |
504 | 23 | } |
505 | 23 | |
506 | 23 | // The elapsed compute timer is updated when the value is dropped. |
507 | 23 | // There is no need for an explicit call to drop. |
508 | 23 | let elapsed_compute = metrics.elapsed_compute().clone(); |
509 | 23 | let _timer = elapsed_compute.timer(); |
510 | 23 | |
511 | 23 | if self.in_mem_batches.len() == 1 { |
512 | 7 | let batch = self.in_mem_batches.swap_remove(0); |
513 | 7 | let reservation = self.reservation.take(); |
514 | 7 | return self.sort_batch_stream(batch, metrics, reservation); |
515 | 16 | } |
516 | 16 | |
517 | 16 | // If less than sort_in_place_threshold_bytes, concatenate and sort in place |
518 | 16 | if self.reservation.size() < self.sort_in_place_threshold_bytes { |
519 | | // Concatenate memory batches together and sort |
520 | 16 | let batch = concat_batches(&self.schema, &self.in_mem_batches)?0 ; |
521 | 16 | self.in_mem_batches.clear(); |
522 | 16 | self.reservation.try_resize(batch.get_array_memory_size())?0 ; |
523 | 16 | let reservation = self.reservation.take(); |
524 | 16 | return self.sort_batch_stream(batch, metrics, reservation); |
525 | 0 | } |
526 | | |
527 | 0 | let streams = std::mem::take(&mut self.in_mem_batches) |
528 | 0 | .into_iter() |
529 | 0 | .map(|batch| { |
530 | 0 | let metrics = self.metrics.baseline.intermediate(); |
531 | 0 | let reservation = self.reservation.split(batch.get_array_memory_size()); |
532 | 0 | let input = self.sort_batch_stream(batch, metrics, reservation)?; |
533 | 0 | Ok(spawn_buffered(input, 1)) |
534 | 0 | }) |
535 | 0 | .collect::<Result<_>>()?; |
536 | | |
537 | 0 | StreamingMergeBuilder::new() |
538 | 0 | .with_streams(streams) |
539 | 0 | .with_schema(Arc::clone(&self.schema)) |
540 | 0 | .with_expressions(&self.expr) |
541 | 0 | .with_metrics(metrics) |
542 | 0 | .with_batch_size(self.batch_size) |
543 | 0 | .with_fetch(self.fetch) |
544 | 0 | .with_reservation(self.merge_reservation.new_empty()) |
545 | 0 | .build() |
546 | 23 | } |
547 | | |
548 | | /// Sorts a single `RecordBatch` into a single stream. |
549 | | /// |
550 | | /// `reservation` accounts for the memory used by this batch and |
551 | | /// is released when the sort is complete |
552 | 23 | fn sort_batch_stream( |
553 | 23 | &self, |
554 | 23 | batch: RecordBatch, |
555 | 23 | metrics: BaselineMetrics, |
556 | 23 | reservation: MemoryReservation, |
557 | 23 | ) -> Result<SendableRecordBatchStream> { |
558 | 23 | assert_eq!(batch.get_array_memory_size(), reservation.size()); |
559 | 23 | let schema = batch.schema(); |
560 | 23 | |
561 | 23 | let fetch = self.fetch; |
562 | 23 | let expressions = Arc::clone(&self.expr); |
563 | 23 | let stream = futures::stream::once(futures::future::lazy(move |_| { |
564 | 23 | let timer = metrics.elapsed_compute().timer(); |
565 | 23 | let sorted = sort_batch(&batch, &expressions, fetch)?0 ; |
566 | 23 | timer.done(); |
567 | 23 | metrics.record_output(sorted.num_rows()); |
568 | 23 | drop(batch); |
569 | 23 | drop(reservation); |
570 | 23 | Ok(sorted) |
571 | 23 | })); |
572 | 23 | Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream))) |
573 | 23 | } |
574 | | |
575 | | /// If this sort may spill, pre-allocates |
576 | | /// `sort_spill_reservation_bytes` of memory to gurarantee memory |
577 | | /// left for the in memory sort/merge. |
578 | 774 | fn reserve_memory_for_merge(&mut self) -> Result<()> { |
579 | 774 | // Reserve headroom for next merge sort |
580 | 774 | if self.runtime.disk_manager.tmp_files_enabled() { |
581 | 774 | let size = self.sort_spill_reservation_bytes; |
582 | 774 | if self.merge_reservation.size() != size { |
583 | 23 | self.merge_reservation.try_resize(size)?0 ; |
584 | 751 | } |
585 | 0 | } |
586 | | |
587 | 774 | Ok(()) |
588 | 774 | } |
589 | | } |
590 | | |
591 | | impl Debug for ExternalSorter { |
592 | 0 | fn fmt(&self, f: &mut Formatter) -> fmt::Result { |
593 | 0 | f.debug_struct("ExternalSorter") |
594 | 0 | .field("memory_used", &self.used()) |
595 | 0 | .field("spilled_bytes", &self.spilled_bytes()) |
596 | 0 | .field("spilled_rows", &self.spilled_rows()) |
597 | 0 | .field("spill_count", &self.spill_count()) |
598 | 0 | .finish() |
599 | 0 | } |
600 | | } |
601 | | |
602 | 55 | pub fn sort_batch( |
603 | 55 | batch: &RecordBatch, |
604 | 55 | expressions: &[PhysicalSortExpr], |
605 | 55 | fetch: Option<usize>, |
606 | 55 | ) -> Result<RecordBatch> { |
607 | 55 | let sort_columns = expressions |
608 | 55 | .iter() |
609 | 101 | .map(|expr| expr.evaluate_to_sort_column(batch)) |
610 | 55 | .collect::<Result<Vec<_>>>()?0 ; |
611 | | |
612 | 55 | let indices = if is_multi_column_with_lists(&sort_columns) { |
613 | | // lex_sort_to_indices doesn't support List with more than one column |
614 | | // https://github.com/apache/arrow-rs/issues/5454 |
615 | 1 | lexsort_to_indices_multi_columns(sort_columns, fetch)?0 |
616 | | } else { |
617 | 54 | lexsort_to_indices(&sort_columns, fetch)?0 |
618 | | }; |
619 | | |
620 | 55 | let columns = take_arrays(batch.columns(), &indices)?0 ; |
621 | | |
622 | 55 | let options = RecordBatchOptions::new().with_row_count(Some(indices.len())); |
623 | 55 | Ok(RecordBatch::try_new_with_options( |
624 | 55 | batch.schema(), |
625 | 55 | columns, |
626 | 55 | &options, |
627 | 55 | )?0 ) |
628 | 55 | } |
629 | | |
630 | | #[inline] |
631 | 55 | fn is_multi_column_with_lists(sort_columns: &[SortColumn]) -> bool { |
632 | 101 | sort_columns.iter().any(|c| { |
633 | 100 | matches!( |
634 | 101 | c.values.data_type(), |
635 | | DataType::List(_) | DataType::LargeList(_) | DataType::FixedSizeList(_, _) |
636 | | ) |
637 | 101 | }) |
638 | 55 | } |
639 | | |
640 | 1 | pub(crate) fn lexsort_to_indices_multi_columns( |
641 | 1 | sort_columns: Vec<SortColumn>, |
642 | 1 | limit: Option<usize>, |
643 | 1 | ) -> Result<UInt32Array> { |
644 | 1 | let (fields, columns) = sort_columns.into_iter().fold( |
645 | 1 | (vec![], vec![]), |
646 | 2 | |(mut fields, mut columns), sort_column| { |
647 | 2 | fields.push(SortField::new_with_options( |
648 | 2 | sort_column.values.data_type().clone(), |
649 | 2 | sort_column.options.unwrap_or_default(), |
650 | 2 | )); |
651 | 2 | columns.push(sort_column.values); |
652 | 2 | (fields, columns) |
653 | 2 | }, |
654 | 1 | ); |
655 | | |
656 | | // TODO reuse converter and rows, refer to TopK. |
657 | 1 | let converter = RowConverter::new(fields)?0 ; |
658 | 1 | let rows = converter.convert_columns(&columns)?0 ; |
659 | 1 | let mut sort: Vec<_> = rows.iter().enumerate().collect(); |
660 | 5 | sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); |
661 | 1 | |
662 | 1 | let mut len = rows.num_rows(); |
663 | 1 | if let Some(limit0 ) = limit { |
664 | 0 | len = limit.min(len); |
665 | 1 | } |
666 | 1 | let indices = |
667 | 4 | UInt32Array::from_iter_values(sort.iter().take(len).map(|(i, _)| *i as u32)); |
668 | 1 | |
669 | 1 | Ok(indices) |
670 | 1 | } |
671 | | |
672 | | /// Sort execution plan. |
673 | | /// |
674 | | /// Support sorting datasets that are larger than the memory allotted |
675 | | /// by the memory manager, by spilling to disk. |
676 | | #[derive(Debug)] |
677 | | pub struct SortExec { |
678 | | /// Input schema |
679 | | pub(crate) input: Arc<dyn ExecutionPlan>, |
680 | | /// Sort expressions |
681 | | expr: Vec<PhysicalSortExpr>, |
682 | | /// Containing all metrics set created during sort |
683 | | metrics_set: ExecutionPlanMetricsSet, |
684 | | /// Preserve partitions of input plan. If false, the input partitions |
685 | | /// will be sorted and merged into a single output partition. |
686 | | preserve_partitioning: bool, |
687 | | /// Fetch highest/lowest n results |
688 | | fetch: Option<usize>, |
689 | | /// Cache holding plan properties like equivalences, output partitioning etc. |
690 | | cache: PlanProperties, |
691 | | } |
692 | | |
693 | | impl SortExec { |
694 | | /// Create a new sort execution plan that produces a single, |
695 | | /// sorted output partition. |
696 | 22 | pub fn new(expr: Vec<PhysicalSortExpr>, input: Arc<dyn ExecutionPlan>) -> Self { |
697 | 22 | let preserve_partitioning = false; |
698 | 22 | let cache = Self::compute_properties(&input, expr.clone(), preserve_partitioning); |
699 | 22 | Self { |
700 | 22 | expr, |
701 | 22 | input, |
702 | 22 | metrics_set: ExecutionPlanMetricsSet::new(), |
703 | 22 | preserve_partitioning, |
704 | 22 | fetch: None, |
705 | 22 | cache, |
706 | 22 | } |
707 | 22 | } |
708 | | |
709 | | /// Whether this `SortExec` preserves partitioning of the children |
710 | 0 | pub fn preserve_partitioning(&self) -> bool { |
711 | 0 | self.preserve_partitioning |
712 | 0 | } |
713 | | |
714 | | /// Specify the partitioning behavior of this sort exec |
715 | | /// |
716 | | /// If `preserve_partitioning` is true, sorts each partition |
717 | | /// individually, producing one sorted stream for each input partition. |
718 | | /// |
719 | | /// If `preserve_partitioning` is false, sorts and merges all |
720 | | /// input partitions producing a single, sorted partition. |
721 | 1 | pub fn with_preserve_partitioning(mut self, preserve_partitioning: bool) -> Self { |
722 | 1 | self.preserve_partitioning = preserve_partitioning; |
723 | 1 | self.cache = self |
724 | 1 | .cache |
725 | 1 | .with_partitioning(Self::output_partitioning_helper( |
726 | 1 | &self.input, |
727 | 1 | self.preserve_partitioning, |
728 | 1 | )); |
729 | 1 | self |
730 | 1 | } |
731 | | |
732 | | /// Modify how many rows to include in the result |
733 | | /// |
734 | | /// If None, then all rows will be returned, in sorted order. |
735 | | /// If Some, then only the top `fetch` rows will be returned. |
736 | | /// This can reduce the memory pressure required by the sort |
737 | | /// operation since rows that are not going to be included |
738 | | /// can be dropped. |
739 | 7 | pub fn with_fetch(&self, fetch: Option<usize>) -> Self { |
740 | 7 | let mut cache = self.cache.clone(); |
741 | 7 | if fetch.is_some() && self.cache.execution_mode == ExecutionMode::Unbounded6 { |
742 | 1 | // When a theoretically unnecessary sort becomes a top-K (which |
743 | 1 | // sometimes arises as an intermediate state before full removal), |
744 | 1 | // its execution mode should become `Bounded`. |
745 | 1 | cache.execution_mode = ExecutionMode::Bounded; |
746 | 6 | } |
747 | 7 | SortExec { |
748 | 7 | input: Arc::clone(&self.input), |
749 | 7 | expr: self.expr.clone(), |
750 | 7 | metrics_set: self.metrics_set.clone(), |
751 | 7 | preserve_partitioning: self.preserve_partitioning, |
752 | 7 | fetch, |
753 | 7 | cache, |
754 | 7 | } |
755 | 7 | } |
756 | | |
757 | | /// Input schema |
758 | 0 | pub fn input(&self) -> &Arc<dyn ExecutionPlan> { |
759 | 0 | &self.input |
760 | 0 | } |
761 | | |
762 | | /// Sort expressions |
763 | 0 | pub fn expr(&self) -> &[PhysicalSortExpr] { |
764 | 0 | &self.expr |
765 | 0 | } |
766 | | |
767 | | /// If `Some(fetch)`, limits output to only the first "fetch" items |
768 | 0 | pub fn fetch(&self) -> Option<usize> { |
769 | 0 | self.fetch |
770 | 0 | } |
771 | | |
772 | 23 | fn output_partitioning_helper( |
773 | 23 | input: &Arc<dyn ExecutionPlan>, |
774 | 23 | preserve_partitioning: bool, |
775 | 23 | ) -> Partitioning { |
776 | 23 | // Get output partitioning: |
777 | 23 | if preserve_partitioning { |
778 | 1 | input.output_partitioning().clone() |
779 | | } else { |
780 | 22 | Partitioning::UnknownPartitioning(1) |
781 | | } |
782 | 23 | } |
783 | | |
784 | | /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. |
785 | 22 | fn compute_properties( |
786 | 22 | input: &Arc<dyn ExecutionPlan>, |
787 | 22 | sort_exprs: LexOrdering, |
788 | 22 | preserve_partitioning: bool, |
789 | 22 | ) -> PlanProperties { |
790 | 22 | // Determine execution mode: |
791 | 22 | let sort_satisfied = input.equivalence_properties().ordering_satisfy_requirement( |
792 | 22 | PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()) |
793 | 22 | .inner |
794 | 22 | .as_slice(), |
795 | 22 | ); |
796 | 22 | let mode = match input.execution_mode() { |
797 | 1 | ExecutionMode::Unbounded if sort_satisfied => ExecutionMode::Unbounded, |
798 | 21 | ExecutionMode::Bounded => ExecutionMode::Bounded, |
799 | 0 | _ => ExecutionMode::PipelineBreaking, |
800 | | }; |
801 | | |
802 | | // Calculate equivalence properties; i.e. reset the ordering equivalence |
803 | | // class with the new ordering: |
804 | 22 | let eq_properties = input |
805 | 22 | .equivalence_properties() |
806 | 22 | .clone() |
807 | 22 | .with_reorder(sort_exprs); |
808 | 22 | |
809 | 22 | // Get output partitioning: |
810 | 22 | let output_partitioning = |
811 | 22 | Self::output_partitioning_helper(input, preserve_partitioning); |
812 | 22 | |
813 | 22 | PlanProperties::new(eq_properties, output_partitioning, mode) |
814 | 22 | } |
815 | | } |
816 | | |
817 | | impl DisplayAs for SortExec { |
818 | 0 | fn fmt_as( |
819 | 0 | &self, |
820 | 0 | t: DisplayFormatType, |
821 | 0 | f: &mut std::fmt::Formatter, |
822 | 0 | ) -> std::fmt::Result { |
823 | 0 | match t { |
824 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
825 | 0 | let expr = PhysicalSortExpr::format_list(&self.expr); |
826 | 0 | let preserve_partitioning = self.preserve_partitioning; |
827 | 0 | match self.fetch { |
828 | 0 | Some(fetch) => { |
829 | 0 | write!(f, "SortExec: TopK(fetch={fetch}), expr=[{expr}], preserve_partitioning=[{preserve_partitioning}]",) |
830 | | } |
831 | 0 | None => write!(f, "SortExec: expr=[{expr}], preserve_partitioning=[{preserve_partitioning}]"), |
832 | | } |
833 | | } |
834 | | } |
835 | 0 | } |
836 | | } |
837 | | |
838 | | impl ExecutionPlan for SortExec { |
839 | 0 | fn name(&self) -> &'static str { |
840 | 0 | "SortExec" |
841 | 0 | } |
842 | | |
843 | 0 | fn as_any(&self) -> &dyn Any { |
844 | 0 | self |
845 | 0 | } |
846 | | |
847 | 52 | fn properties(&self) -> &PlanProperties { |
848 | 52 | &self.cache |
849 | 52 | } |
850 | | |
851 | 0 | fn required_input_distribution(&self) -> Vec<Distribution> { |
852 | 0 | if self.preserve_partitioning { |
853 | 0 | vec![Distribution::UnspecifiedDistribution] |
854 | | } else { |
855 | | // global sort |
856 | | // TODO support RangePartition and OrderedDistribution |
857 | 0 | vec![Distribution::SinglePartition] |
858 | | } |
859 | 0 | } |
860 | | |
861 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
862 | 0 | vec![&self.input] |
863 | 0 | } |
864 | | |
865 | 0 | fn benefits_from_input_partitioning(&self) -> Vec<bool> { |
866 | 0 | vec![false] |
867 | 0 | } |
868 | | |
869 | 0 | fn with_new_children( |
870 | 0 | self: Arc<Self>, |
871 | 0 | children: Vec<Arc<dyn ExecutionPlan>>, |
872 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
873 | 0 | let new_sort = SortExec::new(self.expr.clone(), Arc::clone(&children[0])) |
874 | 0 | .with_fetch(self.fetch) |
875 | 0 | .with_preserve_partitioning(self.preserve_partitioning); |
876 | 0 |
|
877 | 0 | Ok(Arc::new(new_sort)) |
878 | 0 | } |
879 | | |
880 | 25 | fn execute( |
881 | 25 | &self, |
882 | 25 | partition: usize, |
883 | 25 | context: Arc<TaskContext>, |
884 | 25 | ) -> Result<SendableRecordBatchStream> { |
885 | 25 | trace!("Start SortExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()0 ); |
886 | | |
887 | 25 | let mut input = self.input.execute(partition, Arc::clone(&context))?0 ; |
888 | | |
889 | 25 | let execution_options = &context.session_config().options().execution; |
890 | 25 | |
891 | 25 | trace!("End SortExec's input.execute for partition: {}"0 , partition); |
892 | | |
893 | 25 | let sort_satisfied = self |
894 | 25 | .input |
895 | 25 | .equivalence_properties() |
896 | 25 | .ordering_satisfy_requirement( |
897 | 25 | PhysicalSortRequirement::from_sort_exprs(self.expr.iter()) |
898 | 25 | .inner |
899 | 25 | .as_slice(), |
900 | 25 | ); |
901 | 25 | |
902 | 25 | match (sort_satisfied, self.fetch.as_ref()) { |
903 | 1 | (true, Some(fetch)) => Ok(Box::pin(LimitStream::new( |
904 | 1 | input, |
905 | 1 | 0, |
906 | 1 | Some(*fetch), |
907 | 1 | BaselineMetrics::new(&self.metrics_set, partition), |
908 | 1 | ))), |
909 | 0 | (true, None) => Ok(input), |
910 | 5 | (false, Some(fetch)) => { |
911 | 5 | let mut topk = TopK::try_new( |
912 | 5 | partition, |
913 | 5 | input.schema(), |
914 | 5 | self.expr.clone(), |
915 | 5 | *fetch, |
916 | 5 | context.session_config().batch_size(), |
917 | 5 | context.runtime_env(), |
918 | 5 | &self.metrics_set, |
919 | 5 | partition, |
920 | 5 | )?0 ; |
921 | 5 | Ok(Box::pin(RecordBatchStreamAdapter::new( |
922 | 5 | self.schema(), |
923 | 5 | futures::stream::once(async move { |
924 | 25 | while let Some(batch20 ) = input.next().await1 { |
925 | 20 | let batch = batch?0 ; |
926 | 20 | topk.insert_batch(batch)?0 ; |
927 | | } |
928 | 5 | topk.emit() |
929 | 5 | }) |
930 | 5 | .try_flatten(), |
931 | 5 | ))) |
932 | | } |
933 | | (false, None) => { |
934 | 19 | let mut sorter = ExternalSorter::new( |
935 | 19 | partition, |
936 | 19 | input.schema(), |
937 | 19 | self.expr.clone(), |
938 | 19 | context.session_config().batch_size(), |
939 | 19 | self.fetch, |
940 | 19 | execution_options.sort_spill_reservation_bytes, |
941 | 19 | execution_options.sort_in_place_threshold_bytes, |
942 | 19 | &self.metrics_set, |
943 | 19 | context.runtime_env(), |
944 | 19 | ); |
945 | 19 | Ok(Box::pin(RecordBatchStreamAdapter::new( |
946 | 19 | self.schema(), |
947 | 19 | futures::stream::once(async move { |
948 | 788 | while let Some(batch769 ) = input.next().await221 { |
949 | 769 | let batch = batch?0 ; |
950 | 769 | sorter.insert_batch(batch).await0 ?0 ; |
951 | | } |
952 | 18 | sorter.sort() |
953 | 19 | }18 ) |
954 | 19 | .try_flatten(), |
955 | 19 | ))) |
956 | | } |
957 | | } |
958 | 25 | } |
959 | | |
960 | 5 | fn metrics(&self) -> Option<MetricsSet> { |
961 | 5 | Some(self.metrics_set.clone_inner()) |
962 | 5 | } |
963 | | |
964 | 0 | fn statistics(&self) -> Result<Statistics> { |
965 | 0 | Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1) |
966 | 0 | } |
967 | | |
968 | 0 | fn with_fetch(&self, limit: Option<usize>) -> Option<Arc<dyn ExecutionPlan>> { |
969 | 0 | Some(Arc::new(SortExec::with_fetch(self, limit))) |
970 | 0 | } |
971 | | |
972 | 0 | fn fetch(&self) -> Option<usize> { |
973 | 0 | self.fetch |
974 | 0 | } |
975 | | } |
976 | | |
977 | | #[cfg(test)] |
978 | | mod tests { |
979 | | use std::collections::HashMap; |
980 | | use std::pin::Pin; |
981 | | use std::task::{Context, Poll}; |
982 | | |
983 | | use super::*; |
984 | | use crate::coalesce_partitions::CoalescePartitionsExec; |
985 | | use crate::collect; |
986 | | use crate::expressions::col; |
987 | | use crate::memory::MemoryExec; |
988 | | use crate::test; |
989 | | use crate::test::assert_is_pending; |
990 | | use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; |
991 | | |
992 | | use arrow::array::*; |
993 | | use arrow::compute::SortOptions; |
994 | | use arrow::datatypes::*; |
995 | | use datafusion_common::cast::as_primitive_array; |
996 | | use datafusion_common::{assert_batches_eq, Result, ScalarValue}; |
997 | | use datafusion_execution::config::SessionConfig; |
998 | | use datafusion_execution::runtime_env::RuntimeEnvBuilder; |
999 | | use datafusion_execution::RecordBatchStream; |
1000 | | use datafusion_physical_expr::expressions::{Column, Literal}; |
1001 | | use datafusion_physical_expr::EquivalenceProperties; |
1002 | | |
1003 | | use futures::{FutureExt, Stream}; |
1004 | | |
1005 | | #[derive(Debug, Clone)] |
1006 | | pub struct SortedUnboundedExec { |
1007 | | schema: Schema, |
1008 | | batch_size: u64, |
1009 | | cache: PlanProperties, |
1010 | | } |
1011 | | |
1012 | | impl DisplayAs for SortedUnboundedExec { |
1013 | 0 | fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result { |
1014 | 0 | match t { |
1015 | | DisplayFormatType::Default | DisplayFormatType::Verbose => { |
1016 | 0 | write!(f, "UnboundableExec",).unwrap() |
1017 | 0 | } |
1018 | 0 | } |
1019 | 0 | Ok(()) |
1020 | 0 | } |
1021 | | } |
1022 | | |
1023 | | impl SortedUnboundedExec { |
1024 | 1 | fn compute_properties(schema: SchemaRef) -> PlanProperties { |
1025 | 1 | let mut eq_properties = EquivalenceProperties::new(schema); |
1026 | 1 | eq_properties.add_new_orderings(vec![vec![PhysicalSortExpr::new_default( |
1027 | 1 | Arc::new(Column::new("c1", 0)), |
1028 | 1 | )]]); |
1029 | 1 | let mode = ExecutionMode::Unbounded; |
1030 | 1 | PlanProperties::new(eq_properties, Partitioning::UnknownPartitioning(1), mode) |
1031 | 1 | } |
1032 | | } |
1033 | | |
1034 | | impl ExecutionPlan for SortedUnboundedExec { |
1035 | 0 | fn name(&self) -> &'static str { |
1036 | 0 | Self::static_name() |
1037 | 0 | } |
1038 | | |
1039 | 0 | fn as_any(&self) -> &dyn Any { |
1040 | 0 | self |
1041 | 0 | } |
1042 | | |
1043 | 4 | fn properties(&self) -> &PlanProperties { |
1044 | 4 | &self.cache |
1045 | 4 | } |
1046 | | |
1047 | 0 | fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> { |
1048 | 0 | vec![] |
1049 | 0 | } |
1050 | | |
1051 | 0 | fn with_new_children( |
1052 | 0 | self: Arc<Self>, |
1053 | 0 | _: Vec<Arc<dyn ExecutionPlan>>, |
1054 | 0 | ) -> Result<Arc<dyn ExecutionPlan>> { |
1055 | 0 | Ok(self) |
1056 | 0 | } |
1057 | | |
1058 | 1 | fn execute( |
1059 | 1 | &self, |
1060 | 1 | _partition: usize, |
1061 | 1 | _context: Arc<TaskContext>, |
1062 | 1 | ) -> Result<SendableRecordBatchStream> { |
1063 | 1 | Ok(Box::pin(SortedUnboundedStream { |
1064 | 1 | schema: Arc::new(self.schema.clone()), |
1065 | 1 | batch_size: self.batch_size, |
1066 | 1 | offset: 0, |
1067 | 1 | })) |
1068 | 1 | } |
1069 | | } |
1070 | | |
1071 | | #[derive(Debug)] |
1072 | | pub struct SortedUnboundedStream { |
1073 | | schema: SchemaRef, |
1074 | | batch_size: u64, |
1075 | | offset: u64, |
1076 | | } |
1077 | | |
1078 | | impl Stream for SortedUnboundedStream { |
1079 | | type Item = Result<RecordBatch>; |
1080 | | |
1081 | 5 | fn poll_next( |
1082 | 5 | mut self: Pin<&mut Self>, |
1083 | 5 | _cx: &mut Context<'_>, |
1084 | 5 | ) -> Poll<Option<Self::Item>> { |
1085 | 5 | let batch = SortedUnboundedStream::create_record_batch( |
1086 | 5 | Arc::clone(&self.schema), |
1087 | 5 | self.offset, |
1088 | 5 | self.batch_size, |
1089 | 5 | ); |
1090 | 5 | self.offset += self.batch_size; |
1091 | 5 | Poll::Ready(Some(Ok(batch))) |
1092 | 5 | } |
1093 | | } |
1094 | | |
1095 | | impl RecordBatchStream for SortedUnboundedStream { |
1096 | 1 | fn schema(&self) -> SchemaRef { |
1097 | 1 | Arc::clone(&self.schema) |
1098 | 1 | } |
1099 | | } |
1100 | | |
1101 | | impl SortedUnboundedStream { |
1102 | 5 | fn create_record_batch( |
1103 | 5 | schema: SchemaRef, |
1104 | 5 | offset: u64, |
1105 | 5 | batch_size: u64, |
1106 | 5 | ) -> RecordBatch { |
1107 | 10 | let values = (0..batch_size).map(|i| offset + i).collect::<Vec<_>>(); |
1108 | 5 | let array = UInt64Array::from(values); |
1109 | 5 | let array_ref: ArrayRef = Arc::new(array); |
1110 | 5 | RecordBatch::try_new(schema, vec![array_ref]).unwrap() |
1111 | 5 | } |
1112 | | } |
1113 | | |
1114 | | #[tokio::test] |
1115 | 1 | async fn test_in_mem_sort() -> Result<()> { |
1116 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
1117 | 1 | let partitions = 4; |
1118 | 1 | let csv = test::scan_partitioned(partitions); |
1119 | 1 | let schema = csv.schema(); |
1120 | 1 | |
1121 | 1 | let sort_exec = Arc::new(SortExec::new( |
1122 | 1 | vec![PhysicalSortExpr { |
1123 | 1 | expr: col("i", &schema)?0 , |
1124 | 1 | options: SortOptions::default(), |
1125 | 1 | }], |
1126 | 1 | Arc::new(CoalescePartitionsExec::new(csv)), |
1127 | 1 | )); |
1128 | 1 | |
1129 | 1 | let result = collect(sort_exec, Arc::clone(&task_ctx)).await?0 ; |
1130 | 1 | |
1131 | 1 | assert_eq!(result.len(), 1); |
1132 | 1 | assert_eq!(result[0].num_rows(), 400); |
1133 | 1 | |
1134 | 1 | assert_eq!( |
1135 | 1 | task_ctx.runtime_env().memory_pool.reserved(), |
1136 | 1 | 0, |
1137 | 1 | "The sort should have returned all memory used back to the memory manager"0 |
1138 | 1 | ); |
1139 | 1 | |
1140 | 1 | Ok(()) |
1141 | 1 | } |
1142 | | |
1143 | | #[tokio::test] |
1144 | 1 | async fn test_sort_spill() -> Result<()> { |
1145 | 1 | // trigger spill w/ 100 batches |
1146 | 1 | let session_config = SessionConfig::new(); |
1147 | 1 | let sort_spill_reservation_bytes = session_config |
1148 | 1 | .options() |
1149 | 1 | .execution |
1150 | 1 | .sort_spill_reservation_bytes; |
1151 | 1 | let runtime = RuntimeEnvBuilder::new() |
1152 | 1 | .with_memory_limit(sort_spill_reservation_bytes + 12288, 1.0) |
1153 | 1 | .build_arc()?0 ; |
1154 | 1 | let task_ctx = Arc::new( |
1155 | 1 | TaskContext::default() |
1156 | 1 | .with_session_config(session_config) |
1157 | 1 | .with_runtime(runtime), |
1158 | 1 | ); |
1159 | 1 | |
1160 | 1 | let partitions = 100; |
1161 | 1 | let input = test::scan_partitioned(partitions); |
1162 | 1 | let schema = input.schema(); |
1163 | 1 | |
1164 | 1 | let sort_exec = Arc::new(SortExec::new( |
1165 | 1 | vec![PhysicalSortExpr { |
1166 | 1 | expr: col("i", &schema)?0 , |
1167 | 1 | options: SortOptions::default(), |
1168 | 1 | }], |
1169 | 1 | Arc::new(CoalescePartitionsExec::new(input)), |
1170 | 1 | )); |
1171 | 1 | |
1172 | 1 | let result = collect( |
1173 | 1 | Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>, |
1174 | 1 | Arc::clone(&task_ctx), |
1175 | 1 | ) |
1176 | 2 | .await?0 ; |
1177 | 1 | |
1178 | 1 | assert_eq!(result.len(), 2); |
1179 | 1 | |
1180 | 1 | // Now, validate metrics |
1181 | 1 | let metrics = sort_exec.metrics().unwrap(); |
1182 | 1 | |
1183 | 1 | assert_eq!(metrics.output_rows().unwrap(), 10000); |
1184 | 1 | assert!(metrics.elapsed_compute().unwrap() > 0); |
1185 | 1 | assert_eq!(metrics.spill_count().unwrap(), 4); |
1186 | 1 | assert_eq!(metrics.spilled_bytes().unwrap(), 38784); |
1187 | 1 | assert_eq!(metrics.spilled_rows().unwrap(), 9600); |
1188 | 1 | |
1189 | 1 | let columns = result[0].columns(); |
1190 | 1 | |
1191 | 1 | let i = as_primitive_array::<Int32Type>(&columns[0])?0 ; |
1192 | 1 | assert_eq!(i.value(0), 0); |
1193 | 1 | assert_eq!(i.value(i.len() - 1), 81); |
1194 | 1 | |
1195 | 1 | assert_eq!( |
1196 | 1 | task_ctx.runtime_env().memory_pool.reserved(), |
1197 | 1 | 0, |
1198 | 1 | "The sort should have returned all memory used back to the memory manager"0 |
1199 | 1 | ); |
1200 | 1 | |
1201 | 1 | Ok(()) |
1202 | 1 | } |
1203 | | |
1204 | | #[tokio::test] |
1205 | 1 | async fn test_sort_fetch_memory_calculation() -> Result<()> { |
1206 | 1 | // This test mirrors down the size from the example above. |
1207 | 1 | let avg_batch_size = 400; |
1208 | 1 | let partitions = 4; |
1209 | 1 | |
1210 | 1 | // A tuple of (fetch, expect_spillage) |
1211 | 1 | let test_options = vec![ |
1212 | 1 | // Since we don't have a limit (and the memory is less than the total size of |
1213 | 1 | // all the batches we are processing, we expect it to spill. |
1214 | 1 | (None, true), |
1215 | 1 | // When we have a limit however, the buffered size of batches should fit in memory |
1216 | 1 | // since it is much lower than the total size of the input batch. |
1217 | 1 | (Some(1), false), |
1218 | 1 | ]; |
1219 | 1 | |
1220 | 3 | for (fetch, expect_spillage2 ) in test_options { |
1221 | 2 | let session_config = SessionConfig::new(); |
1222 | 2 | let sort_spill_reservation_bytes = session_config |
1223 | 2 | .options() |
1224 | 2 | .execution |
1225 | 2 | .sort_spill_reservation_bytes; |
1226 | 1 | |
1227 | 2 | let runtime = RuntimeEnvBuilder::new() |
1228 | 2 | .with_memory_limit( |
1229 | 2 | sort_spill_reservation_bytes + avg_batch_size * (partitions - 1), |
1230 | 2 | 1.0, |
1231 | 2 | ) |
1232 | 2 | .build_arc()?0 ; |
1233 | 2 | let task_ctx = Arc::new( |
1234 | 2 | TaskContext::default() |
1235 | 2 | .with_runtime(runtime) |
1236 | 2 | .with_session_config(session_config), |
1237 | 2 | ); |
1238 | 2 | |
1239 | 2 | let csv = test::scan_partitioned(partitions); |
1240 | 2 | let schema = csv.schema(); |
1241 | 1 | |
1242 | 2 | let sort_exec = Arc::new( |
1243 | 1 | SortExec::new( |
1244 | 2 | vec![PhysicalSortExpr { |
1245 | 2 | expr: col("i", &schema)?0 , |
1246 | 2 | options: SortOptions::default(), |
1247 | 2 | }], |
1248 | 2 | Arc::new(CoalescePartitionsExec::new(csv)), |
1249 | 2 | ) |
1250 | 2 | .with_fetch(fetch), |
1251 | 1 | ); |
1252 | 1 | |
1253 | 2 | let result = collect( |
1254 | 2 | Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>, |
1255 | 2 | Arc::clone(&task_ctx), |
1256 | 2 | ) |
1257 | 2 | .await?0 ; |
1258 | 2 | assert_eq!(result.len(), 1); |
1259 | 1 | |
1260 | 2 | let metrics = sort_exec.metrics().unwrap(); |
1261 | 2 | let did_it_spill = metrics.spill_count().unwrap_or(0) > 0; |
1262 | 2 | assert_eq!(did_it_spill, expect_spillage, "with fetch: {fetch:?}"0 ); |
1263 | 1 | } |
1264 | 1 | Ok(()) |
1265 | 1 | } |
1266 | | |
1267 | | #[tokio::test] |
1268 | 1 | async fn test_sort_metadata() -> Result<()> { |
1269 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
1270 | 1 | let field_metadata: HashMap<String, String> = |
1271 | 1 | vec![("foo".to_string(), "bar".to_string())] |
1272 | 1 | .into_iter() |
1273 | 1 | .collect(); |
1274 | 1 | let schema_metadata: HashMap<String, String> = |
1275 | 1 | vec![("baz".to_string(), "barf".to_string())] |
1276 | 1 | .into_iter() |
1277 | 1 | .collect(); |
1278 | 1 | |
1279 | 1 | let mut field = Field::new("field_name", DataType::UInt64, true); |
1280 | 1 | field.set_metadata(field_metadata.clone()); |
1281 | 1 | let schema = Schema::new_with_metadata(vec![field], schema_metadata.clone()); |
1282 | 1 | let schema = Arc::new(schema); |
1283 | 1 | |
1284 | 1 | let data: ArrayRef = |
1285 | 1 | Arc::new(vec![3, 2, 1].into_iter().map(Some).collect::<UInt64Array>()); |
1286 | 1 | |
1287 | 1 | let batch = RecordBatch::try_new(Arc::clone(&schema), vec![data]).unwrap(); |
1288 | 1 | let input = Arc::new( |
1289 | 1 | MemoryExec::try_new(&[vec![batch]], Arc::clone(&schema), None).unwrap(), |
1290 | 1 | ); |
1291 | 1 | |
1292 | 1 | let sort_exec = Arc::new(SortExec::new( |
1293 | 1 | vec![PhysicalSortExpr { |
1294 | 1 | expr: col("field_name", &schema)?0 , |
1295 | 1 | options: SortOptions::default(), |
1296 | 1 | }], |
1297 | 1 | input, |
1298 | 1 | )); |
1299 | 1 | |
1300 | 1 | let result: Vec<RecordBatch> = collect(sort_exec, task_ctx).await0 ?0 ; |
1301 | 1 | |
1302 | 1 | let expected_data: ArrayRef = |
1303 | 1 | Arc::new(vec![1, 2, 3].into_iter().map(Some).collect::<UInt64Array>()); |
1304 | 1 | let expected_batch = |
1305 | 1 | RecordBatch::try_new(Arc::clone(&schema), vec![expected_data]).unwrap(); |
1306 | 1 | |
1307 | 1 | // Data is correct |
1308 | 1 | assert_eq!(&vec![expected_batch], &result); |
1309 | 1 | |
1310 | 1 | // explicitlty ensure the metadata is present |
1311 | 1 | assert_eq!(result[0].schema().fields()[0].metadata(), &field_metadata); |
1312 | 1 | assert_eq!(result[0].schema().metadata(), &schema_metadata); |
1313 | 1 | |
1314 | 1 | Ok(()) |
1315 | 1 | } |
1316 | | |
1317 | | #[tokio::test] |
1318 | 1 | async fn test_lex_sort_by_mixed_types() -> Result<()> { |
1319 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
1320 | 1 | let schema = Arc::new(Schema::new(vec![ |
1321 | 1 | Field::new("a", DataType::Int32, true), |
1322 | 1 | Field::new( |
1323 | 1 | "b", |
1324 | 1 | DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), |
1325 | 1 | true, |
1326 | 1 | ), |
1327 | 1 | ])); |
1328 | 1 | |
1329 | 1 | // define data. |
1330 | 1 | let batch = RecordBatch::try_new( |
1331 | 1 | Arc::clone(&schema), |
1332 | 1 | vec![ |
1333 | 1 | Arc::new(Int32Array::from(vec![Some(2), None, Some(1), Some(2)])), |
1334 | 1 | Arc::new(ListArray::from_iter_primitive::<Int32Type, _, _>(vec![ |
1335 | 1 | Some(vec![Some(3)]), |
1336 | 1 | Some(vec![Some(1)]), |
1337 | 1 | Some(vec![Some(6), None]), |
1338 | 1 | Some(vec![Some(5)]), |
1339 | 1 | ])), |
1340 | 1 | ], |
1341 | 1 | )?0 ; |
1342 | 1 | |
1343 | 1 | let sort_exec = Arc::new(SortExec::new( |
1344 | 1 | vec![ |
1345 | 1 | PhysicalSortExpr { |
1346 | 1 | expr: col("a", &schema)?0 , |
1347 | 1 | options: SortOptions { |
1348 | 1 | descending: false, |
1349 | 1 | nulls_first: true, |
1350 | 1 | }, |
1351 | 1 | }, |
1352 | 1 | PhysicalSortExpr { |
1353 | 1 | expr: col("b", &schema)?0 , |
1354 | 1 | options: SortOptions { |
1355 | 1 | descending: true, |
1356 | 1 | nulls_first: false, |
1357 | 1 | }, |
1358 | 1 | }, |
1359 | 1 | ], |
1360 | 1 | Arc::new(MemoryExec::try_new( |
1361 | 1 | &[vec![batch]], |
1362 | 1 | Arc::clone(&schema), |
1363 | 1 | None, |
1364 | 1 | )?0 ), |
1365 | 1 | )); |
1366 | 1 | |
1367 | 1 | assert_eq!(DataType::Int32, *sort_exec.schema().field(0).data_type()); |
1368 | 1 | assert_eq!( |
1369 | 1 | DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), |
1370 | 1 | *sort_exec.schema().field(1).data_type() |
1371 | 1 | ); |
1372 | 1 | |
1373 | 1 | let result: Vec<RecordBatch> = |
1374 | 1 | collect(Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>, task_ctx).await0 ?0 ; |
1375 | 1 | let metrics = sort_exec.metrics().unwrap(); |
1376 | 1 | assert!(metrics.elapsed_compute().unwrap() > 0); |
1377 | 1 | assert_eq!(metrics.output_rows().unwrap(), 4); |
1378 | 1 | assert_eq!(result.len(), 1); |
1379 | 1 | |
1380 | 1 | let expected = RecordBatch::try_new( |
1381 | 1 | schema, |
1382 | 1 | vec![ |
1383 | 1 | Arc::new(Int32Array::from(vec![None, Some(1), Some(2), Some(2)])), |
1384 | 1 | Arc::new(ListArray::from_iter_primitive::<Int32Type, _, _>(vec![ |
1385 | 1 | Some(vec![Some(1)]), |
1386 | 1 | Some(vec![Some(6), None]), |
1387 | 1 | Some(vec![Some(5)]), |
1388 | 1 | Some(vec![Some(3)]), |
1389 | 1 | ])), |
1390 | 1 | ], |
1391 | 1 | )?0 ; |
1392 | 1 | |
1393 | 1 | assert_eq!(expected, result[0]); |
1394 | 1 | |
1395 | 1 | Ok(()) |
1396 | 1 | } |
1397 | | |
1398 | | #[tokio::test] |
1399 | 1 | async fn test_lex_sort_by_float() -> Result<()> { |
1400 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
1401 | 1 | let schema = Arc::new(Schema::new(vec![ |
1402 | 1 | Field::new("a", DataType::Float32, true), |
1403 | 1 | Field::new("b", DataType::Float64, true), |
1404 | 1 | ])); |
1405 | 1 | |
1406 | 1 | // define data. |
1407 | 1 | let batch = RecordBatch::try_new( |
1408 | 1 | Arc::clone(&schema), |
1409 | 1 | vec![ |
1410 | 1 | Arc::new(Float32Array::from(vec![ |
1411 | 1 | Some(f32::NAN), |
1412 | 1 | None, |
1413 | 1 | None, |
1414 | 1 | Some(f32::NAN), |
1415 | 1 | Some(1.0_f32), |
1416 | 1 | Some(1.0_f32), |
1417 | 1 | Some(2.0_f32), |
1418 | 1 | Some(3.0_f32), |
1419 | 1 | ])), |
1420 | 1 | Arc::new(Float64Array::from(vec![ |
1421 | 1 | Some(200.0_f64), |
1422 | 1 | Some(20.0_f64), |
1423 | 1 | Some(10.0_f64), |
1424 | 1 | Some(100.0_f64), |
1425 | 1 | Some(f64::NAN), |
1426 | 1 | None, |
1427 | 1 | None, |
1428 | 1 | Some(f64::NAN), |
1429 | 1 | ])), |
1430 | 1 | ], |
1431 | 1 | )?0 ; |
1432 | 1 | |
1433 | 1 | let sort_exec = Arc::new(SortExec::new( |
1434 | 1 | vec![ |
1435 | 1 | PhysicalSortExpr { |
1436 | 1 | expr: col("a", &schema)?0 , |
1437 | 1 | options: SortOptions { |
1438 | 1 | descending: true, |
1439 | 1 | nulls_first: true, |
1440 | 1 | }, |
1441 | 1 | }, |
1442 | 1 | PhysicalSortExpr { |
1443 | 1 | expr: col("b", &schema)?0 , |
1444 | 1 | options: SortOptions { |
1445 | 1 | descending: false, |
1446 | 1 | nulls_first: false, |
1447 | 1 | }, |
1448 | 1 | }, |
1449 | 1 | ], |
1450 | 1 | Arc::new(MemoryExec::try_new(&[vec![batch]], schema, None)?0 ), |
1451 | 1 | )); |
1452 | 1 | |
1453 | 1 | assert_eq!(DataType::Float32, *sort_exec.schema().field(0).data_type()); |
1454 | 1 | assert_eq!(DataType::Float64, *sort_exec.schema().field(1).data_type()); |
1455 | 1 | |
1456 | 1 | let result: Vec<RecordBatch> = |
1457 | 1 | collect(Arc::clone(&sort_exec) as Arc<dyn ExecutionPlan>, task_ctx).await0 ?0 ; |
1458 | 1 | let metrics = sort_exec.metrics().unwrap(); |
1459 | 1 | assert!(metrics.elapsed_compute().unwrap() > 0); |
1460 | 1 | assert_eq!(metrics.output_rows().unwrap(), 8); |
1461 | 1 | assert_eq!(result.len(), 1); |
1462 | 1 | |
1463 | 1 | let columns = result[0].columns(); |
1464 | 1 | |
1465 | 1 | assert_eq!(DataType::Float32, *columns[0].data_type()); |
1466 | 1 | assert_eq!(DataType::Float64, *columns[1].data_type()); |
1467 | 1 | |
1468 | 1 | let a = as_primitive_array::<Float32Type>(&columns[0])?0 ; |
1469 | 1 | let b = as_primitive_array::<Float64Type>(&columns[1])?0 ; |
1470 | 1 | |
1471 | 1 | // convert result to strings to allow comparing to expected result containing NaN |
1472 | 1 | let result: Vec<(Option<String>, Option<String>)> = (0..result[0].num_rows()) |
1473 | 8 | .map(|i| { |
1474 | 8 | let aval = if a.is_valid(i) { |
1475 | 6 | Some(a.value(i).to_string()) |
1476 | 1 | } else { |
1477 | 2 | None |
1478 | 1 | }; |
1479 | 8 | let bval = if b.is_valid(i) { |
1480 | 6 | Some(b.value(i).to_string()) |
1481 | 1 | } else { |
1482 | 2 | None |
1483 | 1 | }; |
1484 | 8 | (aval, bval) |
1485 | 8 | }) |
1486 | 1 | .collect(); |
1487 | 1 | |
1488 | 1 | let expected: Vec<(Option<String>, Option<String>)> = vec![ |
1489 | 1 | (None, Some("10".to_owned())), |
1490 | 1 | (None, Some("20".to_owned())), |
1491 | 1 | (Some("NaN".to_owned()), Some("100".to_owned())), |
1492 | 1 | (Some("NaN".to_owned()), Some("200".to_owned())), |
1493 | 1 | (Some("3".to_owned()), Some("NaN".to_owned())), |
1494 | 1 | (Some("2".to_owned()), None), |
1495 | 1 | (Some("1".to_owned()), Some("NaN".to_owned())), |
1496 | 1 | (Some("1".to_owned()), None), |
1497 | 1 | ]; |
1498 | 1 | |
1499 | 1 | assert_eq!(expected, result); |
1500 | 1 | |
1501 | 1 | Ok(()) |
1502 | 1 | } |
1503 | | |
1504 | | #[tokio::test] |
1505 | 1 | async fn test_drop_cancel() -> Result<()> { |
1506 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
1507 | 1 | let schema = |
1508 | 1 | Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])); |
1509 | 1 | |
1510 | 1 | let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); |
1511 | 1 | let refs = blocking_exec.refs(); |
1512 | 1 | let sort_exec = Arc::new(SortExec::new( |
1513 | 1 | vec![PhysicalSortExpr { |
1514 | 1 | expr: col("a", &schema)?0 , |
1515 | 1 | options: SortOptions::default(), |
1516 | 1 | }], |
1517 | 1 | blocking_exec, |
1518 | 1 | )); |
1519 | 1 | |
1520 | 1 | let fut = collect(sort_exec, Arc::clone(&task_ctx)); |
1521 | 1 | let mut fut = fut.boxed(); |
1522 | 1 | |
1523 | 1 | assert_is_pending(&mut fut); |
1524 | 1 | drop(fut); |
1525 | 1 | assert_strong_count_converges_to_zero(refs).await0 ; |
1526 | 1 | |
1527 | 1 | assert_eq!( |
1528 | 1 | task_ctx.runtime_env().memory_pool.reserved(), |
1529 | 1 | 0, |
1530 | 1 | "The sort should have returned all memory used back to the memory manager"0 |
1531 | 1 | ); |
1532 | 1 | |
1533 | 1 | Ok(()) |
1534 | 1 | } |
1535 | | |
1536 | | #[test] |
1537 | 1 | fn test_empty_sort_batch() { |
1538 | 1 | let schema = Arc::new(Schema::empty()); |
1539 | 1 | let options = RecordBatchOptions::new().with_row_count(Some(1)); |
1540 | 1 | let batch = |
1541 | 1 | RecordBatch::try_new_with_options(Arc::clone(&schema), vec![], &options) |
1542 | 1 | .unwrap(); |
1543 | 1 | |
1544 | 1 | let expressions = vec![PhysicalSortExpr { |
1545 | 1 | expr: Arc::new(Literal::new(ScalarValue::Int64(Some(1)))), |
1546 | 1 | options: SortOptions::default(), |
1547 | 1 | }]; |
1548 | 1 | |
1549 | 1 | let result = sort_batch(&batch, &expressions, None).unwrap(); |
1550 | 1 | assert_eq!(result.num_rows(), 1); |
1551 | 1 | } |
1552 | | |
1553 | | #[tokio::test] |
1554 | 1 | async fn topk_unbounded_source() -> Result<()> { |
1555 | 1 | let task_ctx = Arc::new(TaskContext::default()); |
1556 | 1 | let schema = Schema::new(vec![Field::new("c1", DataType::UInt64, false)]); |
1557 | 1 | let source = SortedUnboundedExec { |
1558 | 1 | schema: schema.clone(), |
1559 | 1 | batch_size: 2, |
1560 | 1 | cache: SortedUnboundedExec::compute_properties(Arc::new(schema.clone())), |
1561 | 1 | }; |
1562 | 1 | let mut plan = SortExec::new( |
1563 | 1 | vec![PhysicalSortExpr::new_default(Arc::new(Column::new( |
1564 | 1 | "c1", 0, |
1565 | 1 | )))], |
1566 | 1 | Arc::new(source), |
1567 | 1 | ); |
1568 | 1 | plan = plan.with_fetch(Some(9)); |
1569 | 1 | |
1570 | 1 | let batches = collect(Arc::new(plan), task_ctx).await0 ?0 ; |
1571 | 1 | #[rustfmt::skip] |
1572 | 1 | let expected = [ |
1573 | 1 | "+----+", |
1574 | 1 | "| c1 |", |
1575 | 1 | "+----+", |
1576 | 1 | "| 0 |", |
1577 | 1 | "| 1 |", |
1578 | 1 | "| 2 |", |
1579 | 1 | "| 3 |", |
1580 | 1 | "| 4 |", |
1581 | 1 | "| 5 |", |
1582 | 1 | "| 6 |", |
1583 | 1 | "| 7 |", |
1584 | 1 | "| 8 |", |
1585 | 1 | "+----+",]; |
1586 | 1 | assert_batches_eq!(expected, &batches); |
1587 | 1 | Ok(()) |
1588 | 1 | } |
1589 | | } |