-
Notifications
You must be signed in to change notification settings - Fork 807
/
mod.rs
3742 lines (3313 loc) · 135 KB
/
mod.rs
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.
//! Contains reader which reads parquet data into arrow [`RecordBatch`]
use std::collections::VecDeque;
use std::sync::Arc;
use arrow_array::cast::AsArray;
use arrow_array::Array;
use arrow_array::{RecordBatch, RecordBatchReader};
use arrow_schema::{ArrowError, DataType as ArrowType, Schema, SchemaRef};
use arrow_select::filter::prep_null_mask_filter;
pub use filter::{ArrowPredicate, ArrowPredicateFn, RowFilter};
pub use selection::{RowSelection, RowSelector};
pub use crate::arrow::array_reader::RowGroups;
use crate::arrow::array_reader::{build_array_reader, ArrayReader};
use crate::arrow::schema::{parquet_to_arrow_schema_and_fields, ParquetField};
use crate::arrow::{parquet_to_arrow_field_levels, FieldLevels, ProjectionMask};
use crate::column::page::{PageIterator, PageReader};
use crate::errors::{ParquetError, Result};
use crate::file::footer;
use crate::file::metadata::ParquetMetaData;
use crate::file::page_index::index_reader;
use crate::file::reader::{ChunkReader, SerializedPageReader};
use crate::schema::types::SchemaDescriptor;
mod filter;
mod selection;
/// Builder for constructing parquet readers into arrow.
///
/// Most users should use one of the following specializations:
///
/// * synchronous API: [`ParquetRecordBatchReaderBuilder::try_new`]
/// * `async` API: [`ParquetRecordBatchStreamBuilder::new`]
///
/// [`ParquetRecordBatchStreamBuilder::new`]: crate::arrow::async_reader::ParquetRecordBatchStreamBuilder::new
pub struct ArrowReaderBuilder<T> {
pub(crate) input: T,
pub(crate) metadata: Arc<ParquetMetaData>,
pub(crate) schema: SchemaRef,
pub(crate) fields: Option<Arc<ParquetField>>,
pub(crate) batch_size: usize,
pub(crate) row_groups: Option<Vec<usize>>,
pub(crate) projection: ProjectionMask,
pub(crate) filter: Option<RowFilter>,
pub(crate) selection: Option<RowSelection>,
pub(crate) limit: Option<usize>,
pub(crate) offset: Option<usize>,
}
impl<T> ArrowReaderBuilder<T> {
pub(crate) fn new_builder(input: T, metadata: ArrowReaderMetadata) -> Self {
Self {
input,
metadata: metadata.metadata,
schema: metadata.schema,
fields: metadata.fields,
batch_size: 1024,
row_groups: None,
projection: ProjectionMask::all(),
filter: None,
selection: None,
limit: None,
offset: None,
}
}
/// Returns a reference to the [`ParquetMetaData`] for this parquet file
pub fn metadata(&self) -> &Arc<ParquetMetaData> {
&self.metadata
}
/// Returns the parquet [`SchemaDescriptor`] for this parquet file
pub fn parquet_schema(&self) -> &SchemaDescriptor {
self.metadata.file_metadata().schema_descr()
}
/// Returns the arrow [`SchemaRef`] for this parquet file
pub fn schema(&self) -> &SchemaRef {
&self.schema
}
/// Set the size of [`RecordBatch`] to produce. Defaults to 1024
/// If the batch_size more than the file row count, use the file row count.
pub fn with_batch_size(self, batch_size: usize) -> Self {
// Try to avoid allocate large buffer
let batch_size = batch_size.min(self.metadata.file_metadata().num_rows() as usize);
Self { batch_size, ..self }
}
/// Only read data from the provided row group indexes
///
/// This is also called row group filtering
pub fn with_row_groups(self, row_groups: Vec<usize>) -> Self {
Self {
row_groups: Some(row_groups),
..self
}
}
/// Only read data from the provided column indexes
pub fn with_projection(self, mask: ProjectionMask) -> Self {
Self {
projection: mask,
..self
}
}
/// Provide a [`RowSelection`] to filter out rows, and avoid fetching their
/// data into memory.
///
/// This feature is used to restrict which rows are decoded within row
/// groups, skipping ranges of rows that are not needed. Such selections
/// could be determined by evaluating predicates against the parquet page
/// [`Index`] or some other external information available to a query
/// engine.
///
/// # Notes
///
/// Row group filtering (see [`Self::with_row_groups`]) is applied prior to
/// applying the row selection, and therefore rows from skipped row groups
/// should not be included in the [`RowSelection`] (see example below)
///
/// It is recommended to enable writing the page index if using this
/// functionality, to allow more efficient skipping over data pages. See
/// [`ArrowReaderOptions::with_page_index`].
///
/// # Example
///
/// Given a parquet file with 4 row groups, and a row group filter of `[0,
/// 2, 3]`, in order to scan rows 50-100 in row group 2 and rows 200-300 in
/// row group 3:
///
/// ```text
/// Row Group 0, 1000 rows (selected)
/// Row Group 1, 1000 rows (skipped)
/// Row Group 2, 1000 rows (selected, but want to only scan rows 50-100)
/// Row Group 3, 1000 rows (selected, but want to only scan rows 200-300)
/// ```
///
/// You could pass the following [`RowSelection`]:
///
/// ```text
/// Select 1000 (scan all rows in row group 0)
/// Skip 50 (skip the first 50 rows in row group 2)
/// Select 50 (scan rows 50-100 in row group 2)
/// Skip 900 (skip the remaining rows in row group 2)
/// Skip 200 (skip the first 200 rows in row group 3)
/// Select 100 (scan rows 200-300 in row group 3)
/// Skip 700 (skip the remaining rows in row group 3)
/// ```
/// Note there is no entry for the (entirely) skipped row group 1.
///
/// Note you can represent the same selection with fewer entries. Instead of
///
/// ```text
/// Skip 900 (skip the remaining rows in row group 2)
/// Skip 200 (skip the first 200 rows in row group 3)
/// ```
///
/// you could use
///
/// ```text
/// Skip 1100 (skip the remaining 900 rows in row group 2 and the first 200 rows in row group 3)
/// ```
///
/// [`Index`]: crate::file::page_index::index::Index
pub fn with_row_selection(self, selection: RowSelection) -> Self {
Self {
selection: Some(selection),
..self
}
}
/// Provide a [`RowFilter`] to skip decoding rows
///
/// Row filters are applied after row group selection and row selection
///
/// It is recommended to enable reading the page index if using this functionality, to allow
/// more efficient skipping over data pages. See [`ArrowReaderOptions::with_page_index`].
pub fn with_row_filter(self, filter: RowFilter) -> Self {
Self {
filter: Some(filter),
..self
}
}
/// Provide a limit to the number of rows to be read
///
/// The limit will be applied after any [`Self::with_row_selection`] and [`Self::with_row_filter`]
/// allowing it to limit the final set of rows decoded after any pushed down predicates
///
/// It is recommended to enable reading the page index if using this functionality, to allow
/// more efficient skipping over data pages. See [`ArrowReaderOptions::with_page_index`]
pub fn with_limit(self, limit: usize) -> Self {
Self {
limit: Some(limit),
..self
}
}
/// Provide an offset to skip over the given number of rows
///
/// The offset will be applied after any [`Self::with_row_selection`] and [`Self::with_row_filter`]
/// allowing it to skip rows after any pushed down predicates
///
/// It is recommended to enable reading the page index if using this functionality, to allow
/// more efficient skipping over data pages. See [`ArrowReaderOptions::with_page_index`]
pub fn with_offset(self, offset: usize) -> Self {
Self {
offset: Some(offset),
..self
}
}
}
/// Options that control how metadata is read for a parquet file
///
/// See [`ArrowReaderBuilder`] for how to configure how the column data
/// is then read from the file, including projection and filter pushdown
#[derive(Debug, Clone, Default)]
pub struct ArrowReaderOptions {
/// Should the reader strip any user defined metadata from the Arrow schema
skip_arrow_metadata: bool,
/// If provided used as the schema for the file, otherwise the schema is read from the file
supplied_schema: Option<SchemaRef>,
/// If true, attempt to read `OffsetIndex` and `ColumnIndex`
pub(crate) page_index: bool,
}
impl ArrowReaderOptions {
/// Create a new [`ArrowReaderOptions`] with the default settings
pub fn new() -> Self {
Self::default()
}
/// Skip decoding the embedded arrow metadata (defaults to `false`)
///
/// Parquet files generated by some writers may contain embedded arrow
/// schema and metadata.
/// This may not be correct or compatible with your system,
/// for example: [ARROW-16184](https://issues.apache.org/jira/browse/ARROW-16184)
pub fn with_skip_arrow_metadata(self, skip_arrow_metadata: bool) -> Self {
Self {
skip_arrow_metadata,
..self
}
}
/// Provide a schema to use when reading the parquet file. If provided it
/// takes precedence over the schema inferred from the file or the schema defined
/// in the file's metadata. If the schema is not compatible with the file's
/// schema an error will be returned when constructing the builder.
///
/// This option is only required if you want to cast columns to a different type.
/// For example, if you wanted to cast from an Int64 in the Parquet file to a Timestamp
/// in the Arrow schema.
///
/// The supplied schema must have the same number of columns as the parquet schema and
/// the column names need to be the same.
///
/// # Example
/// ```
/// use std::io::Bytes;
/// use std::sync::Arc;
/// use tempfile::tempfile;
/// use arrow_array::{ArrayRef, Int32Array, RecordBatch};
/// use arrow_schema::{DataType, Field, Schema, TimeUnit};
/// use parquet::arrow::arrow_reader::{ArrowReaderOptions, ParquetRecordBatchReaderBuilder};
/// use parquet::arrow::ArrowWriter;
///
/// // Write data - schema is inferred from the data to be Int32
/// let file = tempfile().unwrap();
/// let batch = RecordBatch::try_from_iter(vec![
/// ("col_1", Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef),
/// ]).unwrap();
/// let mut writer = ArrowWriter::try_new(file.try_clone().unwrap(), batch.schema(), None).unwrap();
/// writer.write(&batch).unwrap();
/// writer.close().unwrap();
///
/// // Read the file back.
/// // Supply a schema that interprets the Int32 column as a Timestamp.
/// let supplied_schema = Arc::new(Schema::new(vec![
/// Field::new("col_1", DataType::Timestamp(TimeUnit::Nanosecond, None), false)
/// ]));
/// let options = ArrowReaderOptions::new().with_schema(supplied_schema.clone());
/// let mut builder = ParquetRecordBatchReaderBuilder::try_new_with_options(
/// file.try_clone().unwrap(),
/// options
/// ).expect("Error if the schema is not compatible with the parquet file schema.");
///
/// // Create the reader and read the data using the supplied schema.
/// let mut reader = builder.build().unwrap();
/// let _batch = reader.next().unwrap().unwrap();
/// ```
pub fn with_schema(self, schema: SchemaRef) -> Self {
Self {
supplied_schema: Some(schema),
skip_arrow_metadata: true,
..self
}
}
/// Enable reading [`PageIndex`], if present (defaults to `false`)
///
/// The `PageIndex` can be used to push down predicates to the parquet scan,
/// potentially eliminating unnecessary IO, by some query engines.
///
/// If this is enabled, [`ParquetMetaData::column_index`] and
/// [`ParquetMetaData::offset_index`] will be populated if the corresponding
/// information is present in the file.
///
/// [`PageIndex`]: https://github.com/apache/parquet-format/blob/master/PageIndex.md
/// [`ParquetMetaData::column_index`]: crate::file::metadata::ParquetMetaData::column_index
/// [`ParquetMetaData::offset_index`]: crate::file::metadata::ParquetMetaData::offset_index
pub fn with_page_index(self, page_index: bool) -> Self {
Self { page_index, ..self }
}
}
/// The metadata necessary to construct a [`ArrowReaderBuilder`]
///
/// Note this structure is cheaply clone-able as it consists of several arcs.
///
/// This structure allows
///
/// 1. Loading metadata for a file once and then using that same metadata to
/// construct multiple separate readers, for example, to distribute readers
/// across multiple threads
///
/// 2. Using a cached copy of the [`ParquetMetadata`] rather than reading it
/// from the file each time a reader is constructed.
///
/// [`ParquetMetadata`]: crate::file::metadata::ParquetMetaData
#[derive(Debug, Clone)]
pub struct ArrowReaderMetadata {
/// The Parquet Metadata, if known aprior
pub(crate) metadata: Arc<ParquetMetaData>,
/// The Arrow Schema
pub(crate) schema: SchemaRef,
pub(crate) fields: Option<Arc<ParquetField>>,
}
impl ArrowReaderMetadata {
/// Loads [`ArrowReaderMetadata`] from the provided [`ChunkReader`], if necessary
///
/// See [`ParquetRecordBatchReaderBuilder::new_with_metadata`] for an
/// example of how this can be used
///
/// # Notes
///
/// If `options` has [`ArrowReaderOptions::with_page_index`] true, but
/// `Self::metadata` is missing the page index, this function will attempt
/// to load the page index by making an object store request.
pub fn load<T: ChunkReader>(reader: &T, options: ArrowReaderOptions) -> Result<Self> {
let mut metadata = footer::parse_metadata(reader)?;
if options.page_index {
let column_index = metadata
.row_groups()
.iter()
.map(|rg| index_reader::read_columns_indexes(reader, rg.columns()))
.collect::<Result<Vec<_>>>()?;
metadata.set_column_index(Some(column_index));
let offset_index = metadata
.row_groups()
.iter()
.map(|rg| index_reader::read_pages_locations(reader, rg.columns()))
.collect::<Result<Vec<_>>>()?;
metadata.set_offset_index(Some(offset_index))
}
Self::try_new(Arc::new(metadata), options)
}
/// Create a new [`ArrowReaderMetadata`]
///
/// # Notes
///
/// This function does not attempt to load the PageIndex if not present in the metadata.
/// See [`Self::load`] for more details.
pub fn try_new(metadata: Arc<ParquetMetaData>, options: ArrowReaderOptions) -> Result<Self> {
match options.supplied_schema {
Some(supplied_schema) => Self::with_supplied_schema(metadata, supplied_schema.clone()),
None => {
let kv_metadata = match options.skip_arrow_metadata {
true => None,
false => metadata.file_metadata().key_value_metadata(),
};
let (schema, fields) = parquet_to_arrow_schema_and_fields(
metadata.file_metadata().schema_descr(),
ProjectionMask::all(),
kv_metadata,
)?;
Ok(Self {
metadata,
schema: Arc::new(schema),
fields: fields.map(Arc::new),
})
}
}
}
fn with_supplied_schema(
metadata: Arc<ParquetMetaData>,
supplied_schema: SchemaRef,
) -> Result<Self> {
let parquet_schema = metadata.file_metadata().schema_descr();
let field_levels = parquet_to_arrow_field_levels(
parquet_schema,
ProjectionMask::all(),
Some(supplied_schema.fields()),
)?;
let fields = field_levels.fields;
let inferred_len = fields.len();
let supplied_len = supplied_schema.fields().len();
// Ensure the supplied schema has the same number of columns as the parquet schema.
// parquet_to_arrow_field_levels is expected to throw an error if the schemas have
// different lengths, but we check here to be safe.
if inferred_len != supplied_len {
Err(arrow_err!(format!(
"incompatible arrow schema, expected {} columns received {}",
inferred_len, supplied_len
)))
} else {
let diff_fields: Vec<_> = supplied_schema
.fields()
.iter()
.zip(fields.iter())
.filter_map(|(field1, field2)| {
if field1 != field2 {
Some(field1.name().clone())
} else {
None
}
})
.collect();
if !diff_fields.is_empty() {
Err(ParquetError::ArrowError(format!(
"incompatible arrow schema, the following fields could not be cast: [{}]",
diff_fields.join(", ")
)))
} else {
Ok(Self {
metadata,
schema: supplied_schema,
fields: field_levels.levels.map(Arc::new),
})
}
}
}
/// Returns a reference to the [`ParquetMetaData`] for this parquet file
pub fn metadata(&self) -> &Arc<ParquetMetaData> {
&self.metadata
}
/// Returns the parquet [`SchemaDescriptor`] for this parquet file
pub fn parquet_schema(&self) -> &SchemaDescriptor {
self.metadata.file_metadata().schema_descr()
}
/// Returns the arrow [`SchemaRef`] for this parquet file
pub fn schema(&self) -> &SchemaRef {
&self.schema
}
}
#[doc(hidden)]
/// A newtype used within [`ReaderOptionsBuilder`] to distinguish sync readers from async
pub struct SyncReader<T: ChunkReader>(T);
/// A synchronous builder used to construct [`ParquetRecordBatchReader`] for a file
///
/// For an async API see [`crate::arrow::async_reader::ParquetRecordBatchStreamBuilder`]
///
/// See [`ArrowReaderBuilder`] for additional member functions
pub type ParquetRecordBatchReaderBuilder<T> = ArrowReaderBuilder<SyncReader<T>>;
impl<T: ChunkReader + 'static> ParquetRecordBatchReaderBuilder<T> {
/// Create a new [`ParquetRecordBatchReaderBuilder`]
///
/// ```
/// # use std::sync::Arc;
/// # use bytes::Bytes;
/// # use arrow_array::{Int32Array, RecordBatch};
/// # use arrow_schema::{DataType, Field, Schema};
/// # use parquet::arrow::arrow_reader::{ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder};
/// # use parquet::arrow::ArrowWriter;
/// # let mut file: Vec<u8> = Vec::with_capacity(1024);
/// # let schema = Arc::new(Schema::new(vec![Field::new("i32", DataType::Int32, false)]));
/// # let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap();
/// # let batch = RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![1, 2, 3]))]).unwrap();
/// # writer.write(&batch).unwrap();
/// # writer.close().unwrap();
/// # let file = Bytes::from(file);
/// #
/// let mut builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap();
///
/// // Inspect metadata
/// assert_eq!(builder.metadata().num_row_groups(), 1);
///
/// // Construct reader
/// let mut reader: ParquetRecordBatchReader = builder.with_row_groups(vec![0]).build().unwrap();
///
/// // Read data
/// let _batch = reader.next().unwrap().unwrap();
/// ```
pub fn try_new(reader: T) -> Result<Self> {
Self::try_new_with_options(reader, Default::default())
}
/// Create a new [`ParquetRecordBatchReaderBuilder`] with [`ArrowReaderOptions`]
pub fn try_new_with_options(reader: T, options: ArrowReaderOptions) -> Result<Self> {
let metadata = ArrowReaderMetadata::load(&reader, options)?;
Ok(Self::new_with_metadata(reader, metadata))
}
/// Create a [`ParquetRecordBatchReaderBuilder`] from the provided [`ArrowReaderMetadata`]
///
/// This interface allows:
///
/// 1. Loading metadata once and using it to create multiple builders with
/// potentially different settings or run on different threads
///
/// 2. Using a cached copy of the metadata rather than re-reading it from the
/// file each time a reader is constructed.
///
/// See the docs on [`ArrowReaderMetadata`] for more details
///
/// # Example
/// ```
/// # use std::fs::metadata;
/// # use std::sync::Arc;
/// # use bytes::Bytes;
/// # use arrow_array::{Int32Array, RecordBatch};
/// # use arrow_schema::{DataType, Field, Schema};
/// # use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ParquetRecordBatchReader, ParquetRecordBatchReaderBuilder};
/// # use parquet::arrow::ArrowWriter;
/// # let mut file: Vec<u8> = Vec::with_capacity(1024);
/// # let schema = Arc::new(Schema::new(vec![Field::new("i32", DataType::Int32, false)]));
/// # let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap();
/// # let batch = RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![1, 2, 3]))]).unwrap();
/// # writer.write(&batch).unwrap();
/// # writer.close().unwrap();
/// # let file = Bytes::from(file);
/// #
/// let metadata = ArrowReaderMetadata::load(&file, Default::default()).unwrap();
/// let mut a = ParquetRecordBatchReaderBuilder::new_with_metadata(file.clone(), metadata.clone()).build().unwrap();
/// let mut b = ParquetRecordBatchReaderBuilder::new_with_metadata(file, metadata).build().unwrap();
///
/// // Should be able to read from both in parallel
/// assert_eq!(a.next().unwrap().unwrap(), b.next().unwrap().unwrap());
/// ```
pub fn new_with_metadata(input: T, metadata: ArrowReaderMetadata) -> Self {
Self::new_builder(SyncReader(input), metadata)
}
/// Build a [`ParquetRecordBatchReader`]
///
/// Note: this will eagerly evaluate any `RowFilter` before returning
pub fn build(self) -> Result<ParquetRecordBatchReader> {
// Try to avoid allocate large buffer
let batch_size = self
.batch_size
.min(self.metadata.file_metadata().num_rows() as usize);
let row_groups = self
.row_groups
.unwrap_or_else(|| (0..self.metadata.num_row_groups()).collect());
let reader = ReaderRowGroups {
reader: Arc::new(self.input.0),
metadata: self.metadata,
row_groups,
};
let mut filter = self.filter;
let mut selection = self.selection;
if let Some(filter) = filter.as_mut() {
for predicate in filter.predicates.iter_mut() {
if !selects_any(selection.as_ref()) {
break;
}
let array_reader =
build_array_reader(self.fields.as_deref(), predicate.projection(), &reader)?;
selection = Some(evaluate_predicate(
batch_size,
array_reader,
selection,
predicate.as_mut(),
)?);
}
}
let array_reader = build_array_reader(self.fields.as_deref(), &self.projection, &reader)?;
// If selection is empty, truncate
if !selects_any(selection.as_ref()) {
selection = Some(RowSelection::from(vec![]));
}
Ok(ParquetRecordBatchReader::new(
batch_size,
array_reader,
apply_range(selection, reader.num_rows(), self.offset, self.limit),
))
}
}
struct ReaderRowGroups<T: ChunkReader> {
reader: Arc<T>,
metadata: Arc<ParquetMetaData>,
/// Optional list of row group indices to scan
row_groups: Vec<usize>,
}
impl<T: ChunkReader + 'static> RowGroups for ReaderRowGroups<T> {
fn num_rows(&self) -> usize {
let meta = self.metadata.row_groups();
self.row_groups
.iter()
.map(|x| meta[*x].num_rows() as usize)
.sum()
}
fn column_chunks(&self, i: usize) -> Result<Box<dyn PageIterator>> {
Ok(Box::new(ReaderPageIterator {
column_idx: i,
reader: self.reader.clone(),
metadata: self.metadata.clone(),
row_groups: self.row_groups.clone().into_iter(),
}))
}
}
struct ReaderPageIterator<T: ChunkReader> {
reader: Arc<T>,
column_idx: usize,
row_groups: std::vec::IntoIter<usize>,
metadata: Arc<ParquetMetaData>,
}
impl<T: ChunkReader + 'static> Iterator for ReaderPageIterator<T> {
type Item = Result<Box<dyn PageReader>>;
fn next(&mut self) -> Option<Self::Item> {
let rg_idx = self.row_groups.next()?;
let rg = self.metadata.row_group(rg_idx);
let meta = rg.column(self.column_idx);
let offset_index = self.metadata.offset_index();
// `offset_index` may not exist and `i[rg_idx]` will be empty.
// To avoid `i[rg_idx][self.oolumn_idx`] panic, we need to filter out empty `i[rg_idx]`.
let page_locations = offset_index
.filter(|i| !i[rg_idx].is_empty())
.map(|i| i[rg_idx][self.column_idx].clone());
let total_rows = rg.num_rows() as usize;
let reader = self.reader.clone();
let ret = SerializedPageReader::new(reader, meta, total_rows, page_locations);
Some(ret.map(|x| Box::new(x) as _))
}
}
impl<T: ChunkReader + 'static> PageIterator for ReaderPageIterator<T> {}
/// An `Iterator<Item = ArrowResult<RecordBatch>>` that yields [`RecordBatch`]
/// read from a parquet data source
pub struct ParquetRecordBatchReader {
batch_size: usize,
array_reader: Box<dyn ArrayReader>,
schema: SchemaRef,
selection: Option<VecDeque<RowSelector>>,
}
impl Iterator for ParquetRecordBatchReader {
type Item = Result<RecordBatch, ArrowError>;
fn next(&mut self) -> Option<Self::Item> {
let mut read_records = 0;
match self.selection.as_mut() {
Some(selection) => {
while read_records < self.batch_size && !selection.is_empty() {
let front = selection.pop_front().unwrap();
if front.skip {
let skipped = match self.array_reader.skip_records(front.row_count) {
Ok(skipped) => skipped,
Err(e) => return Some(Err(e.into())),
};
if skipped != front.row_count {
return Some(Err(general_err!(
"failed to skip rows, expected {}, got {}",
front.row_count,
skipped
)
.into()));
}
continue;
}
//Currently, when RowSelectors with row_count = 0 are included then its interpreted as end of reader.
//Fix is to skip such entries. See https://github.com/apache/arrow-rs/issues/2669
if front.row_count == 0 {
continue;
}
// try to read record
let need_read = self.batch_size - read_records;
let to_read = match front.row_count.checked_sub(need_read) {
Some(remaining) if remaining != 0 => {
// if page row count less than batch_size we must set batch size to page row count.
// add check avoid dead loop
selection.push_front(RowSelector::select(remaining));
need_read
}
_ => front.row_count,
};
match self.array_reader.read_records(to_read) {
Ok(0) => break,
Ok(rec) => read_records += rec,
Err(error) => return Some(Err(error.into())),
}
}
}
None => {
if let Err(error) = self.array_reader.read_records(self.batch_size) {
return Some(Err(error.into()));
}
}
};
match self.array_reader.consume_batch() {
Err(error) => Some(Err(error.into())),
Ok(array) => {
let struct_array = array.as_struct_opt().ok_or_else(|| {
ArrowError::ParquetError(
"Struct array reader should return struct array".to_string(),
)
});
match struct_array {
Err(err) => Some(Err(err)),
Ok(e) => (e.len() > 0).then(|| Ok(RecordBatch::from(e))),
}
}
}
}
}
impl RecordBatchReader for ParquetRecordBatchReader {
/// Returns the projected [`SchemaRef`] for reading the parquet file.
///
/// Note that the schema metadata will be stripped here. See
/// [`ParquetRecordBatchReaderBuilder::schema`] if the metadata is desired.
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
}
impl ParquetRecordBatchReader {
/// Create a new [`ParquetRecordBatchReader`] from the provided chunk reader
///
/// See [`ParquetRecordBatchReaderBuilder`] for more options
pub fn try_new<T: ChunkReader + 'static>(reader: T, batch_size: usize) -> Result<Self> {
ParquetRecordBatchReaderBuilder::try_new(reader)?
.with_batch_size(batch_size)
.build()
}
/// Create a new [`ParquetRecordBatchReader`] from the provided [`RowGroups`]
///
/// Note: this is a low-level interface see [`ParquetRecordBatchReader::try_new`] for a
/// higher-level interface for reading parquet data from a file
pub fn try_new_with_row_groups(
levels: &FieldLevels,
row_groups: &dyn RowGroups,
batch_size: usize,
selection: Option<RowSelection>,
) -> Result<Self> {
let array_reader =
build_array_reader(levels.levels.as_ref(), &ProjectionMask::all(), row_groups)?;
Ok(Self {
batch_size,
array_reader,
schema: Arc::new(Schema::new(levels.fields.clone())),
selection: selection.map(|s| s.trim().into()),
})
}
/// Create a new [`ParquetRecordBatchReader`] that will read at most `batch_size` rows at
/// a time from [`ArrayReader`] based on the configured `selection`. If `selection` is `None`
/// all rows will be returned
pub(crate) fn new(
batch_size: usize,
array_reader: Box<dyn ArrayReader>,
selection: Option<RowSelection>,
) -> Self {
let schema = match array_reader.get_data_type() {
ArrowType::Struct(ref fields) => Schema::new(fields.clone()),
_ => unreachable!("Struct array reader's data type is not struct!"),
};
Self {
batch_size,
array_reader,
schema: Arc::new(schema),
selection: selection.map(|s| s.trim().into()),
}
}
}
/// Returns `true` if `selection` is `None` or selects some rows
pub(crate) fn selects_any(selection: Option<&RowSelection>) -> bool {
selection.map(|x| x.selects_any()).unwrap_or(true)
}
/// Applies an optional offset and limit to an optional [`RowSelection`]
pub(crate) fn apply_range(
mut selection: Option<RowSelection>,
row_count: usize,
offset: Option<usize>,
limit: Option<usize>,
) -> Option<RowSelection> {
// If an offset is defined, apply it to the `selection`
if let Some(offset) = offset {
selection = Some(match row_count.checked_sub(offset) {
None => RowSelection::from(vec![]),
Some(remaining) => selection
.map(|selection| selection.offset(offset))
.unwrap_or_else(|| {
RowSelection::from(vec![
RowSelector::skip(offset),
RowSelector::select(remaining),
])
}),
});
}
// If a limit is defined, apply it to the final `selection`
if let Some(limit) = limit {
selection = Some(
selection
.map(|selection| selection.limit(limit))
.unwrap_or_else(|| {
RowSelection::from(vec![RowSelector::select(limit.min(row_count))])
}),
);
}
selection
}
/// Evaluates an [`ArrowPredicate`], returning a [`RowSelection`] indicating
/// which rows to return.
///
/// `input_selection`: Optional pre-existing selection. If `Some`, then the
/// final [`RowSelection`] will be the conjunction of it and the rows selected
/// by `predicate`.
///
/// Note: A pre-existing selection may come from evaluating a previous predicate
/// or if the [`ParquetRecordBatchReader`] specified an explicit
/// [`RowSelection`] in addition to one or more predicates.
pub(crate) fn evaluate_predicate(
batch_size: usize,
array_reader: Box<dyn ArrayReader>,
input_selection: Option<RowSelection>,
predicate: &mut dyn ArrowPredicate,
) -> Result<RowSelection> {
let reader = ParquetRecordBatchReader::new(batch_size, array_reader, input_selection.clone());
let mut filters = vec![];
for maybe_batch in reader {
let maybe_batch = maybe_batch?;
let input_rows = maybe_batch.num_rows();
let filter = predicate.evaluate(maybe_batch)?;
// Since user supplied predicate, check error here to catch bugs quickly
if filter.len() != input_rows {
return Err(arrow_err!(
"ArrowPredicate predicate returned {} rows, expected {input_rows}",
filter.len()
));
}
match filter.null_count() {
0 => filters.push(filter),
_ => filters.push(prep_null_mask_filter(&filter)),
};
}
let raw = RowSelection::from_filters(&filters);
Ok(match input_selection {
Some(selection) => selection.and_then(&raw),
None => raw,
})
}
#[cfg(test)]
mod tests {
use std::cmp::min;
use std::collections::VecDeque;
use std::fmt::Formatter;
use std::fs::File;
use std::io::Seek;
use std::path::PathBuf;
use std::sync::Arc;
use bytes::Bytes;
use half::f16;
use num::PrimInt;
use rand::{thread_rng, Rng, RngCore};
use tempfile::tempfile;
use arrow_array::builder::*;
use arrow_array::cast::AsArray;
use arrow_array::types::{
Decimal128Type, Decimal256Type, DecimalType, Float16Type, Float32Type, Float64Type,
};
use arrow_array::*;
use arrow_buffer::{i256, ArrowNativeType, Buffer, IntervalDayTime};
use arrow_data::ArrayDataBuilder;
use arrow_schema::{ArrowError, DataType as ArrowDataType, Field, Fields, Schema, SchemaRef};
use arrow_select::concat::concat_batches;
use crate::arrow::arrow_reader::{
ArrowPredicateFn, ArrowReaderBuilder, ArrowReaderOptions, ParquetRecordBatchReader,
ParquetRecordBatchReaderBuilder, RowFilter, RowSelection, RowSelector,
};
use crate::arrow::schema::add_encoded_arrow_schema_to_metadata;
use crate::arrow::{ArrowWriter, ProjectionMask};
use crate::basic::{ConvertedType, Encoding, Repetition, Type as PhysicalType};
use crate::column::reader::decoder::REPETITION_LEVELS_BATCH_SIZE;
use crate::data_type::{
BoolType, ByteArray, ByteArrayType, DataType, FixedLenByteArray, FixedLenByteArrayType,
FloatType, Int32Type, Int64Type, Int96Type,
};
use crate::errors::Result;
use crate::file::properties::{EnabledStatistics, WriterProperties, WriterVersion};
use crate::file::writer::SerializedFileWriter;
use crate::schema::parser::parse_message_type;
use crate::schema::types::{Type, TypePtr};
use crate::util::test_common::rand_gen::RandGen;
#[test]
fn test_arrow_reader_all_columns() {
let file = get_test_file("parquet/generated_simple_numerics/blogs.parquet");
let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap();
let original_schema = Arc::clone(builder.schema());
let reader = builder.build().unwrap();
// Verify that the schema was correctly parsed
assert_eq!(original_schema.fields(), reader.schema().fields());
}
#[test]
fn test_arrow_reader_single_column() {
let file = get_test_file("parquet/generated_simple_numerics/blogs.parquet");
let builder = ParquetRecordBatchReaderBuilder::try_new(file).unwrap();
let original_schema = Arc::clone(builder.schema());
let mask = ProjectionMask::leaves(builder.parquet_schema(), [2]);
let reader = builder.with_projection(mask).build().unwrap();
// Verify that the schema was correctly parsed
assert_eq!(1, reader.schema().fields().len());