jonathanc-n commented on code in PR #982: URL: https://github.com/apache/iceberg-rust/pull/982#discussion_r1994474161
########## crates/iceberg/src/arrow/delete_file_manager.rs: ########## @@ -0,0 +1,564 @@ +// 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 std::future::Future; +use std::ops::BitOrAssign; +use std::pin::Pin; +use std::sync::{Arc, OnceLock, RwLock}; +use std::task::{Context, Poll}; + +use futures::channel::oneshot; +use futures::future::join_all; +use futures::{StreamExt, TryStreamExt}; +use roaring::RoaringTreemap; + +use crate::arrow::ArrowReader; +use crate::expr::Predicate::AlwaysTrue; +use crate::expr::{Bind, BoundPredicate, Predicate}; +use crate::io::FileIO; +use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskDeleteFile}; +use crate::spec::DataContentType; +use crate::{Error, ErrorKind, Result}; + +// Equality deletes may apply to more than one DataFile in a scan, and so +// the same equality delete file may be present in more than one invocation of +// DeleteFileManager::load_deletes in the same scan. We want to deduplicate these +// to avoid having to load them twice, so we immediately store cloneable futures in the +// state that can be awaited upon to get te EQ deletes. That way we can check to see if +// a load of each Eq delete file is already in progress and avoid starting another one. +#[derive(Debug, Clone)] +struct EqDelFuture { + result: OnceLock<Predicate>, +} + +impl EqDelFuture { + pub fn new() -> (oneshot::Sender<Predicate>, Self) { + let (tx, rx) = oneshot::channel(); + let result = OnceLock::new(); + + crate::runtime::spawn({ + let result = result.clone(); + async move { result.set(rx.await.unwrap()) } + }); + + (tx, Self { result }) + } +} + +impl Future for EqDelFuture { + type Output = Predicate; + + fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<Self::Output> { + match self.result.get() { + None => Poll::Pending, + Some(predicate) => Poll::Ready(predicate.clone()), + } + } +} + +#[derive(Debug, Default)] +struct DeleteFileManagerState { + // delete vectors and positional deletes get merged when loaded into a single delete vector + // per data file + delete_vectors: HashMap<String, RoaringTreemap>, Review Comment: I think a function should be included for enabling deletion vectors for the future when a property is added. ########## crates/iceberg/src/arrow/delete_file_manager.rs: ########## @@ -0,0 +1,564 @@ +// 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 std::future::Future; +use std::ops::BitOrAssign; +use std::pin::Pin; +use std::sync::{Arc, OnceLock, RwLock}; +use std::task::{Context, Poll}; + +use futures::channel::oneshot; +use futures::future::join_all; +use futures::{StreamExt, TryStreamExt}; +use roaring::RoaringTreemap; + +use crate::arrow::ArrowReader; +use crate::expr::Predicate::AlwaysTrue; +use crate::expr::{Bind, BoundPredicate, Predicate}; +use crate::io::FileIO; +use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskDeleteFile}; +use crate::spec::DataContentType; +use crate::{Error, ErrorKind, Result}; + +// Equality deletes may apply to more than one DataFile in a scan, and so +// the same equality delete file may be present in more than one invocation of +// DeleteFileManager::load_deletes in the same scan. We want to deduplicate these +// to avoid having to load them twice, so we immediately store cloneable futures in the +// state that can be awaited upon to get te EQ deletes. That way we can check to see if +// a load of each Eq delete file is already in progress and avoid starting another one. +#[derive(Debug, Clone)] +struct EqDelFuture { + result: OnceLock<Predicate>, +} + +impl EqDelFuture { + pub fn new() -> (oneshot::Sender<Predicate>, Self) { + let (tx, rx) = oneshot::channel(); + let result = OnceLock::new(); + + crate::runtime::spawn({ + let result = result.clone(); + async move { result.set(rx.await.unwrap()) } + }); + + (tx, Self { result }) + } +} + +impl Future for EqDelFuture { + type Output = Predicate; + + fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<Self::Output> { + match self.result.get() { + None => Poll::Pending, + Some(predicate) => Poll::Ready(predicate.clone()), + } + } +} + +#[derive(Debug, Default)] +struct DeleteFileManagerState { + // delete vectors and positional deletes get merged when loaded into a single delete vector + // per data file + delete_vectors: HashMap<String, RoaringTreemap>, + + // equality delete files are parsed into unbound `Predicate`s. We store them here as + // cloneable futures (see note below) + equality_deletes: HashMap<String, EqDelFuture>, +} + +type StateRef = Arc<RwLock<DeleteFileManagerState>>; + +#[derive(Clone, Debug)] +pub(crate) struct DeleteFileManager { + state: Arc<RwLock<DeleteFileManagerState>>, +} + +// Intermediate context during processing of a delete file task. +enum DeleteFileContext { + // TODO: Delete Vector loader from Puffin files + InProgEqDel(EqDelFuture), + PosDels(ArrowRecordBatchStream), + FreshEqDel { + batch_stream: ArrowRecordBatchStream, + sender: 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 { + InProgEqDel(EqDelFuture), + DelVecs(HashMap<String, RoaringTreemap>), + EqDel, +} + +#[allow(unused_variables)] +impl DeleteFileManager { + pub(crate) fn new() -> DeleteFileManager { + Self { + state: Default::default(), + } + } + + pub(crate) async fn load_deletes( + &self, + delete_file_entries: &[FileScanTaskDeleteFile], + file_io: FileIO, + concurrency_limit_data_files: usize, + ) -> Result<()> { + /* + * 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 return a future + for the pre-existing task from the load phase. If not, we create such a future + and store it in the state to prevent other data file tasks from starting to load + the same equality delete file, and return a record batch stream from the load phase + as per the other delete file types - only this time it is accompanied by a one-shot + channel sender that we will eventually use to resolve the shared future that we stored + in the state. + * 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 manager's 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 of 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: + + FileScanTaskDeleteFile + | + Already-loading EQ Delete | Everything Else + +---------------------------------------------------+ + | | + [get existing future] [load recordbatch stream / puffin] + DeleteFileContext::InProgEqDel 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!] + */ + + let stream_items = delete_file_entries + .iter() + .map(|t| (t.clone(), file_io.clone(), self.state.clone())) + .collect::<Vec<_>>(); + // NOTE: removing the collect and just passing the iterator to futures::stream:iter + // results in an error 'implementation of `std::ops::FnOnce` is not general enough' + + let task_stream = futures::stream::iter(stream_items.into_iter()); + + let results: Vec<ParsedDeleteFileContext> = task_stream + .map(move |(task, file_io, state_ref)| async { + Self::load_file_for_task(task, file_io, state_ref).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?; + + // wait for all in-progress EQ deletes from other tasks + let _ = join_all(results.iter().filter_map(|i| { + if let ParsedDeleteFileContext::InProgEqDel(fut) = i { + Some(fut.clone()) + } else { + None + } + })) + .await; + + let merged_delete_vectors = results + .into_iter() + .fold(HashMap::default(), Self::merge_delete_vectors); + + self.state.write().unwrap().delete_vectors = merged_delete_vectors; + + Ok(()) + } + + async fn load_file_for_task( + task: FileScanTaskDeleteFile, + file_io: FileIO, + state: StateRef, + ) -> Result<DeleteFileContext> { + match task.file_type { + DataContentType::PositionDeletes => Ok(DeleteFileContext::PosDels( + Self::parquet_to_batch_stream(&task.file_path, file_io).await?, + )), + + DataContentType::EqualityDeletes => { + let (sender, fut) = EqDelFuture::new(); Review Comment: I think we can do the check before creating the future ########## crates/iceberg/src/arrow/delete_file_manager.rs: ########## @@ -0,0 +1,564 @@ +// 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 std::future::Future; +use std::ops::BitOrAssign; +use std::pin::Pin; +use std::sync::{Arc, OnceLock, RwLock}; +use std::task::{Context, Poll}; + +use futures::channel::oneshot; +use futures::future::join_all; +use futures::{StreamExt, TryStreamExt}; +use roaring::RoaringTreemap; + +use crate::arrow::ArrowReader; +use crate::expr::Predicate::AlwaysTrue; +use crate::expr::{Bind, BoundPredicate, Predicate}; +use crate::io::FileIO; +use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskDeleteFile}; +use crate::spec::DataContentType; +use crate::{Error, ErrorKind, Result}; + +// Equality deletes may apply to more than one DataFile in a scan, and so +// the same equality delete file may be present in more than one invocation of +// DeleteFileManager::load_deletes in the same scan. We want to deduplicate these +// to avoid having to load them twice, so we immediately store cloneable futures in the +// state that can be awaited upon to get te EQ deletes. That way we can check to see if +// a load of each Eq delete file is already in progress and avoid starting another one. +#[derive(Debug, Clone)] +struct EqDelFuture { + result: OnceLock<Predicate>, +} + +impl EqDelFuture { + pub fn new() -> (oneshot::Sender<Predicate>, Self) { + let (tx, rx) = oneshot::channel(); + let result = OnceLock::new(); + + crate::runtime::spawn({ + let result = result.clone(); + async move { result.set(rx.await.unwrap()) } + }); + + (tx, Self { result }) + } +} + +impl Future for EqDelFuture { + type Output = Predicate; + + fn poll(self: Pin<&mut Self>, _cx: &mut Context<'_>) -> Poll<Self::Output> { + match self.result.get() { + None => Poll::Pending, + Some(predicate) => Poll::Ready(predicate.clone()), + } + } +} + +#[derive(Debug, Default)] +struct DeleteFileManagerState { + // delete vectors and positional deletes get merged when loaded into a single delete vector + // per data file + delete_vectors: HashMap<String, RoaringTreemap>, + + // equality delete files are parsed into unbound `Predicate`s. We store them here as + // cloneable futures (see note below) + equality_deletes: HashMap<String, EqDelFuture>, +} + +type StateRef = Arc<RwLock<DeleteFileManagerState>>; + +#[derive(Clone, Debug)] +pub(crate) struct DeleteFileManager { + state: Arc<RwLock<DeleteFileManagerState>>, +} + +// Intermediate context during processing of a delete file task. +enum DeleteFileContext { + // TODO: Delete Vector loader from Puffin files + InProgEqDel(EqDelFuture), + PosDels(ArrowRecordBatchStream), + FreshEqDel { + batch_stream: ArrowRecordBatchStream, + sender: 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 { + InProgEqDel(EqDelFuture), + DelVecs(HashMap<String, RoaringTreemap>), + EqDel, +} + +#[allow(unused_variables)] +impl DeleteFileManager { + pub(crate) fn new() -> DeleteFileManager { + Self { + state: Default::default(), + } + } + + pub(crate) async fn load_deletes( + &self, + delete_file_entries: &[FileScanTaskDeleteFile], + file_io: FileIO, + concurrency_limit_data_files: usize, + ) -> Result<()> { + /* + * Create a single stream of all delete file tasks irrespective of type, Review Comment: This is really nice. Are we able to make this into rust doc instead? would be nice for people looking into the crate -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org