polars-io 0.54.1

IO related logic for the Polars DataFrame library
Documentation
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
use std::borrow::Cow;

use arrow::bitmap::Bitmap;
use arrow::datatypes::ArrowSchemaRef;
use polars_buffer::Buffer;
use polars_core::chunked_array::builder::NullChunkedBuilder;
use polars_core::config;
use polars_core::prelude::*;
use polars_core::runtime::RAYON;
use polars_core::series::IsSorted;
use polars_core::utils::accumulate_dataframes_vertical;
use polars_parquet::read::{self, ColumnChunkMetadata, FileMetadata, Filter, RowGroupMetadata};
use rayon::prelude::*;

use super::mmap::mmap_columns;
use super::utils::materialize_empty_df;
use super::{ParallelStrategy, mmap};
use crate::RowIndex;
use crate::hive::materialize_hive_partitions;
use crate::mmap::{MmapBytesReader, ReaderBytes};
use crate::parquet::metadata::FileMetadataRef;
use crate::parquet::read::ROW_COUNT_OVERFLOW_ERR;
use crate::utils::slice::split_slice_at_file;

#[cfg(debug_assertions)]
// Ensure we get the proper polars types from schema inference
// This saves unneeded casts.
fn assert_dtypes(dtype: &ArrowDataType) {
    use ArrowDataType as D;

    match dtype {
        // These should all be cast to the BinaryView / Utf8View variants
        D::Utf8 | D::Binary | D::LargeUtf8 | D::LargeBinary => unreachable!(),

        // These should be cast to Float32
        D::Float16 => unreachable!(),

        // This should have been converted to a LargeList
        D::List(_) => unreachable!(),

        // This should have been converted to a LargeList(Struct(_))
        D::Map(_, _) => unreachable!(),

        // Recursive checks
        D::Dictionary(_, dtype, _) => assert_dtypes(dtype),
        D::Extension(ext) => assert_dtypes(&ext.inner),
        D::LargeList(inner) => assert_dtypes(&inner.dtype),
        D::FixedSizeList(inner, _) => assert_dtypes(&inner.dtype),
        D::Struct(fields) => fields.iter().for_each(|f| assert_dtypes(f.dtype())),

        _ => {},
    }
}

fn should_copy_sortedness(dtype: &DataType) -> bool {
    // @NOTE: For now, we are a bit conservative with this.
    use DataType as D;

    matches!(
        dtype,
        D::Int8 | D::Int16 | D::Int32 | D::Int64 | D::UInt8 | D::UInt16 | D::UInt32 | D::UInt64
    )
}

pub fn try_set_sorted_flag(series: &mut Series, col_idx: usize, sorting_map: &[(usize, IsSorted)]) {
    let Some((sorted_col, is_sorted)) = sorting_map.first() else {
        return;
    };
    if *sorted_col != col_idx || !should_copy_sortedness(series.dtype()) {
        return;
    }
    if config::verbose() {
        eprintln!(
            "Parquet conserved SortingColumn for column chunk of '{}' to {is_sorted:?}",
            series.name()
        );
    }

    series.set_sorted_flag(*is_sorted);
}

pub fn create_sorting_map(md: &RowGroupMetadata) -> Vec<(usize, IsSorted)> {
    let capacity = md.sorting_columns().map_or(0, |s| s.len());
    let mut sorting_map = Vec::with_capacity(capacity);

    if let Some(sorting_columns) = md.sorting_columns() {
        for sorting in sorting_columns {
            sorting_map.push((
                sorting.column_idx as usize,
                if sorting.descending {
                    IsSorted::Descending
                } else {
                    IsSorted::Ascending
                },
            ))
        }
    }

    sorting_map
}

fn column_idx_to_series(
    column_i: usize,
    // The metadata belonging to this column
    field_md: &[&ColumnChunkMetadata],
    filter: Option<Filter>,
    file_schema: &ArrowSchema,
    store: &mmap::ColumnStore,
) -> PolarsResult<(Series, Bitmap)> {
    let field = file_schema.get_at_index(column_i).unwrap().1;

    #[cfg(debug_assertions)]
    {
        assert_dtypes(field.dtype())
    }
    let columns = mmap_columns(store, field_md);
    let (arrays, pred_true_mask) = mmap::to_deserializer(columns, field.clone(), filter)?;
    let series = Series::try_from((field, arrays))?;

    Ok((series, pred_true_mask))
}

#[allow(clippy::too_many_arguments)]
fn rg_to_dfs(
    store: &mmap::ColumnStore,
    previous_row_count: &mut IdxSize,
    row_group_start: usize,
    row_group_end: usize,
    pre_slice: (usize, usize),
    file_metadata: &FileMetadata,
    schema: &ArrowSchemaRef,
    row_index: Option<RowIndex>,
    parallel: ParallelStrategy,
    projection: &[usize],
    hive_partition_columns: Option<&[Series]>,
) -> PolarsResult<Vec<DataFrame>> {
    if config::verbose() {
        eprintln!("parquet scan with parallel = {parallel:?}");
    }

    // If we are only interested in the row_index, we take a little special path here.
    if projection.is_empty() {
        if let Some(row_index) = row_index {
            let placeholder =
                NullChunkedBuilder::new(PlSmallStr::from_static("__PL_TMP"), pre_slice.1).finish();
            return Ok(vec![
                DataFrame::new_infer_height(vec![placeholder.into_series().into_column()])?
                    .with_row_index(
                        row_index.name.clone(),
                        Some(row_index.offset + IdxSize::try_from(pre_slice.0).unwrap()),
                    )?
                    .select(std::iter::once(row_index.name))?,
            ]);
        }
    }

    use ParallelStrategy as S;

    match parallel {
        S::Columns | S::None => rg_to_dfs_optionally_par_over_columns(
            store,
            previous_row_count,
            row_group_start,
            row_group_end,
            pre_slice,
            file_metadata,
            schema,
            row_index,
            parallel,
            projection,
            hive_partition_columns,
        ),
        _ => rg_to_dfs_par_over_rg(
            store,
            row_group_start,
            row_group_end,
            previous_row_count,
            pre_slice,
            file_metadata,
            schema,
            row_index,
            projection,
            hive_partition_columns,
        ),
    }
}

#[allow(clippy::too_many_arguments)]
// might parallelize over columns
fn rg_to_dfs_optionally_par_over_columns(
    store: &mmap::ColumnStore,
    previous_row_count: &mut IdxSize,
    row_group_start: usize,
    row_group_end: usize,
    slice: (usize, usize),
    file_metadata: &FileMetadata,
    schema: &ArrowSchemaRef,
    row_index: Option<RowIndex>,
    parallel: ParallelStrategy,
    projection: &[usize],
    hive_partition_columns: Option<&[Series]>,
) -> PolarsResult<Vec<DataFrame>> {
    let mut dfs = Vec::with_capacity(row_group_end - row_group_start);

    let mut n_rows_processed: usize = (0..row_group_start)
        .map(|i| file_metadata.row_groups[i].num_rows())
        .sum();
    let slice_end = slice.0 + slice.1;

    for rg_idx in row_group_start..row_group_end {
        let md = &file_metadata.row_groups[rg_idx];

        let rg_slice =
            split_slice_at_file(&mut n_rows_processed, md.num_rows(), slice.0, slice_end);
        let current_row_count = md.num_rows() as IdxSize;

        let sorting_map = create_sorting_map(md);

        let f = |column_i: &usize| {
            let (name, field) = schema.get_at_index(*column_i).unwrap();

            let Some(iter) = md.columns_under_root_iter(name) else {
                return Ok(Column::full_null(
                    name.clone(),
                    rg_slice.1,
                    &DataType::from_arrow_field(field),
                ));
            };

            let part = iter.collect::<Vec<_>>();

            let (mut series, _) = column_idx_to_series(
                *column_i,
                part.as_slice(),
                Some(Filter::new_ranged(rg_slice.0, rg_slice.0 + rg_slice.1)),
                schema,
                store,
            )?;

            try_set_sorted_flag(&mut series, *column_i, &sorting_map);
            Ok(series.into_column())
        };

        let columns = if let ParallelStrategy::Columns = parallel {
            RAYON.install(|| {
                projection
                    .par_iter()
                    .map(f)
                    .collect::<PolarsResult<Vec<_>>>()
            })?
        } else {
            projection.iter().map(f).collect::<PolarsResult<Vec<_>>>()?
        };

        let mut df = unsafe { DataFrame::new_unchecked(rg_slice.1, columns) };
        if let Some(rc) = &row_index {
            unsafe {
                df.with_row_index_mut(
                    rc.name.clone(),
                    Some(*previous_row_count + rc.offset + rg_slice.0 as IdxSize),
                )
            };
        }

        materialize_hive_partitions(&mut df, schema.as_ref(), hive_partition_columns);

        *previous_row_count = previous_row_count
            .checked_add(current_row_count)
            .ok_or_else(|| {
                polars_err!(
                    ComputeError: "Parquet file produces more than pow(2, 32) rows; \
                    consider compiling with polars-bigidx feature (pip install polars[rt64]), \
                    or set 'streaming'"
                )
            })?;
        dfs.push(df);

        if *previous_row_count as usize >= slice_end {
            break;
        }
    }

    Ok(dfs)
}

#[allow(clippy::too_many_arguments)]
// parallelizes over row groups
fn rg_to_dfs_par_over_rg(
    store: &mmap::ColumnStore,
    row_group_start: usize,
    row_group_end: usize,
    rows_read: &mut IdxSize,
    slice: (usize, usize),
    file_metadata: &FileMetadata,
    schema: &ArrowSchemaRef,
    row_index: Option<RowIndex>,
    projection: &[usize],
    hive_partition_columns: Option<&[Series]>,
) -> PolarsResult<Vec<DataFrame>> {
    // compute the limits per row group and the row count offsets
    let mut row_groups = Vec::with_capacity(row_group_end - row_group_start);

    let mut n_rows_processed: usize = (0..row_group_start)
        .map(|i| file_metadata.row_groups[i].num_rows())
        .sum();
    let slice_end = slice.0 + slice.1;

    // rows_scanned is the number of rows that have been scanned so far when checking for overlap with the slice.
    // rows_read is the number of rows found to overlap with the slice, and thus the number of rows that will be
    // read into a dataframe.
    let mut rows_scanned: IdxSize;

    if row_group_start > 0 {
        // In the case of async reads, we need to account for the fact that row_group_start may be greater than
        // zero due to earlier processing.
        // For details, see: https://github.com/pola-rs/polars/pull/20508#discussion_r1900165649
        rows_scanned = (0..row_group_start)
            .map(|i| file_metadata.row_groups[i].num_rows() as IdxSize)
            .sum();
    } else {
        rows_scanned = 0;
    }

    for i in row_group_start..row_group_end {
        let row_count_start = rows_scanned;
        let rg_md = &file_metadata.row_groups[i];
        let n_rows_this_file = rg_md.num_rows();
        let rg_slice =
            split_slice_at_file(&mut n_rows_processed, n_rows_this_file, slice.0, slice_end);
        rows_scanned = rows_scanned
            .checked_add(n_rows_this_file as IdxSize)
            .ok_or(ROW_COUNT_OVERFLOW_ERR)?;

        *rows_read += rg_slice.1 as IdxSize;

        if rg_slice.1 == 0 {
            continue;
        }

        row_groups.push((rg_md, rg_slice, row_count_start));
    }

    let dfs = RAYON.install(|| {
        // Set partitioned fields to prevent quadratic behavior.
        // Ensure all row groups are partitioned.
        row_groups
            .into_par_iter()
            .map(|(md, slice, row_count_start)| {
                if slice.1 == 0 {
                    return Ok(None);
                }
                // test we don't read the parquet file if this env var is set
                #[cfg(debug_assertions)]
                {
                    assert!(std::env::var("POLARS_PANIC_IF_PARQUET_PARSED").is_err())
                }

                let sorting_map = create_sorting_map(md);

                let columns = projection
                    .iter()
                    .map(|column_i| {
                        let (name, field) = schema.get_at_index(*column_i).unwrap();

                        let Some(iter) = md.columns_under_root_iter(name) else {
                            return Ok(Column::full_null(
                                name.clone(),
                                md.num_rows(),
                                &DataType::from_arrow_field(field),
                            ));
                        };

                        let part = iter.collect::<Vec<_>>();

                        let (mut series, _) = column_idx_to_series(
                            *column_i,
                            part.as_slice(),
                            Some(Filter::new_ranged(slice.0, slice.0 + slice.1)),
                            schema,
                            store,
                        )?;

                        try_set_sorted_flag(&mut series, *column_i, &sorting_map);
                        Ok(series.into_column())
                    })
                    .collect::<PolarsResult<Vec<_>>>()?;

                let mut df = unsafe { DataFrame::new_unchecked(slice.1, columns) };

                if let Some(rc) = &row_index {
                    unsafe {
                        df.with_row_index_mut(
                            rc.name.clone(),
                            Some(row_count_start as IdxSize + rc.offset + slice.0 as IdxSize),
                        )
                    };
                }

                materialize_hive_partitions(&mut df, schema.as_ref(), hive_partition_columns);

                Ok(Some(df))
            })
            .collect::<PolarsResult<Vec<_>>>()
    })?;
    Ok(dfs.into_iter().flatten().collect())
}

#[allow(clippy::too_many_arguments)]
pub fn read_parquet<R: MmapBytesReader>(
    mut reader: R,
    pre_slice: (usize, usize),
    projection: Option<&[usize]>,
    reader_schema: &ArrowSchemaRef,
    metadata: Option<FileMetadataRef>,
    mut parallel: ParallelStrategy,
    row_index: Option<RowIndex>,
    hive_partition_columns: Option<&[Series]>,
) -> PolarsResult<DataFrame> {
    // Fast path.
    if pre_slice.1 == 0 {
        return Ok(materialize_empty_df(
            projection,
            reader_schema,
            hive_partition_columns,
            row_index.as_ref(),
        ));
    }

    let file_metadata = metadata
        .map(Ok)
        .unwrap_or_else(|| read::read_metadata(&mut reader).map(Arc::new))?;
    let n_row_groups = file_metadata.row_groups.len();

    let materialized_projection = projection
        .map(Cow::Borrowed)
        .unwrap_or_else(|| Cow::Owned((0usize..reader_schema.len()).collect::<Vec<_>>()));

    if ParallelStrategy::Auto == parallel {
        if n_row_groups > materialized_projection.len()
            || n_row_groups > RAYON.current_num_threads()
        {
            parallel = ParallelStrategy::RowGroups;
        } else {
            parallel = ParallelStrategy::Columns;
        }
    }

    if let (ParallelStrategy::Columns, true) = (parallel, materialized_projection.len() == 1) {
        parallel = ParallelStrategy::None;
    }

    let reader = ReaderBytes::from(&mut reader);
    Buffer::with_slice(&reader, |buf| {
        let store = mmap::ColumnStore::Local(buf);
        let dfs = rg_to_dfs(
            &store,
            &mut 0,
            0,
            n_row_groups,
            pre_slice,
            &file_metadata,
            reader_schema,
            row_index.clone(),
            parallel,
            &materialized_projection,
            hive_partition_columns,
        )?;

        if dfs.is_empty() {
            Ok(materialize_empty_df(
                projection,
                reader_schema,
                hive_partition_columns,
                row_index.as_ref(),
            ))
        } else {
            accumulate_dataframes_vertical(dfs)
        }
    })
}

pub fn calc_prefilter_cost(mask: &arrow::bitmap::Bitmap) -> f64 {
    let num_edges = mask.num_edges() as f64;
    let rg_len = mask.len() as f64;

    // @GB: I did quite some analysis on this.
    //
    // Pre-filtered and Post-filtered can both be faster in certain scenarios.
    //
    // - Pre-filtered is faster when there is some amount of clustering or
    // sorting involved or if the number of values selected is small.
    // - Post-filtering is faster when the predicate selects a somewhat random
    // elements throughout the row group.
    //
    // The following is a heuristic value to try and estimate which one is
    // faster. Essentially, it sees how many times it needs to switch between
    // skipping items and collecting items and compares it against the number
    // of values that it will collect.
    //
    // Closer to 0: pre-filtering is probably better.
    // Closer to 1: post-filtering is probably better.
    (num_edges / rg_len).clamp(0.0, 1.0)
}

#[derive(Clone, Copy)]
pub enum PrefilterMaskSetting {
    Auto,
    Pre,
    Post,
}

impl PrefilterMaskSetting {
    pub fn init_from_env() -> Self {
        std::env::var("POLARS_PQ_PREFILTERED_MASK").map_or(Self::Auto, |v| match &v[..] {
            "auto" => Self::Auto,
            "pre" => Self::Pre,
            "post" => Self::Post,
            _ => panic!("Invalid `POLARS_PQ_PREFILTERED_MASK` value '{v}'."),
        })
    }

    pub fn should_prefilter(&self, prefilter_cost: f64, dtype: &ArrowDataType) -> bool {
        match self {
            Self::Auto => {
                // Prefiltering is only expensive for nested types so we make the cut-off quite
                // high.
                let is_nested = dtype.is_nested();

                // We empirically selected these numbers.
                !is_nested && prefilter_cost <= 0.01
            },
            Self::Pre => true,
            Self::Post => false,
        }
    }
}