-
Notifications
You must be signed in to change notification settings - Fork 2.1k
Expand file tree
/
Copy pathopener.rs
More file actions
1992 lines (1784 loc) · 81.2 KB
/
opener.rs
File metadata and controls
1992 lines (1784 loc) · 81.2 KB
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
// 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.
//! [`ParquetOpener`] for opening Parquet files
use crate::page_filter::PagePruningAccessPlanFilter;
use crate::row_group_filter::RowGroupAccessPlanFilter;
use crate::{
ParquetAccessPlan, ParquetFileMetrics, ParquetFileReaderFactory,
apply_file_schema_type_coercions, coerce_int96_to_resolution, row_filter,
};
use arrow::array::{RecordBatch, RecordBatchOptions};
use arrow::datatypes::DataType;
use datafusion_datasource::file_stream::{FileOpenFuture, FileOpener};
use datafusion_physical_expr::projection::ProjectionExprs;
use datafusion_physical_expr::utils::reassign_expr_columns;
use datafusion_physical_expr_adapter::replace_columns_with_literals;
use std::collections::HashMap;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};
use arrow::datatypes::{SchemaRef, TimeUnit};
use datafusion_common::encryption::FileDecryptionProperties;
use datafusion_common::stats::Precision;
use datafusion_common::{
ColumnStatistics, DataFusionError, Result, ScalarValue, Statistics, exec_err,
};
use datafusion_datasource::{PartitionedFile, TableSchema};
use datafusion_physical_expr::simplifier::PhysicalExprSimplifier;
use datafusion_physical_expr_adapter::PhysicalExprAdapterFactory;
use datafusion_physical_expr_common::physical_expr::{
PhysicalExpr, is_dynamic_physical_expr,
};
use datafusion_physical_plan::metrics::{
Count, ExecutionPlanMetricsSet, MetricBuilder, PruningMetrics,
};
use datafusion_pruning::{FilePruner, PruningPredicate, build_pruning_predicate};
use crate::sort::reverse_row_selection;
#[cfg(feature = "parquet_encryption")]
use datafusion_common::config::EncryptionFactoryOptions;
#[cfg(feature = "parquet_encryption")]
use datafusion_execution::parquet_encryption::EncryptionFactory;
use futures::{Stream, StreamExt, TryStreamExt, ready};
use log::debug;
use parquet::arrow::arrow_reader::metrics::ArrowReaderMetrics;
use parquet::arrow::arrow_reader::{
ArrowReaderMetadata, ArrowReaderOptions, RowSelectionPolicy,
};
use parquet::arrow::async_reader::AsyncFileReader;
use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask};
use parquet::file::metadata::{PageIndexPolicy, ParquetMetaDataReader, RowGroupMetaData};
/// Implements [`FileOpener`] for a parquet file
pub(super) struct ParquetOpener {
/// Execution partition index
pub partition_index: usize,
/// Projection to apply on top of the table schema (i.e. can reference partition columns).
pub projection: ProjectionExprs,
/// Target number of rows in each output RecordBatch
pub batch_size: usize,
/// Optional limit on the number of rows to read
pub limit: Option<usize>,
/// Optional predicate to apply during the scan
pub predicate: Option<Arc<dyn PhysicalExpr>>,
/// Table schema, including partition columns.
pub table_schema: TableSchema,
/// Optional hint for how large the initial request to read parquet metadata
/// should be
pub metadata_size_hint: Option<usize>,
/// Metrics for reporting
pub metrics: ExecutionPlanMetricsSet,
/// Factory for instantiating parquet reader
pub parquet_file_reader_factory: Arc<dyn ParquetFileReaderFactory>,
/// Should the filters be evaluated during the parquet scan using
/// [`DataFusionArrowPredicate`](row_filter::DatafusionArrowPredicate)?
pub pushdown_filters: bool,
/// Should the filters be reordered to optimize the scan?
pub reorder_filters: bool,
/// Should we force the reader to use RowSelections for filtering
pub force_filter_selections: bool,
/// Should the page index be read from parquet files, if present, to skip
/// data pages
pub enable_page_index: bool,
/// Should the bloom filter be read from parquet, if present, to skip row
/// groups
pub enable_bloom_filter: bool,
/// Should row group pruning be applied
pub enable_row_group_stats_pruning: bool,
/// Coerce INT96 timestamps to specific TimeUnit
pub coerce_int96: Option<TimeUnit>,
/// Optional parquet FileDecryptionProperties
#[cfg(feature = "parquet_encryption")]
pub file_decryption_properties: Option<Arc<FileDecryptionProperties>>,
/// Rewrite expressions in the context of the file schema
pub(crate) expr_adapter_factory: Arc<dyn PhysicalExprAdapterFactory>,
/// Optional factory to create file decryption properties dynamically
#[cfg(feature = "parquet_encryption")]
pub encryption_factory:
Option<(Arc<dyn EncryptionFactory>, EncryptionFactoryOptions)>,
/// Maximum size of the predicate cache, in bytes. If none, uses
/// the arrow-rs default.
pub max_predicate_cache_size: Option<usize>,
/// Whether to read row groups in reverse order
pub reverse_row_groups: bool,
}
/// Represents a prepared access plan with optional row selection
pub(crate) struct PreparedAccessPlan {
/// Row group indexes to read
pub(crate) row_group_indexes: Vec<usize>,
/// Optional row selection for filtering within row groups
pub(crate) row_selection: Option<parquet::arrow::arrow_reader::RowSelection>,
}
impl PreparedAccessPlan {
/// Create a new prepared access plan from a ParquetAccessPlan
pub(crate) fn from_access_plan(
access_plan: ParquetAccessPlan,
rg_metadata: &[RowGroupMetaData],
) -> Result<Self> {
let row_group_indexes = access_plan.row_group_indexes();
let row_selection = access_plan.into_overall_row_selection(rg_metadata)?;
Ok(Self {
row_group_indexes,
row_selection,
})
}
/// Reverse the access plan for reverse scanning
pub(crate) fn reverse(
mut self,
file_metadata: &parquet::file::metadata::ParquetMetaData,
) -> Result<Self> {
// Get the row group indexes before reversing
let row_groups_to_scan = self.row_group_indexes.clone();
// Reverse the row group indexes
self.row_group_indexes = self.row_group_indexes.into_iter().rev().collect();
// If we have a row selection, reverse it to match the new row group order
if let Some(row_selection) = self.row_selection {
self.row_selection = Some(reverse_row_selection(
&row_selection,
file_metadata,
&row_groups_to_scan, // Pass the original (non-reversed) row group indexes
)?);
}
Ok(self)
}
/// Apply this access plan to a ParquetRecordBatchStreamBuilder
fn apply_to_builder(
self,
mut builder: ParquetRecordBatchStreamBuilder<Box<dyn AsyncFileReader>>,
) -> ParquetRecordBatchStreamBuilder<Box<dyn AsyncFileReader>> {
if let Some(row_selection) = self.row_selection {
builder = builder.with_row_selection(row_selection);
}
builder.with_row_groups(self.row_group_indexes)
}
}
impl FileOpener for ParquetOpener {
fn open(&self, partitioned_file: PartitionedFile) -> Result<FileOpenFuture> {
// -----------------------------------
// Step: prepare configurations, etc.
// -----------------------------------
let file_range = partitioned_file.range.clone();
let extensions = partitioned_file.extensions.clone();
let file_location = partitioned_file.object_meta.location.clone();
let file_name = file_location.to_string();
let file_metrics =
ParquetFileMetrics::new(self.partition_index, &file_name, &self.metrics);
let metadata_size_hint = partitioned_file
.metadata_size_hint
.or(self.metadata_size_hint);
let mut async_file_reader: Box<dyn AsyncFileReader> =
self.parquet_file_reader_factory.create_reader(
self.partition_index,
partitioned_file.clone(),
metadata_size_hint,
&self.metrics,
)?;
let batch_size = self.batch_size;
// Calculate the output schema from the original projection (before literal replacement)
// so we get correct field names from column references
let logical_file_schema = Arc::clone(self.table_schema.file_schema());
let output_schema = Arc::new(
self.projection
.project_schema(self.table_schema.table_schema())?,
);
// Build a combined map for replacing column references with literal values.
// This includes:
// 1. Partition column values from the file path (e.g., region=us-west-2)
// 2. Constant columns detected from file statistics (where min == max)
//
// Although partition columns *are* constant columns, we don't want to rely on
// statistics for them being populated if we can use the partition values
// (which are guaranteed to be present).
//
// For example, given a partition column `region` and predicate
// `region IN ('us-east-1', 'eu-central-1')` with file path
// `/data/region=us-west-2/...`, the predicate is rewritten to
// `'us-west-2' IN ('us-east-1', 'eu-central-1')` which simplifies to FALSE.
//
// While partition column optimization is done during logical planning,
// there are cases where partition columns may appear in more complex
// predicates that cannot be simplified until we open the file (such as
// dynamic predicates).
let mut literal_columns: HashMap<String, ScalarValue> = self
.table_schema
.table_partition_cols()
.iter()
.zip(partitioned_file.partition_values.iter())
.map(|(field, value)| (field.name().clone(), value.clone()))
.collect();
// Add constant columns from file statistics.
// Note that if there are statistics for partition columns there will be overlap,
// but since we use a HashMap, we'll just overwrite the partition values with the
// constant values from statistics (which should be the same).
literal_columns.extend(constant_columns_from_stats(
partitioned_file.statistics.as_deref(),
&logical_file_schema,
));
// Apply literal replacements to projection and predicate
let mut projection = self.projection.clone();
let mut predicate = self.predicate.clone();
if !literal_columns.is_empty() {
projection = projection.try_map_exprs(|expr| {
replace_columns_with_literals(Arc::clone(&expr), &literal_columns)
})?;
predicate = predicate
.map(|p| replace_columns_with_literals(p, &literal_columns))
.transpose()?;
}
let reorder_predicates = self.reorder_filters;
let pushdown_filters = self.pushdown_filters;
let force_filter_selections = self.force_filter_selections;
let coerce_int96 = self.coerce_int96;
let enable_bloom_filter = self.enable_bloom_filter;
let enable_row_group_stats_pruning = self.enable_row_group_stats_pruning;
let limit = self.limit;
let predicate_creation_errors = MetricBuilder::new(&self.metrics)
.global_counter("num_predicate_creation_errors");
let expr_adapter_factory = Arc::clone(&self.expr_adapter_factory);
let enable_page_index = self.enable_page_index;
#[cfg(feature = "parquet_encryption")]
let encryption_context = self.get_encryption_context();
let max_predicate_cache_size = self.max_predicate_cache_size;
let reverse_row_groups = self.reverse_row_groups;
Ok(Box::pin(async move {
#[cfg(feature = "parquet_encryption")]
let file_decryption_properties = encryption_context
.get_file_decryption_properties(&file_location)
.await?;
// ---------------------------------------------
// Step: try to prune the current file partition
// ---------------------------------------------
// Prune this file using the file level statistics and partition values.
// Since dynamic filters may have been updated since planning it is possible that we are able
// to prune files now that we couldn't prune at planning time.
// It is assumed that there is no point in doing pruning here if the predicate is not dynamic,
// as it would have been done at planning time.
// We'll also check this after every record batch we read,
// and if at some point we are able to prove we can prune the file using just the file level statistics
// we can end the stream early.
let mut file_pruner = predicate
.as_ref()
.filter(|p| {
// Make a FilePruner only if there is either
// 1. a dynamic expr in the predicate
// 2. the file has file-level statistics.
//
// File-level statistics may prune the file without loading
// any row groups or metadata.
//
// Dynamic filters may prune the file after initial
// planning, as the dynamic filter is updated during
// execution.
//
// The case where there is a dynamic filter but no
// statistics corresponds to a dynamic filter that
// references partition columns. While rare, this is possible
// e.g. `select * from table order by partition_col limit
// 10` could hit this condition.
is_dynamic_physical_expr(p) || partitioned_file.has_statistics()
})
.and_then(|p| {
FilePruner::try_new(
Arc::clone(p),
&logical_file_schema,
&partitioned_file,
predicate_creation_errors.clone(),
)
});
if let Some(file_pruner) = &mut file_pruner
&& file_pruner.should_prune()?
{
// Return an empty stream immediately to skip the work of setting up the actual stream
file_metrics.files_ranges_pruned_statistics.add_pruned(1);
return Ok(futures::stream::empty().boxed());
}
file_metrics.files_ranges_pruned_statistics.add_matched(1);
// --------------------------------------------------------
// Step: fetch Parquet metadata (and optionally page index)
// --------------------------------------------------------
// Don't load the page index yet. Since it is not stored inline in
// the footer, loading the page index if it is not needed will do
// unnecessary I/O. We decide later if it is needed to evaluate the
// pruning predicates. Thus default to not requesting if from the
// underlying reader.
let mut options = ArrowReaderOptions::new().with_page_index(false);
#[cfg(feature = "parquet_encryption")]
if let Some(fd_val) = file_decryption_properties {
options = options.with_file_decryption_properties(Arc::clone(&fd_val));
}
let mut metadata_timer = file_metrics.metadata_load_time.timer();
// Begin by loading the metadata from the underlying reader (note
// the returned metadata may actually include page indexes as some
// readers may return page indexes even when not requested -- for
// example when they are cached)
let mut reader_metadata =
ArrowReaderMetadata::load_async(&mut async_file_reader, options.clone())
.await?;
// Note about schemas: we are actually dealing with **3 different schemas** here:
// - The table schema as defined by the TableProvider.
// This is what the user sees, what they get when they `SELECT * FROM table`, etc.
// - The logical file schema: this is the table schema minus any hive partition columns and projections.
// This is what the physical file schema is coerced to.
// - The physical file schema: this is the schema that the arrow-rs
// parquet reader will actually produce.
let mut physical_file_schema = Arc::clone(reader_metadata.schema());
// The schema loaded from the file may not be the same as the
// desired schema (for example if we want to instruct the parquet
// reader to read strings using Utf8View instead). Update if necessary
if let Some(merged) = apply_file_schema_type_coercions(
&logical_file_schema,
&physical_file_schema,
) {
physical_file_schema = Arc::new(merged);
options = options.with_schema(Arc::clone(&physical_file_schema));
reader_metadata = ArrowReaderMetadata::try_new(
Arc::clone(reader_metadata.metadata()),
options.clone(),
)?;
}
if let Some(ref coerce) = coerce_int96
&& let Some(merged) = coerce_int96_to_resolution(
reader_metadata.parquet_schema(),
&physical_file_schema,
coerce,
)
{
physical_file_schema = Arc::new(merged);
options = options.with_schema(Arc::clone(&physical_file_schema));
reader_metadata = ArrowReaderMetadata::try_new(
Arc::clone(reader_metadata.metadata()),
options.clone(),
)?;
}
// Adapt the projection & filter predicate to the physical file schema.
// This evaluates missing columns and inserts any necessary casts.
// After rewriting to the file schema, further simplifications may be possible.
// For example, if `'a' = col_that_is_missing` becomes `'a' = NULL` that can then be simplified to `FALSE`
// and we can avoid doing any more work on the file (bloom filters, loading the page index, etc.).
// Additionally, if any casts were inserted we can move casts from the column to the literal side:
// `CAST(col AS INT) = 5` can become `col = CAST(5 AS <col type>)`, which can be evaluated statically.
let rewriter = expr_adapter_factory.create(
Arc::clone(&logical_file_schema),
Arc::clone(&physical_file_schema),
);
let simplifier = PhysicalExprSimplifier::new(&physical_file_schema);
predicate = predicate
.map(|p| simplifier.simplify(rewriter.rewrite(p)?))
.transpose()?;
// Adapt projections to the physical file schema as well
projection = projection
.try_map_exprs(|p| simplifier.simplify(rewriter.rewrite(p)?))?;
// Build predicates for this specific file
let (pruning_predicate, page_pruning_predicate) = build_pruning_predicates(
predicate.as_ref(),
&physical_file_schema,
&predicate_creation_errors,
);
// The page index is not stored inline in the parquet footer so the
// code above may not have read the page index structures yet. If we
// need them for reading and they aren't yet loaded, we need to load them now.
if should_enable_page_index(enable_page_index, &page_pruning_predicate) {
reader_metadata = load_page_index(
reader_metadata,
&mut async_file_reader,
// Since we're manually loading the page index the option here should not matter but we pass it in for consistency
options.with_page_index(true),
)
.await?;
}
metadata_timer.stop();
// ---------------------------------------------------------
// Step: construct builder for the final RecordBatch stream
// ---------------------------------------------------------
let mut builder = ParquetRecordBatchStreamBuilder::new_with_metadata(
async_file_reader,
reader_metadata,
);
// ---------------------------------------------------------------------
// Step: optionally add row filter to the builder
//
// Row filter is used for late materialization in parquet decoding, see
// `row_filter` for details.
// ---------------------------------------------------------------------
// Filter pushdown: evaluate predicates during scan
if let Some(predicate) = pushdown_filters.then_some(predicate).flatten() {
let row_filter = row_filter::build_row_filter(
&predicate,
&physical_file_schema,
builder.metadata(),
reorder_predicates,
&file_metrics,
);
match row_filter {
Ok(Some(filter)) => {
builder = builder.with_row_filter(filter);
}
Ok(None) => {}
Err(e) => {
debug!(
"Ignoring error building row filter for '{predicate:?}': {e}"
);
}
};
};
if force_filter_selections {
builder =
builder.with_row_selection_policy(RowSelectionPolicy::Selectors);
}
// ------------------------------------------------------------
// Step: prune row groups by range, predicate and bloom filter
// ------------------------------------------------------------
// Determine which row groups to actually read. The idea is to skip
// as many row groups as possible based on the metadata and query
let file_metadata = Arc::clone(builder.metadata());
let predicate = pruning_predicate.as_ref().map(|p| p.as_ref());
let rg_metadata = file_metadata.row_groups();
// track which row groups to actually read
let access_plan =
create_initial_plan(&file_name, extensions, rg_metadata.len())?;
let mut row_groups = RowGroupAccessPlanFilter::new(access_plan);
// if there is a range restricting what parts of the file to read
if let Some(range) = file_range.as_ref() {
row_groups.prune_by_range(rg_metadata, range);
}
// If there is a predicate that can be evaluated against the metadata
if let Some(predicate) = predicate.as_ref() {
if enable_row_group_stats_pruning {
row_groups.prune_by_statistics(
&physical_file_schema,
builder.parquet_schema(),
rg_metadata,
predicate,
&file_metrics,
);
} else {
// Update metrics: statistics unavailable, so all row groups are
// matched (not pruned)
file_metrics
.row_groups_pruned_statistics
.add_matched(row_groups.remaining_row_group_count());
}
if enable_bloom_filter && !row_groups.is_empty() {
row_groups
.prune_by_bloom_filters(
&physical_file_schema,
&mut builder,
predicate,
&file_metrics,
)
.await;
} else {
// Update metrics: bloom filter unavailable, so all row groups are
// matched (not pruned)
file_metrics
.row_groups_pruned_bloom_filter
.add_matched(row_groups.remaining_row_group_count());
}
} else {
// Update metrics: no predicate, so all row groups are matched (not pruned)
let n_remaining_row_groups = row_groups.remaining_row_group_count();
file_metrics
.row_groups_pruned_statistics
.add_matched(n_remaining_row_groups);
file_metrics
.row_groups_pruned_bloom_filter
.add_matched(n_remaining_row_groups);
}
let mut access_plan = row_groups.build();
// --------------------------------------------------------
// Step: prune pages from the kept row groups
//
// page index pruning: if all data on individual pages can
// be ruled using page metadata, rows from other columns
// with that range can be skipped as well
// --------------------------------------------------------
if enable_page_index
&& !access_plan.is_empty()
&& let Some(p) = page_pruning_predicate
{
access_plan = p.prune_plan_with_page_index(
access_plan,
&physical_file_schema,
builder.parquet_schema(),
file_metadata.as_ref(),
&file_metrics,
);
}
// Prepare the access plan (extract row groups and row selection)
let mut prepared_plan =
PreparedAccessPlan::from_access_plan(access_plan, rg_metadata)?;
// ----------------------------------------------------------
// Step: potentially reverse the access plan for performance.
// See `ParquetSource::try_reverse_output` for the rationale.
// ----------------------------------------------------------
if reverse_row_groups {
prepared_plan = prepared_plan.reverse(file_metadata.as_ref())?;
}
// Apply the prepared plan to the builder
builder = prepared_plan.apply_to_builder(builder);
if let Some(limit) = limit {
builder = builder.with_limit(limit)
}
if let Some(max_predicate_cache_size) = max_predicate_cache_size {
builder = builder.with_max_predicate_cache_size(max_predicate_cache_size);
}
// metrics from the arrow reader itself
let arrow_reader_metrics = ArrowReaderMetrics::enabled();
let indices = projection.column_indices();
let mask = ProjectionMask::roots(builder.parquet_schema(), indices);
let stream = builder
.with_projection(mask)
.with_batch_size(batch_size)
.with_metrics(arrow_reader_metrics.clone())
.build()?;
let files_ranges_pruned_statistics =
file_metrics.files_ranges_pruned_statistics.clone();
let predicate_cache_inner_records =
file_metrics.predicate_cache_inner_records.clone();
let predicate_cache_records = file_metrics.predicate_cache_records.clone();
let stream_schema = Arc::clone(stream.schema());
// Check if we need to replace the schema to handle things like differing nullability or metadata.
// See note below about file vs. output schema.
let replace_schema = !stream_schema.eq(&output_schema);
// Rebase column indices to match the narrowed stream schema.
// The projection expressions have indices based on physical_file_schema,
// but the stream only contains the columns selected by the ProjectionMask.
let projection = projection
.try_map_exprs(|expr| reassign_expr_columns(expr, &stream_schema))?;
let projector = projection.make_projector(&stream_schema)?;
let stream = stream.map_err(DataFusionError::from).map(move |b| {
b.and_then(|mut b| {
copy_arrow_reader_metrics(
&arrow_reader_metrics,
&predicate_cache_inner_records,
&predicate_cache_records,
);
b = projector.project_batch(&b)?;
if replace_schema {
// Ensure the output batch has the expected schema.
// This handles things like schema level and field level metadata, which may not be present
// in the physical file schema.
// It is also possible for nullability to differ; some writers create files with
// OPTIONAL fields even when there are no nulls in the data.
// In these cases it may make sense for the logical schema to be `NOT NULL`.
// RecordBatch::try_new_with_options checks that if the schema is NOT NULL
// the array cannot contain nulls, amongst other checks.
let (_stream_schema, arrays, num_rows) = b.into_parts();
let options =
RecordBatchOptions::new().with_row_count(Some(num_rows));
RecordBatch::try_new_with_options(
Arc::clone(&output_schema),
arrays,
&options,
)
.map_err(Into::into)
} else {
Ok(b)
}
})
});
// ----------------------------------------------------------------------
// Step: wrap the stream so a dynamic filter can stop the file scan early
// ----------------------------------------------------------------------
if let Some(file_pruner) = file_pruner {
Ok(EarlyStoppingStream::new(
stream,
file_pruner,
files_ranges_pruned_statistics,
)
.boxed())
} else {
Ok(stream.boxed())
}
}))
}
}
/// Copies metrics from ArrowReaderMetrics (the metrics collected by the
/// arrow-rs parquet reader) to the parquet file metrics for DataFusion
fn copy_arrow_reader_metrics(
arrow_reader_metrics: &ArrowReaderMetrics,
predicate_cache_inner_records: &Count,
predicate_cache_records: &Count,
) {
if let Some(v) = arrow_reader_metrics.records_read_from_inner() {
predicate_cache_inner_records.add(v);
}
if let Some(v) = arrow_reader_metrics.records_read_from_cache() {
predicate_cache_records.add(v);
}
}
type ConstantColumns = HashMap<String, ScalarValue>;
/// Extract constant column values from statistics, keyed by column name in the logical file schema.
fn constant_columns_from_stats(
statistics: Option<&Statistics>,
file_schema: &SchemaRef,
) -> ConstantColumns {
let mut constants = HashMap::new();
let Some(statistics) = statistics else {
return constants;
};
let num_rows = match statistics.num_rows {
Precision::Exact(num_rows) => Some(num_rows),
_ => None,
};
for (idx, column_stats) in statistics
.column_statistics
.iter()
.take(file_schema.fields().len())
.enumerate()
{
let field = file_schema.field(idx);
if let Some(value) =
constant_value_from_stats(column_stats, num_rows, field.data_type())
{
constants.insert(field.name().clone(), value);
}
}
constants
}
fn constant_value_from_stats(
column_stats: &ColumnStatistics,
num_rows: Option<usize>,
data_type: &DataType,
) -> Option<ScalarValue> {
if let (Precision::Exact(min), Precision::Exact(max)) =
(&column_stats.min_value, &column_stats.max_value)
&& min == max
&& !min.is_null()
&& matches!(column_stats.null_count, Precision::Exact(0))
{
return Some(min.clone());
}
if let (Some(num_rows), Precision::Exact(nulls)) =
(num_rows, &column_stats.null_count)
&& *nulls == num_rows
{
return ScalarValue::try_new_null(data_type).ok();
}
None
}
/// Wraps an inner RecordBatchStream and a [`FilePruner`]
///
/// This can terminate the scan early when some dynamic filters is updated after
/// the scan starts, so we discover after the scan starts that the file can be
/// pruned (can't have matching rows).
struct EarlyStoppingStream<S> {
/// Has the stream finished processing? All subsequent polls will return
/// None
done: bool,
file_pruner: FilePruner,
files_ranges_pruned_statistics: PruningMetrics,
/// The inner stream
inner: S,
}
impl<S> EarlyStoppingStream<S> {
pub fn new(
stream: S,
file_pruner: FilePruner,
files_ranges_pruned_statistics: PruningMetrics,
) -> Self {
Self {
done: false,
inner: stream,
file_pruner,
files_ranges_pruned_statistics,
}
}
}
impl<S> EarlyStoppingStream<S>
where
S: Stream<Item = Result<RecordBatch>> + Unpin,
{
fn check_prune(&mut self, input: Result<RecordBatch>) -> Result<Option<RecordBatch>> {
let batch = input?;
// Since dynamic filters may have been updated, see if we can stop
// reading this stream entirely.
if self.file_pruner.should_prune()? {
self.files_ranges_pruned_statistics.add_pruned(1);
// Previously this file range has been counted as matched
self.files_ranges_pruned_statistics.subtract_matched(1);
self.done = true;
Ok(None)
} else {
// Return the adapted batch
Ok(Some(batch))
}
}
}
impl<S> Stream for EarlyStoppingStream<S>
where
S: Stream<Item = Result<RecordBatch>> + Unpin,
{
type Item = Result<RecordBatch>;
fn poll_next(
mut self: Pin<&mut Self>,
cx: &mut Context<'_>,
) -> Poll<Option<Self::Item>> {
if self.done {
return Poll::Ready(None);
}
match ready!(self.inner.poll_next_unpin(cx)) {
None => {
// input done
self.done = true;
Poll::Ready(None)
}
Some(input_batch) => {
let output = self.check_prune(input_batch);
Poll::Ready(output.transpose())
}
}
}
}
#[derive(Default)]
struct EncryptionContext {
#[cfg(feature = "parquet_encryption")]
file_decryption_properties: Option<Arc<FileDecryptionProperties>>,
#[cfg(feature = "parquet_encryption")]
encryption_factory: Option<(Arc<dyn EncryptionFactory>, EncryptionFactoryOptions)>,
}
#[cfg(feature = "parquet_encryption")]
impl EncryptionContext {
fn new(
file_decryption_properties: Option<Arc<FileDecryptionProperties>>,
encryption_factory: Option<(
Arc<dyn EncryptionFactory>,
EncryptionFactoryOptions,
)>,
) -> Self {
Self {
file_decryption_properties,
encryption_factory,
}
}
async fn get_file_decryption_properties(
&self,
file_location: &object_store::path::Path,
) -> Result<Option<Arc<FileDecryptionProperties>>> {
match &self.file_decryption_properties {
Some(file_decryption_properties) => {
Ok(Some(Arc::clone(file_decryption_properties)))
}
None => match &self.encryption_factory {
Some((encryption_factory, encryption_config)) => Ok(encryption_factory
.get_file_decryption_properties(encryption_config, file_location)
.await?),
None => Ok(None),
},
}
}
}
#[cfg(not(feature = "parquet_encryption"))]
#[expect(dead_code)]
impl EncryptionContext {
async fn get_file_decryption_properties(
&self,
_file_location: &object_store::path::Path,
) -> Result<Option<Arc<FileDecryptionProperties>>> {
Ok(None)
}
}
impl ParquetOpener {
#[cfg(feature = "parquet_encryption")]
fn get_encryption_context(&self) -> EncryptionContext {
EncryptionContext::new(
self.file_decryption_properties.clone(),
self.encryption_factory.clone(),
)
}
#[cfg(not(feature = "parquet_encryption"))]
#[expect(dead_code)]
fn get_encryption_context(&self) -> EncryptionContext {
EncryptionContext::default()
}
}
/// Return the initial [`ParquetAccessPlan`]
///
/// If the user has supplied one as an extension, use that
/// otherwise return a plan that scans all row groups
///
/// Returns an error if an invalid `ParquetAccessPlan` is provided
///
/// Note: file_name is only used for error messages
fn create_initial_plan(
file_name: &str,
extensions: Option<Arc<dyn std::any::Any + Send + Sync>>,
row_group_count: usize,
) -> Result<ParquetAccessPlan> {
if let Some(extensions) = extensions {
if let Some(access_plan) = extensions.downcast_ref::<ParquetAccessPlan>() {
let plan_len = access_plan.len();
if plan_len != row_group_count {
return exec_err!(
"Invalid ParquetAccessPlan for {file_name}. Specified {plan_len} row groups, but file has {row_group_count}"
);
}
// check row group count matches the plan
return Ok(access_plan.clone());
} else {
debug!("DataSourceExec Ignoring unknown extension specified for {file_name}");
}
}
// default to scanning all row groups
Ok(ParquetAccessPlan::new_all(row_group_count))
}
/// Build a page pruning predicate from an optional predicate expression.
/// If the predicate is None or the predicate cannot be converted to a page pruning
/// predicate, return None.
pub(crate) fn build_page_pruning_predicate(
predicate: &Arc<dyn PhysicalExpr>,
file_schema: &SchemaRef,
) -> Arc<PagePruningAccessPlanFilter> {
Arc::new(PagePruningAccessPlanFilter::new(
predicate,
Arc::clone(file_schema),
))
}
pub(crate) fn build_pruning_predicates(
predicate: Option<&Arc<dyn PhysicalExpr>>,
file_schema: &SchemaRef,
predicate_creation_errors: &Count,
) -> (
Option<Arc<PruningPredicate>>,
Option<Arc<PagePruningAccessPlanFilter>>,
) {
let Some(predicate) = predicate.as_ref() else {
return (None, None);
};
let pruning_predicate = build_pruning_predicate(
Arc::clone(predicate),
file_schema,
predicate_creation_errors,
);
let page_pruning_predicate = build_page_pruning_predicate(predicate, file_schema);
(pruning_predicate, Some(page_pruning_predicate))
}
/// Returns a `ArrowReaderMetadata` with the page index loaded, loading
/// it from the underlying `AsyncFileReader` if necessary.
async fn load_page_index<T: AsyncFileReader>(
reader_metadata: ArrowReaderMetadata,
input: &mut T,
options: ArrowReaderOptions,
) -> Result<ArrowReaderMetadata> {
let parquet_metadata = reader_metadata.metadata();
let missing_column_index = parquet_metadata.column_index().is_none();
let missing_offset_index = parquet_metadata.offset_index().is_none();
// You may ask yourself: why are we even checking if the page index is already loaded here?
// Didn't we explicitly *not* load it above?
// Well it's possible that a custom implementation of `AsyncFileReader` gives you
// the page index even if you didn't ask for it (e.g. because it's cached)
// so it's important to check that here to avoid extra work.
if missing_column_index || missing_offset_index {
let m = Arc::try_unwrap(Arc::clone(parquet_metadata))
.unwrap_or_else(|e| e.as_ref().clone());
let mut reader = ParquetMetaDataReader::new_with_metadata(m)
.with_page_index_policy(PageIndexPolicy::Optional);
reader.load_page_index(input).await?;
let new_parquet_metadata = reader.finish()?;
let new_arrow_reader =
ArrowReaderMetadata::try_new(Arc::new(new_parquet_metadata), options)?;
Ok(new_arrow_reader)
} else {
// No need to load the page index again, just return the existing metadata
Ok(reader_metadata)
}
}
fn should_enable_page_index(
enable_page_index: bool,
page_pruning_predicate: &Option<Arc<PagePruningAccessPlanFilter>>,
) -> bool {
enable_page_index
&& page_pruning_predicate.is_some()
&& page_pruning_predicate
.as_ref()
.map(|p| p.filter_number() > 0)
.unwrap_or(false)