forked from apache/datafusion
-
Notifications
You must be signed in to change notification settings - Fork 1
Expand file tree
/
Copy pathfile_format.rs
More file actions
1992 lines (1819 loc) · 74.5 KB
/
file_format.rs
File metadata and controls
1992 lines (1819 loc) · 74.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
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.
//! [`ParquetFormat`]: Parquet [`FileFormat`] abstractions
use std::any::Any;
use std::cell::RefCell;
use std::fmt::Debug;
use std::ops::Range;
use std::rc::Rc;
use std::sync::Arc;
use std::{fmt, vec};
use arrow::array::RecordBatch;
use arrow::datatypes::{Fields, Schema, SchemaRef, TimeUnit};
use datafusion_datasource::TableSchema;
use datafusion_datasource::file_compression_type::FileCompressionType;
use datafusion_datasource::file_sink_config::{FileSink, FileSinkConfig};
use datafusion_datasource::write::{
ObjectWriterBuilder, SharedBuffer, get_writer_schema,
};
use datafusion_datasource::file_format::{FileFormat, FileFormatFactory};
use datafusion_datasource::write::demux::DemuxedStreamReceiver;
use arrow::datatypes::{DataType, Field, FieldRef};
use datafusion_common::config::{ConfigField, ConfigFileType, TableParquetOptions};
use datafusion_common::encryption::FileDecryptionProperties;
use datafusion_common::parsers::CompressionTypeVariant;
use datafusion_common::{
DEFAULT_PARQUET_EXTENSION, DataFusionError, GetExt, HashSet, Result,
internal_datafusion_err, internal_err, not_impl_err,
};
use datafusion_common::{HashMap, Statistics};
use datafusion_common_runtime::{JoinSet, SpawnedTask};
use datafusion_datasource::display::FileGroupDisplay;
use datafusion_datasource::file::FileSource;
use datafusion_datasource::file_scan_config::{FileScanConfig, FileScanConfigBuilder};
use datafusion_datasource::sink::{DataSink, DataSinkExec};
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation};
use datafusion_execution::{SendableRecordBatchStream, TaskContext};
use datafusion_expr::dml::InsertOp;
use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement};
use datafusion_physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan};
use datafusion_session::Session;
use crate::metadata::{DFParquetMetadata, lex_ordering_to_sorting_columns};
use crate::reader::CachedParquetFileReaderFactory;
use crate::source::{ParquetSource, parse_coerce_int96_string};
use async_trait::async_trait;
use bytes::Bytes;
use datafusion_datasource::source::DataSourceExec;
use datafusion_execution::cache::cache_manager::FileMetadataCache;
use datafusion_execution::runtime_env::RuntimeEnv;
use futures::future::BoxFuture;
use futures::{FutureExt, StreamExt, TryStreamExt};
use object_store::buffered::BufWriter;
use object_store::path::Path;
use object_store::{ObjectMeta, ObjectStore};
use parquet::arrow::arrow_writer::{
ArrowColumnChunk, ArrowColumnWriter, ArrowLeafColumn, ArrowRowGroupWriterFactory,
ArrowWriterOptions, compute_leaves,
};
use parquet::arrow::async_reader::MetadataFetch;
use parquet::arrow::{ArrowWriter, AsyncArrowWriter};
use parquet::basic::Type;
#[cfg(feature = "parquet_encryption")]
use parquet::encryption::encrypt::FileEncryptionProperties;
use parquet::errors::ParquetError;
use parquet::file::metadata::{ParquetMetaData, SortingColumn};
use parquet::file::properties::{WriterProperties, WriterPropertiesBuilder};
use parquet::file::writer::SerializedFileWriter;
use parquet::schema::types::SchemaDescriptor;
use tokio::io::{AsyncWrite, AsyncWriteExt};
use tokio::sync::mpsc::{self, Receiver, Sender};
/// Initial writing buffer size. Note this is just a size hint for efficiency. It
/// will grow beyond the set value if needed.
const INITIAL_BUFFER_BYTES: usize = 1048576;
/// When writing parquet files in parallel, if the buffered Parquet data exceeds
/// this size, it is flushed to object store
const BUFFER_FLUSH_BYTES: usize = 1024000;
#[derive(Default)]
/// Factory struct used to create [ParquetFormat]
pub struct ParquetFormatFactory {
/// inner options for parquet
pub options: Option<TableParquetOptions>,
}
impl ParquetFormatFactory {
/// Creates an instance of [ParquetFormatFactory]
pub fn new() -> Self {
Self { options: None }
}
/// Creates an instance of [ParquetFormatFactory] with customized default options
pub fn new_with_options(options: TableParquetOptions) -> Self {
Self {
options: Some(options),
}
}
}
impl FileFormatFactory for ParquetFormatFactory {
fn create(
&self,
state: &dyn Session,
format_options: &std::collections::HashMap<String, String>,
) -> Result<Arc<dyn FileFormat>> {
let parquet_options = match &self.options {
None => {
let mut table_options = state.default_table_options();
table_options.set_config_format(ConfigFileType::PARQUET);
table_options.alter_with_string_hash_map(format_options)?;
table_options.parquet
}
Some(parquet_options) => {
let mut parquet_options = parquet_options.clone();
for (k, v) in format_options {
parquet_options.set(k, v)?;
}
parquet_options
}
};
Ok(Arc::new(
ParquetFormat::default().with_options(parquet_options),
))
}
fn default(&self) -> Arc<dyn FileFormat> {
Arc::new(ParquetFormat::default())
}
fn as_any(&self) -> &dyn Any {
self
}
}
impl GetExt for ParquetFormatFactory {
fn get_ext(&self) -> String {
// Removes the dot, i.e. ".parquet" -> "parquet"
DEFAULT_PARQUET_EXTENSION[1..].to_string()
}
}
impl Debug for ParquetFormatFactory {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
f.debug_struct("ParquetFormatFactory")
.field("ParquetFormatFactory", &self.options)
.finish()
}
}
/// The Apache Parquet `FileFormat` implementation
#[derive(Debug, Default)]
pub struct ParquetFormat {
options: TableParquetOptions,
}
impl ParquetFormat {
/// Construct a new Format with no local overrides
pub fn new() -> Self {
Self::default()
}
/// Activate statistics based row group level pruning
/// - If `None`, defaults to value on `config_options`
pub fn with_enable_pruning(mut self, enable: bool) -> Self {
self.options.global.pruning = enable;
self
}
/// Return `true` if pruning is enabled
pub fn enable_pruning(&self) -> bool {
self.options.global.pruning
}
/// Provide a hint to the size of the file metadata. If a hint is provided
/// the reader will try and fetch the last `size_hint` bytes of the parquet file optimistically.
/// Without a hint, two read are required. One read to fetch the 8-byte parquet footer and then
/// another read to fetch the metadata length encoded in the footer.
///
/// - If `None`, defaults to value on `config_options`
pub fn with_metadata_size_hint(mut self, size_hint: Option<usize>) -> Self {
self.options.global.metadata_size_hint = size_hint;
self
}
/// Return the metadata size hint if set
pub fn metadata_size_hint(&self) -> Option<usize> {
self.options.global.metadata_size_hint
}
/// Tell the parquet reader to skip any metadata that may be in
/// the file Schema. This can help avoid schema conflicts due to
/// metadata.
///
/// - If `None`, defaults to value on `config_options`
pub fn with_skip_metadata(mut self, skip_metadata: bool) -> Self {
self.options.global.skip_metadata = skip_metadata;
self
}
/// Returns `true` if schema metadata will be cleared prior to
/// schema merging.
pub fn skip_metadata(&self) -> bool {
self.options.global.skip_metadata
}
/// Set Parquet options for the ParquetFormat
pub fn with_options(mut self, options: TableParquetOptions) -> Self {
self.options = options;
self
}
/// Parquet options
pub fn options(&self) -> &TableParquetOptions {
&self.options
}
/// Return `true` if should use view types.
///
/// If this returns true, DataFusion will instruct the parquet reader
/// to read string / binary columns using view `StringView` or `BinaryView`
/// if the table schema specifies those types, regardless of any embedded metadata
/// that may specify an alternate Arrow type. The parquet reader is optimized
/// for reading `StringView` and `BinaryView` and such queries are significantly faster.
///
/// If this returns false, the parquet reader will read the columns according to the
/// defaults or any embedded Arrow type information. This may result in reading
/// `StringArrays` and then casting to `StringViewArray` which is less efficient.
pub fn force_view_types(&self) -> bool {
self.options.global.schema_force_view_types
}
/// If true, will use view types. See [`Self::force_view_types`] for details
pub fn with_force_view_types(mut self, use_views: bool) -> Self {
self.options.global.schema_force_view_types = use_views;
self
}
/// Return `true` if binary types will be read as strings.
///
/// If this returns true, DataFusion will instruct the parquet reader
/// to read binary columns such as `Binary` or `BinaryView` as the
/// corresponding string type such as `Utf8` or `LargeUtf8`.
/// The parquet reader has special optimizations for `Utf8` and `LargeUtf8`
/// validation, and such queries are significantly faster than reading
/// binary columns and then casting to string columns.
pub fn binary_as_string(&self) -> bool {
self.options.global.binary_as_string
}
/// If true, will read binary types as strings. See [`Self::binary_as_string`] for details
pub fn with_binary_as_string(mut self, binary_as_string: bool) -> Self {
self.options.global.binary_as_string = binary_as_string;
self
}
pub fn coerce_int96(&self) -> Option<String> {
self.options.global.coerce_int96.clone()
}
pub fn with_coerce_int96(mut self, time_unit: Option<String>) -> Self {
self.options.global.coerce_int96 = time_unit;
self
}
}
/// Clears all metadata (Schema level and field level) on an iterator
/// of Schemas
fn clear_metadata(
schemas: impl IntoIterator<Item = Schema>,
) -> impl Iterator<Item = Schema> {
schemas.into_iter().map(|schema| {
let fields = schema
.fields()
.iter()
.map(|field| {
field.as_ref().clone().with_metadata(Default::default()) // clear meta
})
.collect::<Fields>();
Schema::new(fields)
})
}
#[cfg(feature = "parquet_encryption")]
async fn get_file_decryption_properties(
state: &dyn Session,
options: &TableParquetOptions,
file_path: &Path,
) -> Result<Option<Arc<FileDecryptionProperties>>> {
Ok(match &options.crypto.file_decryption {
Some(cfd) => Some(Arc::new(FileDecryptionProperties::from(cfd.clone()))),
None => match &options.crypto.factory_id {
Some(factory_id) => {
let factory =
state.runtime_env().parquet_encryption_factory(factory_id)?;
factory
.get_file_decryption_properties(
&options.crypto.factory_options,
file_path,
)
.await?
}
None => None,
},
})
}
#[cfg(not(feature = "parquet_encryption"))]
async fn get_file_decryption_properties(
_state: &dyn Session,
_options: &TableParquetOptions,
_file_path: &Path,
) -> Result<Option<Arc<FileDecryptionProperties>>> {
Ok(None)
}
#[async_trait]
impl FileFormat for ParquetFormat {
fn as_any(&self) -> &dyn Any {
self
}
fn get_ext(&self) -> String {
ParquetFormatFactory::new().get_ext()
}
fn get_ext_with_compression(
&self,
file_compression_type: &FileCompressionType,
) -> Result<String> {
let ext = self.get_ext();
match file_compression_type.get_variant() {
CompressionTypeVariant::UNCOMPRESSED => Ok(ext),
_ => internal_err!("Parquet FileFormat does not support compression."),
}
}
fn compression_type(&self) -> Option<FileCompressionType> {
None
}
async fn infer_schema(
&self,
state: &dyn Session,
store: &Arc<dyn ObjectStore>,
objects: &[ObjectMeta],
) -> Result<SchemaRef> {
let coerce_int96 = match self.coerce_int96() {
Some(time_unit) => Some(parse_coerce_int96_string(time_unit.as_str())?),
None => None,
};
let file_metadata_cache =
state.runtime_env().cache_manager.get_file_metadata_cache();
let mut schemas: Vec<_> = futures::stream::iter(objects)
.map(|object| async {
let file_decryption_properties = get_file_decryption_properties(
state,
&self.options,
&object.location,
)
.await?;
let result = DFParquetMetadata::new(store.as_ref(), object)
.with_metadata_size_hint(self.metadata_size_hint())
.with_decryption_properties(file_decryption_properties)
.with_file_metadata_cache(Some(Arc::clone(&file_metadata_cache)))
.with_coerce_int96(coerce_int96)
.fetch_schema_with_location()
.await?;
Ok::<_, DataFusionError>(result)
})
.boxed() // Workaround https://github.com/rust-lang/rust/issues/64552
// fetch schemas concurrently, if requested
.buffer_unordered(state.config_options().execution.meta_fetch_concurrency)
.try_collect()
.await?;
// Schema inference adds fields based the order they are seen
// which depends on the order the files are processed. For some
// object stores (like local file systems) the order returned from list
// is not deterministic. Thus, to ensure deterministic schema inference
// sort the files first.
// https://github.com/apache/datafusion/pull/6629
schemas
.sort_unstable_by(|(location1, _), (location2, _)| location1.cmp(location2));
let schemas = schemas.into_iter().map(|(_, schema)| schema);
let schema = if self.skip_metadata() {
Schema::try_merge(clear_metadata(schemas))
} else {
Schema::try_merge(schemas)
}?;
let schema = if self.binary_as_string() {
transform_binary_to_string(&schema)
} else {
schema
};
let schema = if self.force_view_types() {
transform_schema_to_view(&schema)
} else {
schema
};
Ok(Arc::new(schema))
}
async fn infer_stats(
&self,
state: &dyn Session,
store: &Arc<dyn ObjectStore>,
table_schema: SchemaRef,
object: &ObjectMeta,
) -> Result<Statistics> {
let file_decryption_properties =
get_file_decryption_properties(state, &self.options, &object.location)
.await?;
let file_metadata_cache =
state.runtime_env().cache_manager.get_file_metadata_cache();
DFParquetMetadata::new(store, object)
.with_metadata_size_hint(self.metadata_size_hint())
.with_decryption_properties(file_decryption_properties)
.with_file_metadata_cache(Some(file_metadata_cache))
.fetch_statistics(&table_schema)
.await
}
async fn infer_ordering(
&self,
state: &dyn Session,
store: &Arc<dyn ObjectStore>,
table_schema: SchemaRef,
object: &ObjectMeta,
) -> Result<Option<LexOrdering>> {
let file_decryption_properties =
get_file_decryption_properties(state, &self.options, &object.location)
.await?;
let file_metadata_cache =
state.runtime_env().cache_manager.get_file_metadata_cache();
let metadata = DFParquetMetadata::new(store, object)
.with_metadata_size_hint(self.metadata_size_hint())
.with_decryption_properties(file_decryption_properties)
.with_file_metadata_cache(Some(file_metadata_cache))
.fetch_metadata()
.await?;
crate::metadata::ordering_from_parquet_metadata(&metadata, &table_schema)
}
async fn infer_stats_and_ordering(
&self,
state: &dyn Session,
store: &Arc<dyn ObjectStore>,
table_schema: SchemaRef,
object: &ObjectMeta,
) -> Result<datafusion_datasource::file_format::FileMeta> {
let file_decryption_properties =
get_file_decryption_properties(state, &self.options, &object.location)
.await?;
let file_metadata_cache =
state.runtime_env().cache_manager.get_file_metadata_cache();
let metadata = DFParquetMetadata::new(store, object)
.with_metadata_size_hint(self.metadata_size_hint())
.with_decryption_properties(file_decryption_properties)
.with_file_metadata_cache(Some(file_metadata_cache))
.fetch_metadata()
.await?;
let statistics = DFParquetMetadata::statistics_from_parquet_metadata(
&metadata,
&table_schema,
)?;
let ordering =
crate::metadata::ordering_from_parquet_metadata(&metadata, &table_schema)?;
Ok(
datafusion_datasource::file_format::FileMeta::new(statistics)
.with_ordering(ordering),
)
}
async fn create_physical_plan(
&self,
state: &dyn Session,
conf: FileScanConfig,
) -> Result<Arc<dyn ExecutionPlan>> {
let mut metadata_size_hint = None;
if let Some(metadata) = self.metadata_size_hint() {
metadata_size_hint = Some(metadata);
}
let mut source = conf
.file_source()
.as_any()
.downcast_ref::<ParquetSource>()
.cloned()
.ok_or_else(|| internal_datafusion_err!("Expected ParquetSource"))?;
source = source.with_table_parquet_options(self.options.clone());
// Use the CachedParquetFileReaderFactory
let metadata_cache = state.runtime_env().cache_manager.get_file_metadata_cache();
let store = state
.runtime_env()
.object_store(conf.object_store_url.clone())?;
let cached_parquet_read_factory =
Arc::new(CachedParquetFileReaderFactory::new(store, metadata_cache));
source = source.with_parquet_file_reader_factory(cached_parquet_read_factory);
if let Some(metadata_size_hint) = metadata_size_hint {
source = source.with_metadata_size_hint(metadata_size_hint)
}
source = self.set_source_encryption_factory(source, state)?;
let conf = FileScanConfigBuilder::from(conf)
.with_source(Arc::new(source))
.with_morsel_driven(self.options.global.allow_morsel_driven)
.build();
Ok(DataSourceExec::from_data_source(conf))
}
async fn create_writer_physical_plan(
&self,
input: Arc<dyn ExecutionPlan>,
_state: &dyn Session,
conf: FileSinkConfig,
order_requirements: Option<LexRequirement>,
) -> Result<Arc<dyn ExecutionPlan>> {
if conf.insert_op != InsertOp::Append {
return not_impl_err!("Overwrites are not implemented yet for Parquet");
}
// Convert ordering requirements to Parquet SortingColumns for file metadata
let sorting_columns = if let Some(ref requirements) = order_requirements {
let ordering: LexOrdering = requirements.clone().into();
// In cases like `COPY (... ORDER BY ...) TO ...` the ORDER BY clause
// may not be compatible with Parquet sorting columns (e.g. ordering on `random()`).
// So if we cannot create a Parquet sorting column from the ordering requirement,
// we skip setting sorting columns on the Parquet sink.
lex_ordering_to_sorting_columns(&ordering).ok()
} else {
None
};
let sink = Arc::new(
ParquetSink::new(conf, self.options.clone())
.with_sorting_columns(sorting_columns),
);
Ok(Arc::new(DataSinkExec::new(input, sink, order_requirements)) as _)
}
fn file_source(&self, table_schema: TableSchema) -> Arc<dyn FileSource> {
Arc::new(
ParquetSource::new(table_schema)
.with_table_parquet_options(self.options.clone()),
)
}
}
#[cfg(feature = "parquet_encryption")]
impl ParquetFormat {
fn set_source_encryption_factory(
&self,
source: ParquetSource,
state: &dyn Session,
) -> Result<ParquetSource> {
if let Some(encryption_factory_id) = &self.options.crypto.factory_id {
Ok(source.with_encryption_factory(
state
.runtime_env()
.parquet_encryption_factory(encryption_factory_id)?,
))
} else {
Ok(source)
}
}
}
#[cfg(not(feature = "parquet_encryption"))]
impl ParquetFormat {
fn set_source_encryption_factory(
&self,
source: ParquetSource,
_state: &dyn Session,
) -> Result<ParquetSource> {
if let Some(encryption_factory_id) = &self.options.crypto.factory_id {
Err(DataFusionError::Configuration(format!(
"Parquet encryption factory id is set to '{encryption_factory_id}' but the parquet_encryption feature is disabled"
)))
} else {
Ok(source)
}
}
}
/// Apply necessary schema type coercions to make file schema match table schema.
///
/// This function performs two main types of transformations in a single pass:
/// 1. Binary types to string types conversion - Converts binary data types to their
/// corresponding string types when the table schema expects string data
/// 2. Regular to view types conversion - Converts standard string/binary types to
/// view types when the table schema uses view types
///
/// # Arguments
/// * `table_schema` - The table schema containing the desired types
/// * `file_schema` - The file schema to be transformed
///
/// # Returns
/// * `Some(Schema)` - If any transformations were applied, returns the transformed schema
/// * `None` - If no transformations were needed
pub fn apply_file_schema_type_coercions(
table_schema: &Schema,
file_schema: &Schema,
) -> Option<Schema> {
let mut needs_view_transform = false;
let mut needs_string_transform = false;
// Create a mapping of table field names to their data types for fast lookup
// and simultaneously check if we need any transformations
let table_fields: HashMap<_, _> = table_schema
.fields()
.iter()
.map(|f| {
let dt = f.data_type();
// Check if we need view type transformation
if matches!(dt, &DataType::Utf8View | &DataType::BinaryView) {
needs_view_transform = true;
}
// Check if we need string type transformation
if matches!(
dt,
&DataType::Utf8 | &DataType::LargeUtf8 | &DataType::Utf8View
) {
needs_string_transform = true;
}
(f.name(), dt)
})
.collect();
// Early return if no transformation needed
if !needs_view_transform && !needs_string_transform {
return None;
}
let transformed_fields: Vec<Arc<Field>> = file_schema
.fields()
.iter()
.map(|field| {
let field_name = field.name();
let field_type = field.data_type();
// Look up the corresponding field type in the table schema
if let Some(table_type) = table_fields.get(field_name) {
match (table_type, field_type) {
// table schema uses string type, coerce the file schema to use string type
(
&DataType::Utf8,
DataType::Binary | DataType::LargeBinary | DataType::BinaryView,
) => {
return field_with_new_type(field, DataType::Utf8);
}
// table schema uses large string type, coerce the file schema to use large string type
(
&DataType::LargeUtf8,
DataType::Binary | DataType::LargeBinary | DataType::BinaryView,
) => {
return field_with_new_type(field, DataType::LargeUtf8);
}
// table schema uses string view type, coerce the file schema to use view type
(
&DataType::Utf8View,
DataType::Binary | DataType::LargeBinary | DataType::BinaryView,
) => {
return field_with_new_type(field, DataType::Utf8View);
}
// Handle view type conversions
(&DataType::Utf8View, DataType::Utf8 | DataType::LargeUtf8) => {
return field_with_new_type(field, DataType::Utf8View);
}
(&DataType::BinaryView, DataType::Binary | DataType::LargeBinary) => {
return field_with_new_type(field, DataType::BinaryView);
}
_ => {}
}
}
// If no transformation is needed, keep the original field
Arc::clone(field)
})
.collect();
Some(Schema::new_with_metadata(
transformed_fields,
file_schema.metadata.clone(),
))
}
/// Coerces the file schema's Timestamps to the provided TimeUnit if Parquet schema contains INT96.
pub fn coerce_int96_to_resolution(
parquet_schema: &SchemaDescriptor,
file_schema: &Schema,
time_unit: &TimeUnit,
) -> Option<Schema> {
// Traverse the parquet_schema columns looking for int96 physical types. If encountered, insert
// the field's full path into a set.
let int96_fields: HashSet<_> = parquet_schema
.columns()
.iter()
.filter(|f| f.physical_type() == Type::INT96)
.map(|f| f.path().string())
.collect();
if int96_fields.is_empty() {
// The schema doesn't contain any int96 fields, so skip the remaining logic.
return None;
}
// Do a DFS into the schema using a stack, looking for timestamp(nanos) fields that originated
// as int96 to coerce to the provided time_unit.
type NestedFields = Rc<RefCell<Vec<FieldRef>>>;
type StackContext<'a> = (
Vec<&'a str>, // The Parquet column path (e.g., "c0.list.element.c1") for the current field.
&'a FieldRef, // The current field to be processed.
NestedFields, // The parent's fields that this field will be (possibly) type-coerced and
// inserted into. All fields have a parent, so this is not an Option type.
Option<NestedFields>, // Nested types need to create their own vector of fields for their
// children. For primitive types this will remain None. For nested
// types it is None the first time they are processed. Then, we
// instantiate a vector for its children, push the field back onto the
// stack to be processed again, and DFS into its children. The next
// time we process the field, we know we have DFS'd into the children
// because this field is Some.
);
// This is our top-level fields from which we will construct our schema. We pass this into our
// initial stack context as the parent fields, and the DFS populates it.
let fields = Rc::new(RefCell::new(Vec::with_capacity(file_schema.fields.len())));
// TODO: It might be possible to only DFS into nested fields that we know contain an int96 if we
// use some sort of LPM data structure to check if we're currently DFS'ing nested types that are
// in a column path that contains an int96. That can be a future optimization for large schemas.
let transformed_schema = {
// Populate the stack with our top-level fields.
let mut stack: Vec<StackContext> = file_schema
.fields()
.iter()
.rev()
.map(|f| (vec![f.name().as_str()], f, Rc::clone(&fields), None))
.collect();
// Pop fields to DFS into until we have exhausted the stack.
while let Some((parquet_path, current_field, parent_fields, child_fields)) =
stack.pop()
{
match (current_field.data_type(), child_fields) {
(DataType::Struct(unprocessed_children), None) => {
// This is the first time popping off this struct. We don't yet know the
// correct types of its children (i.e., if they need coercing) so we create
// a vector for child_fields, push the struct node back onto the stack to be
// processed again (see below) after processing all its children.
let child_fields = Rc::new(RefCell::new(Vec::with_capacity(
unprocessed_children.len(),
)));
// Note that here we push the struct back onto the stack with its
// parent_fields in the same position, now with Some(child_fields).
stack.push((
parquet_path.clone(),
current_field,
parent_fields,
Some(Rc::clone(&child_fields)),
));
// Push all the children in reverse to maintain original schema order due to
// stack processing.
for child in unprocessed_children.into_iter().rev() {
let mut child_path = parquet_path.clone();
// Build up a normalized path that we'll use as a key into the original
// int96_fields set above to test if this originated as int96.
child_path.push(".");
child_path.push(child.name());
// Note that here we push the field onto the stack using the struct's
// new child_fields vector as the field's parent_fields.
stack.push((child_path, child, Rc::clone(&child_fields), None));
}
}
(DataType::Struct(unprocessed_children), Some(processed_children)) => {
// This is the second time popping off this struct. The child_fields vector
// now contains each field that has been DFS'd into, and we can construct
// the resulting struct with correct child types.
let processed_children = processed_children.borrow();
assert_eq!(processed_children.len(), unprocessed_children.len());
let processed_struct = Field::new_struct(
current_field.name(),
processed_children.as_slice(),
current_field.is_nullable(),
);
parent_fields.borrow_mut().push(Arc::new(processed_struct));
}
(DataType::List(unprocessed_child), None) => {
// This is the first time popping off this list. See struct docs above.
let child_fields = Rc::new(RefCell::new(Vec::with_capacity(1)));
stack.push((
parquet_path.clone(),
current_field,
parent_fields,
Some(Rc::clone(&child_fields)),
));
let mut child_path = parquet_path.clone();
// Spark uses a definition for arrays/lists that results in a group
// named "list" that is not maintained when parsing to Arrow. We just push
// this name into the path.
child_path.push(".list.");
child_path.push(unprocessed_child.name());
stack.push((
child_path.clone(),
unprocessed_child,
Rc::clone(&child_fields),
None,
));
}
(DataType::List(_), Some(processed_children)) => {
// This is the second time popping off this list. See struct docs above.
let processed_children = processed_children.borrow();
assert_eq!(processed_children.len(), 1);
let processed_list = Field::new_list(
current_field.name(),
Arc::clone(&processed_children[0]),
current_field.is_nullable(),
);
parent_fields.borrow_mut().push(Arc::new(processed_list));
}
(DataType::Map(unprocessed_child, _), None) => {
// This is the first time popping off this map. See struct docs above.
let child_fields = Rc::new(RefCell::new(Vec::with_capacity(1)));
stack.push((
parquet_path.clone(),
current_field,
parent_fields,
Some(Rc::clone(&child_fields)),
));
let mut child_path = parquet_path.clone();
child_path.push(".");
child_path.push(unprocessed_child.name());
stack.push((
child_path.clone(),
unprocessed_child,
Rc::clone(&child_fields),
None,
));
}
(DataType::Map(_, sorted), Some(processed_children)) => {
// This is the second time popping off this map. See struct docs above.
let processed_children = processed_children.borrow();
assert_eq!(processed_children.len(), 1);
let processed_map = Field::new(
current_field.name(),
DataType::Map(Arc::clone(&processed_children[0]), *sorted),
current_field.is_nullable(),
);
parent_fields.borrow_mut().push(Arc::new(processed_map));
}
(DataType::Timestamp(TimeUnit::Nanosecond, None), None)
if int96_fields.contains(parquet_path.concat().as_str()) =>
// We found a timestamp(nanos) and it originated as int96. Coerce it to the correct
// time_unit.
{
parent_fields.borrow_mut().push(field_with_new_type(
current_field,
DataType::Timestamp(*time_unit, None),
));
}
// Other types can be cloned as they are.
_ => parent_fields.borrow_mut().push(Arc::clone(current_field)),
}
}
assert_eq!(fields.borrow().len(), file_schema.fields.len());
Schema::new_with_metadata(
fields.borrow_mut().clone(),
file_schema.metadata.clone(),
)
};
Some(transformed_schema)
}
/// Coerces the file schema if the table schema uses a view type.
#[deprecated(
since = "47.0.0",
note = "Use `apply_file_schema_type_coercions` instead"
)]
pub fn coerce_file_schema_to_view_type(
table_schema: &Schema,
file_schema: &Schema,
) -> Option<Schema> {
let mut transform = false;
let table_fields: HashMap<_, _> = table_schema
.fields
.iter()
.map(|f| {
let dt = f.data_type();
if dt.equals_datatype(&DataType::Utf8View)
|| dt.equals_datatype(&DataType::BinaryView)
{
transform = true;
}
(f.name(), dt)
})
.collect();
if !transform {
return None;
}
let transformed_fields: Vec<Arc<Field>> = file_schema
.fields
.iter()
.map(
|field| match (table_fields.get(field.name()), field.data_type()) {
(Some(DataType::Utf8View), DataType::Utf8 | DataType::LargeUtf8) => {
field_with_new_type(field, DataType::Utf8View)
}
(
Some(DataType::BinaryView),
DataType::Binary | DataType::LargeBinary,
) => field_with_new_type(field, DataType::BinaryView),
_ => Arc::clone(field),
},
)
.collect();
Some(Schema::new_with_metadata(
transformed_fields,
file_schema.metadata.clone(),
))
}
/// If the table schema uses a string type, coerce the file schema to use a string type.
///
/// See [ParquetFormat::binary_as_string] for details
#[deprecated(
since = "47.0.0",
note = "Use `apply_file_schema_type_coercions` instead"
)]
pub fn coerce_file_schema_to_string_type(
table_schema: &Schema,
file_schema: &Schema,
) -> Option<Schema> {
let mut transform = false;
let table_fields: HashMap<_, _> = table_schema
.fields
.iter()
.map(|f| (f.name(), f.data_type()))
.collect();
let transformed_fields: Vec<Arc<Field>> = file_schema
.fields
.iter()
.map(
|field| match (table_fields.get(field.name()), field.data_type()) {
// table schema uses string type, coerce the file schema to use string type
(
Some(DataType::Utf8),
DataType::Binary | DataType::LargeBinary | DataType::BinaryView,
) => {
transform = true;
field_with_new_type(field, DataType::Utf8)
}
// table schema uses large string type, coerce the file schema to use large string type
(
Some(DataType::LargeUtf8),
DataType::Binary | DataType::LargeBinary | DataType::BinaryView,
) => {
transform = true;
field_with_new_type(field, DataType::LargeUtf8)
}
// table schema uses string view type, coerce the file schema to use view type
(
Some(DataType::Utf8View),
DataType::Binary | DataType::LargeBinary | DataType::BinaryView,