polars_io/parquet/read/
read_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
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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
use std::borrow::Cow;
use std::collections::VecDeque;
use std::ops::{Deref, Range};

use arrow::array::BooleanArray;
use arrow::bitmap::MutableBitmap;
use arrow::datatypes::ArrowSchemaRef;
use polars_core::chunked_array::builder::NullChunkedBuilder;
use polars_core::prelude::*;
use polars_core::series::IsSorted;
use polars_core::utils::{accumulate_dataframes_vertical, split_df};
use polars_core::{config, POOL};
use polars_parquet::parquet::error::ParquetResult;
use polars_parquet::parquet::statistics::Statistics;
use polars_parquet::read::{
    self, ColumnChunkMetadata, FileMetadata, Filter, PhysicalType, RowGroupMetadata,
};
use rayon::prelude::*;

#[cfg(feature = "cloud")]
use super::async_impl::FetchRowGroupsFromObjectStore;
use super::mmap::{mmap_columns, ColumnStore};
use super::predicates::read_this_row_group;
use super::to_metadata::ToMetadata;
use super::utils::materialize_empty_df;
use super::{mmap, ParallelStrategy};
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::predicates::{apply_predicate, PhysicalIoExpr};
use crate::utils::get_reader_bytes;
use crate::utils::slice::split_slice_at_file;
use crate::RowIndex;

#[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 casted to the BinaryView / Utf8View variants
        D::Utf8 | D::Binary | D::LargeUtf8 | D::LargeBinary => unreachable!(),

        // These should be casted 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(_, dtype, _) => assert_dtypes(dtype),
        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: &PlHashMap<usize, IsSorted>,
) {
    if let Some(is_sorted) = sorting_map.get(&col_idx) {
        if should_copy_sortedness(series.dtype()) {
            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) -> PlHashMap<usize, IsSorted> {
    let capacity = md.sorting_columns().map_or(0, |s| s.len());
    let mut sorting_map = PlHashMap::with_capacity(capacity);

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

            debug_assert!(prev_value.is_none());
        }
    }

    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> {
    let did_filter = filter.is_some();

    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 stats = columns
        .iter()
        .map(|(col_md, _)| col_md.statistics().transpose())
        .collect::<ParquetResult<Vec<Option<Statistics>>>>();
    let array = mmap::to_deserializer(columns, field.clone(), filter)?;
    let mut series = Series::try_from((field, array))?;

    // We cannot really handle nested metadata at the moment. Just skip it.
    use ArrowDataType as AD;
    match field.dtype() {
        AD::List(_) | AD::LargeList(_) | AD::Struct(_) | AD::FixedSizeList(_, _) => {
            return Ok(series)
        },
        _ => {},
    }

    // We cannot trust the statistics if we filtered the parquet already.
    if did_filter {
        return Ok(series);
    }

    // See if we can find some statistics for this series. If we cannot find anything just return
    // the series as is.
    let Ok(Some(stats)) = stats.map(|mut s| s.pop().flatten()) else {
        return Ok(series);
    };

    let series_trait = series.as_ref();

    macro_rules! match_dtypes_into_metadata {
        ($(($dtype:pat, $phystype:pat) => ($stats:ident, $pldtype:ty),)+) => {
            match (series_trait.dtype(), stats.physical_type()) {
                $(
                ($dtype, $phystype) => {
                    series.try_set_metadata(
                        ToMetadata::<$pldtype>::to_metadata(stats.$stats())
                    );
                })+
                _ => {},
            }
        };
    }

    // Match the data types used by the Series and by the Statistics. If we find a match, set some
    // Metadata for the underlying ChunkedArray.
    use {DataType as D, PhysicalType as P};
    match_dtypes_into_metadata! {
        (D::Boolean, P::Boolean  ) => (expect_as_boolean, BooleanType),
        (D::UInt8,   P::Int32    ) => (expect_as_int32,   UInt8Type  ),
        (D::UInt16,  P::Int32    ) => (expect_as_int32,   UInt16Type ),
        (D::UInt32,  P::Int32    ) => (expect_as_int32,   UInt32Type ),
        (D::UInt64,  P::Int64    ) => (expect_as_int64,   UInt64Type ),
        (D::Int8,    P::Int32    ) => (expect_as_int32,   Int8Type   ),
        (D::Int16,   P::Int32    ) => (expect_as_int32,   Int16Type  ),
        (D::Int32,   P::Int32    ) => (expect_as_int32,   Int32Type  ),
        (D::Int64,   P::Int64    ) => (expect_as_int64,   Int64Type  ),
        (D::Float32, P::Float    ) => (expect_as_float,   Float32Type),
        (D::Float64, P::Double   ) => (expect_as_double,  Float64Type),
        (D::String,  P::ByteArray) => (expect_as_binary,  StringType ),
        (D::Binary,  P::ByteArray) => (expect_as_binary,  BinaryType ),
    }

    Ok(series)
}

#[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,
    slice: (usize, usize),
    file_metadata: &FileMetadata,
    schema: &ArrowSchemaRef,
    predicate: Option<&dyn PhysicalIoExpr>,
    row_index: Option<RowIndex>,
    parallel: ParallelStrategy,
    projection: &[usize],
    use_statistics: bool,
    hive_partition_columns: Option<&[Series]>,
) -> PolarsResult<Vec<DataFrame>> {
    // 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"), slice.1).finish();
            return Ok(vec![DataFrame::new(vec![placeholder
                .into_series()
                .into_column()])?
            .with_row_index(
                row_index.name.clone(),
                Some(row_index.offset + IdxSize::try_from(slice.0).unwrap()),
            )?
            .select(std::iter::once(row_index.name))?]);
        }
    }

    use ParallelStrategy as S;

    if parallel == S::Prefiltered {
        if let Some(predicate) = predicate {
            if let Some(live_variables) = predicate.live_variables() {
                return rg_to_dfs_prefiltered(
                    store,
                    previous_row_count,
                    row_group_start,
                    row_group_end,
                    file_metadata,
                    schema,
                    live_variables,
                    predicate,
                    row_index,
                    projection,
                    use_statistics,
                    hive_partition_columns,
                );
            }
        }
    }

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

/// Load several Parquet row groups as DataFrames while filtering predicate items.
///
/// This strategy works as follows:
///
/// ```text
/// For each Row Group:
///     1. Skip this row group if statistics already filter it out
///     2. Load all the data for the columns needed for the predicate (i.e. the live columns)
///     3. Create a predicate mask.
///     4. Load the filtered data for the columns not in the predicate (i.e. the dead columns)
///     5. Merge the columns into the right DataFrame
/// ```
#[allow(clippy::too_many_arguments)]
fn rg_to_dfs_prefiltered(
    store: &mmap::ColumnStore,
    previous_row_count: &mut IdxSize,
    row_group_start: usize,
    row_group_end: usize,
    file_metadata: &FileMetadata,
    schema: &ArrowSchemaRef,
    live_variables: Vec<PlSmallStr>,
    predicate: &dyn PhysicalIoExpr,
    row_index: Option<RowIndex>,
    projection: &[usize],
    use_statistics: bool,
    hive_partition_columns: Option<&[Series]>,
) -> PolarsResult<Vec<DataFrame>> {
    if row_group_end > u32::MAX as usize {
        polars_bail!(ComputeError: "Parquet file contains too many row groups (> {})", u32::MAX);
    }

    let mut row_offset = *previous_row_count;
    let rg_offsets: Vec<IdxSize> = match row_index {
        None => Vec::new(),
        Some(_) => (row_group_start..row_group_end)
            .map(|index| {
                let md = &file_metadata.row_groups[index];

                let current_offset = row_offset;
                let current_row_count = md.num_rows() as IdxSize;
                row_offset += current_row_count;

                current_offset
            })
            .collect(),
    };

    // Deduplicate the live variables
    let live_variables = live_variables
        .iter()
        .map(Deref::deref)
        .collect::<PlHashSet<_>>();

    // Get the number of live columns
    let num_live_columns = live_variables.len();
    let num_dead_columns = projection.len() - num_live_columns;

    // @NOTE: This is probably already sorted, but just to be sure.
    let mut projection_sorted = projection.to_vec();
    projection_sorted.sort();

    // We create two look-up tables that map indexes offsets into the live- and dead-set onto
    // column indexes of the schema.
    let mut live_idx_to_col_idx = Vec::with_capacity(num_live_columns);
    let mut dead_idx_to_col_idx = Vec::with_capacity(num_dead_columns);
    for &i in projection_sorted.iter() {
        let name = schema.get_at_index(i).unwrap().0.as_str();

        if live_variables.contains(name) {
            live_idx_to_col_idx.push(i);
        } else {
            dead_idx_to_col_idx.push(i);
        }
    }

    debug_assert_eq!(live_idx_to_col_idx.len(), num_live_columns);
    debug_assert_eq!(dead_idx_to_col_idx.len(), num_dead_columns);

    let mask_setting = PrefilterMaskSetting::init_from_env();

    let dfs: Vec<Option<DataFrame>> = POOL.install(move || {
        // Set partitioned fields to prevent quadratic behavior.
        // Ensure all row groups are partitioned.

        (row_group_start..row_group_end)
            .into_par_iter()
            .map(|rg_idx| {
                let md = &file_metadata.row_groups[rg_idx];

                if use_statistics {
                    match read_this_row_group(Some(predicate), md, schema) {
                        Ok(false) => return Ok(None),
                        Ok(true) => {},
                        Err(e) => return Err(e),
                    }
                }

                let sorting_map = create_sorting_map(md);

                // Collect the data for the live columns
                let live_columns = (0..num_live_columns)
                    .into_par_iter()
                    .map(|i| {
                        let col_idx = live_idx_to_col_idx[i];

                        let (name, field) = schema.get_at_index(col_idx).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.dtype, true),
                            ));
                        };

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

                        let mut series =
                            column_idx_to_series(col_idx, part.as_slice(), None, schema, store)?;

                        try_set_sorted_flag(&mut series, col_idx, &sorting_map);

                        Ok(series.into_column())
                    })
                    .collect::<PolarsResult<Vec<_>>>()?;

                // Apply the predicate to the live columns and save the dataframe and the bitmask
                let md = &file_metadata.row_groups[rg_idx];
                let mut df = unsafe { DataFrame::new_no_checks(md.num_rows(), live_columns) };

                materialize_hive_partitions(
                    &mut df,
                    schema.as_ref(),
                    hive_partition_columns,
                    md.num_rows(),
                );
                let s = predicate.evaluate_io(&df)?;
                let mask = s.bool().expect("filter predicates was not of type boolean");

                if let Some(rc) = &row_index {
                    df.with_row_index_mut(rc.name.clone(), Some(rg_offsets[rg_idx] + rc.offset));
                }
                df = df.filter(mask)?;

                let mut filter_mask = MutableBitmap::with_capacity(mask.len());

                // We need to account for the validity of the items
                for chunk in mask.downcast_iter() {
                    match chunk.validity() {
                        None => filter_mask.extend_from_bitmap(chunk.values()),
                        Some(validity) => {
                            filter_mask.extend_from_bitmap(&(validity & chunk.values()))
                        },
                    }
                }

                let filter_mask = filter_mask.freeze();

                debug_assert_eq!(md.num_rows(), filter_mask.len());
                debug_assert_eq!(df.height(), filter_mask.set_bits());

                if filter_mask.set_bits() == 0 {
                    return Ok(None);
                }

                // We don't need to do any further work if there are no dead columns
                if num_dead_columns == 0 {
                    return Ok(Some(df));
                }

                let prefilter_cost = matches!(mask_setting, PrefilterMaskSetting::Auto)
                    .then(|| calc_prefilter_cost(&filter_mask))
                    .unwrap_or_default();

                #[cfg(debug_assertions)]
                {
                    let md = &file_metadata.row_groups[rg_idx];
                    debug_assert_eq!(md.num_rows(), mask.len());
                }

                let n_rows_in_result = filter_mask.set_bits();

                let mut dead_columns = (0..num_dead_columns)
                    .into_par_iter()
                    .map(|i| {
                        let col_idx = dead_idx_to_col_idx[i];

                        let (name, field) = schema.get_at_index(col_idx).unwrap();

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

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

                        let pre = || {
                            column_idx_to_series(
                                col_idx,
                                field_md.as_slice(),
                                Some(Filter::new_masked(filter_mask.clone())),
                                schema,
                                store,
                            )
                        };
                        let post = || {
                            let array = column_idx_to_series(
                                col_idx,
                                field_md.as_slice(),
                                None,
                                schema,
                                store,
                            )?;

                            debug_assert_eq!(array.len(), mask.len());

                            let mask_arr = BooleanArray::new(
                                ArrowDataType::Boolean,
                                filter_mask.clone(),
                                None,
                            );
                            let mask_arr = BooleanChunked::from(mask_arr);
                            array.filter(&mask_arr)
                        };

                        let mut series = if mask_setting.should_prefilter(
                            prefilter_cost,
                            &schema.get_at_index(col_idx).unwrap().1.dtype,
                        ) {
                            pre()?
                        } else {
                            post()?
                        };

                        debug_assert_eq!(series.len(), filter_mask.set_bits());

                        try_set_sorted_flag(&mut series, col_idx, &sorting_map);

                        Ok(series.into_column())
                    })
                    .collect::<PolarsResult<Vec<Column>>>()?;

                debug_assert!(dead_columns.iter().all(|v| v.len() == df.height()));

                let height = df.height();
                let mut live_columns = df.take_columns();

                assert_eq!(
                    live_columns.len() + dead_columns.len(),
                    projection_sorted.len()
                );

                let mut live_idx = 0;
                let mut dead_idx = 0;

                // We create need to re-sort the columns by merging the live and dead columns.
                let columns = projection_sorted
                    .iter()
                    .map(|&i| {
                        let name = schema.get_at_index(i).unwrap().0.as_str();

                        if live_variables.contains(name) {
                            debug_assert!(live_idx < live_columns.len());
                            // SAFETY: We calculate the amount of live_columns in the same way.
                            let column = unsafe { live_columns.as_ptr().add(live_idx).read() };
                            live_idx += 1;
                            column
                        } else {
                            debug_assert!(dead_idx < dead_columns.len());
                            // SAFETY: We calculate the amount of dead_columns in the same way.
                            let column = unsafe { dead_columns.as_ptr().add(dead_idx).read() };
                            dead_idx += 1;
                            column
                        }
                    })
                    .collect::<Vec<Column>>();

                debug_assert_eq!(live_idx, live_columns.len());
                debug_assert_eq!(dead_idx, dead_columns.len());
                debug_assert_eq!(columns.len(), projection_sorted.len());

                // SAFETY: We have now moved all items from live_columns and dead_columns to
                // columns. So we should set the length to 0 to avoid a double free.
                unsafe {
                    live_columns.set_len(0);
                    dead_columns.set_len(0);
                }

                // SAFETY: This is completely based on the schema so all column names are unique
                // and the length is given by the parquet file which should always be the same.
                let df = unsafe { DataFrame::new_no_checks(height, columns) };

                PolarsResult::Ok(Some(df))
            })
            .collect::<PolarsResult<Vec<Option<DataFrame>>>>()
    })?;

    let dfs: Vec<DataFrame> = dfs.into_iter().flatten().collect();

    let row_count: usize = dfs.iter().map(|df| df.height()).sum();
    let row_count = IdxSize::try_from(row_count).map_err(|_| ROW_COUNT_OVERFLOW_ERR)?;
    *previous_row_count = previous_row_count
        .checked_add(row_count)
        .ok_or(ROW_COUNT_OVERFLOW_ERR)?;

    Ok(dfs)
}

#[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,
    predicate: Option<&dyn PhysicalIoExpr>,
    row_index: Option<RowIndex>,
    parallel: ParallelStrategy,
    projection: &[usize],
    use_statistics: bool,
    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;

        if use_statistics
            && !read_this_row_group(predicate, &file_metadata.row_groups[rg_idx], schema)?
        {
            *previous_row_count += rg_slice.1 as IdxSize;
            continue;
        }
        // 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 = if let ParallelStrategy::Columns = parallel {
            POOL.install(|| {
                projection
                    .par_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(),
                                rg_slice.1,
                                &DataType::from_arrow(&field.dtype, true),
                            ));
                        };

                        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())
                    })
                    .collect::<PolarsResult<Vec<_>>>()
            })?
        } else {
            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(),
                            rg_slice.1,
                            &DataType::from_arrow(&field.dtype, true),
                        ));
                    };

                    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())
                })
                .collect::<PolarsResult<Vec<_>>>()?
        };

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

        materialize_hive_partitions(&mut df, schema.as_ref(), hive_partition_columns, rg_slice.1);
        apply_predicate(&mut df, predicate, true)?;

        *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 (polars-u64-idx package on python), \
                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,
    previous_row_count: &mut IdxSize,
    slice: (usize, usize),
    file_metadata: &FileMetadata,
    schema: &ArrowSchemaRef,
    predicate: Option<&dyn PhysicalIoExpr>,
    row_index: Option<RowIndex>,
    projection: &[usize],
    use_statistics: bool,
    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;

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

        if rg_slice.1 == 0 {
            continue;
        }

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

    let dfs = POOL.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 || use_statistics && !read_this_row_group(predicate, md, schema)? {
                    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.dtype, true),
                            ));
                        };

                        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_no_checks(slice.1, columns) };

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

                materialize_hive_partitions(
                    &mut df,
                    schema.as_ref(),
                    hive_partition_columns,
                    slice.1,
                );
                apply_predicate(&mut df, predicate, false)?;

                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,
    slice: (usize, usize),
    projection: Option<&[usize]>,
    reader_schema: &ArrowSchemaRef,
    metadata: Option<FileMetadataRef>,
    predicate: Option<&dyn PhysicalIoExpr>,
    mut parallel: ParallelStrategy,
    row_index: Option<RowIndex>,
    use_statistics: bool,
    hive_partition_columns: Option<&[Series]>,
) -> PolarsResult<DataFrame> {
    // Fast path.
    if 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();

    // if there are multiple row groups and categorical data
    // we need a string cache
    // we keep it alive until the end of the function
    let _sc = if n_row_groups > 1 {
        #[cfg(feature = "dtype-categorical")]
        {
            Some(polars_core::StringCacheHolder::hold())
        }
        #[cfg(not(feature = "dtype-categorical"))]
        {
            Some(0u8)
        }
    } else {
        None
    };

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

    if let Some(predicate) = predicate {
        if std::env::var("POLARS_PARQUET_AUTO_PREFILTERED").is_ok_and(|v| v == "1")
            && predicate.live_variables().map_or(0, |v| v.len()) * n_row_groups
                >= POOL.current_num_threads()
        {
            parallel = ParallelStrategy::Prefiltered;
        }
    }
    if ParallelStrategy::Auto == parallel {
        if n_row_groups > materialized_projection.len() || n_row_groups > POOL.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);
    let store = mmap::ColumnStore::Local(unsafe {
        std::mem::transmute::<ReaderBytes<'_>, ReaderBytes<'static>>(reader).to_memslice()
    });

    let dfs = rg_to_dfs(
        &store,
        &mut 0,
        0,
        n_row_groups,
        slice,
        &file_metadata,
        reader_schema,
        predicate,
        row_index.clone(),
        parallel,
        &materialized_projection,
        use_statistics,
        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 struct FetchRowGroupsFromMmapReader(ReaderBytes<'static>);

impl FetchRowGroupsFromMmapReader {
    pub fn new(mut reader: Box<dyn MmapBytesReader>) -> PolarsResult<Self> {
        // SAFETY: we will keep ownership on the struct and reference the bytes on the heap.
        // this should not work with passed bytes so we check if it is a file
        assert!(reader.to_file().is_some());
        let reader_ptr = unsafe {
            std::mem::transmute::<&mut dyn MmapBytesReader, &'static mut dyn MmapBytesReader>(
                reader.as_mut(),
            )
        };
        let reader_bytes = get_reader_bytes(reader_ptr)?;
        Ok(FetchRowGroupsFromMmapReader(reader_bytes))
    }

    fn fetch_row_groups(&mut self, _row_groups: Range<usize>) -> PolarsResult<ColumnStore> {
        // @TODO: we can something smarter here with mmap
        Ok(mmap::ColumnStore::Local(self.0.to_memslice()))
    }
}

// We couldn't use a trait as async trait gave very hard HRT lifetime errors.
// Maybe a puzzle for another day.
pub enum RowGroupFetcher {
    #[cfg(feature = "cloud")]
    ObjectStore(FetchRowGroupsFromObjectStore),
    Local(FetchRowGroupsFromMmapReader),
}

#[cfg(feature = "cloud")]
impl From<FetchRowGroupsFromObjectStore> for RowGroupFetcher {
    fn from(value: FetchRowGroupsFromObjectStore) -> Self {
        RowGroupFetcher::ObjectStore(value)
    }
}

impl From<FetchRowGroupsFromMmapReader> for RowGroupFetcher {
    fn from(value: FetchRowGroupsFromMmapReader) -> Self {
        RowGroupFetcher::Local(value)
    }
}

impl RowGroupFetcher {
    async fn fetch_row_groups(&mut self, _row_groups: Range<usize>) -> PolarsResult<ColumnStore> {
        match self {
            RowGroupFetcher::Local(f) => f.fetch_row_groups(_row_groups),
            #[cfg(feature = "cloud")]
            RowGroupFetcher::ObjectStore(f) => f.fetch_row_groups(_row_groups).await,
        }
    }
}

pub(super) fn compute_row_group_range(
    row_group_start: usize,
    row_group_end: usize,
    slice: (usize, usize),
    row_groups: &[RowGroupMetadata],
) -> std::ops::Range<usize> {
    let mut start = row_group_start;
    let mut cum_rows: usize = (0..row_group_start).map(|i| row_groups[i].num_rows()).sum();
    let row_group_end = row_groups.len().min(row_group_end);

    loop {
        if start == row_group_end {
            break;
        }

        cum_rows += row_groups[start].num_rows();

        if cum_rows >= slice.0 {
            break;
        }

        start += 1;
    }

    let slice_end = slice.0 + slice.1;
    let mut end = (1 + start).min(row_group_end);

    loop {
        if end == row_group_end {
            break;
        }

        if cum_rows >= slice_end {
            break;
        }

        cum_rows += row_groups[end].num_rows();
        end += 1;
    }

    start..end
}

pub struct BatchedParquetReader {
    // use to keep ownership
    #[allow(dead_code)]
    row_group_fetcher: RowGroupFetcher,
    slice: (usize, usize),
    projection: Arc<[usize]>,
    schema: ArrowSchemaRef,
    metadata: FileMetadataRef,
    predicate: Option<Arc<dyn PhysicalIoExpr>>,
    row_index: Option<RowIndex>,
    rows_read: IdxSize,
    row_group_offset: usize,
    n_row_groups: usize,
    chunks_fifo: VecDeque<DataFrame>,
    parallel: ParallelStrategy,
    chunk_size: usize,
    use_statistics: bool,
    hive_partition_columns: Option<Arc<[Series]>>,
    include_file_path: Option<StringChunked>,
    /// Has returned at least one materialized frame.
    has_returned: bool,
}

impl BatchedParquetReader {
    #[allow(clippy::too_many_arguments)]
    pub fn new(
        row_group_fetcher: RowGroupFetcher,
        metadata: FileMetadataRef,
        schema: ArrowSchemaRef,
        slice: (usize, usize),
        projection: Option<Vec<usize>>,
        predicate: Option<Arc<dyn PhysicalIoExpr>>,
        row_index: Option<RowIndex>,
        chunk_size: usize,
        use_statistics: bool,
        hive_partition_columns: Option<Vec<Series>>,
        include_file_path: Option<(PlSmallStr, Arc<str>)>,
        mut parallel: ParallelStrategy,
    ) -> PolarsResult<Self> {
        let n_row_groups = metadata.row_groups.len();
        let projection = projection
            .map(Arc::from)
            .unwrap_or_else(|| (0usize..schema.len()).collect::<Arc<[_]>>());

        parallel = match parallel {
            ParallelStrategy::Auto => {
                if n_row_groups > projection.len() || n_row_groups > POOL.current_num_threads() {
                    ParallelStrategy::RowGroups
                } else {
                    ParallelStrategy::Columns
                }
            },
            _ => parallel,
        };

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

        Ok(BatchedParquetReader {
            row_group_fetcher,
            slice,
            projection,
            schema,
            metadata,
            row_index,
            rows_read: 0,
            predicate,
            row_group_offset: 0,
            n_row_groups,
            chunks_fifo: VecDeque::with_capacity(POOL.current_num_threads()),
            parallel,
            chunk_size,
            use_statistics,
            hive_partition_columns: hive_partition_columns.map(Arc::from),
            include_file_path: include_file_path
                .map(|(col, path)| StringChunked::full(col, &path, 1)),
            has_returned: false,
        })
    }

    pub fn schema(&self) -> &ArrowSchemaRef {
        &self.schema
    }

    pub fn is_finished(&self) -> bool {
        self.row_group_offset >= self.n_row_groups
    }

    pub fn finishes_this_batch(&self, n: usize) -> bool {
        self.row_group_offset + n > self.n_row_groups
    }

    pub async fn next_batches(&mut self, n: usize) -> PolarsResult<Option<Vec<DataFrame>>> {
        if self.rows_read as usize == self.slice.0 + self.slice.1 && self.has_returned {
            return if self.chunks_fifo.is_empty() {
                Ok(None)
            } else {
                // the range end point must not be greater than the length of the deque
                let n_drainable = std::cmp::min(n, self.chunks_fifo.len());
                Ok(Some(self.chunks_fifo.drain(..n_drainable).collect()))
            };
        }

        let mut skipped_all_rgs = false;
        // fill up fifo stack
        if (self.rows_read as usize) < self.slice.0 + self.slice.1
            && self.row_group_offset < self.n_row_groups
            && self.chunks_fifo.len() < n
        {
            // Ensure we apply the limit on the metadata, before we download the row-groups.
            let row_group_range = compute_row_group_range(
                self.row_group_offset,
                self.row_group_offset + n,
                self.slice,
                &self.metadata.row_groups,
            );

            let store = self
                .row_group_fetcher
                .fetch_row_groups(row_group_range.clone())
                .await?;

            let mut dfs = match store {
                ColumnStore::Local(_) => rg_to_dfs(
                    &store,
                    &mut self.rows_read,
                    row_group_range.start,
                    row_group_range.end,
                    self.slice,
                    &self.metadata,
                    &self.schema,
                    self.predicate.as_deref(),
                    self.row_index.clone(),
                    self.parallel,
                    &self.projection,
                    self.use_statistics,
                    self.hive_partition_columns.as_deref(),
                ),
                #[cfg(feature = "async")]
                ColumnStore::Fetched(b) => {
                    // This branch we spawn the decoding and decompression of the bytes on a rayon task.
                    // This will ensure we don't block the async thread.

                    // Reconstruct as that makes it a 'static.
                    let store = ColumnStore::Fetched(b);
                    let (tx, rx) = tokio::sync::oneshot::channel();

                    // Make everything 'static.
                    let mut rows_read = self.rows_read;
                    let row_index = self.row_index.clone();
                    let predicate = self.predicate.clone();
                    let schema = self.schema.clone();
                    let metadata = self.metadata.clone();
                    let parallel = self.parallel;
                    let projection = self.projection.clone();
                    let use_statistics = self.use_statistics;
                    let hive_partition_columns = self.hive_partition_columns.clone();
                    let slice = self.slice;

                    let f = move || {
                        let dfs = rg_to_dfs(
                            &store,
                            &mut rows_read,
                            row_group_range.start,
                            row_group_range.end,
                            slice,
                            &metadata,
                            &schema,
                            predicate.as_deref(),
                            row_index,
                            parallel,
                            &projection,
                            use_statistics,
                            hive_partition_columns.as_deref(),
                        );

                        // Don't unwrap send attempt - async task could be cancelled.
                        let _ = tx.send((dfs, rows_read));
                    };

                    // Spawn the task and wait on it asynchronously.
                    if POOL.current_thread_index().is_some() {
                        // We are a rayon thread, so we can't use POOL.spawn as it would mean we spawn a task and block until
                        // another rayon thread executes it - we would deadlock if all rayon threads did this.
                        // Safety: The tokio runtime flavor is multi-threaded.
                        tokio::task::block_in_place(f);
                    } else {
                        POOL.spawn(f);
                    };

                    let (dfs, rows_read) = rx.await.unwrap();
                    self.rows_read = rows_read;
                    dfs
                },
            }?;

            if let Some(ca) = self.include_file_path.as_mut() {
                let mut max_len = 0;

                if self.projection.is_empty() {
                    max_len = self.metadata.num_rows;
                } else {
                    for df in &dfs {
                        max_len = std::cmp::max(max_len, df.height());
                    }
                }

                // Re-use the same ChunkedArray
                if ca.len() < max_len {
                    *ca = ca.new_from_index(0, max_len);
                }

                for df in &mut dfs {
                    unsafe {
                        df.with_column_unchecked(
                            ca.slice(
                                0,
                                if !self.projection.is_empty() {
                                    df.height()
                                } else {
                                    self.metadata.num_rows
                                },
                            )
                            .into_column(),
                        )
                    };
                }
            }

            self.row_group_offset += n;

            // case where there is no data in the file
            // the streaming engine needs at least a single chunk
            if self.rows_read == 0 && dfs.is_empty() {
                let mut df = materialize_empty_df(
                    Some(self.projection.as_ref()),
                    &self.schema,
                    self.hive_partition_columns.as_deref(),
                    self.row_index.as_ref(),
                );

                if let Some(ca) = &self.include_file_path {
                    unsafe {
                        df.with_column_unchecked(ca.clear().into_column());
                    }
                };

                return Ok(Some(vec![df]));
            }

            // TODO! this is slower than it needs to be
            // we also need to parallelize over row groups here.

            skipped_all_rgs |= dfs.is_empty();
            for mut df in dfs {
                // make sure that the chunks are not too large
                let n = df.height() / self.chunk_size;
                if n > 1 {
                    for df in split_df(&mut df, n, false) {
                        self.chunks_fifo.push_back(df)
                    }
                } else {
                    self.chunks_fifo.push_back(df)
                }
            }
        } else {
            skipped_all_rgs = !self.has_returned;
        };

        if self.chunks_fifo.is_empty() {
            if skipped_all_rgs {
                self.has_returned = true;
                let mut df = materialize_empty_df(
                    Some(self.projection.as_ref()),
                    &self.schema,
                    self.hive_partition_columns.as_deref(),
                    self.row_index.as_ref(),
                );

                if let Some(ca) = &self.include_file_path {
                    unsafe {
                        df.with_column_unchecked(ca.clear().into_column());
                    }
                };

                Ok(Some(vec![df]))
            } else {
                Ok(None)
            }
        } else {
            let mut chunks = Vec::with_capacity(n);
            let mut i = 0;
            while let Some(df) = self.chunks_fifo.pop_front() {
                chunks.push(df);
                i += 1;
                if i == n {
                    break;
                }
            }

            self.has_returned = true;
            Ok(Some(chunks))
        }
    }

    /// Turn the batched reader into an iterator.
    #[cfg(feature = "async")]
    pub fn iter(self, batches_per_iter: usize) -> BatchedParquetIter {
        BatchedParquetIter {
            batches_per_iter,
            inner: self,
            current_batch: vec![].into_iter(),
        }
    }
}

#[cfg(feature = "async")]
pub struct BatchedParquetIter {
    batches_per_iter: usize,
    inner: BatchedParquetReader,
    current_batch: std::vec::IntoIter<DataFrame>,
}

#[cfg(feature = "async")]
impl BatchedParquetIter {
    // todo! implement stream
    pub(crate) async fn next_(&mut self) -> Option<PolarsResult<DataFrame>> {
        match self.current_batch.next() {
            Some(df) => Some(Ok(df)),
            None => match self.inner.next_batches(self.batches_per_iter).await {
                Err(e) => Some(Err(e)),
                Ok(opt_batch) => {
                    let batch = opt_batch?;
                    self.current_batch = batch.into_iter();
                    self.current_batch.next().map(Ok)
                },
            },
        }
    }
}

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)
}

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 more expensive for nested types so we make the cut-off
                // higher.
                let is_nested = dtype.is_nested();

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