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
mod compression;
mod indexes;
pub mod levels;
mod metadata;
mod page;
#[cfg(feature = "stream")]
mod stream;

use std::io::{Read, Seek, SeekFrom};
use std::sync::Arc;
use std::vec::IntoIter;

pub use compression::{decompress, BasicDecompressor, Decompressor};
pub use metadata::read_metadata;
#[cfg(feature = "stream")]
pub use page::get_page_stream;
pub use page::{IndexedPageReader, PageFilter, PageIterator, PageReader};
#[cfg(feature = "stream")]
pub use stream::read_metadata as read_metadata_async;

use crate::error::Error;
use crate::metadata::{ColumnChunkMetaData, RowGroupMetaData};
use crate::page::CompressedDataPage;
use crate::schema::types::ParquetType;
use crate::{error::Result, metadata::FileMetaData};

pub use indexes::{read_columns_indexes, read_pages_locations};

/// Filters row group metadata to only those row groups,
/// for which the predicate function returns true
pub fn filter_row_groups(
    metadata: &FileMetaData,
    predicate: &dyn Fn(&RowGroupMetaData, usize) -> bool,
) -> FileMetaData {
    let mut filtered_row_groups = Vec::<RowGroupMetaData>::new();
    for (i, row_group_metadata) in metadata.row_groups.iter().enumerate() {
        if predicate(row_group_metadata, i) {
            filtered_row_groups.push(row_group_metadata.clone());
        }
    }
    let mut metadata = metadata.clone();
    metadata.row_groups = filtered_row_groups;
    metadata
}

/// Returns a new [`PageReader`] by seeking `reader` to the begining of `column_chunk`.
pub fn get_page_iterator<R: Read + Seek>(
    column_chunk: &ColumnChunkMetaData,
    mut reader: R,
    pages_filter: Option<PageFilter>,
    buffer: Vec<u8>,
) -> Result<PageReader<R>> {
    let pages_filter = pages_filter.unwrap_or_else(|| Arc::new(|_, _| true));

    let (col_start, _) = column_chunk.byte_range();
    reader.seek(SeekFrom::Start(col_start))?;
    Ok(PageReader::new(reader, column_chunk, pages_filter, buffer))
}

/// Returns an [`Iterator`] of [`ColumnChunkMetaData`] corresponding to the columns
/// from `field` at `row_group`.
/// For primitive fields (e.g. `i64`), the iterator has exactly one item.
pub fn get_field_columns<'a>(
    metadata: &'a FileMetaData,
    row_group: usize,
    field: &'a ParquetType,
) -> impl Iterator<Item = &'a ColumnChunkMetaData> {
    metadata
        .schema()
        .columns()
        .iter()
        .enumerate()
        .filter(move |x| x.1.path_in_schema[0] == field.name())
        .map(move |x| &metadata.row_groups[row_group].columns()[x.0])
}

/// Returns a [`ColumnIterator`] of column chunks corresponding to `field`.
/// Contrarily to [`get_page_iterator`] that returns a single iterator of pages, this iterator
/// returns multiple iterators, one per physical column of the `field`.
/// For primitive fields (e.g. `i64`), [`ColumnIterator`] yields exactly one column.
/// For complex fields, it yields multiple columns.
pub fn get_column_iterator<R: Read + Seek>(
    reader: R,
    metadata: &FileMetaData,
    row_group: usize,
    field: usize,
    page_filter: Option<PageFilter>,
    page_buffer: Vec<u8>,
) -> ColumnIterator<R> {
    let field = metadata.schema().fields()[field].clone();
    let columns = get_field_columns(metadata, row_group, &field)
        .cloned()
        .collect::<Vec<_>>();

    ColumnIterator::new(reader, field, columns, page_filter, page_buffer)
}

/// State of [`MutStreamingIterator`].
#[derive(Debug)]
pub enum State<T> {
    /// Iterator still has elements
    Some(T),
    /// Iterator finished
    Finished(Vec<u8>),
}

/// A special kind of fallible streaming iterator where `advance` consumes the iterator.
pub trait MutStreamingIterator: Sized {
    type Item;
    type Error;

    fn advance(self) -> std::result::Result<State<Self>, Self::Error>;
    fn get(&mut self) -> Option<&mut Self::Item>;
}

/// Trait describing a [`MutStreamingIterator`] of column chunks.
pub trait ColumnChunkIter<I>:
    MutStreamingIterator<Item = (I, ColumnChunkMetaData), Error = Error>
{
    /// The field associated to the set of column chunks this iterator iterates over.
    fn field(&self) -> &ParquetType;
}

/// A [`MutStreamingIterator`] that reads column chunks one by one,
/// returning a [`PageReader`] per column.
pub struct ColumnIterator<R: Read + Seek> {
    reader: Option<R>,
    field: ParquetType,
    columns: Vec<ColumnChunkMetaData>,
    page_filter: Option<PageFilter>,
    current: Option<(PageReader<R>, ColumnChunkMetaData)>,
    page_buffer: Vec<u8>,
}

impl<R: Read + Seek> ColumnIterator<R> {
    /// Returns a new [`ColumnIterator`]
    pub fn new(
        reader: R,
        field: ParquetType,
        mut columns: Vec<ColumnChunkMetaData>,
        page_filter: Option<PageFilter>,
        page_buffer: Vec<u8>,
    ) -> Self {
        columns.reverse();
        Self {
            reader: Some(reader),
            field,
            page_buffer,
            columns,
            page_filter,
            current: None,
        }
    }
}

impl<R: Read + Seek> MutStreamingIterator for ColumnIterator<R> {
    type Item = (PageReader<R>, ColumnChunkMetaData);
    type Error = Error;

    fn advance(mut self) -> Result<State<Self>> {
        let (reader, buffer) = if let Some((iter, _)) = self.current {
            iter.into_inner()
        } else {
            (self.reader.unwrap(), self.page_buffer)
        };
        if self.columns.is_empty() {
            return Ok(State::Finished(buffer));
        };
        let column = self.columns.pop().unwrap();

        let iter = get_page_iterator(&column, reader, self.page_filter.clone(), buffer)?;
        let current = Some((iter, column));
        Ok(State::Some(Self {
            reader: None,
            field: self.field,
            columns: self.columns,
            page_filter: self.page_filter,
            current,
            page_buffer: vec![],
        }))
    }

    fn get(&mut self) -> Option<&mut Self::Item> {
        self.current.as_mut()
    }
}

impl<R: Read + Seek> ColumnChunkIter<PageReader<R>> for ColumnIterator<R> {
    fn field(&self) -> &ParquetType {
        &self.field
    }
}

/// A [`MutStreamingIterator`] of pre-read column chunks
#[derive(Debug)]
pub struct ReadColumnIterator {
    field: ParquetType,
    chunks: Vec<(Vec<Result<CompressedDataPage>>, ColumnChunkMetaData)>,
    current: Option<(IntoIter<Result<CompressedDataPage>>, ColumnChunkMetaData)>,
}

impl ReadColumnIterator {
    /// Returns a new [`ReadColumnIterator`]
    pub fn new(
        field: ParquetType,
        chunks: Vec<(Vec<Result<CompressedDataPage>>, ColumnChunkMetaData)>,
    ) -> Self {
        Self {
            field,
            chunks,
            current: None,
        }
    }
}

impl MutStreamingIterator for ReadColumnIterator {
    type Item = (IntoIter<Result<CompressedDataPage>>, ColumnChunkMetaData);
    type Error = Error;

    fn advance(mut self) -> Result<State<Self>> {
        if self.chunks.is_empty() {
            return Ok(State::Finished(vec![]));
        }
        self.current = self
            .chunks
            .pop()
            .map(|(pages, meta)| (pages.into_iter(), meta));
        Ok(State::Some(Self {
            field: self.field,
            chunks: self.chunks,
            current: self.current,
        }))
    }

    fn get(&mut self) -> Option<&mut Self::Item> {
        self.current.as_mut()
    }
}

impl ColumnChunkIter<IntoIter<Result<CompressedDataPage>>> for ReadColumnIterator {
    fn field(&self) -> &ParquetType {
        &self.field
    }
}

#[cfg(test)]
mod tests {
    use std::fs::File;

    use crate::FallibleStreamingIterator;

    use super::*;

    use crate::tests::get_path;

    #[test]
    fn basic() -> Result<()> {
        let mut testdata = get_path();
        testdata.push("alltypes_plain.parquet");
        let mut file = File::open(testdata).unwrap();

        let metadata = read_metadata(&mut file)?;

        let row_group = 0;
        let column = 0;
        let column_metadata = &metadata.row_groups[row_group].columns()[column];
        let buffer = vec![];
        let mut iter = get_page_iterator(column_metadata, &mut file, None, buffer)?;

        let page = iter.next().unwrap().unwrap();
        assert_eq!(page.num_values(), 8);
        Ok(())
    }

    #[test]
    fn reuse_buffer() -> Result<()> {
        let mut testdata = get_path();
        testdata.push("alltypes_plain.snappy.parquet");
        let mut file = File::open(testdata).unwrap();

        let metadata = read_metadata(&mut file)?;

        let row_group = 0;
        let column = 0;
        let column_metadata = &metadata.row_groups[row_group].columns()[column];
        let buffer = vec![0];
        let iterator = get_page_iterator(column_metadata, &mut file, None, buffer)?;

        let buffer = vec![];
        let mut iterator = Decompressor::new(iterator, buffer);

        let _ = iterator.next()?.unwrap();

        assert!(iterator.next()?.is_none());
        let (a, b) = iterator.into_buffers();
        assert_eq!(a.len(), 11); // note: compressed is higher in this example.
        assert_eq!(b.len(), 9);

        Ok(())
    }

    #[test]
    fn reuse_buffer_decompress() -> Result<()> {
        let mut testdata = get_path();
        testdata.push("alltypes_plain.parquet");
        let mut file = File::open(testdata).unwrap();

        let metadata = read_metadata(&mut file)?;

        let row_group = 0;
        let column = 0;
        let column_metadata = &metadata.row_groups[row_group].columns()[column];
        let buffer = vec![1];
        let iterator = get_page_iterator(column_metadata, &mut file, None, buffer)?;

        let buffer = vec![];
        let mut iterator = Decompressor::new(iterator, buffer);

        iterator.next()?.unwrap();

        assert!(iterator.next()?.is_none());
        let (a, b) = iterator.into_buffers();

        assert_eq!(a.len(), 11);
        assert_eq!(b.len(), 0); // the decompressed buffer is never used because it is always swapped with the other buffer.

        Ok(())
    }

    #[test]
    fn column_iter() -> Result<()> {
        let mut testdata = get_path();
        testdata.push("alltypes_plain.parquet");
        let mut file = File::open(testdata).unwrap();

        let metadata = read_metadata(&mut file)?;

        let row_group = 0;
        let column = 0;
        let column_metadata = &metadata.row_groups[row_group].columns()[column];
        let buffer = vec![];
        let iter: Vec<_> = get_page_iterator(column_metadata, &mut file, None, buffer)?.collect();

        let field = metadata.schema().fields()[0].clone();
        let mut iter = ReadColumnIterator::new(field, vec![(iter, column_metadata.clone())]);

        loop {
            match iter.advance()? {
                State::Some(mut new_iter) => {
                    if let Some((pages, _descriptor)) = new_iter.get() {
                        let mut iterator = BasicDecompressor::new(pages, vec![]);
                        while let Some(_page) = iterator.next()? {
                            // do something with it
                        }
                        let _internal_buffer = iterator.into_inner();
                    }
                    iter = new_iter;
                }
                State::Finished(_buffer) => {
                    assert!(_buffer.is_empty()); // data is uncompressed => buffer is always moved
                    break;
                }
            }
        }
        Ok(())
    }

    #[test]
    fn basics_column_iterator() -> Result<()> {
        let mut testdata = get_path();
        testdata.push("alltypes_plain.parquet");
        let mut file = File::open(testdata).unwrap();

        let metadata = read_metadata(&mut file)?;

        let mut iter = ColumnIterator::new(
            file,
            metadata.schema().fields()[0].clone(),
            metadata.row_groups[0].columns().to_vec(),
            None,
            vec![],
        );

        loop {
            match iter.advance()? {
                State::Some(mut new_iter) => {
                    if let Some((pages, _descriptor)) = new_iter.get() {
                        let mut iterator = BasicDecompressor::new(pages, vec![]);
                        while let Some(_page) = iterator.next()? {
                            // do something with it
                        }
                        let _internal_buffer = iterator.into_inner();
                    }
                    iter = new_iter;
                }
                State::Finished(_buffer) => {
                    assert!(_buffer.is_empty()); // data is uncompressed => buffer is always moved
                    break;
                }
            }
        }
        Ok(())
    }
}