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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

//! Utilities and traits for scheduling & decoding data
//!
//! Reading data involves two steps: scheduling and decoding.  The
//! scheduling step is responsible for figuring out what data is needed
//! and issuing the appropriate I/O requests.  The decoding step is
//! responsible for taking the loaded data and turning it into Arrow
//! arrays.
//!
//! # Scheduling
//!
//! Scheduling is split into [`self::FieldScheduler`] and [`self::PageScheduler`].
//! There is one field scheduler for each output field, which may map to many
//! columns of actual data.  A field scheduler is responsible for figuring out
//! the order in which pages should be scheduled.  Field schedulers then delegate
//! to page schedulers to figure out the I/O requests that need to be made for
//! the page.
//!
//! Page schedulers also create the decoders that will be used to decode the
//! scheduled data.
//!
//! # Decoding
//!
//! Decoders are split into [`self::PhysicalPageDecoder`] and
//! [`self::LogicalPageDecoder`].  Note that both physical and logical decoding
//! happens on a per-page basis.  There is no concept of a "field decoder" or
//! "column decoder".
//!
//! The physical decoders handle lower level encodings.  They have a few advantages:
//!
//!  * They do not need to decode into an Arrow array and so they don't need
//!    to be enveloped into the Arrow filesystem (e.g. Arrow doesn't have a
//!    bit-packed type.  We can use variable-length binary but that is kind
//!    of awkward)
//!  * They can decode into an existing allocation.  This can allow for "page
//!    bridging".  If we are trying to decode into a batch of 1024 rows and
//!    the rows 0..1024 are spread across two pages then we can avoid a memory
//!    copy by allocating once and decoding each page into the outer allocation.
//!    (note: page bridging is not actually implemented yet)
//!
//! However, there are some limitations for physical decoders:
//!
//!  * They are constrained to a single column
//!  * The API is more complex
//!
//! The logical decoders are designed to map one or more columns of Lance
//! data into an Arrow array.
//!
//! Typically, a "logical encoding" will have both a logical decoder and a field scheduler.
//! Meanwhile, a "physical encoding" will have a physical decoder but no corresponding field
//! scheduler.git add --all
//!
//!
//! # General notes
//!
//! Encodings are typically nested into each other to form a tree.  The top of the tree is
//! the user requested schema.  Each field in that schema is assigned to one top-level logical
//! encoding.  That encoding can then contain other logical encodings or physical encodings.
//! Physical encodings can also contain other physical encodings.
//!
//! So, for example, a single field in the Arrow schema might have the type List<UInt32>
//!
//! The encoding tree could then be:
//!
//! root: List (logical encoding)
//!  - indices: Primitive (logical encoding)
//!    - column: Basic (physical encoding)
//!      - validity: Bitmap (physical encoding)
//!      - values: RLE (physical encoding)
//!        - runs: Value (physical encoding)
//!        - values: Value (physical encoding)
//!  - items: Primitive (logical encoding)
//!    - column: Basic (physical encoding)
//!      - values: Value (phsyical encoding)
//!
//! Note that, in this example, root.items.column does not have a validity because there were
//! no nulls in the page.
//!
//! ## Multiple buffers or multiple columns?
//!
//! Note that there are many different ways we can write encodings.  For example, we might
//! store primitive fields in a single column with two buffers (one for validity and one for
//! values)
//!
//! On the other hand, we could also store a primitive field as two different columns.  One
//! that yields a non-nullable boolean array and one that yields a non-nullable array of items.
//! Then we could combine these two arrays into a single array where the boolean array is the
//! bitmap.  There are a few subtle differences between the approaches:
//!
//! * Storing things as multiple buffers within the same column is generally more efficient and
//!   easier to schedule.  For example, in-batch coalescing is very easy but can only be done
//!   on data that is in the same page.
//! * When things are stored in multiple columns you have to worry about their pages not being
//!   in sync.  In our previous validity / values example this means we might have to do some
//!   memory copies to get the validity array and values arrays to be the same length as
//!   decode.
//! * When things are stored in a single column, projection is impossible.  For example, if we
//!   tried to store all the struct fields in a single column with lots of buffers then we wouldn't
//!   be able to read back individual fields of the struct.
//!
//! The fixed size list decoding is an interesting example because it is actually both a physical
//! encoding and a logical encoding.  A fixed size list of a physical encoding is, itself, a physical
//! encoding (e.g. a fixed size list of doubles).  However, a fixed size list of a logical encoding
//! is a logical encoding (e.g. a fixed size list of structs).
//!
//! # The scheduling loop
//!
//! Reading a Lance file involves both scheduling and decoding.  Its generally expected that these
//! will run as two separate threads.
//!
//! ```text
//!
//!                                    I/O PARALLELISM
//!                       Issues
//!                       Requests   ┌─────────────────┐
//!                                  │                 │        Wait for
//!                       ┌──────────►   I/O Service   ├─────►  Enough I/O ◄─┐
//!                       │          │                 │        For batch    │
//!                       │          └─────────────────┘             │3      │
//!                       │                                          │       │
//!                       │                                          │       │2
//! ┌─────────────────────┴─┐                              ┌─────────▼───────┴┐
//! │                       │                              │                  │Poll
//! │       Batch Decode    │ Decode tasks sent via channel│   Batch Decode   │1
//! │       Scheduler       ├─────────────────────────────►│   Stream         ◄─────
//! │                       │                              │                  │
//! └─────▲─────────────┬───┘                              └─────────┬────────┘
//!       │             │                                            │4
//!       │             │                                            │
//!       └─────────────┘                                   ┌────────┴────────┐
//!  Caller of schedule_range                Buffer polling │                 │
//!  will be scheduler thread                to achieve CPU │ Decode Batch    ├────►
//!  and schedule one decode                 parallelism    │ Task            │
//!  task (and all needed I/O)               (thread per    │                 │
//!  per logical page                         batch)        └─────────────────┘
//! ```
//!
//! The scheduling thread will work through the file from the
//! start to the end as quickly as possible.  Data is scheduled one page at a time in a row-major
//! fashion.  For example, imagine we have a file with the following page structure:
//!
//! ```text
//! Score (Float32)     | C0P0 |
//! Id (16-byte UUID)   | C1P0 | C1P1 | C1P2 | C1P3 |
//! Vector (4096 bytes) | C2P0 | C2P1 | C2P2 | C2P3 | .. | C2P1024 |
//! ```
//!
//! This would be quite common as each of these pages has the same number of bytes.  Let's pretend
//! each page is 1MiB and so there are 256Ki rows of data.  Each page of `Score` has 256Ki rows.
//! Each page of `Id` has 64Ki rows.  Each page of `Vector` has 256 rows.  The scheduler would then
//! schedule in the following order:
//!
//! C0 P0
//! C1 P0
//! C2 P0
//! C2 P1
//! ... (254 pages omitted)
//! C2 P255
//! C1 P1
//! C2 P256
//! ... (254 pages omitted)
//! C2 P511
//! C1 P2
//! C2 P512
//! ... (254 pages omitted)
//! C2 P767
//! C1 P3
//! C2 P768
//! ... (254 pages omitted)
//! C2 P1024
//!
//! This is the ideal scheduling order because it means we can decode complete rows as quickly as possible.
//! Note that the scheduler thread does not need to wait for I/O to happen at any point.  As soon as it starts
//! it will start scheduling one page of I/O after another until it has scheduled the entire file's worth of
//! I/O.  This is slightly different than other file readers which have "row group parallelism" and will
//! typically only schedule X row groups worth of reads at a time.
//!
//! In the near future there will be a backpressure mechanism and so it may need to stop/pause if the compute
//! falls behind.
//!
//! ## Indirect I/O
//!
//! Regrettably, there are times where we cannot know exactly what data we need until we have partially decoded
//! the file.  This happens when we have variable sized list data.  In that case the scheduling task for that
//! page will only schedule the first part of the read (loading the list offsets).  It will then immediately
//! spawn a new tokio task to wait for that I/O and decode the list offsets.  That follow-up task is not part
//! of the scheduling loop or the decode loop.  It is a free task.  Once the list offsets are decoded we submit
//! a follow-up I/O task.  This task is scheduled at a high priority because the decoder is going to need it soon.
//!
//! # The decode loop
//!
//! As soon as the scheduler starts we can start decoding.  Each time we schedule a page we
//! push a decoder for that page's data into a channel.  The decode loop
//! ([`BatchDecodeStream`]) reads from that channel.  Each time it receives a decoder it
//! waits until the decoder has all of its data.  Then it grabs the next decoder.  Once it has
//! enough loaded decoders to complete a batch worth of rows it will spawn a "decode batch task".
//!
//! These batch decode tasks perform the actual CPU work of decoding the loaded data into Arrow
//! arrays.  This may involve signifciant CPU processing like decompression or arithmetic in order
//! to restore the data to its correct in-memory representation.
//!
//! ## Batch size
//!
//! The `BatchDecodeStream` is configured with a batch size.  This does not need to have any
//! relation to the page size(s) used to write the data.  This keeps our compute work completely
//! independent of our I/O work.  We suggest using small batch sizes:
//!
//!  * Batches should fit in CPU cache (at least L3)
//!  * More batches means more opportunity for parallelism
//!  * The "batch overhead" is very small in Lance compared to other formats because it has no
//!    relation to the way the data is stored.

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

use arrow_array::cast::AsArray;
use arrow_array::{ArrayRef, RecordBatch};
use arrow_schema::{DataType, Field as ArrowField, Fields, Schema as ArrowSchema};
use bytes::Bytes;
use futures::future::BoxFuture;
use futures::stream::BoxStream;
use futures::{FutureExt, StreamExt};
use lance_arrow::DataTypeExt;
use lance_core::datatypes::{Field, Schema};
use log::{trace, warn};
use snafu::{location, Location};
use tokio::sync::mpsc::{self, unbounded_channel};

use lance_core::{Error, Result};
use tracing::instrument;

use crate::data::DataBlock;
use crate::encoder::{values_column_encoding, EncodedBatch};
use crate::encodings::logical::list::{ListFieldScheduler, OffsetPageInfo};
use crate::encodings::logical::primitive::PrimitiveFieldScheduler;
use crate::encodings::logical::r#struct::{SimpleStructDecoder, SimpleStructScheduler};
use crate::encodings::physical::{ColumnBuffers, FileBuffers};
use crate::format::pb;
use crate::{BufferScheduler, EncodingsIo};

// If users are getting batches over 10MiB large then it's time to reduce the batch size
const BATCH_SIZE_BYTES_WARNING: u64 = 10 * 1024 * 1024;

/// Metadata describing a page in a file
///
/// This is typically created by reading the metadata section of a Lance file
#[derive(Debug)]
pub struct PageInfo {
    /// The number of rows in the page
    pub num_rows: u64,
    /// The encoding that explains the buffers in the page
    pub encoding: pb::ArrayEncoding,
    /// The offsets and sizes of the buffers in the file
    pub buffer_offsets_and_sizes: Arc<[(u64, u64)]>,
}

/// Metadata describing a column in a file
///
/// This is typically created by reading the metadata section of a Lance file
#[derive(Debug, Clone)]
pub struct ColumnInfo {
    /// The index of the column in the file
    pub index: u32,
    /// The metadata for each page in the column
    pub page_infos: Arc<[PageInfo]>,
    /// File positions and their sizes of the column-level buffers
    pub buffer_offsets_and_sizes: Arc<[(u64, u64)]>,
    pub encoding: pb::ColumnEncoding,
}

impl ColumnInfo {
    /// Create a new instance
    pub fn new(
        index: u32,
        page_infos: Arc<[PageInfo]>,
        buffer_offsets_and_sizes: Vec<(u64, u64)>,
        encoding: pb::ColumnEncoding,
    ) -> Self {
        Self {
            index,
            page_infos,
            buffer_offsets_and_sizes: buffer_offsets_and_sizes.into_boxed_slice().into(),
            encoding,
        }
    }
}

/// The scheduler for decoding batches
///
/// Lance decoding is done in two steps, scheduling, and decoding.  The
/// scheduling tends to be lightweight and should quickly figure what data
/// is needed from the disk issue the appropriate I/O requests.  A decode task is
/// created to eventually decode the data (once it is loaded) and scheduling
/// moves on to scheduling the next page.
///
/// Meanwhile, it's expected that a decode stream will be setup to run at the
/// same time.  Decode tasks take the data that is loaded and turn it into
/// Arrow arrays.
///
/// This approach allows us to keep our I/O parallelism and CPU parallelism
/// completely separate since those are often two very different values.
///
/// Backpressure should be achieved via the I/O service.  Requests that are
/// issued will pile up if the decode stream is not polling quickly enough.
/// The [`crate::EncodingsIo::submit_request`] function should return a pending
/// future once there are too many I/O requests in flight.
///
/// TODO: Implement backpressure
pub struct DecodeBatchScheduler {
    pub root_scheduler: Arc<dyn FieldScheduler>,
    pub root_fields: Fields,
}

/// Represents a series of decoder strategies
///
/// These strategies will be applied, in order, to determine
/// which decoder to use for a field.
#[derive(Debug, Clone)]
pub struct DecoderMiddlewareChain {
    chain: Vec<Arc<dyn FieldDecoderStrategy>>,
}

impl Default for DecoderMiddlewareChain {
    fn default() -> Self {
        Self {
            chain: Default::default(),
        }
        .add_strategy(Arc::new(CoreFieldDecoderStrategy::default()))
    }
}

impl DecoderMiddlewareChain {
    /// Creates an empty decoder chain
    pub fn new() -> Self {
        Self { chain: Vec::new() }
    }

    /// Adds a decoder to the end of the chain
    pub fn add_strategy(mut self, decoder: Arc<dyn FieldDecoderStrategy>) -> Self {
        self.chain.push(decoder);
        self
    }

    /// Obtain a cursor into the chain that can be used to create
    /// field schedulers
    pub(crate) fn cursor<'a>(
        &'a self,
        io: &'a Arc<dyn EncodingsIo>,
    ) -> DecoderMiddlewareChainCursor<'a> {
        DecoderMiddlewareChainCursor {
            chain: self,
            io,
            cur_idx: 0,
            path: VecDeque::new(),
        }
    }
}

/// A cursor into a decoder middleware chain
///
/// Each field scheduler is given a cursor during the create_field_scheduler
/// call.  This cursor can be used both to create child field schedulers and
/// to create a scheduler from an inner encoding.
pub struct DecoderMiddlewareChainCursor<'a> {
    chain: &'a DecoderMiddlewareChain,
    io: &'a Arc<dyn EncodingsIo>,
    path: VecDeque<u32>,
    cur_idx: usize,
}

pub type ChosenFieldScheduler<'a> = (
    DecoderMiddlewareChainCursor<'a>,
    Result<Arc<dyn FieldScheduler>>,
);

impl<'a> DecoderMiddlewareChainCursor<'a> {
    /// Returns the current path into the field being decoded
    pub fn current_path(&self) -> &VecDeque<u32> {
        &self.path
    }

    /// Returns the I/O service which can be used to grab column metadata
    pub fn io(&self) -> &Arc<dyn EncodingsIo> {
        self.io
    }

    /// Delegates responsibilty to the next encoder in the chain
    ///
    /// Field schedulers should call this method when:
    ///
    /// * They do not understand or handle the encoding
    /// * They wrap an encoding and want a scheduler for the inner encoding
    pub fn next(
        mut self,
        field: &Field,
        column_infos: &mut ColumnInfoIter,
        buffers: FileBuffers,
    ) -> Result<ChosenFieldScheduler<'a>> {
        if self.cur_idx >= self.chain.chain.len() {
            return Err(Error::invalid_input(
                format!(
                    "The user requested a field {:?} but no decoders were registered to handle it",
                    field
                ),
                location!(),
            ));
        }
        let item = &self.chain.chain[self.cur_idx];
        self.cur_idx += 1;
        item.create_field_scheduler(field, column_infos, buffers, self)
    }

    /// Restarts the decoder chain without creating a new "child"
    ///
    /// This can be useful, for example, when a field scheduler has
    /// an inner scheduler, and the current / parent strategies might
    /// apply to the inner scheduler.
    ///
    /// If the current / parent strategies should not be consulted
    /// then call [`Self::next`] instead.
    pub fn restart_at_current(
        mut self,
        field: &Field,
        column_infos: &mut ColumnInfoIter,
        buffers: FileBuffers,
    ) -> Result<ChosenFieldScheduler<'a>> {
        self.cur_idx = 0;
        self.next(field, column_infos, buffers)
    }

    /// Restarts the decoder chain for a new "child" field.  The main
    /// difference between this and [`Self::restart_at_current`] is that
    /// this method will modify [`Self::current_path`]
    pub fn new_child(
        mut self,
        child_idx: u32,
        field: &Field,
        column_infos: &mut ColumnInfoIter,
        buffers: FileBuffers,
    ) -> Result<ChosenFieldScheduler<'a>> {
        self.path.push_back(child_idx);
        self.cur_idx = 0;
        self.next(field, column_infos, buffers)
    }

    /// Starts the decoding process for a field
    pub(crate) fn start(
        mut self,
        field: &Field,
        column_infos: &mut ColumnInfoIter,
        buffers: FileBuffers,
    ) -> Result<ChosenFieldScheduler<'a>> {
        self.path.clear();
        self.cur_idx = 0;
        self.next(field, column_infos, buffers)
    }
}

pub struct ColumnInfoIter<'a> {
    column_infos: &'a [ColumnInfo],
    column_indices: &'a [u32],
    column_info_pos: usize,
    column_indices_pos: usize,
}

impl<'a> ColumnInfoIter<'a> {
    pub fn new(column_infos: &'a [ColumnInfo], column_indices: &'a [u32]) -> Self {
        let initial_pos = column_indices[0] as usize;
        Self {
            column_infos,
            column_indices,
            column_info_pos: initial_pos,
            column_indices_pos: 0,
        }
    }

    pub fn peek(&self) -> &'a ColumnInfo {
        &self.column_infos[self.column_info_pos]
    }

    pub(crate) fn next_top_level(&mut self) {
        self.column_indices_pos += 1;
        if self.column_indices_pos < self.column_indices.len() {
            self.column_info_pos = self.column_indices[self.column_indices_pos] as usize;
        } else {
            self.column_info_pos = self.column_infos.len();
        }
    }
}

impl<'a> Iterator for ColumnInfoIter<'a> {
    type Item = &'a ColumnInfo;

    fn next(&mut self) -> Option<Self::Item> {
        if self.column_info_pos < self.column_infos.len() {
            let info = &self.column_infos[self.column_info_pos];
            self.column_info_pos += 1;
            Some(info)
        } else {
            None
        }
    }
}

// A trait that handles the mapping from Arrow schema to field decoders.
//
// Note that the decoders can only be figured out using both the schema AND
// the column metadata.  In theory, one could infer the decoder / column type
// using only the column metadata.  However, field nullability would be
// missing / incorrect and its also not as easy as it sounds since pages can
// have different encodings and those encodings often have various layers.
// Also, sometimes the inference is just impossible.  For example,
// Timestamp, Float64, Int64, and UInt64 will all be encoded as 8-byte value
// encoding.  The only way to know the data type is to look at the schema.
//
// We also can't just guess the encoding based on the schema.  This is because
// there may be multiple different ways to encode a field and it may even
// change on a page-by-page basis.
//
// For example, if a field is a struct field then we expect a header
// column that could have one of a few different encodings.
//
// This could be encoded with "simple struct" and an empty header column
// followed by the shredded child columns.  It could be encoded as a nullable
// struct where the nulls are in a dense bitmap.  It could even be encoded
// as a packed (row-major) struct where there is only a single column containing
// all of the data!
//
// TODO: Still lots of research to do here in different ways that
// we can map schemas to buffers.
//
// Example: repetition levels - the validity bitmaps for nested
// fields are fatter (more than one bit per row) and contain
// validity information about parent fields (e.g. is this a
// struct-struct-null or struct-null-null or null-null-null?)
//
// Examples: sentinel-shredding - instead of creating a wider
// validity bitmap we assign more sentinels to each column.  So
// if the values of an int32 array have a max of 1000 then we can
// use 1001 to mean null int32 and 1002 to mean null parent.
//
// Examples: Sparse structs - the struct column has a validity
// bitmap that must be read if you plan on reading the struct
// or any nested field.  However, this could be a compressed
// bitmap stored in metadata.  A perk for this approach is that
// the child fields can then have a smaller size than the parent
// field.  E.g. if a struct is 1000 rows and 900 of them are
// null then there is one validity bitmap of length 1000 and
// 100 rows of each of the children.
pub trait FieldDecoderStrategy: Send + Sync + std::fmt::Debug {
    /// Called to create a field scheduler for a field
    ///
    /// Stratgies can examine:
    /// * The target field
    /// * The column metadata (potentially consuming multiple columns)
    ///
    /// If a strategy does not handle an encoding it should call
    /// `chain.next` to delegate to the next strategy in the chain.
    ///
    /// The actual scheduler creation is asynchronous.  This is because
    /// the scheduler may need to read column metadata from disk.
    fn create_field_scheduler<'a>(
        &self,
        field: &Field,
        column_infos: &mut ColumnInfoIter,
        buffers: FileBuffers,
        chain: DecoderMiddlewareChainCursor<'a>,
    ) -> Result<ChosenFieldScheduler<'a>>;
}

/// The core decoder strategy handles all the various Arrow types
#[derive(Debug, Default)]
pub struct CoreFieldDecoderStrategy {
    pub validate_data: bool,
}

impl CoreFieldDecoderStrategy {
    /// This is just a sanity check to ensure there is no "wrapped encodings"
    /// that haven't been handled.
    fn ensure_values_encoded(column_info: &ColumnInfo, path: &VecDeque<u32>) -> Result<()> {
        let column_encoding = column_info
            .encoding
            .column_encoding
            .as_ref()
            .ok_or_else(|| {
                Error::invalid_input(
                    format!(
                        "the column at index {} was missing a ColumnEncoding",
                        column_info.index
                    ),
                    location!(),
                )
            })?;
        if matches!(
            column_encoding,
            pb::column_encoding::ColumnEncoding::Values(_)
        ) {
            Ok(())
        } else {
            Err(Error::invalid_input(format!("the column at index {} mapping to the input field at {:?} has column encoding {:?} and no decoder is registered to handle it", column_info.index, path, column_encoding), location!()))
        }
    }

    fn is_primitive(data_type: &DataType) -> bool {
        if data_type.is_primitive() | data_type.is_binary_like() {
            true
        } else {
            match data_type {
                // DataType::is_primitive doesn't consider these primitive but we do
                DataType::Boolean | DataType::Null | DataType::FixedSizeBinary(_) => true,
                DataType::FixedSizeList(inner, _) => Self::is_primitive(inner.data_type()),
                _ => false,
            }
        }
    }

    fn create_primitive_scheduler(
        &self,
        data_type: &DataType,
        path: &VecDeque<u32>,
        column: &ColumnInfo,
        buffers: FileBuffers,
    ) -> Result<Arc<dyn FieldScheduler>> {
        Self::ensure_values_encoded(column, path)?;
        // Primitive fields map to a single column
        let column_buffers = ColumnBuffers {
            file_buffers: buffers,
            positions_and_sizes: &column.buffer_offsets_and_sizes,
        };
        Ok(Arc::new(PrimitiveFieldScheduler::new(
            data_type.clone(),
            column.page_infos.clone(),
            column_buffers,
            self.validate_data,
        )))
    }

    /// Helper method to verify the page encoding of a struct header column
    fn check_simple_struct(column_info: &ColumnInfo, path: &VecDeque<u32>) -> Result<()> {
        Self::ensure_values_encoded(column_info, path)?;
        if column_info.page_infos.len() != 1 {
            return Err(Error::InvalidInput { source: format!("Due to schema we expected a struct column but we received a column with {} pages and right now we only support struct columns with 1 page", column_info.page_infos.len()).into(), location: location!() });
        }
        let encoding = &column_info.page_infos[0].encoding;
        match encoding.array_encoding.as_ref().unwrap() {
            pb::array_encoding::ArrayEncoding::Struct(_) => Ok(()),
            _ => Err(Error::InvalidInput { source: format!("Expected a struct encoding because we have a struct field in the schema but got the encoding {:?}", encoding).into(), location: location!() }),
        }
    }

    fn check_packed_struct(column_info: &ColumnInfo) -> bool {
        let encoding = &column_info.page_infos[0].encoding;
        matches!(
            encoding.array_encoding.as_ref().unwrap(),
            pb::array_encoding::ArrayEncoding::PackedStruct(_)
        )
    }
}

impl FieldDecoderStrategy for CoreFieldDecoderStrategy {
    fn create_field_scheduler<'a>(
        &self,
        field: &Field,
        column_infos: &mut ColumnInfoIter,
        buffers: FileBuffers,
        chain: DecoderMiddlewareChainCursor<'a>,
    ) -> Result<ChosenFieldScheduler<'a>> {
        let data_type = field.data_type();
        if Self::is_primitive(&data_type) {
            let primitive_col = column_infos.next().unwrap();
            let scheduler = self.create_primitive_scheduler(
                &data_type,
                chain.current_path(),
                primitive_col,
                buffers,
            )?;
            return Ok((chain, Ok(scheduler)));
        }
        match &data_type {
            DataType::FixedSizeList(inner, _dimension) => {
                // A fixed size list column could either be a physical or a logical decoder
                // depending on the child data type.
                if Self::is_primitive(inner.data_type()) {
                    let primitive_col = column_infos.next().unwrap();
                    let scheduler = self.create_primitive_scheduler(
                        &data_type,
                        chain.current_path(),
                        primitive_col,
                        buffers,
                    )?;
                    Ok((chain, Ok(scheduler)))
                } else {
                    todo!()
                }
            }
            DataType::Dictionary(_key_type, value_type) => {
                if Self::is_primitive(value_type) {
                    let primitive_col = column_infos.next().unwrap();
                    let scheduler = self.create_primitive_scheduler(
                        &data_type,
                        chain.current_path(),
                        primitive_col,
                        buffers,
                    )?;
                    Ok((chain, Ok(scheduler)))
                } else {
                    Err(Error::NotSupported {
                        source: format!(
                            "No way to decode into a dictionary field of type {}",
                            value_type
                        )
                        .into(),
                        location: location!(),
                    })
                }
            }
            DataType::List(items_field) | DataType::LargeList(items_field) => {
                let offsets_column = column_infos.next().unwrap();
                Self::ensure_values_encoded(offsets_column, chain.current_path())?;
                let offsets_column_buffers = ColumnBuffers {
                    file_buffers: buffers,
                    positions_and_sizes: &offsets_column.buffer_offsets_and_sizes,
                };
                let item_field_name = items_field.name().clone();
                let (chain, items_scheduler) = chain.new_child(
                    /*child_idx=*/ 0,
                    &field.children[0],
                    column_infos,
                    buffers,
                )?;
                let items_scheduler = items_scheduler?;

                let (inner_infos, null_offset_adjustments): (Vec<_>, Vec<_>) = offsets_column
                    .page_infos
                    .iter()
                    .map(|offsets_page| {
                        if let Some(pb::array_encoding::ArrayEncoding::List(list_encoding)) =
                            &offsets_page.encoding.array_encoding
                        {
                            let inner = PageInfo {
                                buffer_offsets_and_sizes: offsets_page
                                    .buffer_offsets_and_sizes
                                    .clone(),
                                encoding: list_encoding.offsets.as_ref().unwrap().as_ref().clone(),
                                num_rows: offsets_page.num_rows,
                            };
                            (
                                inner,
                                OffsetPageInfo {
                                    offsets_in_page: offsets_page.num_rows,
                                    null_offset_adjustment: list_encoding.null_offset_adjustment,
                                    num_items_referenced_by_page: list_encoding.num_items,
                                },
                            )
                        } else {
                            // TODO: Should probably return Err here
                            panic!("Expected a list column");
                        }
                    })
                    .unzip();
                let inner = Arc::new(PrimitiveFieldScheduler::new(
                    DataType::UInt64,
                    Arc::from(inner_infos.into_boxed_slice()),
                    offsets_column_buffers,
                    self.validate_data,
                )) as Arc<dyn FieldScheduler>;
                let offset_type = if matches!(data_type, DataType::List(_)) {
                    DataType::Int32
                } else {
                    DataType::Int64
                };
                let items_type = items_field.data_type().clone();
                let list_scheduler = Ok(Arc::new(ListFieldScheduler::new(
                    inner,
                    items_scheduler,
                    item_field_name.clone(),
                    items_type,
                    offset_type,
                    null_offset_adjustments,
                )) as Arc<dyn FieldScheduler>);
                Ok((chain, list_scheduler))
            }
            DataType::Struct(fields) => {
                let column_info = column_infos.next().unwrap();

                if Self::check_packed_struct(column_info) {
                    // use packed struct encoding
                    let scheduler = self.create_primitive_scheduler(
                        &data_type,
                        chain.current_path(),
                        column_info,
                        buffers,
                    )?;
                    Ok((chain, Ok(scheduler)))
                } else {
                    // use default struct encoding
                    Self::check_simple_struct(column_info, chain.current_path()).unwrap();
                    let is_root = field.metadata.contains_key("__lance_decoder_root");
                    let mut child_schedulers = Vec::with_capacity(field.children.len());
                    let mut chain = chain;
                    for (i, field) in field.children.iter().enumerate() {
                        if is_root {
                            column_infos.next_top_level();
                        }
                        let (next_chain, field_scheduler) =
                            chain.new_child(i as u32, field, column_infos, buffers)?;
                        child_schedulers.push(field_scheduler?);
                        chain = next_chain;
                    }

                    let fields = fields.clone();
                    let struct_scheduler = Ok(Arc::new(SimpleStructScheduler::new(
                        child_schedulers,
                        fields,
                    )) as Arc<dyn FieldScheduler>);

                    // For now, we don't record nullability for structs.  As a result, there is always
                    // only one "page" of struct data.  In the future, this will change.  A null-aware
                    // struct scheduler will need to first calculate how many rows are in the struct page
                    // and then find the child pages that overlap.  This should be doable.
                    Ok((chain, struct_scheduler))
                }
            }
            // TODO: Still need support for dictionary / RLE
            _ => chain.next(field, column_infos, buffers),
        }
    }
}

/// Create's a dummy ColumnInfo for the root column
fn root_column(num_rows: u64) -> ColumnInfo {
    let num_root_pages = num_rows.div_ceil(u32::MAX as u64);
    let final_page_num_rows = num_rows % (u32::MAX as u64);
    let root_pages = (0..num_root_pages)
        .map(|i| PageInfo {
            num_rows: if i == num_root_pages - 1 {
                final_page_num_rows
            } else {
                u64::MAX
            },
            encoding: pb::ArrayEncoding {
                array_encoding: Some(pb::array_encoding::ArrayEncoding::Struct(
                    pb::SimpleStruct {},
                )),
            },
            buffer_offsets_and_sizes: Arc::new([]),
        })
        .collect::<Vec<_>>();
    ColumnInfo {
        buffer_offsets_and_sizes: Arc::new([]),
        encoding: values_column_encoding(),
        index: u32::MAX,
        page_infos: Arc::from(root_pages),
    }
}

impl DecodeBatchScheduler {
    /// Creates a new decode scheduler with the expected schema and the column
    /// metadata of the file.
    pub fn try_new<'a>(
        schema: &'a Schema,
        column_indices: &[u32],
        column_infos: &[Arc<ColumnInfo>],
        file_buffer_positions_and_sizes: &'a Vec<(u64, u64)>,
        num_rows: u64,
        decoder_strategy: &DecoderMiddlewareChain,
        io: &Arc<dyn EncodingsIo>,
    ) -> Result<Self> {
        let buffers = FileBuffers {
            positions_and_sizes: file_buffer_positions_and_sizes,
        };
        let arrow_schema = ArrowSchema::from(schema);
        let root_fields = arrow_schema.fields().clone();
        let mut columns = Vec::with_capacity(column_infos.len() + 1);
        columns.push(root_column(num_rows));
        columns.extend(column_infos.iter().map(|col| col.as_ref().clone()));
        let adjusted_column_indices = [0_u32]
            .into_iter()
            .chain(column_indices.iter().map(|i| *i + 1))
            .collect::<Vec<_>>();
        let mut column_iter = ColumnInfoIter::new(&columns, &adjusted_column_indices);
        let root_type = DataType::Struct(root_fields.clone());
        let mut root_field = Field::try_from(&ArrowField::new("root", root_type, false))?;
        root_field
            .metadata
            .insert("__lance_decoder_root".to_string(), "true".to_string());
        let (_, root_scheduler) =
            decoder_strategy
                .cursor(io)
                .start(&root_field, &mut column_iter, buffers)?;
        let root_scheduler = root_scheduler?;
        Ok(Self {
            root_scheduler,
            root_fields,
        })
    }

    pub fn from_scheduler(root_scheduler: Arc<dyn FieldScheduler>, root_fields: Fields) -> Self {
        Self {
            root_scheduler,
            root_fields,
        }
    }

    fn do_schedule_ranges(
        &mut self,
        ranges: &[Range<u64>],
        filter: &FilterExpression,
        io: Arc<dyn EncodingsIo>,
        mut schedule_action: impl FnMut(Result<DecoderMessage>),
    ) {
        let rows_requested = ranges.iter().map(|r| r.end - r.start).sum::<u64>();
        trace!("Scheduling ranges {:?} ({} rows)", ranges, rows_requested);

        let mut context = SchedulerContext::new(io);
        let maybe_root_job = self.root_scheduler.schedule_ranges(ranges, filter);
        if let Err(schedule_ranges_err) = maybe_root_job {
            schedule_action(Err(schedule_ranges_err));
            return;
        }
        let mut root_job = maybe_root_job.unwrap();
        let mut num_rows_scheduled = 0;
        let mut rows_to_schedule = root_job.num_rows();
        trace!("Scheduled ranges refined to {} rows", rows_to_schedule);
        while rows_to_schedule > 0 {
            let maybe_next_scan_line = root_job.schedule_next(&mut context, num_rows_scheduled);
            if let Err(schedule_next_err) = maybe_next_scan_line {
                schedule_action(Err(schedule_next_err));
                return;
            }
            let next_scan_line = maybe_next_scan_line.unwrap();
            num_rows_scheduled += next_scan_line.rows_scheduled;
            rows_to_schedule -= next_scan_line.rows_scheduled;
            trace!(
                "Scheduled scan line of {} rows and {} decoders",
                next_scan_line.rows_scheduled,
                next_scan_line.decoders.len()
            );
            schedule_action(Ok(DecoderMessage {
                scheduled_so_far: num_rows_scheduled,
                decoders: next_scan_line.decoders,
            }));
        }

        trace!("Finished scheduling {} ranges", ranges.len());
    }

    // This method is similar to schedule_ranges but instead of
    // sending the decoders to a channel it collects them all into a vector
    pub fn schedule_ranges_to_vec(
        &mut self,
        ranges: &[Range<u64>],
        filter: &FilterExpression,
        io: Arc<dyn EncodingsIo>,
    ) -> Result<Vec<DecoderMessage>> {
        let mut decode_messages = Vec::new();
        self.do_schedule_ranges(ranges, filter, io, |msg| decode_messages.push(msg));
        decode_messages.into_iter().collect::<Result<Vec<_>>>()
    }

    /// Schedules the load of a multiple ranges of rows
    ///
    /// Ranges must be non-overlapping and in sorted order
    ///
    /// # Arguments
    ///
    /// * `ranges` - The ranges of rows to load
    /// * `sink` - A channel to send the decode tasks
    /// * `scheduler` An I/O scheduler to issue I/O requests
    #[instrument(skip_all)]
    pub fn schedule_ranges(
        &mut self,
        ranges: &[Range<u64>],
        filter: &FilterExpression,
        sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
        scheduler: Arc<dyn EncodingsIo>,
    ) {
        self.do_schedule_ranges(ranges, filter, scheduler, |msg| {
            sink.send(msg).unwrap();
        })
    }

    /// Schedules the load of a range of rows
    ///
    /// # Arguments
    ///
    /// * `range` - The range of rows to load
    /// * `sink` - A channel to send the decode tasks
    /// * `scheduler` An I/O scheduler to issue I/O requests
    #[instrument(skip_all)]
    pub fn schedule_range(
        &mut self,
        range: Range<u64>,
        filter: &FilterExpression,
        sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
        scheduler: Arc<dyn EncodingsIo>,
    ) {
        self.schedule_ranges(&[range.clone()], filter, sink, scheduler)
    }

    /// Schedules the load of selected rows
    ///
    /// # Arguments
    ///
    /// * `indices` - The row indices to load (these must be in ascending order!)
    /// * `sink` - A channel to send the decode tasks
    /// * `scheduler` An I/O scheduler to issue I/O requests
    pub fn schedule_take(
        &mut self,
        indices: &[u64],
        filter: &FilterExpression,
        sink: mpsc::UnboundedSender<Result<DecoderMessage>>,
        scheduler: Arc<dyn EncodingsIo>,
    ) {
        debug_assert!(indices.windows(2).all(|w| w[0] <= w[1]));
        if indices.is_empty() {
            return;
        }
        trace!("Scheduling take of {} rows", indices.len());
        let ranges = indices
            .iter()
            .map(|&idx| idx..(idx + 1))
            .collect::<Vec<_>>();
        self.schedule_ranges(&ranges, filter, sink, scheduler)
    }

    pub fn new_root_decoder_ranges(&self, ranges: &[Range<u64>]) -> SimpleStructDecoder {
        let rows_to_read = ranges
            .iter()
            .map(|range| range.end - range.start)
            .sum::<u64>();
        SimpleStructDecoder::new(self.root_fields.clone(), rows_to_read)
    }

    pub fn new_root_decoder_indices(&self, indices: &[u64]) -> SimpleStructDecoder {
        SimpleStructDecoder::new(self.root_fields.clone(), indices.len() as u64)
    }
}

pub struct ReadBatchTask {
    pub task: BoxFuture<'static, Result<RecordBatch>>,
    pub num_rows: u32,
}

/// A stream that takes scheduled jobs and generates decode tasks from them.
pub struct BatchDecodeStream {
    context: DecoderContext,
    root_decoder: SimpleStructDecoder,
    rows_remaining: u64,
    rows_per_batch: u32,
    rows_scheduled: u64,
    rows_drained: u64,
    scheduler_exhuasted: bool,
    emitted_batch_size_warning: Arc<Once>,
}

impl BatchDecodeStream {
    /// Create a new instance of a batch decode stream
    ///
    /// # Arguments
    ///
    /// * `scheduled` - an incoming stream of decode tasks from a
    ///   [`crate::decode::DecodeBatchScheduler`]
    /// * `schema` - the scheam of the data to create
    /// * `rows_per_batch` the number of rows to create before making a batch
    /// * `num_rows` the total number of rows scheduled
    /// * `num_columns` the total number of columns in the file
    pub fn new(
        scheduled: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
        rows_per_batch: u32,
        num_rows: u64,
        root_decoder: SimpleStructDecoder,
    ) -> Self {
        Self {
            context: DecoderContext::new(scheduled),
            root_decoder,
            rows_remaining: num_rows,
            rows_per_batch,
            rows_scheduled: 0,
            rows_drained: 0,
            scheduler_exhuasted: false,
            emitted_batch_size_warning: Arc::new(Once::new()),
        }
    }

    fn accept_decoder(&mut self, decoder: DecoderReady) -> Result<()> {
        if decoder.path.is_empty() {
            // The root decoder we can ignore
            Ok(())
        } else {
            self.root_decoder.accept_child(decoder)
        }
    }

    async fn wait_for_scheduled(&mut self, scheduled_need: u64) -> Result<u64> {
        if self.scheduler_exhuasted {
            return Ok(self.rows_scheduled);
        }
        while self.rows_scheduled < scheduled_need {
            let next_message = self.context.source.recv().await;
            match next_message {
                Some(scan_line) => {
                    let scan_line = scan_line?;
                    self.rows_scheduled = scan_line.scheduled_so_far;
                    for decoder in scan_line.decoders {
                        self.accept_decoder(decoder)?;
                    }
                }
                None => {
                    // Schedule ended before we got all the data we expected.  This probably
                    // means some kind of pushdown filter was applied and we didn't load as
                    // much data as we thought we would.
                    self.scheduler_exhuasted = true;
                    return Ok(self.rows_scheduled);
                }
            }
        }
        Ok(scheduled_need)
    }

    #[instrument(level = "debug", skip_all)]
    async fn next_batch_task(&mut self) -> Result<Option<NextDecodeTask>> {
        trace!(
            "Draining batch task (rows_remaining={} rows_drained={} rows_scheduled={})",
            self.rows_remaining,
            self.rows_drained,
            self.rows_scheduled,
        );
        if self.rows_remaining == 0 {
            return Ok(None);
        }

        let mut to_take = self.rows_remaining.min(self.rows_per_batch as u64);
        self.rows_remaining -= to_take;

        let scheduled_need = (self.rows_drained + to_take).saturating_sub(self.rows_scheduled);
        trace!("scheduled_need = {} because rows_drained = {} and to_take = {} and rows_scheduled = {}", scheduled_need, self.rows_drained, to_take, self.rows_scheduled);
        if scheduled_need > 0 {
            let desired_scheduled = scheduled_need + self.rows_scheduled;
            trace!(
                "Draining from scheduler (desire at least {} scheduled rows)",
                desired_scheduled
            );
            let actually_scheduled = self.wait_for_scheduled(desired_scheduled).await?;
            if actually_scheduled < desired_scheduled {
                let under_scheduled = desired_scheduled - actually_scheduled;
                to_take -= under_scheduled;
            }
        }

        if to_take == 0 {
            return Ok(None);
        }

        let avail = self.root_decoder.avail();
        trace!("Top level page has {} rows already available", avail);
        if avail < to_take {
            trace!(
                "Top level page waiting for an additional {} rows",
                to_take - avail
            );
            self.root_decoder.wait(to_take).await?;
        }
        let next_task = self.root_decoder.drain(to_take)?;
        self.rows_drained += to_take;
        Ok(Some(next_task))
    }

    #[instrument(level = "debug", skip_all)]
    fn task_to_batch(
        task: NextDecodeTask,
        emitted_batch_size_warning: Arc<Once>,
    ) -> Result<RecordBatch> {
        let struct_arr = task.task.decode();
        match struct_arr {
            Ok(struct_arr) => {
                let batch = RecordBatch::from(struct_arr.as_struct());
                let size_bytes = batch.get_array_memory_size() as u64;
                if size_bytes > BATCH_SIZE_BYTES_WARNING {
                    emitted_batch_size_warning.call_once(|| {
                        let size_mb = size_bytes / 1024 / 1024;
                        warn!("Lance read in a single batch that contained more than {}MiB of data.  You may want to consider reducing the batch size.", size_mb);
                    });
                }
                Ok(batch)
            }
            Err(e) => {
                let e = Error::Internal {
                    message: format!("Error decoding batch: {}", e),
                    location: location!(),
                };
                Err(e)
            }
        }
    }

    pub fn into_stream(self) -> BoxStream<'static, ReadBatchTask> {
        let stream = futures::stream::unfold(self, |mut slf| async move {
            let next_task = slf.next_batch_task().await;
            let next_task = next_task.transpose().map(|next_task| {
                let num_rows = next_task.as_ref().map(|t| t.num_rows).unwrap_or(0);
                let emitted_batch_size_warning = slf.emitted_batch_size_warning.clone();
                let task = tokio::spawn(async move {
                    let next_task = next_task?;
                    Self::task_to_batch(next_task, emitted_batch_size_warning)
                });
                (task, num_rows)
            });
            next_task.map(|(task, num_rows)| {
                let task = task.map(|join_wrapper| join_wrapper.unwrap()).boxed();
                // This should be true since batch size is u32
                debug_assert!(num_rows <= u32::MAX as u64);
                let next_task = ReadBatchTask {
                    task,
                    num_rows: num_rows as u32,
                };
                (next_task, slf)
            })
        });
        stream.boxed()
    }
}

/// A decoder for single-column encodings of primitive data (this includes fixed size
/// lists of primitive data)
///
/// Physical decoders are able to decode into existing buffers for zero-copy operation.
///
/// Instances should be stateless and `Send` / `Sync`.  This is because multiple decode
/// tasks could reference the same page.  For example, imagine a page covers rows 0-2000
/// and the decoder stream has a batch size of 1024.  The decoder will be needed by both
/// the decode task for batch 0 and the decode task for batch 1.
///
/// See [`crate::decoder`] for more information
pub trait PrimitivePageDecoder: Send + Sync {
    /// Decode data into buffers
    ///
    /// This may be a simple zero-copy from a disk buffer or could involve complex decoding
    /// such as decompressing from some compressed representation.
    ///
    /// Capacity is stored as a tuple of (num_bytes: u64, is_needed: bool).  The `is_needed`
    /// portion only needs to be updated if the encoding has some concept of an "optional"
    /// buffer.
    ///
    /// Encodings can have any number of input or output buffers.  For example, a dictionary
    /// decoding will convert two buffers (indices + dictionary) into a single buffer
    ///
    /// Binary decodings have two output buffers (one for values, one for offsets)
    ///
    /// Other decodings could even expand the # of output buffers.  For example, we could decode
    /// fixed size strings into variable length strings going from one input buffer to multiple output
    /// buffers.
    ///
    /// Each Arrow data type typically has a fixed structure of buffers and the encoding chain will
    /// generally end at one of these structures.  However, intermediate structures may exist which
    /// do not correspond to any Arrow type at all.  For example, a bitpacking encoding will deal
    /// with buffers that have bits-per-value that is not a multiple of 8.
    ///
    /// The `primitive_array_from_buffers` method has an expected buffer layout for each arrow
    /// type (order matters) and encodings that aim to decode into arrow types should respect
    /// this layout.
    /// # Arguments
    ///
    /// * `rows_to_skip` - how many rows to skip (within the page) before decoding
    /// * `num_rows` - how many rows to decode
    /// * `all_null` - A mutable bool, set to true if a decoder determines all values are null
    fn decode(&self, rows_to_skip: u64, num_rows: u64) -> Result<Box<dyn DataBlock>>;
}

/// A scheduler for single-column encodings of primitive data
///
/// The scheduler is responsible for calculating what I/O is needed for the requested rows
///
/// Instances should be stateless and `Send` and `Sync`.  This is because instances can
/// be shared in follow-up I/O tasks.
///
/// See [`crate::decoder`] for more information
pub trait PageScheduler: Send + Sync + std::fmt::Debug {
    /// Schedules a batch of I/O to load the data needed for the requested ranges
    ///
    /// Returns a future that will yield a decoder once the data has been loaded
    ///
    /// # Arguments
    ///
    /// * `range` - the range of row offsets (relative to start of page) requested
    ///             these must be ordered and must not overlap
    /// * `scheduler` - a scheduler to submit the I/O request to
    /// * `top_level_row` - the row offset of the top level field currently being
    ///   scheduled.  This can be used to assign priority to I/O requests
    fn schedule_ranges(
        &self,
        ranges: &[Range<u64>],
        scheduler: &Arc<dyn EncodingsIo>,
        top_level_row: u64,
    ) -> BoxFuture<'static, Result<Box<dyn PrimitivePageDecoder>>>;
}

/// Contains the context for a scheduler
pub struct SchedulerContext {
    recv: Option<mpsc::UnboundedReceiver<DecoderMessage>>,
    io: Arc<dyn EncodingsIo>,
    name: String,
    path: Vec<u32>,
    path_names: Vec<String>,
}

pub struct ScopedSchedulerContext<'a> {
    pub context: &'a mut SchedulerContext,
}

impl<'a> ScopedSchedulerContext<'a> {
    pub fn pop(self) -> &'a mut SchedulerContext {
        self.context.pop();
        self.context
    }
}

impl SchedulerContext {
    pub fn new(io: Arc<dyn EncodingsIo>) -> Self {
        Self {
            io,
            recv: None,
            name: "".to_string(),
            path: Vec::new(),
            path_names: Vec::new(),
        }
    }

    pub fn io(&self) -> &Arc<dyn EncodingsIo> {
        &self.io
    }

    pub fn push(&mut self, name: &str, index: u32) -> ScopedSchedulerContext {
        self.path.push(index);
        self.path_names.push(name.to_string());
        ScopedSchedulerContext { context: self }
    }

    pub fn pop(&mut self) {
        self.path.pop();
        self.path_names.pop();
    }

    pub fn path_name(&self) -> String {
        let path = self.path_names.join("/");
        if self.recv.is_some() {
            format!("TEMP({}){}", self.name, path)
        } else {
            format!("ROOT{}", path)
        }
    }

    pub fn locate_decoder(&mut self, decoder: Box<dyn LogicalPageDecoder>) -> DecoderReady {
        trace!(
            "Scheduling decoder of type {:?} for {:?}",
            decoder.data_type(),
            self.path,
        );
        DecoderReady {
            decoder,
            path: VecDeque::from_iter(self.path.iter().copied()),
        }
    }
}

#[derive(Debug)]
pub struct ScheduledScanLine {
    pub rows_scheduled: u64,
    pub decoders: Vec<DecoderReady>,
}

pub trait SchedulingJob: std::fmt::Debug {
    fn schedule_next(
        &mut self,
        context: &mut SchedulerContext,
        top_level_row: u64,
    ) -> Result<ScheduledScanLine>;

    fn num_rows(&self) -> u64;
}

/// A filter expression to apply to the data
///
/// The core decoders do not currently take advantage of filtering in
/// any way.  In order to maintain the abstraction we represent filters
/// as an arbitrary byte sequence.
///
/// We recommend that encodings use Substrait for filters.
pub struct FilterExpression(pub Bytes);

impl FilterExpression {
    /// Create a filter expression that does not filter any data
    ///
    /// This is currently represented by an empty byte array.  Encoders
    /// that are "filter aware" should make sure they handle this case.
    pub fn no_filter() -> Self {
        Self(Bytes::new())
    }
}

/// A scheduler for a field's worth of data
///
/// Each field in a reader's output schema maps to one field scheduler.  This scheduler may
/// map to more than one column.  For example, one field of struct data may
/// cover many columns of child data.  In fact, the entire file is treated as one
/// top-level struct field.
///
/// The scheduler is responsible for calculating the neccesary I/O.  One schedule_range
/// request could trigger mulitple batches of I/O across multiple columns.  The scheduler
/// should emit decoders into the sink as quickly as possible.
///
/// As soon as the scheduler encounters a batch of data that can decoded then the scheduler
/// should emit a decoder in the "unloaded" state.  The decode stream will pull the decoder
/// and start decoding.
///
/// The order in which decoders are emitted is important.  Pages should be emitted in
/// row-major order allowing decode of complete rows as quickly as possible.
///
/// The `FieldScheduler` should be stateless and `Send` and `Sync`.  This is
/// because it might need to be shared.  For example, a list page has a reference to
/// the field schedulers for its items column.  This is shared with the follow-up I/O
/// task created when the offsets are loaded.
///
/// See [`crate::decoder`] for more information
pub trait FieldScheduler: Send + Sync + std::fmt::Debug {
    /// Schedules I/O for the requested portions of the field.
    ///
    /// Note: `ranges` must be ordered and non-overlapping
    /// TODO: Support unordered or overlapping ranges in file scheduler
    fn schedule_ranges<'a>(
        &'a self,
        ranges: &[Range<u64>],
        filter: &FilterExpression,
    ) -> Result<Box<dyn SchedulingJob + 'a>>;
    /// The number of rows in this field
    fn num_rows(&self) -> u64;
}

/// A trait for tasks that decode data into an Arrow array
pub trait DecodeArrayTask: Send {
    /// Decodes the data into an Arrow array
    fn decode(self: Box<Self>) -> Result<ArrayRef>;
}

/// A task to decode data into an Arrow array
pub struct NextDecodeTask {
    /// The decode task itself
    pub task: Box<dyn DecodeArrayTask>,
    /// The number of rows that will be created
    pub num_rows: u64,
    /// Whether or not the decoder that created this still has more rows to decode
    pub has_more: bool,
}

#[derive(Debug)]
pub struct DecoderReady {
    // The decoder that is ready to be decoded
    pub decoder: Box<dyn LogicalPageDecoder>,
    // The path to the decoder, the first value is the column index
    // following values, if present, are nested child indices
    //
    // For example, a path of [1, 1, 0] would mean to grab the second
    // column, then the second child, and then the first child.
    //
    // It could represent x in the following schema:
    //
    // score: float64
    // points: struct
    //   color: string
    //   location: struct
    //     x: float64
    //
    // Currently, only struct decoders have "children" although other
    // decoders may at some point as well.  List children are only
    // handled through indirect I/O at the moment and so they don't
    // need to be represented (yet)
    pub path: VecDeque<u32>,
}

pub struct DecoderMessage {
    pub scheduled_so_far: u64,
    pub decoders: Vec<DecoderReady>,
}

pub struct DecoderContext {
    source: mpsc::UnboundedReceiver<Result<DecoderMessage>>,
}

impl DecoderContext {
    pub fn new(source: mpsc::UnboundedReceiver<Result<DecoderMessage>>) -> Self {
        Self { source }
    }
}

/// A decoder for a field's worth of data
///
/// The decoder is initially "unloaded" (doesn't have all its data).  The [`Self::wait`]
/// method should be called to wait for the needed I/O data before attempting to decode
/// any further.
///
/// Unlike the other decoder types it is assumed that `LogicalPageDecoder` is stateful
/// and only `Send`.  This is why we don't need a `rows_to_skip` argument in [`Self::drain`]
pub trait LogicalPageDecoder: std::fmt::Debug + Send {
    /// Add a newly scheduled child decoder
    ///
    /// The default implementation does not expect children and returns
    /// an error.
    fn accept_child(&mut self, _child: DecoderReady) -> Result<()> {
        Err(Error::Internal {
            message: format!(
                "The decoder {:?} does not expect children but received a child",
                self
            ),
            location: location!(),
        })
    }
    /// Waits for enough data to be loaded to decode `num_rows` of data
    fn wait(&mut self, num_rows: u64) -> BoxFuture<Result<()>>;
    /// Creates a task to decode `num_rows` of data into an array
    fn drain(&mut self, num_rows: u64) -> Result<NextDecodeTask>;
    /// The number of rows that are in the page but haven't yet been "waited"
    fn unawaited(&self) -> u64;
    /// The number of rows that have been "waited" but not yet decoded
    fn avail(&self) -> u64;
    /// The data type of the decoded data
    fn data_type(&self) -> &DataType;
}

/// Decodes a batch of data from an in-memory structure created by [`crate::encoder::encode_batch`]
pub async fn decode_batch(
    batch: &EncodedBatch,
    filter: &FilterExpression,
    field_decoder_strategy: &DecoderMiddlewareChain,
) -> Result<RecordBatch> {
    let io_scheduler = Arc::new(BufferScheduler::new(batch.data.clone())) as Arc<dyn EncodingsIo>;
    let mut decode_scheduler = DecodeBatchScheduler::try_new(
        batch.schema.as_ref(),
        &batch.top_level_columns,
        &batch.page_table,
        &vec![],
        batch.num_rows,
        field_decoder_strategy,
        &io_scheduler,
    )?;
    let (tx, rx) = unbounded_channel();
    decode_scheduler.schedule_range(0..batch.num_rows, filter, tx, io_scheduler);
    #[allow(clippy::single_range_in_vec_init)]
    let root_decoder = decode_scheduler.new_root_decoder_ranges(&[0..batch.num_rows]);
    let stream = BatchDecodeStream::new(rx, batch.num_rows as u32, batch.num_rows, root_decoder);
    stream.into_stream().next().await.unwrap().task.await
}