@@ -25,6 +25,7 @@ use arrow::datatypes::Schema;
25
25
use arrow:: datatypes:: SchemaRef ;
26
26
use async_trait:: async_trait;
27
27
use futures:: TryStreamExt ;
28
+ use hashbrown:: HashMap ;
28
29
use parquet:: arrow:: ArrowReader ;
29
30
use parquet:: arrow:: ParquetFileArrowReader ;
30
31
use parquet:: errors:: ParquetError ;
@@ -46,7 +47,7 @@ use crate::error::Result;
46
47
use crate :: logical_plan:: combine_filters;
47
48
use crate :: logical_plan:: Expr ;
48
49
use crate :: physical_plan:: expressions:: { MaxAccumulator , MinAccumulator } ;
49
- use crate :: physical_plan:: file_format:: ParquetExec ;
50
+ use crate :: physical_plan:: file_format:: { ParquetExec , SchemaAdapter } ;
50
51
use crate :: physical_plan:: ExecutionPlan ;
51
52
use crate :: physical_plan:: { Accumulator , Statistics } ;
52
53
use datafusion_data_access:: object_store:: { ObjectReader , ObjectReaderStream } ;
@@ -99,8 +100,12 @@ impl FileFormat for ParquetFormat {
99
100
Ok ( Arc :: new ( merged_schema) )
100
101
}
101
102
102
- async fn infer_stats ( & self , reader : Arc < dyn ObjectReader > ) -> Result < Statistics > {
103
- let stats = fetch_statistics ( reader) ?;
103
+ async fn infer_stats (
104
+ & self ,
105
+ reader : Arc < dyn ObjectReader > ,
106
+ table_schema : SchemaRef ,
107
+ ) -> Result < Statistics > {
108
+ let stats = fetch_statistics ( reader, table_schema) ?;
104
109
Ok ( stats)
105
110
}
106
111
@@ -279,46 +284,65 @@ fn fetch_schema(object_reader: Arc<dyn ObjectReader>) -> Result<Schema> {
279
284
}
280
285
281
286
/// Read and parse the statistics of the Parquet file at location `path`
282
- fn fetch_statistics ( object_reader : Arc < dyn ObjectReader > ) -> Result < Statistics > {
287
+ fn fetch_statistics (
288
+ object_reader : Arc < dyn ObjectReader > ,
289
+ table_schema : SchemaRef ,
290
+ ) -> Result < Statistics > {
283
291
let obj_reader = ChunkObjectReader ( object_reader) ;
284
292
let file_reader = Arc :: new ( SerializedFileReader :: new ( obj_reader) ?) ;
285
293
let mut arrow_reader = ParquetFileArrowReader :: new ( file_reader) ;
286
- let schema = arrow_reader. get_schema ( ) ?;
287
- let num_fields = schema . fields ( ) . len ( ) ;
288
- let fields = schema . fields ( ) . to_vec ( ) ;
294
+ let file_schema = arrow_reader. get_schema ( ) ?;
295
+ let num_fields = table_schema . fields ( ) . len ( ) ;
296
+ let fields = table_schema . fields ( ) . to_vec ( ) ;
289
297
let meta_data = arrow_reader. get_metadata ( ) ;
290
298
291
299
let mut num_rows = 0 ;
292
300
let mut total_byte_size = 0 ;
293
301
let mut null_counts = vec ! [ 0 ; num_fields] ;
294
302
let mut has_statistics = false ;
295
303
296
- let ( mut max_values, mut min_values) = create_max_min_accs ( & schema) ;
304
+ let schema_adapter = SchemaAdapter :: new ( table_schema. clone ( ) ) ;
305
+
306
+ let ( mut max_values, mut min_values) = create_max_min_accs ( & table_schema) ;
297
307
298
308
for row_group_meta in meta_data. row_groups ( ) {
299
309
num_rows += row_group_meta. num_rows ( ) ;
300
310
total_byte_size += row_group_meta. total_byte_size ( ) ;
301
311
302
- let columns_null_counts = row_group_meta
303
- . columns ( )
304
- . iter ( )
305
- . flat_map ( |c| c. statistics ( ) . map ( |stats| stats. null_count ( ) ) ) ;
306
-
307
- for ( i, cnt) in columns_null_counts. enumerate ( ) {
308
- null_counts[ i] += cnt as usize
309
- }
312
+ let mut column_stats: HashMap < usize , ( u64 , & ParquetStatistics ) > = HashMap :: new ( ) ;
310
313
311
314
for ( i, column) in row_group_meta. columns ( ) . iter ( ) . enumerate ( ) {
312
315
if let Some ( stat) = column. statistics ( ) {
313
316
has_statistics = true ;
314
- summarize_min_max ( & mut max_values, & mut min_values, & fields, i, stat)
317
+ column_stats. insert ( i, ( stat. null_count ( ) , stat) ) ;
318
+ }
319
+ }
320
+
321
+ if has_statistics {
322
+ for ( table_idx, null_cnt) in null_counts. iter_mut ( ) . enumerate ( ) {
323
+ if let Some ( file_idx) =
324
+ schema_adapter. map_column_index ( table_idx, & file_schema)
325
+ {
326
+ if let Some ( ( null_count, stats) ) = column_stats. get ( & file_idx) {
327
+ * null_cnt += * null_count as usize ;
328
+ summarize_min_max (
329
+ & mut max_values,
330
+ & mut min_values,
331
+ & fields,
332
+ table_idx,
333
+ stats,
334
+ )
335
+ }
336
+ } else {
337
+ * null_cnt += num_rows as usize ;
338
+ }
315
339
}
316
340
}
317
341
}
318
342
319
343
let column_stats = if has_statistics {
320
344
Some ( get_col_stats (
321
- & schema ,
345
+ & table_schema ,
322
346
null_counts,
323
347
& mut max_values,
324
348
& mut min_values,
@@ -369,10 +393,102 @@ mod tests {
369
393
370
394
use crate :: prelude:: { SessionConfig , SessionContext } ;
371
395
use arrow:: array:: {
372
- BinaryArray , BooleanArray , Float32Array , Float64Array , Int32Array ,
373
- TimestampNanosecondArray ,
396
+ ArrayRef , BinaryArray , BooleanArray , Float32Array , Float64Array , Int32Array ,
397
+ StringArray , TimestampNanosecondArray ,
374
398
} ;
399
+ use arrow:: record_batch:: RecordBatch ;
400
+ use datafusion_common:: ScalarValue ;
375
401
use futures:: StreamExt ;
402
+ use parquet:: arrow:: ArrowWriter ;
403
+ use parquet:: file:: properties:: WriterProperties ;
404
+ use tempfile:: NamedTempFile ;
405
+
406
+ // Add a new column with the specified field name to the RecordBatch
407
+ fn add_to_batch (
408
+ batch : & RecordBatch ,
409
+ field_name : & str ,
410
+ array : ArrayRef ,
411
+ ) -> RecordBatch {
412
+ let mut fields = batch. schema ( ) . fields ( ) . clone ( ) ;
413
+ fields. push ( Field :: new ( field_name, array. data_type ( ) . clone ( ) , true ) ) ;
414
+ let schema = Arc :: new ( Schema :: new ( fields) ) ;
415
+
416
+ let mut columns = batch. columns ( ) . to_vec ( ) ;
417
+ columns. push ( array) ;
418
+ RecordBatch :: try_new ( schema, columns) . expect ( "error; creating record batch" )
419
+ }
420
+
421
+ fn create_batch ( columns : Vec < ( & str , ArrayRef ) > ) -> RecordBatch {
422
+ columns. into_iter ( ) . fold (
423
+ RecordBatch :: new_empty ( Arc :: new ( Schema :: new ( vec ! [ ] ) ) ) ,
424
+ |batch, ( field_name, arr) | add_to_batch ( & batch, field_name, arr. clone ( ) ) ,
425
+ )
426
+ }
427
+
428
+ async fn create_table (
429
+ batches : Vec < RecordBatch > ,
430
+ ) -> Result < ( Vec < NamedTempFile > , Schema ) > {
431
+ let merged_schema =
432
+ Schema :: try_merge ( batches. iter ( ) . map ( |b| b. schema ( ) . as_ref ( ) . clone ( ) ) ) ?;
433
+
434
+ let files: Vec < _ > = batches
435
+ . into_iter ( )
436
+ . map ( |batch| {
437
+ let output = tempfile:: NamedTempFile :: new ( ) . expect ( "creating temp file" ) ;
438
+
439
+ let props = WriterProperties :: builder ( ) . build ( ) ;
440
+ let file: std:: fs:: File = ( * output. as_file ( ) )
441
+ . try_clone ( )
442
+ . expect ( "cloning file descriptor" ) ;
443
+ let mut writer = ArrowWriter :: try_new ( file, batch. schema ( ) , Some ( props) )
444
+ . expect ( "creating writer" ) ;
445
+
446
+ writer. write ( & batch) . expect ( "Writing batch" ) ;
447
+ writer. close ( ) . unwrap ( ) ;
448
+ output
449
+ } )
450
+ . collect ( ) ;
451
+
452
+ Ok ( ( files, merged_schema) )
453
+ }
454
+
455
+ #[ tokio:: test]
456
+ async fn read_merged_batches ( ) -> Result < ( ) > {
457
+ let c1: ArrayRef =
458
+ Arc :: new ( StringArray :: from ( vec ! [ Some ( "Foo" ) , None , Some ( "bar" ) ] ) ) ;
459
+
460
+ let c2: ArrayRef = Arc :: new ( Int64Array :: from ( vec ! [ Some ( 1 ) , Some ( 2 ) , None ] ) ) ;
461
+
462
+ let batch1 = create_batch ( vec ! [ ( "c1" , c1. clone( ) ) ] ) ;
463
+
464
+ let batch2 = create_batch ( vec ! [ ( "c2" , c2) ] ) ;
465
+
466
+ let ( files, schema) = create_table ( vec ! [ batch1, batch2] ) . await ?;
467
+ let table_schema = Arc :: new ( schema) ;
468
+
469
+ let reader = local_object_reader ( files[ 0 ] . path ( ) . to_string_lossy ( ) . to_string ( ) ) ;
470
+
471
+ let stats = fetch_statistics ( reader, table_schema. clone ( ) ) ?;
472
+
473
+ assert_eq ! ( stats. num_rows, Some ( 3 ) ) ;
474
+ let c1_stats = & stats. column_statistics . as_ref ( ) . expect ( "missing c1 stats" ) [ 0 ] ;
475
+ let c2_stats = & stats. column_statistics . as_ref ( ) . expect ( "missing c2 stats" ) [ 1 ] ;
476
+ assert_eq ! ( c1_stats. null_count, Some ( 1 ) ) ;
477
+ assert_eq ! ( c2_stats. null_count, Some ( 3 ) ) ;
478
+
479
+ let reader = local_object_reader ( files[ 1 ] . path ( ) . to_string_lossy ( ) . to_string ( ) ) ;
480
+
481
+ let stats = fetch_statistics ( reader, table_schema) ?;
482
+ assert_eq ! ( stats. num_rows, Some ( 3 ) ) ;
483
+ let c1_stats = & stats. column_statistics . as_ref ( ) . expect ( "missing c1 stats" ) [ 0 ] ;
484
+ let c2_stats = & stats. column_statistics . as_ref ( ) . expect ( "missing c2 stats" ) [ 1 ] ;
485
+ assert_eq ! ( c1_stats. null_count, Some ( 3 ) ) ;
486
+ assert_eq ! ( c2_stats. null_count, Some ( 1 ) ) ;
487
+ assert_eq ! ( c2_stats. max_value, Some ( ScalarValue :: Int64 ( Some ( 2 ) ) ) ) ;
488
+ assert_eq ! ( c2_stats. min_value, Some ( ScalarValue :: Int64 ( Some ( 1 ) ) ) ) ;
489
+
490
+ Ok ( ( ) )
491
+ }
376
492
377
493
#[ tokio:: test]
378
494
async fn read_small_batches ( ) -> Result < ( ) > {
@@ -645,7 +761,7 @@ mod tests {
645
761
. await
646
762
. expect ( "Schema inference" ) ;
647
763
let statistics = format
648
- . infer_stats ( local_object_reader ( filename. clone ( ) ) )
764
+ . infer_stats ( local_object_reader ( filename. clone ( ) ) , file_schema . clone ( ) )
649
765
. await
650
766
. expect ( "Stats inference" ) ;
651
767
let file_groups = vec ! [ vec![ local_unpartitioned_file( filename. clone( ) ) ] ] ;
0 commit comments