polars-plan 0.54.1

Lazy query engine for the Polars DataFrame library
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
use std::io::{BufReader, Cursor};
use std::sync::{LazyLock, RwLock};

use either::Either;
use polars_buffer::Buffer;
use polars_core::runtime::ASYNC;
use polars_io::RowIndex;
use polars_io::csv::read::streaming::read_until_start_and_infer_schema;
use polars_io::prelude::*;
use polars_io::utils::byte_source::{ByteSource, DynByteSourceBuilder};
use polars_io::utils::compression::{ByteSourceReader, CompressedReader, SupportedCompression};
use polars_io::utils::stream_buf_reader::ReaderSource;

use super::*;

pub(super) async fn dsl_to_ir(
    sources: ScanSources,
    mut unified_scan_args_box: Box<UnifiedScanArgs>,
    scan_type: Box<FileScanDsl>,
    cached_ir: Arc<Mutex<Option<IR>>>,
    cache_file_info: SourcesToFileInfo,
    verbose: bool,
) -> PolarsResult<()> {
    // Note that the first metadata can still end up being `None` later if the files were
    // filtered from predicate pushdown.
    // Check and drop the lock in its own scope
    let is_not_cached = {
        let cached_ir_guard = cached_ir.lock().unwrap();
        cached_ir_guard.is_none()
    };

    if is_not_cached {
        let unified_scan_args = unified_scan_args_box.as_mut();

        if let Some(hive_schema) = unified_scan_args.hive_options.schema.as_deref() {
            match unified_scan_args.hive_options.enabled {
                // Enable hive_partitioning if it is unspecified but a non-empty hive_schema given
                None if !hive_schema.is_empty() => {
                    unified_scan_args.hive_options.enabled = Some(true)
                },
                // hive_partitioning was explicitly disabled
                Some(false) => polars_bail!(
                    ComputeError:
                    "a hive schema was given but hive_partitioning was disabled"
                ),
                Some(true) | None => {},
            }
        }

        let sources_before_expansion = &sources;

        let sources = match &*scan_type {
            #[cfg(feature = "parquet")]
            FileScanDsl::Parquet { .. } => {
                sources
                    .expand_paths_with_hive_update(unified_scan_args)
                    .await?
            },
            #[cfg(feature = "ipc")]
            FileScanDsl::Ipc { .. } => {
                sources
                    .expand_paths_with_hive_update(unified_scan_args)
                    .await?
            },
            #[cfg(feature = "csv")]
            FileScanDsl::Csv { .. } => sources.expand_paths(unified_scan_args).await?,
            #[cfg(feature = "json")]
            FileScanDsl::NDJson { .. } => sources.expand_paths(unified_scan_args).await?,
            #[cfg(feature = "python")]
            FileScanDsl::PythonDataset { .. } => {
                // There are a lot of places that short-circuit if the paths is empty,
                // so we just give a dummy path here.
                ScanSources::Paths(Buffer::from_iter([PlRefPath::new("PL_PY_DSET")]))
            },
            #[cfg(feature = "scan_lines")]
            FileScanDsl::Lines { .. } => sources.expand_paths(unified_scan_args).await?,
            FileScanDsl::ExpandedPaths { .. } => sources.expand_paths(unified_scan_args).await?,
            FileScanDsl::Anonymous { .. } => sources.clone(),
        };

        // For cloud we must deduplicate files. Serialization/deserialization leads to Arc's losing there
        // sharing.
        let (mut file_info, scan_type_ir) = {
            cache_file_info
                .get_or_insert(
                    &scan_type,
                    &sources,
                    sources_before_expansion,
                    unified_scan_args,
                    verbose,
                )
                .await?
        };

        if unified_scan_args.hive_options.enabled.is_none() {
            // We expect this to be `Some(_)` after this point. If it hasn't been auto-enabled
            // we explicitly set it to disabled.
            unified_scan_args.hive_options.enabled = Some(false);
        }

        let hive_parts = if unified_scan_args.hive_options.enabled.unwrap()
            && let Some(file_schema) = file_info.reader_schema.as_ref()
        {
            let paths = sources
                .as_paths()
                .ok_or_else(|| polars_err!(nyi = "Hive-partitioning of in-memory buffers"))?;

            #[allow(unused_assignments)]
            let mut owned = None;

            hive_partitions_from_paths(
                paths,
                unified_scan_args.hive_options.hive_start_idx,
                unified_scan_args.hive_options.schema.clone(),
                match file_schema {
                    Either::Left(v) => {
                        owned = Some(Schema::from_arrow_schema(v.as_ref()));
                        owned.as_ref().unwrap()
                    },
                    Either::Right(v) => v.as_ref(),
                },
                unified_scan_args.hive_options.try_parse_dates,
            )?
        } else {
            None
        };

        if let Some(ref hive_parts) = hive_parts {
            let hive_schema = hive_parts.schema();
            file_info.update_schema_with_hive_schema(hive_schema.clone());
        } else if let Some(hive_schema) = unified_scan_args.hive_options.schema.clone() {
            // We hit here if we are passed the `hive_schema` to `scan_parquet` but end up with an empty file
            // list during path expansion. In this case we still want to return an empty DataFrame with this
            // schema.
            file_info.update_schema_with_hive_schema(hive_schema);
        }

        if let Some(ref file_path_col) = unified_scan_args.include_file_paths {
            let schema: &mut Schema = Arc::make_mut(&mut file_info.schema);

            if schema.contains(file_path_col) {
                polars_bail!(
                    Duplicate: r#"column name for file paths "{}" conflicts with column name from file"#,
                    file_path_col
                );
            }

            schema.insert_at_index(schema.len(), file_path_col.clone(), DataType::String)?;
        }

        unified_scan_args.projection = if let Some(file_schema) = file_info.reader_schema.as_ref() {
            maybe_init_projection_excluding_hive(
                file_schema,
                hive_parts.as_ref().map(|h| h.schema()),
            )
        } else {
            None
        };

        if let Some(row_index) = &unified_scan_args.row_index {
            let schema = Arc::make_mut(&mut file_info.schema);
            *schema = schema
                .new_inserting_at_index(0, row_index.name.clone(), IDX_DTYPE)
                .unwrap();
        }

        let ir = if sources.is_empty() && !matches!(&(*scan_type), FileScanDsl::Anonymous { .. }) {
            IR::DataFrameScan {
                df: Arc::new(DataFrame::empty_with_schema(&file_info.schema)),
                schema: file_info.schema,
                output_schema: None,
            }
        } else {
            let unified_scan_args = unified_scan_args_box;

            IR::Scan {
                sources,
                file_info,
                hive_parts,
                predicate: None,
                predicate_file_skip_applied: None,
                scan_type: Box::new(scan_type_ir),
                output_schema: None,
                unified_scan_args,
            }
        };

        let mut cached_ir = cached_ir.lock().unwrap();
        cached_ir.replace(ir);
    }

    Ok(())
}

pub(super) fn insert_row_index_to_schema(
    schema: &mut Schema,
    name: PlSmallStr,
) -> PolarsResult<()> {
    if schema.contains(&name) {
        polars_bail!(
            Duplicate:
            "cannot add row_index with name '{}': \
            column already exists in file.",
            name,
        )
    }

    schema.insert_at_index(0, name, IDX_DTYPE)?;

    Ok(())
}

#[cfg(any(feature = "parquet", feature = "ipc"))]
fn prepare_output_schema(
    mut schema: Schema,
    row_index: Option<&RowIndex>,
) -> PolarsResult<SchemaRef> {
    if let Some(rc) = row_index {
        insert_row_index_to_schema(&mut schema, rc.name.clone())?;
    }
    Ok(Arc::new(schema))
}

#[cfg(any(feature = "json", feature = "csv"))]
fn prepare_schemas(
    mut schema: Schema,
    row_index: Option<&RowIndex>,
) -> PolarsResult<(SchemaRef, SchemaRef)> {
    Ok(if let Some(rc) = row_index {
        let reader_schema = schema.clone();
        insert_row_index_to_schema(&mut schema, rc.name.clone())?;
        (Arc::new(reader_schema), Arc::new(schema))
    } else {
        let schema = Arc::new(schema);
        (schema.clone(), schema)
    })
}

#[cfg(feature = "parquet")]
pub(super) async fn parquet_file_info(
    sources: &ScanSources,
    row_index: Option<&RowIndex>,
    #[allow(unused)] cloud_options: Option<&polars_io::cloud::CloudOptions>,
) -> PolarsResult<(
    FileInfo,
    Option<FileMetadataRef>,
    Option<Arc<[FileMetadataRef]>>,
)> {
    use futures::stream::{FuturesOrdered, FuturesUnordered, StreamExt};
    use polars_core::error::feature_gated;

    let n_sources = sources.len();
    let first_scan_source = sources.iter().next().expect("at least one source");

    // First file: schema + num_rows + full metadata. Schema comes from
    // file 0 only (no cross-file schema evolution).
    let (reader_schema, first_num_rows, first_metadata) = {
        if first_scan_source.is_cloud_url() {
            let first_path = first_scan_source.as_path().unwrap();
            feature_gated!("cloud", {
                let mut reader =
                    ParquetObjectStore::from_uri(first_path.clone(), cloud_options, None).await?;

                (
                    reader.schema().await?,
                    reader.num_rows().await?,
                    reader.get_metadata().await?.clone(),
                )
            })
        } else {
            let memslice = first_scan_source.to_memslice()?;
            let mut reader = ParquetReader::new(Cursor::new(memslice));
            (
                reader.schema()?,
                reader.num_rows()?,
                reader.get_metadata()?.clone(),
            )
        }
    };

    let schema =
        prepare_output_schema(Schema::from_arrow_schema(reader_schema.as_ref()), row_index)?;

    // Resolve metadata for sources past the first, dispatched by
    // `POLARS_RESOLVE_METADATA_LEVEL`:
    // - `None`: extrapolate `first_num_rows * n_sources`.
    // - `RowCounts` (OSS default): per-source thrift field 3 only.
    // - `Full` (cloud default): per-source footer, populates
    //   `metadata_per_source` for the distributed scheduler.
    let mode = polars_config::config().resolve_metadata_level();
    let (metadata_per_source, known_size, estimated_size) = if n_sources == 1 {
        (None, Some(first_num_rows), first_num_rows)
    } else {
        use polars_config::ResolveMode;
        match mode {
            ResolveMode::None => (None, None, first_num_rows * n_sources),
            ResolveMode::RowCounts => {
                let mut futures =
                    (1..n_sources)
                        .map(|i| async move {
                            read_parquet_num_rows(sources.at(i), cloud_options).await
                        })
                        .collect::<FuturesUnordered<_>>();

                // Best-effort: a file that fails to decode at plan time (e.g.
                // an invalid file in a hive partition not yet pruned) simply
                // contributes 0 to the estimate. If execution needs the file
                // it will error then; if predicate pushdown prunes it first,
                // it never matters.
                let mut total: usize = first_num_rows;
                while let Some(res) = futures.next().await {
                    if let Ok(n) = res {
                        total = total.saturating_add(n as usize);
                    }
                }
                (None, Some(total), total)
            },
            ResolveMode::Full => {
                // Each file decoded with its own schema: per-file schemas
                // may differ in columns, dtypes, or column order.
                let mut futures = (1..n_sources)
                    .map(|i| read_parquet_metadata(sources.at(i), cloud_options))
                    .collect::<FuturesOrdered<_>>();

                // Push slot 0 (satisfying the `metadata_per_source[0] ==
                // first_metadata` invariant), then push each future result.
                // Best-effort: on error push `first_metadata.clone()`; the
                // cloud scheduler re-fetches if it needs accurate row_groups
                // for that file.
                let mut per_file: Vec<FileMetadataRef> = Vec::with_capacity(n_sources);
                per_file.push(first_metadata.clone());
                let mut total: usize = first_num_rows;
                while let Some(file_result) = futures.next().await {
                    match file_result {
                        Ok(m) => {
                            total = total.saturating_add(m.num_rows);
                            per_file.push(m);
                        },
                        Err(_) => per_file.push(first_metadata.clone()),
                    }
                }
                let dense: Arc<[FileMetadataRef]> = per_file.into();
                (Some(dense), Some(total), total)
            },
        }
    };

    let file_info = FileInfo::new(
        schema,
        Some(Either::Left(reader_schema)),
        (known_size, estimated_size),
    );

    Ok((file_info, Some(first_metadata), metadata_per_source))
}

/// Fetch one source's full footer. Used by [`parquet_file_info`] in
/// `Full` resolve mode.
#[cfg(feature = "parquet")]
async fn read_parquet_metadata(
    source: ScanSourceRef<'_>,
    #[allow(unused)] cloud_options: Option<&polars_io::cloud::CloudOptions>,
) -> PolarsResult<FileMetadataRef> {
    use polars_core::error::feature_gated;

    if source.is_cloud_url() {
        let path = source.as_path().unwrap();
        feature_gated!("cloud", {
            let mut reader =
                ParquetObjectStore::from_uri(path.clone(), cloud_options, None).await?;
            reader.get_metadata().await.cloned()
        })
    } else {
        let memslice = source.to_memslice()?;
        let mut cursor = Cursor::new(memslice);
        let md = polars_parquet::parquet::read::read_metadata(&mut cursor)?;
        Ok(Arc::new(md))
    }
}

/// Fetch one source's `num_rows` (thrift field 3 only); skips
/// schema, row_groups, and the rest. Used by [`parquet_file_info`]
/// in `RowCounts` resolve mode.
#[cfg(feature = "parquet")]
async fn read_parquet_num_rows(
    source: ScanSourceRef<'_>,
    #[allow(unused)] cloud_options: Option<&polars_io::cloud::CloudOptions>,
) -> PolarsResult<i64> {
    use polars_core::error::feature_gated;

    if source.is_cloud_url() {
        let path = source.as_path().unwrap();
        feature_gated!("cloud", {
            let mut reader =
                ParquetObjectStore::from_uri(path.clone(), cloud_options, None).await?;
            reader.num_rows_only().await
        })
    } else {
        let memslice = source.to_memslice()?;
        let mut cursor = Cursor::new(memslice);
        polars_parquet::parquet::read::read_num_rows(&mut cursor).map_err(Into::into)
    }
}

pub fn max_metadata_scan_cached() -> usize {
    static MAX_SCANS_METADATA_CACHED: LazyLock<usize> = LazyLock::new(|| {
        let value = std::env::var("POLARS_MAX_CACHED_METADATA_SCANS").map_or(8, |v| {
            v.parse::<usize>()
                .expect("invalid `POLARS_MAX_CACHED_METADATA_SCANS` value")
        });
        if value == 0 {
            return usize::MAX;
        }
        value
    });
    *MAX_SCANS_METADATA_CACHED
}

// TODO! return metadata arced
#[cfg(feature = "ipc")]
pub(super) async fn ipc_file_info(
    first_scan_source: ScanSourceRef<'_>,
    row_index: Option<&RowIndex>,
    cloud_options: Option<&polars_io::cloud::CloudOptions>,
) -> PolarsResult<(FileInfo, arrow::io::ipc::read::FileMetadata)> {
    use polars_core::error::feature_gated;

    let metadata = match first_scan_source {
        ScanSourceRef::Path(path) => {
            if path.has_scheme() {
                feature_gated!("cloud", {
                    polars_io::ipc::IpcReaderAsync::from_uri(path.clone(), cloud_options)
                        .await?
                        .metadata()
                        .await?
                })
            } else {
                arrow::io::ipc::read::read_file_metadata(&mut std::io::BufReader::new(
                    polars_utils::open_file(path.as_std_path())?,
                ))?
            }
        },
        ScanSourceRef::File(file) => {
            arrow::io::ipc::read::read_file_metadata(&mut std::io::BufReader::new(file))?
        },
        ScanSourceRef::Buffer(buff) => {
            arrow::io::ipc::read::read_file_metadata(&mut std::io::Cursor::new(buff))?
        },
    };

    let file_info = FileInfo::new(
        prepare_output_schema(
            Schema::from_arrow_schema(metadata.schema.as_ref()),
            row_index,
        )?,
        Some(Either::Left(Arc::clone(&metadata.schema))),
        (None, usize::MAX),
    );

    Ok((file_info, metadata))
}

#[cfg(feature = "csv")]
pub async fn csv_file_info(
    sources: &ScanSources,
    _first_scan_source: ScanSourceRef<'_>,
    row_index: Option<&RowIndex>,
    csv_options: &mut CsvReadOptions,
    cloud_options: Option<&polars_io::cloud::CloudOptions>,
    missing_columns_policy: MissingColumnsPolicy,
) -> PolarsResult<FileInfo> {
    use polars_core::error::feature_gated;
    use polars_core::runtime::RAYON;
    use rayon::iter::{IntoParallelIterator, ParallelIterator};

    // Holding _first_scan_source should guarantee sources is not empty.
    debug_assert!(!sources.is_empty());

    // TODO:
    // * See if we can do better than scanning all files if there is a row limit

    // prints the error message if paths is empty.
    let run_async =
        sources.is_cloud_url() || (sources.is_paths() && polars_config::config().force_async());

    let cache_entries = {
        if run_async {
            let sources = sources.clone();
            assert!(sources.as_paths().is_some());

            feature_gated!("cloud", {
                Some(
                    polars_io::file_cache::init_entries_from_uri_list(
                        (0..sources.len())
                            .map(move |i| sources.as_paths().unwrap().get(i).unwrap().clone()),
                        cloud_options,
                    )
                    .await?,
                )
            })
        } else {
            None
        }
    };

    let infer_schema_length = csv_options.infer_schema_length;
    let infer_schema_func = |i| {
        const ASSUMED_COMPRESSION_RATIO: usize = 4;
        let source = sources.at(i);

        let (mem_slice_raw, file_size, decompressed_slice_size_hint) = if run_async
            && let Some(infer_schema_length) = infer_schema_length
        {
            // Only download what we need for schema inference.
            // To do so, we use an iterative two-way progressive trial-and-error download strategy
            // until we either have enough rows, or reached EOF. In every iteration, we either
            // increase fetch_size (download progressively more), or try_read_size (try and
            // decompress more of what we have, in the case of compressed).
            const INITIAL_FETCH: usize = 64 * 1024;

            // Collect metadata.
            let byte_source = ASYNC.block_on(async move {
                source
                    .to_dyn_byte_source(&DynByteSourceBuilder::ObjectStore, cloud_options, None)
                    .await
            })?;
            let byte_source = Arc::new(byte_source);

            let file_size = {
                let byte_source = byte_source.clone();
                ASYNC.block_on(async move { byte_source.get_size().await })?
            };

            let compression = if file_size >= 4 {
                let byte_source = byte_source.clone();
                let magic_range = 0..4;
                let magic_bytes =
                    ASYNC.block_on(async move { byte_source.get_range(magic_range).await })?;
                SupportedCompression::check(&magic_bytes)
            } else {
                None
            };

            let mut offset = 0;
            let mut fetch_size = INITIAL_FETCH;
            let mut try_read_size = INITIAL_FETCH * ASSUMED_COMPRESSION_RATIO;
            let mut truncated_bytes: Vec<u8> = Vec::with_capacity(INITIAL_FETCH);
            let mut reached_eof = false;

            // Collect enough rows to satisfy infer_schema_length.
            let (mem_slice_raw, decompressed_slice_size_hint) = loop {
                let range = offset..std::cmp::min(file_size, offset + fetch_size);

                if range.is_empty() {
                    reached_eof = true
                } else {
                    let byte_source = byte_source.clone();
                    let fetch_bytes =
                        ASYNC.block_on(async move { byte_source.get_range(range).await })?;
                    offset += fetch_bytes.len();
                    truncated_bytes.extend_from_slice(fetch_bytes.as_ref());
                }

                let decompressed_size_hint =
                    Some(offset * compression.map_or(1, |_| ASSUMED_COMPRESSION_RATIO));
                let mut reader = ByteSourceReader::<ReaderSource>::from_memory(
                    Buffer::from_owner(truncated_bytes.clone()),
                )?;

                let read_size = if compression.is_none() {
                    offset
                } else if reached_eof {
                    usize::MAX
                } else {
                    try_read_size
                };

                // Note: if `count_rows_from_reader_par` and therefore also `read_next_slice` were to
                // handle truncated compressed bytes gracefully, we could avoid the following EoF check
                // and remove `try_read_size` from the loop.
                let (slice, bytes_read) =
                    match reader.read_next_slice(&Buffer::new(), read_size, decompressed_size_hint)
                    {
                        Ok(v) => v,
                        // We assume that unexpected EOF indicates that we lack sufficient data.
                        Err(e) if e.kind() == std::io::ErrorKind::UnexpectedEof => {
                            fetch_size *= 2;
                            continue;
                        },
                        Err(e) => Err(e)?,
                    };

                let row_count = polars_io::csv::read::count_rows_from_slice_par(
                    slice.clone(),
                    csv_options.parse_options.quote_char,
                    csv_options.parse_options.comment_prefix.as_ref(),
                    csv_options.parse_options.eol_char,
                    csv_options.has_header,
                    csv_options.skip_lines,
                    csv_options.skip_rows,
                    csv_options.skip_rows_after_header,
                    csv_options.raise_if_empty,
                )?;

                if row_count < infer_schema_length && !reached_eof {
                    if compression.is_some() && bytes_read == read_size {
                        // Decompressor had more to give — read_size too small
                        try_read_size *= 2;
                    } else {
                        // Decompressor exhausted input — need more compressed bytes
                        // Or, no compression
                        fetch_size *= 2;
                    }
                    continue;
                }

                break (Buffer::from_owner(truncated_bytes), Some(bytes_read));
            };
            (mem_slice_raw, file_size, decompressed_slice_size_hint)
        } else {
            let mem_slice_raw =
                source.to_buffer_possibly_async(run_async, cache_entries.as_ref(), i)?;
            let file_size = mem_slice_raw.len();
            let compression = SupportedCompression::check(&mem_slice_raw);
            let decompressed_slice_size_hint = Some(match compression {
                None => file_size,
                Some(_) => file_size * ASSUMED_COMPRESSION_RATIO,
            });
            (mem_slice_raw, file_size, decompressed_slice_size_hint)
        };

        let mut reader = ByteSourceReader::from_memory(mem_slice_raw)?;
        let compression = reader.compression();

        let mut first_row_len = 0;
        let (schema, _) = read_until_start_and_infer_schema(
            csv_options,
            None,
            decompressed_slice_size_hint,
            Some(Box::new(|line| {
                first_row_len = line.len() + 1;
            })),
            &mut reader,
        )?;

        let decompressed_file_size_hint = match compression {
            None => file_size,
            Some(_) => file_size * ASSUMED_COMPRESSION_RATIO,
        };

        // TODO. We can do (much) better by collect statistics as part of row count and/or schema
        // inference, including observed average row_length and compression ratio.
        let estimated_rows =
            (decompressed_file_size_hint as f64 / first_row_len as f64).round() as usize;

        Ok((schema, estimated_rows))
    };

    let merge_func =
        |a: PolarsResult<(Schema, usize)>, b: PolarsResult<(Schema, usize)>| match (a, b) {
            (Err(e), _) | (_, Err(e)) => Err(e),
            (Ok((mut schema_a, row_estimate_a)), Ok((schema_b, row_estimate_b))) => {
                match (schema_a.is_empty(), schema_b.is_empty()) {
                    (true, _) => Ok((schema_b, row_estimate_b)),
                    (_, true) => Ok((schema_a, row_estimate_a)),
                    _ => match missing_columns_policy {
                        MissingColumnsPolicy::Raise => {
                            schema_a.to_supertype(&schema_b)?;
                            Ok((schema_a, row_estimate_a.saturating_add(row_estimate_b)))
                        },
                        MissingColumnsPolicy::Insert => {
                            // Union merge: keep all columns from both schemas,
                            // supertype columns that exist in both.
                            use polars_core::utils::try_get_supertype;
                            for (name, dtype) in schema_b.iter() {
                                match schema_a.get(name) {
                                    Some(existing_dtype) => {
                                        let st = try_get_supertype(existing_dtype, dtype)?;
                                        schema_a.with_column(name.clone(), st);
                                    },
                                    None => {
                                        schema_a.with_column(name.clone(), dtype.clone());
                                    },
                                }
                            }
                            Ok((schema_a, row_estimate_a.saturating_add(row_estimate_b)))
                        },
                    },
                }
            },
        };

    assert!(
        csv_options.schema.is_none(),
        "DSL to IR schema inference should not run if user provides a schema."
    );
    // Run inference in parallel with a specific merge order.
    // TODO: flatten to single level once Schema::to_supertype is commutative.
    let si_results = RAYON.join(
        || infer_schema_func(0),
        || {
            (1..sources.len())
                .into_par_iter()
                .map(infer_schema_func)
                .reduce(|| Ok(Default::default()), merge_func)
        },
    );

    let (inferred_schema, estimated_n_rows) = merge_func(si_results.0, si_results.1)?;
    let inferred_schema_ref = Arc::new(inferred_schema);

    let (schema, reader_schema) = if let Some(rc) = row_index {
        let mut output_schema = (*inferred_schema_ref).clone();
        insert_row_index_to_schema(&mut output_schema, rc.name.clone())?;

        (Arc::new(output_schema), inferred_schema_ref)
    } else {
        (inferred_schema_ref.clone(), inferred_schema_ref)
    };

    Ok(FileInfo::new(
        schema,
        Some(Either::Right(reader_schema)),
        (None, estimated_n_rows),
    ))
}

#[cfg(feature = "json")]
pub async fn ndjson_file_info(
    sources: &ScanSources,
    first_scan_source: ScanSourceRef<'_>,
    row_index: Option<&RowIndex>,
    ndjson_options: &NDJsonReadOptions,
    cloud_options: Option<&polars_io::cloud::CloudOptions>,
) -> PolarsResult<FileInfo> {
    use polars_core::error::feature_gated;

    let run_async =
        sources.is_cloud_url() || (sources.is_paths() && polars_config::config().force_async());

    let cache_entries = {
        if run_async {
            let sources = sources.clone();
            assert!(sources.as_paths().is_some());

            feature_gated!("cloud", {
                Some(
                    polars_io::file_cache::init_entries_from_uri_list(
                        (0..sources.len())
                            .map(move |i| sources.as_paths().unwrap().get(i).unwrap().clone()),
                        cloud_options,
                    )
                    .await?,
                )
            })
        } else {
            None
        }
    };

    let infer_schema_length = ndjson_options.infer_schema_length;

    let mut schema = if let Some(schema) = ndjson_options.schema.clone() {
        schema
    } else if run_async && let Some(infer_schema_length) = infer_schema_length {
        // Only download what we need for schema inference.
        // To do so, we use an iterative two-way progressive trial-and-error download strategy
        // until we either have enough rows, or reached EOF. In every iteration, we either
        // increase fetch_size (download progressively more), or try_read_size (try and
        // decompress more of what we have, in the case of compressed).
        use polars_io::utils::compression::{ByteSourceReader, SupportedCompression};
        use polars_io::utils::stream_buf_reader::ReaderSource;

        const INITIAL_FETCH: usize = 64 * 1024;
        const ASSUMED_COMPRESSION_RATIO: usize = 4;

        let first_scan_source = first_scan_source.into_owned()?.clone();
        let cloud_options = cloud_options.cloned();
        // TODO. Support IOMetrics collection during planning phase.
        let byte_source = ASYNC
            .spawn(async move {
                first_scan_source
                    .as_scan_source_ref()
                    .to_dyn_byte_source(
                        &DynByteSourceBuilder::ObjectStore,
                        cloud_options.as_ref(),
                        None,
                    )
                    .await
            })
            .await
            .unwrap()?;
        let byte_source = Arc::new(byte_source);

        let file_size = {
            let byte_source = byte_source.clone();
            ASYNC
                .spawn(async move { byte_source.get_size().await })
                .await
                .unwrap()?
        };

        let mut offset = 0;
        let mut fetch_size = INITIAL_FETCH;
        let mut try_read_size = INITIAL_FETCH * ASSUMED_COMPRESSION_RATIO;
        let mut truncated_bytes: Vec<u8> = Vec::with_capacity(INITIAL_FETCH);
        let mut reached_eof = false;

        // Collect enough rows to satisfy infer_schema_length
        let memslice = loop {
            let range = offset..std::cmp::min(file_size, offset + fetch_size);

            if range.is_empty() {
                reached_eof = true
            } else {
                let byte_source = byte_source.clone();
                let fetch_bytes = ASYNC
                    .spawn(async move { byte_source.get_range(range).await })
                    .await
                    .unwrap()?;
                offset += fetch_bytes.len();
                truncated_bytes.extend_from_slice(fetch_bytes.as_ref());
            }

            let compression = SupportedCompression::check(&truncated_bytes);
            let mut reader = ByteSourceReader::<ReaderSource>::from_memory(Buffer::from_owner(
                truncated_bytes.clone(),
            ))?;
            let read_size = if compression.is_none() {
                offset
            } else if reached_eof {
                usize::MAX
            } else {
                try_read_size
            };

            let uncompressed_size_hint = Some(
                offset
                    * if compression.is_none() {
                        1
                    } else {
                        ASSUMED_COMPRESSION_RATIO
                    },
            );

            let (slice, bytes_read) =
                match reader.read_next_slice(&Buffer::new(), read_size, uncompressed_size_hint) {
                    Ok(v) => v,
                    // We assume that unexpected EOF indicates that we lack sufficient data.
                    Err(e) if e.kind() == std::io::ErrorKind::UnexpectedEof => {
                        fetch_size *= 2;
                        continue;
                    },
                    Err(e) => Err(e)?,
                };

            if polars_io::ndjson::count_rows(&slice) < infer_schema_length.into() && !reached_eof {
                if compression.is_some() && bytes_read == read_size {
                    // Decompressor had more to give — read_size too small
                    try_read_size *= 2;
                } else {
                    // Decompressor exhausted input — need more compressed bytes
                    // Or, no compression
                    fetch_size *= 2;
                }
                continue;
            }

            break slice;
        };

        let mut buf_reader = BufReader::new(Cursor::new(memslice));
        Arc::new(polars_io::ndjson::infer_schema(
            &mut buf_reader,
            ndjson_options.infer_schema_length,
        )?)
    } else {
        // Download the entire object.
        // Warning - this is potentially memory-expensive in the case of a cloud source, and goes
        // against the design goal of a streaming reader. This can be optimized.
        let mem_slice =
            first_scan_source.to_buffer_possibly_async(run_async, cache_entries.as_ref(), 0)?;
        let mut reader = BufReader::new(CompressedReader::try_new(mem_slice)?);

        Arc::new(polars_io::ndjson::infer_schema(
            &mut reader,
            ndjson_options.infer_schema_length,
        )?)
    };

    if let Some(overwriting_schema) = &ndjson_options.schema_overwrite {
        overwrite_schema(Arc::make_mut(&mut schema), overwriting_schema)?;
    }

    let mut reader_schema = schema.clone();

    if row_index.is_some() {
        (schema, reader_schema) = prepare_schemas(Arc::unwrap_or_clone(schema), row_index)?
    }

    Ok(FileInfo::new(
        schema,
        Some(Either::Right(reader_schema)),
        (None, usize::MAX),
    ))
}

// Add flags that influence metadata/schema here
#[derive(Eq, Hash, PartialEq)]
enum CachedSourceKey {
    ParquetIpc {
        first_path: PlRefPath,
        schema_overwrite: Option<SchemaRef>,
    },
    CsvJson {
        paths: Buffer<PlRefPath>,
        schema: Option<SchemaRef>,
        schema_overwrite: Option<SchemaRef>,
    },
}

#[derive(Default, Clone)]
pub(super) struct SourcesToFileInfo {
    inner: Arc<RwLock<PlHashMap<CachedSourceKey, (FileInfo, FileScanIR)>>>,
}

impl SourcesToFileInfo {
    async fn infer_or_parse(
        &self,
        scan_type: FileScanDsl,
        sources: &ScanSources,
        sources_before_expansion: &ScanSources,
        unified_scan_args: &mut UnifiedScanArgs,
    ) -> PolarsResult<(FileInfo, FileScanIR)> {
        let require_first_source = |failed_operation_name: &'static str, hint: &'static str| {
            sources.first_or_empty_expand_err(
                failed_operation_name,
                sources_before_expansion,
                unified_scan_args.glob,
                hint,
            )
        };

        let cloud_options = unified_scan_args.cloud_options.as_ref();

        Ok(match scan_type {
            #[cfg(feature = "parquet")]
            FileScanDsl::Parquet { options } => {
                if let Some(schema) = &options.schema {
                    // We were passed a schema, we don't have to call `parquet_file_info`,
                    // but this does mean we don't have `row_estimation` and `first_metadata`.

                    (
                        FileInfo {
                            schema: schema.clone(),
                            reader_schema: Some(either::Either::Left(Arc::new(
                                schema.to_arrow(CompatLevel::newest()),
                            ))),
                            row_estimation: (None, usize::MAX),
                        },
                        FileScanIR::Parquet {
                            options,
                            // Schema was passed in; no footer was resolved.
                            first_metadata: None,
                            metadata_per_source: None,
                        },
                    )
                } else {
                    {
                        let first_scan_source = require_first_source(
                            "failed to retrieve first file schema (parquet)",
                            "\
passing a schema can allow \
this scan to succeed with an empty DataFrame.",
                        )?;

                        if verbose() {
                            eprintln!(
                                "sourcing parquet scan file schema from: '{}'",
                                first_scan_source.to_include_path_name()
                            )
                        }

                        let (mut file_info, mut first_metadata, mut metadata_per_source) =
                            scans::parquet_file_info(
                                sources,
                                unified_scan_args.row_index.as_ref(),
                                cloud_options,
                            )
                            .await?;

                        if let Some((total, deleted)) = unified_scan_args.row_count {
                            let size = (total - deleted) as usize;
                            file_info.row_estimation = (Some(size), size);
                        }

                        if self.inner.read().unwrap().len() > max_metadata_scan_cached() {
                            // Cache pressure: drop both pre-decoded slots so
                            // we don't blow memory. Streaming readers fall
                            // back to fetching footers at scan time.
                            first_metadata = None;
                            metadata_per_source = None;
                        }

                        PolarsResult::Ok((
                            file_info,
                            FileScanIR::Parquet {
                                options,
                                first_metadata,
                                metadata_per_source,
                            },
                        ))
                    }
                    .map_err(|e| e.context(failed_here!(parquet scan)))?
                }
            },
            #[cfg(feature = "ipc")]
            FileScanDsl::Ipc { options } => {
                let first_scan_source =
                    require_first_source("failed to retrieve first file schema (ipc)", "")?;

                if verbose() {
                    eprintln!(
                        "sourcing ipc scan file schema from: '{}'",
                        first_scan_source.to_include_path_name()
                    )
                }

                let (file_info, md) = scans::ipc_file_info(
                    first_scan_source,
                    unified_scan_args.row_index.as_ref(),
                    cloud_options,
                )
                .await?;

                PolarsResult::Ok((
                    file_info,
                    FileScanIR::Ipc {
                        options,
                        metadata: Some(Arc::new(md)),
                    },
                ))
            }
            .map_err(|e| e.context(failed_here!(ipc scan)))?,
            #[cfg(feature = "csv")]
            FileScanDsl::Csv { mut options } => {
                let file_info = if let Some(schema) = options.schema.clone() {
                    FileInfo {
                        schema: schema.clone(),
                        reader_schema: Some(either::Either::Right(schema)),
                        row_estimation: (None, usize::MAX),
                    }
                } else {
                    let first_scan_source =
                        require_first_source("failed to retrieve file schemas (csv)", "")?;

                    if verbose() {
                        eprintln!(
                            "sourcing csv scan file schema from: '{}'",
                            first_scan_source.to_include_path_name()
                        )
                    }

                    scans::csv_file_info(
                        sources,
                        first_scan_source,
                        unified_scan_args.row_index.as_ref(),
                        Arc::make_mut(&mut options),
                        cloud_options,
                        unified_scan_args.missing_columns_policy,
                    )
                    .await?
                };

                PolarsResult::Ok((file_info, FileScanIR::Csv { options }))
            }
            .map_err(|e| e.context(failed_here!(csv scan)))?,
            #[cfg(feature = "json")]
            FileScanDsl::NDJson { options } => {
                let file_info = if let Some(schema) = options.schema.clone() {
                    FileInfo {
                        schema: schema.clone(),
                        reader_schema: Some(either::Either::Right(schema)),
                        row_estimation: (None, usize::MAX),
                    }
                } else {
                    let first_scan_source =
                        require_first_source("failed to retrieve first file schema (ndjson)", "")?;

                    if verbose() {
                        eprintln!(
                            "sourcing ndjson scan file schema from: '{}'",
                            first_scan_source.to_include_path_name()
                        )
                    }

                    scans::ndjson_file_info(
                        sources,
                        first_scan_source,
                        unified_scan_args.row_index.as_ref(),
                        &options,
                        cloud_options,
                    )
                    .await?
                };

                PolarsResult::Ok((file_info, FileScanIR::NDJson { options }))
            }
            .map_err(|e| e.context(failed_here!(ndjson scan)))?,
            #[cfg(feature = "python")]
            FileScanDsl::PythonDataset { dataset_object } => (|| {
                if crate::dsl::DATASET_PROVIDER_VTABLE.get().is_none() {
                    polars_bail!(ComputeError: "DATASET_PROVIDER_VTABLE (python) not initialized")
                }

                let mut schema = dataset_object.schema()?;
                let reader_schema = schema.clone();

                if let Some(row_index) = &unified_scan_args.row_index {
                    insert_row_index_to_schema(Arc::make_mut(&mut schema), row_index.name.clone())?;
                }

                PolarsResult::Ok((
                    FileInfo {
                        schema,
                        reader_schema: Some(either::Either::Right(reader_schema)),
                        row_estimation: (None, usize::MAX),
                    },
                    FileScanIR::PythonDataset {
                        dataset_object,
                        cached_ir: Default::default(),
                    },
                ))
            })()
            .map_err(|e| e.context(failed_here!(python dataset scan)))?,
            #[cfg(feature = "scan_lines")]
            FileScanDsl::Lines { name } => {
                let schema = Arc::new(Schema::from_iter([(name.clone(), DataType::String)]));

                (
                    FileInfo {
                        schema: schema.clone(),
                        reader_schema: Some(either::Either::Right(schema.clone())),
                        row_estimation: (None, usize::MAX),
                    },
                    FileScanIR::Lines { name },
                )
            },
            FileScanDsl::ExpandedPaths { name } => {
                let schema = Arc::new(Schema::from_iter([(name.clone(), DataType::String)]));

                (
                    FileInfo {
                        schema: schema.clone(),
                        reader_schema: Some(either::Either::Right(schema.clone())),
                        row_estimation: (Some(sources.len()), sources.len()),
                    },
                    FileScanIR::ExpandedPaths { name },
                )
            },
            FileScanDsl::Anonymous {
                file_info,
                options,
                function,
            } => (file_info, FileScanIR::Anonymous { options, function }),
        })
    }

    pub(super) async fn get_or_insert(
        &self,
        scan_type: &FileScanDsl,
        sources: &ScanSources,
        sources_before_expansion: &ScanSources,
        unified_scan_args: &mut UnifiedScanArgs,
        verbose: bool,
    ) -> PolarsResult<(FileInfo, FileScanIR)> {
        // Only cache non-empty paths. Others are directly parsed.
        let paths = match sources {
            ScanSources::Paths(paths) if !paths.is_empty() => paths.clone(),

            _ => {
                return self
                    .infer_or_parse(
                        scan_type.clone(),
                        sources,
                        sources_before_expansion,
                        unified_scan_args,
                    )
                    .await;
            },
        };

        let (k, v): (CachedSourceKey, Option<(FileInfo, FileScanIR)>) = match scan_type {
            #[cfg(feature = "parquet")]
            FileScanDsl::Parquet { options } => {
                let key = CachedSourceKey::ParquetIpc {
                    first_path: paths[0].clone(),
                    schema_overwrite: options.schema.clone(),
                };

                let guard = self.inner.read().unwrap();
                let v = guard.get(&key);
                (key, v.cloned())
            },
            #[cfg(feature = "ipc")]
            FileScanDsl::Ipc { options: _ } => {
                let key = CachedSourceKey::ParquetIpc {
                    first_path: paths[0].clone(),
                    schema_overwrite: None,
                };

                let guard = self.inner.read().unwrap();
                let v = guard.get(&key);
                (key, v.cloned())
            },
            #[cfg(feature = "csv")]
            FileScanDsl::Csv { options } => {
                let key = CachedSourceKey::CsvJson {
                    paths: paths.clone(),
                    schema: options.schema.clone(),
                    schema_overwrite: options.schema_overwrite.clone(),
                };
                let guard = self.inner.read().unwrap();
                let v = guard.get(&key);
                (key, v.cloned())
            },
            #[cfg(feature = "json")]
            FileScanDsl::NDJson { options } => {
                let key = CachedSourceKey::CsvJson {
                    paths: paths.clone(),
                    schema: options.schema.clone(),
                    schema_overwrite: options.schema_overwrite.clone(),
                };
                let guard = self.inner.read().unwrap();
                let v = guard.get(&key);
                (key, v.cloned())
            },
            _ => {
                return self
                    .infer_or_parse(
                        scan_type.clone(),
                        sources,
                        sources_before_expansion,
                        unified_scan_args,
                    )
                    .await;
            },
        };

        if let Some(out) = v {
            if verbose {
                eprintln!("FILE_INFO CACHE HIT")
            }
            Ok(out)
        } else {
            let v = self
                .infer_or_parse(
                    scan_type.clone(),
                    sources,
                    sources_before_expansion,
                    unified_scan_args,
                )
                .await?;
            self.inner.write().unwrap().insert(k, v.clone());
            Ok(v)
        }
    }
}