/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/sorts/builder.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 | | use arrow::compute::interleave; |
19 | | use arrow::datatypes::SchemaRef; |
20 | | use arrow::record_batch::RecordBatch; |
21 | | use datafusion_common::Result; |
22 | | use datafusion_execution::memory_pool::MemoryReservation; |
23 | | use std::sync::Arc; |
24 | | |
25 | | #[derive(Debug, Copy, Clone, Default)] |
26 | | struct BatchCursor { |
27 | | /// The index into BatchBuilder::batches |
28 | | batch_idx: usize, |
29 | | /// The row index within the given batch |
30 | | row_idx: usize, |
31 | | } |
32 | | |
33 | | /// Provides an API to incrementally build a [`RecordBatch`] from partitioned [`RecordBatch`] |
34 | | #[derive(Debug)] |
35 | | pub struct BatchBuilder { |
36 | | /// The schema of the RecordBatches yielded by this stream |
37 | | schema: SchemaRef, |
38 | | |
39 | | /// Maintain a list of [`RecordBatch`] and their corresponding stream |
40 | | batches: Vec<(usize, RecordBatch)>, |
41 | | |
42 | | /// Accounts for memory used by buffered batches |
43 | | reservation: MemoryReservation, |
44 | | |
45 | | /// The current [`BatchCursor`] for each stream |
46 | | cursors: Vec<BatchCursor>, |
47 | | |
48 | | /// The accumulated stream indexes from which to pull rows |
49 | | /// Consists of a tuple of `(batch_idx, row_idx)` |
50 | | indices: Vec<(usize, usize)>, |
51 | | } |
52 | | |
53 | | impl BatchBuilder { |
54 | | /// Create a new [`BatchBuilder`] with the provided `stream_count` and `batch_size` |
55 | 19 | pub fn new( |
56 | 19 | schema: SchemaRef, |
57 | 19 | stream_count: usize, |
58 | 19 | batch_size: usize, |
59 | 19 | reservation: MemoryReservation, |
60 | 19 | ) -> Self { |
61 | 19 | Self { |
62 | 19 | schema, |
63 | 19 | batches: Vec::with_capacity(stream_count * 2), |
64 | 19 | cursors: vec![BatchCursor::default(); stream_count], |
65 | 19 | indices: Vec::with_capacity(batch_size), |
66 | 19 | reservation, |
67 | 19 | } |
68 | 19 | } |
69 | | |
70 | | /// Append a new batch in `stream_idx` |
71 | 676 | pub fn push_batch(&mut self, stream_idx: usize, batch: RecordBatch) -> Result<()> { |
72 | 676 | self.reservation.try_grow(batch.get_array_memory_size())?0 ; |
73 | 676 | let batch_idx = self.batches.len(); |
74 | 676 | self.batches.push((stream_idx, batch)); |
75 | 676 | self.cursors[stream_idx] = BatchCursor { |
76 | 676 | batch_idx, |
77 | 676 | row_idx: 0, |
78 | 676 | }; |
79 | 676 | Ok(()) |
80 | 676 | } |
81 | | |
82 | | /// Append the next row from `stream_idx` |
83 | 14.4k | pub fn push_row(&mut self, stream_idx: usize) { |
84 | 14.4k | let cursor = &mut self.cursors[stream_idx]; |
85 | 14.4k | let row_idx = cursor.row_idx; |
86 | 14.4k | cursor.row_idx += 1; |
87 | 14.4k | self.indices.push((cursor.batch_idx, row_idx)); |
88 | 14.4k | } |
89 | | |
90 | | /// Returns the number of in-progress rows in this [`BatchBuilder`] |
91 | 14.5k | pub fn len(&self) -> usize { |
92 | 14.5k | self.indices.len() |
93 | 14.5k | } |
94 | | |
95 | | /// Returns `true` if this [`BatchBuilder`] contains no in-progress rows |
96 | 96 | pub fn is_empty(&self) -> bool { |
97 | 96 | self.indices.is_empty() |
98 | 96 | } |
99 | | |
100 | | /// Returns the schema of this [`BatchBuilder`] |
101 | 0 | pub fn schema(&self) -> &SchemaRef { |
102 | 0 | &self.schema |
103 | 0 | } |
104 | | |
105 | | /// Drains the in_progress row indexes, and builds a new RecordBatch from them |
106 | | /// |
107 | | /// Will then drop any batches for which all rows have been yielded to the output |
108 | | /// |
109 | | /// Returns `None` if no pending rows |
110 | 96 | pub fn build_record_batch(&mut self) -> Result<Option<RecordBatch>> { |
111 | 96 | if self.is_empty() { |
112 | 18 | return Ok(None); |
113 | 78 | } |
114 | | |
115 | 78 | let columns = (0..self.schema.fields.len()) |
116 | 120 | .map(|column_idx| { |
117 | 120 | let arrays: Vec<_> = self |
118 | 120 | .batches |
119 | 120 | .iter() |
120 | 947 | .map(|(_, batch)| batch.column(column_idx).as_ref()) |
121 | 120 | .collect(); |
122 | 120 | Ok(interleave(&arrays, &self.indices)?0 ) |
123 | 120 | }) |
124 | 78 | .collect::<Result<Vec<_>>>()?0 ; |
125 | | |
126 | 78 | self.indices.clear(); |
127 | 78 | |
128 | 78 | // New cursors are only created once the previous cursor for the stream |
129 | 78 | // is finished. This means all remaining rows from all but the last batch |
130 | 78 | // for each stream have been yielded to the newly created record batch |
131 | 78 | // |
132 | 78 | // We can therefore drop all but the last batch for each stream |
133 | 78 | let mut batch_idx = 0; |
134 | 78 | let mut retained = 0; |
135 | 853 | self.batches.retain(|(stream_idx, batch)| { |
136 | 853 | let stream_cursor = &mut self.cursors[*stream_idx]; |
137 | 853 | let retain = stream_cursor.batch_idx == batch_idx; |
138 | 853 | batch_idx += 1; |
139 | 853 | |
140 | 853 | if retain { |
141 | 228 | stream_cursor.batch_idx = retained; |
142 | 228 | retained += 1; |
143 | 625 | } else { |
144 | 625 | self.reservation.shrink(batch.get_array_memory_size()); |
145 | 625 | } |
146 | 853 | retain |
147 | 853 | }); |
148 | 78 | |
149 | 78 | Ok(Some(RecordBatch::try_new( |
150 | 78 | Arc::clone(&self.schema), |
151 | 78 | columns, |
152 | 78 | )?0 )) |
153 | 96 | } |
154 | | } |