forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathsource.rs
More file actions
782 lines (731 loc) · 32.5 KB
/
source.rs
File metadata and controls
782 lines (731 loc) · 32.5 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
// 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.
//! ParquetSource implementation for reading parquet files
use std::any::Any;
use std::fmt::Debug;
use std::fmt::Formatter;
use std::sync::Arc;
use crate::opener::build_pruning_predicates;
use crate::opener::ParquetOpener;
use crate::row_filter::can_expr_be_pushed_down_with_schemas;
use crate::DefaultParquetFileReaderFactory;
use crate::ParquetFileReaderFactory;
use datafusion_common::config::ConfigOptions;
#[cfg(feature = "parquet_encryption")]
use datafusion_common::config::EncryptionFactoryOptions;
use datafusion_datasource::as_file_source;
use datafusion_datasource::file_stream::FileOpener;
use datafusion_datasource::schema_adapter::{
DefaultSchemaAdapterFactory, SchemaAdapterFactory,
};
use arrow::datatypes::{SchemaRef, TimeUnit};
use datafusion_common::config::TableParquetOptions;
use datafusion_common::{DataFusionError, Statistics};
use datafusion_datasource::file::FileSource;
use datafusion_datasource::file_scan_config::FileScanConfig;
use datafusion_physical_expr::conjunction;
use datafusion_physical_expr_adapter::DefaultPhysicalExprAdapterFactory;
use datafusion_physical_expr_common::physical_expr::fmt_sql;
use datafusion_physical_expr_common::physical_expr::PhysicalExpr;
use datafusion_physical_plan::filter_pushdown::PushedDown;
use datafusion_physical_plan::filter_pushdown::{
FilterPushdownPropagation, PushedDownPredicate,
};
use datafusion_physical_plan::metrics::Count;
use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet;
use datafusion_physical_plan::DisplayFormatType;
#[cfg(feature = "parquet_encryption")]
use datafusion_common::encryption::map_config_decryption_to_decryption;
#[cfg(feature = "parquet_encryption")]
use datafusion_execution::parquet_encryption::EncryptionFactory;
use itertools::Itertools;
use object_store::ObjectStore;
/// Execution plan for reading one or more Parquet files.
///
/// ```text
/// ▲
/// │
/// │ Produce a stream of
/// │ RecordBatches
/// │
/// ┌───────────────────────┐
/// │ │
/// │ DataSourceExec │
/// │ │
/// └───────────────────────┘
/// ▲
/// │ Asynchronously read from one
/// │ or more parquet files via
/// │ ObjectStore interface
/// │
/// │
/// .───────────────────.
/// │ )
/// │`───────────────────'│
/// │ ObjectStore │
/// │.───────────────────.│
/// │ )
/// `───────────────────'
///
/// ```
///
/// # Example: Create a `DataSourceExec`
/// ```
/// # use std::sync::Arc;
/// # use arrow::datatypes::Schema;
/// # use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder};
/// # use datafusion_datasource_parquet::source::ParquetSource;
/// # use datafusion_datasource::PartitionedFile;
/// # use datafusion_execution::object_store::ObjectStoreUrl;
/// # use datafusion_physical_expr::expressions::lit;
/// # use datafusion_datasource::source::DataSourceExec;
/// # use datafusion_common::config::TableParquetOptions;
///
/// # let file_schema = Arc::new(Schema::empty());
/// # let object_store_url = ObjectStoreUrl::local_filesystem();
/// # let predicate = lit(true);
/// let source = Arc::new(
/// ParquetSource::default()
/// .with_predicate(predicate)
/// );
/// // Create a DataSourceExec for reading `file1.parquet` with a file size of 100MB
/// let config = FileScanConfigBuilder::new(object_store_url, file_schema, source)
/// .with_file(PartitionedFile::new("file1.parquet", 100*1024*1024)).build();
/// let exec = DataSourceExec::from_data_source(config);
/// ```
///
/// # Features
///
/// Supports the following optimizations:
///
/// * Concurrent reads: reads from one or more files in parallel as multiple
/// partitions, including concurrently reading multiple row groups from a single
/// file.
///
/// * Predicate push down: skips row groups, pages, rows based on metadata
/// and late materialization. See "Predicate Pushdown" below.
///
/// * Projection pushdown: reads and decodes only the columns required.
///
/// * Limit pushdown: stop execution early after some number of rows are read.
///
/// * Custom readers: customize reading parquet files, e.g. to cache metadata,
/// coalesce I/O operations, etc. See [`ParquetFileReaderFactory`] for more
/// details.
///
/// * Schema evolution: read parquet files with different schemas into a unified
/// table schema. See [`SchemaAdapterFactory`] for more details.
///
/// * metadata_size_hint: controls the number of bytes read from the end of the
/// file in the initial I/O when the default [`ParquetFileReaderFactory`]. If a
/// custom reader is used, it supplies the metadata directly and this parameter
/// is ignored. [`ParquetSource::with_metadata_size_hint`] for more details.
///
/// * User provided `ParquetAccessPlan`s to skip row groups and/or pages
/// based on external information. See "Implementing External Indexes" below
///
/// # Predicate Pushdown
///
/// `DataSourceExec` uses the provided [`PhysicalExpr`] predicate as a filter to
/// skip reading unnecessary data and improve query performance using several techniques:
///
/// * Row group pruning: skips entire row groups based on min/max statistics
/// found in [`ParquetMetaData`] and any Bloom filters that are present.
///
/// * Page pruning: skips individual pages within a ColumnChunk using the
/// [Parquet PageIndex], if present.
///
/// * Row filtering: skips rows within a page using a form of late
/// materialization. When possible, predicates are applied by the parquet
/// decoder *during* decode (see [`ArrowPredicate`] and [`RowFilter`] for more
/// details). This is only enabled if `ParquetScanOptions::pushdown_filters` is set to true.
///
/// Note: If the predicate can not be used to accelerate the scan, it is ignored
/// (no error is raised on predicate evaluation errors).
///
/// [`ArrowPredicate`]: parquet::arrow::arrow_reader::ArrowPredicate
/// [`RowFilter`]: parquet::arrow::arrow_reader::RowFilter
/// [Parquet PageIndex]: https://github.com/apache/parquet-format/blob/master/PageIndex.md
///
/// # Example: rewriting `DataSourceExec`
///
/// You can modify a `DataSourceExec` using [`ParquetSource`], for example
/// to change files or add a predicate.
///
/// ```no_run
/// # use std::sync::Arc;
/// # use arrow::datatypes::Schema;
/// # use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder};
/// # use datafusion_datasource::PartitionedFile;
/// # use datafusion_datasource::source::DataSourceExec;
///
/// # fn parquet_exec() -> DataSourceExec { unimplemented!() }
/// // Split a single DataSourceExec into multiple DataSourceExecs, one for each file
/// let exec = parquet_exec();
/// let data_source = exec.data_source();
/// let base_config = data_source.as_any().downcast_ref::<FileScanConfig>().unwrap();
/// let existing_file_groups = &base_config.file_groups;
/// let new_execs = existing_file_groups
/// .iter()
/// .map(|file_group| {
/// // create a new exec by copying the existing exec's source config
/// let new_config = FileScanConfigBuilder::from(base_config.clone())
/// .with_file_groups(vec![file_group.clone()])
/// .build();
///
/// (DataSourceExec::from_data_source(new_config))
/// })
/// .collect::<Vec<_>>();
/// ```
///
/// # Implementing External Indexes
///
/// It is possible to restrict the row groups and selections within those row
/// groups that the DataSourceExec will consider by providing an initial
/// `ParquetAccessPlan` as `extensions` on `PartitionedFile`. This can be
/// used to implement external indexes on top of parquet files and select only
/// portions of the files.
///
/// The `DataSourceExec` will try and reduce any provided `ParquetAccessPlan`
/// further based on the contents of `ParquetMetadata` and other settings.
///
/// ## Example of providing a ParquetAccessPlan
///
/// ```
/// # use std::sync::Arc;
/// # use arrow::datatypes::{Schema, SchemaRef};
/// # use datafusion_datasource::PartitionedFile;
/// # use datafusion_datasource_parquet::ParquetAccessPlan;
/// # use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder};
/// # use datafusion_datasource_parquet::source::ParquetSource;
/// # use datafusion_execution::object_store::ObjectStoreUrl;
/// # use datafusion_datasource::source::DataSourceExec;
///
/// # fn schema() -> SchemaRef {
/// # Arc::new(Schema::empty())
/// # }
/// // create an access plan to scan row group 0, 1 and 3 and skip row groups 2 and 4
/// let mut access_plan = ParquetAccessPlan::new_all(5);
/// access_plan.skip(2);
/// access_plan.skip(4);
/// // provide the plan as extension to the FileScanConfig
/// let partitioned_file = PartitionedFile::new("my_file.parquet", 1234)
/// .with_extensions(Arc::new(access_plan));
/// // create a FileScanConfig to scan this file
/// let config = FileScanConfigBuilder::new(ObjectStoreUrl::local_filesystem(), schema(), Arc::new(ParquetSource::default()))
/// .with_file(partitioned_file).build();
/// // this parquet DataSourceExec will not even try to read row groups 2 and 4. Additional
/// // pruning based on predicates may also happen
/// let exec = DataSourceExec::from_data_source(config);
/// ```
///
/// For a complete example, see the [`advanced_parquet_index` example]).
///
/// [`parquet_index_advanced` example]: https://github.com/apache/datafusion/blob/main/datafusion-examples/examples/advanced_parquet_index.rs
///
/// # Execution Overview
///
/// * Step 1: `DataSourceExec::execute` is called, returning a `FileStream`
/// configured to open parquet files with a `ParquetOpener`.
///
/// * Step 2: When the stream is polled, the `ParquetOpener` is called to open
/// the file.
///
/// * Step 3: The `ParquetOpener` gets the [`ParquetMetaData`] (file metadata)
/// via [`ParquetFileReaderFactory`], creating a `ParquetAccessPlan` by
/// applying predicates to metadata. The plan and projections are used to
/// determine what pages must be read.
///
/// * Step 4: The stream begins reading data, fetching the required parquet
/// pages incrementally decoding them, and applying any row filters (see
/// [`Self::with_pushdown_filters`]).
///
/// * Step 5: As each [`RecordBatch`] is read, it may be adapted by a
/// [`SchemaAdapter`] to match the table schema. By default missing columns are
/// filled with nulls, but this can be customized via [`SchemaAdapterFactory`].
///
/// [`RecordBatch`]: arrow::record_batch::RecordBatch
/// [`SchemaAdapter`]: datafusion_datasource::schema_adapter::SchemaAdapter
/// [`ParquetMetadata`]: parquet::file::metadata::ParquetMetaData
#[derive(Clone, Default, Debug)]
pub struct ParquetSource {
/// Options for reading Parquet files
pub(crate) table_parquet_options: TableParquetOptions,
/// Optional metrics
pub(crate) metrics: ExecutionPlanMetricsSet,
/// The schema of the file.
/// In particular, this is the schema of the table without partition columns,
/// *not* the physical schema of the file.
pub(crate) file_schema: Option<SchemaRef>,
/// Optional predicate for row filtering during parquet scan
pub(crate) predicate: Option<Arc<dyn PhysicalExpr>>,
/// Optional user defined parquet file reader factory
pub(crate) parquet_file_reader_factory: Option<Arc<dyn ParquetFileReaderFactory>>,
/// Optional user defined schema adapter
pub(crate) schema_adapter_factory: Option<Arc<dyn SchemaAdapterFactory>>,
/// Batch size configuration
pub(crate) batch_size: Option<usize>,
/// Optional hint for the size of the parquet metadata
pub(crate) metadata_size_hint: Option<usize>,
pub(crate) projected_statistics: Option<Statistics>,
#[cfg(feature = "parquet_encryption")]
pub(crate) encryption_factory: Option<Arc<dyn EncryptionFactory>>,
}
impl ParquetSource {
/// Create a new ParquetSource to read the data specified in the file scan
/// configuration with the provided `TableParquetOptions`.
/// if default values are going to be used, use `ParguetConfig::default()` instead
pub fn new(table_parquet_options: TableParquetOptions) -> Self {
Self {
table_parquet_options,
..Self::default()
}
}
/// Set the metadata size hint
///
/// This value determines how many bytes at the end of the file the default
/// [`ParquetFileReaderFactory`] will request in the initial IO. If this is
/// too small, the ParquetSource will need to make additional IO requests to
/// read the footer.
pub fn with_metadata_size_hint(mut self, metadata_size_hint: usize) -> Self {
self.metadata_size_hint = Some(metadata_size_hint);
self
}
/// Set predicate information
pub fn with_predicate(&self, predicate: Arc<dyn PhysicalExpr>) -> Self {
let mut conf = self.clone();
conf.predicate = Some(Arc::clone(&predicate));
conf
}
/// Set the encryption factory to use to generate file decryption properties
#[cfg(feature = "parquet_encryption")]
pub fn with_encryption_factory(
mut self,
encryption_factory: Arc<dyn EncryptionFactory>,
) -> Self {
self.encryption_factory = Some(encryption_factory);
self
}
/// Options passed to the parquet reader for this scan
pub fn table_parquet_options(&self) -> &TableParquetOptions {
&self.table_parquet_options
}
/// Optional predicate.
pub fn predicate(&self) -> Option<&Arc<dyn PhysicalExpr>> {
self.predicate.as_ref()
}
/// return the optional file reader factory
pub fn parquet_file_reader_factory(
&self,
) -> Option<&Arc<dyn ParquetFileReaderFactory>> {
self.parquet_file_reader_factory.as_ref()
}
/// Optional user defined parquet file reader factory.
///
pub fn with_parquet_file_reader_factory(
mut self,
parquet_file_reader_factory: Arc<dyn ParquetFileReaderFactory>,
) -> Self {
self.parquet_file_reader_factory = Some(parquet_file_reader_factory);
self
}
/// If true, the predicate will be used during the parquet scan.
/// Defaults to false.
pub fn with_pushdown_filters(mut self, pushdown_filters: bool) -> Self {
self.table_parquet_options.global.pushdown_filters = pushdown_filters;
self
}
/// Return the value described in [`Self::with_pushdown_filters`]
pub(crate) fn pushdown_filters(&self) -> bool {
self.table_parquet_options.global.pushdown_filters
}
/// If true, the `RowFilter` made by `pushdown_filters` may try to
/// minimize the cost of filter evaluation by reordering the
/// predicate [`Expr`]s. If false, the predicates are applied in
/// the same order as specified in the query. Defaults to false.
///
/// [`Expr`]: datafusion_expr::Expr
pub fn with_reorder_filters(mut self, reorder_filters: bool) -> Self {
self.table_parquet_options.global.reorder_filters = reorder_filters;
self
}
/// Return the value described in [`Self::with_reorder_filters`]
fn reorder_filters(&self) -> bool {
self.table_parquet_options.global.reorder_filters
}
/// If enabled, the reader will read the page index
/// This is used to optimize filter pushdown
/// via `RowSelector` and `RowFilter` by
/// eliminating unnecessary IO and decoding
pub fn with_enable_page_index(mut self, enable_page_index: bool) -> Self {
self.table_parquet_options.global.enable_page_index = enable_page_index;
self
}
/// Return the value described in [`Self::with_enable_page_index`]
fn enable_page_index(&self) -> bool {
self.table_parquet_options.global.enable_page_index
}
/// If enabled, the reader will read by the bloom filter
pub fn with_bloom_filter_on_read(mut self, bloom_filter_on_read: bool) -> Self {
self.table_parquet_options.global.bloom_filter_on_read = bloom_filter_on_read;
self
}
/// If enabled, the writer will write by the bloom filter
pub fn with_bloom_filter_on_write(
mut self,
enable_bloom_filter_on_write: bool,
) -> Self {
self.table_parquet_options.global.bloom_filter_on_write =
enable_bloom_filter_on_write;
self
}
/// Return the value described in [`Self::with_bloom_filter_on_read`]
fn bloom_filter_on_read(&self) -> bool {
self.table_parquet_options.global.bloom_filter_on_read
}
/// Return the maximum predicate cache size, in bytes, used when
/// `pushdown_filters`
pub fn max_predicate_cache_size(&self) -> Option<usize> {
self.table_parquet_options.global.max_predicate_cache_size
}
/// Applies schema adapter factory from the FileScanConfig if present.
///
/// # Arguments
/// * `conf` - FileScanConfig that may contain a schema adapter factory
/// # Returns
/// The converted FileSource with schema adapter factory applied if provided
pub fn apply_schema_adapter(
self,
conf: &FileScanConfig,
) -> datafusion_common::Result<Arc<dyn FileSource>> {
let file_source: Arc<dyn FileSource> = self.into();
// If the FileScanConfig.file_source() has a schema adapter factory, apply it
if let Some(factory) = conf.file_source().schema_adapter_factory() {
file_source.with_schema_adapter_factory(
Arc::<dyn SchemaAdapterFactory>::clone(&factory),
)
} else {
Ok(file_source)
}
}
#[cfg(feature = "parquet_encryption")]
fn get_encryption_factory_with_config(
&self,
) -> Option<(Arc<dyn EncryptionFactory>, EncryptionFactoryOptions)> {
match &self.encryption_factory {
None => None,
Some(factory) => Some((
Arc::clone(factory),
self.table_parquet_options.crypto.factory_options.clone(),
)),
}
}
}
/// Parses datafusion.common.config.ParquetOptions.coerce_int96 String to a arrow_schema.datatype.TimeUnit
pub(crate) fn parse_coerce_int96_string(
str_setting: &str,
) -> datafusion_common::Result<TimeUnit> {
let str_setting_lower: &str = &str_setting.to_lowercase();
match str_setting_lower {
"ns" => Ok(TimeUnit::Nanosecond),
"us" => Ok(TimeUnit::Microsecond),
"ms" => Ok(TimeUnit::Millisecond),
"s" => Ok(TimeUnit::Second),
_ => Err(DataFusionError::Configuration(format!(
"Unknown or unsupported parquet coerce_int96: \
{str_setting}. Valid values are: ns, us, ms, and s."
))),
}
}
/// Allows easy conversion from ParquetSource to Arc<dyn FileSource>
impl From<ParquetSource> for Arc<dyn FileSource> {
fn from(source: ParquetSource) -> Self {
as_file_source(source)
}
}
impl FileSource for ParquetSource {
fn create_file_opener(
&self,
object_store: Arc<dyn ObjectStore>,
base_config: &FileScanConfig,
partition: usize,
) -> Arc<dyn FileOpener> {
let projection = base_config
.file_column_projection_indices()
.unwrap_or_else(|| (0..base_config.file_schema.fields().len()).collect());
let (expr_adapter_factory, schema_adapter_factory) = match (
base_config.expr_adapter_factory.as_ref(),
self.schema_adapter_factory.as_ref(),
) {
(Some(expr_adapter_factory), Some(schema_adapter_factory)) => {
// Use both the schema adapter factory and the expr adapter factory.
// This results in the the SchemaAdapter being used for projections (e.g. a column was selected that is a UInt32 in the file and a UInt64 in the table schema)
// but the PhysicalExprAdapterFactory being used for predicate pushdown and stats pruning.
(
Some(Arc::clone(expr_adapter_factory)),
Arc::clone(schema_adapter_factory),
)
}
(Some(expr_adapter_factory), None) => {
// If no custom schema adapter factory is provided but an expr adapter factory is provided use the expr adapter factory alongside the default schema adapter factory.
// This means that the PhysicalExprAdapterFactory will be used for predicate pushdown and stats pruning, while the default schema adapter factory will be used for projections.
(
Some(Arc::clone(expr_adapter_factory)),
Arc::new(DefaultSchemaAdapterFactory) as _,
)
}
(None, Some(schema_adapter_factory)) => {
// If a custom schema adapter factory is provided but no expr adapter factory is provided use the custom SchemaAdapter for both projections and predicate pushdown.
// This maximizes compatibility with existing code that uses the SchemaAdapter API and did not explicitly opt into the PhysicalExprAdapterFactory API.
(None, Arc::clone(schema_adapter_factory) as _)
}
(None, None) => {
// If no custom schema adapter factory or expr adapter factory is provided, use the default schema adapter factory and the default physical expr adapter factory.
// This means that the default SchemaAdapter will be used for projections (e.g. a column was selected that is a UInt32 in the file and a UInt64 in the table schema)
// and the default PhysicalExprAdapterFactory will be used for predicate pushdown and stats pruning.
// This is the default behavior with not customization and means that most users of DataFusion will be cut over to the new PhysicalExprAdapterFactory API.
(
Some(Arc::new(DefaultPhysicalExprAdapterFactory) as _),
Arc::new(DefaultSchemaAdapterFactory) as _,
)
}
};
let parquet_file_reader_factory =
self.parquet_file_reader_factory.clone().unwrap_or_else(|| {
Arc::new(DefaultParquetFileReaderFactory::new(object_store)) as _
});
#[cfg(feature = "parquet_encryption")]
let file_decryption_properties = self
.table_parquet_options()
.crypto
.file_decryption
.as_ref()
.map(map_config_decryption_to_decryption)
.map(Arc::new);
let coerce_int96 = self
.table_parquet_options
.global
.coerce_int96
.as_ref()
.map(|time_unit| parse_coerce_int96_string(time_unit.as_str()).unwrap());
Arc::new(ParquetOpener {
partition_index: partition,
projection: Arc::from(projection),
batch_size: self
.batch_size
.expect("Batch size must set before creating ParquetOpener"),
limit: base_config.limit,
predicate: self.predicate.clone(),
logical_file_schema: Arc::clone(&base_config.file_schema),
partition_fields: base_config.table_partition_cols.clone(),
metadata_size_hint: self.metadata_size_hint,
metrics: self.metrics().clone(),
parquet_file_reader_factory,
pushdown_filters: self.pushdown_filters(),
reorder_filters: self.reorder_filters(),
enable_page_index: self.enable_page_index(),
enable_bloom_filter: self.bloom_filter_on_read(),
enable_row_group_stats_pruning: self.table_parquet_options.global.pruning,
schema_adapter_factory,
coerce_int96,
#[cfg(feature = "parquet_encryption")]
file_decryption_properties,
expr_adapter_factory,
#[cfg(feature = "parquet_encryption")]
encryption_factory: self.get_encryption_factory_with_config(),
max_predicate_cache_size: self.max_predicate_cache_size(),
})
}
fn as_any(&self) -> &dyn Any {
self
}
fn filter(&self) -> Option<Arc<dyn PhysicalExpr>> {
self.predicate.clone()
}
fn with_batch_size(&self, batch_size: usize) -> Arc<dyn FileSource> {
let mut conf = self.clone();
conf.batch_size = Some(batch_size);
Arc::new(conf)
}
fn with_schema(&self, schema: SchemaRef) -> Arc<dyn FileSource> {
Arc::new(Self {
file_schema: Some(schema),
..self.clone()
})
}
fn with_statistics(&self, statistics: Statistics) -> Arc<dyn FileSource> {
let mut conf = self.clone();
conf.projected_statistics = Some(statistics);
Arc::new(conf)
}
fn with_projection(&self, _config: &FileScanConfig) -> Arc<dyn FileSource> {
Arc::new(Self { ..self.clone() })
}
fn metrics(&self) -> &ExecutionPlanMetricsSet {
&self.metrics
}
fn statistics(&self) -> datafusion_common::Result<Statistics> {
let statistics = &self.projected_statistics;
let statistics = statistics
.clone()
.expect("projected_statistics must be set");
// When filters are pushed down, we have no way of knowing the exact statistics.
// Note that pruning predicate is also a kind of filter pushdown.
// (bloom filters use `pruning_predicate` too).
// Because filter pushdown may happen dynamically as long as there is a predicate
// if we have *any* predicate applied, we can't guarantee the statistics are exact.
if self.predicate().is_some() {
Ok(statistics.to_inexact())
} else {
Ok(statistics)
}
}
fn file_type(&self) -> &str {
"parquet"
}
fn fmt_extra(&self, t: DisplayFormatType, f: &mut Formatter) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
let predicate_string = self
.predicate()
.map(|p| format!(", predicate={p}"))
.unwrap_or_default();
write!(f, "{predicate_string}")?;
// Try to build a the pruning predicates.
// These are only generated here because it's useful to have *some*
// idea of what pushdown is happening when viewing plans.
// However it is important to note that these predicates are *not*
// necessarily the predicates that are actually evaluated:
// the actual predicates are built in reference to the physical schema of
// each file, which we do not have at this point and hence cannot use.
// Instead we use the logical schema of the file (the table schema without partition columns).
if let (Some(file_schema), Some(predicate)) =
(&self.file_schema, &self.predicate)
{
let predicate_creation_errors = Count::new();
if let (Some(pruning_predicate), _) = build_pruning_predicates(
Some(predicate),
file_schema,
&predicate_creation_errors,
) {
let mut guarantees = pruning_predicate
.literal_guarantees()
.iter()
.map(|item| format!("{item}"))
.collect_vec();
guarantees.sort();
write!(
f,
", pruning_predicate={}, required_guarantees=[{}]",
pruning_predicate.predicate_expr(),
guarantees.join(", ")
)?;
}
};
Ok(())
}
DisplayFormatType::TreeRender => {
if let Some(predicate) = self.predicate() {
writeln!(f, "predicate={}", fmt_sql(predicate.as_ref()))?;
}
Ok(())
}
}
}
fn try_pushdown_filters(
&self,
filters: Vec<Arc<dyn PhysicalExpr>>,
config: &ConfigOptions,
) -> datafusion_common::Result<FilterPushdownPropagation<Arc<dyn FileSource>>> {
let Some(file_schema) = self.file_schema.clone() else {
return Ok(FilterPushdownPropagation::with_parent_pushdown_result(
vec![PushedDown::No; filters.len()],
));
};
// Determine if based on configs we should push filters down.
// If either the table / scan itself or the config has pushdown enabled,
// we will push down the filters.
// If both are disabled, we will not push down the filters.
// By default they are both disabled.
// Regardless of pushdown, we will update the predicate to include the filters
// because even if scan pushdown is disabled we can still use the filters for stats pruning.
let config_pushdown_enabled = config.execution.parquet.pushdown_filters;
let table_pushdown_enabled = self.pushdown_filters();
let pushdown_filters = table_pushdown_enabled || config_pushdown_enabled;
let mut source = self.clone();
let filters: Vec<PushedDownPredicate> = filters
.into_iter()
.map(|filter| {
if can_expr_be_pushed_down_with_schemas(&filter, &file_schema) {
PushedDownPredicate::supported(filter)
} else {
PushedDownPredicate::unsupported(filter)
}
})
.collect();
if filters
.iter()
.all(|f| matches!(f.discriminant, PushedDown::No))
{
// No filters can be pushed down, so we can just return the remaining filters
// and avoid replacing the source in the physical plan.
return Ok(FilterPushdownPropagation::with_parent_pushdown_result(
vec![PushedDown::No; filters.len()],
));
}
let allowed_filters = filters
.iter()
.filter_map(|f| match f.discriminant {
PushedDown::Yes => Some(Arc::clone(&f.predicate)),
PushedDown::No => None,
})
.collect_vec();
let predicate = match source.predicate {
Some(predicate) => {
conjunction(std::iter::once(predicate).chain(allowed_filters))
}
None => conjunction(allowed_filters),
};
source.predicate = Some(predicate);
source = source.with_pushdown_filters(pushdown_filters);
let source = Arc::new(source);
// If pushdown_filters is false we tell our parents that they still have to handle the filters,
// even if we updated the predicate to include the filters (they will only be used for stats pruning).
if !pushdown_filters {
return Ok(FilterPushdownPropagation::with_parent_pushdown_result(
vec![PushedDown::No; filters.len()],
)
.with_updated_node(source));
}
Ok(FilterPushdownPropagation::with_parent_pushdown_result(
filters.iter().map(|f| f.discriminant).collect(),
)
.with_updated_node(source))
}
fn with_schema_adapter_factory(
&self,
schema_adapter_factory: Arc<dyn SchemaAdapterFactory>,
) -> datafusion_common::Result<Arc<dyn FileSource>> {
Ok(Arc::new(Self {
schema_adapter_factory: Some(schema_adapter_factory),
..self.clone()
}))
}
fn schema_adapter_factory(&self) -> Option<Arc<dyn SchemaAdapterFactory>> {
self.schema_adapter_factory.clone()
}
}