-
Notifications
You must be signed in to change notification settings - Fork 1.3k
/
Copy pathmerge.rs
329 lines (294 loc) · 11.1 KB
/
merge.rs
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
// 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.
//! Merge that deals with an arbitrary size of streaming inputs.
//! This is an order-preserving merge.
use crate::metrics::BaselineMetrics;
use crate::sorts::builder::BatchBuilder;
use crate::sorts::cursor::{Cursor, CursorValues};
use crate::sorts::stream::PartitionedStream;
use crate::RecordBatchStream;
use arrow::datatypes::SchemaRef;
use arrow::record_batch::RecordBatch;
use datafusion_common::Result;
use datafusion_execution::memory_pool::MemoryReservation;
use futures::Stream;
use std::pin::Pin;
use std::task::{ready, Context, Poll};
/// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`]
type CursorStream<C> = Box<dyn PartitionedStream<Output = Result<(C, RecordBatch)>>>;
#[derive(Debug)]
pub(crate) struct SortPreservingMergeStream<C: CursorValues> {
in_progress: BatchBuilder,
/// The sorted input streams to merge together
streams: CursorStream<C>,
/// used to record execution metrics
metrics: BaselineMetrics,
/// If the stream has encountered an error
aborted: bool,
/// A loser tree that always produces the minimum cursor
///
/// Node 0 stores the top winner, Nodes 1..num_streams store
/// the loser nodes
///
/// This implements a "Tournament Tree" (aka Loser Tree) to keep
/// track of the current smallest element at the top. When the top
/// record is taken, the tree structure is not modified, and only
/// the path from bottom to top is visited, keeping the number of
/// comparisons close to the theoretical limit of `log(S)`.
///
/// The current implementation uses a vector to store the tree.
/// Conceptually, it looks like this (assuming 8 streams):
///
/// ```text
/// 0 (winner)
///
/// 1
/// / \
/// 2 3
/// / \ / \
/// 4 5 6 7
/// ```
///
/// Where element at index 0 in the vector is the current winner. Element
/// at index 1 is the root of the loser tree, element at index 2 is the
/// left child of the root, and element at index 3 is the right child of
/// the root and so on.
///
/// reference: <https://en.wikipedia.org/wiki/K-way_merge_algorithm#Tournament_Tree>
loser_tree: Vec<usize>,
/// If the most recently yielded overall winner has been replaced
/// within the loser tree. A value of `false` indicates that the
/// overall winner has been yielded but the loser tree has not
/// been updated
loser_tree_adjusted: bool,
/// target batch size
batch_size: usize,
/// Cursors for each input partition. `None` means the input is exhausted
cursors: Vec<Option<Cursor<C>>>,
/// Optional number of rows to fetch
fetch: Option<usize>,
/// number of rows produced
produced: usize,
}
impl<C: CursorValues> SortPreservingMergeStream<C> {
pub(crate) fn new(
streams: CursorStream<C>,
schema: SchemaRef,
metrics: BaselineMetrics,
batch_size: usize,
fetch: Option<usize>,
reservation: MemoryReservation,
) -> Self {
let stream_count = streams.partitions();
Self {
in_progress: BatchBuilder::new(schema, stream_count, batch_size, reservation),
streams,
metrics,
aborted: false,
cursors: (0..stream_count).map(|_| None).collect(),
loser_tree: vec![],
loser_tree_adjusted: false,
batch_size,
fetch,
produced: 0,
}
}
/// If the stream at the given index is not exhausted, and the last cursor for the
/// stream is finished, poll the stream for the next RecordBatch and create a new
/// cursor for the stream from the returned result
fn maybe_poll_stream(
&mut self,
cx: &mut Context<'_>,
idx: usize,
) -> Poll<Result<()>> {
if self.cursors[idx].is_some() {
// Cursor is not finished - don't need a new RecordBatch yet
return Poll::Ready(Ok(()));
}
match futures::ready!(self.streams.poll_next(cx, idx)) {
None => Poll::Ready(Ok(())),
Some(Err(e)) => Poll::Ready(Err(e)),
Some(Ok((cursor, batch))) => {
self.cursors[idx] = Some(Cursor::new(cursor));
Poll::Ready(self.in_progress.push_batch(idx, batch))
}
}
}
fn poll_next_inner(
&mut self,
cx: &mut Context<'_>,
) -> Poll<Option<Result<RecordBatch>>> {
if self.aborted {
return Poll::Ready(None);
}
// try to initialize the loser tree
if self.loser_tree.is_empty() {
// Ensure all non-exhausted streams have a cursor from which
// rows can be pulled
for i in 0..self.streams.partitions() {
if let Err(e) = ready!(self.maybe_poll_stream(cx, i)) {
self.aborted = true;
return Poll::Ready(Some(Err(e)));
}
}
self.init_loser_tree();
}
// NB timer records time taken on drop, so there are no
// calls to `timer.done()` below.
let elapsed_compute = self.metrics.elapsed_compute().clone();
let _timer = elapsed_compute.timer();
loop {
// Adjust the loser tree if necessary, returning control if needed
if !self.loser_tree_adjusted {
let winner = self.loser_tree[0];
if let Err(e) = ready!(self.maybe_poll_stream(cx, winner)) {
self.aborted = true;
return Poll::Ready(Some(Err(e)));
}
self.update_loser_tree();
}
let stream_idx = self.loser_tree[0];
if self.advance(stream_idx) {
self.loser_tree_adjusted = false;
self.in_progress.push_row(stream_idx);
// stop sorting if fetch has been reached
if self.fetch_reached() {
self.aborted = true;
} else if self.in_progress.len() < self.batch_size {
continue;
}
}
self.produced += self.in_progress.len();
return Poll::Ready(self.in_progress.build_record_batch().transpose());
}
}
fn fetch_reached(&mut self) -> bool {
self.fetch
.map(|fetch| self.produced + self.in_progress.len() >= fetch)
.unwrap_or(false)
}
fn advance(&mut self, stream_idx: usize) -> bool {
let slot = &mut self.cursors[stream_idx];
match slot.as_mut() {
Some(c) => {
c.advance();
if c.is_finished() {
*slot = None;
}
true
}
None => false,
}
}
/// Returns `true` if the cursor at index `a` is greater than at index `b`
#[inline]
fn is_gt(&self, a: usize, b: usize) -> bool {
match (&self.cursors[a], &self.cursors[b]) {
(None, _) => true,
(_, None) => false,
(Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(),
}
}
/// Find the leaf node index in the loser tree for the given cursor index
///
/// Note that this is not necessarily a leaf node in the tree, but it can
/// also be a half-node (a node with only one child). This happens when the
/// number of cursors/streams is not a power of two. Thus, the loser tree
/// will be unbalanced, but it will still work correctly.
///
/// For example, with 5 streams, the loser tree will look like this:
///
/// ```text
/// 0 (winner)
///
/// 1
/// / \
/// 2 3
/// / \ / \
/// 4 | | |
/// / \ | | |
/// -+---+--+---+---+---- Below is not a part of loser tree
/// S3 S4 S0 S1 S2
/// ```
///
/// S0, S1, ... S4 are the streams (read: stream at index 0, stream at
/// index 1, etc.)
///
/// Zooming in at node 2 in the loser tree as an example, we can see that
/// it takes as input the next item at (S0) and the loser of (S3, S4).
///
#[inline]
fn lt_leaf_node_index(&self, cursor_index: usize) -> usize {
(self.cursors.len() + cursor_index) / 2
}
/// Find the parent node index for the given node index
#[inline]
fn lt_parent_node_index(&self, node_idx: usize) -> usize {
node_idx / 2
}
/// Attempts to initialize the loser tree with one value from each
/// non exhausted input, if possible
fn init_loser_tree(&mut self) {
// Init loser tree
self.loser_tree = vec![usize::MAX; self.cursors.len()];
for i in 0..self.cursors.len() {
let mut winner = i;
let mut cmp_node = self.lt_leaf_node_index(i);
while cmp_node != 0 && self.loser_tree[cmp_node] != usize::MAX {
let challenger = self.loser_tree[cmp_node];
if self.is_gt(winner, challenger) {
self.loser_tree[cmp_node] = winner;
winner = challenger;
}
cmp_node = self.lt_parent_node_index(cmp_node);
}
self.loser_tree[cmp_node] = winner;
}
self.loser_tree_adjusted = true;
}
/// Attempts to update the loser tree, following winner replacement, if possible
fn update_loser_tree(&mut self) {
let mut winner = self.loser_tree[0];
// Replace overall winner by walking tree of losers
let mut cmp_node = self.lt_leaf_node_index(winner);
while cmp_node != 0 {
let challenger = self.loser_tree[cmp_node];
if self.is_gt(winner, challenger) {
self.loser_tree[cmp_node] = winner;
winner = challenger;
}
cmp_node = self.lt_parent_node_index(cmp_node);
}
self.loser_tree[0] = winner;
self.loser_tree_adjusted = true;
}
}
impl<C: CursorValues + Unpin> Stream for SortPreservingMergeStream<C> {
type Item = Result<RecordBatch>;
fn poll_next(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
let poll = self.poll_next_inner(cx);
self.metrics.record_poll(poll)
}
}
impl<C: CursorValues + Unpin> RecordBatchStream for SortPreservingMergeStream<C> {
fn schema(&self) -> SchemaRef {
self.in_progress.schema().clone()
}
}