1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
use std::collections::{HashMap, HashSet};
use std::sync::Arc;
use super::{ParquetAccessPlan, ParquetFileMetrics};
use arrow::array::{ArrayRef, BooleanArray};
use arrow::datatypes::Schema;
use datafusion_common::pruning::PruningStatistics;
use datafusion_common::{Column, Result, ScalarValue};
use datafusion_datasource::FileRange;
use datafusion_physical_expr::PhysicalExprSimplifier;
use datafusion_physical_expr::expressions::NotExpr;
use datafusion_pruning::PruningPredicate;
use parquet::arrow::arrow_reader::statistics::StatisticsConverter;
use parquet::arrow::parquet_column;
use parquet::basic::Type;
use parquet::data_type::Decimal;
use parquet::schema::types::SchemaDescriptor;
use parquet::{
arrow::{ParquetRecordBatchStreamBuilder, async_reader::AsyncFileReader},
bloom_filter::Sbbf,
file::metadata::RowGroupMetaData,
};
/// Reduces the [`ParquetAccessPlan`] based on row group level metadata.
///
/// This struct implements the various types of pruning that are applied to a
/// set of row groups within a parquet file, progressively narrowing down the
/// set of row groups (and ranges/selections within those row groups) that
/// should be scanned, based on the available metadata.
#[derive(Debug, Clone, PartialEq)]
pub struct RowGroupAccessPlanFilter {
/// which row groups should be accessed
access_plan: ParquetAccessPlan,
/// Row groups where ALL rows are known to match the pruning predicate
/// (the predicate does not filter any rows)
is_fully_matched: Vec<bool>,
}
impl RowGroupAccessPlanFilter {
/// Create a new `RowGroupPlanBuilder` for pruning out the groups to scan
/// based on metadata and statistics
pub fn new(access_plan: ParquetAccessPlan) -> Self {
let num_row_groups = access_plan.len();
Self {
access_plan,
is_fully_matched: vec![false; num_row_groups],
}
}
/// Return true if there are no row groups
pub fn is_empty(&self) -> bool {
self.access_plan.is_empty()
}
/// Return the number of row groups that are currently expected to be scanned
pub fn remaining_row_group_count(&self) -> usize {
self.access_plan.row_group_index_iter().count()
}
/// Returns the inner access plan
pub fn build(self) -> ParquetAccessPlan {
self.access_plan
}
/// Returns the is_fully_matched vector
pub fn is_fully_matched(&self) -> &Vec<bool> {
&self.is_fully_matched
}
/// Prunes the access plan based on the limit and fully contained row groups.
///
/// The pruning works by leveraging the concept of fully matched row groups. Consider a query like:
/// `WHERE species LIKE 'Alpine%' AND s >= 50 LIMIT N`
///
/// After initial filtering, row groups can be classified into three states:
///
/// 1. Not Matching / Pruned
/// 2. Partially Matching (Row Group/Page contains some matches)
/// 3. Fully Matching (Entire range is within predicate)
///
/// +-----------------------------------------------------------------------+
/// | NOT MATCHING |
/// | Row group 1 |
/// | +-----------------------------------+-----------------------------+ |
/// | | SPECIES | S | |
/// | +-----------------------------------+-----------------------------+ |
/// | | Snow Vole | 7 | |
/// | | Brown Bear | 133 ✅ | |
/// | | Gray Wolf | 82 ✅ | |
/// | +-----------------------------------+-----------------------------+ |
/// +-----------------------------------------------------------------------+
///
/// +---------------------------------------------------------------------------+
/// | PARTIALLY MATCHING |
/// | |
/// | Row group 2 Row group 4 |
/// | +------------------+--------------+ +------------------+----------+ |
/// | | SPECIES | S | | SPECIES | S | |
/// | +------------------+--------------+ +------------------+----------+ |
/// | | Lynx | 71 ✅ | | Europ. Mole | 4 | |
/// | | Red Fox | 40 | | Polecat | 16 | |
/// | | Alpine Bat ✅ | 6 | | Alpine Ibex ✅ | 97 ✅ | |
/// | +------------------+--------------+ +------------------+----------+ |
/// +---------------------------------------------------------------------------+
///
/// +-----------------------------------------------------------------------+
/// | FULLY MATCHING |
/// | Row group 3 |
/// | +-----------------------------------+-----------------------------+ |
/// | | SPECIES | S | |
/// | +-----------------------------------+-----------------------------+ |
/// | | Alpine Ibex ✅ | 101 ✅ | |
/// | | Alpine Goat ✅ | 76 ✅ | |
/// | | Alpine Sheep ✅ | 83 ✅ | |
/// | +-----------------------------------+-----------------------------+ |
/// +-----------------------------------------------------------------------+
///
/// ### Identification of Fully Matching Row Groups
///
/// DataFusion identifies row groups where ALL rows satisfy the filter by inverting the
/// predicate and checking if statistics prove the inverted version is false for the group.
///
/// For example, prefix matches like `species LIKE 'Alpine%'` are pruned using ranges:
/// 1. Candidate Range: `species >= 'Alpine' AND species < 'Alpinf'`
/// 2. Inverted Condition (to prove full match): `species < 'Alpine' OR species >= 'Alpinf'`
/// 3. Statistical Evaluation (check if any row *could* satisfy the inverted condition):
/// `min < 'Alpine' OR max >= 'Alpinf'`
///
/// If this evaluation is **false**, it proves no row can fail the original filter,
/// so the row group is **FULLY MATCHING**.
///
/// ### Impact of Statistics Truncation
///
/// The precision of pruning depends on the metadata quality. Truncated statistics
/// may prevent the system from proving a full match.
///
/// **Example**: `WHERE species LIKE 'Alpine%'` (Target range: `['Alpine', 'Alpinf')`)
///
/// | Truncation Length | min / max | Inverted Evaluation | Status |
/// |-------------------|---------------------|---------------------------------------------------------------------|------------------------|
/// | **Length 6** | `Alpine` / `Alpine` | `"Alpine" < "Alpine" (F) OR "Alpine" >= "Alpinf" (F)` -> **false** | **FULLY MATCHING** |
/// | **Length 3** | `Alp` / `Alq` | `"Alp" < "Alpine" (T) OR "Alq" >= "Alpinf" (T)` -> **true** | **PARTIALLY MATCHING** |
///
/// Even though Row Group 3 only contains matching rows, truncation to length 3 makes
/// the statistics `[Alp, Alq]` too broad to prove it (they could include "Alpha").
/// The system must conservatively scan the group.
///
/// Without limit pruning: Scan Partition 2 → Partition 3 → Partition 4 (until limit reached)
/// With limit pruning: If Partition 3 contains enough rows to satisfy the limit,
/// skip Partitions 2 and 4 entirely and go directly to Partition 3.
///
/// This optimization is particularly effective when:
/// - The limit is small relative to the total dataset size
/// - There are row groups that are fully matched by the filter predicates
/// - The fully matched row groups contain sufficient rows to satisfy the limit
///
/// For more information, see the [paper](https://arxiv.org/pdf/2504.11540)'s "Pruning for LIMIT Queries" part
pub fn prune_by_limit(
&mut self,
limit: usize,
rg_metadata: &[RowGroupMetaData],
metrics: &ParquetFileMetrics,
) {
let mut fully_matched_row_group_indexes: Vec<usize> = Vec::new();
let mut fully_matched_rows_count: usize = 0;
// Iterate through the currently accessible row groups and try to
// find a set of matching row groups that can satisfy the limit
for &idx in self.access_plan.row_group_indexes().iter() {
if self.is_fully_matched[idx] {
let row_group_row_count = rg_metadata[idx].num_rows() as usize;
fully_matched_row_group_indexes.push(idx);
fully_matched_rows_count += row_group_row_count;
if fully_matched_rows_count >= limit {
break;
}
}
}
// If we can satisfy the limit with fully matching row groups,
// rewrite the plan to do so
if fully_matched_rows_count >= limit {
let original_num_accessible_row_groups =
self.access_plan.row_group_indexes().len();
let new_num_accessible_row_groups = fully_matched_row_group_indexes.len();
let pruned_count = original_num_accessible_row_groups
.saturating_sub(new_num_accessible_row_groups);
metrics.limit_pruned_row_groups.add_pruned(pruned_count);
let mut new_access_plan = ParquetAccessPlan::new_none(rg_metadata.len());
for &idx in &fully_matched_row_group_indexes {
new_access_plan.scan(idx);
}
self.access_plan = new_access_plan;
}
}
/// Prune remaining row groups to only those within the specified range.
///
/// Updates this set to mark row groups that should not be scanned
///
/// # Panics
/// if `groups.len() != self.len()`
pub fn prune_by_range(&mut self, groups: &[RowGroupMetaData], range: &FileRange) {
assert_eq!(groups.len(), self.access_plan.len());
for (idx, metadata) in groups.iter().enumerate() {
if !self.access_plan.should_scan(idx) {
continue;
}
// Skip the row group if the first dictionary/data page are not
// within the range.
//
// note don't use the location of metadata
// <https://github.com/apache/datafusion/issues/5995>
let col = metadata.column(0);
let offset = col
.dictionary_page_offset()
.unwrap_or_else(|| col.data_page_offset());
if !range.contains(offset) {
self.access_plan.skip(idx);
}
}
}
/// Prune remaining row groups using min/max/null_count statistics and
/// the [`PruningPredicate`] to determine if the predicate can not be true.
///
/// Updates this set to mark row groups that should not be scanned
///
/// Note: This method currently ignores ColumnOrder
/// <https://github.com/apache/datafusion/issues/8335>
///
/// # Panics
/// if `groups.len() != self.len()`
pub fn prune_by_statistics(
&mut self,
arrow_schema: &Schema,
parquet_schema: &SchemaDescriptor,
groups: &[RowGroupMetaData],
predicate: &PruningPredicate,
metrics: &ParquetFileMetrics,
) {
// scoped timer updates on drop
let _timer_guard = metrics.statistics_eval_time.timer();
assert_eq!(groups.len(), self.access_plan.len());
// Indexes of row groups still to scan
let row_group_indexes = self.access_plan.row_group_indexes();
let row_group_metadatas = row_group_indexes
.iter()
.map(|&i| &groups[i])
.collect::<Vec<_>>();
let pruning_stats = RowGroupPruningStatistics {
parquet_schema,
row_group_metadatas,
arrow_schema,
};
// try to prune the row groups in a single call
match predicate.prune(&pruning_stats) {
Ok(values) => {
let mut fully_contained_candidates_original_idx: Vec<usize> = Vec::new();
for (idx, &value) in row_group_indexes.iter().zip(values.iter()) {
if !value {
self.access_plan.skip(*idx);
metrics.row_groups_pruned_statistics.add_pruned(1);
} else {
metrics.row_groups_pruned_statistics.add_matched(1);
fully_contained_candidates_original_idx.push(*idx);
}
}
// Check if any of the matched row groups are fully contained by the predicate
self.identify_fully_matched_row_groups(
&fully_contained_candidates_original_idx,
arrow_schema,
parquet_schema,
groups,
predicate,
metrics,
);
}
// stats filter array could not be built, so we can't prune
Err(e) => {
log::debug!("Error evaluating row group predicate values {e}");
metrics.predicate_evaluation_errors.add(1);
}
}
}
/// Identifies row groups that are fully matched by the predicate.
///
/// This optimization checks whether all rows in a row group satisfy the predicate
/// by inverting the predicate and checking if it prunes the row group. If the
/// inverted predicate prunes a row group, it means no rows match the inverted
/// predicate, which implies all rows match the original predicate.
///
/// Note: This optimization is relatively inexpensive for a limited number of row groups.
fn identify_fully_matched_row_groups(
&mut self,
candidate_row_group_indices: &[usize],
arrow_schema: &Schema,
parquet_schema: &SchemaDescriptor,
groups: &[RowGroupMetaData],
predicate: &PruningPredicate,
metrics: &ParquetFileMetrics,
) {
if candidate_row_group_indices.is_empty() {
return;
}
// Use NotExpr to create the inverted predicate
let inverted_expr = Arc::new(NotExpr::new(Arc::clone(predicate.orig_expr())));
// Simplify the NOT expression (e.g., NOT(c1 = 0) -> c1 != 0)
// before building the pruning predicate
let simplifier = PhysicalExprSimplifier::new(arrow_schema);
let Ok(inverted_expr) = simplifier.simplify(inverted_expr) else {
return;
};
let Ok(inverted_predicate) =
PruningPredicate::try_new(inverted_expr, Arc::clone(predicate.schema()))
else {
return;
};
let inverted_pruning_stats = RowGroupPruningStatistics {
parquet_schema,
row_group_metadatas: candidate_row_group_indices
.iter()
.map(|&i| &groups[i])
.collect::<Vec<_>>(),
arrow_schema,
};
let Ok(inverted_values) = inverted_predicate.prune(&inverted_pruning_stats)
else {
return;
};
for (i, &original_row_group_idx) in candidate_row_group_indices.iter().enumerate()
{
// If the inverted predicate *also* prunes this row group (meaning inverted_values[i] is false),
// it implies that *all* rows in this group satisfy the original predicate.
if !inverted_values[i] {
self.is_fully_matched[original_row_group_idx] = true;
metrics.row_groups_pruned_statistics.add_fully_matched(1);
}
}
}
/// Prune remaining row groups using available bloom filters and the
/// [`PruningPredicate`].
///
/// Updates this set with row groups that should not be scanned
///
/// # Panics
/// if the builder does not have the same number of row groups as this set
pub async fn prune_by_bloom_filters<T: AsyncFileReader + Send + 'static>(
&mut self,
arrow_schema: &Schema,
builder: &mut ParquetRecordBatchStreamBuilder<T>,
predicate: &PruningPredicate,
metrics: &ParquetFileMetrics,
) {
// scoped timer updates on drop
let _timer_guard = metrics.bloom_filter_eval_time.timer();
assert_eq!(builder.metadata().num_row_groups(), self.access_plan.len());
for idx in 0..self.access_plan.len() {
if !self.access_plan.should_scan(idx) {
continue;
}
// Attempt to find bloom filters for filtering this row group
let literal_columns = predicate.literal_columns();
let mut column_sbbf = HashMap::with_capacity(literal_columns.len());
for column_name in literal_columns {
let Some((column_idx, _field)) =
parquet_column(builder.parquet_schema(), arrow_schema, &column_name)
else {
continue;
};
let bf = match builder
.get_row_group_column_bloom_filter(idx, column_idx)
.await
{
Ok(Some(bf)) => bf,
Ok(None) => continue, // no bloom filter for this column
Err(e) => {
log::debug!("Ignoring error reading bloom filter: {e}");
metrics.predicate_evaluation_errors.add(1);
continue;
}
};
let physical_type =
builder.parquet_schema().column(column_idx).physical_type();
column_sbbf.insert(column_name.to_string(), (bf, physical_type));
}
let stats = BloomFilterStatistics { column_sbbf };
// Can this group be pruned?
let prune_group = match predicate.prune(&stats) {
Ok(values) => !values[0],
Err(e) => {
log::debug!(
"Error evaluating row group predicate on bloom filter: {e}"
);
metrics.predicate_evaluation_errors.add(1);
false
}
};
if prune_group {
metrics.row_groups_pruned_bloom_filter.add_pruned(1);
self.access_plan.skip(idx)
} else {
metrics.row_groups_pruned_bloom_filter.add_matched(1);
}
}
}
}
/// Implements [`PruningStatistics`] for Parquet Split Block Bloom Filters (SBBF)
struct BloomFilterStatistics {
/// Maps column name to the parquet bloom filter and parquet physical type
column_sbbf: HashMap<String, (Sbbf, Type)>,
}
impl BloomFilterStatistics {
/// Helper function for checking if [`Sbbf`] filter contains [`ScalarValue`].
///
/// In case the type of scalar is not supported, returns `true`, assuming that the
/// value may be present.
fn check_scalar(sbbf: &Sbbf, value: &ScalarValue, parquet_type: &Type) -> bool {
match value {
ScalarValue::Utf8(Some(v))
| ScalarValue::Utf8View(Some(v))
| ScalarValue::LargeUtf8(Some(v)) => sbbf.check(&v.as_str()),
ScalarValue::Binary(Some(v))
| ScalarValue::BinaryView(Some(v))
| ScalarValue::LargeBinary(Some(v)) => sbbf.check(v),
ScalarValue::FixedSizeBinary(_size, Some(v)) => sbbf.check(v),
ScalarValue::Boolean(Some(v)) => sbbf.check(v),
ScalarValue::Float64(Some(v)) => sbbf.check(v),
ScalarValue::Float32(Some(v)) => sbbf.check(v),
ScalarValue::Int64(Some(v)) => sbbf.check(v),
ScalarValue::Int32(Some(v)) => sbbf.check(v),
ScalarValue::UInt64(Some(v)) => sbbf.check(v),
ScalarValue::UInt32(Some(v)) => sbbf.check(v),
ScalarValue::Decimal128(Some(v), p, s) => match parquet_type {
Type::INT32 => {
//https://github.com/apache/parquet-format/blob/eb4b31c1d64a01088d02a2f9aefc6c17c54cc6fc/Encodings.md?plain=1#L35-L42
// All physical type are little-endian
if *p > 9 {
//DECIMAL can be used to annotate the following types:
//
// int32: for 1 <= precision <= 9
// int64: for 1 <= precision <= 18
return true;
}
let b = (*v as i32).to_le_bytes();
// Use Decimal constructor after https://github.com/apache/arrow-rs/issues/5325
let decimal = Decimal::Int32 {
value: b,
precision: *p as i32,
scale: *s as i32,
};
sbbf.check(&decimal)
}
Type::INT64 => {
if *p > 18 {
return true;
}
let b = (*v as i64).to_le_bytes();
let decimal = Decimal::Int64 {
value: b,
precision: *p as i32,
scale: *s as i32,
};
sbbf.check(&decimal)
}
Type::FIXED_LEN_BYTE_ARRAY => {
// keep with from_bytes_to_i128
let b = v.to_be_bytes().to_vec();
// Use Decimal constructor after https://github.com/apache/arrow-rs/issues/5325
let decimal = Decimal::Bytes {
value: b.into(),
precision: *p as i32,
scale: *s as i32,
};
sbbf.check(&decimal)
}
_ => true,
},
ScalarValue::Dictionary(_, inner) => {
BloomFilterStatistics::check_scalar(sbbf, inner, parquet_type)
}
_ => true,
}
}
}
impl PruningStatistics for BloomFilterStatistics {
fn min_values(&self, _column: &Column) -> Option<ArrayRef> {
None
}
fn max_values(&self, _column: &Column) -> Option<ArrayRef> {
None
}
fn num_containers(&self) -> usize {
1
}
fn null_counts(&self, _column: &Column) -> Option<ArrayRef> {
None
}
fn row_counts(&self, _column: &Column) -> Option<ArrayRef> {
None
}
/// Use bloom filters to determine if we are sure this column can not
/// possibly contain `values`
///
/// The `contained` API returns false if the bloom filters knows that *ALL*
/// of the values in a column are not present.
fn contained(
&self,
column: &Column,
values: &HashSet<ScalarValue>,
) -> Option<BooleanArray> {
let (sbbf, parquet_type) = self.column_sbbf.get(column.name.as_str())?;
// Bloom filters are probabilistic data structures that can return false
// positives (i.e. it might return true even if the value is not
// present) however, the bloom filter will return `false` if the value is
// definitely not present.
let known_not_present = values
.iter()
.map(|value| BloomFilterStatistics::check_scalar(sbbf, value, parquet_type))
// The row group doesn't contain any of the values if
// all the checks are false
.all(|v| !v);
let contains = if known_not_present {
Some(false)
} else {
// Given the bloom filter is probabilistic, we can't be sure that
// the row group actually contains the values. Return `None` to
// indicate this uncertainty
None
};
Some(BooleanArray::from(vec![contains]))
}
}
/// Wraps a slice of [`RowGroupMetaData`] in a way that implements [`PruningStatistics`]
struct RowGroupPruningStatistics<'a> {
parquet_schema: &'a SchemaDescriptor,
row_group_metadatas: Vec<&'a RowGroupMetaData>,
arrow_schema: &'a Schema,
}
impl<'a> RowGroupPruningStatistics<'a> {
/// Return an iterator over the row group metadata
fn metadata_iter(&'a self) -> impl Iterator<Item = &'a RowGroupMetaData> + 'a {
self.row_group_metadatas.iter().copied()
}
fn statistics_converter<'b>(
&'a self,
column: &'b Column,
) -> Result<StatisticsConverter<'a>> {
Ok(StatisticsConverter::try_new(
&column.name,
self.arrow_schema,
self.parquet_schema,
)?)
}
}
impl PruningStatistics for RowGroupPruningStatistics<'_> {
fn min_values(&self, column: &Column) -> Option<ArrayRef> {
self.statistics_converter(column)
.and_then(|c| Ok(c.row_group_mins(self.metadata_iter())?))
.ok()
}
fn max_values(&self, column: &Column) -> Option<ArrayRef> {
self.statistics_converter(column)
.and_then(|c| Ok(c.row_group_maxes(self.metadata_iter())?))
.ok()
}
fn num_containers(&self) -> usize {
self.row_group_metadatas.len()
}
fn null_counts(&self, column: &Column) -> Option<ArrayRef> {
self.statistics_converter(column)
.and_then(|c| Ok(c.row_group_null_counts(self.metadata_iter())?))
.ok()
.map(|counts| Arc::new(counts) as ArrayRef)
}
fn row_counts(&self, column: &Column) -> Option<ArrayRef> {
// row counts are the same for all columns in a row group
self.statistics_converter(column)
.and_then(|c| Ok(c.row_group_row_counts(self.metadata_iter())?))
.ok()
.flatten()
.map(|counts| Arc::new(counts) as ArrayRef)
}
fn contained(
&self,
_column: &Column,
_values: &HashSet<ScalarValue>,
) -> Option<BooleanArray> {
None
}
}
#[cfg(test)]
mod tests {
use std::ops::Rem;
use std::sync::Arc;
use super::*;
use crate::reader::ParquetFileReader;
use arrow::datatypes::DataType::Decimal128;
use arrow::datatypes::{DataType, Field};
use datafusion_common::Result;
use datafusion_expr::{Expr, cast, col, lit};
use datafusion_physical_expr::planner::logical2physical;
use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet;
use object_store::ObjectStoreExt;
use parquet::arrow::ArrowSchemaConverter;
use parquet::arrow::async_reader::ParquetObjectReader;
use parquet::basic::LogicalType;
use parquet::data_type::{ByteArray, FixedLenByteArray};
use parquet::file::metadata::ColumnChunkMetaData;
use parquet::{
basic::Type as PhysicalType, file::statistics::Statistics as ParquetStatistics,
schema::types::SchemaDescPtr,
};
struct PrimitiveTypeField {
name: &'static str,
physical_ty: PhysicalType,
logical_ty: Option<LogicalType>,
precision: Option<i32>,
scale: Option<i32>,
byte_len: Option<i32>,
}
impl PrimitiveTypeField {
fn new(name: &'static str, physical_ty: PhysicalType) -> Self {
Self {
name,
physical_ty,
logical_ty: None,
precision: None,
scale: None,
byte_len: None,
}
}
fn with_logical_type(mut self, logical_type: LogicalType) -> Self {
self.logical_ty = Some(logical_type);
self
}
fn with_precision(mut self, precision: i32) -> Self {
self.precision = Some(precision);
self
}
fn with_scale(mut self, scale: i32) -> Self {
self.scale = Some(scale);
self
}
fn with_byte_len(mut self, byte_len: i32) -> Self {
self.byte_len = Some(byte_len);
self
}
}
#[test]
fn remaining_row_group_count_reports_non_skipped_groups() {
let mut filter = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(4));
assert_eq!(filter.remaining_row_group_count(), 4);
filter.access_plan.skip(1);
assert_eq!(filter.remaining_row_group_count(), 3);
filter.access_plan.skip(3);
assert_eq!(filter.remaining_row_group_count(), 2);
}
#[test]
fn row_group_pruning_predicate_simple_expr() {
use datafusion_expr::{col, lit};
// int > 1 => c1_max > 1
let schema =
Arc::new(Schema::new(vec![Field::new("c1", DataType::Int32, false)]));
let expr = col("c1").gt(lit(15));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let field = PrimitiveTypeField::new("c1", PhysicalType::INT32);
let schema_descr = get_test_schema_descr(vec![field]);
let rgm1 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::int32(
Some(1),
Some(10),
None,
Some(0),
false,
)],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::int32(
Some(11),
Some(20),
None,
Some(0),
false,
)],
);
let metrics = parquet_file_metrics();
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(2));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&[rgm1, rgm2],
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![1]))
}
#[test]
fn row_group_pruning_predicate_missing_stats() {
use datafusion_expr::{col, lit};
// int > 1 => c1_max > 1
let schema =
Arc::new(Schema::new(vec![Field::new("c1", DataType::Int32, false)]));
let expr = col("c1").gt(lit(15));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let field = PrimitiveTypeField::new("c1", PhysicalType::INT32);
let schema_descr = get_test_schema_descr(vec![field]);
let rgm1 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::int32(None, None, None, Some(0), false)],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::int32(
Some(11),
Some(20),
None,
Some(0),
false,
)],
);
let metrics = parquet_file_metrics();
// missing statistics for first row group mean that the result from the predicate expression
// is null / undefined so the first row group can't be filtered out
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(2));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&[rgm1, rgm2],
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::None);
}
#[test]
fn row_group_pruning_predicate_partial_expr() {
use datafusion_expr::{col, lit};
// test row group predicate with partially supported expression
// (int > 1) and ((int % 2) = 0) => c1_max > 1 and true
let schema = Arc::new(Schema::new(vec![
Field::new("c1", DataType::Int32, false),
Field::new("c2", DataType::Int32, false),
]));
let expr = col("c1").gt(lit(15)).and(col("c2").rem(lit(2)).eq(lit(0)));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let schema_descr = get_test_schema_descr(vec![
PrimitiveTypeField::new("c1", PhysicalType::INT32),
PrimitiveTypeField::new("c2", PhysicalType::INT32),
]);
let rgm1 = get_row_group_meta_data(
&schema_descr,
vec![
ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false),
ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false),
],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
vec![
ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false),
ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false),
],
);
let metrics = parquet_file_metrics();
let groups = &[rgm1, rgm2];
// the first row group is still filtered out because the predicate expression can be partially evaluated
// when conditions are joined using AND
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(2));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
groups,
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![1]));
// if conditions in predicate are joined with OR and an unsupported expression is used
// this bypasses the entire predicate expression and no row groups are filtered out
let expr = col("c1").gt(lit(15)).or(col("c2").rem(lit(2)).eq(lit(0)));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
// if conditions in predicate are joined with OR and an unsupported expression is used
// this bypasses the entire predicate expression and no row groups are filtered out
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(2));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
groups,
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::None);
}
#[test]
fn row_group_pruning_predicate_file_schema() {
use datafusion_expr::{col, lit};
// test row group predicate when file schema is different than table schema
// c1 > 0
let table_schema = Arc::new(Schema::new(vec![
Field::new("c1", DataType::Int32, false),
Field::new("c2", DataType::Int32, false),
]));
let expr = col("c1").gt(lit(0));
let expr = logical2physical(&expr, &table_schema);
let pruning_predicate =
PruningPredicate::try_new(expr, table_schema.clone()).unwrap();
// Model a file schema's column order c2 then c1, which is the opposite
// of the table schema
let file_schema = Arc::new(Schema::new(vec![
Field::new("c2", DataType::Int32, false),
Field::new("c1", DataType::Int32, false),
]));
let schema_descr = get_test_schema_descr(vec![
PrimitiveTypeField::new("c2", PhysicalType::INT32),
PrimitiveTypeField::new("c1", PhysicalType::INT32),
]);
// rg1 has c2 less than zero, c1 greater than zero
let rgm1 = get_row_group_meta_data(
&schema_descr,
vec![
ParquetStatistics::int32(Some(-10), Some(-1), None, Some(0), false), // c2
ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false),
],
);
// rg1 has c2 greater than zero, c1 less than zero
let rgm2 = get_row_group_meta_data(
&schema_descr,
vec![
ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false),
ParquetStatistics::int32(Some(-10), Some(-1), None, Some(0), false),
],
);
let metrics = parquet_file_metrics();
let groups = &[rgm1, rgm2];
// the first row group should be left because c1 is greater than zero
// the second should be filtered out because c1 is less than zero
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(2));
row_groups.prune_by_statistics(
&file_schema,
&schema_descr,
groups,
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![0]));
}
fn gen_row_group_meta_data_for_pruning_predicate() -> Vec<RowGroupMetaData> {
let schema_descr = get_test_schema_descr(vec![
PrimitiveTypeField::new("c1", PhysicalType::INT32),
PrimitiveTypeField::new("c2", PhysicalType::BOOLEAN),
]);
let rgm1 = get_row_group_meta_data(
&schema_descr,
vec![
ParquetStatistics::int32(Some(1), Some(10), None, Some(0), false),
ParquetStatistics::boolean(Some(false), Some(true), None, Some(0), false),
],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
vec![
ParquetStatistics::int32(Some(11), Some(20), None, Some(0), false),
ParquetStatistics::boolean(Some(false), Some(true), None, Some(1), false),
],
);
vec![rgm1, rgm2]
}
#[test]
fn row_group_pruning_predicate_null_expr() {
use datafusion_expr::{col, lit};
// int > 1 and IsNull(bool) => c1_max > 1 and bool_null_count > 0
let schema = Arc::new(Schema::new(vec![
Field::new("c1", DataType::Int32, false),
Field::new("c2", DataType::Boolean, false),
]));
let schema_descr = ArrowSchemaConverter::new().convert(&schema).unwrap();
let expr = col("c1").gt(lit(15)).and(col("c2").is_null());
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let groups = gen_row_group_meta_data_for_pruning_predicate();
let metrics = parquet_file_metrics();
// First row group was filtered out because it contains no null value on "c2".
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(2));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&groups,
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![1]));
}
#[test]
fn row_group_pruning_predicate_eq_null_expr() {
use datafusion_expr::{col, lit};
// test row group predicate with an unknown (Null) expr
//
// int > 1 and bool = NULL => c1_max > 1 and null
let schema = Arc::new(Schema::new(vec![
Field::new("c1", DataType::Int32, false),
Field::new("c2", DataType::Boolean, false),
]));
let schema_descr = ArrowSchemaConverter::new().convert(&schema).unwrap();
let expr = col("c1")
.gt(lit(15))
.and(col("c2").eq(lit(ScalarValue::Boolean(None))));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let groups = gen_row_group_meta_data_for_pruning_predicate();
let metrics = parquet_file_metrics();
// bool = NULL always evaluates to NULL (and thus will not
// pass predicates. Ideally these should both be false
let mut row_groups =
RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(groups.len()));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&groups,
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![1]));
}
#[test]
fn row_group_pruning_predicate_decimal_type() {
// For the decimal data type, parquet can use `INT32`, `INT64`, `BYTE_ARRAY`, `FIXED_LENGTH_BYTE_ARRAY` to
// store the data.
// In this case, construct four types of statistics to filtered with the decimal predication.
// INT32: c1 > 5, the c1 is decimal(9,2)
// The type of scalar value if decimal(9,2), don't need to do cast
let schema =
Arc::new(Schema::new(vec![Field::new("c1", Decimal128(9, 2), false)]));
let field = PrimitiveTypeField::new("c1", PhysicalType::INT32)
.with_logical_type(LogicalType::Decimal {
scale: 2,
precision: 9,
})
.with_scale(2)
.with_precision(9);
let schema_descr = get_test_schema_descr(vec![field]);
let expr = col("c1").gt(lit(ScalarValue::Decimal128(Some(500), 9, 2)));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let rgm1 = get_row_group_meta_data(
&schema_descr,
// [1.00, 6.00]
// c1 > 5, this row group will be included in the results.
vec![ParquetStatistics::int32(
Some(100),
Some(600),
None,
Some(0),
false,
)],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
// [0.1, 0.2]
// c1 > 5, this row group will not be included in the results.
vec![ParquetStatistics::int32(
Some(10),
Some(20),
None,
Some(0),
false,
)],
);
let rgm3 = get_row_group_meta_data(
&schema_descr,
// [1, None]
// c1 > 5, this row group can not be filtered out, so will be included in the results.
vec![ParquetStatistics::int32(
Some(100),
None,
None,
Some(0),
false,
)],
);
let metrics = parquet_file_metrics();
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&[rgm1, rgm2, rgm3],
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![0, 2]));
}
#[test]
fn row_group_pruning_predicate_decimal_type2() {
// INT32: c1 > 5, but parquet decimal type has different precision or scale to arrow decimal
// The c1 type is decimal(9,0) in the parquet file, and the type of scalar is decimal(5,2).
// We should convert all type to the coercion type, which is decimal(11,2)
// The decimal of arrow is decimal(5,2), the decimal of parquet is decimal(9,0)
let schema =
Arc::new(Schema::new(vec![Field::new("c1", Decimal128(9, 0), false)]));
let field = PrimitiveTypeField::new("c1", PhysicalType::INT32)
.with_logical_type(LogicalType::Decimal {
scale: 0,
precision: 9,
})
.with_scale(0)
.with_precision(9);
let schema_descr = get_test_schema_descr(vec![field]);
let expr = cast(col("c1"), Decimal128(11, 2)).gt(cast(
lit(ScalarValue::Decimal128(Some(500), 5, 2)),
Decimal128(11, 2),
));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let rgm1 = get_row_group_meta_data(
&schema_descr,
// [100, 600]
// c1 > 5, this row group will be included in the results.
vec![ParquetStatistics::int32(
Some(100),
Some(600),
None,
Some(0),
false,
)],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
// [10, 20]
// c1 > 5, this row group will be included in the results.
vec![ParquetStatistics::int32(
Some(10),
Some(20),
None,
Some(0),
false,
)],
);
let rgm3 = get_row_group_meta_data(
&schema_descr,
// [0, 2]
// c1 > 5, this row group will not be included in the results.
vec![ParquetStatistics::int32(
Some(0),
Some(2),
None,
Some(0),
false,
)],
);
let rgm4 = get_row_group_meta_data(
&schema_descr,
// [None, 2]
// c1 > 5, this row group will also not be included in the results
// (the min value is unknown, but the max value is 2, so no values can be greater than 5)
vec![ParquetStatistics::int32(
None,
Some(2),
None,
Some(0),
false,
)],
);
let rgm5 = get_row_group_meta_data(
&schema_descr,
// [2, None]
// c1 > 5, this row group must be included
// (the min value is 2, but the max value is unknown, so it may have values greater than 5)
vec![ParquetStatistics::int32(
Some(2),
None,
None,
Some(0),
false,
)],
);
let metrics = parquet_file_metrics();
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(5));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&[rgm1, rgm2, rgm3, rgm4, rgm5],
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![0, 1, 4]));
}
#[test]
fn row_group_pruning_predicate_decimal_type3() {
// INT64: c1 < 5, the c1 is decimal(18,2)
let schema = Arc::new(Schema::new(vec![Field::new(
"c1",
Decimal128(18, 2),
false,
)]));
let field = PrimitiveTypeField::new("c1", PhysicalType::INT64)
.with_logical_type(LogicalType::Decimal {
scale: 2,
precision: 18,
})
.with_scale(2)
.with_precision(18);
let schema_descr = get_test_schema_descr(vec![field]);
let expr = col("c1").lt(lit(ScalarValue::Decimal128(Some(500), 18, 2)));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
let rgm1 = get_row_group_meta_data(
&schema_descr,
// [6.00, 8.00]
vec![ParquetStatistics::int32(
Some(600),
Some(800),
None,
Some(0),
false,
)],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
// [0.1, 0.2]
vec![ParquetStatistics::int64(
Some(10),
Some(20),
None,
Some(0),
false,
)],
);
let rgm3 = get_row_group_meta_data(
&schema_descr,
// [0.1, 0.2]
vec![ParquetStatistics::int64(None, None, None, Some(0), false)],
);
let metrics = parquet_file_metrics();
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&[rgm1, rgm2, rgm3],
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![1, 2]));
}
#[test]
fn row_group_pruning_predicate_decimal_type4() {
// FIXED_LENGTH_BYTE_ARRAY: c1 = decimal128(100000, 28, 3), the c1 is decimal(18,2)
// the type of parquet is decimal(18,2)
let schema = Arc::new(Schema::new(vec![Field::new(
"c1",
Decimal128(18, 2),
false,
)]));
let field = PrimitiveTypeField::new("c1", PhysicalType::FIXED_LEN_BYTE_ARRAY)
.with_logical_type(LogicalType::Decimal {
scale: 2,
precision: 18,
})
.with_scale(2)
.with_precision(18)
.with_byte_len(16);
let schema_descr = get_test_schema_descr(vec![field]);
// cast the type of c1 to decimal(28,3)
let left = cast(col("c1"), Decimal128(28, 3));
let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3)));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
// we must use the big-endian when encode the i128 to bytes or vec[u8].
let rgm1 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::fixed_len_byte_array(
// 5.00
Some(FixedLenByteArray::from(ByteArray::from(
500i128.to_be_bytes().to_vec(),
))),
// 80.00
Some(FixedLenByteArray::from(ByteArray::from(
8000i128.to_be_bytes().to_vec(),
))),
None,
Some(0),
false,
)],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::fixed_len_byte_array(
// 5.00
Some(FixedLenByteArray::from(ByteArray::from(
500i128.to_be_bytes().to_vec(),
))),
// 200.00
Some(FixedLenByteArray::from(ByteArray::from(
20000i128.to_be_bytes().to_vec(),
))),
None,
Some(0),
false,
)],
);
let rgm3 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::fixed_len_byte_array(
None,
None,
None,
Some(0),
false,
)],
);
let metrics = parquet_file_metrics();
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&[rgm1, rgm2, rgm3],
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![1, 2]));
}
#[test]
fn row_group_pruning_predicate_decimal_type5() {
// BYTE_ARRAY: c1 = decimal128(100000, 28, 3), the c1 is decimal(18,2)
// the type of parquet is decimal(18,2)
let schema = Arc::new(Schema::new(vec![Field::new(
"c1",
Decimal128(18, 2),
false,
)]));
let field = PrimitiveTypeField::new("c1", PhysicalType::BYTE_ARRAY)
.with_logical_type(LogicalType::Decimal {
scale: 2,
precision: 18,
})
.with_scale(2)
.with_precision(18)
.with_byte_len(16);
let schema_descr = get_test_schema_descr(vec![field]);
// cast the type of c1 to decimal(28,3)
let left = cast(col("c1"), Decimal128(28, 3));
let expr = left.eq(lit(ScalarValue::Decimal128(Some(100000), 28, 3)));
let expr = logical2physical(&expr, &schema);
let pruning_predicate = PruningPredicate::try_new(expr, schema.clone()).unwrap();
// we must use the big-endian when encode the i128 to bytes or vec[u8].
let rgm1 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::byte_array(
// 5.00
Some(ByteArray::from(500i128.to_be_bytes().to_vec())),
// 80.00
Some(ByteArray::from(8000i128.to_be_bytes().to_vec())),
None,
Some(0),
false,
)],
);
let rgm2 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::byte_array(
// 5.00
Some(ByteArray::from(500i128.to_be_bytes().to_vec())),
// 200.00
Some(ByteArray::from(20000i128.to_be_bytes().to_vec())),
None,
Some(0),
false,
)],
);
let rgm3 = get_row_group_meta_data(
&schema_descr,
vec![ParquetStatistics::byte_array(
None,
None,
None,
Some(0),
false,
)],
);
let metrics = parquet_file_metrics();
let mut row_groups = RowGroupAccessPlanFilter::new(ParquetAccessPlan::new_all(3));
row_groups.prune_by_statistics(
&schema,
&schema_descr,
&[rgm1, rgm2, rgm3],
&pruning_predicate,
&metrics,
);
assert_pruned(row_groups, ExpectedPruning::Some(vec![1, 2]));
}
fn get_row_group_meta_data(
schema_descr: &SchemaDescPtr,
column_statistics: Vec<ParquetStatistics>,
) -> RowGroupMetaData {
let mut columns = vec![];
let number_row = 1000;
for (i, s) in column_statistics.iter().enumerate() {
let column = ColumnChunkMetaData::builder(schema_descr.column(i))
.set_statistics(s.clone())
.set_num_values(number_row)
.build()
.unwrap();
columns.push(column);
}
RowGroupMetaData::builder(schema_descr.clone())
.set_num_rows(number_row)
.set_total_byte_size(2000)
.set_column_metadata(columns)
.build()
.unwrap()
}
fn get_test_schema_descr(fields: Vec<PrimitiveTypeField>) -> SchemaDescPtr {
use parquet::schema::types::Type as SchemaType;
let schema_fields = fields
.iter()
.map(|field| {
let mut builder =
SchemaType::primitive_type_builder(field.name, field.physical_ty);
// add logical type for the parquet field
if let Some(logical_type) = &field.logical_ty {
builder = builder.with_logical_type(Some(logical_type.clone()));
}
if let Some(precision) = field.precision {
builder = builder.with_precision(precision);
}
if let Some(scale) = field.scale {
builder = builder.with_scale(scale);
}
if let Some(byte_len) = field.byte_len {
builder = builder.with_length(byte_len);
}
Arc::new(builder.build().unwrap())
})
.collect::<Vec<_>>();
let schema = SchemaType::group_type_builder("schema")
.with_fields(schema_fields)
.build()
.unwrap();
Arc::new(SchemaDescriptor::new(Arc::new(schema)))
}
fn parquet_file_metrics() -> ParquetFileMetrics {
let metrics = Arc::new(ExecutionPlanMetricsSet::new());
ParquetFileMetrics::new(0, "file.parquet", &metrics)
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_simple_expr() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_all_pruned()
// generate pruning predicate `(String = "Hello_Not_exists")`
.run(col(r#""String""#).eq(lit("Hello_Not_Exists")))
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_multiple_expr() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_all_pruned()
// generate pruning predicate `(String = "Hello_Not_exists" OR String = "Hello_Not_exists2")`
.run(
lit("1").eq(lit("1")).and(
col(r#""String""#)
.eq(lit("Hello_Not_Exists"))
.or(col(r#""String""#).eq(lit("Hello_Not_Exists2"))),
),
)
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_multiple_expr_view() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_all_pruned()
// generate pruning predicate `(String = "Hello_Not_exists" OR String = "Hello_Not_exists2")`
.run(
lit("1").eq(lit("1")).and(
col(r#""String""#)
.eq(Expr::Literal(
ScalarValue::Utf8View(Some(String::from("Hello_Not_Exists"))),
None,
))
.or(col(r#""String""#).eq(Expr::Literal(
ScalarValue::Utf8View(Some(String::from(
"Hello_Not_Exists2",
))),
None,
))),
),
)
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_sql_in() {
// load parquet file
let testdata = datafusion_common::test_util::parquet_test_data();
let file_name = "data_index_bloom_encoding_stats.parquet";
let path = format!("{testdata}/{file_name}");
let data = bytes::Bytes::from(std::fs::read(path).unwrap());
// generate pruning predicate
let schema = Schema::new(vec![Field::new("String", DataType::Utf8, false)]);
let expr = col(r#""String""#).in_list(
(1..25)
.map(|i| lit(format!("Hello_Not_Exists{i}")))
.collect::<Vec<_>>(),
false,
);
let expr = logical2physical(&expr, &schema);
let pruning_predicate =
PruningPredicate::try_new(expr, Arc::new(schema)).unwrap();
let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate(
file_name,
data,
&pruning_predicate,
)
.await
.unwrap();
assert!(pruned_row_groups.access_plan.row_group_indexes().is_empty());
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_with_exists_value() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_none_pruned()
// generate pruning predicate `(String = "Hello")`
.run(col(r#""String""#).eq(lit("Hello")))
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_with_exists_2_values() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_none_pruned()
// generate pruning predicate `(String = "Hello") OR (String = "the quick")`
.run(
col(r#""String""#)
.eq(lit("Hello"))
.or(col(r#""String""#).eq(lit("the quick"))),
)
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_with_exists_3_values() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_none_pruned()
// generate pruning predicate `(String = "Hello") OR (String = "the quick") OR (String = "are you")`
.run(
col(r#""String""#)
.eq(lit("Hello"))
.or(col(r#""String""#).eq(lit("the quick")))
.or(col(r#""String""#).eq(lit("are you"))),
)
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_with_exists_3_values_view() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_none_pruned()
// generate pruning predicate `(String = "Hello") OR (String = "the quick") OR (String = "are you")`
.run(
col(r#""String""#)
.eq(Expr::Literal(
ScalarValue::Utf8View(Some(String::from("Hello"))),
None,
))
.or(col(r#""String""#).eq(Expr::Literal(
ScalarValue::Utf8View(Some(String::from("the quick"))),
None,
)))
.or(col(r#""String""#).eq(Expr::Literal(
ScalarValue::Utf8View(Some(String::from("are you"))),
None,
))),
)
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_with_or_not_eq() {
BloomFilterTest::new_data_index_bloom_encoding_stats()
.with_expect_none_pruned()
// generate pruning predicate `(String = "foo") OR (String != "bar")`
.run(
col(r#""String""#)
.not_eq(lit("foo"))
.or(col(r#""String""#).not_eq(lit("bar"))),
)
.await
}
#[tokio::test]
async fn test_row_group_bloom_filter_pruning_predicate_without_bloom_filter() {
// generate pruning predicate on a column without a bloom filter
BloomFilterTest::new_all_types()
.with_expect_none_pruned()
.run(col(r#""string_col""#).eq(lit("0")))
.await
}
// What row groups are expected to be left after pruning
#[derive(Debug)]
enum ExpectedPruning {
All,
/// Only the specified row groups are expected to REMAIN (not what is pruned)
Some(Vec<usize>),
None,
}
impl ExpectedPruning {
/// asserts that the pruned row group match this expectation
fn assert(&self, row_groups: &RowGroupAccessPlanFilter) {
let num_row_groups = row_groups.access_plan.len();
assert!(num_row_groups > 0);
let num_pruned = (0..num_row_groups)
.filter_map(|i| {
if row_groups.access_plan.should_scan(i) {
None
} else {
Some(1)
}
})
.sum::<usize>();
match self {
Self::All => {
assert_eq!(
num_row_groups, num_pruned,
"Expected all row groups to be pruned, but got {row_groups:?}"
);
}
ExpectedPruning::None => {
assert_eq!(
num_pruned, 0,
"Expected no row groups to be pruned, but got {row_groups:?}"
);
}
ExpectedPruning::Some(expected) => {
let actual = row_groups.access_plan.row_group_indexes();
assert_eq!(
expected, &actual,
"Unexpected row groups pruned. Expected {expected:?}, got {actual:?}"
);
}
}
}
}
fn assert_pruned(row_groups: RowGroupAccessPlanFilter, expected: ExpectedPruning) {
expected.assert(&row_groups);
}
struct BloomFilterTest {
file_name: String,
schema: Schema,
// which row groups are expected to be left after pruning
post_pruning_row_groups: ExpectedPruning,
}
impl BloomFilterTest {
/// Return a test for data_index_bloom_encoding_stats.parquet
/// Note the values in the `String` column are:
/// ```sql
/// > select * from './parquet-testing/data/data_index_bloom_encoding_stats.parquet';
/// +-----------+
/// | String |
/// +-----------+
/// | Hello |
/// | This is |
/// | a |
/// | test |
/// | How |
/// | are you |
/// | doing |
/// | today |
/// | the quick |
/// | brown fox |
/// | jumps |
/// | over |
/// | the lazy |
/// | dog |
/// +-----------+
/// ```
fn new_data_index_bloom_encoding_stats() -> Self {
Self {
file_name: String::from("data_index_bloom_encoding_stats.parquet"),
schema: Schema::new(vec![Field::new("String", DataType::Utf8, false)]),
post_pruning_row_groups: ExpectedPruning::None,
}
}
// Return a test for alltypes_plain.parquet
fn new_all_types() -> Self {
Self {
file_name: String::from("alltypes_plain.parquet"),
schema: Schema::new(vec![Field::new(
"string_col",
DataType::Utf8,
false,
)]),
post_pruning_row_groups: ExpectedPruning::None,
}
}
/// Expect all row groups to be pruned
pub fn with_expect_all_pruned(mut self) -> Self {
self.post_pruning_row_groups = ExpectedPruning::All;
self
}
/// Expect all row groups not to be pruned
pub fn with_expect_none_pruned(mut self) -> Self {
self.post_pruning_row_groups = ExpectedPruning::None;
self
}
/// Prune this file using the specified expression and check that the expected row groups are left
async fn run(self, expr: Expr) {
let Self {
file_name,
schema,
post_pruning_row_groups,
} = self;
let testdata = datafusion_common::test_util::parquet_test_data();
let path = format!("{testdata}/{file_name}");
let data = bytes::Bytes::from(std::fs::read(path).unwrap());
let expr = logical2physical(&expr, &schema);
let pruning_predicate =
PruningPredicate::try_new(expr, Arc::new(schema)).unwrap();
let pruned_row_groups = test_row_group_bloom_filter_pruning_predicate(
&file_name,
data,
&pruning_predicate,
)
.await
.unwrap();
post_pruning_row_groups.assert(&pruned_row_groups);
}
}
/// Evaluates the pruning predicate on the specified row groups and returns the row groups that are left
async fn test_row_group_bloom_filter_pruning_predicate(
file_name: &str,
data: bytes::Bytes,
pruning_predicate: &PruningPredicate,
) -> Result<RowGroupAccessPlanFilter> {
use datafusion_datasource::PartitionedFile;
use object_store::ObjectMeta;
let object_meta = ObjectMeta {
location: object_store::path::Path::parse(file_name).expect("creating path"),
last_modified: chrono::DateTime::from(std::time::SystemTime::now()),
size: data.len() as u64,
e_tag: None,
version: None,
};
let in_memory = object_store::memory::InMemory::new();
in_memory
.put(&object_meta.location, data.into())
.await
.expect("put parquet file into in memory object store");
let metrics = ExecutionPlanMetricsSet::new();
let file_metrics =
ParquetFileMetrics::new(0, object_meta.location.as_ref(), &metrics);
let inner =
ParquetObjectReader::new(Arc::new(in_memory), object_meta.location.clone())
.with_file_size(object_meta.size);
let partitioned_file = PartitionedFile::new_from_meta(object_meta);
let reader = ParquetFileReader {
inner,
file_metrics: file_metrics.clone(),
partitioned_file,
};
let mut builder = ParquetRecordBatchStreamBuilder::new(reader).await.unwrap();
let access_plan = ParquetAccessPlan::new_all(builder.metadata().num_row_groups());
let mut pruned_row_groups = RowGroupAccessPlanFilter::new(access_plan);
pruned_row_groups
.prune_by_bloom_filters(
pruning_predicate.schema(),
&mut builder,
pruning_predicate,
&file_metrics,
)
.await;
Ok(pruned_row_groups)
}
}