1717
1818use std:: collections:: HashMap ;
1919
20- use arrow_array:: { Int64Array , StringArray } ;
20+ use arrow_array:: {
21+ BinaryArray , BooleanArray , Date32Array , Decimal128Array , Float32Array , Float64Array ,
22+ Int32Array , Int64Array , LargeBinaryArray , LargeStringArray , StringArray ,
23+ TimestampMicrosecondArray ,
24+ } ;
25+ use arrow_schema;
2126use futures:: { StreamExt , TryStreamExt } ;
2227use tokio:: sync:: oneshot:: { Receiver , channel} ;
2328
@@ -38,7 +43,7 @@ pub(crate) struct CachingDeleteFileLoader {
3843
3944// Intermediate context during processing of a delete file task.
4045enum DeleteFileContext {
41- // TODO: Delete Vector loader from Puffin files
46+ DelVecs ( HashMap < String , DeleteVector > ) ,
4247 ExistingEqDel ,
4348 PosDels ( ArrowRecordBatchStream ) ,
4449 FreshEqDel {
@@ -200,6 +205,11 @@ impl CachingDeleteFileLoader {
200205 del_filter : DeleteFilter ,
201206 schema : SchemaRef ,
202207 ) -> Result < DeleteFileContext > {
208+ // Check if the file is a Puffin file (by extension or by trying to read it as such)
209+ if Self :: is_puffin_file ( & task. file_path ) {
210+ return Self :: load_puffin_delete_vectors ( & task. file_path , & basic_delete_file_loader) . await ;
211+ }
212+
203213 match task. file_type {
204214 DataContentType :: PositionDeletes => Ok ( DeleteFileContext :: PosDels (
205215 basic_delete_file_loader
@@ -238,6 +248,7 @@ impl CachingDeleteFileLoader {
238248 ctx : DeleteFileContext ,
239249 ) -> Result < ParsedDeleteFileContext > {
240250 match ctx {
251+ DeleteFileContext :: DelVecs ( hash_map) => Ok ( ParsedDeleteFileContext :: DelVecs ( hash_map) ) ,
241252 DeleteFileContext :: ExistingEqDel => Ok ( ParsedDeleteFileContext :: EqDel ) ,
242253 DeleteFileContext :: PosDels ( batch_stream) => {
243254 let del_vecs =
@@ -311,15 +322,214 @@ impl CachingDeleteFileLoader {
311322 /// Parses record batch streams from individual equality delete files
312323 ///
313324 /// Returns an unbound Predicate for each batch stream
325+ ///
326+ /// Equality delete files contain rows where each row represents values that should be deleted.
327+ /// For example, if the equality IDs are [1, 2] representing columns "name" and "age",
328+ /// and the file contains rows [("Alice", 25), ("Bob", 30)], then any data rows matching
329+ /// (name="Alice" AND age=25) OR (name="Bob" AND age=30) should be deleted.
314330 async fn parse_equality_deletes_record_batch_stream (
315- streams : ArrowRecordBatchStream ,
331+ mut stream : ArrowRecordBatchStream ,
316332 ) -> Result < Predicate > {
317- // TODO
333+ use crate :: expr:: Predicate :: * ;
334+ use crate :: expr:: { Reference , Literal as ExprLiteral } ;
335+ use crate :: spec:: { Literal , PrimitiveLiteral } ;
336+ use arrow_array:: Array ;
337+
338+ let mut combined_predicates = Vec :: new ( ) ;
339+
340+ while let Some ( batch) = stream. next ( ) . await {
341+ let batch = batch?;
342+ let schema = batch. schema ( ) ;
343+
344+ // Process each row in the batch
345+ for row_idx in 0 ..batch. num_rows ( ) {
346+ let mut row_conditions = Vec :: new ( ) ;
347+
348+ // For each column in the equality delete file, create an equality condition
349+ for col_idx in 0 ..batch. num_columns ( ) {
350+ let column = batch. column ( col_idx) ;
351+ let field = schema. field ( col_idx) ;
352+
353+ // Extract the field ID from metadata
354+ let field_id = field
355+ . metadata ( )
356+ . get ( "parquet_field_id" )
357+ . or_else ( || field. metadata ( ) . get ( "PARQUET:field_id" ) )
358+ . ok_or_else ( || {
359+ Error :: new (
360+ ErrorKind :: DataInvalid ,
361+ format ! ( "Missing field ID for column '{}'" , field. name( ) ) ,
362+ )
363+ } ) ?
364+ . parse :: < i32 > ( )
365+ . map_err ( |_| {
366+ Error :: new (
367+ ErrorKind :: DataInvalid ,
368+ "Invalid field ID format" ,
369+ )
370+ } ) ?;
371+
372+ // Skip if the value is null
373+ if column. is_null ( row_idx) {
374+ continue ;
375+ }
376+
377+ // Convert Arrow value to Iceberg Literal based on data type
378+ let literal = match field. data_type ( ) {
379+ arrow_schema:: DataType :: Boolean => {
380+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: BooleanArray > ( )
381+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected BooleanArray" ) ) ?;
382+ Literal :: bool ( array. value ( row_idx) )
383+ } ,
384+ arrow_schema:: DataType :: Int32 => {
385+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: Int32Array > ( )
386+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected Int32Array" ) ) ?;
387+ Literal :: int ( array. value ( row_idx) )
388+ } ,
389+ arrow_schema:: DataType :: Int64 => {
390+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: Int64Array > ( )
391+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected Int64Array" ) ) ?;
392+ Literal :: long ( array. value ( row_idx) )
393+ } ,
394+ arrow_schema:: DataType :: Float32 => {
395+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: Float32Array > ( )
396+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected Float32Array" ) ) ?;
397+ Literal :: float ( array. value ( row_idx) )
398+ } ,
399+ arrow_schema:: DataType :: Float64 => {
400+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: Float64Array > ( )
401+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected Float64Array" ) ) ?;
402+ Literal :: double ( array. value ( row_idx) )
403+ } ,
404+ arrow_schema:: DataType :: Utf8 => {
405+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: StringArray > ( )
406+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected StringArray" ) ) ?;
407+ Literal :: string ( array. value ( row_idx) )
408+ } ,
409+ arrow_schema:: DataType :: LargeUtf8 => {
410+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: LargeStringArray > ( )
411+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected LargeStringArray" ) ) ?;
412+ Literal :: string ( array. value ( row_idx) )
413+ } ,
414+ arrow_schema:: DataType :: Binary => {
415+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: BinaryArray > ( )
416+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected BinaryArray" ) ) ?;
417+ Literal :: binary ( array. value ( row_idx) . to_vec ( ) )
418+ } ,
419+ arrow_schema:: DataType :: LargeBinary => {
420+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: LargeBinaryArray > ( )
421+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected LargeBinaryArray" ) ) ?;
422+ Literal :: binary ( array. value ( row_idx) . to_vec ( ) )
423+ } ,
424+ arrow_schema:: DataType :: Date32 => {
425+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: Date32Array > ( )
426+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected Date32Array" ) ) ?;
427+ Literal :: date ( array. value ( row_idx) )
428+ } ,
429+ arrow_schema:: DataType :: Timestamp ( _, _) => {
430+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: TimestampMicrosecondArray > ( )
431+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected TimestampMicrosecondArray" ) ) ?;
432+ Literal :: timestamp_micros ( array. value ( row_idx) )
433+ } ,
434+ arrow_schema:: DataType :: Decimal128 ( precision, scale) => {
435+ let array = column. as_any ( ) . downcast_ref :: < arrow_array:: Decimal128Array > ( )
436+ . ok_or_else ( || Error :: new ( ErrorKind :: DataInvalid , "Expected Decimal128Array" ) ) ?;
437+ Literal :: decimal_from_i128 ( array. value ( row_idx) , * precision as u32 , * scale as u32 ) ?
438+ } ,
439+ _ => {
440+ return Err ( Error :: new (
441+ ErrorKind :: FeatureUnsupported ,
442+ format ! ( "Unsupported data type for equality delete: {:?}" , field. data_type( ) ) ,
443+ ) ) ;
444+ }
445+ } ;
446+
447+ // Create equality condition: field_id = literal
448+ let condition = Equal {
449+ term : Box :: new ( Reference :: new ( field. name ( ) . to_string ( ) ) ) ,
450+ literal : ExprLiteral :: new ( literal) ,
451+ } ;
452+
453+ row_conditions. push ( condition) ;
454+ }
455+
456+ // If we have conditions for this row, combine them with AND
457+ if !row_conditions. is_empty ( ) {
458+ let row_predicate = row_conditions. into_iter ( ) . reduce ( |acc, condition| And {
459+ left : Box :: new ( acc) ,
460+ right : Box :: new ( condition) ,
461+ } ) . unwrap ( ) ;
462+
463+ combined_predicates. push ( row_predicate) ;
464+ }
465+ }
466+ }
467+
468+ // Combine all row predicates with OR (any matching row should be deleted)
469+ if combined_predicates. is_empty ( ) {
470+ Ok ( AlwaysFalse ) // No rows to delete
471+ } else {
472+ let final_predicate = combined_predicates. into_iter ( ) . reduce ( |acc, predicate| Or {
473+ left : Box :: new ( acc) ,
474+ right : Box :: new ( predicate) ,
475+ } ) . unwrap ( ) ;
476+
477+ Ok ( final_predicate)
478+ }
479+ }
480+
481+ /// Check if a file is a Puffin file based on file extension or magic bytes
482+ fn is_puffin_file ( file_path : & str ) -> bool {
483+ file_path. ends_with ( ".puffin" ) || file_path. ends_with ( ".bin" )
484+ }
485+
486+ /// Load Delete Vectors from a Puffin file
487+ async fn load_puffin_delete_vectors (
488+ file_path : & str ,
489+ basic_delete_file_loader : & BasicDeleteFileLoader ,
490+ ) -> Result < DeleteFileContext > {
491+ use crate :: puffin:: { PuffinReader , DELETION_VECTOR_V1 } ;
492+
493+ let input_file = basic_delete_file_loader. file_io ( ) . new_input ( file_path) ?;
494+ let puffin_reader = PuffinReader :: new ( input_file) ;
495+ let file_metadata = puffin_reader. file_metadata ( ) . await ?;
496+
497+ let mut delete_vectors = HashMap :: new ( ) ;
498+
499+ // Process each blob in the Puffin file
500+ for blob_metadata in file_metadata. blobs ( ) {
501+ if blob_metadata. blob_type ( ) == DELETION_VECTOR_V1 {
502+ let blob = puffin_reader. blob ( blob_metadata) . await ?;
503+ let delete_vector = Self :: parse_delete_vector_blob ( & blob) ?;
504+
505+ // For now, we'll assume the delete vector applies to all files
506+ // In a real implementation, we would need to determine which data files
507+ // this delete vector applies to based on the blob metadata properties
508+ if let Some ( data_file_path) = blob. properties ( ) . get ( "data-file-path" ) {
509+ delete_vectors. insert ( data_file_path. clone ( ) , delete_vector) ;
510+ }
511+ }
512+ }
513+
514+ Ok ( DeleteFileContext :: DelVecs ( delete_vectors) )
515+ }
318516
319- Err ( Error :: new (
320- ErrorKind :: FeatureUnsupported ,
321- "parsing of equality deletes is not yet supported" ,
322- ) )
517+ /// Parse a deletion vector blob from Puffin format into a DeleteVector
518+ fn parse_delete_vector_blob ( blob : & crate :: puffin:: Blob ) -> Result < DeleteVector > {
519+ use roaring:: RoaringTreemap ;
520+
521+ // According to the Iceberg spec, deletion vectors are stored as RoaringBitmap
522+ // in the "portable" format for 64-bit implementations
523+ let data = blob. data ( ) ;
524+
525+ // Parse the RoaringTreemap from the blob data
526+ let roaring_treemap = RoaringTreemap :: deserialize_from ( std:: io:: Cursor :: new ( data) )
527+ . map_err ( |e| Error :: new (
528+ ErrorKind :: DataInvalid ,
529+ format ! ( "Failed to deserialize deletion vector: {}" , e) ,
530+ ) ) ?;
531+
532+ Ok ( DeleteVector :: new ( roaring_treemap) )
323533 }
324534}
325535
0 commit comments