@@ -33,7 +33,9 @@ use bytes::Bytes;
3333use fnv:: FnvHashSet ;
3434use futures:: future:: BoxFuture ;
3535use futures:: { try_join, FutureExt , StreamExt , TryFutureExt , TryStreamExt } ;
36- use parquet:: arrow:: arrow_reader:: { ArrowPredicateFn , ArrowReaderOptions , RowFilter , RowSelection } ;
36+ use parquet:: arrow:: arrow_reader:: {
37+ ArrowPredicateFn , ArrowReaderOptions , RowFilter , RowSelection , RowSelector ,
38+ } ;
3739use parquet:: arrow:: async_reader:: AsyncFileReader ;
3840use parquet:: arrow:: { ParquetRecordBatchStreamBuilder , ProjectionMask , PARQUET_FIELD_ID_META_KEY } ;
3941use parquet:: file:: metadata:: { ParquetMetaData , ParquetMetaDataReader , RowGroupMetaData } ;
@@ -341,15 +343,98 @@ impl ArrowReader {
341343 /// Using the Parquet page index, we build a `RowSelection` that rejects rows that are indicated
342344 /// as having been deleted by a positional delete, taking into account any row groups that have
343345 /// been skipped entirely by the filter predicate
344- #[ allow( unused) ]
345346 fn build_deletes_row_selection (
346347 row_group_metadata : & [ RowGroupMetaData ] ,
347348 selected_row_groups : & Option < Vec < usize > > ,
348349 mut positional_deletes : RoaringTreemap ,
349350 ) -> Result < RowSelection > {
350- // TODO
351+ let mut results: Vec < RowSelector > = Vec :: new ( ) ;
352+ let mut selected_row_groups_idx = 0 ;
353+ let mut current_page_base_idx: u64 = 0 ;
354+
355+ ' outer: for ( idx, row_group_metadata) in row_group_metadata. iter ( ) . enumerate ( ) {
356+ let page_num_rows = row_group_metadata. num_rows ( ) as u64 ;
357+ let next_page_base_idx = current_page_base_idx + page_num_rows;
358+
359+ // if row group selection is enabled,
360+ if let Some ( selected_row_groups) = selected_row_groups {
361+ // if we've consumed all the selected row groups, we're done
362+ if selected_row_groups_idx == selected_row_groups. len ( ) {
363+ break ;
364+ }
365+
366+ if idx == selected_row_groups[ selected_row_groups_idx] {
367+ // we're in a selected row group. Increment selected_row_groups_idx
368+ // so that next time around the for loop we're looking for the next
369+ // selected row group
370+ selected_row_groups_idx += 1 ;
371+ } else {
372+ // remove any positional deletes from the skipped page so that
373+ // `positional.deletes.min()` can be used
374+ positional_deletes. remove_range ( current_page_base_idx..next_page_base_idx) ;
375+
376+ // still increment the current page base index but then skip to the next row group
377+ // in the file
378+ current_page_base_idx += page_num_rows;
379+ continue ;
380+ }
381+ }
382+
383+ let mut next_deleted_row_idx = match positional_deletes. min ( ) {
384+ Some ( next_deleted_row_idx) => {
385+ // if the index of the next deleted row is beyond this page, skip to the next page
386+ if next_deleted_row_idx >= next_page_base_idx {
387+ continue ;
388+ }
389+
390+ next_deleted_row_idx
391+ }
392+
393+ // If there are no more pos deletes, stop building row selections and return what we
394+ // have. NB this depends on behavior of ParquetRecordBatchReader that appears to be
395+ // undocumented (not that I can find!): if there are no further RowSelections
396+ // then everything else is assumed to be selected.
397+ // see https://github.com/apache/arrow-rs/blob/c245a45efef9d9453f121587365e56d53c39d28f/parquet/src/arrow/arrow_reader/mod.rs#L703
398+ _ => break
399+ } ;
351400
352- Ok ( RowSelection :: default ( ) )
401+ let mut current_idx = current_page_base_idx;
402+ while next_deleted_row_idx < next_page_base_idx {
403+ // `select` all rows that precede the next delete index
404+ if current_idx < next_deleted_row_idx {
405+ let run_length = next_deleted_row_idx - current_idx;
406+ results. push ( RowSelector :: select ( run_length as usize ) ) ;
407+ current_idx += run_length;
408+ }
409+
410+ // `skip` all consecutive deleted rows
411+ let mut run_length = 0 ;
412+ while next_deleted_row_idx == current_idx {
413+ run_length += 1 ;
414+ current_idx += 1 ;
415+ positional_deletes. remove ( next_deleted_row_idx) ;
416+
417+ next_deleted_row_idx = match positional_deletes. min ( ) {
418+ Some ( next_deleted_row_idx) => next_deleted_row_idx,
419+ _ => {
420+ results. push ( RowSelector :: skip ( run_length) ) ;
421+ break ' outer;
422+ }
423+ } ;
424+ }
425+ results. push ( RowSelector :: skip ( run_length) ) ;
426+ }
427+
428+ if let Some ( selected_row_groups) = selected_row_groups {
429+ if selected_row_groups_idx == selected_row_groups. len ( ) {
430+ break ;
431+ }
432+ }
433+
434+ current_page_base_idx += page_num_rows;
435+ }
436+
437+ Ok ( results. into ( ) )
353438 }
354439
355440 fn build_field_id_set_and_map (
@@ -1255,10 +1340,12 @@ mod tests {
12551340 use std:: sync:: Arc ;
12561341
12571342 use arrow_schema:: { DataType , Field , Schema as ArrowSchema , TimeUnit } ;
1343+ use parquet:: arrow:: arrow_reader:: { RowSelection , RowSelector } ;
12581344 use parquet:: arrow:: ProjectionMask ;
1345+ use parquet:: file:: metadata:: { ColumnChunkMetaData , RowGroupMetaData } ;
12591346 use parquet:: schema:: parser:: parse_message_type;
1260- use parquet:: schema:: types:: SchemaDescriptor ;
1261-
1347+ use parquet:: schema:: types:: { SchemaDescPtr , SchemaDescriptor } ;
1348+ use roaring :: RoaringTreemap ;
12621349 use crate :: arrow:: reader:: { CollectFieldIdVisitor , PARQUET_FIELD_ID_META_KEY } ;
12631350 use crate :: arrow:: ArrowReader ;
12641351 use crate :: expr:: visitors:: bound_predicate_visitor:: visit;
@@ -1423,4 +1510,99 @@ message schema {
14231510 . expect ( "Some ProjectionMask" ) ;
14241511 assert_eq ! ( mask, ProjectionMask :: leaves( & parquet_schema, vec![ 0 ] ) ) ;
14251512 }
1513+
1514+ #[ test]
1515+ fn test_build_deletes_row_selection ( ) {
1516+ let schema_descr = get_test_schema_descr ( ) ;
1517+
1518+ let mut columns = vec ! [ ] ;
1519+ for ptr in schema_descr. columns ( ) {
1520+ let column = ColumnChunkMetaData :: builder ( ptr. clone ( ) ) . build ( ) . unwrap ( ) ;
1521+ columns. push ( column) ;
1522+ }
1523+
1524+ let row_groups_metadata = vec ! [
1525+ build_test_row_group_meta( schema_descr. clone( ) , columns. clone( ) , 1000 , 0 ) ,
1526+ build_test_row_group_meta( schema_descr. clone( ) , columns. clone( ) , 500 , 1 ) ,
1527+ build_test_row_group_meta( schema_descr. clone( ) , columns. clone( ) , 500 , 2 ) ,
1528+ build_test_row_group_meta( schema_descr. clone( ) , columns. clone( ) , 1000 , 3 ) ,
1529+ build_test_row_group_meta( schema_descr. clone( ) , columns. clone( ) , 500 , 4 ) ,
1530+ ] ;
1531+
1532+ let selected_row_groups = Some ( vec ! [ 1 , 3 ] ) ;
1533+
1534+ let positional_deletes = RoaringTreemap :: from_iter ( & [
1535+ 1 , // in skipped row group 0, should be ignored
1536+ 3 , // in skipped row group 0, should be ignored
1537+ 4 , // in skipped row group 0, should be ignored
1538+ 5 , // in skipped row group 0, should be ignored
1539+
1540+ 1002 , // solitary row in selected row group 1
1541+
1542+ 1010 , // run of 5 rows in selected row group 1
1543+ 1011 ,
1544+ 1012 ,
1545+ 1013 ,
1546+ 1014 ,
1547+
1548+ 1600 , // should ignore, in skipped row group 2
1549+
1550+ 2100 , // single row in selected row group 3,
1551+
1552+ 2200 , // run of 3 consecutive rows in selected row group 3
1553+ 2201 ,
1554+ 2202
1555+ ] ) ;
1556+
1557+ let result = ArrowReader :: build_deletes_row_selection (
1558+ & row_groups_metadata,
1559+ & selected_row_groups,
1560+ positional_deletes
1561+ ) . unwrap ( ) ;
1562+
1563+ let expected = RowSelection :: from ( vec ! [
1564+ RowSelector :: select( 2 ) ,
1565+ RowSelector :: skip( 1 ) ,
1566+ RowSelector :: select( 7 ) ,
1567+ RowSelector :: skip( 5 ) ,
1568+ RowSelector :: select( 100 ) ,
1569+ RowSelector :: skip( 1 ) ,
1570+ RowSelector :: select( 99 ) ,
1571+ RowSelector :: skip( 3 )
1572+ ] ) ;
1573+
1574+ assert_eq ! ( result, expected) ;
1575+ }
1576+
1577+ fn build_test_row_group_meta ( schema_descr : SchemaDescPtr , columns : Vec < ColumnChunkMetaData > , num_rows : i64 , ordinal : i16 ) -> RowGroupMetaData {
1578+ RowGroupMetaData :: builder ( schema_descr. clone ( ) )
1579+ . set_num_rows ( num_rows)
1580+ . set_total_byte_size ( 2000 )
1581+ . set_column_metadata ( columns)
1582+ . set_ordinal ( ordinal)
1583+ . build ( )
1584+ . unwrap ( )
1585+ }
1586+
1587+ fn get_test_schema_descr ( ) -> SchemaDescPtr {
1588+ use parquet:: schema:: types:: Type as SchemaType ;
1589+
1590+ let schema = SchemaType :: group_type_builder ( "schema" )
1591+ . with_fields ( vec ! [
1592+ Arc :: new(
1593+ SchemaType :: primitive_type_builder( "a" , parquet:: basic:: Type :: INT32 )
1594+ . build( )
1595+ . unwrap( ) ,
1596+ ) ,
1597+ Arc :: new(
1598+ SchemaType :: primitive_type_builder( "b" , parquet:: basic:: Type :: INT32 )
1599+ . build( )
1600+ . unwrap( ) ,
1601+ ) ,
1602+ ] )
1603+ . build ( )
1604+ . unwrap ( ) ;
1605+
1606+ Arc :: new ( SchemaDescriptor :: new ( Arc :: new ( schema) ) )
1607+ }
14261608}
0 commit comments