@@ -21,6 +21,7 @@ use std::pin::Pin;
21
21
use std:: sync:: { Arc , OnceLock , RwLock } ;
22
22
use std:: task:: { Context , Poll } ;
23
23
24
+ use arrow_array:: { Int64Array , StringArray } ;
24
25
use futures:: channel:: oneshot;
25
26
use futures:: future:: join_all;
26
27
use futures:: { StreamExt , TryStreamExt } ;
@@ -169,7 +170,7 @@ impl CachingDeleteFileManager {
169
170
/// vector maps that resulted from any positional delete or delete vector files into a
170
171
/// single map and persist it in the state.
171
172
///
172
- ///
173
+ ///
173
174
/// Conceptually, the data flow is like this:
174
175
/// ```none
175
176
/// FileScanTaskDeleteFile
@@ -393,14 +394,44 @@ impl CachingDeleteFileManager {
393
394
///
394
395
/// Returns a map of data file path to a delete vector
395
396
async fn parse_positional_deletes_record_batch_stream (
396
- stream : ArrowRecordBatchStream ,
397
+ mut stream : ArrowRecordBatchStream ,
397
398
) -> Result < HashMap < String , DeleteVector > > {
398
- // TODO
399
+ let mut result: HashMap < String , DeleteVector > = HashMap :: default ( ) ;
400
+
401
+ while let Some ( batch) = stream. next ( ) . await {
402
+ let batch = batch?;
403
+ let schema = batch. schema ( ) ;
404
+ let columns = batch. columns ( ) ;
405
+
406
+ let Some ( file_paths) = columns[ 0 ] . as_any ( ) . downcast_ref :: < StringArray > ( ) else {
407
+ return Err ( Error :: new (
408
+ ErrorKind :: DataInvalid ,
409
+ "Could not downcast file paths array to StringArray" ,
410
+ ) ) ;
411
+ } ;
412
+ let Some ( positions) = columns[ 1 ] . as_any ( ) . downcast_ref :: < Int64Array > ( ) else {
413
+ return Err ( Error :: new (
414
+ ErrorKind :: DataInvalid ,
415
+ "Could not downcast positions array to Int64Array" ,
416
+ ) ) ;
417
+ } ;
399
418
400
- Err ( Error :: new (
401
- ErrorKind :: FeatureUnsupported ,
402
- "parsing of positional deletes is not yet supported" ,
403
- ) )
419
+ for ( file_path, pos) in file_paths. iter ( ) . zip ( positions. iter ( ) ) {
420
+ let ( Some ( file_path) , Some ( pos) ) = ( file_path, pos) else {
421
+ return Err ( Error :: new (
422
+ ErrorKind :: DataInvalid ,
423
+ "null values in delete file" ,
424
+ ) ) ;
425
+ } ;
426
+
427
+ result
428
+ . entry ( file_path. to_string ( ) )
429
+ . or_default ( )
430
+ . insert ( pos as u64 ) ;
431
+ }
432
+ }
433
+
434
+ Ok ( result)
404
435
}
405
436
406
437
/// Parses record batch streams from individual equality delete files
@@ -514,38 +545,74 @@ mod tests {
514
545
. build ( )
515
546
. unwrap ( ) ;
516
547
517
- // Note that with the delete file parsing not yet in place, all we can test here is that
518
- // the call to the loader fails with the expected FeatureUnsupportedError.
519
548
let delete_file_manager = CachingDeleteFileManager :: new ( file_io. clone ( ) , 10 ) ;
520
549
521
550
let file_scan_tasks = setup ( table_location) ;
522
551
523
- let result = delete_file_manager
552
+ let _ = delete_file_manager
524
553
. load_deletes ( & file_scan_tasks[ 0 ] . deletes , file_scan_tasks[ 0 ] . schema_ref ( ) )
525
554
. await ;
526
555
527
- assert ! ( result. is_err_and( |e| e. kind( ) == ErrorKind :: FeatureUnsupported ) ) ;
556
+ let result = delete_file_manager
557
+ . get_delete_vector_for_task ( & file_scan_tasks[ 0 ] )
558
+ . unwrap ( ) ;
559
+ assert_eq ! ( result. read( ) . unwrap( ) . len( ) , 3 ) ; // pos dels from pos del file 1 and 2
560
+
561
+ let result = delete_file_manager
562
+ . get_delete_vector_for_task ( & file_scan_tasks[ 1 ] )
563
+ . unwrap ( ) ;
564
+ assert_eq ! ( result. read( ) . unwrap( ) . len( ) , 3 ) ; // pos dels from pos del file 3
528
565
}
529
566
530
567
fn setup ( table_location : & Path ) -> Vec < FileScanTask > {
531
568
let data_file_schema = Arc :: new ( Schema :: builder ( ) . build ( ) . unwrap ( ) ) ;
532
569
let positional_delete_schema = create_pos_del_schema ( ) ;
533
570
534
- let file_path_values = vec ! [ format!( "{}/1.parquet" , table_location. to_str( ) . unwrap( ) ) ; 8 ] ;
535
- let pos_values = vec ! [ 0 , 1 , 3 , 5 , 6 , 8 , 1022 , 1023 ] ;
536
-
537
- let file_path_col = Arc :: new ( StringArray :: from_iter_values ( file_path_values) ) ;
538
- let pos_col = Arc :: new ( Int64Array :: from_iter_values ( pos_values) ) ;
571
+ let mut file_path_values = vec ! [ ] ;
572
+ let mut pos_values = vec ! [ ] ;
573
+
574
+ file_path_values. push ( vec ! [
575
+ format!(
576
+ "{}/1.parquet" ,
577
+ table_location. to_str( ) . unwrap( )
578
+ ) ;
579
+ 3
580
+ ] ) ;
581
+ pos_values. push ( vec ! [ 0 , 1 , 3 ] ) ;
582
+
583
+ file_path_values. push ( vec ! [
584
+ format!(
585
+ "{}/1.parquet" ,
586
+ table_location. to_str( ) . unwrap( )
587
+ ) ;
588
+ 3
589
+ ] ) ;
590
+ pos_values. push ( vec ! [ 5 , 6 , 8 ] ) ;
591
+
592
+ file_path_values. push ( vec ! [
593
+ format!(
594
+ "{}/2.parquet" ,
595
+ table_location. to_str( ) . unwrap( )
596
+ ) ;
597
+ 3
598
+ ] ) ;
599
+ pos_values. push ( vec ! [ 1022 , 1023 , 1024 ] ) ;
600
+ // 9 rows in total pos deleted across 3 files
539
601
540
602
let props = WriterProperties :: builder ( )
541
603
. set_compression ( Compression :: SNAPPY )
542
604
. build ( ) ;
543
605
544
606
for n in 1 ..=3 {
607
+ let file_path_col = Arc :: new ( StringArray :: from_iter_values (
608
+ file_path_values. pop ( ) . unwrap ( ) ,
609
+ ) ) ;
610
+ let pos_col = Arc :: new ( Int64Array :: from_iter_values ( pos_values. pop ( ) . unwrap ( ) ) ) ;
611
+
545
612
let positional_deletes_to_write =
546
613
RecordBatch :: try_new ( positional_delete_schema. clone ( ) , vec ! [
547
614
file_path_col. clone( ) ,
548
- pos_col. clone ( ) ,
615
+ pos_col,
549
616
] )
550
617
. unwrap ( ) ;
551
618
@@ -596,7 +663,7 @@ mod tests {
596
663
start: 0 ,
597
664
length: 0 ,
598
665
record_count: None ,
599
- data_file_path: "" . to_string ( ) ,
666
+ data_file_path: format! ( "{}/1.parquet" , table_location . to_str ( ) . unwrap ( ) ) ,
600
667
data_file_content: DataContentType :: Data ,
601
668
data_file_format: DataFileFormat :: Parquet ,
602
669
schema: data_file_schema. clone( ) ,
@@ -608,13 +675,13 @@ mod tests {
608
675
start: 0 ,
609
676
length: 0 ,
610
677
record_count: None ,
611
- data_file_path: "" . to_string ( ) ,
678
+ data_file_path: format! ( "{}/2.parquet" , table_location . to_str ( ) . unwrap ( ) ) ,
612
679
data_file_content: DataContentType :: Data ,
613
680
data_file_format: DataFileFormat :: Parquet ,
614
681
schema: data_file_schema. clone( ) ,
615
682
project_field_ids: vec![ ] ,
616
683
predicate: None ,
617
- deletes: vec![ pos_del_2 , pos_del_3] ,
684
+ deletes: vec![ pos_del_3] ,
618
685
} ,
619
686
] ;
620
687
0 commit comments