@@ -75,6 +75,8 @@ pub struct ParquetExec {
75
75
projected_schema : SchemaRef ,
76
76
/// Execution metrics
77
77
metrics : ExecutionPlanMetricsSet ,
78
+ /// Optional predicate for row filtering during parquet scan
79
+ predicate : Option < Expr > ,
78
80
/// Optional predicate for pruning row groups
79
81
pruning_predicate : Option < PruningPredicate > ,
80
82
/// Optional hint for the size of the parquet metadata
@@ -98,6 +100,7 @@ impl ParquetExec {
98
100
MetricBuilder :: new ( & metrics) . global_counter ( "num_predicate_creation_errors" ) ;
99
101
100
102
let pruning_predicate = predicate
103
+ . clone ( )
101
104
. and_then ( |predicate_expr| {
102
105
match PruningPredicate :: try_new (
103
106
predicate_expr,
@@ -127,6 +130,7 @@ impl ParquetExec {
127
130
projected_schema,
128
131
projected_statistics,
129
132
metrics,
133
+ predicate,
130
134
pruning_predicate,
131
135
metadata_size_hint,
132
136
parquet_file_reader_factory : None ,
@@ -284,6 +288,7 @@ impl ExecutionPlan for ParquetExec {
284
288
partition_index,
285
289
projection : Arc :: from ( projection) ,
286
290
batch_size : ctx. session_config ( ) . batch_size ( ) ,
291
+ predicate : self . predicate . clone ( ) ,
287
292
pruning_predicate : self . pruning_predicate . clone ( ) ,
288
293
table_schema : self . base_config . file_schema . clone ( ) ,
289
294
metadata_size_hint : self . metadata_size_hint ,
@@ -312,6 +317,12 @@ impl ExecutionPlan for ParquetExec {
312
317
) -> std:: fmt:: Result {
313
318
match t {
314
319
DisplayFormatType :: Default => {
320
+ let predicate_string = self
321
+ . predicate
322
+ . as_ref ( )
323
+ . map ( |p| format ! ( ", predicate={}" , p) )
324
+ . unwrap_or_default ( ) ;
325
+
315
326
let pruning_predicate_string = self
316
327
. pruning_predicate
317
328
. as_ref ( )
@@ -325,9 +336,10 @@ impl ExecutionPlan for ParquetExec {
325
336
326
337
write ! (
327
338
f,
328
- "ParquetExec: limit={:?}, partitions={}{}{}, projection={}" ,
339
+ "ParquetExec: limit={:?}, partitions={}{}{}{} , projection={}" ,
329
340
self . base_config. limit,
330
341
super :: FileGroupsDisplay ( & self . base_config. file_groups) ,
342
+ predicate_string,
331
343
pruning_predicate_string,
332
344
output_ordering_string,
333
345
super :: ProjectSchemaDisplay ( & self . projected_schema) ,
@@ -364,6 +376,7 @@ struct ParquetOpener {
364
376
partition_index : usize ,
365
377
projection : Arc < [ usize ] > ,
366
378
batch_size : usize ,
379
+ predicate : Option < Expr > ,
367
380
pruning_predicate : Option < PruningPredicate > ,
368
381
table_schema : SchemaRef ,
369
382
metadata_size_hint : Option < usize > ,
@@ -399,6 +412,7 @@ impl FileOpener for ParquetOpener {
399
412
let schema_adapter = SchemaAdapter :: new ( self . table_schema . clone ( ) ) ;
400
413
let batch_size = self . batch_size ;
401
414
let projection = self . projection . clone ( ) ;
415
+ let predicate = self . predicate . clone ( ) ;
402
416
let pruning_predicate = self . pruning_predicate . clone ( ) ;
403
417
let table_schema = self . table_schema . clone ( ) ;
404
418
let reorder_predicates = self . reorder_filters ;
@@ -418,11 +432,8 @@ impl FileOpener for ParquetOpener {
418
432
adapted_projections. iter ( ) . cloned ( ) ,
419
433
) ;
420
434
421
- // Filter pushdown: evlauate predicates during scan
422
- if let Some ( predicate) = pushdown_filters
423
- . then ( || pruning_predicate. as_ref ( ) . map ( |p| p. logical_expr ( ) ) )
424
- . flatten ( )
425
- {
435
+ // Filter pushdown: evaluate predicates during scan
436
+ if let Some ( predicate) = pushdown_filters. then_some ( predicate) . flatten ( ) {
426
437
let row_filter = row_filter:: build_row_filter (
427
438
predicate. clone ( ) ,
428
439
builder. schema ( ) . as_ref ( ) ,
@@ -1564,6 +1575,9 @@ mod tests {
1564
1575
& display,
1565
1576
"pruning_predicate=c1_min@0 != bar OR bar != c1_max@1"
1566
1577
) ;
1578
+
1579
+ assert_contains ! ( & display, r#"predicate=c1 != Utf8("bar")"# ) ;
1580
+
1567
1581
assert_contains ! ( & display, "projection=[c1]" ) ;
1568
1582
}
1569
1583
@@ -1587,15 +1601,20 @@ mod tests {
1587
1601
. otherwise ( lit ( false ) )
1588
1602
. unwrap ( ) ;
1589
1603
1590
- let rt = round_trip ( vec ! [ batch1] , None , None , Some ( filter) , true , false ) . await ;
1604
+ let rt =
1605
+ round_trip ( vec ! [ batch1] , None , None , Some ( filter. clone ( ) ) , true , false ) . await ;
1591
1606
1592
1607
// Should not contain a pruning predicate
1593
1608
let pruning_predicate = & rt. parquet_exec . pruning_predicate ;
1594
1609
assert ! (
1595
1610
pruning_predicate. is_none( ) ,
1596
1611
"Still had pruning predicate: {:?}" ,
1597
1612
pruning_predicate
1598
- )
1613
+ ) ;
1614
+
1615
+ // but does still has a pushdown down predicate
1616
+ let predicate = rt. parquet_exec . predicate . as_ref ( ) ;
1617
+ assert_eq ! ( predicate, Some ( & filter) ) ;
1599
1618
}
1600
1619
1601
1620
/// returns the sum of all the metrics with the specified name
0 commit comments