datafusion-datasource 53.1.0

datafusion-datasource
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
// 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.

//! Logic for managing groups of [`PartitionedFile`]s in DataFusion

use crate::{FileRange, PartitionedFile};
use arrow::compute::SortOptions;
use datafusion_common::Statistics;
use datafusion_common::utils::compare_rows;
use itertools::Itertools;
use std::cmp::{Ordering, min};
use std::collections::{BinaryHeap, HashMap};
use std::iter::repeat_with;
use std::mem;
use std::ops::{Deref, DerefMut, Index, IndexMut};
use std::sync::Arc;

/// Repartition input files into `target_partitions` partitions, if total file size exceed
/// `repartition_file_min_size`
///
/// This partitions evenly by file byte range, and does not have any knowledge
/// of how data is laid out in specific files. The specific `FileOpener` are
/// responsible for the actual partitioning on specific data source type. (e.g.
/// the `CsvOpener` will read lines overlap with byte range as well as
/// handle boundaries to ensure all lines will be read exactly once)
///
/// # Example
///
/// For example, if there are two files `A` and `B` that we wish to read with 4
/// partitions (with 4 threads) they will be divided as follows:
///
/// ```text
///                                    ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
///                                      ┌─────────────────┐
///                                    │ │                 │ │
///                                      │     File A      │
///                                    │ │  Range: 0-2MB   │ │
///                                      │                 │
///                                    │ └─────────────────┘ │
///                                     ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
/// ┌─────────────────┐                ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
/// │                 │                  ┌─────────────────┐
/// │                 │                │ │                 │ │
/// │                 │                  │     File A      │
/// │                 │                │ │   Range 2-4MB   │ │
/// │                 │                  │                 │
/// │                 │                │ └─────────────────┘ │
/// │  File A (7MB)   │   ────────▶     ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
/// │                 │                ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
/// │                 │                  ┌─────────────────┐
/// │                 │                │ │                 │ │
/// │                 │                  │     File A      │
/// │                 │                │ │  Range: 4-6MB   │ │
/// │                 │                  │                 │
/// │                 │                │ └─────────────────┘ │
/// └─────────────────┘                 ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
/// ┌─────────────────┐                ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
/// │  File B (1MB)   │                  ┌─────────────────┐
/// │                 │                │ │     File A      │ │
/// └─────────────────┘                  │  Range: 6-7MB   │
///                                    │ └─────────────────┘ │
///                                      ┌─────────────────┐
///                                    │ │  File B (1MB)   │ │
///                                      │                 │
///                                    │ └─────────────────┘ │
///                                     ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
///
///                                    If target_partitions = 4,
///                                      divides into 4 groups
/// ```
///
/// # Maintaining Order
///
/// Within each group files are read sequentially. Thus, if the overall order of
/// tuples must be preserved, multiple files can not be mixed in the same group.
///
/// In this case, the code will split the largest files evenly into any
/// available empty groups, but the overall distribution may not be as even
/// as if the order did not need to be preserved.
///
/// ```text
///                                   ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
///                                      ┌─────────────────┐
///                                    │ │                 │ │
///                                      │     File A      │
///                                    │ │  Range: 0-2MB   │ │
///                                      │                 │
/// ┌─────────────────┐                │ └─────────────────┘ │
/// │                 │                 ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
/// │                 │                ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
/// │                 │                  ┌─────────────────┐
/// │                 │                │ │                 │ │
/// │                 │                  │     File A      │
/// │                 │                │ │   Range 2-4MB   │ │
/// │  File A (6MB)   │   ────────▶      │                 │
/// │    (ordered)    │                │ └─────────────────┘ │
/// │                 │                 ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
/// │                 │                ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
/// │                 │                  ┌─────────────────┐
/// │                 │                │ │                 │ │
/// │                 │                  │     File A      │
/// │                 │                │ │  Range: 4-6MB   │ │
/// └─────────────────┘                  │                 │
/// ┌─────────────────┐                │ └─────────────────┘ │
/// │  File B (1MB)   │                 ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
/// │    (ordered)    │                ┌ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ┐
/// └─────────────────┘                  ┌─────────────────┐
///                                    │ │  File B (1MB)   │ │
///                                      │                 │
///                                    │ └─────────────────┘ │
///                                     ─ ─ ─ ─ ─ ─ ─ ─ ─ ─ ─
///
///                                    If target_partitions = 4,
///                                      divides into 4 groups
/// ```
#[derive(Debug, Clone, Copy)]
pub struct FileGroupPartitioner {
    /// how many partitions should be created
    target_partitions: usize,
    /// the minimum size for a file to be repartitioned.
    repartition_file_min_size: usize,
    /// if the order when reading the files must be preserved
    preserve_order_within_groups: bool,
}

impl Default for FileGroupPartitioner {
    fn default() -> Self {
        Self::new()
    }
}

impl FileGroupPartitioner {
    /// Creates a new [`FileGroupPartitioner`] with default values:
    /// 1. `target_partitions = 1`
    /// 2. `repartition_file_min_size = 10MB`
    /// 3. `preserve_order_within_groups = false`
    pub fn new() -> Self {
        Self {
            target_partitions: 1,
            repartition_file_min_size: 10 * 1024 * 1024,
            preserve_order_within_groups: false,
        }
    }

    /// Set the target partitions
    pub fn with_target_partitions(mut self, target_partitions: usize) -> Self {
        self.target_partitions = target_partitions;
        self
    }

    /// Set the minimum size at which to repartition a file
    pub fn with_repartition_file_min_size(
        mut self,
        repartition_file_min_size: usize,
    ) -> Self {
        self.repartition_file_min_size = repartition_file_min_size;
        self
    }

    /// Set whether the order of tuples within a file must be preserved
    pub fn with_preserve_order_within_groups(
        mut self,
        preserve_order_within_groups: bool,
    ) -> Self {
        self.preserve_order_within_groups = preserve_order_within_groups;
        self
    }

    /// Repartition input files according to the settings on this [`FileGroupPartitioner`].
    ///
    /// If no repartitioning is needed or possible, return `None`.
    pub fn repartition_file_groups(
        &self,
        file_groups: &[FileGroup],
    ) -> Option<Vec<FileGroup>> {
        if file_groups.is_empty() {
            return None;
        }

        //  special case when order must be preserved
        if self.preserve_order_within_groups {
            self.repartition_preserving_order(file_groups)
        } else {
            self.repartition_evenly_by_size(file_groups)
        }
    }

    /// Evenly repartition files across partitions by size, ignoring any
    /// existing grouping / ordering
    fn repartition_evenly_by_size(
        &self,
        file_groups: &[FileGroup],
    ) -> Option<Vec<FileGroup>> {
        let target_partitions = self.target_partitions;
        let repartition_file_min_size = self.repartition_file_min_size;
        let flattened_files = file_groups.iter().flat_map(FileGroup::iter).collect_vec();

        let total_size = flattened_files
            .iter()
            .map(|f| f.effective_size())
            .sum::<u64>();
        if total_size < (repartition_file_min_size as u64) || total_size == 0 {
            return None;
        }

        let target_partition_size = total_size.div_ceil(target_partitions as u64);

        let current_partition_index: usize = 0;
        let current_partition_size: u64 = 0;

        // Partition byte range evenly for all `PartitionedFile`s
        let repartitioned_files = flattened_files
            .into_iter()
            .scan(
                (current_partition_index, current_partition_size),
                |(current_partition_index, current_partition_size), source_file| {
                    let mut produced_files = vec![];
                    let (mut range_start, file_end) = source_file.range();
                    while range_start < file_end {
                        let range_end = min(
                            range_start
                                + (target_partition_size - *current_partition_size),
                            file_end,
                        );

                        let mut produced_file = source_file.clone();
                        produced_file.range = Some(FileRange {
                            start: range_start as i64,
                            end: range_end as i64,
                        });
                        produced_files.push((*current_partition_index, produced_file));

                        if *current_partition_size + (range_end - range_start)
                            >= target_partition_size
                        {
                            *current_partition_index += 1;
                            *current_partition_size = 0;
                        } else {
                            *current_partition_size += range_end - range_start;
                        }
                        range_start = range_end;
                    }
                    Some(produced_files)
                },
            )
            .flatten()
            .chunk_by(|(partition_idx, _)| *partition_idx)
            .into_iter()
            .map(|(_, group)| FileGroup::new(group.map(|(_, vals)| vals).collect_vec()))
            .collect_vec();

        Some(repartitioned_files)
    }

    /// Redistribute file groups across size preserving order
    fn repartition_preserving_order(
        &self,
        file_groups: &[FileGroup],
    ) -> Option<Vec<FileGroup>> {
        // Can't repartition and preserve order if there are more groups
        // than partitions
        if file_groups.len() >= self.target_partitions {
            return None;
        }
        let num_new_groups = self.target_partitions - file_groups.len();

        // If there is only a single file
        if file_groups.len() == 1 && file_groups[0].len() == 1 {
            return self.repartition_evenly_by_size(file_groups);
        }

        // Find which files could be split (single file groups)
        let mut heap: BinaryHeap<_> = file_groups
            .iter()
            .enumerate()
            .filter_map(|(group_index, group)| {
                // ignore groups that do not have exactly 1 file
                if group.len() == 1 {
                    Some(ToRepartition {
                        source_index: group_index,
                        file_size: group[0].effective_size(),
                        new_groups: vec![group_index],
                    })
                } else {
                    None
                }
            })
            .map(CompareByRangeSize)
            .collect();

        // No files can be redistributed
        if heap.is_empty() {
            return None;
        }

        // Add new empty groups to which we will redistribute ranges of existing files
        // Add new empty groups to which we will redistribute ranges of existing files
        let mut file_groups: Vec<_> = file_groups
            .iter()
            .cloned()
            .chain(repeat_with(|| FileGroup::new(Vec::new())).take(num_new_groups))
            .collect();

        // Divide up empty groups
        for (group_index, group) in file_groups.iter().enumerate() {
            if !group.is_empty() {
                continue;
            }
            // Pick the file that has the largest ranges to read so far
            let mut largest_group = heap.pop().unwrap();
            largest_group.new_groups.push(group_index);
            heap.push(largest_group);
        }

        // Distribute files to their newly assigned groups
        while let Some(to_repartition) = heap.pop() {
            let range_size = to_repartition.range_size();
            let ToRepartition {
                source_index,
                file_size: _,
                new_groups,
            } = to_repartition.into_inner();
            assert_eq!(file_groups[source_index].len(), 1);
            let original_file = file_groups[source_index].pop().unwrap();

            let last_group = new_groups.len() - 1;
            let (mut range_start, file_end) = original_file.range();
            let mut range_end = range_start + range_size;
            for (i, group_index) in new_groups.into_iter().enumerate() {
                let target_group = &mut file_groups[group_index];
                assert!(target_group.is_empty());

                // adjust last range to include the entire file
                if i == last_group {
                    range_end = file_end;
                }
                target_group.push(
                    original_file
                        .clone()
                        .with_range(range_start as i64, range_end as i64),
                );
                range_start = range_end;
                range_end += range_size;
            }
        }

        Some(file_groups)
    }
}

/// Represents a group of partitioned files that'll be processed by a single thread.
/// Maintains optional statistics across all files in the group.
///
/// # Statistics
///
/// The group-level [`FileGroup::file_statistics`] field contains merged statistics from all files
/// in the group for the **full table schema** (file columns + partition columns).
///
/// Partition column statistics are derived from the individual file partition values:
/// - `min` = minimum partition value across all files in the group
/// - `max` = maximum partition value across all files in the group
/// - `null_count` = 0 (partition values are never null)
///
/// This allows query optimizers to prune entire file groups based on partition bounds.
#[derive(Debug, Clone)]
pub struct FileGroup {
    /// The files in this group
    files: Vec<PartitionedFile>,
    /// Optional statistics for the data across all files in the group.
    ///
    /// These statistics cover the full table schema: file columns plus partition columns.
    /// Partition column statistics are merged from individual [`PartitionedFile::statistics`],
    /// which compute exact values from [`PartitionedFile::partition_values`].
    statistics: Option<Arc<Statistics>>,
}

impl FileGroup {
    /// Creates a new FileGroup from a vector of PartitionedFile objects
    pub fn new(files: Vec<PartitionedFile>) -> Self {
        Self {
            files,
            statistics: None,
        }
    }

    /// Returns the number of files in this group
    pub fn len(&self) -> usize {
        self.files.len()
    }

    /// Set the statistics for this group
    pub fn with_statistics(mut self, statistics: Arc<Statistics>) -> Self {
        self.statistics = Some(statistics);
        self
    }

    /// Returns a slice of the files in this group
    pub fn files(&self) -> &[PartitionedFile] {
        &self.files
    }

    pub fn iter(&self) -> impl Iterator<Item = &PartitionedFile> {
        self.files.iter()
    }

    pub fn into_inner(self) -> Vec<PartitionedFile> {
        self.files
    }

    pub fn is_empty(&self) -> bool {
        self.files.is_empty()
    }

    /// Removes the last element from the files vector and returns it, or None if empty
    pub fn pop(&mut self) -> Option<PartitionedFile> {
        self.files.pop()
    }

    /// Adds a file to the group
    pub fn push(&mut self, partitioned_file: PartitionedFile) {
        self.files.push(partitioned_file);
    }

    /// Get the specific file statistics for the given index
    /// If the index is None, return the `FileGroup` statistics
    pub fn file_statistics(&self, index: Option<usize>) -> Option<&Statistics> {
        if let Some(index) = index {
            self.files.get(index).and_then(|f| f.statistics.as_deref())
        } else {
            self.statistics.as_deref()
        }
    }

    /// Get the mutable reference to the statistics for this group
    pub fn statistics_mut(&mut self) -> Option<&mut Statistics> {
        self.statistics.as_mut().map(Arc::make_mut)
    }

    /// Partition the list of files into `n` groups
    pub fn split_files(mut self, n: usize) -> Vec<FileGroup> {
        if self.is_empty() {
            return vec![];
        }

        // ObjectStore::list does not guarantee any consistent order and for some
        // implementations such as LocalFileSystem, it may be inconsistent. Thus
        // Sort files by path to ensure consistent plans when run more than once.
        self.files.sort_by(|a, b| a.path().cmp(b.path()));

        // effectively this is div with rounding up instead of truncating
        let chunk_size = self.len().div_ceil(n);
        let mut chunks = Vec::with_capacity(n);
        let mut current_chunk = Vec::with_capacity(chunk_size);
        for file in self.files.drain(..) {
            current_chunk.push(file);
            if current_chunk.len() == chunk_size {
                let full_chunk = FileGroup::new(mem::replace(
                    &mut current_chunk,
                    Vec::with_capacity(chunk_size),
                ));
                chunks.push(full_chunk);
            }
        }

        if !current_chunk.is_empty() {
            chunks.push(FileGroup::new(current_chunk))
        }

        chunks
    }

    /// Groups files by their partition values, ensuring all files with same
    /// partition values are in the same group.
    ///
    /// Note: May return fewer groups than `max_target_partitions` when the
    /// number of unique partition values is less than the target.
    pub fn group_by_partition_values(
        self,
        max_target_partitions: usize,
    ) -> Vec<FileGroup> {
        if self.is_empty() || max_target_partitions == 0 {
            return vec![];
        }

        let mut partition_groups: HashMap<
            Vec<datafusion_common::ScalarValue>,
            Vec<PartitionedFile>,
        > = HashMap::new();

        for file in self.files {
            partition_groups
                .entry(file.partition_values.clone())
                .or_default()
                .push(file);
        }

        let num_unique_partitions = partition_groups.len();

        // Sort for deterministic bucket assignment across query executions.
        let mut sorted_partitions: Vec<_> = partition_groups.into_iter().collect();
        let sort_options =
            vec![
                SortOptions::default();
                sorted_partitions.first().map(|(k, _)| k.len()).unwrap_or(0)
            ];
        sorted_partitions.sort_by(|a, b| {
            compare_rows(&a.0, &b.0, &sort_options).unwrap_or(Ordering::Equal)
        });

        if num_unique_partitions <= max_target_partitions {
            sorted_partitions
                .into_iter()
                .map(|(_, files)| FileGroup::new(files))
                .collect()
        } else {
            // Merge into max_target_partitions buckets using round-robin.
            // This maintains grouping by partition value as we are merging groups which already
            // contain all values for a partition key.
            let mut target_groups = vec![vec![]; max_target_partitions];

            for (idx, (_, files)) in sorted_partitions.into_iter().enumerate() {
                let bucket = idx % max_target_partitions;
                target_groups[bucket].extend(files);
            }

            target_groups.into_iter().map(FileGroup::new).collect()
        }
    }
}

impl Index<usize> for FileGroup {
    type Output = PartitionedFile;

    fn index(&self, index: usize) -> &Self::Output {
        &self.files[index]
    }
}

impl IndexMut<usize> for FileGroup {
    fn index_mut(&mut self, index: usize) -> &mut Self::Output {
        &mut self.files[index]
    }
}

impl FromIterator<PartitionedFile> for FileGroup {
    fn from_iter<I: IntoIterator<Item = PartitionedFile>>(iter: I) -> Self {
        let files = iter.into_iter().collect();
        FileGroup::new(files)
    }
}

impl From<Vec<PartitionedFile>> for FileGroup {
    fn from(files: Vec<PartitionedFile>) -> Self {
        FileGroup::new(files)
    }
}

impl Default for FileGroup {
    fn default() -> Self {
        Self::new(Vec::new())
    }
}

/// Tracks how a individual file will be repartitioned
#[derive(Debug, Clone)]
struct ToRepartition {
    /// the index from which the original file will be taken
    source_index: usize,
    /// the size of the original file
    file_size: u64,
    /// indexes of which group(s) will this be distributed to (including `source_index`)
    new_groups: Vec<usize>,
}

impl ToRepartition {
    /// How big will each file range be when this file is read in its new groups?
    fn range_size(&self) -> u64 {
        self.file_size / (self.new_groups.len() as u64)
    }
}

struct CompareByRangeSize(ToRepartition);
impl CompareByRangeSize {
    fn into_inner(self) -> ToRepartition {
        self.0
    }
}
impl Ord for CompareByRangeSize {
    fn cmp(&self, other: &Self) -> Ordering {
        self.0.range_size().cmp(&other.0.range_size())
    }
}
impl PartialOrd for CompareByRangeSize {
    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
        Some(self.cmp(other))
    }
}
impl PartialEq for CompareByRangeSize {
    fn eq(&self, other: &Self) -> bool {
        // PartialEq must be consistent with PartialOrd
        self.cmp(other) == Ordering::Equal
    }
}
impl Eq for CompareByRangeSize {}
impl Deref for CompareByRangeSize {
    type Target = ToRepartition;
    fn deref(&self) -> &Self::Target {
        &self.0
    }
}
impl DerefMut for CompareByRangeSize {
    fn deref_mut(&mut self) -> &mut Self::Target {
        &mut self.0
    }
}

#[cfg(test)]
mod test {
    use super::*;
    use datafusion_common::ScalarValue;

    /// Empty file won't get partitioned
    #[test]
    fn repartition_empty_file_only() {
        let partitioned_file_empty = pfile("empty", 0);
        let file_group = vec![FileGroup::new(vec![partitioned_file_empty])];

        let partitioned_files = FileGroupPartitioner::new()
            .with_target_partitions(4)
            .with_repartition_file_min_size(0)
            .repartition_file_groups(&file_group);

        assert_partitioned_files(None, partitioned_files);
    }

    /// Repartition when there is a empty file in file groups
    #[test]
    fn repartition_empty_files() {
        let pfile_a = pfile("a", 10);
        let pfile_b = pfile("b", 10);
        let pfile_empty = pfile("empty", 0);

        let empty_first = vec![
            FileGroup::new(vec![pfile_empty.clone()]),
            FileGroup::new(vec![pfile_a.clone()]),
            FileGroup::new(vec![pfile_b.clone()]),
        ];
        let empty_middle = vec![
            FileGroup::new(vec![pfile_a.clone()]),
            FileGroup::new(vec![pfile_empty.clone()]),
            FileGroup::new(vec![pfile_b.clone()]),
        ];
        let empty_last = vec![
            FileGroup::new(vec![pfile_a]),
            FileGroup::new(vec![pfile_b]),
            FileGroup::new(vec![pfile_empty]),
        ];

        // Repartition file groups into x partitions
        let expected_2 = vec![
            FileGroup::new(vec![pfile("a", 10).with_range(0, 10)]),
            FileGroup::new(vec![pfile("b", 10).with_range(0, 10)]),
        ];
        let expected_3 = vec![
            FileGroup::new(vec![pfile("a", 10).with_range(0, 7)]),
            FileGroup::new(vec![
                pfile("a", 10).with_range(7, 10),
                pfile("b", 10).with_range(0, 4),
            ]),
            FileGroup::new(vec![pfile("b", 10).with_range(4, 10)]),
        ];

        let file_groups_tests = [empty_first, empty_middle, empty_last];

        for fg in file_groups_tests {
            let all_expected = [(2, expected_2.clone()), (3, expected_3.clone())];
            for (n_partition, expected) in all_expected {
                let actual = FileGroupPartitioner::new()
                    .with_target_partitions(n_partition)
                    .with_repartition_file_min_size(10)
                    .repartition_file_groups(&fg);

                assert_partitioned_files(Some(expected), actual);
            }
        }
    }

    #[test]
    fn repartition_single_file() {
        // Single file, single partition into multiple partitions
        let single_partition = vec![FileGroup::new(vec![pfile("a", 123)])];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&single_partition);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 123).with_range(0, 31)]),
            FileGroup::new(vec![pfile("a", 123).with_range(31, 62)]),
            FileGroup::new(vec![pfile("a", 123).with_range(62, 93)]),
            FileGroup::new(vec![pfile("a", 123).with_range(93, 123)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_single_file_with_range() {
        // Single file, single partition into multiple partitions
        let single_partition =
            vec![FileGroup::new(vec![pfile("a", 123).with_range(0, 123)])];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&single_partition);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 123).with_range(0, 31)]),
            FileGroup::new(vec![pfile("a", 123).with_range(31, 62)]),
            FileGroup::new(vec![pfile("a", 123).with_range(62, 93)]),
            FileGroup::new(vec![pfile("a", 123).with_range(93, 123)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_single_file_with_incomplete_range() {
        // Single file, single partition into multiple partitions
        let single_partition =
            vec![FileGroup::new(vec![pfile("a", 123).with_range(10, 100)])];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&single_partition);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 123).with_range(10, 33)]),
            FileGroup::new(vec![pfile("a", 123).with_range(33, 56)]),
            FileGroup::new(vec![pfile("a", 123).with_range(56, 79)]),
            FileGroup::new(vec![pfile("a", 123).with_range(79, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_single_file_duplicated_with_range() {
        // Single file, two partitions into multiple partitions
        let single_partition = vec![FileGroup::new(vec![
            pfile("a", 100).with_range(0, 50),
            pfile("a", 100).with_range(50, 100),
        ])];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&single_partition);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 100).with_range(0, 25)]),
            FileGroup::new(vec![pfile("a", 100).with_range(25, 50)]),
            FileGroup::new(vec![pfile("a", 100).with_range(50, 75)]),
            FileGroup::new(vec![pfile("a", 100).with_range(75, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_too_much_partitions() {
        // Single file, single partition into 96 partitions
        let partitioned_file = pfile("a", 8);
        let single_partition = vec![FileGroup::new(vec![partitioned_file])];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(96)
            .with_repartition_file_min_size(5)
            .repartition_file_groups(&single_partition);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 8).with_range(0, 1)]),
            FileGroup::new(vec![pfile("a", 8).with_range(1, 2)]),
            FileGroup::new(vec![pfile("a", 8).with_range(2, 3)]),
            FileGroup::new(vec![pfile("a", 8).with_range(3, 4)]),
            FileGroup::new(vec![pfile("a", 8).with_range(4, 5)]),
            FileGroup::new(vec![pfile("a", 8).with_range(5, 6)]),
            FileGroup::new(vec![pfile("a", 8).with_range(6, 7)]),
            FileGroup::new(vec![pfile("a", 8).with_range(7, 8)]),
        ]);

        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_multiple_partitions() {
        // Multiple files in single partition after redistribution
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 40)]),
            FileGroup::new(vec![pfile("b", 60)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(3)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 40).with_range(0, 34)]),
            FileGroup::new(vec![
                pfile("a", 40).with_range(34, 40),
                pfile("b", 60).with_range(0, 28),
            ]),
            FileGroup::new(vec![pfile("b", 60).with_range(28, 60)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_same_num_partitions() {
        // "Rebalance" files across partitions
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 40)]),
            FileGroup::new(vec![pfile("b", 60)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(2)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            FileGroup::new(vec![
                pfile("a", 40).with_range(0, 40),
                pfile("b", 60).with_range(0, 10),
            ]),
            FileGroup::new(vec![pfile("b", 60).with_range(10, 60)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_no_action_min_size() {
        // No action due to target_partition_size
        let single_partition = vec![FileGroup::new(vec![pfile("a", 123)])];

        let actual = FileGroupPartitioner::new()
            .with_target_partitions(65)
            .with_repartition_file_min_size(500)
            .repartition_file_groups(&single_partition);

        assert_partitioned_files(None, actual)
    }

    #[test]
    fn repartition_no_action_zero_files() {
        // No action due to no files
        let empty_partition = vec![];

        let partitioner = FileGroupPartitioner::new()
            .with_target_partitions(65)
            .with_repartition_file_min_size(500);

        assert_partitioned_files(None, repartition_test(partitioner, empty_partition))
    }

    #[test]
    fn repartition_ordered_no_action_too_few_partitions() {
        // No action as there are no new groups to redistribute to
        let input_partitions = vec![
            FileGroup::new(vec![pfile("a", 100)]),
            FileGroup::new(vec![pfile("b", 200)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(2)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&input_partitions);

        assert_partitioned_files(None, actual)
    }

    #[test]
    fn repartition_ordered_no_action_file_too_small() {
        // No action as there are no new groups to redistribute to
        let single_partition = vec![FileGroup::new(vec![pfile("a", 100)])];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(2)
            // file is too small to repartition
            .with_repartition_file_min_size(1000)
            .repartition_file_groups(&single_partition);

        assert_partitioned_files(None, actual)
    }

    #[test]
    fn repartition_ordered_one_large_file() {
        // "Rebalance" the single large file across partitions
        let source_partitions = vec![FileGroup::new(vec![pfile("a", 100)])];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(3)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 100).with_range(0, 34)]),
            FileGroup::new(vec![pfile("a", 100).with_range(34, 68)]),
            FileGroup::new(vec![pfile("a", 100).with_range(68, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_one_large_file_with_range() {
        // "Rebalance" the single large file across partitions
        let source_partitions =
            vec![FileGroup::new(vec![pfile("a", 100).with_range(0, 100)])];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(3)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            FileGroup::new(vec![pfile("a", 100).with_range(0, 34)]),
            FileGroup::new(vec![pfile("a", 100).with_range(34, 68)]),
            FileGroup::new(vec![pfile("a", 100).with_range(68, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_one_large_one_small_file() {
        // "Rebalance" the single large file across empty partitions, but can't split
        // small file
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 100)]),
            FileGroup::new(vec![pfile("b", 30)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            // scan first third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(0, 33)]),
            // only b in this group (can't do this)
            FileGroup::new(vec![pfile("b", 30).with_range(0, 30)]),
            // second third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(33, 66)]),
            // final third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(66, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_one_large_one_small_file_with_full_range() {
        // "Rebalance" the single large file across empty partitions, but can't split
        // small file
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 100).with_range(0, 100)]),
            FileGroup::new(vec![pfile("b", 30)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            // scan first third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(0, 33)]),
            // only b in this group (can't do this)
            FileGroup::new(vec![pfile("b", 30).with_range(0, 30)]),
            // second third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(33, 66)]),
            // final third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(66, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_one_large_one_small_file_with_split_range() {
        // "Rebalance" the single large file across empty partitions, but can't split
        // small file
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 100).with_range(0, 50)]),
            FileGroup::new(vec![pfile("a", 100).with_range(50, 100)]),
            FileGroup::new(vec![pfile("b", 30)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            // scan first half of first "a"
            FileGroup::new(vec![pfile("a", 100).with_range(0, 25)]),
            // second "a" fully (not split)
            FileGroup::new(vec![pfile("a", 100).with_range(50, 100)]),
            // only b in this group (can't do this)
            FileGroup::new(vec![pfile("b", 30).with_range(0, 30)]),
            // second half of first "a"
            FileGroup::new(vec![pfile("a", 100).with_range(25, 50)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_one_large_one_small_file_with_non_full_range() {
        // "Rebalance" the single large file across empty partitions, but can't split
        // small file
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 100).with_range(20, 80)]),
            FileGroup::new(vec![pfile("b", 30).with_range(5, 25)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            // scan first third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(20, 40)]),
            // only b in this group (can't split this)
            FileGroup::new(vec![pfile("b", 30).with_range(5, 25)]),
            // second third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(40, 60)]),
            // final third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(60, 80)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_two_large_files() {
        // "Rebalance" two large files across empty partitions, but can't mix them
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 100)]),
            FileGroup::new(vec![pfile("b", 100)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(4)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            // scan first half of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(0, 50)]),
            // scan first half of "b"
            FileGroup::new(vec![pfile("b", 100).with_range(0, 50)]),
            // second half of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(50, 100)]),
            // second half of "b"
            FileGroup::new(vec![pfile("b", 100).with_range(50, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_two_large_one_small_files() {
        // "Rebalance" two large files and one small file across empty partitions
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 100)]),
            FileGroup::new(vec![pfile("b", 100)]),
            FileGroup::new(vec![pfile("c", 30)]),
        ];

        let partitioner = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_repartition_file_min_size(10);

        // with 4 partitions, can only split the first large file "a"
        let actual = partitioner
            .with_target_partitions(4)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            // scan first half of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(0, 50)]),
            // All of "b"
            FileGroup::new(vec![pfile("b", 100).with_range(0, 100)]),
            // All of "c"
            FileGroup::new(vec![pfile("c", 30).with_range(0, 30)]),
            // second half of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(50, 100)]),
        ]);
        assert_partitioned_files(expected, actual);

        // With 5 partitions, we can split both "a" and "b", but they can't be intermixed
        let actual = partitioner
            .with_target_partitions(5)
            .repartition_file_groups(&source_partitions);

        let expected = Some(vec![
            // scan first half of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(0, 50)]),
            // scan first half of "b"
            FileGroup::new(vec![pfile("b", 100).with_range(0, 50)]),
            // All of "c"
            FileGroup::new(vec![pfile("c", 30).with_range(0, 30)]),
            // second half of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(50, 100)]),
            // second half of "b"
            FileGroup::new(vec![pfile("b", 100).with_range(50, 100)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    #[test]
    fn repartition_ordered_one_large_one_small_existing_empty() {
        // "Rebalance" files using existing empty partition
        let source_partitions = vec![
            FileGroup::new(vec![pfile("a", 100)]),
            FileGroup::default(),
            FileGroup::new(vec![pfile("b", 40)]),
            FileGroup::default(),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(5)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        // Of the three available groups (2 original empty and 1 new from the
        // target partitions), assign two to "a" and one to "b"
        let expected = Some(vec![
            // Scan of "a" across three groups
            FileGroup::new(vec![pfile("a", 100).with_range(0, 33)]),
            FileGroup::new(vec![pfile("a", 100).with_range(33, 66)]),
            // scan first half of "b"
            FileGroup::new(vec![pfile("b", 40).with_range(0, 20)]),
            // final third of "a"
            FileGroup::new(vec![pfile("a", 100).with_range(66, 100)]),
            // second half of "b"
            FileGroup::new(vec![pfile("b", 40).with_range(20, 40)]),
        ]);
        assert_partitioned_files(expected, actual);
    }
    #[test]
    fn repartition_ordered_existing_group_multiple_files() {
        // groups with multiple files in a group can not be changed, but can divide others
        let source_partitions = vec![
            // two files in an existing partition
            FileGroup::new(vec![pfile("a", 100), pfile("b", 100)]),
            FileGroup::new(vec![pfile("c", 40)]),
        ];

        let actual = FileGroupPartitioner::new()
            .with_preserve_order_within_groups(true)
            .with_target_partitions(3)
            .with_repartition_file_min_size(10)
            .repartition_file_groups(&source_partitions);

        // Of the three available groups (2 original empty and 1 new from the
        // target partitions), assign two to "a" and one to "b"
        let expected = Some(vec![
            // don't try and rearrange files in the existing partition
            // assuming that the caller had a good reason to put them that way.
            // (it is technically possible to split off ranges from the files if desired)
            FileGroup::new(vec![pfile("a", 100), pfile("b", 100)]),
            // first half of "c"
            FileGroup::new(vec![pfile("c", 40).with_range(0, 20)]),
            // second half of "c"
            FileGroup::new(vec![pfile("c", 40).with_range(20, 40)]),
        ]);
        assert_partitioned_files(expected, actual);
    }

    /// Asserts that the two groups of [`PartitionedFile`] are the same
    /// (PartitionedFile doesn't implement PartialEq)
    fn assert_partitioned_files(
        expected: Option<Vec<FileGroup>>,
        actual: Option<Vec<FileGroup>>,
    ) {
        match (expected, actual) {
            (None, None) => {}
            (Some(_), None) => panic!("Expected Some, got None"),
            (None, Some(_)) => panic!("Expected None, got Some"),
            (Some(expected), Some(actual)) => {
                let expected_string = format!("{expected:#?}");
                let actual_string = format!("{actual:#?}");
                assert_eq!(expected_string, actual_string);
            }
        }
    }

    /// returns a partitioned file with the specified path and size
    fn pfile(path: impl Into<String>, file_size: u64) -> PartitionedFile {
        PartitionedFile::new(path, file_size)
    }

    /// Creates a file with partition value with a static size of 10.
    fn pfile_with_pv(path: &str, pv: &str) -> PartitionedFile {
        let mut file = pfile(path, 10);
        file.partition_values = vec![ScalarValue::from(pv)];
        file
    }

    /// repartition the file groups both with and without preserving order
    /// asserting they return the same value and returns that value
    fn repartition_test(
        partitioner: FileGroupPartitioner,
        file_groups: Vec<FileGroup>,
    ) -> Option<Vec<FileGroup>> {
        let repartitioned = partitioner.repartition_file_groups(&file_groups);

        let repartitioned_preserving_sort = partitioner
            .with_preserve_order_within_groups(true)
            .repartition_file_groups(&file_groups);

        assert_partitioned_files(repartitioned.clone(), repartitioned_preserving_sort);
        repartitioned
    }

    #[test]
    fn test_group_by_partition_values_edge_cases() {
        // Edge cases: empty and zero target
        assert!(FileGroup::default().group_by_partition_values(4).is_empty());
        assert!(
            FileGroup::new(vec![pfile("a", 100)])
                .group_by_partition_values(0)
                .is_empty()
        );
    }

    #[test]
    fn test_group_by_partition_values_less_groups_than_target() {
        // File a and b have partition value p1.
        // File c has partition value p2.
        // Grouping by partition value should not redistribute any files since the number of partition
        // values <= max_target_partitions.
        let fg = FileGroup::new(vec![
            pfile_with_pv("a", "p1"),
            pfile_with_pv("b", "p1"),
            pfile_with_pv("c", "p2"),
        ]);
        let groups = fg.group_by_partition_values(4);
        assert_eq!(groups.len(), 2);
        assert_eq!(groups[0].len(), 2);
        assert_eq!(groups[1].len(), 1);
    }

    #[test]
    fn test_group_by_partition_values_more_groups_than_target() {
        // Each file has a single partition value. The number of partition values > max_target_partitions, so
        // they should be round-robin distributed into groups.
        let fg = FileGroup::new(vec![
            pfile_with_pv("a", "p1"),
            pfile_with_pv("b", "p2"),
            pfile_with_pv("c", "p3"),
            pfile_with_pv("d", "p4"),
            pfile_with_pv("e", "p5"),
        ]);
        let groups = fg.group_by_partition_values(3);
        assert_eq!(groups.len(), 3);
        assert_eq!(groups[0].len(), 2);
        assert_eq!(groups[1].len(), 2);
        assert_eq!(groups[2].len(), 1);
    }
}