sdd commented on code in PR #982:
URL: https://github.com/apache/iceberg-rust/pull/982#discussion_r2048330445


##########
crates/iceberg/src/arrow/delete_file_manager.rs:
##########
@@ -47,47 +60,533 @@ impl DeleteFileManager for CachingDeleteFileManager {
         ))
     }
 }
+// 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, Arc<RwLock<DeleteVector>>>,
+
+    // 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>>;
+
+// 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, DeleteVector>),
+    EqDel,
+}
 
 #[allow(unused_variables)]
 impl CachingDeleteFileManager {
-    pub fn new(file_io: FileIO, concurrency_limit_data_files: usize) -> 
CachingDeleteFileManager {
-        Self {
+    pub(crate) fn new(file_io: FileIO, concurrency_limit_data_files: usize) -> 
Self {
+        CachingDeleteFileManager {
             file_io,
             concurrency_limit_data_files,
+            state: Arc::new(Default::default()),
         }
     }
 
+    /// Load the deletes for all the specified tasks
+    ///
+    /// Returned future completes once all loading has finished.
+    ///
+    ///  * 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 
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 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:
+    /// ```none
+    ///                                          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!]
+    /// ```
     pub(crate) async fn load_deletes(
         &self,
-        delete_file_entries: Vec<FileScanTaskDeleteFile>,
+        delete_file_entries: &[FileScanTaskDeleteFile],
     ) -> Result<()> {
-        // TODO
+        let stream_items = delete_file_entries
+            .iter()
+            .map(|t| (t.clone(), self.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'
 
-        if !delete_file_entries.is_empty() {
-            Err(Error::new(
-                ErrorKind::FeatureUnsupported,
-                "Reading delete files is not yet supported",
-            ))
-        } else {
-            Ok(())
+        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(self.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 = {
+                    let mut state = state.write().unwrap();
+                    if let Some(existing) = 
state.equality_deletes.get(&task.file_path) {
+                        return 
Ok(DeleteFileContext::InProgEqDel(existing.clone()));
+                    }
+
+                    let (sender, fut) = EqDelFuture::new();
+
+                    state
+                        .equality_deletes
+                        .insert(task.file_path.to_string(), fut);
+
+                    sender
+                };
+
+                Ok(DeleteFileContext::FreshEqDel {
+                    batch_stream: 
Self::parquet_to_batch_stream(&task.file_path, file_io).await?,
+                    sender,
+                })
+            }
+
+            DataContentType::Data => Err(Error::new(
+                ErrorKind::Unexpected,
+                "tasks with files of type Data not expected here",
+            )),
         }
     }
 
-    pub(crate) fn build_delete_predicate(
+    async fn parse_file_content_for_task(
+        ctx: DeleteFileContext,
+    ) -> Result<ParsedDeleteFileContext> {
+        match ctx {
+            DeleteFileContext::InProgEqDel(fut) => 
Ok(ParsedDeleteFileContext::InProgEqDel(fut)),
+            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)
+            }
+        }
+    }
+
+    fn merge_delete_vectors(
+        mut merged_delete_vectors: HashMap<String, Arc<RwLock<DeleteVector>>>,
+        item: ParsedDeleteFileContext,
+    ) -> HashMap<String, Arc<RwLock<DeleteVector>>> {
+        if let ParsedDeleteFileContext::DelVecs(del_vecs) = item {
+            del_vecs.into_iter().for_each(|(key, val)| {
+                let entry = merged_delete_vectors.entry(key).or_default();
+                {
+                    let mut inner = entry.write().unwrap();
+                    (*inner).intersect_assign(&val);
+                }
+            });
+        }
+
+        merged_delete_vectors
+    }
+
+    /// Loads a RecordBatchStream for a given datafile.
+    async fn parquet_to_batch_stream(
+        data_file_path: &str,
+        file_io: FileIO,
+    ) -> Result<ArrowRecordBatchStream> {
+        /*
+           Essentially a super-cut-down ArrowReader. We can't use ArrowReader 
directly
+           as that introduces a circular dependency.
+        */
+        let record_batch_stream = 
ArrowReader::create_parquet_record_batch_stream_builder(
+            data_file_path,
+            file_io.clone(),
+            false,
+        )
+        .await?
+        .build()?
+        .map_err(|e| Error::new(ErrorKind::Unexpected, format!("{}", e)));
+
+        Ok(Box::pin(record_batch_stream) as ArrowRecordBatchStream)
+    }
+
+    /// 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
+
+        Ok(HashMap::default())

Review Comment:
   Done



-- 
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

Reply via email to