lance_encoding/encodings/logical/
list.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::{collections::VecDeque, ops::Range, sync::Arc};

use arrow_array::{
    cast::AsArray,
    new_empty_array,
    types::{Int32Type, Int64Type, UInt64Type},
    Array, ArrayRef, BooleanArray, Int32Array, Int64Array, LargeListArray, ListArray, UInt64Array,
};
use arrow_buffer::{BooleanBuffer, BooleanBufferBuilder, Buffer, NullBuffer, OffsetBuffer};
use arrow_schema::{DataType, Field, Fields};
use futures::{future::BoxFuture, FutureExt};
use lance_arrow::list::ListArrayExt;
use log::trace;
use snafu::{location, Location};
use tokio::task::JoinHandle;

use lance_core::{cache::FileMetadataCache, Error, Result};

use crate::{
    buffer::LanceBuffer,
    data::{BlockInfo, DataBlock, FixedWidthDataBlock},
    decoder::{
        DecodeArrayTask, DecodeBatchScheduler, DecodedArray, FieldScheduler, FilterExpression,
        ListPriorityRange, LogicalPageDecoder, MessageType, NextDecodeTask, PageEncoding,
        PriorityRange, ScheduledScanLine, SchedulerContext, SchedulingJob,
        StructuralDecodeArrayTask, StructuralFieldDecoder, StructuralFieldScheduler,
        StructuralSchedulingJob,
    },
    encoder::{
        ArrayEncoder, EncodeTask, EncodedArray, EncodedColumn, EncodedPage, FieldEncoder,
        OutOfLineBuffers,
    },
    encodings::logical::r#struct::SimpleStructScheduler,
    format::pb,
    repdef::RepDefBuilder,
    EncodingsIo,
};

use super::{primitive::AccumulationQueue, r#struct::SimpleStructDecoder};

// Scheduling lists is tricky.  Imagine the following scenario:
//
// * There are 2000 offsets per offsets page
// * The user requests range 8000..8500
//
// First, since 8000 matches the start of an offsets page, we don't need to read an extra offset.
//
// Since this range matches the start of a page, we know we will get an offsets array like
// [0, ...]
//
// We need to restore nulls, which relies on a null offset adjustment, which is unique to each offsets
// page.
//
// We need to map this to [X, ...] where X is the sum of the number of items in the 0-2000, 2000-4000,
// and 4000-6000 pages.
//
// This gets even trickier if a range spans multiple offsets pages.  For example, given the same
// scenario but the user requests 7999..8500.  In this case the first page read will include an
// extra offset (e.g. we need to read 7998..8000), the null adjustment will be different between the
// two, and the items offset will be different.
//
// To handle this, we take the incoming row requests, look at the page info, and then calculate
// list requests.

#[derive(Debug)]
struct ListRequest {
    /// How many lists this request maps to
    num_lists: u64,
    /// Did this request include an extra offset
    includes_extra_offset: bool,
    /// The null offset adjustment for this request
    null_offset_adjustment: u64,
    /// items offset to apply
    items_offset: u64,
}

#[derive(Debug)]
struct ListRequestsIter {
    // The bool triggers whether we need to skip an offset or not
    list_requests: VecDeque<ListRequest>,
    offsets_requests: Vec<Range<u64>>,
}

impl ListRequestsIter {
    // TODO: This logic relies on row_ranges being ordered and may be a problem when we
    // add proper support for out-of-order take
    fn new(row_ranges: &[Range<u64>], page_infos: &[OffsetPageInfo]) -> Self {
        let mut items_offset = 0;
        let mut offsets_offset = 0;
        let mut page_infos_iter = page_infos.iter();
        let mut cur_page_info = page_infos_iter.next().unwrap();
        let mut list_requests = VecDeque::new();
        let mut offsets_requests = Vec::new();

        // Each row range maps to at least one list request.  It may map to more if the
        // range spans multiple offsets pages.
        for range in row_ranges {
            let mut range = range.clone();

            // Skip any offsets pages that are before the range
            while offsets_offset + (cur_page_info.offsets_in_page) <= range.start {
                trace!("Skipping null offset adjustment chunk {:?}", offsets_offset);
                offsets_offset += cur_page_info.offsets_in_page;
                items_offset += cur_page_info.num_items_referenced_by_page;
                cur_page_info = page_infos_iter.next().unwrap();
            }

            // If the range starts at the beginning of an offsets page we don't need
            // to read an extra offset
            let mut includes_extra_offset = range.start != offsets_offset;
            if includes_extra_offset {
                offsets_requests.push(range.start - 1..range.end);
            } else {
                offsets_requests.push(range.clone());
            }

            // At this point our range overlaps the current page (cur_page_info) and
            // we can start slicing it into list requests
            while !range.is_empty() {
                // The end of the list request is the min of the end of the range
                // and the end of the current page
                let end = offsets_offset + cur_page_info.offsets_in_page;
                let last = end >= range.end;
                let end = end.min(range.end);
                list_requests.push_back(ListRequest {
                    num_lists: end - range.start,
                    includes_extra_offset,
                    null_offset_adjustment: cur_page_info.null_offset_adjustment,
                    items_offset,
                });

                includes_extra_offset = false;
                range.start = end;
                // If there is still more data in the range, we need to move to the
                // next page
                if !last {
                    offsets_offset += cur_page_info.offsets_in_page;
                    items_offset += cur_page_info.num_items_referenced_by_page;
                    cur_page_info = page_infos_iter.next().unwrap();
                }
            }
        }
        Self {
            list_requests,
            offsets_requests,
        }
    }

    // Given a page of offset data, grab the corresponding list requests
    fn next(&mut self, mut num_offsets: u64) -> Vec<ListRequest> {
        let mut list_requests = Vec::new();
        while num_offsets > 0 {
            let req = self.list_requests.front_mut().unwrap();
            // If the request did not start at zero then we need to read an extra offset
            if req.includes_extra_offset {
                num_offsets -= 1;
                debug_assert_ne!(num_offsets, 0);
            }
            if num_offsets >= req.num_lists {
                num_offsets -= req.num_lists;
                list_requests.push(self.list_requests.pop_front().unwrap());
            } else {
                let sub_req = ListRequest {
                    num_lists: num_offsets,
                    includes_extra_offset: req.includes_extra_offset,
                    null_offset_adjustment: req.null_offset_adjustment,
                    items_offset: req.items_offset,
                };

                list_requests.push(sub_req);
                req.includes_extra_offset = false;
                req.num_lists -= num_offsets;
                num_offsets = 0;
            }
        }
        list_requests
    }
}

/// Given a list of offsets and a list of requested list row ranges we need to rewrite the offsets so that
/// they appear as expected for a list array.  This involves a number of tasks:
///
///  * Nulls in the offsets are represented by oversize values and these need to be converted to
///    the appropriate length
///  * For each range we (usually) load N + 1 offsets, so if we have 5 ranges we have 5 extra values
///    and we need to drop 4 of those.
///  * Ranges may not start at 0 and, while we don't strictly need to, we want to go ahead and normalize
///    the offsets so that the first offset is 0.
///
/// Throughout the comments we will consider the following example case:
///
/// The user requests the following ranges of lists (list_row_ranges): [0..3, 5..6]
///
/// This is a total of 4 lists.  The loaded offsets are [10, 20, 120, 150, 60].  The last valid offset is 99.
/// The null_offset_adjustment will be 100.
///
/// Our desired output offsets are going to be [0, 10, 20, 20, 30] and the item ranges are [0..20] and [50..60]
/// The validity array is [true, true, false, true]
fn decode_offsets(
    offsets: &dyn Array,
    list_requests: &[ListRequest],
    null_offset_adjustment: u64,
) -> (VecDeque<Range<u64>>, Vec<u64>, BooleanBuffer) {
    // In our example this is [10, 20, 120, 50, 60]
    let numeric_offsets = offsets.as_primitive::<UInt64Type>();
    // In our example there are 4 total lists
    let total_num_lists = list_requests.iter().map(|req| req.num_lists).sum::<u64>() as u32;
    let mut normalized_offsets = Vec::with_capacity(total_num_lists as usize);
    let mut validity_buffer = BooleanBufferBuilder::new(total_num_lists as usize);
    // The first output offset is always 0 no matter what
    normalized_offsets.push(0);
    let mut last_normalized_offset = 0;
    let offsets_values = numeric_offsets.values();

    let mut item_ranges = VecDeque::new();
    let mut offsets_offset: u32 = 0;
    // All ranges should be non-empty
    debug_assert!(list_requests.iter().all(|r| r.num_lists > 0));
    for req in list_requests {
        // The # of lists in this particular range
        let num_lists = req.num_lists;

        // Because we know the first offset is always 0 we don't store that.  This means we have special
        // logic if a range starts at 0 (we didn't need to read an extra offset value in that case)
        // In our example we enter this special case on the first range (0..3) but not the second (5..6)
        // This means the first range, which has 3 lists, maps to 3 values in our offsets array [10, 20, 120]
        // However, the second range, which has 1 list, maps to 2 values in our offsets array [150, 60]
        let (items_range, offsets_to_norm_start, num_offsets_to_norm) =
            if !req.includes_extra_offset {
                // In our example items start is 0 and items_end is 20
                let first_offset_idx = 0_usize;
                let num_offsets = num_lists as usize;
                let items_start = 0;
                let items_end = offsets_values[num_offsets - 1] % null_offset_adjustment;
                let items_range = items_start..items_end;
                (items_range, first_offset_idx, num_offsets)
            } else {
                // In our example, offsets_offset will be 3, items_start will be 50, and items_end will
                // be 60
                let first_offset_idx = offsets_offset as usize;
                let num_offsets = num_lists as usize + 1;
                let items_start = offsets_values[first_offset_idx] % null_offset_adjustment;
                let items_end =
                    offsets_values[first_offset_idx + num_offsets - 1] % null_offset_adjustment;
                let items_range = items_start..items_end;
                (items_range, first_offset_idx, num_offsets)
            };

        // TODO: Maybe consider writing whether there are nulls or not as part of the
        // page description.  Then we can skip all validity work.  Not clear if that will
        // be any benefit though.

        // We calculate validity from all elements but the first (or all elements
        // if this is the special zero-start case)
        //
        // So, in our first pass through, we consider [10, 20, 120] (1 null)
        // In our second pass through we only consider [60] (0 nulls)
        // Note that the 150 is null but we only loaded it to know where the 50-60 list started
        // and it doesn't actually correspond to a list (e.g. list 4 is null but we aren't loading it
        // here)
        let validity_start = if !req.includes_extra_offset {
            0
        } else {
            offsets_to_norm_start + 1
        };
        for off in offsets_values
            .slice(validity_start, num_lists as usize)
            .iter()
        {
            validity_buffer.append(*off < null_offset_adjustment);
        }

        // In our special case we need to account for the offset 0-first_item
        if !req.includes_extra_offset {
            let first_item = offsets_values[0] % null_offset_adjustment;
            normalized_offsets.push(first_item);
            last_normalized_offset = first_item;
        }

        // Finally, we go through and shift the offsets.  If we just returned them as is (taking care of
        // nulls) we would get [0, 10, 20, 20, 60] but our last list only has 10 items, not 40 and so we
        // need to shift that 60 to a 40.
        normalized_offsets.extend(
                offsets_values
                    .slice(offsets_to_norm_start, num_offsets_to_norm)
                    .windows(2)
                    .map(|w| {
                        let start = w[0] % null_offset_adjustment;
                        let end = w[1] % null_offset_adjustment;
                        if end < start {
                            panic!("End is less than start in window {:?} with null_offset_adjustment={} we get start={} and end={}", w, null_offset_adjustment, start, end);
                        }
                        let length = end - start;
                        last_normalized_offset += length;
                        last_normalized_offset
                    }),
            );
        trace!(
            "List offsets range of {} lists maps to item range {:?}",
            num_lists,
            items_range
        );
        offsets_offset += num_offsets_to_norm as u32;
        if !items_range.is_empty() {
            let items_range =
                items_range.start + req.items_offset..items_range.end + req.items_offset;
            item_ranges.push_back(items_range);
        }
    }

    let validity = validity_buffer.finish();
    (item_ranges, normalized_offsets, validity)
}

/// After scheduling the offsets we immediately launch this task as a new tokio task
/// This task waits for the offsets to arrive, decodes them, and then schedules the I/O
/// for the items.
///
/// This task does not wait for the items data.  That happens on the main decode loop (unless
/// we have list of list of ... in which case it happens in the outer indirect decode loop)
#[allow(clippy::too_many_arguments)]
async fn indirect_schedule_task(
    mut offsets_decoder: Box<dyn LogicalPageDecoder>,
    list_requests: Vec<ListRequest>,
    null_offset_adjustment: u64,
    items_scheduler: Arc<dyn FieldScheduler>,
    items_type: DataType,
    io: Arc<dyn EncodingsIo>,
    cache: Arc<FileMetadataCache>,
    priority: Box<dyn PriorityRange>,
) -> Result<IndirectlyLoaded> {
    let num_offsets = offsets_decoder.num_rows();
    // We know the offsets are a primitive array and thus will not need additional
    // pages.  We can use a dummy receiver to match the decoder API
    offsets_decoder.wait_for_loaded(num_offsets - 1).await?;
    let decode_task = offsets_decoder.drain(num_offsets)?;
    let offsets = decode_task.task.decode()?;

    let (item_ranges, offsets, validity) =
        decode_offsets(offsets.as_ref(), &list_requests, null_offset_adjustment);

    trace!(
        "Indirectly scheduling items ranges {:?} from list items column with {} rows (and priority {:?})",
        item_ranges,
        items_scheduler.num_rows(),
        priority
    );
    let offsets: Arc<[u64]> = offsets.into();

    // All requested lists are empty
    if item_ranges.is_empty() {
        debug_assert!(item_ranges.iter().all(|r| r.start == r.end));
        return Ok(IndirectlyLoaded {
            root_decoder: None,
            offsets,
            validity,
        });
    }
    let item_ranges = item_ranges.into_iter().collect::<Vec<_>>();
    let num_items = item_ranges.iter().map(|r| r.end - r.start).sum::<u64>();

    // Create a new root scheduler, which has one column, which is our items data
    let root_fields = Fields::from(vec![Field::new("item", items_type, true)]);
    let indirect_root_scheduler =
        SimpleStructScheduler::new(vec![items_scheduler], root_fields.clone());
    let mut indirect_scheduler = DecodeBatchScheduler::from_scheduler(
        Arc::new(indirect_root_scheduler),
        root_fields.clone(),
        cache,
    );
    let mut root_decoder = SimpleStructDecoder::new(root_fields, num_items);

    let priority = Box::new(ListPriorityRange::new(priority, offsets.clone()));

    let indirect_messages = indirect_scheduler.schedule_ranges_to_vec(
        &item_ranges,
        // Can't push filters into list items
        &FilterExpression::no_filter(),
        io,
        Some(priority),
    )?;

    for message in indirect_messages {
        for decoder in message.decoders {
            let decoder = decoder.into_legacy();
            if !decoder.path.is_empty() {
                root_decoder.accept_child(decoder)?;
            }
        }
    }

    Ok(IndirectlyLoaded {
        offsets,
        validity,
        root_decoder: Some(root_decoder),
    })
}

#[derive(Debug)]
struct ListFieldSchedulingJob<'a> {
    scheduler: &'a ListFieldScheduler,
    offsets: Box<dyn SchedulingJob + 'a>,
    num_rows: u64,
    list_requests_iter: ListRequestsIter,
}

impl<'a> ListFieldSchedulingJob<'a> {
    fn try_new(
        scheduler: &'a ListFieldScheduler,
        ranges: &[Range<u64>],
        filter: &FilterExpression,
    ) -> Result<Self> {
        let list_requests_iter = ListRequestsIter::new(ranges, &scheduler.offset_page_info);
        let num_rows = ranges.iter().map(|r| r.end - r.start).sum::<u64>();
        let offsets = scheduler
            .offsets_scheduler
            .schedule_ranges(&list_requests_iter.offsets_requests, filter)?;
        Ok(Self {
            scheduler,
            offsets,
            list_requests_iter,
            num_rows,
        })
    }
}

impl SchedulingJob for ListFieldSchedulingJob<'_> {
    fn schedule_next(
        &mut self,
        context: &mut SchedulerContext,
        priority: &dyn PriorityRange,
    ) -> Result<ScheduledScanLine> {
        let next_offsets = self.offsets.schedule_next(context, priority)?;
        let offsets_scheduled = next_offsets.rows_scheduled;
        let list_reqs = self.list_requests_iter.next(offsets_scheduled);
        trace!(
            "Scheduled {} offsets which maps to list requests: {:?}",
            offsets_scheduled,
            list_reqs
        );
        let null_offset_adjustment = list_reqs[0].null_offset_adjustment;
        // It shouldn't be possible for `list_reqs` to span more than one offsets page and so it shouldn't
        // be possible for the null_offset_adjustment to change
        debug_assert!(list_reqs
            .iter()
            .all(|req| req.null_offset_adjustment == null_offset_adjustment));
        let num_rows = list_reqs.iter().map(|req| req.num_lists).sum::<u64>();
        // offsets is a uint64 which is guaranteed to create one decoder on each call to schedule_next
        let next_offsets_decoder = next_offsets
            .decoders
            .into_iter()
            .next()
            .unwrap()
            .into_legacy()
            .decoder;

        let items_scheduler = self.scheduler.items_scheduler.clone();
        let items_type = self.scheduler.items_field.data_type().clone();
        let io = context.io().clone();
        let cache = context.cache().clone();

        // Immediately spawn the indirect scheduling
        let indirect_fut = tokio::spawn(indirect_schedule_task(
            next_offsets_decoder,
            list_reqs,
            null_offset_adjustment,
            items_scheduler,
            items_type,
            io,
            cache,
            priority.box_clone(),
        ));

        // Return a decoder
        let decoder = Box::new(ListPageDecoder {
            offsets: Arc::new([]),
            validity: BooleanBuffer::new(Buffer::from_vec(Vec::<u8>::default()), 0, 0),
            item_decoder: None,
            rows_drained: 0,
            rows_loaded: 0,
            items_field: self.scheduler.items_field.clone(),
            num_rows,
            unloaded: Some(indirect_fut),
            offset_type: self.scheduler.offset_type.clone(),
            data_type: self.scheduler.list_type.clone(),
        });
        let decoder = context.locate_decoder(decoder);
        Ok(ScheduledScanLine {
            decoders: vec![MessageType::DecoderReady(decoder)],
            rows_scheduled: num_rows,
        })
    }

    fn num_rows(&self) -> u64 {
        self.num_rows
    }
}

/// A page scheduler for list fields that encodes offsets in one field and items in another
///
/// The list scheduler is somewhat unique because it requires indirect I/O.  We cannot know the
/// ranges we need simply by looking at the metadata.  This means that list scheduling doesn't
/// fit neatly into the two-thread schedule-loop / decode-loop model.  To handle this, when a
/// list page is scheduled, we only schedule the I/O for the offsets and then we immediately
/// launch a new tokio task.  This new task waits for the offsets, decodes them, and then
/// schedules the I/O for the items.  Keep in mind that list items can be lists themselves.  If
/// that is the case then this indirection will continue.  The decode task that is returned will
/// only finish `wait`ing when all of the I/O has completed.
///
/// Whenever we schedule follow-up I/O like this the priority is based on the top-level row
/// index.  This helps ensure that earlier rows get finished completely (including follow up
/// tasks) before we perform I/O for later rows.
#[derive(Debug)]
pub struct ListFieldScheduler {
    offsets_scheduler: Arc<dyn FieldScheduler>,
    items_scheduler: Arc<dyn FieldScheduler>,
    items_field: Arc<Field>,
    offset_type: DataType,
    list_type: DataType,
    offset_page_info: Vec<OffsetPageInfo>,
}

/// The offsets are stored in a uint64 encoded column.  For each page we
/// store some supplementary data that helps us understand the offsets.
/// This is needed to construct the scheduler
#[derive(Debug)]
pub struct OffsetPageInfo {
    pub offsets_in_page: u64,
    pub null_offset_adjustment: u64,
    pub num_items_referenced_by_page: u64,
}

impl ListFieldScheduler {
    // Create a new ListPageScheduler
    pub fn new(
        offsets_scheduler: Arc<dyn FieldScheduler>,
        items_scheduler: Arc<dyn FieldScheduler>,
        items_field: Arc<Field>,
        // Should be int32 or int64
        offset_type: DataType,
        offset_page_info: Vec<OffsetPageInfo>,
    ) -> Self {
        let list_type = match &offset_type {
            DataType::Int32 => DataType::List(items_field.clone()),
            DataType::Int64 => DataType::LargeList(items_field.clone()),
            _ => panic!("Unexpected offset type {}", offset_type),
        };
        Self {
            offsets_scheduler,
            items_scheduler,
            items_field,
            offset_type,
            offset_page_info,
            list_type,
        }
    }
}

impl FieldScheduler for ListFieldScheduler {
    fn schedule_ranges<'a>(
        &'a self,
        ranges: &[Range<u64>],
        filter: &FilterExpression,
    ) -> Result<Box<dyn SchedulingJob + 'a>> {
        Ok(Box::new(ListFieldSchedulingJob::try_new(
            self, ranges, filter,
        )?))
    }

    fn num_rows(&self) -> u64 {
        self.offsets_scheduler.num_rows()
    }

    fn initialize<'a>(
        &'a self,
        _filter: &'a FilterExpression,
        _context: &'a SchedulerContext,
    ) -> BoxFuture<'a, Result<()>> {
        // 2.0 schedulers do not need to initialize
        std::future::ready(Ok(())).boxed()
    }
}

/// As soon as the first call to decode comes in we wait for all indirect I/O to
/// complete.
///
/// Once the indirect I/O is finished we pull items out of `unawaited`, wait them
/// (this wait should return immediately) and then push them into `item_decoders`.
///
/// We then drain from `item_decoders`, popping item pages off as we finish with
/// them.
///
/// TODO: Test the case where a single list page has multiple items pages
#[derive(Debug)]
struct ListPageDecoder {
    unloaded: Option<JoinHandle<Result<IndirectlyLoaded>>>,
    // offsets and validity will have already been decoded as part of the indirect I/O
    offsets: Arc<[u64]>,
    validity: BooleanBuffer,
    item_decoder: Option<SimpleStructDecoder>,
    num_rows: u64,
    rows_drained: u64,
    rows_loaded: u64,
    items_field: Arc<Field>,
    offset_type: DataType,
    data_type: DataType,
}

struct ListDecodeTask {
    offsets: Vec<u64>,
    validity: BooleanBuffer,
    // Will be None if there are no items (all empty / null lists)
    items: Option<Box<dyn DecodeArrayTask>>,
    items_field: Arc<Field>,
    offset_type: DataType,
}

impl DecodeArrayTask for ListDecodeTask {
    fn decode(self: Box<Self>) -> Result<ArrayRef> {
        let items = self
            .items
            .map(|items| {
                // When we run the indirect I/O we wrap things in a struct array with a single field
                // named "item".  We can unwrap that now.
                let wrapped_items = items.decode()?;
                Result::Ok(wrapped_items.as_struct().column(0).clone())
            })
            .unwrap_or_else(|| Ok(new_empty_array(self.items_field.data_type())))?;

        // The offsets are already decoded but they need to be shifted back to 0 and cast
        // to the appropriate type
        //
        // Although, in some cases, the shift IS strictly required since the unshifted offsets
        // may cross i32::MAX even though the shifted offsets do not
        let offsets = UInt64Array::from(self.offsets);
        let validity = NullBuffer::new(self.validity);
        let validity = if validity.null_count() == 0 {
            None
        } else {
            Some(validity)
        };
        let min_offset = UInt64Array::new_scalar(offsets.value(0));
        let offsets = arrow_arith::numeric::sub(&offsets, &min_offset)?;
        match &self.offset_type {
            DataType::Int32 => {
                let offsets = arrow_cast::cast(&offsets, &DataType::Int32)?;
                let offsets_i32 = offsets.as_primitive::<Int32Type>();
                let offsets = OffsetBuffer::new(offsets_i32.values().clone());

                Ok(Arc::new(ListArray::try_new(
                    self.items_field.clone(),
                    offsets,
                    items,
                    validity,
                )?))
            }
            DataType::Int64 => {
                let offsets = arrow_cast::cast(&offsets, &DataType::Int64)?;
                let offsets_i64 = offsets.as_primitive::<Int64Type>();
                let offsets = OffsetBuffer::new(offsets_i64.values().clone());

                Ok(Arc::new(LargeListArray::try_new(
                    self.items_field.clone(),
                    offsets,
                    items,
                    validity,
                )?))
            }
            _ => panic!("ListDecodeTask with data type that is not i32 or i64"),
        }
    }
}

// Helper method that performs binary search.  However, once the
// target is found it walks past any duplicates.  E.g. if the
// input list is [0, 3, 5, 5, 5, 7] then this will only return
// 0, 1, 4, or 5.
fn binary_search_to_end(to_search: &[u64], target: u64) -> u64 {
    let mut result = match to_search.binary_search(&target) {
        Ok(idx) => idx,
        Err(idx) => idx - 1,
    };
    while result < (to_search.len() - 1) && to_search[result + 1] == target {
        result += 1;
    }
    result as u64
}

impl LogicalPageDecoder for ListPageDecoder {
    fn wait_for_loaded(&mut self, loaded_need: u64) -> BoxFuture<Result<()>> {
        async move {
            // wait for the indirect I/O to finish, run the scheduler for the indirect
            // I/O and then wait for enough items to arrive
            if self.unloaded.is_some() {
                trace!("List scheduler needs to wait for indirect I/O to complete");
                let indirectly_loaded = self.unloaded.take().unwrap().await;
                if indirectly_loaded.is_err() {
                    match indirectly_loaded.unwrap_err().try_into_panic() {
                        Ok(err) => std::panic::resume_unwind(err),
                        Err(err) => panic!("{:?}", err),
                    };
                }
                let indirectly_loaded = indirectly_loaded.unwrap()?;

                self.offsets = indirectly_loaded.offsets;
                self.validity = indirectly_loaded.validity;
                self.item_decoder = indirectly_loaded.root_decoder;
            }
            if self.rows_loaded > loaded_need {
                return Ok(());
            }

            let boundary = loaded_need as usize;
            debug_assert!(boundary < self.num_rows as usize);
            // We need more than X lists which means we need at least X+1 lists which means
            // we need at least offsets[X+1] items which means we need more than offsets[X+1]-1 items.
            let items_needed = self.offsets[boundary + 1].saturating_sub(1);
            trace!(
                "List decoder is waiting for more than {} rows to be loaded and {}/{} are already loaded.  To satisfy this we need more than {} loaded items",
                loaded_need,
                self.rows_loaded,
                self.num_rows,
                items_needed,
            );

            let items_loaded = if let Some(item_decoder) = self.item_decoder.as_mut() {
                item_decoder.wait_for_loaded(items_needed).await?;
                item_decoder.rows_loaded()
            } else {
                0
            };

            self.rows_loaded = binary_search_to_end(&self.offsets, items_loaded);
            trace!("List decoder now has {} loaded rows", self.rows_loaded);

            Ok(())
        }
        .boxed()
    }

    fn drain(&mut self, num_rows: u64) -> Result<NextDecodeTask> {
        // We already have the offsets but need to drain the item pages
        let mut actual_num_rows = num_rows;
        let item_start = self.offsets[self.rows_drained as usize];
        if self.offset_type != DataType::Int64 {
            // We might not be able to drain `num_rows` because that request might contain more than 2^31 items
            // so we need to figure out how many rows we can actually drain.
            while actual_num_rows > 0 {
                let num_items =
                    self.offsets[(self.rows_drained + actual_num_rows) as usize] - item_start;
                if num_items <= i32::MAX as u64 {
                    break;
                }
                // TODO: This could be slow.  Maybe faster to start from zero or do binary search.  Investigate when
                // actually adding support for smaller than requested batches
                actual_num_rows -= 1;
            }
        }
        if actual_num_rows < num_rows {
            // TODO: We should be able to automatically
            // shrink the read batch size if we detect the batches are going to be huge (maybe
            // even achieve this with a read_batch_bytes parameter, though some estimation may
            // still be required)
            return Err(Error::NotSupported { source: format!("loading a batch of {} lists would require creating an array with over i32::MAX items and we don't yet support returning smaller than requested batches", num_rows).into(), location: location!() });
        }
        let offsets = self.offsets
            [self.rows_drained as usize..(self.rows_drained + actual_num_rows + 1) as usize]
            .to_vec();
        let validity = self
            .validity
            .slice(self.rows_drained as usize, actual_num_rows as usize);
        let start = offsets[0];
        let end = offsets[offsets.len() - 1];
        let num_items_to_drain = end - start;

        let item_decode = if num_items_to_drain == 0 {
            None
        } else {
            self.item_decoder
                .as_mut()
                .map(|item_decoder| Result::Ok(item_decoder.drain(num_items_to_drain)?.task))
                .transpose()?
        };

        self.rows_drained += num_rows;
        let has_more = self.rows_left() > 0;
        Ok(NextDecodeTask {
            has_more,
            num_rows,
            task: Box::new(ListDecodeTask {
                offsets,
                validity,
                items_field: self.items_field.clone(),
                items: item_decode,
                offset_type: self.offset_type.clone(),
            }) as Box<dyn DecodeArrayTask>,
        })
    }

    fn num_rows(&self) -> u64 {
        self.num_rows
    }

    fn rows_loaded(&self) -> u64 {
        self.rows_loaded
    }

    fn rows_drained(&self) -> u64 {
        self.rows_drained
    }

    fn data_type(&self) -> &DataType {
        &self.data_type
    }
}

struct IndirectlyLoaded {
    offsets: Arc<[u64]>,
    validity: BooleanBuffer,
    root_decoder: Option<SimpleStructDecoder>,
}

impl std::fmt::Debug for IndirectlyLoaded {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("IndirectlyLoaded")
            .field("offsets", &self.offsets)
            .field("validity", &self.validity)
            .finish()
    }
}

/// An encoder for list offsets that "stitches" offsets and encodes nulls into the offsets
///
/// If we need to encode several list arrays into a single page then we need to "stitch" the offsets
/// For example, imagine we have list arrays [[0, 1], [2]] and [[3, 4, 5]].
///
/// We will have offset arrays [0, 2, 3] and [0, 3].  We don't want to encode [0, 2, 3, 0, 3].  What
/// we want is [0, 2, 3, 6]
///
/// This encoder also handles validity by converting a null value into an oversized offset.  For example,
/// if we have four lists with offsets [0, 20, 20, 20, 30] and the list at index 2 is null (note that
/// the list at index 1 is empty) then we turn this into offsets [0, 20, 20, 51, 30].  We replace a null
/// offset with previous_offset + max_offset + 1.  This makes it possible to load a single item from the
/// list array.
///
/// These offsets are always stored on disk as a u64 array.  First, this is because its simply much more
/// likely than one expects that this is needed, even if our lists are not massive.  This is because we
/// only write an offsets page when we have enough data.  This means we will probably accumulate a million
/// offsets or more before we bother to write a page. If our lists have a few thousand items a piece then
/// we end up passing the u32::MAX boundary.
///
/// The second reason is that list offsets are very easily compacted with delta + bit packing and so those
/// u64 offsets should easily be shrunk down before being put on disk.
///
/// This encoder can encode both lists and large lists.  It can decode the resulting column into either type
/// as well. (TODO: Test and enable large lists)
///
/// You can even write as a large list and decode as a regular list (as long as no single list has more than
/// 2^31 items) or vice versa.  You could even encode a mixed stream of list and large list (but unclear that
/// would ever be useful)
#[derive(Debug)]
struct ListOffsetsEncoder {
    // An accumulation queue, we insert both offset arrays and validity arrays into this queue
    accumulation_queue: AccumulationQueue,
    // The inner encoder of offset values
    inner_encoder: Arc<dyn ArrayEncoder>,
    column_index: u32,
}

impl ListOffsetsEncoder {
    fn new(
        cache_bytes: u64,
        keep_original_array: bool,
        column_index: u32,
        inner_encoder: Arc<dyn ArrayEncoder>,
    ) -> Self {
        Self {
            accumulation_queue: AccumulationQueue::new(
                cache_bytes,
                column_index,
                keep_original_array,
            ),
            inner_encoder,
            column_index,
        }
    }

    /// Given a list array, return the offsets as a standalone ArrayRef (either an Int32Array or Int64Array)
    fn extract_offsets(list_arr: &dyn Array) -> ArrayRef {
        match list_arr.data_type() {
            DataType::List(_) => {
                let offsets = list_arr.as_list::<i32>().offsets().clone();
                Arc::new(Int32Array::new(offsets.into_inner(), None))
            }
            DataType::LargeList(_) => {
                let offsets = list_arr.as_list::<i64>().offsets().clone();
                Arc::new(Int64Array::new(offsets.into_inner(), None))
            }
            _ => panic!(),
        }
    }

    /// Converts the validity of a list array into a boolean array.  If there is no validity information
    /// then this is an empty boolean array.
    fn extract_validity(list_arr: &dyn Array) -> ArrayRef {
        if let Some(validity) = list_arr.nulls() {
            Arc::new(BooleanArray::new(validity.inner().clone(), None))
        } else {
            // We convert None validity into an empty array because the accumulation queue can't
            // handle Option<ArrayRef>
            new_empty_array(&DataType::Boolean)
        }
    }

    fn make_encode_task(&self, arrays: Vec<ArrayRef>) -> EncodeTask {
        let inner_encoder = self.inner_encoder.clone();
        let column_idx = self.column_index;
        // At this point we should have 2*N arrays where the even-indexed arrays are integer offsets
        // and the odd-indexed arrays are boolean validity bitmaps
        let offset_arrays = arrays.iter().step_by(2).cloned().collect::<Vec<_>>();
        let validity_arrays = arrays.into_iter().skip(1).step_by(2).collect::<Vec<_>>();

        tokio::task::spawn(async move {
            let num_rows =
                offset_arrays.iter().map(|arr| arr.len()).sum::<usize>() - offset_arrays.len();
            let num_rows = num_rows as u64;
            let mut buffer_index = 0;
            let array = Self::do_encode(
                offset_arrays,
                validity_arrays,
                &mut buffer_index,
                num_rows,
                inner_encoder,
            )?;
            let (data, description) = array.into_buffers();
            Ok(EncodedPage {
                data,
                description: PageEncoding::Legacy(description),
                num_rows,
                column_idx,
                row_number: 0, // Legacy encoders do not use
            })
        })
        .map(|res_res| res_res.unwrap())
        .boxed()
    }

    fn maybe_encode_offsets_and_validity(&mut self, list_arr: &dyn Array) -> Option<EncodeTask> {
        let offsets = Self::extract_offsets(list_arr);
        let validity = Self::extract_validity(list_arr);
        let num_rows = offsets.len() as u64;
        // Either inserting the offsets OR inserting the validity could cause the
        // accumulation queue to fill up
        if let Some(mut arrays) = self
            .accumulation_queue
            .insert(offsets, /*row_number=*/ 0, num_rows)
        {
            arrays.0.push(validity);
            Some(self.make_encode_task(arrays.0))
        } else if let Some(arrays) = self
            .accumulation_queue
            .insert(validity, /*row_number=*/ 0, num_rows)
        {
            Some(self.make_encode_task(arrays.0))
        } else {
            None
        }
    }

    fn flush(&mut self) -> Option<EncodeTask> {
        if let Some(arrays) = self.accumulation_queue.flush() {
            Some(self.make_encode_task(arrays.0))
        } else {
            None
        }
    }

    // Get's the total number of items covered by an array of offsets (keeping in
    // mind that the first offset may not be zero)
    fn get_offset_span(array: &dyn Array) -> u64 {
        match array.data_type() {
            DataType::Int32 => {
                let arr_i32 = array.as_primitive::<Int32Type>();
                (arr_i32.value(arr_i32.len() - 1) - arr_i32.value(0)) as u64
            }
            DataType::Int64 => {
                let arr_i64 = array.as_primitive::<Int64Type>();
                (arr_i64.value(arr_i64.len() - 1) - arr_i64.value(0)) as u64
            }
            _ => panic!(),
        }
    }

    // This is where we do the work to actually shift the offsets and encode nulls
    // Note that the output is u64 and the input could be i32 OR i64.
    fn extend_offsets_vec_u64(
        dest: &mut Vec<u64>,
        offsets: &dyn Array,
        validity: Option<&BooleanArray>,
        // The offset of this list into the destination
        base: u64,
        null_offset_adjustment: u64,
    ) {
        match offsets.data_type() {
            DataType::Int32 => {
                let offsets_i32 = offsets.as_primitive::<Int32Type>();
                let start = offsets_i32.value(0) as u64;
                // If we want to take a list from start..X and change it into
                // a list from end..X then we need to add (base - start) to all elements
                // Note that `modifier` may be negative but (item + modifier) will always be >= 0
                let modifier = base as i64 - start as i64;
                if let Some(validity) = validity {
                    dest.extend(
                        offsets_i32
                            .values()
                            .iter()
                            .skip(1)
                            .zip(validity.values().iter())
                            .map(|(&off, valid)| {
                                (off as i64 + modifier) as u64
                                    + (!valid as u64 * null_offset_adjustment)
                            }),
                    );
                } else {
                    dest.extend(
                        offsets_i32
                            .values()
                            .iter()
                            .skip(1)
                            // Subtract by `start` so offsets start at 0
                            .map(|&v| (v as i64 + modifier) as u64),
                    );
                }
            }
            DataType::Int64 => {
                let offsets_i64 = offsets.as_primitive::<Int64Type>();
                let start = offsets_i64.value(0) as u64;
                // If we want to take a list from start..X and change it into
                // a list from end..X then we need to add (base - start) to all elements
                // Note that `modifier` may be negative but (item + modifier) will always be >= 0
                let modifier = base as i64 - start as i64;
                if let Some(validity) = validity {
                    dest.extend(
                        offsets_i64
                            .values()
                            .iter()
                            .skip(1)
                            .zip(validity.values().iter())
                            .map(|(&off, valid)| {
                                (off + modifier) as u64 + (!valid as u64 * null_offset_adjustment)
                            }),
                    )
                } else {
                    dest.extend(
                        offsets_i64
                            .values()
                            .iter()
                            .skip(1)
                            .map(|&v| (v + modifier) as u64),
                    );
                }
            }
            _ => panic!("Invalid list offsets data type {:?}", offsets.data_type()),
        }
    }

    fn do_encode_u64(
        offset_arrays: Vec<ArrayRef>,
        validity: Vec<Option<&BooleanArray>>,
        num_offsets: u64,
        null_offset_adjustment: u64,
        buffer_index: &mut u32,
        inner_encoder: Arc<dyn ArrayEncoder>,
    ) -> Result<EncodedArray> {
        let mut offsets = Vec::with_capacity(num_offsets as usize);
        for (offsets_arr, validity_arr) in offset_arrays.iter().zip(validity) {
            let last_prev_offset = offsets.last().copied().unwrap_or(0) % null_offset_adjustment;
            Self::extend_offsets_vec_u64(
                &mut offsets,
                &offsets_arr,
                validity_arr,
                last_prev_offset,
                null_offset_adjustment,
            );
        }
        let offsets_data = DataBlock::FixedWidth(FixedWidthDataBlock {
            bits_per_value: 64,
            data: LanceBuffer::reinterpret_vec(offsets),
            num_values: num_offsets,
            block_info: BlockInfo::new(),
        });
        inner_encoder.encode(offsets_data, &DataType::UInt64, buffer_index)
    }

    fn do_encode(
        offset_arrays: Vec<ArrayRef>,
        validity_arrays: Vec<ArrayRef>,
        buffer_index: &mut u32,
        num_offsets: u64,
        inner_encoder: Arc<dyn ArrayEncoder>,
    ) -> Result<EncodedArray> {
        let validity_arrays = validity_arrays
            .iter()
            .map(|v| {
                if v.is_empty() {
                    None
                } else {
                    Some(v.as_boolean())
                }
            })
            .collect::<Vec<_>>();
        debug_assert_eq!(offset_arrays.len(), validity_arrays.len());
        let total_span = offset_arrays
            .iter()
            .map(|arr| Self::get_offset_span(arr.as_ref()))
            .sum::<u64>();
        // See encodings.proto for reasoning behind this value
        let null_offset_adjustment = total_span + 1;
        let encoded_offsets = Self::do_encode_u64(
            offset_arrays,
            validity_arrays,
            num_offsets,
            null_offset_adjustment,
            buffer_index,
            inner_encoder,
        )?;
        Ok(EncodedArray {
            data: encoded_offsets.data,
            encoding: pb::ArrayEncoding {
                array_encoding: Some(pb::array_encoding::ArrayEncoding::List(Box::new(
                    pb::List {
                        offsets: Some(Box::new(encoded_offsets.encoding)),
                        null_offset_adjustment,
                        num_items: total_span,
                    },
                ))),
            },
        })
    }
}

pub struct ListFieldEncoder {
    offsets_encoder: ListOffsetsEncoder,
    items_encoder: Box<dyn FieldEncoder>,
}

impl ListFieldEncoder {
    pub fn new(
        items_encoder: Box<dyn FieldEncoder>,
        inner_offsets_encoder: Arc<dyn ArrayEncoder>,
        cache_bytes_per_columns: u64,
        keep_original_array: bool,
        column_index: u32,
    ) -> Self {
        Self {
            offsets_encoder: ListOffsetsEncoder::new(
                cache_bytes_per_columns,
                keep_original_array,
                column_index,
                inner_offsets_encoder,
            ),
            items_encoder,
        }
    }

    fn combine_tasks(
        offsets_tasks: Vec<EncodeTask>,
        item_tasks: Vec<EncodeTask>,
    ) -> Result<Vec<EncodeTask>> {
        let mut all_tasks = offsets_tasks;
        let item_tasks = item_tasks;
        all_tasks.extend(item_tasks);
        Ok(all_tasks)
    }
}

impl FieldEncoder for ListFieldEncoder {
    fn maybe_encode(
        &mut self,
        array: ArrayRef,
        external_buffers: &mut OutOfLineBuffers,
        repdef: RepDefBuilder,
        row_number: u64,
        num_rows: u64,
    ) -> Result<Vec<EncodeTask>> {
        // The list may have an offset / shorter length which means the underlying
        // values array could be longer than what we need to encode and so we need
        // to slice down to the region of interest.
        let items = match array.data_type() {
            DataType::List(_) => {
                let list_arr = array.as_list::<i32>();
                let items_start = list_arr.value_offsets()[list_arr.offset()] as usize;
                let items_end =
                    list_arr.value_offsets()[list_arr.offset() + list_arr.len()] as usize;
                list_arr
                    .values()
                    .slice(items_start, items_end - items_start)
            }
            DataType::LargeList(_) => {
                let list_arr = array.as_list::<i64>();
                let items_start = list_arr.value_offsets()[list_arr.offset()] as usize;
                let items_end =
                    list_arr.value_offsets()[list_arr.offset() + list_arr.len()] as usize;
                list_arr
                    .values()
                    .slice(items_start, items_end - items_start)
            }
            _ => panic!(),
        };
        let offsets_tasks = self
            .offsets_encoder
            .maybe_encode_offsets_and_validity(array.as_ref())
            .map(|task| vec![task])
            .unwrap_or_default();
        let mut item_tasks = self.items_encoder.maybe_encode(
            items,
            external_buffers,
            repdef,
            row_number,
            num_rows,
        )?;
        if !offsets_tasks.is_empty() && item_tasks.is_empty() {
            // An items page cannot currently be shared by two different offsets pages.  This is
            // a limitation in the current scheduler and could be addressed in the future.  As a result
            // we always need to encode the items page if we encode the offsets page.
            //
            // In practice this isn't usually too bad unless we are targeting very small pages.
            item_tasks = self.items_encoder.flush(external_buffers)?;
        }
        Self::combine_tasks(offsets_tasks, item_tasks)
    }

    fn flush(&mut self, external_buffers: &mut OutOfLineBuffers) -> Result<Vec<EncodeTask>> {
        let offsets_tasks = self
            .offsets_encoder
            .flush()
            .map(|task| vec![task])
            .unwrap_or_default();
        let item_tasks = self.items_encoder.flush(external_buffers)?;
        Self::combine_tasks(offsets_tasks, item_tasks)
    }

    fn num_columns(&self) -> u32 {
        self.items_encoder.num_columns() + 1
    }

    fn finish(
        &mut self,
        external_buffers: &mut OutOfLineBuffers,
    ) -> BoxFuture<'_, Result<Vec<EncodedColumn>>> {
        let inner_columns = self.items_encoder.finish(external_buffers);
        async move {
            let mut columns = vec![EncodedColumn::default()];
            let inner_columns = inner_columns.await?;
            columns.extend(inner_columns);
            Ok(columns)
        }
        .boxed()
    }
}

/// A structural encoder for list fields
///
/// The list's offsets are added to the rep/def builder
/// and the list array's values are passed to the child encoder
///
/// The values will have any garbage values removed and will be trimmed
/// to only include the values that are actually used.
pub struct ListStructuralEncoder {
    child: Box<dyn FieldEncoder>,
}

impl ListStructuralEncoder {
    pub fn new(child: Box<dyn FieldEncoder>) -> Self {
        Self { child }
    }
}

impl FieldEncoder for ListStructuralEncoder {
    fn maybe_encode(
        &mut self,
        array: ArrayRef,
        external_buffers: &mut OutOfLineBuffers,
        mut repdef: RepDefBuilder,
        row_number: u64,
        num_rows: u64,
    ) -> Result<Vec<EncodeTask>> {
        let values = if let Some(list_arr) = array.as_list_opt::<i32>() {
            let has_garbage_values =
                repdef.add_offsets(list_arr.offsets().clone(), array.nulls().cloned());
            if has_garbage_values {
                list_arr.filter_garbage_nulls().trimmed_values()
            } else {
                list_arr.trimmed_values()
            }
        } else if let Some(list_arr) = array.as_list_opt::<i64>() {
            let has_garbage_values =
                repdef.add_offsets(list_arr.offsets().clone(), array.nulls().cloned());
            if has_garbage_values {
                list_arr.filter_garbage_nulls().trimmed_values()
            } else {
                list_arr.trimmed_values()
            }
        } else {
            panic!("List encoder used for non-list data")
        };
        self.child
            .maybe_encode(values, external_buffers, repdef, row_number, num_rows)
    }

    fn flush(&mut self, external_buffers: &mut OutOfLineBuffers) -> Result<Vec<EncodeTask>> {
        self.child.flush(external_buffers)
    }

    fn num_columns(&self) -> u32 {
        self.child.num_columns()
    }

    fn finish(
        &mut self,
        external_buffers: &mut OutOfLineBuffers,
    ) -> BoxFuture<'_, Result<Vec<crate::encoder::EncodedColumn>>> {
        self.child.finish(external_buffers)
    }
}

#[derive(Debug)]
pub struct StructuralListScheduler {
    child: Box<dyn StructuralFieldScheduler>,
}

impl StructuralListScheduler {
    pub fn new(child: Box<dyn StructuralFieldScheduler>) -> Self {
        Self { child }
    }
}

impl StructuralFieldScheduler for StructuralListScheduler {
    fn schedule_ranges<'a>(
        &'a self,
        ranges: &[Range<u64>],
        filter: &FilterExpression,
    ) -> Result<Box<dyn StructuralSchedulingJob + 'a>> {
        let child = self.child.schedule_ranges(ranges, filter)?;

        Ok(Box::new(StructuralListSchedulingJob::new(child)))
    }

    fn initialize<'a>(
        &'a mut self,
        filter: &'a FilterExpression,
        context: &'a SchedulerContext,
    ) -> BoxFuture<'a, Result<()>> {
        self.child.initialize(filter, context)
    }
}

/// Scheduling job for list data
///
/// Scheduling is handled by the primitive encoder and nothing special
/// happens here.
#[derive(Debug)]
struct StructuralListSchedulingJob<'a> {
    child: Box<dyn StructuralSchedulingJob + 'a>,
}

impl<'a> StructuralListSchedulingJob<'a> {
    fn new(child: Box<dyn StructuralSchedulingJob + 'a>) -> Self {
        Self { child }
    }
}

impl StructuralSchedulingJob for StructuralListSchedulingJob<'_> {
    fn schedule_next(
        &mut self,
        context: &mut SchedulerContext,
    ) -> Result<Option<ScheduledScanLine>> {
        self.child.schedule_next(context)
    }
}

#[derive(Debug)]
pub struct StructuralListDecoder {
    child: Box<dyn StructuralFieldDecoder>,
    data_type: DataType,
}

impl StructuralListDecoder {
    pub fn new(child: Box<dyn StructuralFieldDecoder>, data_type: DataType) -> Self {
        Self { child, data_type }
    }
}

impl StructuralFieldDecoder for StructuralListDecoder {
    fn accept_page(&mut self, child: crate::decoder::LoadedPage) -> Result<()> {
        self.child.accept_page(child)
    }

    fn drain(&mut self, num_rows: u64) -> Result<Box<dyn StructuralDecodeArrayTask>> {
        let child_task = self.child.drain(num_rows)?;
        Ok(Box::new(StructuralListDecodeTask::new(
            child_task,
            self.data_type.clone(),
        )))
    }

    fn data_type(&self) -> &DataType {
        &self.data_type
    }
}

#[derive(Debug)]
struct StructuralListDecodeTask {
    child_task: Box<dyn StructuralDecodeArrayTask>,
    data_type: DataType,
}

impl StructuralListDecodeTask {
    fn new(child_task: Box<dyn StructuralDecodeArrayTask>, data_type: DataType) -> Self {
        Self {
            child_task,
            data_type,
        }
    }
}

impl StructuralDecodeArrayTask for StructuralListDecodeTask {
    fn decode(self: Box<Self>) -> Result<DecodedArray> {
        let DecodedArray { array, mut repdef } = self.child_task.decode()?;
        match &self.data_type {
            DataType::List(child_field) => {
                let (offsets, validity) = repdef.unravel_offsets::<i32>()?;
                let list_array = ListArray::try_new(child_field.clone(), offsets, array, validity)?;
                Ok(DecodedArray {
                    array: Arc::new(list_array),
                    repdef,
                })
            }
            DataType::LargeList(child_field) => {
                let (offsets, validity) = repdef.unravel_offsets::<i64>()?;
                let list_array =
                    LargeListArray::try_new(child_field.clone(), offsets, array, validity)?;
                Ok(DecodedArray {
                    array: Arc::new(list_array),
                    repdef,
                })
            }
            _ => panic!("List decoder did not have a list field"),
        }
    }
}

#[cfg(test)]
mod tests {

    use std::{collections::HashMap, sync::Arc};

    use arrow::array::{Int64Builder, LargeListBuilder, StringBuilder};
    use arrow_array::{
        builder::{Int32Builder, ListBuilder},
        Array, ArrayRef, BooleanArray, ListArray, StructArray, UInt64Array,
    };
    use arrow_buffer::{BooleanBuffer, NullBuffer, OffsetBuffer, ScalarBuffer};
    use arrow_schema::{DataType, Field, Fields};
    use rstest::rstest;

    use crate::{
        testing::{check_round_trip_encoding_of_data, check_round_trip_encoding_random, TestCases},
        version::LanceFileVersion,
    };

    fn make_list_type(inner_type: DataType) -> DataType {
        DataType::List(Arc::new(Field::new("item", inner_type, true)))
    }

    fn make_large_list_type(inner_type: DataType) -> DataType {
        DataType::LargeList(Arc::new(Field::new("item", inner_type, true)))
    }

    #[rstest]
    #[test_log::test(tokio::test)]
    async fn test_list(
        #[values(LanceFileVersion::V2_0, LanceFileVersion::V2_1)] version: LanceFileVersion,
    ) {
        let field = Field::new("", make_list_type(DataType::Int32), true);
        check_round_trip_encoding_random(field, version).await;
    }

    #[test_log::test(tokio::test)]
    async fn test_large_list() {
        let field = Field::new("", make_large_list_type(DataType::Int32), true);
        check_round_trip_encoding_random(field, LanceFileVersion::V2_0).await;
    }

    #[test_log::test(tokio::test)]
    async fn test_nested_strings() {
        let field = Field::new("", make_list_type(DataType::Utf8), true);
        check_round_trip_encoding_random(field, LanceFileVersion::V2_0).await;
    }

    #[test_log::test(tokio::test)]
    async fn test_nested_list() {
        let field = Field::new("", make_list_type(make_list_type(DataType::Int32)), true);
        check_round_trip_encoding_random(field, LanceFileVersion::V2_0).await;
    }

    #[test_log::test(tokio::test)]
    async fn test_list_struct_list() {
        let struct_type = DataType::Struct(Fields::from(vec![Field::new(
            "inner_str",
            DataType::Utf8,
            false,
        )]));

        let field = Field::new("", make_list_type(struct_type), true);
        check_round_trip_encoding_random(field, LanceFileVersion::V2_0).await;
    }

    #[test_log::test(tokio::test)]
    async fn test_list_struct_empty() {
        let fields = Fields::from(vec![Field::new("inner", DataType::UInt64, true)]);
        let items = UInt64Array::from(Vec::<u64>::new());
        let structs = StructArray::new(fields, vec![Arc::new(items)], None);
        let offsets = OffsetBuffer::new(ScalarBuffer::<i32>::from(vec![0; 2 * 1024 * 1024 + 1]));
        let lists = ListArray::new(
            Arc::new(Field::new("item", structs.data_type().clone(), true)),
            offsets,
            Arc::new(structs),
            None,
        );

        check_round_trip_encoding_of_data(
            vec![Arc::new(lists)],
            &TestCases::default(),
            HashMap::new(),
        )
        .await;
    }

    #[rstest]
    #[test_log::test(tokio::test)]
    async fn test_simple_list(
        #[values(LanceFileVersion::V2_0, LanceFileVersion::V2_1)] version: LanceFileVersion,
    ) {
        let items_builder = Int32Builder::new();
        let mut list_builder = ListBuilder::new(items_builder);
        list_builder.append_value([Some(1), Some(2), Some(3)]);
        list_builder.append_value([Some(4), Some(5)]);
        list_builder.append_null();
        list_builder.append_value([Some(6), Some(7), Some(8)]);
        let list_array = list_builder.finish();

        let test_cases = TestCases::default()
            .with_range(0..2)
            .with_range(0..3)
            .with_range(1..3)
            .with_indices(vec![1, 3])
            .with_indices(vec![2])
            .with_file_version(version);
        check_round_trip_encoding_of_data(vec![Arc::new(list_array)], &test_cases, HashMap::new())
            .await;
    }

    #[rstest]
    #[test_log::test(tokio::test)]
    async fn test_simple_sliced_list() {
        let items_builder = Int32Builder::new();
        let mut list_builder = ListBuilder::new(items_builder);
        list_builder.append_value([Some(1), Some(2), Some(3)]);
        list_builder.append_value([Some(4), Some(5)]);
        list_builder.append_null();
        list_builder.append_value([Some(6), Some(7), Some(8)]);
        let list_array = list_builder.finish();

        let list_array = list_array.slice(1, 2);

        let test_cases = TestCases::default()
            .with_range(0..2)
            .with_range(1..2)
            .with_indices(vec![0])
            .with_indices(vec![1])
            .with_file_version(LanceFileVersion::V2_1);
        check_round_trip_encoding_of_data(vec![Arc::new(list_array)], &test_cases, HashMap::new())
            .await;
    }

    #[rstest]
    #[test_log::test(tokio::test)]
    async fn test_list_with_garbage_nulls() {
        // In Arrow, list nulls are allowed to be non-empty, with masked garbage values
        // Here we make a list with a null row in the middle with 3 garbage values
        let items = UInt64Array::from(vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]);
        let offsets = ScalarBuffer::<i32>::from(vec![0, 5, 8, 10]);
        let offsets = OffsetBuffer::new(offsets);
        let list_validity = NullBuffer::new(BooleanBuffer::from(vec![true, false, true]));
        let list_arr = ListArray::new(
            Arc::new(Field::new("item", DataType::UInt64, true)),
            offsets,
            Arc::new(items),
            Some(list_validity),
        );

        let test_cases = TestCases::default()
            .with_range(0..3)
            .with_range(1..2)
            .with_indices(vec![1])
            .with_indices(vec![2])
            .with_file_version(LanceFileVersion::V2_1);
        check_round_trip_encoding_of_data(vec![Arc::new(list_arr)], &test_cases, HashMap::new())
            .await;
    }

    #[test_log::test(tokio::test)]
    async fn test_simple_two_page_list() {
        // This is a simple pre-defined list that spans two pages.  This test is useful for
        // debugging the repetition index
        let items_builder = Int64Builder::new();
        let mut list_builder = ListBuilder::new(items_builder);
        for i in 0..512 {
            list_builder.append_value([Some(i), Some(i * 2)]);
        }
        let list_array_1 = list_builder.finish();

        let items_builder = Int64Builder::new();
        let mut list_builder = ListBuilder::new(items_builder);
        for i in 0..512 {
            let i = i + 512;
            list_builder.append_value([Some(i), Some(i * 2)]);
        }
        let list_array_2 = list_builder.finish();

        let test_cases = TestCases::default()
            .with_file_version(LanceFileVersion::V2_1)
            .with_page_sizes(vec![100])
            .with_range(800..900);
        check_round_trip_encoding_of_data(
            vec![Arc::new(list_array_1), Arc::new(list_array_2)],
            &test_cases,
            HashMap::new(),
        )
        .await;
    }

    #[test_log::test(tokio::test)]
    async fn test_simple_large_list() {
        let items_builder = Int32Builder::new();
        let mut list_builder = LargeListBuilder::new(items_builder);
        list_builder.append_value([Some(1), Some(2), Some(3)]);
        list_builder.append_value([Some(4), Some(5)]);
        list_builder.append_null();
        list_builder.append_value([Some(6), Some(7), Some(8)]);
        let list_array = list_builder.finish();

        let test_cases = TestCases::default()
            .with_range(0..2)
            .with_range(0..3)
            .with_range(1..3)
            .with_indices(vec![1, 3]);
        check_round_trip_encoding_of_data(vec![Arc::new(list_array)], &test_cases, HashMap::new())
            .await;
    }

    #[test_log::test(tokio::test)]
    async fn test_empty_lists() {
        // Scenario 1: Some lists are empty

        let values = [vec![Some(1), Some(2), Some(3)], vec![], vec![None]];
        // Test empty list at beginning, middle, and end
        for order in [[0, 1, 2], [1, 0, 2], [2, 0, 1]] {
            let items_builder = Int32Builder::new();
            let mut list_builder = ListBuilder::new(items_builder);
            for idx in order {
                list_builder.append_value(values[idx].clone());
            }
            let list_array = Arc::new(list_builder.finish());
            let test_cases = TestCases::default()
                .with_indices(vec![1])
                .with_indices(vec![0])
                .with_indices(vec![2])
                .with_indices(vec![0, 1]);
            check_round_trip_encoding_of_data(
                vec![list_array.clone()],
                &test_cases,
                HashMap::new(),
            )
            .await;
            let test_cases = test_cases.with_batch_size(1);
            check_round_trip_encoding_of_data(vec![list_array], &test_cases, HashMap::new()).await;
        }

        // Scenario 2: All lists are empty

        // When encoding a list of empty lists there are no items to encode
        // which is strange and we want to ensure we handle it
        let items_builder = Int32Builder::new();
        let mut list_builder = ListBuilder::new(items_builder);
        list_builder.append(true);
        list_builder.append_null();
        list_builder.append(true);
        let list_array = Arc::new(list_builder.finish());

        let test_cases = TestCases::default().with_range(0..2).with_indices(vec![1]);
        check_round_trip_encoding_of_data(vec![list_array.clone()], &test_cases, HashMap::new())
            .await;
        let test_cases = test_cases.with_batch_size(1);
        check_round_trip_encoding_of_data(vec![list_array], &test_cases, HashMap::new()).await;

        // Scenario 2: All lists are empty (but now with strings)

        // When encoding a list of empty lists there are no items to encode
        // which is strange and we want to ensure we handle it
        let items_builder = StringBuilder::new();
        let mut list_builder = ListBuilder::new(items_builder);
        list_builder.append(true);
        list_builder.append_null();
        list_builder.append(true);
        let list_array = Arc::new(list_builder.finish());

        let test_cases = TestCases::default().with_range(0..2).with_indices(vec![1]);
        check_round_trip_encoding_of_data(vec![list_array.clone()], &test_cases, HashMap::new())
            .await;
        let test_cases = test_cases.with_batch_size(1);
        check_round_trip_encoding_of_data(vec![list_array], &test_cases, HashMap::new()).await;
    }

    #[test_log::test(tokio::test)]
    #[ignore] // This test is quite slow in debug mode
    async fn test_jumbo_list() {
        // This is an overflow test.  We have a list of lists where each list
        // has 1Mi items.  We encode 5000 of these lists and so we have over 4Gi in the
        // offsets range
        let items = BooleanArray::new_null(1024 * 1024);
        let offsets = OffsetBuffer::new(ScalarBuffer::from(vec![0, 1024 * 1024]));
        let list_arr = Arc::new(ListArray::new(
            Arc::new(Field::new("item", DataType::Boolean, true)),
            offsets,
            Arc::new(items),
            None,
        )) as ArrayRef;
        let arrs = vec![list_arr; 5000];

        // We can't validate because our validation relies on concatenating all input arrays
        let test_cases = TestCases::default().without_validation();
        check_round_trip_encoding_of_data(arrs, &test_cases, HashMap::new()).await;
    }
}