-
Notifications
You must be signed in to change notification settings - Fork 261
Scan Delete Support Part 4: Delete File Loading; Skeleton for Processing #982
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
Open
sdd
wants to merge
12
commits into
apache:main
Choose a base branch
from
sdd:feat/delete-fila-manager-loading
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
12 commits
Select commit
Hold shift + click to select a range
3776aeb
feat: delete file manager loading
sdd 0078be5
feat: changes suggested in review
sdd e2903e1
feat: return Err for unimplemented delete vec parse methods and make …
sdd 65dd638
feat: schema evolution of equality delete file record batches
sdd 79415ac
refactor: split DeleteFileManager into DeleteFileLoader and DeleteFilter
sdd 2207257
fix: add waker for DeleteFileIndex
sdd 2cf7692
refactor: remove DeleteFileFilter from CachingDeleteFileLoader
sdd 839232f
refactor: extract BasicDeleteFileLoader from CachingDeleteFileLoader
sdd d6a4a4d
feat: remove flag to selectively enable delete file processing in vav…
sdd 462b2f7
changes required after rebase on main
sdd ef28809
fix: handle WouldBlock correctly in DeleteFileIndex
sdd b147098
refactor: use Notify and oneshot channel rather than custom Future
sdd 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
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
This file contains hidden or 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,324 @@ | ||
// 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 std::collections::HashMap; | ||
|
||
use futures::{StreamExt, TryStreamExt}; | ||
use tokio::sync::oneshot::{Receiver, channel}; | ||
|
||
use super::delete_filter::DeleteFilter; | ||
use crate::arrow::delete_file_loader::BasicDeleteFileLoader; | ||
use crate::delete_vector::DeleteVector; | ||
use crate::expr::Predicate; | ||
use crate::io::FileIO; | ||
use crate::scan::{ArrowRecordBatchStream, FileScanTaskDeleteFile}; | ||
use crate::spec::{DataContentType, SchemaRef}; | ||
use crate::{Error, ErrorKind, Result}; | ||
|
||
#[derive(Clone, Debug)] | ||
pub(crate) struct CachingDeleteFileLoader { | ||
basic_delete_file_loader: BasicDeleteFileLoader, | ||
concurrency_limit_data_files: usize, | ||
} | ||
|
||
// Intermediate context during processing of a delete file task. | ||
enum DeleteFileContext { | ||
// TODO: Delete Vector loader from Puffin files | ||
ExistingEqDel, | ||
PosDels(ArrowRecordBatchStream), | ||
FreshEqDel { | ||
batch_stream: ArrowRecordBatchStream, | ||
sender: tokio::sync::oneshot::Sender<Predicate>, | ||
}, | ||
} | ||
|
||
// Final result of the processing of a delete file task before | ||
// results are fully merged into the DeleteFileManager's state | ||
enum ParsedDeleteFileContext { | ||
DelVecs(HashMap<String, DeleteVector>), | ||
EqDel, | ||
} | ||
|
||
#[allow(unused_variables)] | ||
impl CachingDeleteFileLoader { | ||
pub(crate) fn new(file_io: FileIO, concurrency_limit_data_files: usize) -> Self { | ||
CachingDeleteFileLoader { | ||
basic_delete_file_loader: BasicDeleteFileLoader::new(file_io), | ||
concurrency_limit_data_files, | ||
} | ||
} | ||
|
||
/// Initiates loading of all deletes for all the specified tasks | ||
/// | ||
/// Returned future completes once all positional deletes and delete vectors | ||
/// have loaded. EQ deletes are not waited for in this method but the returned | ||
/// DeleteFilter will await their loading when queried for them. | ||
/// | ||
/// * Create a single stream of all delete file tasks irrespective of type, | ||
/// so that we can respect the combined concurrency limit | ||
/// * We then process each in two phases: load and parse. | ||
/// * for positional deletes the load phase instantiates an ArrowRecordBatchStream to | ||
/// stream the file contents out | ||
/// * for eq deletes, we first check if the EQ delete is already loaded or being loaded by | ||
/// another concurrently processing data file scan task. If it is, we skip it. | ||
/// If not, the DeleteFilter is updated to contain a notifier to prevent other data file | ||
/// tasks from starting to load the same equality delete file. We spawn a task to load | ||
/// the EQ delete's record batch stream, convert it to a predicate, update the delete filter, | ||
/// and notify any task that was waiting for it. | ||
/// * When this gets updated to add support for delete vectors, the load phase will return | ||
/// a PuffinReader for them. | ||
/// * The parse phase parses each record batch stream according to its associated data type. | ||
/// The result of this is a map of data file paths to delete vectors for the positional | ||
/// delete tasks (and in future for the delete vector tasks). For equality delete | ||
/// file tasks, this results in an unbound Predicate. | ||
/// * The unbound Predicates resulting from equality deletes are sent to their associated oneshot | ||
/// channel to store them in the right place in the delete file managers state. | ||
/// * The results of all of these futures are awaited on in parallel with the specified | ||
/// level of concurrency and collected into a vec. We then combine all the delete | ||
/// vector maps that resulted from any positional delete or delete vector files into a | ||
/// single map and persist it in the state. | ||
/// | ||
/// | ||
/// Conceptually, the data flow is like this: | ||
/// ```none | ||
/// FileScanTaskDeleteFile | ||
/// | | ||
/// Skip Started EQ Deletes | ||
/// | | ||
/// | | ||
/// [load recordbatch stream / puffin] | ||
/// DeleteFileContext | ||
/// | | ||
/// | | ||
/// +-----------------------------+--------------------------+ | ||
/// Pos Del Del Vec (Not yet Implemented) EQ Del | ||
/// | | | | ||
/// [parse pos del stream] [parse del vec puffin] [parse eq del] | ||
/// HashMap<String, RoaringTreeMap> HashMap<String, RoaringTreeMap> (Predicate, Sender) | ||
/// | | | | ||
/// | | [persist to state] | ||
/// | | () | ||
/// | | | | ||
/// +-----------------------------+--------------------------+ | ||
/// | | ||
/// [buffer unordered] | ||
/// | | ||
/// [combine del vectors] | ||
/// HashMap<String, RoaringTreeMap> | ||
/// | | ||
/// [persist del vectors to state] | ||
/// () | ||
/// | | ||
/// | | ||
/// [join!] | ||
/// ``` | ||
pub(crate) fn load_deletes( | ||
&self, | ||
delete_file_entries: &[FileScanTaskDeleteFile], | ||
schema: SchemaRef, | ||
) -> Receiver<Result<DeleteFilter>> { | ||
let (tx, rx) = channel(); | ||
let del_filter = DeleteFilter::default(); | ||
|
||
let stream_items = delete_file_entries | ||
.iter() | ||
.map(|t| { | ||
( | ||
t.clone(), | ||
self.basic_delete_file_loader.clone(), | ||
del_filter.clone(), | ||
schema.clone(), | ||
) | ||
}) | ||
.collect::<Vec<_>>(); | ||
let task_stream = futures::stream::iter(stream_items); | ||
|
||
let del_filter = del_filter.clone(); | ||
let concurrency_limit_data_files = self.concurrency_limit_data_files; | ||
let basic_delete_file_loader = self.basic_delete_file_loader.clone(); | ||
crate::runtime::spawn(async move { | ||
let result = async move { | ||
let mut del_filter = del_filter; | ||
let basic_delete_file_loader = basic_delete_file_loader.clone(); | ||
|
||
let results: Vec<ParsedDeleteFileContext> = task_stream | ||
.map(move |(task, file_io, del_filter, schema)| { | ||
let basic_delete_file_loader = basic_delete_file_loader.clone(); | ||
async move { | ||
Self::load_file_for_task( | ||
&task, | ||
basic_delete_file_loader.clone(), | ||
del_filter, | ||
schema, | ||
) | ||
.await | ||
} | ||
}) | ||
.map(move |ctx| { | ||
Ok(async { Self::parse_file_content_for_task(ctx.await?).await }) | ||
}) | ||
.try_buffer_unordered(concurrency_limit_data_files) | ||
.try_collect::<Vec<_>>() | ||
.await?; | ||
|
||
for item in results { | ||
if let ParsedDeleteFileContext::DelVecs(hash_map) = item { | ||
for (data_file_path, delete_vector) in hash_map.into_iter() { | ||
del_filter.upsert_delete_vector(data_file_path, delete_vector); | ||
} | ||
} | ||
} | ||
|
||
Ok(del_filter) | ||
} | ||
.await; | ||
|
||
let _ = tx.send(result); | ||
}); | ||
|
||
rx | ||
} | ||
|
||
async fn load_file_for_task( | ||
task: &FileScanTaskDeleteFile, | ||
basic_delete_file_loader: BasicDeleteFileLoader, | ||
del_filter: DeleteFilter, | ||
schema: SchemaRef, | ||
) -> Result<DeleteFileContext> { | ||
match task.file_type { | ||
DataContentType::PositionDeletes => Ok(DeleteFileContext::PosDels( | ||
basic_delete_file_loader | ||
.parquet_to_batch_stream(&task.file_path) | ||
.await?, | ||
)), | ||
|
||
DataContentType::EqualityDeletes => { | ||
let Some(notify) = del_filter.try_start_eq_del_load(&task.file_path) else { | ||
return Ok(DeleteFileContext::ExistingEqDel); | ||
}; | ||
|
||
let (sender, receiver) = channel(); | ||
del_filter.insert_equality_delete(&task.file_path, receiver); | ||
|
||
Ok(DeleteFileContext::FreshEqDel { | ||
batch_stream: BasicDeleteFileLoader::evolve_schema( | ||
basic_delete_file_loader | ||
.parquet_to_batch_stream(&task.file_path) | ||
.await?, | ||
schema, | ||
) | ||
.await?, | ||
sender, | ||
}) | ||
} | ||
|
||
DataContentType::Data => Err(Error::new( | ||
ErrorKind::Unexpected, | ||
"tasks with files of type Data not expected here", | ||
)), | ||
} | ||
} | ||
|
||
async fn parse_file_content_for_task( | ||
ctx: DeleteFileContext, | ||
) -> Result<ParsedDeleteFileContext> { | ||
match ctx { | ||
DeleteFileContext::ExistingEqDel => Ok(ParsedDeleteFileContext::EqDel), | ||
DeleteFileContext::PosDels(batch_stream) => { | ||
let del_vecs = | ||
Self::parse_positional_deletes_record_batch_stream(batch_stream).await?; | ||
Ok(ParsedDeleteFileContext::DelVecs(del_vecs)) | ||
} | ||
DeleteFileContext::FreshEqDel { | ||
sender, | ||
batch_stream, | ||
} => { | ||
let predicate = | ||
Self::parse_equality_deletes_record_batch_stream(batch_stream).await?; | ||
|
||
sender | ||
.send(predicate) | ||
.map_err(|err| { | ||
Error::new( | ||
ErrorKind::Unexpected, | ||
"Could not send eq delete predicate to state", | ||
) | ||
}) | ||
.map(|_| ParsedDeleteFileContext::EqDel) | ||
} | ||
} | ||
} | ||
|
||
/// Parses a record batch stream coming from positional delete files | ||
/// | ||
/// Returns a map of data file path to a delete vector | ||
async fn parse_positional_deletes_record_batch_stream( | ||
stream: ArrowRecordBatchStream, | ||
) -> Result<HashMap<String, DeleteVector>> { | ||
// TODO | ||
|
||
Err(Error::new( | ||
ErrorKind::FeatureUnsupported, | ||
"parsing of positional deletes is not yet supported", | ||
)) | ||
} | ||
|
||
/// Parses record batch streams from individual equality delete files | ||
/// | ||
/// Returns an unbound Predicate for each batch stream | ||
async fn parse_equality_deletes_record_batch_stream( | ||
streams: ArrowRecordBatchStream, | ||
) -> Result<Predicate> { | ||
// TODO | ||
|
||
Err(Error::new( | ||
ErrorKind::FeatureUnsupported, | ||
"parsing of equality deletes is not yet supported", | ||
)) | ||
} | ||
} | ||
|
||
#[cfg(test)] | ||
mod tests { | ||
use tempfile::TempDir; | ||
|
||
use super::*; | ||
use crate::arrow::delete_file_loader::tests::setup; | ||
|
||
#[tokio::test] | ||
async fn test_delete_file_manager_load_deletes() { | ||
let tmp_dir = TempDir::new().unwrap(); | ||
let table_location = tmp_dir.path(); | ||
let file_io = FileIO::from_path(table_location.as_os_str().to_str().unwrap()) | ||
.unwrap() | ||
.build() | ||
.unwrap(); | ||
|
||
// Note that with the delete file parsing not yet in place, all we can test here is that | ||
// the call to the loader fails with the expected FeatureUnsupportedError. | ||
let delete_file_manager = CachingDeleteFileLoader::new(file_io.clone(), 10); | ||
|
||
let file_scan_tasks = setup(table_location); | ||
|
||
let result = delete_file_manager | ||
.load_deletes(&file_scan_tasks[0].deletes, file_scan_tasks[0].schema_ref()) | ||
.await | ||
.unwrap(); | ||
|
||
assert!(result.is_err_and(|e| e.kind() == ErrorKind::FeatureUnsupported)); | ||
} | ||
} |
Oops, something went wrong.
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.
Looking forward to the puffin / deletion vector support!
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.
Me too! 😁