datafusion 53.1.0

DataFusion is an in-memory query engine that uses Apache Arrow as the memory model
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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

use crate::execution::SessionState;
use async_trait::async_trait;
use datafusion_catalog_listing::{ListingOptions, ListingTableConfig};
use datafusion_common::{config_datafusion_err, internal_datafusion_err};
use datafusion_session::Session;
use futures::StreamExt;
use std::collections::HashMap;

/// Extension trait for [`ListingTableConfig`] that supports inferring schemas
///
/// This trait exists because the following inference methods only
/// work for [`SessionState`] implementations of [`Session`].
/// See [`ListingTableConfig`] for the remaining inference methods.
#[async_trait]
pub trait ListingTableConfigExt {
    /// Infer `ListingOptions` based on `table_path` and file suffix.
    ///
    /// The format is inferred based on the first `table_path`.
    async fn infer_options(
        self,
        state: &dyn Session,
    ) -> datafusion_common::Result<ListingTableConfig>;

    /// Convenience method to call both [`Self::infer_options`] and [`ListingTableConfig::infer_schema`]
    async fn infer(
        self,
        state: &dyn Session,
    ) -> datafusion_common::Result<ListingTableConfig>;
}

#[async_trait]
impl ListingTableConfigExt for ListingTableConfig {
    async fn infer_options(
        self,
        state: &dyn Session,
    ) -> datafusion_common::Result<ListingTableConfig> {
        let store = if let Some(url) = self.table_paths.first() {
            state.runtime_env().object_store(url)?
        } else {
            return Ok(self);
        };

        let file = self
            .table_paths
            .first()
            .unwrap()
            .list_all_files(state, store.as_ref(), "")
            .await?
            .next()
            .await
            .ok_or_else(|| internal_datafusion_err!("No files for table"))??;

        let (file_extension, maybe_compression_type) =
            ListingTableConfig::infer_file_extension_and_compression_type(
                file.location.as_ref(),
            )?;

        let mut format_options = HashMap::new();
        if let Some(ref compression_type) = maybe_compression_type {
            format_options
                .insert("format.compression".to_string(), compression_type.clone());
        }
        let state = state.as_any().downcast_ref::<SessionState>().unwrap();
        let file_format = state
            .get_file_format_factory(&file_extension)
            .ok_or(config_datafusion_err!(
                "No file_format found with extension {file_extension}"
            ))?
            .create(state, &format_options)?;

        let listing_file_extension =
            if let Some(compression_type) = maybe_compression_type {
                format!("{}.{}", &file_extension, &compression_type)
            } else {
                file_extension
            };

        let listing_options = ListingOptions::new(file_format)
            .with_file_extension(listing_file_extension)
            .with_target_partitions(state.config().target_partitions())
            .with_collect_stat(state.config().collect_statistics());

        Ok(self.with_listing_options(listing_options))
    }

    async fn infer(self, state: &dyn Session) -> datafusion_common::Result<Self> {
        self.infer_options(state).await?.infer_schema(state).await
    }
}

#[cfg(test)]
mod tests {
    #[cfg(feature = "parquet")]
    use crate::datasource::file_format::parquet::ParquetFormat;
    use crate::datasource::listing::table::ListingTableConfigExt;
    use crate::execution::options::JsonReadOptions;
    use crate::prelude::*;
    use crate::{
        datasource::{
            DefaultTableSource, MemTable, file_format::csv::CsvFormat,
            file_format::json::JsonFormat, provider_as_source,
        },
        execution::options::ArrowReadOptions,
        test::{
            columns, object_store::ensure_head_concurrency,
            object_store::make_test_store_and_state, object_store::register_test_store,
        },
    };
    use arrow::{compute::SortOptions, record_batch::RecordBatch};
    use arrow_schema::{DataType, Field, Schema, SchemaRef};
    use datafusion_catalog::TableProvider;
    use datafusion_catalog_listing::{
        ListingOptions, ListingTable, ListingTableConfig, SchemaSource,
    };
    use datafusion_common::{
        DataFusionError, Result, ScalarValue, assert_contains,
        stats::Precision,
        test_util::{batches_to_string, datafusion_test_data},
    };
    use datafusion_datasource::ListingTableUrl;
    use datafusion_datasource::file_compression_type::FileCompressionType;
    use datafusion_datasource::file_format::FileFormat;
    use datafusion_expr::dml::InsertOp;
    use datafusion_expr::{BinaryExpr, LogicalPlanBuilder, Operator};
    use datafusion_physical_expr::PhysicalSortExpr;
    use datafusion_physical_expr::expressions::binary;
    use datafusion_physical_expr_common::sort_expr::LexOrdering;
    use datafusion_physical_plan::empty::EmptyExec;
    use datafusion_physical_plan::{ExecutionPlanProperties, collect};
    use std::collections::HashMap;
    use std::io::Write;
    use std::sync::Arc;
    use tempfile::TempDir;
    use url::Url;

    /// Creates a test schema with standard field types used in tests
    fn create_test_schema() -> SchemaRef {
        Arc::new(Schema::new(vec![
            Field::new("c1", DataType::Float32, true),
            Field::new("c2", DataType::Float64, true),
            Field::new("c3", DataType::Boolean, true),
            Field::new("c4", DataType::Utf8, true),
        ]))
    }

    /// Helper function to generate test file paths with given prefix, count, and optional start index
    fn generate_test_files(prefix: &str, count: usize) -> Vec<String> {
        generate_test_files_with_start(prefix, count, 0)
    }

    /// Helper function to generate test file paths with given prefix, count, and start index
    fn generate_test_files_with_start(
        prefix: &str,
        count: usize,
        start_index: usize,
    ) -> Vec<String> {
        (start_index..start_index + count)
            .map(|i| format!("{prefix}/file{i}"))
            .collect()
    }

    #[tokio::test]
    async fn test_schema_source_tracking_comprehensive() -> Result<()> {
        let ctx = SessionContext::new();
        let testdata = datafusion_test_data();
        let filename = format!("{testdata}/aggregate_simple.csv");
        let table_path = ListingTableUrl::parse(filename)?;

        // Test default schema source
        let format = CsvFormat::default();
        let options = ListingOptions::new(Arc::new(format));
        let config =
            ListingTableConfig::new(table_path.clone()).with_listing_options(options);
        assert_eq!(config.schema_source(), SchemaSource::Unset);

        // Test schema source after setting a schema explicitly
        let provided_schema = create_test_schema();
        let config_with_schema = config.clone().with_schema(provided_schema.clone());
        assert_eq!(config_with_schema.schema_source(), SchemaSource::Specified);

        // Test schema source after inferring schema
        assert_eq!(config.schema_source(), SchemaSource::Unset);

        let config_with_inferred = config.infer_schema(&ctx.state()).await?;
        assert_eq!(config_with_inferred.schema_source(), SchemaSource::Inferred);

        // Test schema preservation through operations
        let config_with_schema_and_options = config_with_schema.clone();
        assert_eq!(
            config_with_schema_and_options.schema_source(),
            SchemaSource::Specified
        );

        // Make sure inferred schema doesn't override specified schema
        let config_with_schema_and_infer = config_with_schema_and_options
            .clone()
            .infer(&ctx.state())
            .await?;
        assert_eq!(
            config_with_schema_and_infer.schema_source(),
            SchemaSource::Specified
        );

        // Verify sources in actual ListingTable objects
        let table_specified = ListingTable::try_new(config_with_schema_and_options)?;
        assert_eq!(table_specified.schema_source(), SchemaSource::Specified);

        let table_inferred = ListingTable::try_new(config_with_inferred)?;
        assert_eq!(table_inferred.schema_source(), SchemaSource::Inferred);

        Ok(())
    }

    #[tokio::test]
    async fn read_single_file() -> Result<()> {
        let ctx = SessionContext::new_with_config(
            SessionConfig::new().with_collect_statistics(true),
        );

        let table = load_table(&ctx, "alltypes_plain.parquet").await?;
        let projection = None;
        let exec = table
            .scan(&ctx.state(), projection, &[], None)
            .await
            .expect("Scan table");

        assert_eq!(exec.children().len(), 0);
        assert_eq!(exec.output_partitioning().partition_count(), 1);

        // test metadata
        assert_eq!(
            exec.partition_statistics(None)?.num_rows,
            Precision::Exact(8)
        );
        assert_eq!(
            exec.partition_statistics(None)?.total_byte_size,
            Precision::Absent,
        );

        Ok(())
    }

    #[cfg(feature = "parquet")]
    #[tokio::test]
    async fn test_try_create_output_ordering() {
        let testdata = crate::test_util::parquet_test_data();
        let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
        let table_path = ListingTableUrl::parse(filename).unwrap();

        let ctx = SessionContext::new();
        let state = ctx.state();
        let options = ListingOptions::new(Arc::new(ParquetFormat::default()));
        let schema = options.infer_schema(&state, &table_path).await.unwrap();

        use crate::datasource::file_format::parquet::ParquetFormat;
        use datafusion_physical_plan::expressions::col as physical_col;
        use datafusion_physical_plan::expressions::lit as physical_lit;
        use std::ops::Add;

        // (file_sort_order, expected_result)
        let cases = vec![
            (
                vec![],
                Ok::<Vec<LexOrdering>, DataFusionError>(Vec::<LexOrdering>::new()),
            ),
            // sort expr, but non column
            (
                vec![vec![col("int_col").add(lit(1)).sort(true, true)]],
                Ok(vec![
                    [PhysicalSortExpr {
                        expr: binary(
                            physical_col("int_col", &schema).unwrap(),
                            Operator::Plus,
                            physical_lit(1),
                            &schema,
                        )
                        .unwrap(),
                        options: SortOptions {
                            descending: false,
                            nulls_first: true,
                        },
                    }]
                    .into(),
                ]),
            ),
            // ok with one column
            (
                vec![vec![col("string_col").sort(true, false)]],
                Ok(vec![
                    [PhysicalSortExpr {
                        expr: physical_col("string_col", &schema).unwrap(),
                        options: SortOptions {
                            descending: false,
                            nulls_first: false,
                        },
                    }]
                    .into(),
                ]),
            ),
            // ok with two columns, different options
            (
                vec![vec![
                    col("string_col").sort(true, false),
                    col("int_col").sort(false, true),
                ]],
                Ok(vec![
                    [
                        PhysicalSortExpr::new_default(
                            physical_col("string_col", &schema).unwrap(),
                        )
                        .asc()
                        .nulls_last(),
                        PhysicalSortExpr::new_default(
                            physical_col("int_col", &schema).unwrap(),
                        )
                        .desc()
                        .nulls_first(),
                    ]
                    .into(),
                ]),
            ),
        ];

        for (file_sort_order, expected_result) in cases {
            let options = options.clone().with_file_sort_order(file_sort_order);

            let config = ListingTableConfig::new(table_path.clone())
                .with_listing_options(options)
                .with_schema(schema.clone());

            let table =
                ListingTable::try_new(config.clone()).expect("Creating the table");
            let ordering_result =
                table.try_create_output_ordering(state.execution_props(), &[]);

            match (expected_result, ordering_result) {
                (Ok(expected), Ok(result)) => {
                    assert_eq!(expected, result);
                }
                (Err(expected), Err(result)) => {
                    // can't compare the DataFusionError directly
                    let result = result.to_string();
                    let expected = expected.to_string();
                    assert_contains!(result.to_string(), expected);
                }
                (expected_result, ordering_result) => {
                    panic!(
                        "expected: {expected_result:#?}\n\nactual:{ordering_result:#?}"
                    );
                }
            }
        }
    }

    #[tokio::test]
    async fn read_empty_table() -> Result<()> {
        let ctx = SessionContext::new();
        let path = String::from("table/p1=v1/file.json");
        register_test_store(&ctx, &[(&path, 100)]);

        let format = JsonFormat::default();
        let ext = format.get_ext();

        let opt = ListingOptions::new(Arc::new(format))
            .with_file_extension(ext)
            .with_table_partition_cols(vec![(String::from("p1"), DataType::Utf8)])
            .with_target_partitions(4);

        let table_path = ListingTableUrl::parse("test:///table/")?;
        let file_schema =
            Arc::new(Schema::new(vec![Field::new("a", DataType::Boolean, false)]));
        let config = ListingTableConfig::new(table_path)
            .with_listing_options(opt)
            .with_schema(file_schema);
        let table = ListingTable::try_new(config)?;

        assert_eq!(
            columns(&table.schema()),
            vec!["a".to_owned(), "p1".to_owned()]
        );

        // this will filter out the only file in the store
        let filter = Expr::not_eq(col("p1"), lit("v1"));

        let scan = table
            .scan(&ctx.state(), None, &[filter], None)
            .await
            .expect("Empty execution plan");

        assert!(scan.as_any().is::<EmptyExec>());
        assert_eq!(
            columns(&scan.schema()),
            vec!["a".to_owned(), "p1".to_owned()]
        );

        Ok(())
    }

    async fn load_table(
        ctx: &SessionContext,
        name: &str,
    ) -> Result<Arc<dyn TableProvider>> {
        let testdata = crate::test_util::parquet_test_data();
        let filename = format!("{testdata}/{name}");
        let table_path = ListingTableUrl::parse(filename)?;

        let config = ListingTableConfig::new(table_path)
            .infer(&ctx.state())
            .await?;
        let table = ListingTable::try_new(config)?;
        Ok(Arc::new(table))
    }

    /// Check that the files listed by the table match the specified `output_partitioning`
    /// when the object store contains `files`.
    async fn assert_list_files_for_scan_grouping(
        files: &[&str],
        table_prefix: &str,
        target_partitions: usize,
        output_partitioning: usize,
        file_ext: Option<&str>,
    ) -> Result<()> {
        let ctx = SessionContext::new();
        register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>());

        let opt = ListingOptions::new(Arc::new(JsonFormat::default()))
            .with_file_extension_opt(file_ext)
            .with_target_partitions(target_partitions);

        let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);

        let table_path = ListingTableUrl::parse(table_prefix)?;
        let config = ListingTableConfig::new(table_path)
            .with_listing_options(opt)
            .with_schema(Arc::new(schema));

        let table = ListingTable::try_new(config)?;

        let result = table.list_files_for_scan(&ctx.state(), &[], None).await?;

        assert_eq!(result.file_groups.len(), output_partitioning);

        Ok(())
    }

    /// Check that the files listed by the table match the specified `output_partitioning`
    /// when the object store contains `files`.
    async fn assert_list_files_for_multi_paths(
        files: &[&str],
        table_prefix: &[&str],
        target_partitions: usize,
        output_partitioning: usize,
        file_ext: Option<&str>,
    ) -> Result<()> {
        let ctx = SessionContext::new();
        register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>());

        let opt = ListingOptions::new(Arc::new(JsonFormat::default()))
            .with_file_extension_opt(file_ext)
            .with_target_partitions(target_partitions);

        let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);

        let table_paths = table_prefix
            .iter()
            .map(|t| ListingTableUrl::parse(t).unwrap())
            .collect();
        let config = ListingTableConfig::new_with_multi_paths(table_paths)
            .with_listing_options(opt)
            .with_schema(Arc::new(schema));

        let table = ListingTable::try_new(config)?;

        let result = table.list_files_for_scan(&ctx.state(), &[], None).await?;

        assert_eq!(result.file_groups.len(), output_partitioning);

        Ok(())
    }

    /// Check that the files listed by the table match the specified `output_partitioning`
    /// when the object store contains `files`, and validate that file metadata is fetched
    /// concurrently
    async fn assert_list_files_for_exact_paths(
        files: &[&str],
        target_partitions: usize,
        output_partitioning: usize,
        file_ext: Option<&str>,
    ) -> Result<()> {
        let ctx = SessionContext::new();
        let (store, _) = make_test_store_and_state(
            &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>(),
        );

        let meta_fetch_concurrency = ctx
            .state()
            .config_options()
            .execution
            .meta_fetch_concurrency;
        let expected_concurrency = files.len().min(meta_fetch_concurrency);
        let head_concurrency_store = ensure_head_concurrency(store, expected_concurrency);

        let url = Url::parse("test://").unwrap();
        ctx.register_object_store(&url, head_concurrency_store.clone());

        let format = JsonFormat::default();

        let opt = ListingOptions::new(Arc::new(format))
            .with_file_extension_opt(file_ext)
            .with_target_partitions(target_partitions);

        let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);

        let table_paths = files
            .iter()
            .map(|t| ListingTableUrl::parse(format!("test:///{t}")).unwrap())
            .collect();
        let config = ListingTableConfig::new_with_multi_paths(table_paths)
            .with_listing_options(opt)
            .with_schema(Arc::new(schema));

        let table = ListingTable::try_new(config)?;

        let result = table.list_files_for_scan(&ctx.state(), &[], None).await?;

        assert_eq!(result.file_groups.len(), output_partitioning);

        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_sql_csv_defaults() -> Result<()> {
        helper_test_insert_into_sql("csv", FileCompressionType::UNCOMPRESSED, "", None)
            .await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_sql_csv_defaults_header_row() -> Result<()> {
        helper_test_insert_into_sql(
            "csv",
            FileCompressionType::UNCOMPRESSED,
            "",
            Some(HashMap::from([("has_header".into(), "true".into())])),
        )
        .await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_sql_json_defaults() -> Result<()> {
        helper_test_insert_into_sql("json", FileCompressionType::UNCOMPRESSED, "", None)
            .await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_sql_parquet_defaults() -> Result<()> {
        helper_test_insert_into_sql(
            "parquet",
            FileCompressionType::UNCOMPRESSED,
            "",
            None,
        )
        .await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_sql_parquet_session_overrides() -> Result<()> {
        let mut config_map: HashMap<String, String> = HashMap::new();
        config_map.insert(
            "datafusion.execution.parquet.compression".into(),
            "zstd(5)".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.dictionary_enabled".into(),
            "false".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.dictionary_page_size_limit".into(),
            "100".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.statistics_enabled".into(),
            "none".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.max_statistics_size".into(),
            "10".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.max_row_group_size".into(),
            "5".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.created_by".into(),
            "datafusion test".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.column_index_truncate_length".into(),
            "50".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.data_page_row_count_limit".into(),
            "50".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.bloom_filter_on_write".into(),
            "true".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.bloom_filter_fpp".into(),
            "0.01".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.bloom_filter_ndv".into(),
            "1000".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.writer_version".into(),
            "2.0".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.write_batch_size".into(),
            "5".into(),
        );
        helper_test_insert_into_sql(
            "parquet",
            FileCompressionType::UNCOMPRESSED,
            "",
            Some(config_map),
        )
        .await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_append_new_parquet_files_session_overrides() -> Result<()> {
        let mut config_map: HashMap<String, String> = HashMap::new();
        config_map.insert(
            "datafusion.execution.soft_max_rows_per_output_file".into(),
            "10".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.compression".into(),
            "zstd(5)".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.dictionary_enabled".into(),
            "false".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.dictionary_page_size_limit".into(),
            "100".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.statistics_enabled".into(),
            "none".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.max_statistics_size".into(),
            "10".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.max_row_group_size".into(),
            "5".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.created_by".into(),
            "datafusion test".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.column_index_truncate_length".into(),
            "50".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.data_page_row_count_limit".into(),
            "50".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.encoding".into(),
            "delta_binary_packed".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.bloom_filter_on_write".into(),
            "true".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.bloom_filter_fpp".into(),
            "0.01".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.bloom_filter_ndv".into(),
            "1000".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.writer_version".into(),
            "2.0".into(),
        );
        config_map.insert(
            "datafusion.execution.parquet.write_batch_size".into(),
            "5".into(),
        );
        config_map.insert("datafusion.execution.batch_size".into(), "10".into());
        helper_test_append_new_files_to_table(
            ParquetFormat::default().get_ext(),
            FileCompressionType::UNCOMPRESSED,
            Some(config_map),
            2,
        )
        .await?;
        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_append_new_parquet_files_invalid_session_fails()
    -> Result<()> {
        let mut config_map: HashMap<String, String> = HashMap::new();
        config_map.insert(
            "datafusion.execution.parquet.compression".into(),
            "zstd".into(),
        );
        let e = helper_test_append_new_files_to_table(
            ParquetFormat::default().get_ext(),
            FileCompressionType::UNCOMPRESSED,
            Some(config_map),
            2,
        )
        .await
        .expect_err("Example should fail!");
        assert_eq!(
            e.strip_backtrace(),
            "Invalid or Unsupported Configuration: zstd compression requires specifying a level such as zstd(4)"
        );

        Ok(())
    }

    async fn helper_test_append_new_files_to_table(
        file_type_ext: String,
        file_compression_type: FileCompressionType,
        session_config_map: Option<HashMap<String, String>>,
        expected_n_files_per_insert: usize,
    ) -> Result<()> {
        // Create the initial context, schema, and batch.
        let session_ctx = match session_config_map {
            Some(cfg) => {
                let config = SessionConfig::from_string_hash_map(&cfg)?;
                SessionContext::new_with_config(config)
            }
            None => SessionContext::new(),
        };

        // Create a new schema with one field called "a" of type Int32
        let schema = Arc::new(Schema::new(vec![Field::new(
            "column1",
            DataType::Int32,
            false,
        )]));

        let filter_predicate = Expr::BinaryExpr(BinaryExpr::new(
            Box::new(Expr::Column("column1".into())),
            Operator::GtEq,
            Box::new(Expr::Literal(ScalarValue::Int32(Some(0)), None)),
        ));

        // Create a new batch of data to insert into the table
        let batch = RecordBatch::try_new(
            schema.clone(),
            vec![Arc::new(arrow::array::Int32Array::from(vec![
                1, 2, 3, 4, 5, 6, 7, 8, 9, 10,
            ]))],
        )?;

        // Register appropriate table depending on file_type we want to test
        let tmp_dir = TempDir::new()?;
        match file_type_ext.as_str() {
            "csv" => {
                session_ctx
                    .register_csv(
                        "t",
                        tmp_dir.path().to_str().unwrap(),
                        CsvReadOptions::new()
                            .schema(schema.as_ref())
                            .file_compression_type(file_compression_type),
                    )
                    .await?;
            }
            "json" => {
                session_ctx
                    .register_json(
                        "t",
                        tmp_dir.path().to_str().unwrap(),
                        JsonReadOptions::default()
                            .schema(schema.as_ref())
                            .file_compression_type(file_compression_type),
                    )
                    .await?;
            }
            #[cfg(feature = "parquet")]
            "parquet" => {
                session_ctx
                    .register_parquet(
                        "t",
                        tmp_dir.path().to_str().unwrap(),
                        ParquetReadOptions::default().schema(schema.as_ref()),
                    )
                    .await?;
            }
            #[cfg(feature = "avro")]
            "avro" => {
                session_ctx
                    .register_avro(
                        "t",
                        tmp_dir.path().to_str().unwrap(),
                        AvroReadOptions::default().schema(schema.as_ref()),
                    )
                    .await?;
            }
            "arrow" => {
                session_ctx
                    .register_arrow(
                        "t",
                        tmp_dir.path().to_str().unwrap(),
                        ArrowReadOptions::default().schema(schema.as_ref()),
                    )
                    .await?;
            }
            _ => panic!("Unrecognized file extension {file_type_ext}"),
        }

        // Create and register the source table with the provided schema and inserted data
        let source_table = Arc::new(MemTable::try_new(
            schema.clone(),
            vec![vec![batch.clone(), batch.clone()]],
        )?);
        session_ctx.register_table("source", source_table.clone())?;
        // Convert the source table into a provider so that it can be used in a query
        let source = provider_as_source(source_table);
        let target = session_ctx.table_provider("t").await?;
        let target = Arc::new(DefaultTableSource::new(target));
        // Create a table scan logical plan to read from the source table
        let scan_plan = LogicalPlanBuilder::scan("source", source, None)?
            .filter(filter_predicate)?
            .build()?;
        // Since logical plan contains a filter, increasing parallelism is helpful.
        // Therefore, we will have 8 partitions in the final plan.
        // Create an insert plan to insert the source data into the initial table
        let insert_into_table =
            LogicalPlanBuilder::insert_into(scan_plan, "t", target, InsertOp::Append)?
                .build()?;
        // Create a physical plan from the insert plan
        let plan = session_ctx
            .state()
            .create_physical_plan(&insert_into_table)
            .await?;
        // Execute the physical plan and collect the results
        let res = collect(plan, session_ctx.task_ctx()).await?;
        // Insert returns the number of rows written, in our case this would be 6.

        insta::allow_duplicates! {insta::assert_snapshot!(batches_to_string(&res),@r"
        +-------+
        | count |
        +-------+
        | 20    |
        +-------+
        ");}

        // Read the records in the table
        let batches = session_ctx
            .sql("select count(*) as count from t")
            .await?
            .collect()
            .await?;

        insta::allow_duplicates! {insta::assert_snapshot!(batches_to_string(&batches),@r"
        +-------+
        | count |
        +-------+
        | 20    |
        +-------+
        ");}

        // Assert that `target_partition_number` many files were added to the table.
        let num_files = tmp_dir.path().read_dir()?.count();
        assert_eq!(num_files, expected_n_files_per_insert);

        // Create a physical plan from the insert plan
        let plan = session_ctx
            .state()
            .create_physical_plan(&insert_into_table)
            .await?;

        // Again, execute the physical plan and collect the results
        let res = collect(plan, session_ctx.task_ctx()).await?;

        insta::allow_duplicates! {insta::assert_snapshot!(batches_to_string(&res),@r"
        +-------+
        | count |
        +-------+
        | 20    |
        +-------+
        ");}

        // Read the contents of the table
        let batches = session_ctx
            .sql("select count(*) AS count from t")
            .await?
            .collect()
            .await?;

        insta::allow_duplicates! {insta::assert_snapshot!(batches_to_string(&batches),@r"
        +-------+
        | count |
        +-------+
        | 40    |
        +-------+
        ");}

        // Assert that another `target_partition_number` many files were added to the table.
        let num_files = tmp_dir.path().read_dir()?.count();
        assert_eq!(num_files, expected_n_files_per_insert * 2);

        // Return Ok if the function
        Ok(())
    }

    /// tests insert into with end to end sql
    /// create external table + insert into statements
    async fn helper_test_insert_into_sql(
        file_type: &str,
        // TODO test with create statement options such as compression
        _file_compression_type: FileCompressionType,
        external_table_options: &str,
        session_config_map: Option<HashMap<String, String>>,
    ) -> Result<()> {
        // Create the initial context
        let session_ctx = match session_config_map {
            Some(cfg) => {
                let config = SessionConfig::from_string_hash_map(&cfg)?;
                SessionContext::new_with_config(config)
            }
            None => SessionContext::new(),
        };

        // create table
        let tmp_dir = TempDir::new()?;
        let str_path = tmp_dir
            .path()
            .to_str()
            .expect("Temp path should convert to &str");
        session_ctx
            .sql(&format!(
                "create external table foo(a varchar, b varchar, c int) \
                        stored as {file_type} \
                        location '{str_path}' \
                        {external_table_options}"
            ))
            .await?
            .collect()
            .await?;

        // insert data
        session_ctx.sql("insert into foo values ('foo', 'bar', 1),('foo', 'bar', 2), ('foo', 'bar', 3)")
            .await?
            .collect()
            .await?;

        // check count
        let batches = session_ctx
            .sql("select * from foo")
            .await?
            .collect()
            .await?;

        insta::allow_duplicates! {insta::assert_snapshot!(batches_to_string(&batches),@r"
        +-----+-----+---+
        | a   | b   | c |
        +-----+-----+---+
        | foo | bar | 1 |
        | foo | bar | 2 |
        | foo | bar | 3 |
        +-----+-----+---+
        ");}

        Ok(())
    }

    #[tokio::test]
    async fn test_infer_options_compressed_csv() -> Result<()> {
        let testdata = crate::test_util::arrow_test_data();
        let filename = format!("{testdata}/csv/aggregate_test_100.csv.gz");
        let table_path = ListingTableUrl::parse(filename)?;

        let ctx = SessionContext::new();

        let config = ListingTableConfig::new(table_path);
        let config_with_opts = config.infer_options(&ctx.state()).await?;
        let config_with_schema = config_with_opts.infer_schema(&ctx.state()).await?;

        let schema = config_with_schema.file_schema.unwrap();

        assert_eq!(schema.fields.len(), 13);

        Ok(())
    }

    #[tokio::test]
    async fn infer_preserves_provided_schema() -> Result<()> {
        let ctx = SessionContext::new();

        let testdata = datafusion_test_data();
        let filename = format!("{testdata}/aggregate_simple.csv");
        let table_path = ListingTableUrl::parse(filename)?;

        let provided_schema = create_test_schema();

        let format = CsvFormat::default();
        let options = ListingOptions::new(Arc::new(format));
        let config = ListingTableConfig::new(table_path)
            .with_listing_options(options)
            .with_schema(Arc::clone(&provided_schema));

        let config = config.infer(&ctx.state()).await?;

        assert_eq!(*config.file_schema.unwrap(), *provided_schema);

        Ok(())
    }

    #[tokio::test]
    async fn test_listing_table_config_with_multiple_files_comprehensive() -> Result<()> {
        let ctx = SessionContext::new();

        // Create test files with different schemas
        let tmp_dir = TempDir::new()?;
        let file_path1 = tmp_dir.path().join("file1.csv");
        let file_path2 = tmp_dir.path().join("file2.csv");

        // File 1: c1,c2,c3
        let mut file1 = std::fs::File::create(&file_path1)?;
        writeln!(file1, "c1,c2,c3")?;
        writeln!(file1, "1,2,3")?;
        writeln!(file1, "4,5,6")?;

        // File 2: c1,c2,c3,c4
        let mut file2 = std::fs::File::create(&file_path2)?;
        writeln!(file2, "c1,c2,c3,c4")?;
        writeln!(file2, "7,8,9,10")?;
        writeln!(file2, "11,12,13,14")?;

        // Parse paths
        let table_path1 = ListingTableUrl::parse(file_path1.to_str().unwrap())?;
        let table_path2 = ListingTableUrl::parse(file_path2.to_str().unwrap())?;

        // Create format and options
        let format = CsvFormat::default().with_has_header(true);
        let options = ListingOptions::new(Arc::new(format));

        // Test case 1: Infer schema using first file's schema
        let config1 = ListingTableConfig::new_with_multi_paths(vec![
            table_path1.clone(),
            table_path2.clone(),
        ])
        .with_listing_options(options.clone());
        let config1 = config1.infer_schema(&ctx.state()).await?;
        assert_eq!(config1.schema_source(), SchemaSource::Inferred);

        // Verify schema matches first file
        let schema1 = config1.file_schema.as_ref().unwrap().clone();
        assert_eq!(schema1.fields().len(), 3);
        assert_eq!(schema1.field(0).name(), "c1");
        assert_eq!(schema1.field(1).name(), "c2");
        assert_eq!(schema1.field(2).name(), "c3");

        // Test case 2: Use specified schema with 3 columns
        let schema_3cols = Arc::new(Schema::new(vec![
            Field::new("c1", DataType::Utf8, true),
            Field::new("c2", DataType::Utf8, true),
            Field::new("c3", DataType::Utf8, true),
        ]));

        let config2 = ListingTableConfig::new_with_multi_paths(vec![
            table_path1.clone(),
            table_path2.clone(),
        ])
        .with_listing_options(options.clone())
        .with_schema(schema_3cols);
        let config2 = config2.infer_schema(&ctx.state()).await?;
        assert_eq!(config2.schema_source(), SchemaSource::Specified);

        // Verify that the schema is still the one we specified (3 columns)
        let schema2 = config2.file_schema.as_ref().unwrap().clone();
        assert_eq!(schema2.fields().len(), 3);
        assert_eq!(schema2.field(0).name(), "c1");
        assert_eq!(schema2.field(1).name(), "c2");
        assert_eq!(schema2.field(2).name(), "c3");

        // Test case 3: Use specified schema with 4 columns
        let schema_4cols = Arc::new(Schema::new(vec![
            Field::new("c1", DataType::Utf8, true),
            Field::new("c2", DataType::Utf8, true),
            Field::new("c3", DataType::Utf8, true),
            Field::new("c4", DataType::Utf8, true),
        ]));

        let config3 = ListingTableConfig::new_with_multi_paths(vec![
            table_path1.clone(),
            table_path2.clone(),
        ])
        .with_listing_options(options.clone())
        .with_schema(schema_4cols);
        let config3 = config3.infer_schema(&ctx.state()).await?;
        assert_eq!(config3.schema_source(), SchemaSource::Specified);

        // Verify that the schema is still the one we specified (4 columns)
        let schema3 = config3.file_schema.as_ref().unwrap().clone();
        assert_eq!(schema3.fields().len(), 4);
        assert_eq!(schema3.field(0).name(), "c1");
        assert_eq!(schema3.field(1).name(), "c2");
        assert_eq!(schema3.field(2).name(), "c3");
        assert_eq!(schema3.field(3).name(), "c4");

        // Test case 4: Verify order matters when inferring schema
        let config4 = ListingTableConfig::new_with_multi_paths(vec![
            table_path2.clone(),
            table_path1.clone(),
        ])
        .with_listing_options(options);
        let config4 = config4.infer_schema(&ctx.state()).await?;

        // Should use first file's schema, which now has 4 columns
        let schema4 = config4.file_schema.as_ref().unwrap().clone();
        assert_eq!(schema4.fields().len(), 4);
        assert_eq!(schema4.field(0).name(), "c1");
        assert_eq!(schema4.field(1).name(), "c2");
        assert_eq!(schema4.field(2).name(), "c3");
        assert_eq!(schema4.field(3).name(), "c4");

        Ok(())
    }

    #[tokio::test]
    async fn test_list_files_configurations() -> Result<()> {
        // Define common test cases as (description, files, paths, target_partitions, expected_partitions, file_ext)
        let test_cases = vec![
            // Single path cases
            (
                "Single path, more partitions than files",
                generate_test_files("bucket/key-prefix", 5),
                vec!["test:///bucket/key-prefix/"],
                12,
                5,
                Some(""),
            ),
            (
                "Single path, equal partitions and files",
                generate_test_files("bucket/key-prefix", 4),
                vec!["test:///bucket/key-prefix/"],
                4,
                4,
                Some(""),
            ),
            (
                "Single path, more files than partitions",
                generate_test_files("bucket/key-prefix", 5),
                vec!["test:///bucket/key-prefix/"],
                2,
                2,
                Some(""),
            ),
            // Multi path cases
            (
                "Multi path, more partitions than files",
                {
                    let mut files = generate_test_files("bucket/key1", 3);
                    files.extend(generate_test_files_with_start("bucket/key2", 2, 3));
                    files.extend(generate_test_files_with_start("bucket/key3", 1, 5));
                    files
                },
                vec!["test:///bucket/key1/", "test:///bucket/key2/"],
                12,
                5,
                Some(""),
            ),
            // No files case
            (
                "No files",
                vec![],
                vec!["test:///bucket/key-prefix/"],
                2,
                0,
                Some(""),
            ),
            // Exact path cases
            (
                "Exact paths test",
                {
                    let mut files = generate_test_files("bucket/key1", 3);
                    files.extend(generate_test_files_with_start("bucket/key2", 2, 3));
                    files
                },
                vec![
                    "test:///bucket/key1/file0",
                    "test:///bucket/key1/file1",
                    "test:///bucket/key1/file2",
                    "test:///bucket/key2/file3",
                    "test:///bucket/key2/file4",
                ],
                12,
                5,
                Some(""),
            ),
        ];

        // Run each test case
        for (test_name, files, paths, target_partitions, expected_partitions, file_ext) in
            test_cases
        {
            println!("Running test: {test_name}");

            if files.is_empty() {
                // Test empty files case
                assert_list_files_for_multi_paths(
                    &[],
                    &paths,
                    target_partitions,
                    expected_partitions,
                    file_ext,
                )
                .await?;
            } else if paths.len() == 1 {
                // Test using single path API
                let file_refs: Vec<&str> = files.iter().map(|s| s.as_str()).collect();
                assert_list_files_for_scan_grouping(
                    &file_refs,
                    paths[0],
                    target_partitions,
                    expected_partitions,
                    file_ext,
                )
                .await?;
            } else if paths[0].contains("test:///bucket/key") {
                // Test using multi path API
                let file_refs: Vec<&str> = files.iter().map(|s| s.as_str()).collect();
                assert_list_files_for_multi_paths(
                    &file_refs,
                    &paths,
                    target_partitions,
                    expected_partitions,
                    file_ext,
                )
                .await?;
            } else {
                // Test using exact path API for specific cases
                let file_refs: Vec<&str> = files.iter().map(|s| s.as_str()).collect();
                assert_list_files_for_exact_paths(
                    &file_refs,
                    target_partitions,
                    expected_partitions,
                    file_ext,
                )
                .await?;
            }
        }

        Ok(())
    }

    #[tokio::test]
    async fn test_listing_table_prunes_extra_files_in_hive() -> Result<()> {
        let files = [
            "bucket/test/pid=1/file1",
            "bucket/test/pid=1/file2",
            "bucket/test/pid=2/file3",
            "bucket/test/pid=2/file4",
            "bucket/test/other/file5",
        ];

        let ctx = SessionContext::new();
        register_test_store(&ctx, &files.iter().map(|f| (*f, 10)).collect::<Vec<_>>());

        let opt = ListingOptions::new(Arc::new(JsonFormat::default()))
            .with_file_extension_opt(Some(""))
            .with_table_partition_cols(vec![("pid".to_string(), DataType::Int32)]);

        let table_path = ListingTableUrl::parse("test:///bucket/test/").unwrap();
        let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);
        let config = ListingTableConfig::new(table_path)
            .with_listing_options(opt)
            .with_schema(Arc::new(schema));

        let table = ListingTable::try_new(config)?;

        let result = table.list_files_for_scan(&ctx.state(), &[], None).await?;
        assert_eq!(result.file_groups.len(), 1);

        let files = result.file_groups[0].clone();

        assert_eq!(
            files
                .iter()
                .map(|f| f.path().to_string())
                .collect::<Vec<_>>(),
            vec![
                "bucket/test/pid=1/file1",
                "bucket/test/pid=1/file2",
                "bucket/test/pid=2/file3",
                "bucket/test/pid=2/file4",
            ]
        );

        Ok(())
    }

    #[cfg(feature = "parquet")]
    #[tokio::test]
    async fn test_table_stats_behaviors() -> Result<()> {
        use crate::datasource::file_format::parquet::ParquetFormat;

        let testdata = crate::test_util::parquet_test_data();
        let filename = format!("{}/{}", testdata, "alltypes_plain.parquet");
        let table_path = ListingTableUrl::parse(filename)?;

        let ctx = SessionContext::new();
        let state = ctx.state();

        // Test 1: Default behavior - stats not collected
        let opt_default = ListingOptions::new(Arc::new(ParquetFormat::default()));
        let schema_default = opt_default.infer_schema(&state, &table_path).await?;
        let config_default = ListingTableConfig::new(table_path.clone())
            .with_listing_options(opt_default)
            .with_schema(schema_default);

        let table_default = ListingTable::try_new(config_default)?;

        let exec_default = table_default.scan(&state, None, &[], None).await?;
        assert_eq!(
            exec_default.partition_statistics(None)?.num_rows,
            Precision::Absent
        );

        // TODO correct byte size: https://github.com/apache/datafusion/issues/14936
        assert_eq!(
            exec_default.partition_statistics(None)?.total_byte_size,
            Precision::Absent
        );

        // Test 2: Explicitly disable stats
        let opt_disabled = ListingOptions::new(Arc::new(ParquetFormat::default()))
            .with_collect_stat(false);
        let schema_disabled = opt_disabled.infer_schema(&state, &table_path).await?;
        let config_disabled = ListingTableConfig::new(table_path.clone())
            .with_listing_options(opt_disabled)
            .with_schema(schema_disabled);
        let table_disabled = ListingTable::try_new(config_disabled)?;

        let exec_disabled = table_disabled.scan(&state, None, &[], None).await?;
        assert_eq!(
            exec_disabled.partition_statistics(None)?.num_rows,
            Precision::Absent
        );
        assert_eq!(
            exec_disabled.partition_statistics(None)?.total_byte_size,
            Precision::Absent
        );

        // Test 3: Explicitly enable stats
        let opt_enabled = ListingOptions::new(Arc::new(ParquetFormat::default()))
            .with_collect_stat(true);
        let schema_enabled = opt_enabled.infer_schema(&state, &table_path).await?;
        let config_enabled = ListingTableConfig::new(table_path)
            .with_listing_options(opt_enabled)
            .with_schema(schema_enabled);
        let table_enabled = ListingTable::try_new(config_enabled)?;

        let exec_enabled = table_enabled.scan(&state, None, &[], None).await?;
        assert_eq!(
            exec_enabled.partition_statistics(None)?.num_rows,
            Precision::Exact(8)
        );
        // TODO correct byte size: https://github.com/apache/datafusion/issues/14936
        assert_eq!(
            exec_enabled.partition_statistics(None)?.total_byte_size,
            Precision::Absent,
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_insert_into_parameterized() -> Result<()> {
        let test_cases = vec![
            // (file_format, batch_size, soft_max_rows, expected_files)
            ("json", 10, 10, 2),
            ("csv", 10, 10, 2),
            #[cfg(feature = "parquet")]
            ("parquet", 10, 10, 2),
            #[cfg(feature = "parquet")]
            ("parquet", 20, 20, 1),
        ];

        for (format, batch_size, soft_max_rows, expected_files) in test_cases {
            println!(
                "Testing insert with format: {format}, batch_size: {batch_size}, expected files: {expected_files}"
            );

            let mut config_map = HashMap::new();
            config_map.insert(
                "datafusion.execution.batch_size".into(),
                batch_size.to_string(),
            );
            config_map.insert(
                "datafusion.execution.soft_max_rows_per_output_file".into(),
                soft_max_rows.to_string(),
            );

            let file_extension = match format {
                "json" => JsonFormat::default().get_ext(),
                "csv" => CsvFormat::default().get_ext(),
                #[cfg(feature = "parquet")]
                "parquet" => ParquetFormat::default().get_ext(),
                _ => unreachable!("Unsupported format"),
            };

            helper_test_append_new_files_to_table(
                file_extension,
                FileCompressionType::UNCOMPRESSED,
                Some(config_map),
                expected_files,
            )
            .await?;
        }

        Ok(())
    }

    #[tokio::test]
    async fn test_basic_table_scan() -> Result<()> {
        let ctx = SessionContext::new();

        // Test basic table creation and scanning
        let path = "table/file.json";
        register_test_store(&ctx, &[(path, 10)]);

        let format = JsonFormat::default();
        let opt = ListingOptions::new(Arc::new(format)).with_collect_stat(false);
        let schema = Schema::new(vec![Field::new("a", DataType::Boolean, false)]);
        let table_path = ListingTableUrl::parse("test:///table/")?;

        let config = ListingTableConfig::new(table_path)
            .with_listing_options(opt)
            .with_schema(Arc::new(schema));

        let table = ListingTable::try_new(config)?;

        // The scan should work correctly
        let scan_result = table.scan(&ctx.state(), None, &[], None).await;
        assert!(scan_result.is_ok(), "Scan should succeed");

        // Verify file listing works
        let result = table.list_files_for_scan(&ctx.state(), &[], None).await?;
        assert!(
            !result.file_groups.is_empty(),
            "Should list files successfully"
        );

        Ok(())
    }
}