-
Notifications
You must be signed in to change notification settings - Fork 879
/
Copy pathserialized_reader.rs
2124 lines (1881 loc) · 80.8 KB
/
serialized_reader.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 implementations of the reader traits FileReader, RowGroupReader and PageReader
//! Also contains implementations of the ChunkReader for files (with buffering) and byte arrays (RAM)
use std::collections::VecDeque;
use std::iter;
use std::{fs::File, io::Read, path::Path, sync::Arc};
use crate::basic::{Encoding, Type};
use crate::bloom_filter::Sbbf;
use crate::column::page::{Page, PageMetadata, PageReader};
use crate::compression::{create_codec, Codec};
use crate::errors::{ParquetError, Result};
use crate::file::page_index::offset_index::OffsetIndexMetaData;
use crate::file::{
metadata::*,
properties::{ReaderProperties, ReaderPropertiesPtr},
reader::*,
statistics,
};
use crate::format::{PageHeader, PageLocation, PageType};
use crate::record::reader::RowIter;
use crate::record::Row;
use crate::schema::types::Type as SchemaType;
use crate::thrift::{TCompactSliceInputProtocol, TSerializable};
use bytes::Bytes;
use thrift::protocol::TCompactInputProtocol;
impl TryFrom<File> for SerializedFileReader<File> {
type Error = ParquetError;
fn try_from(file: File) -> Result<Self> {
Self::new(file)
}
}
impl TryFrom<&Path> for SerializedFileReader<File> {
type Error = ParquetError;
fn try_from(path: &Path) -> Result<Self> {
let file = File::open(path)?;
Self::try_from(file)
}
}
impl TryFrom<String> for SerializedFileReader<File> {
type Error = ParquetError;
fn try_from(path: String) -> Result<Self> {
Self::try_from(Path::new(&path))
}
}
impl TryFrom<&str> for SerializedFileReader<File> {
type Error = ParquetError;
fn try_from(path: &str) -> Result<Self> {
Self::try_from(Path::new(&path))
}
}
/// Conversion into a [`RowIter`]
/// using the full file schema over all row groups.
impl IntoIterator for SerializedFileReader<File> {
type Item = Result<Row>;
type IntoIter = RowIter<'static>;
fn into_iter(self) -> Self::IntoIter {
RowIter::from_file_into(Box::new(self))
}
}
// ----------------------------------------------------------------------
// Implementations of file & row group readers
/// A serialized implementation for Parquet [`FileReader`].
pub struct SerializedFileReader<R: ChunkReader> {
chunk_reader: Arc<R>,
metadata: Arc<ParquetMetaData>,
props: ReaderPropertiesPtr,
}
/// A predicate for filtering row groups, invoked with the metadata and index
/// of each row group in the file. Only row groups for which the predicate
/// evaluates to `true` will be scanned
pub type ReadGroupPredicate = Box<dyn FnMut(&RowGroupMetaData, usize) -> bool>;
/// A builder for [`ReadOptions`].
/// For the predicates that are added to the builder,
/// they will be chained using 'AND' to filter the row groups.
#[derive(Default)]
pub struct ReadOptionsBuilder {
predicates: Vec<ReadGroupPredicate>,
enable_page_index: bool,
props: Option<ReaderProperties>,
}
impl ReadOptionsBuilder {
/// New builder
pub fn new() -> Self {
Self::default()
}
/// Add a predicate on row group metadata to the reading option,
/// Filter only row groups that match the predicate criteria
pub fn with_predicate(mut self, predicate: ReadGroupPredicate) -> Self {
self.predicates.push(predicate);
self
}
/// Add a range predicate on filtering row groups if their midpoints are within
/// the Closed-Open range `[start..end) {x | start <= x < end}`
pub fn with_range(mut self, start: i64, end: i64) -> Self {
assert!(start < end);
let predicate = move |rg: &RowGroupMetaData, _: usize| {
let mid = get_midpoint_offset(rg);
mid >= start && mid < end
};
self.predicates.push(Box::new(predicate));
self
}
/// Enable reading the page index structures described in
/// "[Column Index] Layout to Support Page Skipping"
///
/// [Column Index]: https://github.com/apache/parquet-format/blob/master/PageIndex.md
pub fn with_page_index(mut self) -> Self {
self.enable_page_index = true;
self
}
/// Set the [`ReaderProperties`] configuration.
pub fn with_reader_properties(mut self, properties: ReaderProperties) -> Self {
self.props = Some(properties);
self
}
/// Seal the builder and return the read options
pub fn build(self) -> ReadOptions {
let props = self
.props
.unwrap_or_else(|| ReaderProperties::builder().build());
ReadOptions {
predicates: self.predicates,
enable_page_index: self.enable_page_index,
props,
}
}
}
/// A collection of options for reading a Parquet file.
///
/// Currently, only predicates on row group metadata are supported.
/// All predicates will be chained using 'AND' to filter the row groups.
pub struct ReadOptions {
predicates: Vec<ReadGroupPredicate>,
enable_page_index: bool,
props: ReaderProperties,
}
impl<R: 'static + ChunkReader> SerializedFileReader<R> {
/// Creates file reader from a Parquet file.
/// Returns error if Parquet file does not exist or is corrupt.
pub fn new(chunk_reader: R) -> Result<Self> {
let metadata = ParquetMetaDataReader::new().parse_and_finish(&chunk_reader)?;
let props = Arc::new(ReaderProperties::builder().build());
Ok(Self {
chunk_reader: Arc::new(chunk_reader),
metadata: Arc::new(metadata),
props,
})
}
/// Creates file reader from a Parquet file with read options.
/// Returns error if Parquet file does not exist or is corrupt.
pub fn new_with_options(chunk_reader: R, options: ReadOptions) -> Result<Self> {
let mut metadata_builder = ParquetMetaDataReader::new()
.parse_and_finish(&chunk_reader)?
.into_builder();
let mut predicates = options.predicates;
// Filter row groups based on the predicates
for (i, rg_meta) in metadata_builder.take_row_groups().into_iter().enumerate() {
let mut keep = true;
for predicate in &mut predicates {
if !predicate(&rg_meta, i) {
keep = false;
break;
}
}
if keep {
metadata_builder = metadata_builder.add_row_group(rg_meta);
}
}
let mut metadata = metadata_builder.build();
// If page indexes are desired, build them with the filtered set of row groups
if options.enable_page_index {
let mut reader =
ParquetMetaDataReader::new_with_metadata(metadata).with_page_indexes(true);
reader.read_page_indexes(&chunk_reader)?;
metadata = reader.finish()?;
}
Ok(Self {
chunk_reader: Arc::new(chunk_reader),
metadata: Arc::new(metadata),
props: Arc::new(options.props),
})
}
}
/// Get midpoint offset for a row group
fn get_midpoint_offset(meta: &RowGroupMetaData) -> i64 {
let col = meta.column(0);
let mut offset = col.data_page_offset();
if let Some(dic_offset) = col.dictionary_page_offset() {
if offset > dic_offset {
offset = dic_offset
}
};
offset + meta.compressed_size() / 2
}
impl<R: 'static + ChunkReader> FileReader for SerializedFileReader<R> {
fn metadata(&self) -> &ParquetMetaData {
&self.metadata
}
fn num_row_groups(&self) -> usize {
self.metadata.num_row_groups()
}
fn get_row_group(&self, i: usize) -> Result<Box<dyn RowGroupReader + '_>> {
let row_group_metadata = self.metadata.row_group(i);
// Row groups should be processed sequentially.
let props = Arc::clone(&self.props);
let f = Arc::clone(&self.chunk_reader);
Ok(Box::new(SerializedRowGroupReader::new(
f,
row_group_metadata,
self.metadata.offset_index().map(|x| x[i].as_slice()),
props,
)?))
}
fn get_row_iter(&self, projection: Option<SchemaType>) -> Result<RowIter> {
RowIter::from_file(projection, self)
}
}
/// A serialized implementation for Parquet [`RowGroupReader`].
pub struct SerializedRowGroupReader<'a, R: ChunkReader> {
chunk_reader: Arc<R>,
metadata: &'a RowGroupMetaData,
offset_index: Option<&'a [OffsetIndexMetaData]>,
props: ReaderPropertiesPtr,
bloom_filters: Vec<Option<Sbbf>>,
}
impl<'a, R: ChunkReader> SerializedRowGroupReader<'a, R> {
/// Creates new row group reader from a file, row group metadata and custom config.
pub fn new(
chunk_reader: Arc<R>,
metadata: &'a RowGroupMetaData,
offset_index: Option<&'a [OffsetIndexMetaData]>,
props: ReaderPropertiesPtr,
) -> Result<Self> {
let bloom_filters = if props.read_bloom_filter() {
metadata
.columns()
.iter()
.map(|col| Sbbf::read_from_column_chunk(col, chunk_reader.clone()))
.collect::<Result<Vec<_>>>()?
} else {
iter::repeat(None).take(metadata.columns().len()).collect()
};
Ok(Self {
chunk_reader,
metadata,
offset_index,
props,
bloom_filters,
})
}
}
impl<R: 'static + ChunkReader> RowGroupReader for SerializedRowGroupReader<'_, R> {
fn metadata(&self) -> &RowGroupMetaData {
self.metadata
}
fn num_columns(&self) -> usize {
self.metadata.num_columns()
}
// TODO: fix PARQUET-816
fn get_column_page_reader(&self, i: usize) -> Result<Box<dyn PageReader>> {
let col = self.metadata.column(i);
let page_locations = self.offset_index.map(|x| x[i].page_locations.clone());
let props = Arc::clone(&self.props);
Ok(Box::new(SerializedPageReader::new_with_properties(
Arc::clone(&self.chunk_reader),
col,
self.metadata.num_rows() as usize,
page_locations,
props,
)?))
}
/// get bloom filter for the `i`th column
fn get_column_bloom_filter(&self, i: usize) -> Option<&Sbbf> {
self.bloom_filters[i].as_ref()
}
fn get_row_iter(&self, projection: Option<SchemaType>) -> Result<RowIter> {
RowIter::from_row_group(projection, self)
}
}
/// Reads a [`PageHeader`] from the provided [`Read`]
pub(crate) fn read_page_header<T: Read>(input: &mut T) -> Result<PageHeader> {
let mut prot = TCompactInputProtocol::new(input);
let page_header = PageHeader::read_from_in_protocol(&mut prot)?;
Ok(page_header)
}
/// Reads a [`PageHeader`] from the provided [`Read`] returning the number of bytes read
fn read_page_header_len<T: Read>(input: &mut T) -> Result<(usize, PageHeader)> {
/// A wrapper around a [`std::io::Read`] that keeps track of the bytes read
struct TrackedRead<R> {
inner: R,
bytes_read: usize,
}
impl<R: Read> Read for TrackedRead<R> {
fn read(&mut self, buf: &mut [u8]) -> std::io::Result<usize> {
let v = self.inner.read(buf)?;
self.bytes_read += v;
Ok(v)
}
}
let mut tracked = TrackedRead {
inner: input,
bytes_read: 0,
};
let header = read_page_header(&mut tracked)?;
Ok((tracked.bytes_read, header))
}
/// Decodes a [`Page`] from the provided `buffer`
pub(crate) fn decode_page(
page_header: PageHeader,
buffer: Bytes,
physical_type: Type,
decompressor: Option<&mut Box<dyn Codec>>,
) -> Result<Page> {
// Verify the 32-bit CRC checksum of the page
#[cfg(feature = "crc")]
if let Some(expected_crc) = page_header.crc {
let crc = crc32fast::hash(&buffer);
if crc != expected_crc as u32 {
return Err(general_err!("Page CRC checksum mismatch"));
}
}
// When processing data page v2, depending on enabled compression for the
// page, we should account for uncompressed data ('offset') of
// repetition and definition levels.
//
// We always use 0 offset for other pages other than v2, `true` flag means
// that compression will be applied if decompressor is defined
let mut offset: usize = 0;
let mut can_decompress = true;
if let Some(ref header_v2) = page_header.data_page_header_v2 {
offset = (header_v2.definition_levels_byte_length + header_v2.repetition_levels_byte_length)
as usize;
// When is_compressed flag is missing the page is considered compressed
can_decompress = header_v2.is_compressed.unwrap_or(true);
}
// TODO: page header could be huge because of statistics. We should set a
// maximum page header size and abort if that is exceeded.
let buffer = match decompressor {
Some(decompressor) if can_decompress => {
let uncompressed_size = page_header.uncompressed_page_size as usize;
let mut decompressed = Vec::with_capacity(uncompressed_size);
let compressed = &buffer.as_ref()[offset..];
decompressed.extend_from_slice(&buffer.as_ref()[..offset]);
decompressor.decompress(
compressed,
&mut decompressed,
Some(uncompressed_size - offset),
)?;
if decompressed.len() != uncompressed_size {
return Err(general_err!(
"Actual decompressed size doesn't match the expected one ({} vs {})",
decompressed.len(),
uncompressed_size
));
}
Bytes::from(decompressed)
}
_ => buffer,
};
let result = match page_header.type_ {
PageType::DICTIONARY_PAGE => {
let dict_header = page_header.dictionary_page_header.as_ref().ok_or_else(|| {
ParquetError::General("Missing dictionary page header".to_string())
})?;
let is_sorted = dict_header.is_sorted.unwrap_or(false);
Page::DictionaryPage {
buf: buffer,
num_values: dict_header.num_values.try_into()?,
encoding: Encoding::try_from(dict_header.encoding)?,
is_sorted,
}
}
PageType::DATA_PAGE => {
let header = page_header
.data_page_header
.ok_or_else(|| ParquetError::General("Missing V1 data page header".to_string()))?;
Page::DataPage {
buf: buffer,
num_values: header.num_values.try_into()?,
encoding: Encoding::try_from(header.encoding)?,
def_level_encoding: Encoding::try_from(header.definition_level_encoding)?,
rep_level_encoding: Encoding::try_from(header.repetition_level_encoding)?,
statistics: statistics::from_thrift(physical_type, header.statistics)?,
}
}
PageType::DATA_PAGE_V2 => {
let header = page_header
.data_page_header_v2
.ok_or_else(|| ParquetError::General("Missing V2 data page header".to_string()))?;
let is_compressed = header.is_compressed.unwrap_or(true);
Page::DataPageV2 {
buf: buffer,
num_values: header.num_values.try_into()?,
encoding: Encoding::try_from(header.encoding)?,
num_nulls: header.num_nulls.try_into()?,
num_rows: header.num_rows.try_into()?,
def_levels_byte_len: header.definition_levels_byte_length.try_into()?,
rep_levels_byte_len: header.repetition_levels_byte_length.try_into()?,
is_compressed,
statistics: statistics::from_thrift(physical_type, header.statistics)?,
}
}
_ => {
// For unknown page type (e.g., INDEX_PAGE), skip and read next.
unimplemented!("Page type {:?} is not supported", page_header.type_)
}
};
Ok(result)
}
enum SerializedPageReaderState {
Values {
/// The current byte offset in the reader
offset: usize,
/// The length of the chunk in bytes
remaining_bytes: usize,
// If the next page header has already been "peeked", we will cache it and it`s length here
next_page_header: Option<Box<PageHeader>>,
},
Pages {
/// Remaining page locations
page_locations: VecDeque<PageLocation>,
/// Remaining dictionary location if any
dictionary_page: Option<PageLocation>,
/// The total number of rows in this column chunk
total_rows: usize,
},
}
/// A serialized implementation for Parquet [`PageReader`].
pub struct SerializedPageReader<R: ChunkReader> {
/// The chunk reader
reader: Arc<R>,
/// The compression codec for this column chunk. Only set for non-PLAIN codec.
decompressor: Option<Box<dyn Codec>>,
/// Column chunk type.
physical_type: Type,
state: SerializedPageReaderState,
}
impl<R: ChunkReader> SerializedPageReader<R> {
/// Creates a new serialized page reader from a chunk reader and metadata
pub fn new(
reader: Arc<R>,
meta: &ColumnChunkMetaData,
total_rows: usize,
page_locations: Option<Vec<PageLocation>>,
) -> Result<Self> {
let props = Arc::new(ReaderProperties::builder().build());
SerializedPageReader::new_with_properties(reader, meta, total_rows, page_locations, props)
}
/// Creates a new serialized page with custom options.
pub fn new_with_properties(
reader: Arc<R>,
meta: &ColumnChunkMetaData,
total_rows: usize,
page_locations: Option<Vec<PageLocation>>,
props: ReaderPropertiesPtr,
) -> Result<Self> {
let decompressor = create_codec(meta.compression(), props.codec_options())?;
let (start, len) = meta.byte_range();
let state = match page_locations {
Some(locations) => {
let dictionary_page = match locations.first() {
Some(dict_offset) if dict_offset.offset as u64 != start => Some(PageLocation {
offset: start as i64,
compressed_page_size: (dict_offset.offset as u64 - start) as i32,
first_row_index: 0,
}),
_ => None,
};
SerializedPageReaderState::Pages {
page_locations: locations.into(),
dictionary_page,
total_rows,
}
}
None => SerializedPageReaderState::Values {
offset: start as usize,
remaining_bytes: len as usize,
next_page_header: None,
},
};
Ok(Self {
reader,
decompressor,
state,
physical_type: meta.column_type(),
})
}
/// Similar to `peek_next_page`, but returns the offset of the next page instead of the page metadata.
/// Unlike page metadata, an offset can uniquely identify a page.
///
/// This is used when we need to read parquet with row-filter, and we don't want to decompress the page twice.
/// This function allows us to check if the next page is being cached or read previously.
#[cfg(test)]
fn peek_next_page_offset(&mut self) -> Result<Option<usize>> {
match &mut self.state {
SerializedPageReaderState::Values {
offset,
remaining_bytes,
next_page_header,
} => {
loop {
if *remaining_bytes == 0 {
return Ok(None);
}
return if let Some(header) = next_page_header.as_ref() {
if let Ok(_page_meta) = PageMetadata::try_from(&**header) {
Ok(Some(*offset))
} else {
// For unknown page type (e.g., INDEX_PAGE), skip and read next.
*next_page_header = None;
continue;
}
} else {
let mut read = self.reader.get_read(*offset as u64)?;
let (header_len, header) = read_page_header_len(&mut read)?;
*offset += header_len;
*remaining_bytes -= header_len;
let page_meta = if let Ok(_page_meta) = PageMetadata::try_from(&header) {
Ok(Some(*offset))
} else {
// For unknown page type (e.g., INDEX_PAGE), skip and read next.
continue;
};
*next_page_header = Some(Box::new(header));
page_meta
};
}
}
SerializedPageReaderState::Pages {
page_locations,
dictionary_page,
..
} => {
if let Some(page) = dictionary_page {
Ok(Some(page.offset as usize))
} else if let Some(page) = page_locations.front() {
Ok(Some(page.offset as usize))
} else {
Ok(None)
}
}
}
}
}
impl<R: ChunkReader> Iterator for SerializedPageReader<R> {
type Item = Result<Page>;
fn next(&mut self) -> Option<Self::Item> {
self.get_next_page().transpose()
}
}
fn verify_page_header_len(header_len: usize, remaining_bytes: usize) -> Result<()> {
if header_len > remaining_bytes {
return Err(eof_err!("Invalid page header"));
}
Ok(())
}
fn verify_page_size(
compressed_size: i32,
uncompressed_size: i32,
remaining_bytes: usize,
) -> Result<()> {
// The page's compressed size should not exceed the remaining bytes that are
// available to read. The page's uncompressed size is the expected size
// after decompression, which can never be negative.
if compressed_size < 0 || compressed_size as usize > remaining_bytes || uncompressed_size < 0 {
return Err(eof_err!("Invalid page header"));
}
Ok(())
}
impl<R: ChunkReader> PageReader for SerializedPageReader<R> {
fn get_next_page(&mut self) -> Result<Option<Page>> {
loop {
let page = match &mut self.state {
SerializedPageReaderState::Values {
offset,
remaining_bytes: remaining,
next_page_header,
} => {
if *remaining == 0 {
return Ok(None);
}
let mut read = self.reader.get_read(*offset as u64)?;
let header = if let Some(header) = next_page_header.take() {
*header
} else {
let (header_len, header) = read_page_header_len(&mut read)?;
verify_page_header_len(header_len, *remaining)?;
*offset += header_len;
*remaining -= header_len;
header
};
verify_page_size(
header.compressed_page_size,
header.uncompressed_page_size,
*remaining,
)?;
let data_len = header.compressed_page_size as usize;
*offset += data_len;
*remaining -= data_len;
if header.type_ == PageType::INDEX_PAGE {
continue;
}
let mut buffer = Vec::with_capacity(data_len);
let read = read.take(data_len as u64).read_to_end(&mut buffer)?;
if read != data_len {
return Err(eof_err!(
"Expected to read {} bytes of page, read only {}",
data_len,
read
));
}
decode_page(
header,
Bytes::from(buffer),
self.physical_type,
self.decompressor.as_mut(),
)?
}
SerializedPageReaderState::Pages {
page_locations,
dictionary_page,
..
} => {
let front = match dictionary_page
.take()
.or_else(|| page_locations.pop_front())
{
Some(front) => front,
None => return Ok(None),
};
let page_len = front.compressed_page_size as usize;
let buffer = self.reader.get_bytes(front.offset as u64, page_len)?;
let mut prot = TCompactSliceInputProtocol::new(buffer.as_ref());
let header = PageHeader::read_from_in_protocol(&mut prot)?;
let offset = buffer.len() - prot.as_slice().len();
let bytes = buffer.slice(offset..);
decode_page(
header,
bytes,
self.physical_type,
self.decompressor.as_mut(),
)?
}
};
return Ok(Some(page));
}
}
fn peek_next_page(&mut self) -> Result<Option<PageMetadata>> {
match &mut self.state {
SerializedPageReaderState::Values {
offset,
remaining_bytes,
next_page_header,
} => {
loop {
if *remaining_bytes == 0 {
return Ok(None);
}
return if let Some(header) = next_page_header.as_ref() {
if let Ok(page_meta) = (&**header).try_into() {
Ok(Some(page_meta))
} else {
// For unknown page type (e.g., INDEX_PAGE), skip and read next.
*next_page_header = None;
continue;
}
} else {
let mut read = self.reader.get_read(*offset as u64)?;
let (header_len, header) = read_page_header_len(&mut read)?;
verify_page_header_len(header_len, *remaining_bytes)?;
*offset += header_len;
*remaining_bytes -= header_len;
let page_meta = if let Ok(page_meta) = (&header).try_into() {
Ok(Some(page_meta))
} else {
// For unknown page type (e.g., INDEX_PAGE), skip and read next.
continue;
};
*next_page_header = Some(Box::new(header));
page_meta
};
}
}
SerializedPageReaderState::Pages {
page_locations,
dictionary_page,
total_rows,
} => {
if dictionary_page.is_some() {
Ok(Some(PageMetadata {
num_rows: None,
num_levels: None,
is_dict: true,
}))
} else if let Some(page) = page_locations.front() {
let next_rows = page_locations
.get(1)
.map(|x| x.first_row_index as usize)
.unwrap_or(*total_rows);
Ok(Some(PageMetadata {
num_rows: Some(next_rows - page.first_row_index as usize),
num_levels: None,
is_dict: false,
}))
} else {
Ok(None)
}
}
}
}
fn skip_next_page(&mut self) -> Result<()> {
match &mut self.state {
SerializedPageReaderState::Values {
offset,
remaining_bytes,
next_page_header,
} => {
if let Some(buffered_header) = next_page_header.take() {
verify_page_size(
buffered_header.compressed_page_size,
buffered_header.uncompressed_page_size,
*remaining_bytes,
)?;
// The next page header has already been peeked, so just advance the offset
*offset += buffered_header.compressed_page_size as usize;
*remaining_bytes -= buffered_header.compressed_page_size as usize;
} else {
let mut read = self.reader.get_read(*offset as u64)?;
let (header_len, header) = read_page_header_len(&mut read)?;
verify_page_header_len(header_len, *remaining_bytes)?;
verify_page_size(
header.compressed_page_size,
header.uncompressed_page_size,
*remaining_bytes,
)?;
let data_page_size = header.compressed_page_size as usize;
*offset += header_len + data_page_size;
*remaining_bytes -= header_len + data_page_size;
}
Ok(())
}
SerializedPageReaderState::Pages { page_locations, .. } => {
page_locations.pop_front();
Ok(())
}
}
}
fn at_record_boundary(&mut self) -> Result<bool> {
match &mut self.state {
SerializedPageReaderState::Values { .. } => Ok(self.peek_next_page()?.is_none()),
SerializedPageReaderState::Pages { .. } => Ok(true),
}
}
}
#[cfg(test)]
mod tests {
use std::collections::HashSet;
use bytes::Buf;
use crate::file::properties::{EnabledStatistics, WriterProperties};
use crate::format::BoundaryOrder;
use crate::basic::{self, ColumnOrder};
use crate::column::reader::ColumnReader;
use crate::data_type::private::ParquetValueType;
use crate::data_type::{AsBytes, FixedLenByteArrayType, Int32Type};
use crate::file::page_index::index::{Index, NativeIndex};
use crate::file::page_index::index_reader::{read_columns_indexes, read_offset_indexes};
use crate::file::writer::SerializedFileWriter;
use crate::record::RowAccessor;
use crate::schema::parser::parse_message_type;
use crate::util::test_common::file_util::{get_test_file, get_test_path};
use super::*;
#[test]
fn test_cursor_and_file_has_the_same_behaviour() {
let mut buf: Vec<u8> = Vec::new();
get_test_file("alltypes_plain.parquet")
.read_to_end(&mut buf)
.unwrap();
let cursor = Bytes::from(buf);
let read_from_cursor = SerializedFileReader::new(cursor).unwrap();
let test_file = get_test_file("alltypes_plain.parquet");
let read_from_file = SerializedFileReader::new(test_file).unwrap();
let file_iter = read_from_file.get_row_iter(None).unwrap();
let cursor_iter = read_from_cursor.get_row_iter(None).unwrap();
for (a, b) in file_iter.zip(cursor_iter) {
assert_eq!(a.unwrap(), b.unwrap())
}
}
#[test]
fn test_file_reader_try_from() {
// Valid file path
let test_file = get_test_file("alltypes_plain.parquet");
let test_path_buf = get_test_path("alltypes_plain.parquet");
let test_path = test_path_buf.as_path();
let test_path_str = test_path.to_str().unwrap();
let reader = SerializedFileReader::try_from(test_file);
assert!(reader.is_ok());
let reader = SerializedFileReader::try_from(test_path);
assert!(reader.is_ok());
let reader = SerializedFileReader::try_from(test_path_str);
assert!(reader.is_ok());
let reader = SerializedFileReader::try_from(test_path_str.to_string());
assert!(reader.is_ok());
// Invalid file path
let test_path = Path::new("invalid.parquet");
let test_path_str = test_path.to_str().unwrap();
let reader = SerializedFileReader::try_from(test_path);
assert!(reader.is_err());
let reader = SerializedFileReader::try_from(test_path_str);
assert!(reader.is_err());
let reader = SerializedFileReader::try_from(test_path_str.to_string());
assert!(reader.is_err());
}
#[test]
fn test_file_reader_into_iter() {
let path = get_test_path("alltypes_plain.parquet");
let reader = SerializedFileReader::try_from(path.as_path()).unwrap();
let iter = reader.into_iter();
let values: Vec<_> = iter.flat_map(|x| x.unwrap().get_int(0)).collect();
assert_eq!(values, &[4, 5, 6, 7, 2, 3, 0, 1]);
}
#[test]
fn test_file_reader_into_iter_project() {
let path = get_test_path("alltypes_plain.parquet");
let reader = SerializedFileReader::try_from(path.as_path()).unwrap();
let schema = "message schema { OPTIONAL INT32 id; }";
let proj = parse_message_type(schema).ok();
let iter = reader.into_iter().project(proj).unwrap();
let values: Vec<_> = iter.flat_map(|x| x.unwrap().get_int(0)).collect();
assert_eq!(values, &[4, 5, 6, 7, 2, 3, 0, 1]);
}
#[test]
fn test_reuse_file_chunk() {
// This test covers the case of maintaining the correct start position in a file
// stream for each column reader after initializing and moving to the next one
// (without necessarily reading the entire column).
let test_file = get_test_file("alltypes_plain.parquet");
let reader = SerializedFileReader::new(test_file).unwrap();
let row_group = reader.get_row_group(0).unwrap();
let mut page_readers = Vec::new();
for i in 0..row_group.num_columns() {
page_readers.push(row_group.get_column_page_reader(i).unwrap());
}
// Now buffer each col reader, we do not expect any failures like:
// General("underlying Thrift error: end of file")
for mut page_reader in page_readers {
assert!(page_reader.get_next_page().is_ok());
}
}
#[test]
fn test_file_reader() {
let test_file = get_test_file("alltypes_plain.parquet");
let reader_result = SerializedFileReader::new(test_file);
assert!(reader_result.is_ok());
let reader = reader_result.unwrap();
// Test contents in Parquet metadata
let metadata = reader.metadata();
assert_eq!(metadata.num_row_groups(), 1);
// Test contents in file metadata
let file_metadata = metadata.file_metadata();
assert!(file_metadata.created_by().is_some());
assert_eq!(
file_metadata.created_by().unwrap(),
"impala version 1.3.0-INTERNAL (build 8a48ddb1eff84592b3fc06bc6f51ec120e1fffc9)"
);
assert!(file_metadata.key_value_metadata().is_none());
assert_eq!(file_metadata.num_rows(), 8);
assert_eq!(file_metadata.version(), 1);
assert_eq!(file_metadata.column_orders(), None);