buoyant_kernel 0.21.103

Buoyant Data distribution of delta-kernel
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
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
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
//! Default Parquet handler implementation

use std::collections::HashMap;
use std::ops::Range;
use std::sync::Arc;

use futures::stream::{self, BoxStream};
use futures::{StreamExt, TryStreamExt};
use uuid::Uuid;

use super::file_stream::{FileOpenFuture, FileOpener, FileStream};
use super::stats::collect_stats;
use super::UrlExt;
use crate::arrow::array::builder::{MapBuilder, MapFieldNames, StringBuilder};
use crate::arrow::array::{Array, Int64Array, RecordBatch, StringArray, StructArray};
use crate::arrow::datatypes::{DataType, Field, Schema};
use crate::engine::arrow_conversion::{TryFromArrow as _, TryIntoArrow as _};
use crate::engine::arrow_data::ArrowEngineData;
use crate::engine::arrow_utils::{
    fixup_parquet_read, generate_mask, get_requested_indices, ordering_needs_row_indexes,
    RowIndexBuilder,
};
use crate::engine::default::executor::TaskExecutor;
use crate::engine::parquet_row_group_skipping::ParquetRowGroupSkipping;
use crate::engine::{reader_options, writer_options};
use crate::expressions::ColumnName;
use crate::metrics::{MetricEvent, MetricsReporter};
use crate::object_store::path::Path;
use crate::object_store::{DynObjectStore, ObjectStoreExt as _};
use crate::parquet::arrow::arrow_reader::{ArrowReaderMetadata, ParquetRecordBatchReaderBuilder};
use crate::parquet::arrow::arrow_writer::ArrowWriter;
use crate::parquet::arrow::async_reader::{ParquetObjectReader, ParquetRecordBatchStreamBuilder};
use crate::parquet::arrow::async_writer::{AsyncArrowWriter, ParquetObjectWriter};
use crate::schema::{SchemaRef, StructType};
use crate::transaction::WriteContext;
use crate::{
    DeltaResult, EngineData, Error, FileDataReadResultIterator, FileMeta, ParquetFooter,
    ParquetHandler, PredicateRef,
};

#[derive(Debug)]
pub struct DefaultParquetHandler<E: TaskExecutor> {
    store: Arc<DynObjectStore>,
    task_executor: Arc<E>,
    readahead: usize,
    /// Optional reporter for emitting [`MetricEvent::ParquetReadCompleted`] events.
    reporter: Option<Arc<dyn MetricsReporter>>,
}

/// Metadata of a data file (typically a parquet file).
#[derive(Debug)]
pub struct DataFileMetadata {
    file_meta: FileMeta,
    /// Collected statistics for this file (includes numRecords, tightBounds, etc.).
    stats: StructArray,
}

impl DataFileMetadata {
    pub fn new(file_meta: FileMeta, stats: StructArray) -> Self {
        Self { file_meta, stats }
    }

    /// Returns the absolute URL of the written file.
    pub fn location(&self) -> &url::Url {
        &self.file_meta.location
    }

    /// Converts this `DataFileMetadata` into an [`EngineData`] record batch matching the schema
    /// returned by [`Transaction::add_files_schema`].
    ///
    /// The `partition_values` map uses physical column names as keys and protocol-serialized
    /// strings as values. `None` represents a null partition value. The serialization layer
    /// converts nulls and empty strings to `None` before reaching this method, so `Some("")`
    /// is not expected in normal usage.
    ///
    /// The `log_path` is the path string written to the Delta log's `add.path` field.
    ///
    /// [`Transaction::add_files_schema`]: crate::transaction::Transaction::add_files_schema
    pub(crate) fn as_record_batch(
        &self,
        partition_values: &HashMap<String, Option<String>>,
        log_path: &str,
    ) -> DeltaResult<Box<dyn EngineData>> {
        let path = Arc::new(StringArray::from(vec![log_path]));
        let key_builder = StringBuilder::new();
        let val_builder = StringBuilder::new();
        let names = MapFieldNames {
            entry: "key_value".to_string(),
            key: "key".to_string(),
            value: "value".to_string(),
        };
        let mut builder = MapBuilder::new(Some(names), key_builder, val_builder);
        for (k, v) in partition_values {
            builder.keys().append_value(k);
            match v.as_deref() {
                // The serialization layer already converts empty strings to None, so
                // Some("") should not occur. The empty check is purely defensive.
                Some(val) if !val.is_empty() => builder.values().append_value(val),
                _ => builder.values().append_null(),
            }
        }
        builder.append(true)?;
        let partitions = Arc::new(builder.finish());
        // this means max size we can write is i64::MAX (~8EB)
        let size: i64 = self
            .file_meta
            .size
            .try_into()
            .map_err(|_| Error::generic("Failed to convert parquet metadata 'size' to i64"))?;
        let size = Arc::new(Int64Array::from(vec![size]));
        let modification_time = Arc::new(Int64Array::from(vec![self.file_meta.last_modified]));

        let stats_array = Arc::new(self.stats.clone());

        // Build schema dynamically based on stats (stats schema varies based on collected
        // statistics)
        let key_value_struct = DataType::Struct(
            vec![
                Field::new("key", DataType::Utf8, false),
                Field::new("value", DataType::Utf8, true),
            ]
            .into(),
        );
        let schema = Schema::new(vec![
            Field::new("path", DataType::Utf8, false),
            Field::new(
                "partitionValues",
                DataType::Map(
                    Arc::new(Field::new("key_value", key_value_struct, false)),
                    false,
                ),
                false,
            ),
            Field::new("size", DataType::Int64, false),
            Field::new("modificationTime", DataType::Int64, false),
            Field::new("stats", stats_array.data_type().clone(), true),
        ]);

        Ok(Box::new(ArrowEngineData::new(RecordBatch::try_new(
            Arc::new(schema),
            vec![path, partitions, size, modification_time, stats_array],
        )?)))
    }
}

impl<E: TaskExecutor> DefaultParquetHandler<E> {
    pub fn new(store: Arc<DynObjectStore>, task_executor: Arc<E>) -> Self {
        Self {
            store,
            task_executor,
            readahead: 10,
            reporter: None,
        }
    }

    /// Max number of batches to read ahead while executing [Self::read_parquet_files()].
    ///
    /// Defaults to 10.
    pub fn with_readahead(mut self, readahead: usize) -> Self {
        self.readahead = readahead;
        self
    }

    /// Set a metrics reporter to receive [`MetricEvent::ParquetReadCompleted`] events.
    pub fn with_reporter(mut self, reporter: Option<Arc<dyn MetricsReporter>>) -> Self {
        self.reporter = reporter;
        self
    }

    // Write `data` to `{path}/<uuid>.parquet` as parquet using ArrowWriter and return the parquet
    // metadata (where `<uuid>` is a generated UUIDv4).
    //
    // Note: after encoding the data as parquet, this issues a PUT followed by a HEAD to storage in
    // order to obtain metadata about the object just written.
    async fn write_parquet(
        &self,
        path: &url::Url,
        data: Box<dyn EngineData>,
        stats_columns: &[ColumnName],
    ) -> DeltaResult<DataFileMetadata> {
        let batch: Box<_> = ArrowEngineData::try_from_engine_data(data)?;
        let record_batch = batch.record_batch();

        // Collect statistics before writing (includes numRecords)
        let stats = collect_stats(record_batch, stats_columns)?;

        let mut buffer = vec![];
        let mut writer = ArrowWriter::try_new_with_options(
            &mut buffer,
            record_batch.schema(),
            writer_options(),
        )?;
        writer.write(record_batch)?;
        writer.close()?; // writer must be closed to write footer

        let size: u64 = buffer
            .len()
            .try_into()
            .map_err(|_| Error::generic("unable to convert usize to u64"))?;
        let name: String = format!("{}.parquet", Uuid::new_v4());
        // fail if path does not end with a trailing slash
        if !path.path().ends_with('/') {
            return Err(Error::generic(format!(
                "Path must end with a trailing slash: {path}"
            )));
        }
        let path = path.join(&name)?;

        self.store
            .put(&Path::from_url_path(path.path())?, buffer.into())
            .await?;

        let metadata = self.store.head(&Path::from_url_path(path.path())?).await?;
        let modification_time = metadata.last_modified.timestamp_millis();
        if size != metadata.size {
            return Err(Error::generic(format!(
                "Size mismatch after writing parquet file: expected {}, got {}",
                size, metadata.size
            )));
        }

        let file_meta = FileMeta::new(path, modification_time, size);
        Ok(DataFileMetadata::new(file_meta, stats))
    }

    /// Write `data` to a new parquet file under the [`WriteContext::write_dir`] and return
    /// Add action metadata ready for [`Transaction::add_files`].
    ///
    /// Note that the schema does not contain the dataChange column. In order to set `data_change`
    /// flag, use [`crate::transaction::Transaction::with_data_change`].
    ///
    /// [`WriteContext::write_dir`]: crate::transaction::WriteContext::write_dir
    /// [`Transaction::add_files`]: crate::transaction::Transaction::add_files
    pub async fn write_parquet_file(
        &self,
        data: Box<dyn EngineData>,
        write_context: &WriteContext,
    ) -> DeltaResult<Box<dyn EngineData>> {
        let file_metadata = self
            .write_parquet(
                &write_context.write_dir(),
                data,
                write_context.stats_columns(),
            )
            .await?;
        super::build_add_file_metadata(file_metadata, write_context)
    }
}

/// Internal async implementation of read_parquet_files
async fn read_parquet_files_impl(
    store: Arc<DynObjectStore>,
    files: Vec<FileMeta>,
    physical_schema: SchemaRef,
    predicate: Option<PredicateRef>,
) -> DeltaResult<BoxStream<'static, DeltaResult<Box<dyn EngineData>>>> {
    if files.is_empty() {
        return Ok(Box::pin(stream::empty()));
    }

    let arrow_schema = Arc::new(physical_schema.as_ref().try_into_arrow()?);

    // get the first FileMeta to decide how to fetch the file.
    // NB: This means that every file in `FileMeta` _must_ have the same scheme or things will break
    // s3://    -> aws   (ParquetOpener)
    // nothing  -> local (ParquetOpener)
    // https:// -> assume presigned URL (and fetch without object_store)
    //   -> reqwest to get data
    //   -> parse to parquet
    // SAFETY: we did is_empty check above, this is ok.
    if files[0].location.is_presigned() {
        let file_opener = Box::new(PresignedUrlOpener::new(
            1024,
            physical_schema.clone(),
            predicate,
        ));
        let stream = FileStream::new(files, arrow_schema, file_opener)?.map_ok(
            |record_batch| -> Box<dyn EngineData> { Box::new(ArrowEngineData::new(record_batch)) },
        );
        return Ok(Box::pin(stream));
    }

    // an iterator of futures that open each file
    let file_futures = files.into_iter().map(move |file| {
        let store = store.clone();
        let schema = physical_schema.clone();
        let predicate = predicate.clone();
        async move {
            open_parquet_file(
                store,
                schema,
                predicate,
                None,
                super::DEFAULT_BATCH_SIZE,
                file,
            )
            .await
        }
    });
    // create a stream from that iterator which buffers up to `buffer_size` futures at a time
    let result_stream = stream::iter(file_futures)
        .buffered(super::DEFAULT_BUFFER_SIZE)
        .try_flatten()
        .map_ok(|record_batch| -> Box<dyn EngineData> {
            Box::new(ArrowEngineData::new(record_batch))
        });

    Ok(Box::pin(result_stream))
}

impl<E: TaskExecutor> ParquetHandler for DefaultParquetHandler<E> {
    fn read_parquet_files(
        &self,
        files: &[FileMeta],
        physical_schema: SchemaRef,
        predicate: Option<PredicateRef>,
    ) -> DeltaResult<FileDataReadResultIterator> {
        let future = read_parquet_files_impl(
            self.store.clone(),
            files.to_vec(),
            physical_schema,
            predicate,
        );
        let inner = super::stream_future_to_iter(self.task_executor.clone(), future)?;
        if let Some(reporter) = &self.reporter {
            let num_files = files.len() as u64;
            let bytes_read = files.iter().map(|f| f.size).sum();
            Ok(Box::new(super::ReadMetricsIterator::new(
                inner,
                reporter.clone(),
                num_files,
                bytes_read,
                |num_files, bytes_read| MetricEvent::ParquetReadCompleted {
                    num_files,
                    bytes_read,
                },
            )))
        } else {
            Ok(inner)
        }
    }

    /// Writes engine data to a Parquet file at the specified location.
    ///
    /// This implementation uses asynchronous file I/O with object_store to write the Parquet file.
    /// If a file already exists at the given location, it will be overwritten.
    ///
    /// # Parameters
    ///
    /// - `location` - The full URL path where the Parquet file should be written (e.g., `s3://bucket/path/file.parquet`,
    ///   `file:///path/to/file.parquet`).
    /// - `data` - An iterator of engine data to be written to the Parquet file.
    ///
    /// # Returns
    ///
    /// A [`DeltaResult`] indicating success or failure.
    fn write_parquet_file(
        &self,
        location: url::Url,
        mut data: Box<dyn Iterator<Item = DeltaResult<Box<dyn EngineData>>> + Send>,
    ) -> DeltaResult<()> {
        use tokio::sync::Mutex;

        let store = self.store.clone();
        let path = Path::from_url_path(location.path())?;

        // Get first batch to initialize writer with schema
        let first_batch = data.next().ok_or_else(|| {
            Error::generic("Cannot write parquet file with empty data iterator")
        })??;
        let first_arrow = ArrowEngineData::try_from_engine_data(first_batch)?;
        let first_record_batch: RecordBatch = (*first_arrow).into();

        let object_writer = ParquetObjectWriter::new(store, path);
        let schema = first_record_batch.schema();
        let writer = Arc::new(Mutex::new(AsyncArrowWriter::try_new_with_options(
            object_writer,
            schema,
            writer_options(),
        )?));

        let w = writer.clone();
        self.task_executor.block_on(async move {
            // Write the first batch
            let mut writer = w.lock().await;
            writer.write(&first_record_batch).await
        })?;

        // In order to avoid a deadlock when running inside a single-threaded runtime, this code
        // iterator over the blocking stream of `data` separately from the `write` awaits` .
        // See: https://github.com/delta-io/delta-kernel-rs/issues/2399>
        for result in data {
            let engine_data = result?;
            let arrow_data = ArrowEngineData::try_from_engine_data(engine_data)?;
            let batch: RecordBatch = (*arrow_data).into();
            let w = writer.clone();
            self.task_executor.block_on(async move {
                let mut writer = w.lock().await;
                writer.write(&batch).await
            })?;
        }

        let w = writer.clone();
        self.task_executor.block_on(async move {
            let mut writer = w.lock().await;
            writer.finish().await
        })?;

        Ok(())
    }

    fn read_parquet_footer(&self, file: &FileMeta) -> DeltaResult<ParquetFooter> {
        let store = self.store.clone();
        let location = file.location.clone();
        let file_size = file.size;

        self.task_executor.block_on(async move {
            let metadata = if location.is_presigned() {
                let client = reqwest::Client::new();
                let response =
                    client.get(location.as_str()).send().await.map_err(|e| {
                        Error::generic(format!("Failed to fetch presigned URL: {e}"))
                    })?;
                let bytes = response
                    .bytes()
                    .await
                    .map_err(|e| Error::generic(format!("Failed to read response bytes: {e}")))?;
                ArrowReaderMetadata::load(&bytes, reader_options())?
            } else {
                let path = Path::from_url_path(location.path())?;
                let mut reader = ParquetObjectReader::new(store, path).with_file_size(file_size);
                ArrowReaderMetadata::load_async(&mut reader, reader_options()).await?
            };

            let schema = StructType::try_from_arrow(metadata.schema().as_ref())
                .map(Arc::new)
                .map_err(Error::Arrow)?;
            Ok(ParquetFooter { schema })
        })
    }
}

/// Opens a Parquet file and returns a stream of record batches
async fn open_parquet_file(
    store: Arc<DynObjectStore>,
    table_schema: SchemaRef,
    predicate: Option<PredicateRef>,
    limit: Option<usize>,
    batch_size: usize,
    file_meta: FileMeta,
) -> DeltaResult<BoxStream<'static, DeltaResult<RecordBatch>>> {
    let file_location = file_meta.location.to_string();
    let path = Path::from_url_path(file_meta.location.path())?;

    let mut reader = {
        use crate::object_store::ObjectStoreScheme;
        // HACK: unfortunately, `ParquetObjectReader` under the hood does a suffix range
        // request which isn't supported by Azure. For now we just detect if the URL is
        // pointing to azure and if so, do a HEAD request so we can pass in file size to the
        // reader which will cause the reader to avoid a suffix range request.
        // see also: https://github.com/delta-io/delta-kernel-rs/issues/968

        // Since the `Remove` action's size value is optional as specified in the delta protocol
        // https://github.com/delta-io/delta/blob/master/PROTOCOL.md#add-file-and-remove-file,
        // the extracted size will be zero in this case. Thus, this function
        // need to handle the case of zero file_meta.size.
        if file_meta.size != 0 {
            ParquetObjectReader::new(store, path).with_file_size(file_meta.size)
        } else if let Ok((ObjectStoreScheme::MicrosoftAzure, _)) =
            ObjectStoreScheme::parse(&file_meta.location)
        {
            // also note doing HEAD then actual GET isn't atomic, and leaves us vulnerable
            // to file changing between the two calls.
            let meta = store.head(&path).await?;
            ParquetObjectReader::new(store, path).with_file_size(meta.size)
        } else {
            ParquetObjectReader::new(store, path)
        }
    };

    let reader_options = reader_options();
    let metadata = ArrowReaderMetadata::load_async(&mut reader, reader_options.clone()).await?;
    let parquet_schema = metadata.schema();
    let (indices, requested_ordering) = get_requested_indices(&table_schema, parquet_schema)?;
    let mut builder =
        ParquetRecordBatchStreamBuilder::new_with_options(reader, reader_options).await?;
    if let Some(mask) = generate_mask(
        &table_schema,
        parquet_schema,
        builder.parquet_schema(),
        &indices,
    ) {
        builder = builder.with_projection(mask)
    }

    // Only create RowIndexBuilder if row indexes are actually needed
    let mut row_indexes = ordering_needs_row_indexes(&requested_ordering)
        .then(|| RowIndexBuilder::new(builder.metadata().row_groups()));

    // Filter row groups and row indexes if a predicate is provided
    if let Some(ref predicate) = predicate {
        builder = builder.with_row_group_filter(predicate, row_indexes.as_mut());
    }
    if let Some(limit) = limit {
        builder = builder.with_limit(limit)
    }

    let mut row_indexes = row_indexes.map(|rb| rb.build()).transpose()?;
    let stream = builder.with_batch_size(batch_size).build()?;

    let arrow_schema: Arc<Schema> = Arc::new(table_schema.as_ref().try_into_arrow()?);
    let stream = stream.map(move |rbr| {
        fixup_parquet_read(
            rbr?,
            &requested_ordering,
            row_indexes.as_mut(),
            Some(&file_location),
            Some(&arrow_schema),
        )
        .map(Into::into)
    });
    Ok(stream.boxed())
}

/// Implements [`FileOpener`] for a opening a parquet file from a presigned URL
struct PresignedUrlOpener {
    batch_size: usize,
    predicate: Option<PredicateRef>,
    limit: Option<usize>,
    table_schema: SchemaRef,
    client: reqwest::Client,
}

impl PresignedUrlOpener {
    pub(crate) fn new(
        batch_size: usize,
        schema: SchemaRef,
        predicate: Option<PredicateRef>,
    ) -> Self {
        Self {
            batch_size,
            table_schema: schema,
            predicate,
            limit: None,
            client: reqwest::Client::new(),
        }
    }
}

impl FileOpener for PresignedUrlOpener {
    fn open(&self, file_meta: FileMeta, _range: Option<Range<i64>>) -> DeltaResult<FileOpenFuture> {
        let batch_size = self.batch_size;
        let table_schema = self.table_schema.clone();
        let predicate = self.predicate.clone();
        let limit = self.limit;
        let client = self.client.clone(); // uses Arc internally according to reqwest docs
        let file_location = file_meta.location.to_string();

        Ok(Box::pin(async move {
            // fetch the file from the interweb
            let reader = client.get(&file_location).send().await?.bytes().await?;
            let reader_options = reader_options();
            let metadata = ArrowReaderMetadata::load(&reader, reader_options.clone())?;
            let parquet_schema = metadata.schema();
            let (indices, requested_ordering) =
                get_requested_indices(&table_schema, parquet_schema)?;

            let mut builder =
                ParquetRecordBatchReaderBuilder::try_new_with_options(reader, reader_options)?;
            if let Some(mask) = generate_mask(
                &table_schema,
                parquet_schema,
                builder.parquet_schema(),
                &indices,
            ) {
                builder = builder.with_projection(mask)
            }

            // Only create RowIndexBuilder if row indexes are actually needed
            let mut row_indexes = ordering_needs_row_indexes(&requested_ordering)
                .then(|| RowIndexBuilder::new(builder.metadata().row_groups()));

            // Filter row groups and row indexes if a predicate is provided
            if let Some(ref predicate) = predicate {
                builder = builder.with_row_group_filter(predicate, row_indexes.as_mut());
            }
            if let Some(limit) = limit {
                builder = builder.with_limit(limit)
            }

            let reader = builder.with_batch_size(batch_size).build()?;

            let mut row_indexes = row_indexes.map(|rb| rb.build()).transpose()?;
            let arrow_schema: Arc<Schema> = Arc::new(table_schema.as_ref().try_into_arrow()?);
            let stream = futures::stream::iter(reader);
            let stream = stream.map(move |rbr| {
                fixup_parquet_read(
                    rbr?,
                    &requested_ordering,
                    row_indexes.as_mut(),
                    Some(&file_location),
                    Some(&arrow_schema),
                )
                .map(Into::into)
            });
            Ok(stream.boxed())
        }))
    }
}

#[cfg(test)]
mod tests {
    use std::collections::HashMap;
    use std::path::PathBuf;
    use std::slice;

    use itertools::Itertools;
    use url::Url;

    use super::*;
    use crate::arrow::array::{
        Array, BinaryArray, BooleanArray, Date32Array, Decimal128Array, Float32Array, Float64Array,
        Int16Array, Int32Array, Int64Array, Int8Array, RecordBatch, StringArray,
        TimestampMicrosecondArray,
    };
    use crate::arrow::datatypes::{DataType as ArrowDataType, Field, Schema as ArrowSchema};
    use crate::engine::arrow_conversion::TryIntoKernel as _;
    use crate::engine::arrow_data::ArrowEngineData;
    use crate::engine::default::executor::tokio::TokioBackgroundExecutor;
    use crate::engine::default::DEFAULT_BATCH_SIZE;
    use crate::object_store::local::LocalFileSystem;
    use crate::object_store::memory::InMemory;
    use crate::parquet::arrow::{ARROW_SCHEMA_META_KEY, PARQUET_FIELD_ID_META_KEY};
    use crate::schema::ColumnMetadataKey;
    use crate::utils::current_time_ms;
    use crate::utils::test_utils::assert_result_error_with_message;
    use crate::EngineData;

    fn into_record_batch(
        engine_data: DeltaResult<Box<dyn EngineData>>,
    ) -> DeltaResult<RecordBatch> {
        engine_data
            .and_then(ArrowEngineData::try_from_engine_data)
            .map(Into::into)
    }

    async fn read_all_rows_helper(file_meta: FileMeta) -> DeltaResult<Vec<RecordBatch>> {
        let store = Arc::new(LocalFileSystem::new());
        let path = Path::from_url_path(file_meta.location.path()).unwrap();
        let reader = ParquetObjectReader::new(store.clone(), path);
        let physical_schema = ParquetRecordBatchStreamBuilder::new(reader)
            .await
            .unwrap()
            .schema()
            .clone();
        let stream = open_parquet_file(
            store,
            Arc::new(physical_schema.try_into_kernel().unwrap()),
            None,
            None,
            DEFAULT_BATCH_SIZE,
            file_meta,
        )
        .await
        .unwrap();

        let batches: Vec<RecordBatch> = stream.try_collect().await.unwrap();
        Ok(batches)
    }

    #[tokio::test]
    async fn test_open_parquet_file_with_size() {
        let path = std::fs::canonicalize(PathBuf::from(
            "./tests/data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet"
        )).unwrap();
        let file_size = std::fs::metadata(&path).unwrap().len();
        let url = Url::from_file_path(path).unwrap();
        let file_meta = FileMeta {
            location: url,
            last_modified: 0,
            size: file_size,
        };
        let data = read_all_rows_helper(file_meta).await.unwrap();

        assert_eq!(data.len(), 1);
        assert_eq!(data[0].num_rows(), 10);
    }

    #[tokio::test]
    async fn test_open_parquet_file_without_size() {
        let path = std::fs::canonicalize(PathBuf::from(
            "./tests/data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet"
        )).unwrap();
        let url = Url::from_file_path(path).unwrap();
        let file_meta = FileMeta {
            location: url,
            last_modified: 0,
            size: 0,
        };
        let data = read_all_rows_helper(file_meta).await.unwrap();

        assert_eq!(data.len(), 1);
        assert_eq!(data[0].num_rows(), 10);
    }

    #[tokio::test]
    async fn test_read_parquet_files() {
        let store = Arc::new(LocalFileSystem::new());

        let path = std::fs::canonicalize(PathBuf::from(
            "./tests/data/table-with-dv-small/part-00000-fae5310a-a37d-4e51-827b-c3d5516560ca-c000.snappy.parquet"
        )).unwrap();
        let url = url::Url::from_file_path(path).unwrap();
        let location = Path::from_url_path(url.path()).unwrap();
        let meta = store.head(&location).await.unwrap();

        let reader = ParquetObjectReader::new(store.clone(), location);
        let physical_schema = ParquetRecordBatchStreamBuilder::new(reader)
            .await
            .unwrap()
            .schema()
            .clone();

        let files = &[FileMeta {
            location: url.clone(),
            last_modified: meta.last_modified.timestamp(),
            size: meta.size,
        }];

        let handler = DefaultParquetHandler::new(store, Arc::new(TokioBackgroundExecutor::new()));
        let data: Vec<RecordBatch> = handler
            .read_parquet_files(
                files,
                Arc::new(physical_schema.try_into_kernel().unwrap()),
                None,
            )
            .unwrap()
            .map(into_record_batch)
            .try_collect()
            .unwrap();

        assert_eq!(data.len(), 1);
        assert_eq!(data[0].num_rows(), 10);
    }

    #[rstest::rstest]
    fn test_as_record_batch(
        #[values(None, Some("a".to_string()))] partition_value: Option<String>,
    ) {
        let location = Url::parse("file:///test_url").unwrap();
        let size = 1_000_000;
        let last_modified = 10000000000;
        let num_records = 10;
        let file_metadata = FileMeta::new(location.clone(), last_modified, size);
        let stats = StructArray::try_new(
            vec![
                Field::new("numRecords", ArrowDataType::Int64, true),
                Field::new("tightBounds", ArrowDataType::Boolean, true),
            ]
            .into(),
            vec![
                Arc::new(Int64Array::from(vec![num_records as i64])),
                Arc::new(BooleanArray::from(vec![true])),
            ],
            None,
        )
        .unwrap();
        let data_file_metadata = DataFileMetadata::new(file_metadata, stats.clone());
        let partition_values = HashMap::from([("partition1".to_string(), partition_value.clone())]);
        let actual = data_file_metadata
            .as_record_batch(&partition_values, "test_url")
            .unwrap();
        let actual = ArrowEngineData::try_from_engine_data(actual).unwrap();

        let mut partition_values_builder = MapBuilder::new(
            Some(MapFieldNames {
                entry: "key_value".to_string(),
                key: "key".to_string(),
                value: "value".to_string(),
            }),
            StringBuilder::new(),
            StringBuilder::new(),
        );

        partition_values_builder.keys().append_value("partition1");
        match &partition_value {
            None => partition_values_builder.values().append_null(),
            Some(v) => partition_values_builder.values().append_value(v),
        }
        partition_values_builder.append(true).unwrap();
        let partition_values = partition_values_builder.finish();

        // Build expected schema dynamically based on stats
        let stats_field = Field::new("stats", stats.data_type().clone(), true);
        let schema = Arc::new(crate::arrow::datatypes::Schema::new(vec![
            Field::new("path", ArrowDataType::Utf8, false),
            Field::new(
                "partitionValues",
                ArrowDataType::Map(
                    Arc::new(Field::new(
                        "key_value",
                        ArrowDataType::Struct(
                            vec![
                                Field::new("key", ArrowDataType::Utf8, false),
                                Field::new("value", ArrowDataType::Utf8, true),
                            ]
                            .into(),
                        ),
                        false,
                    )),
                    false,
                ),
                false,
            ),
            Field::new("size", ArrowDataType::Int64, false),
            Field::new("modificationTime", ArrowDataType::Int64, false),
            stats_field,
        ]));

        let expected = RecordBatch::try_new(
            schema,
            vec![
                Arc::new(StringArray::from(vec!["test_url"])),
                Arc::new(partition_values),
                Arc::new(Int64Array::from(vec![size as i64])),
                Arc::new(Int64Array::from(vec![last_modified])),
                Arc::new(stats),
            ],
        )
        .unwrap();

        assert_eq!(actual.record_batch(), &expected);
    }

    #[tokio::test]
    async fn test_write_parquet() {
        let store = Arc::new(InMemory::new());
        let parquet_handler =
            DefaultParquetHandler::new(store.clone(), Arc::new(TokioBackgroundExecutor::new()));

        let data = Box::new(ArrowEngineData::new(
            RecordBatch::try_from_iter(vec![(
                "a",
                Arc::new(Int64Array::from(vec![1, 2, 3])) as Arc<dyn Array>,
            )])
            .unwrap(),
        ));

        let write_metadata = parquet_handler
            .write_parquet(&Url::parse("memory:///data/").unwrap(), data, &[])
            .await
            .unwrap();

        let DataFileMetadata {
            file_meta:
                ref parquet_file @ FileMeta {
                    ref location,
                    last_modified,
                    size,
                },
            ref stats,
        } = write_metadata;
        let expected_location = Url::parse("memory:///data/").unwrap();

        // head the object to get metadata
        let meta = store
            .head(&Path::from_url_path(location.path()).unwrap())
            .await
            .unwrap();
        let expected_size = meta.size;

        // check that last_modified is within 10s of now
        let now: i64 = current_time_ms().unwrap();

        let filename = location.path().split('/').next_back().unwrap();
        assert_eq!(&expected_location.join(filename).unwrap(), location);
        assert_eq!(expected_size, size);
        assert!(now - last_modified < 10_000);

        // Check numRecords from stats
        let num_records = stats
            .column_by_name("numRecords")
            .unwrap()
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap()
            .value(0);
        assert_eq!(num_records, 3);

        // check we can read back
        let path = Path::from_url_path(location.path()).unwrap();
        let reader = ParquetObjectReader::new(store.clone(), path);
        let physical_schema = ParquetRecordBatchStreamBuilder::new(reader)
            .await
            .unwrap()
            .schema()
            .clone();

        let data: Vec<RecordBatch> = parquet_handler
            .read_parquet_files(
                slice::from_ref(parquet_file),
                Arc::new(physical_schema.try_into_kernel().unwrap()),
                None,
            )
            .unwrap()
            .map(into_record_batch)
            .try_collect()
            .unwrap();

        assert_eq!(data.len(), 1);
        assert_eq!(data[0].num_rows(), 3);
    }

    #[tokio::test]
    async fn test_disallow_non_trailing_slash() {
        let store = Arc::new(InMemory::new());
        let parquet_handler =
            DefaultParquetHandler::new(store.clone(), Arc::new(TokioBackgroundExecutor::new()));

        let data = Box::new(ArrowEngineData::new(
            RecordBatch::try_from_iter(vec![(
                "a",
                Arc::new(Int64Array::from(vec![1, 2, 3])) as Arc<dyn Array>,
            )])
            .unwrap(),
        ));

        assert_result_error_with_message(
            parquet_handler
                .write_parquet(&Url::parse("memory:///data").unwrap(), data, &[])
                .await,
            "Generic delta kernel error: Path must end with a trailing slash: memory:///data",
        );
    }

    #[tokio::test]
    async fn test_parquet_handler_trait_write() {
        let store = Arc::new(InMemory::new());
        let parquet_handler: Arc<dyn ParquetHandler> = Arc::new(DefaultParquetHandler::new(
            store.clone(),
            Arc::new(TokioBackgroundExecutor::new()),
        ));

        let engine_data: Box<dyn EngineData> = Box::new(ArrowEngineData::new(
            RecordBatch::try_from_iter(vec![
                (
                    "x",
                    Arc::new(Int64Array::from(vec![10, 20, 30])) as Arc<dyn Array>,
                ),
                (
                    "y",
                    Arc::new(Int64Array::from(vec![100, 200, 300])) as Arc<dyn Array>,
                ),
            ])
            .unwrap(),
        ));

        // Create iterator with single batch
        let data_iter: Box<dyn Iterator<Item = DeltaResult<Box<dyn EngineData>>> + Send> =
            Box::new(std::iter::once(Ok(engine_data)));

        // Test writing through the trait method
        let file_url = Url::parse("memory:///test/data.parquet").unwrap();
        parquet_handler
            .write_parquet_file(file_url.clone(), data_iter)
            .unwrap();

        // Verify we can read the file back
        let path = Path::from_url_path(file_url.path()).unwrap();
        let metadata = store.head(&path).await.unwrap();
        let reader = ParquetObjectReader::new(store.clone(), path);
        let physical_schema = ParquetRecordBatchStreamBuilder::new(reader)
            .await
            .unwrap()
            .schema()
            .clone();

        let file_meta = FileMeta {
            location: file_url,
            last_modified: 0,
            size: metadata.size,
        };

        let data: Vec<RecordBatch> = parquet_handler
            .read_parquet_files(
                slice::from_ref(&file_meta),
                Arc::new(physical_schema.try_into_kernel().unwrap()),
                None,
            )
            .unwrap()
            .map(into_record_batch)
            .try_collect()
            .unwrap();

        assert_eq!(data.len(), 1);
        assert_eq!(data[0].num_rows(), 3);
        assert_eq!(data[0].num_columns(), 2);
    }

    #[tokio::test]
    async fn test_parquet_handler_trait_write_and_read_roundtrip() {
        let store = Arc::new(InMemory::new());
        let parquet_handler: Arc<dyn ParquetHandler> = Arc::new(DefaultParquetHandler::new(
            store.clone(),
            Arc::new(TokioBackgroundExecutor::new()),
        ));

        // Create test data with all Delta-supported primitive types
        let engine_data: Box<dyn EngineData> = Box::new(ArrowEngineData::new(
            RecordBatch::try_from_iter(vec![
                // Byte (i8)
                (
                    "byte_col",
                    Arc::new(Int8Array::from(vec![1i8, 2, 3, 4, 5])) as Arc<dyn Array>,
                ),
                // Short (i16)
                (
                    "short_col",
                    Arc::new(Int16Array::from(vec![100i16, 200, 300, 400, 500])) as Arc<dyn Array>,
                ),
                // Integer (i32)
                (
                    "int_col",
                    Arc::new(Int32Array::from(vec![1000i32, 2000, 3000, 4000, 5000]))
                        as Arc<dyn Array>,
                ),
                // Long (i64)
                (
                    "long_col",
                    Arc::new(Int64Array::from(vec![10000i64, 20000, 30000, 40000, 50000]))
                        as Arc<dyn Array>,
                ),
                // Float (f32)
                (
                    "float_col",
                    Arc::new(Float32Array::from(vec![1.1f32, 2.2, 3.3, 4.4, 5.5]))
                        as Arc<dyn Array>,
                ),
                // Double (f64)
                (
                    "double_col",
                    Arc::new(Float64Array::from(vec![1.11f64, 2.22, 3.33, 4.44, 5.55]))
                        as Arc<dyn Array>,
                ),
                // Boolean
                (
                    "bool_col",
                    Arc::new(BooleanArray::from(vec![true, false, true, false, true]))
                        as Arc<dyn Array>,
                ),
                // String
                (
                    "string_col",
                    Arc::new(StringArray::from(vec!["a", "b", "c", "d", "e"])) as Arc<dyn Array>,
                ),
                // Binary
                (
                    "binary_col",
                    Arc::new(BinaryArray::from_vec(vec![
                        b"bin1", b"bin2", b"bin3", b"bin4", b"bin5",
                    ])) as Arc<dyn Array>,
                ),
                // Date
                (
                    "date_col",
                    Arc::new(Date32Array::from(vec![18262, 18263, 18264, 18265, 18266]))
                        as Arc<dyn Array>, // Days since epoch (2020-01-01 onwards)
                ),
                // Timestamp (with UTC timezone)
                (
                    "timestamp_col",
                    Arc::new(
                        TimestampMicrosecondArray::from(vec![
                            1609459200000000i64, // 2021-01-01 00:00:00 UTC
                            1609545600000000i64,
                            1609632000000000i64,
                            1609718400000000i64,
                            1609804800000000i64,
                        ])
                        .with_timezone("UTC"),
                    ) as Arc<dyn Array>,
                ),
                // TimestampNtz (without timezone)
                (
                    "timestamp_ntz_col",
                    Arc::new(TimestampMicrosecondArray::from(vec![
                        1609459200000000i64, // 2021-01-01 00:00:00
                        1609545600000000i64,
                        1609632000000000i64,
                        1609718400000000i64,
                        1609804800000000i64,
                    ])) as Arc<dyn Array>,
                ),
                // Decimal (precision 10, scale 2)
                (
                    "decimal_col",
                    Arc::new(
                        Decimal128Array::from(vec![12345i128, 23456, 34567, 45678, 56789])
                            .with_precision_and_scale(10, 2)
                            .unwrap(),
                    ) as Arc<dyn Array>,
                ),
            ])
            .unwrap(),
        ));

        // Create iterator with single batch
        let data_iter: Box<dyn Iterator<Item = DeltaResult<Box<dyn EngineData>>> + Send> =
            Box::new(std::iter::once(Ok(engine_data)));

        // Write the data
        let file_url = Url::parse("memory:///roundtrip/test.parquet").unwrap();
        parquet_handler
            .write_parquet_file(file_url.clone(), data_iter)
            .unwrap();

        // Read it back
        let path = Path::from_url_path(file_url.path()).unwrap();
        let metadata = store.head(&path).await.unwrap();
        let reader = ParquetObjectReader::new(store.clone(), path);
        let physical_schema = ParquetRecordBatchStreamBuilder::new(reader)
            .await
            .unwrap()
            .schema()
            .clone();

        let file_meta = FileMeta {
            location: file_url.clone(),
            last_modified: 0,
            size: metadata.size,
        };

        let data: Vec<RecordBatch> = parquet_handler
            .read_parquet_files(
                slice::from_ref(&file_meta),
                Arc::new(physical_schema.try_into_kernel().unwrap()),
                None,
            )
            .unwrap()
            .map(into_record_batch)
            .try_collect()
            .unwrap();

        // Verify the data
        assert_eq!(data.len(), 1);
        assert_eq!(data[0].num_rows(), 5);
        assert_eq!(data[0].num_columns(), 13);

        let mut col_idx = 0;

        // Verify byte column
        let byte_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Int8Array>()
            .unwrap();
        assert_eq!(byte_col.values(), &[1i8, 2, 3, 4, 5]);
        col_idx += 1;

        // Verify short column
        let short_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Int16Array>()
            .unwrap();
        assert_eq!(short_col.values(), &[100i16, 200, 300, 400, 500]);
        col_idx += 1;

        // Verify int column
        let int_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Int32Array>()
            .unwrap();
        assert_eq!(int_col.values(), &[1000i32, 2000, 3000, 4000, 5000]);
        col_idx += 1;

        // Verify long column
        let long_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();
        assert_eq!(long_col.values(), &[10000i64, 20000, 30000, 40000, 50000]);
        col_idx += 1;

        // Verify float column
        let float_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Float32Array>()
            .unwrap();
        assert_eq!(float_col.values(), &[1.1f32, 2.2, 3.3, 4.4, 5.5]);
        col_idx += 1;

        // Verify double column
        let double_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Float64Array>()
            .unwrap();
        assert_eq!(double_col.values(), &[1.11f64, 2.22, 3.33, 4.44, 5.55]);
        col_idx += 1;

        // Verify bool column
        let bool_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<BooleanArray>()
            .unwrap();
        assert!(bool_col.value(0));
        assert!(!bool_col.value(1));
        col_idx += 1;

        // Verify string column
        let string_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<StringArray>()
            .unwrap();
        assert_eq!(string_col.value(0), "a");
        assert_eq!(string_col.value(4), "e");
        col_idx += 1;

        // Verify binary column
        let binary_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<BinaryArray>()
            .unwrap();
        assert_eq!(binary_col.value(0), b"bin1");
        assert_eq!(binary_col.value(4), b"bin5");
        col_idx += 1;

        // Verify date column
        let date_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Date32Array>()
            .unwrap();
        assert_eq!(date_col.values(), &[18262, 18263, 18264, 18265, 18266]);
        col_idx += 1;

        // Verify timestamp column (with UTC timezone)
        let timestamp_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<TimestampMicrosecondArray>()
            .unwrap();
        assert_eq!(timestamp_col.value(0), 1609459200000000i64);
        assert_eq!(timestamp_col.value(4), 1609804800000000i64);
        assert!(timestamp_col
            .timezone()
            .is_some_and(|tz| tz.eq_ignore_ascii_case("utc")));
        col_idx += 1;

        // Verify timestamp_ntz column (without timezone)
        let timestamp_ntz_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<TimestampMicrosecondArray>()
            .unwrap();
        assert_eq!(timestamp_ntz_col.value(0), 1609459200000000i64);
        assert_eq!(timestamp_ntz_col.value(4), 1609804800000000i64);
        assert!(timestamp_ntz_col.timezone().is_none());
        col_idx += 1;

        // Verify decimal column
        let decimal_col = data[0]
            .column(col_idx)
            .as_any()
            .downcast_ref::<Decimal128Array>()
            .unwrap();
        assert_eq!(decimal_col.value(0), 12345i128);
        assert_eq!(decimal_col.value(4), 56789i128);
        assert_eq!(decimal_col.precision(), 10);
        assert_eq!(decimal_col.scale(), 2);
    }

    /// Test that field IDs are accessible via ColumnMetadataKey::ParquetFieldId as documented.
    ///
    /// Per trait definitions in lib.rs, field IDs should be accessible via
    /// StructField::get_config_value with ColumnMetadataKey::ParquetFieldId.
    #[test]
    fn test_parquet_footer_read_with_field_id() {
        // Write parquet file with field ID
        let field = Field::new("value", ArrowDataType::Int64, false).with_metadata(HashMap::from(
            [(PARQUET_FIELD_ID_META_KEY.to_string(), "42".to_string())],
        ));
        let arrow_schema = Arc::new(ArrowSchema::new(vec![field]));

        let temp_dir = tempfile::tempdir().unwrap();
        let file_path = temp_dir.path().join("field_id_test.parquet");
        let batch = RecordBatch::try_new(
            arrow_schema.clone(),
            vec![Arc::new(Int64Array::from(vec![1, 2, 3]))],
        )
        .unwrap();

        let file = std::fs::File::create(&file_path).unwrap();
        let mut writer = ArrowWriter::try_new(file, arrow_schema, None).unwrap();
        writer.write(&batch).unwrap();
        writer.close().unwrap();

        // Read footer and verify field ID accessibility
        let store = Arc::new(LocalFileSystem::new());
        let handler = DefaultParquetHandler::new(store, Arc::new(TokioBackgroundExecutor::new()));
        let file_size = std::fs::metadata(&file_path).unwrap().len();
        let file_meta = FileMeta {
            location: Url::from_file_path(&file_path).unwrap(),
            last_modified: 0,
            size: file_size,
        };

        let footer = handler.read_parquet_footer(&file_meta).unwrap();
        let field = footer
            .schema
            .fields()
            .find(|f| f.name() == "value")
            .unwrap();

        // Field ID is transformed to kernel key when reading
        assert_eq!(
            field
                .metadata()
                .get(ColumnMetadataKey::ParquetFieldId.as_ref()),
            Some(&"42".into())
        );

        // Field ID should be accessible via documented API
        let field_id = field.get_config_value(&ColumnMetadataKey::ParquetFieldId)
            .expect("Field ID should be accessible via ColumnMetadataKey::ParquetFieldId per lib.rs:836-837");

        match field_id {
            crate::schema::MetadataValue::String(id) => assert_eq!(id, "42"),
            crate::schema::MetadataValue::Number(id) => assert_eq!(*id, 42),
            other => panic!("Expected String or Number, got {other:?}"),
        }
    }

    /// Test that columns are matched by field ID when column names differ.
    ///
    /// Per lib.rs:676-680, field IDs (via [`ColumnMetadataKey::ParquetFieldId`]) should take
    /// precedence over field names for column matching.
    ///
    /// [`ColumnMetadataKey::ParquetFieldId`]: crate::schema::ColumnMetadataKey::ParquetFieldId
    #[test]
    fn test_read_parquet_with_field_id_matching() {
        use crate::schema::{ColumnMetadataKey, MetadataValue, StructField, StructType};

        // Write parquet with field IDs using PARQUET_FIELD_ID_META_KEY (Parquet's native key)
        // The kernel will transform these to parquet.field.id when reading
        let fields = vec![
            Field::new("id", ArrowDataType::Int64, false).with_metadata(HashMap::from([(
                PARQUET_FIELD_ID_META_KEY.to_string(),
                "1".to_string(),
            )])),
            Field::new("name", ArrowDataType::Utf8, false).with_metadata(HashMap::from([(
                PARQUET_FIELD_ID_META_KEY.to_string(),
                "2".to_string(),
            )])),
        ];
        let arrow_schema = Arc::new(ArrowSchema::new(fields));

        let temp_dir = tempfile::tempdir().unwrap();
        let file_path = temp_dir.path().join("field_id_matching.parquet");
        let batch = RecordBatch::try_new(
            arrow_schema.clone(),
            vec![
                Arc::new(Int64Array::from(vec![1, 2, 3])),
                Arc::new(StringArray::from(vec!["alice", "bob", "charlie"])),
            ],
        )
        .unwrap();

        let file = std::fs::File::create(&file_path).unwrap();
        let mut writer = ArrowWriter::try_new(file, arrow_schema, None).unwrap();
        writer.write(&batch).unwrap();
        writer.close().unwrap();

        // Create kernel schema with DIFFERENT names but SAME field IDs
        let kernel_schema = Arc::new(
            StructType::try_new(vec![
                StructField::new("user_id", crate::schema::DataType::LONG, false).with_metadata([
                    (
                        ColumnMetadataKey::ParquetFieldId.as_ref(),
                        MetadataValue::Number(1),
                    ),
                ]),
                StructField::new("user_name", crate::schema::DataType::STRING, false)
                    .with_metadata([(
                        ColumnMetadataKey::ParquetFieldId.as_ref(),
                        MetadataValue::Number(2),
                    )]),
            ])
            .unwrap(),
        );

        // Read using kernel schema with different column names
        let store = Arc::new(LocalFileSystem::new());
        let handler = DefaultParquetHandler::new(store, Arc::new(TokioBackgroundExecutor::new()));
        let file_meta = FileMeta {
            location: Url::from_file_path(&file_path).unwrap(),
            last_modified: 0,
            size: std::fs::metadata(&file_path).unwrap().len(),
        };

        // Should successfully match by field ID despite different names
        let data: Vec<RecordBatch> = handler
            .read_parquet_files(slice::from_ref(&file_meta), kernel_schema, None)
            .unwrap()
            .map(into_record_batch)
            .try_collect()
            .unwrap();

        // Verify data was correctly matched by field ID
        assert_eq!(data.len(), 1);
        let batch = &data[0];

        let id_col = batch
            .column(0)
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();
        assert_eq!(id_col.values(), &[1, 2, 3], "Should match by field ID 1");

        let name_col = batch
            .column(1)
            .as_any()
            .downcast_ref::<StringArray>()
            .unwrap();
        assert_eq!(name_col.value(0), "alice", "Should match by field ID 2");
        assert_eq!(name_col.value(1), "bob");
        assert_eq!(name_col.value(2), "charlie");
    }

    // Verifies that write_parquet (the internal stats-collecting path) does not embed the Arrow
    // IPC schema in the Parquet file metadata.
    #[tokio::test]
    async fn write_parquet_omits_arrow_schema_metadata() {
        let store = Arc::new(InMemory::new());
        let parquet_handler =
            DefaultParquetHandler::new(store.clone(), Arc::new(TokioBackgroundExecutor::new()));

        let data = Box::new(ArrowEngineData::new(
            RecordBatch::try_from_iter(vec![(
                "a",
                Arc::new(Int64Array::from(vec![1, 2, 3])) as Arc<dyn Array>,
            )])
            .unwrap(),
        ));
        let metadata = parquet_handler
            .write_parquet(&Url::parse("memory:///data/").unwrap(), data, &[])
            .await
            .unwrap();

        let path = Path::from_url_path(metadata.file_meta.location.path()).unwrap();
        let reader = ParquetObjectReader::new(store, path);
        let builder = ParquetRecordBatchStreamBuilder::new(reader).await.unwrap();
        let kv = builder.metadata().file_metadata().key_value_metadata();
        let has = kv
            .map(|kv| kv.iter().any(|e| e.key == ARROW_SCHEMA_META_KEY))
            .unwrap_or(false);
        assert!(
            !has,
            "Parquet file should not contain embedded Arrow schema metadata"
        );
    }

    #[tokio::test]
    async fn write_parquet_file_creates_parent_directories() {
        // GIVEN a file path whose parent directories do not exist
        let temp_dir = tempfile::tempdir().unwrap();
        let nested_path = temp_dir.path().join("a/b/c/output.parquet");
        assert!(!nested_path.parent().unwrap().exists());

        let store = Arc::new(LocalFileSystem::new());
        let parquet_handler: Arc<dyn ParquetHandler> = Arc::new(DefaultParquetHandler::new(
            store.clone(),
            Arc::new(TokioBackgroundExecutor::new()),
        ));

        let engine_data: Box<dyn EngineData> = Box::new(ArrowEngineData::new(
            RecordBatch::try_from_iter(vec![(
                "x",
                Arc::new(Int64Array::from(vec![1, 2, 3])) as Arc<dyn Array>,
            )])
            .unwrap(),
        ));
        let data_iter: Box<dyn Iterator<Item = DeltaResult<Box<dyn EngineData>>> + Send> =
            Box::new(std::iter::once(Ok(engine_data)));

        // WHEN we write a parquet file to that path
        let file_url = Url::from_file_path(&nested_path).unwrap();
        parquet_handler
            .write_parquet_file(file_url.clone(), data_iter)
            .unwrap();

        // THEN the file is created and contains the expected data
        assert!(nested_path.exists());

        let path = Path::from_url_path(file_url.path()).unwrap();
        let reader = ParquetObjectReader::new(store.clone(), path);
        let batches: Vec<RecordBatch> = ParquetRecordBatchStreamBuilder::new(reader)
            .await
            .unwrap()
            .build()
            .unwrap()
            .try_collect()
            .await
            .unwrap();
        assert_eq!(batches.len(), 1);
        let col = batches[0]
            .column(0)
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();
        assert_eq!(col.values(), &[1, 2, 3]);
    }
}