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
use std::io::{Read, Seek};
use std::sync::Arc;

use arrow::io::parquet::read;
use arrow::io::parquet::write::FileMetaData;
#[cfg(feature = "async")]
use polars_core::cloud::CloudOptions;
use polars_core::prelude::*;
#[cfg(feature = "serde")]
use serde::{Deserialize, Serialize};

use super::read_impl::FetchRowGroupsFromMmapReader;
use crate::mmap::MmapBytesReader;
#[cfg(feature = "async")]
use crate::parquet::async_impl::FetchRowGroupsFromObjectStore;
#[cfg(feature = "async")]
use crate::parquet::async_impl::ParquetObjectStore;
use crate::parquet::read_impl::read_parquet;
pub use crate::parquet::read_impl::BatchedParquetReader;
use crate::predicates::PhysicalIoExpr;
use crate::prelude::*;
use crate::RowCount;

#[derive(Copy, Clone, Debug, Eq, PartialEq)]
#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
pub enum ParallelStrategy {
    /// Don't parallelize
    None,
    /// Parallelize over the row groups
    Columns,
    /// Parallelize over the columns
    RowGroups,
    /// Automatically determine over which unit to parallelize
    /// This will choose the most occurring unit.
    Auto,
}

impl Default for ParallelStrategy {
    fn default() -> Self {
        ParallelStrategy::Auto
    }
}

/// Read Apache parquet format into a DataFrame.
#[must_use]
pub struct ParquetReader<R: Read + Seek> {
    reader: R,
    rechunk: bool,
    n_rows: Option<usize>,
    columns: Option<Vec<String>>,
    projection: Option<Vec<usize>>,
    parallel: ParallelStrategy,
    row_count: Option<RowCount>,
    low_memory: bool,
    metadata: Option<FileMetaData>,
}

impl<R: MmapBytesReader> ParquetReader<R> {
    #[cfg(feature = "lazy")]
    // todo! hoist to lazy crate
    pub fn _finish_with_scan_ops(
        mut self,
        predicate: Option<Arc<dyn PhysicalIoExpr>>,
        projection: Option<&[usize]>,
    ) -> PolarsResult<DataFrame> {
        // this path takes predicates and parallelism into account
        let metadata = read::read_metadata(&mut self.reader)?;
        let schema = read::schema::infer_schema(&metadata)?;

        let rechunk = self.rechunk;
        read_parquet(
            self.reader,
            self.n_rows.unwrap_or(usize::MAX),
            projection,
            &schema,
            Some(metadata),
            predicate,
            self.parallel,
            self.row_count,
        )
        .map(|mut df| {
            if rechunk {
                df.rechunk();
            };
            df
        })
    }

    /// Try to reduce memory pressure at the expense of performance. If setting this does not reduce memory
    /// enough, turn off parallelization.
    pub fn set_low_memory(mut self, low_memory: bool) -> Self {
        self.low_memory = low_memory;
        self
    }

    /// Read the parquet file in parallel (default). The single threaded reader consumes less memory.
    pub fn read_parallel(mut self, parallel: ParallelStrategy) -> Self {
        self.parallel = parallel;
        self
    }

    /// Stop parsing when `n` rows are parsed. By settings this parameter the csv will be parsed
    /// sequentially.
    pub fn with_n_rows(mut self, num_rows: Option<usize>) -> Self {
        self.n_rows = num_rows;
        self
    }

    /// Columns to select/ project
    pub fn with_columns(mut self, columns: Option<Vec<String>>) -> Self {
        self.columns = columns;
        self
    }

    /// Set the reader's column projection. This counts from 0, meaning that
    /// `vec![0, 4]` would select the 1st and 5th column.
    pub fn with_projection(mut self, projection: Option<Vec<usize>>) -> Self {
        self.projection = projection;
        self
    }

    /// Add a `row_count` column.
    pub fn with_row_count(mut self, row_count: Option<RowCount>) -> Self {
        self.row_count = row_count;
        self
    }

    /// [`Schema`] of the file.
    pub fn schema(&mut self) -> PolarsResult<Schema> {
        let metadata = self.get_metadata()?;
        let schema = read::infer_schema(metadata)?;
        Ok(schema.fields.iter().into())
    }
    /// Number of rows in the parquet file.
    pub fn num_rows(&mut self) -> PolarsResult<usize> {
        let metadata = self.get_metadata()?;
        Ok(metadata.num_rows)
    }
    fn get_metadata(&mut self) -> PolarsResult<&FileMetaData> {
        if self.metadata.is_none() {
            self.metadata = Some(read::read_metadata(&mut self.reader)?);
        }
        Ok(self.metadata.as_ref().unwrap())
    }
}

impl<R: MmapBytesReader + 'static> ParquetReader<R> {
    pub fn batched(mut self, chunk_size: usize) -> PolarsResult<BatchedParquetReader> {
        let metadata = read::read_metadata(&mut self.reader)?;

        let row_group_fetcher = Box::new(FetchRowGroupsFromMmapReader::new(Box::new(self.reader))?);
        BatchedParquetReader::new(
            row_group_fetcher,
            metadata,
            self.n_rows.unwrap_or(usize::MAX),
            self.projection,
            self.row_count,
            chunk_size,
        )
    }
}

impl<R: MmapBytesReader> SerReader<R> for ParquetReader<R> {
    /// Create a new [`ParquetReader`] from an existing `Reader`.
    fn new(reader: R) -> Self {
        ParquetReader {
            reader,
            rechunk: false,
            n_rows: None,
            columns: None,
            projection: None,
            parallel: Default::default(),
            row_count: None,
            low_memory: false,
            metadata: None,
        }
    }

    fn set_rechunk(mut self, rechunk: bool) -> Self {
        self.rechunk = rechunk;
        self
    }

    fn finish(mut self) -> PolarsResult<DataFrame> {
        let metadata = read::read_metadata(&mut self.reader)?;
        let schema = read::schema::infer_schema(&metadata)?;

        if let Some(cols) = &self.columns {
            self.projection = Some(columns_to_projection(cols, &schema)?);
        }

        read_parquet(
            self.reader,
            self.n_rows.unwrap_or(usize::MAX),
            self.projection.as_deref(),
            &schema,
            Some(metadata),
            None,
            self.parallel,
            self.row_count,
        )
        .map(|mut df| {
            if self.rechunk {
                df.rechunk();
            }
            df
        })
    }
}

/// A Parquet reader on top of the async object_store API. Only the batch reader is implemented since
/// parquet files on cloud storage tend to be big and slow to access.
#[cfg(feature = "async")]
pub struct ParquetAsyncReader {
    reader: ParquetObjectStore,
    rechunk: bool,
    n_rows: Option<usize>,
    projection: Option<Vec<usize>>,
    row_count: Option<RowCount>,
    low_memory: bool,
}

#[cfg(feature = "async")]
impl ParquetAsyncReader {
    pub fn from_uri(
        uri: &str,
        cloud_options: Option<&CloudOptions>,
    ) -> PolarsResult<ParquetAsyncReader> {
        Ok(ParquetAsyncReader {
            reader: ParquetObjectStore::from_uri(uri, cloud_options)?,
            rechunk: false,
            n_rows: None,
            projection: None,
            row_count: None,
            low_memory: false,
        })
    }

    /// Fetch the file info in a synchronous way to for the query planning phase.
    #[tokio::main(flavor = "current_thread")]
    pub async fn file_info(
        uri: &str,
        options: Option<&CloudOptions>,
    ) -> PolarsResult<(Schema, usize)> {
        let mut reader = ParquetAsyncReader::from_uri(uri, options)?;
        let schema = reader.schema().await?;
        let num_rows = reader.num_rows().await?;
        Ok((schema, num_rows))
    }

    pub async fn schema(&mut self) -> PolarsResult<Schema> {
        self.reader.schema().await
    }
    pub async fn num_rows(&mut self) -> PolarsResult<usize> {
        self.reader.num_rows().await
    }

    pub fn with_n_rows(mut self, n_rows: Option<usize>) -> Self {
        self.n_rows = n_rows;
        self
    }

    pub fn with_row_count(mut self, row_count: Option<RowCount>) -> Self {
        self.row_count = row_count;
        self
    }

    pub fn set_rechunk(mut self, rechunk: bool) -> Self {
        self.rechunk = rechunk;
        self
    }

    pub fn set_low_memory(mut self, low_memory: bool) -> Self {
        self.low_memory = low_memory;
        self
    }

    pub fn with_projection(mut self, projection: Option<Vec<usize>>) -> Self {
        self.projection = projection;
        self
    }

    #[tokio::main(flavor = "current_thread")]
    pub async fn batched(mut self, chunk_size: usize) -> PolarsResult<BatchedParquetReader> {
        let metadata = self.reader.get_metadata().await?.to_owned();
        let row_group_fetcher = Box::new(FetchRowGroupsFromObjectStore::new(
            self.reader,
            &metadata,
            &self.projection,
        )?);
        BatchedParquetReader::new(
            row_group_fetcher,
            metadata,
            self.n_rows.unwrap_or(usize::MAX),
            self.projection,
            self.row_count,
            chunk_size,
        )
    }
}