-
Notifications
You must be signed in to change notification settings - Fork 1.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat(7181): cascading loser tree merges #7379
Closed
Closed
Changes from all commits
Commits
Show all changes
30 commits
Select commit
Hold shift + click to select a range
5eaaeec
refactor(7181): move the management of input data, and output data, t…
wiedld a3870d0
feat(7181): add cursor.seek()
wiedld e54e92c
refactor(7181): move streaming_merge() into separate mod from the mer…
wiedld 3d43e97
feat(7181): streaming_merge() consumes SortPreservingCascadeStream
wiedld 1a6a364
feat(7181): change BatchBuilder to be a SortOrder Builder, with the S…
wiedld 28454c5
feat(7181): add slice() to Cursor trait
wiedld b766712
feat(7181): make SortOrderBuilder yield in a stream.
wiedld 9ff37f3
feat(7181): enable the ability to create a multi-layer merge cascade.
wiedld eb647ea
feat(7181): build multiple-level cascade tree.
wiedld 8cd22a0
feat(7181): use RecordBatch tracking to avoid expensive slicing of ba…
wiedld 173577b
fix(7181): improve performance by using hasher on tuple (unqiue slice…
wiedld b0f1402
chore(7181): make a zero-cost BatchId type, for more explicit code
wiedld 9ea3a65
refactor: comment the major streaming structures, and how they are in…
wiedld 7be30c2
refactor: use u64 as batch_id in cascading merge sort
wiedld 0e9573d
feat(7181): convert into generic ReceiverStream, such that can be reu…
wiedld c439138
feat(7181): add buffered multithreading to merge streams
wiedld fca522b
test(7181): have sort preserving merge tests, run in both single thre…
wiedld 50c8636
chore: TMP COMMIT pointing at arrow-rs branch, for CI pipeline
wiedld cfa32fa
Merge branch 'main' into 7181/cascading-loser-tree-merges
wiedld d520496
chore: clippy and linter
wiedld a324ef8
fix(7181): have RowCursor slicing be within the a single arc-refed Rows
wiedld d3613bd
feat(7181): have BatchCursor be the primary struct passed around
wiedld 3786021
feat(7181): update documentation for the cascaded merge
wiedld 2932bd5
Merge branch 'main' into 7181/cascading-loser-tree-merges
wiedld 8701220
fix: add apache license header to new mods
wiedld 828a5d1
Merge branch 'main' into 7181/cascading-loser-tree-merges
wiedld 0dfc60c
Merge branch 'main' into 7181/cascading-loser-tree-merges
wiedld e642420
Merge branch 'main' into 7181/cascading-loser-tree-merges
wiedld f97cc4d
feat(7181): remove mutex around polled stream.
wiedld 9b10198
Merge branch 'main' into 7181/cascading-loser-tree-merges
wiedld File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,89 @@ | ||
// 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. | ||
|
||
use datafusion_common::Result; | ||
|
||
use super::cursor::Cursor; | ||
|
||
pub type BatchId = u64; | ||
|
||
#[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)] | ||
pub struct BatchOffset(pub usize); | ||
|
||
pub type SlicedBatchCursorIdentifier = (BatchId, BatchOffset); | ||
|
||
/// The [`BatchCursor`] represents a complete, or partial, [`Cursor`] for a given record batch ([`BatchId`]). | ||
/// | ||
/// A record batch (represented by its [`Cursor`]) can be sliced due to the following reason: | ||
/// 1. a merge node takes in 10 streams | ||
/// 2 at any given time, this means up to 10 cursors (record batches) are being merged (e.g. in the loser tree) | ||
/// 3. merge nodes will yield once it hits a size limit | ||
/// 4. at the moment of yielding, there may be some cursors which are partially yielded | ||
/// | ||
/// Unique representation of sliced cursor is denoted by the [`SlicedBatchCursorIdentifier`]. | ||
#[derive(Debug)] | ||
pub struct BatchCursor<C: Cursor> { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is used in the CursorStream (not the BatchCursorStream which included the actual record batches). Therefore, I think this could have a better name. It wraps the cursor, and maps it to the original (tracked) batch -- as well as tracking the sliced offset. Naming ideas? |
||
/// The index into BatchTrackingStream::batches | ||
batch: BatchId, | ||
/// The offset of the row within the given batch, based on the idea of a sliced cursor. | ||
/// When a batch is partially yielded, then the offset->end will determine how much was yielded. | ||
row_offset: BatchOffset, | ||
|
||
/// The cursor for the given batch. | ||
pub cursor: C, | ||
} | ||
|
||
impl<C: Cursor> BatchCursor<C> { | ||
/// Create a new [`BatchCursor`] from a [`Cursor`] and a [`BatchId`]. | ||
/// | ||
/// New [`BatchCursor`]s will have a [`BatchOffset`] of 0. | ||
/// Subsequent batch_cursors can be created by slicing. | ||
pub fn new(batch: BatchId, cursor: C) -> Self { | ||
Self { | ||
batch, | ||
row_offset: BatchOffset(0), | ||
cursor, | ||
} | ||
} | ||
|
||
/// A unique identifier used to identify a [`BatchCursor`] | ||
pub fn identifier(&self) -> SlicedBatchCursorIdentifier { | ||
(self.batch, self.row_offset) | ||
} | ||
|
||
/// Slicing of a batch cursor is done by slicing the underlying cursor, | ||
/// and adjust the BatchOffset | ||
pub fn slice(&self, offset: usize, length: usize) -> Result<Self> { | ||
Ok(Self { | ||
batch: self.batch, | ||
row_offset: BatchOffset(self.row_offset.0 + offset), | ||
cursor: self.cursor.slice(offset, length)?, | ||
}) | ||
} | ||
} | ||
|
||
impl<C: Cursor> std::fmt::Display for BatchCursor<C> { | ||
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { | ||
write!( | ||
f, | ||
"BatchCursor(batch: {}, offset: {}, num_rows: {})", | ||
self.batch, | ||
self.row_offset.0, | ||
self.cursor.num_rows() | ||
) | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
RecordBatchReceiverStream
was made generic in this commit, such that it could handle a buffered stream of record_batches, or the sort_orders (yielded per each merge node).In order to make generic, did the following:
StreamAdapter::call()
to be used forReceiverStream::run_input()
.RecordBatchReceiverStreamAdaptor
that is used for record batchesPlease let me know if I should have structured this differently.