@@ -92,24 +92,35 @@ impl FileOpener for ParquetOpener {
92
92
// Prune using known statistics
93
93
match ( & file. statistics , & self . predicate ) {
94
94
( Some ( stats) , Some ( predicate) ) => {
95
- let pruning_predicate =
96
- build_pruning_predicate ( Arc :: clone ( predicate) , & self . table_schema , & predicate_creation_errors) ;
95
+ let pruning_predicate = build_pruning_predicate (
96
+ Arc :: clone ( predicate) ,
97
+ & self . table_schema ,
98
+ & predicate_creation_errors,
99
+ ) ;
97
100
if let Some ( pruning_predicate) = pruning_predicate {
98
101
let pruning_stats = PrunableStatistics :: new (
99
- Arc :: clone ( stats) ,
102
+ vec ! [ Arc :: clone( stats) ] ,
100
103
Arc :: clone ( & self . table_schema ) ,
101
104
) ;
102
105
match pruning_predicate. prune ( & pruning_stats) {
103
106
Ok ( values) => {
104
107
// We expect a single container -> if all containers are false skip this file
105
108
if values. into_iter ( ) . all ( |v| !v) {
106
109
// Return an empty stream
107
- todo ! ( )
110
+ return Ok ( Box :: pin ( async move {
111
+ Ok ( futures:: stream:: empty ( ) . boxed ( ) )
112
+ } ) ) ;
108
113
}
109
114
}
110
115
// stats filter array could not be built, so we can't prune
111
116
Err ( e) => {
112
- log:: debug!( "Error evaluating row group predicate values {e}" ) ;
117
+ let err = format ! (
118
+ "Error evaluating row group predicate values {e}"
119
+ ) ;
120
+ println ! ( "{err}" ) ;
121
+ log:: debug!(
122
+ "Error evaluating row group predicate values {e}"
123
+ ) ;
113
124
predicate_creation_errors. add ( 1 ) ;
114
125
}
115
126
}
@@ -469,3 +480,145 @@ async fn load_page_index<T: AsyncFileReader>(
469
480
Ok ( reader_metadata)
470
481
}
471
482
}
483
+
484
+ #[ cfg( test) ]
485
+ mod test {
486
+ use std:: sync:: Arc ;
487
+
488
+ use bytes:: { BufMut , BytesMut } ;
489
+ use chrono:: Utc ;
490
+ use datafusion_common:: {
491
+ record_batch, stats:: Precision , ColumnStatistics , ScalarValue , Statistics ,
492
+ } ;
493
+ use datafusion_datasource:: {
494
+ file_meta:: FileMeta , file_stream:: FileOpener ,
495
+ schema_adapter:: DefaultSchemaAdapterFactory , PartitionedFile ,
496
+ } ;
497
+ use datafusion_expr:: { col, lit} ;
498
+ use datafusion_physical_expr:: planner:: logical2physical;
499
+ use datafusion_physical_plan:: metrics:: ExecutionPlanMetricsSet ;
500
+ use futures:: { Stream , StreamExt } ;
501
+ use object_store:: { memory:: InMemory , path:: Path , ObjectMeta , ObjectStore } ;
502
+ use parquet:: arrow:: ArrowWriter ;
503
+
504
+ use crate :: { opener:: ParquetOpener , DefaultParquetFileReaderFactory } ;
505
+
506
+ async fn count_batches_and_rows (
507
+ mut stream : std:: pin:: Pin <
508
+ Box <
509
+ dyn Stream <
510
+ Item = Result <
511
+ arrow:: array:: RecordBatch ,
512
+ arrow:: error:: ArrowError ,
513
+ > ,
514
+ > + Send ,
515
+ > ,
516
+ > ,
517
+ ) -> ( usize , usize ) {
518
+ let mut num_batches = 0 ;
519
+ let mut num_rows = 0 ;
520
+ while let Some ( Ok ( batch) ) = stream. next ( ) . await {
521
+ num_rows += batch. num_rows ( ) ;
522
+ num_batches += 1 ;
523
+ }
524
+ ( num_batches, num_rows)
525
+ }
526
+
527
+ #[ tokio:: test]
528
+ async fn test_prune_based_on_statistics ( ) {
529
+ let batch = record_batch ! (
530
+ ( "a" , Int32 , vec![ Some ( 1 ) , Some ( 2 ) , Some ( 2 ) ] ) ,
531
+ ( "b" , Float32 , vec![ Some ( 1.0 ) , Some ( 2.0 ) , None ] )
532
+ )
533
+ . unwrap ( ) ;
534
+
535
+ let store = Arc :: new ( InMemory :: new ( ) ) as Arc < dyn ObjectStore > ;
536
+ let mut out = BytesMut :: new ( ) . writer ( ) ;
537
+ {
538
+ let mut writer =
539
+ ArrowWriter :: try_new ( & mut out, batch. schema ( ) , None ) . unwrap ( ) ;
540
+ writer. write ( & batch) . unwrap ( ) ;
541
+ writer. finish ( ) . unwrap ( ) ;
542
+ }
543
+ let data = out. into_inner ( ) . freeze ( ) ;
544
+ let data_size = data. len ( ) ;
545
+ store
546
+ . put ( & Path :: from ( "test.parquet" ) , data. into ( ) )
547
+ . await
548
+ . unwrap ( ) ;
549
+
550
+ let schema = batch. schema ( ) ;
551
+ let file = PartitionedFile :: new (
552
+ "file.parquet" . to_string ( ) ,
553
+ u64:: try_from ( data_size) . unwrap ( ) ,
554
+ )
555
+ . with_statistics ( Arc :: new (
556
+ Statistics :: new_unknown ( & schema)
557
+ . add_column_statistics ( ColumnStatistics :: new_unknown ( ) )
558
+ . add_column_statistics (
559
+ ColumnStatistics :: new_unknown ( )
560
+ . with_min_value ( Precision :: Exact ( ScalarValue :: Float32 ( Some ( 1.0 ) ) ) )
561
+ . with_max_value ( Precision :: Exact ( ScalarValue :: Float32 ( Some ( 2.0 ) ) ) )
562
+ . with_null_count ( Precision :: Exact ( 1 ) ) ,
563
+ ) ,
564
+ ) ) ;
565
+
566
+ let make_opener = |predicate| {
567
+ ParquetOpener {
568
+ partition_index : 0 ,
569
+ projection : Arc :: new ( [ 0 , 1 ] ) ,
570
+ batch_size : 1024 ,
571
+ limit : None ,
572
+ predicate : Some ( predicate) ,
573
+ table_schema : schema. clone ( ) ,
574
+ metadata_size_hint : None ,
575
+ metrics : ExecutionPlanMetricsSet :: new ( ) ,
576
+ parquet_file_reader_factory : Arc :: new (
577
+ DefaultParquetFileReaderFactory :: new ( Arc :: clone ( & store) ) ,
578
+ ) ,
579
+ pushdown_filters : false , // note that this is false!
580
+ reorder_filters : false ,
581
+ enable_page_index : false ,
582
+ enable_bloom_filter : false ,
583
+ schema_adapter_factory : Arc :: new ( DefaultSchemaAdapterFactory ) ,
584
+ enable_row_group_stats_pruning : true ,
585
+ coerce_int96 : None ,
586
+ }
587
+ } ;
588
+
589
+ let make_meta = || FileMeta {
590
+ object_meta : ObjectMeta {
591
+ location : Path :: from ( "test.parquet" ) ,
592
+ last_modified : Utc :: now ( ) ,
593
+ size : u64:: try_from ( data_size) . unwrap ( ) ,
594
+ e_tag : None ,
595
+ version : None ,
596
+ } ,
597
+ range : None ,
598
+ extensions : None ,
599
+ metadata_size_hint : None ,
600
+ } ;
601
+
602
+ // A filter on "a" should not exclude any rows even if it matches the data
603
+ let expr = col ( "a" ) . eq ( lit ( 1 ) ) ;
604
+ let predicate = logical2physical ( & expr, & schema) ;
605
+ let opener = make_opener ( predicate) ;
606
+ let stream = opener
607
+ . open ( make_meta ( ) , file. clone ( ) )
608
+ . unwrap ( )
609
+ . await
610
+ . unwrap ( ) ;
611
+ let ( num_batches, num_rows) = count_batches_and_rows ( stream) . await ;
612
+ assert_eq ! ( num_batches, 1 ) ;
613
+ assert_eq ! ( num_rows, 3 ) ;
614
+
615
+ // A filter on `b = 5.0` should exclude all rows
616
+ let expr = col ( "b" ) . eq ( lit ( ScalarValue :: Float32 ( Some ( 5.0 ) ) ) ) ;
617
+ let predicate = logical2physical ( & expr, & schema) ;
618
+ let opener = make_opener ( predicate) ;
619
+ let stream = opener. open ( make_meta ( ) , file) . unwrap ( ) . await . unwrap ( ) ;
620
+ let ( num_batches, num_rows) = count_batches_and_rows ( stream) . await ;
621
+ assert_eq ! ( num_batches, 0 ) ;
622
+ assert_eq ! ( num_rows, 0 ) ;
623
+ }
624
+ }
0 commit comments