polars_io/parquet/read/
async_impl.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
//! Read parquet files in parallel from the Object Store without a third party crate.
use std::ops::Range;

use arrow::datatypes::ArrowSchemaRef;
use bytes::Bytes;
use object_store::path::Path as ObjectPath;
use polars_core::config::{get_rg_prefetch_size, verbose};
use polars_core::prelude::*;
use polars_parquet::read::RowGroupMetadata;
use polars_parquet::write::FileMetadata;
use polars_utils::pl_str::PlSmallStr;
use tokio::sync::mpsc::{channel, Receiver, Sender};
use tokio::sync::Mutex;

use super::mmap::ColumnStore;
use super::predicates::read_this_row_group;
use crate::cloud::{
    build_object_store, object_path_from_str, CloudLocation, CloudOptions, PolarsObjectStore,
};
use crate::parquet::metadata::FileMetadataRef;
use crate::pl_async::get_runtime;
use crate::predicates::PhysicalIoExpr;

type DownloadedRowGroup = PlHashMap<u64, Bytes>;
type QueuePayload = (usize, DownloadedRowGroup);
type QueueSend = Arc<Sender<PolarsResult<QueuePayload>>>;

pub struct ParquetObjectStore {
    store: PolarsObjectStore,
    path: ObjectPath,
    length: Option<usize>,
    metadata: Option<FileMetadataRef>,
}

impl ParquetObjectStore {
    pub async fn from_uri(
        uri: &str,
        options: Option<&CloudOptions>,
        metadata: Option<FileMetadataRef>,
    ) -> PolarsResult<Self> {
        let (CloudLocation { prefix, .. }, store) = build_object_store(uri, options, false).await?;
        let path = object_path_from_str(&prefix)?;

        Ok(ParquetObjectStore {
            store,
            path,
            length: None,
            metadata,
        })
    }

    async fn get_ranges(&self, ranges: &mut [Range<usize>]) -> PolarsResult<PlHashMap<u64, Bytes>> {
        self.store.get_ranges_sort(&self.path, ranges).await
    }

    /// Initialize the length property of the object, unless it has already been fetched.
    async fn length(&mut self) -> PolarsResult<usize> {
        if self.length.is_none() {
            self.length = Some(self.store.head(&self.path).await?.size);
        }
        Ok(self.length.unwrap())
    }

    /// Number of rows in the parquet file.
    pub async fn num_rows(&mut self) -> PolarsResult<usize> {
        let metadata = self.get_metadata().await?;
        Ok(metadata.num_rows)
    }

    /// Fetch the metadata of the parquet file, do not memoize it.
    async fn fetch_metadata(&mut self) -> PolarsResult<FileMetadata> {
        let length = self.length().await?;
        fetch_metadata(&self.store, &self.path, length).await
    }

    /// Fetch and memoize the metadata of the parquet file.
    pub async fn get_metadata(&mut self) -> PolarsResult<&FileMetadataRef> {
        if self.metadata.is_none() {
            self.metadata = Some(Arc::new(self.fetch_metadata().await?));
        }
        Ok(self.metadata.as_ref().unwrap())
    }
}

fn read_n<const N: usize>(reader: &mut &[u8]) -> Option<[u8; N]> {
    if N <= reader.len() {
        let (head, tail) = reader.split_at(N);
        *reader = tail;
        Some(head.try_into().unwrap())
    } else {
        None
    }
}

fn read_i32le(reader: &mut &[u8]) -> Option<i32> {
    read_n(reader).map(i32::from_le_bytes)
}

/// Asynchronously reads the files' metadata
pub async fn fetch_metadata(
    store: &PolarsObjectStore,
    path: &ObjectPath,
    file_byte_length: usize,
) -> PolarsResult<FileMetadata> {
    let footer_header_bytes = store
        .get_range(
            path,
            file_byte_length
                .checked_sub(polars_parquet::parquet::FOOTER_SIZE as usize)
                .ok_or_else(|| {
                    polars_parquet::parquet::error::ParquetError::OutOfSpec(
                        "not enough bytes to contain parquet footer".to_string(),
                    )
                })?..file_byte_length,
        )
        .await?;

    let footer_byte_length: usize = {
        let reader = &mut footer_header_bytes.as_ref();
        let footer_byte_size = read_i32le(reader).unwrap();
        let magic = read_n(reader).unwrap();
        debug_assert!(reader.is_empty());
        if magic != polars_parquet::parquet::PARQUET_MAGIC {
            return Err(polars_parquet::parquet::error::ParquetError::OutOfSpec(
                "incorrect magic in parquet footer".to_string(),
            )
            .into());
        }
        footer_byte_size.try_into().map_err(|_| {
            polars_parquet::parquet::error::ParquetError::OutOfSpec(
                "negative footer byte length".to_string(),
            )
        })?
    };

    let footer_bytes = store
        .get_range(
            path,
            file_byte_length
                .checked_sub(polars_parquet::parquet::FOOTER_SIZE as usize + footer_byte_length)
                .ok_or_else(|| {
                    polars_parquet::parquet::error::ParquetError::OutOfSpec(
                        "not enough bytes to contain parquet footer".to_string(),
                    )
                })?..file_byte_length,
        )
        .await?;

    Ok(polars_parquet::parquet::read::deserialize_metadata(
        std::io::Cursor::new(footer_bytes.as_ref()),
        // TODO: Describe why this makes sense. Taken from the previous
        // implementation which said "a highly nested but sparse struct could
        // result in many allocations".
        footer_bytes.as_ref().len() * 2 + 1024,
    )?)
}

/// Download rowgroups for the column whose indexes are given in `projection`.
/// We concurrently download the columns for each field.
async fn download_projection(
    fields: Arc<[PlSmallStr]>,
    row_group: RowGroupMetadata,
    async_reader: Arc<ParquetObjectStore>,
    sender: QueueSend,
    rg_index: usize,
) -> bool {
    let async_reader = &async_reader;
    let row_group = &row_group;
    let fields = fields.as_ref();

    let mut ranges = Vec::with_capacity(fields.len());
    let mut offsets = Vec::with_capacity(fields.len());
    fields.iter().for_each(|name| {
        // A single column can have multiple matches (structs).
        let iter = row_group
            .columns_under_root_iter(name)
            .unwrap()
            .map(|meta| {
                let byte_range = meta.byte_range();
                let offset = byte_range.start;
                let byte_range = byte_range.start as usize..byte_range.end as usize;
                (offset, byte_range)
            });

        for (offset, range) in iter {
            offsets.push(offset);
            ranges.push(range);
        }
    });

    let result = async_reader
        .get_ranges(&mut ranges)
        .await
        .map(|bytes_map| (rg_index, bytes_map));
    sender.send(result).await.is_ok()
}

async fn download_row_group(
    rg: RowGroupMetadata,
    async_reader: Arc<ParquetObjectStore>,
    sender: QueueSend,
    rg_index: usize,
) -> bool {
    if rg.n_columns() == 0 {
        return true;
    }

    let mut ranges = rg
        .byte_ranges_iter()
        .map(|x| x.start as usize..x.end as usize)
        .collect::<Vec<_>>();

    sender
        .send(
            async_reader
                .get_ranges(&mut ranges)
                .await
                .map(|bytes_map| (rg_index, bytes_map)),
        )
        .await
        .is_ok()
}

pub struct FetchRowGroupsFromObjectStore {
    rg_q: Arc<Mutex<Receiver<PolarsResult<QueuePayload>>>>,
    prefetched_rg: PlHashMap<usize, DownloadedRowGroup>,
}

impl FetchRowGroupsFromObjectStore {
    pub fn new(
        reader: ParquetObjectStore,
        schema: ArrowSchemaRef,
        projection: Option<&[usize]>,
        predicate: Option<Arc<dyn PhysicalIoExpr>>,
        row_group_range: Range<usize>,
        row_groups: &[RowGroupMetadata],
    ) -> PolarsResult<Self> {
        let projected_fields: Option<Arc<[PlSmallStr]>> = projection.map(|projection| {
            projection
                .iter()
                .map(|i| (schema.get_at_index(*i).as_ref().unwrap().0.clone()))
                .collect()
        });

        let mut prefetched: PlHashMap<usize, DownloadedRowGroup> = PlHashMap::new();

        let mut row_groups = if let Some(pred) = predicate.as_deref() {
            row_group_range
                .filter_map(|i| {
                    let rg = &row_groups[i];

                    let should_be_read =
                        matches!(read_this_row_group(Some(pred), rg, &schema), Ok(true));

                    // Already add the row groups that will be skipped to the prefetched data.
                    if !should_be_read {
                        prefetched.insert(i, Default::default());
                    }

                    should_be_read.then(|| (i, rg.clone()))
                })
                .collect::<Vec<_>>()
        } else {
            row_groups.iter().cloned().enumerate().collect()
        };
        let reader = Arc::new(reader);
        let msg_limit = get_rg_prefetch_size();

        if verbose() {
            eprintln!("POLARS ROW_GROUP PREFETCH_SIZE: {}", msg_limit)
        }

        let (snd, rcv) = channel(msg_limit);
        let snd = Arc::new(snd);

        get_runtime().spawn(async move {
            let chunk_len = msg_limit;
            let mut handles = Vec::with_capacity(chunk_len.clamp(0, row_groups.len()));
            for chunk in row_groups.chunks_mut(chunk_len) {
                // Start downloads concurrently
                for (i, rg) in chunk {
                    let rg = std::mem::take(rg);

                    match &projected_fields {
                        Some(projected_fields) => {
                            let handle = tokio::spawn(download_projection(
                                projected_fields.clone(),
                                rg,
                                reader.clone(),
                                snd.clone(),
                                *i,
                            ));
                            handles.push(handle)
                        },
                        None => {
                            let handle = tokio::spawn(download_row_group(
                                rg,
                                reader.clone(),
                                snd.clone(),
                                *i,
                            ));
                            handles.push(handle)
                        },
                    }
                }

                // Wait n - 3 tasks, so we already start the next downloads earlier.
                for task in handles.drain(..handles.len().saturating_sub(3)) {
                    let succeeded = task.await.unwrap();
                    if !succeeded {
                        return;
                    }
                }
            }

            // Drain remaining tasks.
            for task in handles.drain(..) {
                let succeeded = task.await.unwrap();
                if !succeeded {
                    return;
                }
            }
        });

        Ok(FetchRowGroupsFromObjectStore {
            rg_q: Arc::new(Mutex::new(rcv)),
            prefetched_rg: Default::default(),
        })
    }

    pub(crate) async fn fetch_row_groups(
        &mut self,
        row_groups: Range<usize>,
    ) -> PolarsResult<ColumnStore> {
        let mut guard = self.rg_q.lock().await;

        while !row_groups
            .clone()
            .all(|i| self.prefetched_rg.contains_key(&i))
        {
            let Some(fetched) = guard.recv().await else {
                break;
            };
            let (rg_i, payload) = fetched?;

            self.prefetched_rg.insert(rg_i, payload);
        }

        let received = row_groups
            .flat_map(|i| self.prefetched_rg.remove(&i))
            .flat_map(|rg| rg.into_iter())
            .collect::<PlHashMap<_, _>>();

        Ok(ColumnStore::Fetched(received))
    }
}