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

use std::{
    collections::{BTreeMap, BTreeSet},
    io::Cursor,
    ops::Range,
    pin::Pin,
    sync::Arc,
};

use arrow_schema::Schema as ArrowSchema;
use byteorder::{ByteOrder, LittleEndian, ReadBytesExt};
use bytes::{Bytes, BytesMut};
use deepsize::{Context, DeepSizeOf};
use futures::{stream::BoxStream, Stream, StreamExt};
use lance_encoding::{
    decoder::{
        schedule_and_decode, ColumnInfo, DecoderPlugins, FilterExpression, PageEncoding, PageInfo,
        ReadBatchTask, RequestedRows, SchedulerDecoderConfig,
    },
    encoder::EncodedBatch,
    version::LanceFileVersion,
    EncodingsIo,
};
use log::debug;
use prost::{Message, Name};
use snafu::{location, Location};

use lance_core::{
    cache::FileMetadataCache,
    datatypes::{Field, Schema},
    Error, Result,
};
use lance_encoding::format::pb as pbenc;
use lance_io::{
    scheduler::FileScheduler,
    stream::{RecordBatchStream, RecordBatchStreamAdapter},
    ReadBatchParams,
};

use crate::{
    datatypes::{Fields, FieldsWithMeta},
    format::{pb, pbfile, MAGIC, MAJOR_VERSION, MINOR_VERSION},
    v2::writer::PAGE_BUFFER_ALIGNMENT,
};

use super::io::LanceEncodingsIo;

// For now, we don't use global buffers for anything other than schema.  If we
// use these later we should make them lazily loaded and then cached once loaded.
//
// We store their position / length for debugging purposes
#[derive(Debug, DeepSizeOf)]
pub struct BufferDescriptor {
    pub position: u64,
    pub size: u64,
}

// TODO: Caching
#[derive(Debug)]
pub struct CachedFileMetadata {
    /// The schema of the file
    pub file_schema: Arc<Schema>,
    /// The column metadatas
    pub column_metadatas: Vec<pbfile::ColumnMetadata>,
    pub column_infos: Vec<Arc<ColumnInfo>>,
    /// The number of rows in the file
    pub num_rows: u64,
    pub file_buffers: Vec<BufferDescriptor>,
    /// The number of bytes contained in the data page section of the file
    pub num_data_bytes: u64,
    /// The number of bytes contained in the column metadata (not including buffers
    /// referenced by the metadata)
    pub num_column_metadata_bytes: u64,
    /// The number of bytes contained in global buffers
    pub num_global_buffer_bytes: u64,
    /// The number of bytes contained in the CMO and GBO tables
    pub num_footer_bytes: u64,
    pub major_version: u16,
    pub minor_version: u16,
}

impl DeepSizeOf for CachedFileMetadata {
    // TODO: include size for `column_metadatas` and `column_infos`.
    fn deep_size_of_children(&self, context: &mut Context) -> usize {
        self.file_schema.deep_size_of_children(context)
            + self
                .file_buffers
                .iter()
                .map(|file_buffer| file_buffer.deep_size_of_children(context))
                .sum::<usize>()
    }
}

impl CachedFileMetadata {
    pub fn version(&self) -> LanceFileVersion {
        match (self.major_version, self.minor_version) {
            (0, 3) => LanceFileVersion::V2_0,
            (2, 1) => LanceFileVersion::V2_1,
            _ => panic!(
                "Unsupported version: {}.{}",
                self.major_version, self.minor_version
            ),
        }
    }
}

/// Selecting columns from a lance file requires specifying both the
/// index of the column and the data type of the column
///
/// Partly, this is because it is not strictly required that columns
/// be read into the same type.  For example, a string column may be
/// read as a string, large_string or string_view type.
///
/// A read will only succeed if the decoder for a column is capable
/// of decoding into the requested type.
///
/// Note that this should generally be limited to different in-memory
/// representations of the same semantic type.  An encoding could
/// theoretically support "casting" (e.g. int to string,  etc.) but
/// there is little advantage in doing so here.
///
/// Note: in order to specify a projection the user will need some way
/// to figure out the column indices.  In the table format we do this
/// using field IDs and keeping track of the field id->column index mapping.
///
/// If users are not using the table format then they will need to figure
/// out some way to do this themselves.
#[derive(Debug, Clone)]
pub struct ReaderProjection {
    /// The data types (schema) of the selected columns.  The names
    /// of the schema are arbitrary and ignored.
    pub schema: Arc<Schema>,
    /// The indices of the columns to load.
    ///
    /// The mapping should be as follows:
    ///
    /// - Primitive: the index of the column in the schema
    /// - List: the index of the list column in the schema
    ///         followed by the column indices of the children
    /// - FixedSizeList (of primitive): the index of the column in the schema
    ///         (this case is not nested)
    /// - FixedSizeList (of non-primitive): not yet implemented
    /// - Dictionary: same as primitive
    /// - Struct: the index of the struct column in the schema
    ///          followed by the column indices of the children
    ///
    /// In other words, this should be a DFS listing of the desired schema.
    ///
    /// For example, if the goal is to load:
    ///
    ///   x: int32
    ///   y: struct<z: int32, w: string>
    ///   z: list<int32>
    ///
    /// and the schema originally used to store the data was:
    ///
    ///   a: struct<x: int32>
    ///   b: int64
    ///   y: struct<z: int32, c: int64, w: string>
    ///   z: list<int32>
    ///
    /// Then the column_indices should be [1, 3, 4, 6, 7, 8]
    pub column_indices: Vec<u32>,
}

impl ReaderProjection {
    fn from_field_ids_helper<'a>(
        reader: &FileReader,
        fields: impl Iterator<Item = &'a Field>,
        field_id_to_column_index: &BTreeMap<u32, u32>,
        column_indices: &mut Vec<u32>,
    ) -> Result<()> {
        for field in fields {
            let is_structural = reader.metadata.version() >= LanceFileVersion::V2_1;
            // In the 2.0 system we needed ids for intermediate fields.  In 2.1+
            // we only need ids for leaf fields.
            if !is_structural || field.children.is_empty() {
                if let Some(column_idx) = field_id_to_column_index.get(&(field.id as u32)).copied()
                {
                    column_indices.push(column_idx);
                }
            }
            Self::from_field_ids_helper(
                reader,
                field.children.iter(),
                field_id_to_column_index,
                column_indices,
            )?;
        }
        Ok(())
    }

    /// Creates a projection using a mapping from field IDs to column indices
    ///
    /// You can obtain such a mapping when the file is written using the
    /// [`crate::v2::writer::FileWriter::field_id_to_column_indices`] method.
    pub fn from_field_ids(
        reader: &FileReader,
        schema: &Schema,
        field_id_to_column_index: &BTreeMap<u32, u32>,
    ) -> Result<Self> {
        let mut column_indices = Vec::new();
        Self::from_field_ids_helper(
            reader,
            schema.fields.iter(),
            field_id_to_column_index,
            &mut column_indices,
        )?;
        Ok(Self {
            schema: Arc::new(schema.clone()),
            column_indices,
        })
    }

    /// Creates a projection that reads the entire file
    ///
    /// If the schema provided is not the schema of the entire file then
    /// the projection will be invalid and the read will fail.
    pub fn from_whole_schema(schema: &Schema, version: LanceFileVersion) -> Self {
        let schema = Arc::new(schema.clone());
        let is_structural = version >= LanceFileVersion::V2_1;
        let mut counter = 0;
        let counter = &mut counter;
        let column_indices = schema
            .fields_pre_order()
            .filter_map(|field| {
                if field.children.is_empty() || !is_structural {
                    let col_idx = *counter;
                    *counter += 1;
                    Some(col_idx)
                } else {
                    None
                }
            })
            .collect::<Vec<_>>();
        Self {
            schema,
            column_indices,
        }
    }

    /// Creates a projection that reads the specified columns provided by name
    ///
    /// The syntax for column names is the same as [`lance_core::datatypes::Schema::project`]
    ///
    /// If the schema provided is not the schema of the entire file then
    /// the projection will be invalid and the read will fail.
    pub fn from_column_names(schema: &Schema, column_names: &[&str]) -> Result<Self> {
        let field_id_to_column_index = schema
            .fields_pre_order()
            .enumerate()
            .map(|(idx, field)| (field.id as u32, idx as u32))
            .collect::<BTreeMap<_, _>>();
        let projected = schema.project(column_names)?;
        let column_indices = projected
            .fields_pre_order()
            .map(|f| field_id_to_column_index[&(f.id as u32)])
            .collect::<Vec<_>>();
        Ok(Self {
            schema: Arc::new(projected),
            column_indices,
        })
    }
}

#[derive(Debug, Default)]
pub struct FileReaderOptions {
    validate_on_decode: bool,
}

#[derive(Debug)]
pub struct FileReader {
    scheduler: Arc<LanceEncodingsIo>,
    // The default projection to be applied to all reads
    base_projection: ReaderProjection,
    num_rows: u64,
    metadata: Arc<CachedFileMetadata>,
    decoder_plugins: Arc<DecoderPlugins>,
    cache: Arc<FileMetadataCache>,
    options: FileReaderOptions,
}
#[derive(Debug)]
struct Footer {
    #[allow(dead_code)]
    column_meta_start: u64,
    // We don't use this today because we always load metadata for every column
    // and don't yet support "metadata projection"
    #[allow(dead_code)]
    column_meta_offsets_start: u64,
    global_buff_offsets_start: u64,
    num_global_buffers: u32,
    num_columns: u32,
    major_version: u16,
    minor_version: u16,
}

const FOOTER_LEN: usize = 40;

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

    pub fn metadata(&self) -> &Arc<CachedFileMetadata> {
        &self.metadata
    }

    pub async fn read_global_buffer(&self, index: u32) -> Result<Bytes> {
        let buffer_desc = self.metadata.file_buffers.get(index as usize).ok_or_else(||Error::invalid_input(format!("request for global buffer at index {} but there were only {} global buffers in the file", index, self.metadata.file_buffers.len()), location!()))?;
        self.scheduler
            .submit_single(
                buffer_desc.position..buffer_desc.position + buffer_desc.size,
                0,
            )
            .await
    }

    async fn read_tail(scheduler: &FileScheduler) -> Result<(Bytes, u64)> {
        let file_size = scheduler.reader().size().await? as u64;
        let begin = if file_size < scheduler.reader().block_size() as u64 {
            0
        } else {
            file_size - scheduler.reader().block_size() as u64
        };
        let tail_bytes = scheduler.submit_single(begin..file_size, 0).await?;
        Ok((tail_bytes, file_size))
    }

    // Checks to make sure the footer is written correctly and returns the
    // position of the file descriptor (which comes from the footer)
    fn decode_footer(footer_bytes: &Bytes) -> Result<Footer> {
        let len = footer_bytes.len();
        if len < FOOTER_LEN {
            return Err(Error::io(
                format!(
                    "does not have sufficient data, len: {}, bytes: {:?}",
                    len, footer_bytes
                ),
                location!(),
            ));
        }
        let mut cursor = Cursor::new(footer_bytes.slice(len - FOOTER_LEN..));

        let column_meta_start = cursor.read_u64::<LittleEndian>()?;
        let column_meta_offsets_start = cursor.read_u64::<LittleEndian>()?;
        let global_buff_offsets_start = cursor.read_u64::<LittleEndian>()?;
        let num_global_buffers = cursor.read_u32::<LittleEndian>()?;
        let num_columns = cursor.read_u32::<LittleEndian>()?;
        let major_version = cursor.read_u16::<LittleEndian>()?;
        let minor_version = cursor.read_u16::<LittleEndian>()?;

        if major_version == MAJOR_VERSION as u16 && minor_version == MINOR_VERSION as u16 {
            return Err(Error::version_conflict(
                "Attempt to use the lance v2 reader to read a legacy file".to_string(),
                major_version,
                minor_version,
                location!(),
            ));
        }

        let magic_bytes = footer_bytes.slice(len - 4..);
        if magic_bytes.as_ref() != MAGIC {
            return Err(Error::io(
                format!(
                    "file does not appear to be a Lance file (invalid magic: {:?})",
                    MAGIC
                ),
                location!(),
            ));
        }
        Ok(Footer {
            column_meta_start,
            column_meta_offsets_start,
            global_buff_offsets_start,
            num_global_buffers,
            num_columns,
            major_version,
            minor_version,
        })
    }

    // TODO: Once we have coalesced I/O we should only read the column metadatas that we need
    fn read_all_column_metadata(
        column_metadata_bytes: Bytes,
        footer: &Footer,
    ) -> Result<Vec<pbfile::ColumnMetadata>> {
        let column_metadata_start = footer.column_meta_start;
        // cmo == column_metadata_offsets
        let cmo_table_size = 16 * footer.num_columns as usize;
        let cmo_table = column_metadata_bytes.slice(column_metadata_bytes.len() - cmo_table_size..);

        (0..footer.num_columns)
            .map(|col_idx| {
                let offset = (col_idx * 16) as usize;
                let position = LittleEndian::read_u64(&cmo_table[offset..offset + 8]);
                let length = LittleEndian::read_u64(&cmo_table[offset + 8..offset + 16]);
                let normalized_position = (position - column_metadata_start) as usize;
                let normalized_end = normalized_position + (length as usize);
                Ok(pbfile::ColumnMetadata::decode(
                    &column_metadata_bytes[normalized_position..normalized_end],
                )?)
            })
            .collect::<Result<Vec<_>>>()
    }

    async fn optimistic_tail_read(
        data: &Bytes,
        start_pos: u64,
        scheduler: &FileScheduler,
        file_len: u64,
    ) -> Result<Bytes> {
        let num_bytes_needed = (file_len - start_pos) as usize;
        if data.len() >= num_bytes_needed {
            Ok(data.slice((data.len() - num_bytes_needed)..))
        } else {
            let num_bytes_missing = (num_bytes_needed - data.len()) as u64;
            let start = file_len - num_bytes_needed as u64;
            let missing_bytes = scheduler
                .submit_single(start..start + num_bytes_missing, 0)
                .await?;
            let mut combined = BytesMut::with_capacity(data.len() + num_bytes_missing as usize);
            combined.extend(missing_bytes);
            combined.extend(data);
            Ok(combined.freeze())
        }
    }

    fn do_decode_gbo_table(
        gbo_bytes: &Bytes,
        footer: &Footer,
        version: LanceFileVersion,
    ) -> Result<Vec<BufferDescriptor>> {
        let mut global_bufs_cursor = Cursor::new(gbo_bytes);

        let mut global_buffers = Vec::with_capacity(footer.num_global_buffers as usize);
        for _ in 0..footer.num_global_buffers {
            let buf_pos = global_bufs_cursor.read_u64::<LittleEndian>()?;
            assert!(
                version < LanceFileVersion::V2_1 || buf_pos % PAGE_BUFFER_ALIGNMENT as u64 == 0
            );
            let buf_size = global_bufs_cursor.read_u64::<LittleEndian>()?;
            global_buffers.push(BufferDescriptor {
                position: buf_pos,
                size: buf_size,
            });
        }

        Ok(global_buffers)
    }

    async fn decode_gbo_table(
        tail_bytes: &Bytes,
        file_len: u64,
        scheduler: &FileScheduler,
        footer: &Footer,
        version: LanceFileVersion,
    ) -> Result<Vec<BufferDescriptor>> {
        // This could, in theory, trigger another IOP but the GBO table should never be large
        // enough for that to happen
        let gbo_bytes = Self::optimistic_tail_read(
            tail_bytes,
            footer.global_buff_offsets_start,
            scheduler,
            file_len,
        )
        .await?;
        Self::do_decode_gbo_table(&gbo_bytes, footer, version)
    }

    fn decode_schema(schema_bytes: Bytes) -> Result<(u64, lance_core::datatypes::Schema)> {
        let file_descriptor = pb::FileDescriptor::decode(schema_bytes)?;
        let pb_schema = file_descriptor.schema.unwrap();
        let num_rows = file_descriptor.length;
        let fields_with_meta = FieldsWithMeta {
            fields: Fields(pb_schema.fields),
            metadata: pb_schema.metadata,
        };
        let schema = lance_core::datatypes::Schema::from(fields_with_meta);
        Ok((num_rows, schema))
    }

    // TODO: Support late projection.  Currently, if we want to perform a
    // projected read of a file, we load all of the column metadata, and then
    // only read the column data that is requested.  This is fine for most cases.
    //
    // However, if there are many columns then loading all of the column metadata
    // may be expensive.  We should support a mode where we only load the column
    // metadata for the columns that are requested (the file format supports this).
    //
    // The main challenge is that we either need to ignore the column metadata cache
    // or have a more sophisticated cache that can cache per-column metadata.
    //
    // Also, if the number of columns is fairly small, it's faster to read them as a
    // single IOP, but we can fix this through coalescing.
    pub async fn read_all_metadata(scheduler: &FileScheduler) -> Result<CachedFileMetadata> {
        // 1. read the footer
        let (tail_bytes, file_len) = Self::read_tail(scheduler).await?;
        let footer = Self::decode_footer(&tail_bytes)?;

        let file_version = LanceFileVersion::try_from_major_minor(
            footer.major_version as u32,
            footer.minor_version as u32,
        )?;

        let gbo_table =
            Self::decode_gbo_table(&tail_bytes, file_len, scheduler, &footer, file_version).await?;
        if gbo_table.is_empty() {
            return Err(Error::Internal {
                message: "File did not contain any global buffers, schema expected".to_string(),
                location: location!(),
            });
        }
        let schema_start = gbo_table[0].position;
        let schema_size = gbo_table[0].size;

        let num_footer_bytes = file_len - schema_start;

        // By default we read all column metadatas.  We do NOT read the column metadata buffers
        // at this point.  We only want to read the column metadata for columns we are actually loading.
        let all_metadata_bytes =
            Self::optimistic_tail_read(&tail_bytes, schema_start, scheduler, file_len).await?;

        let schema_bytes = all_metadata_bytes.slice(0..schema_size as usize);
        let (num_rows, schema) = Self::decode_schema(schema_bytes)?;

        // Next, read the metadata for the columns
        // This is both the column metadata and the CMO table
        let column_metadata_start = (footer.column_meta_start - schema_start) as usize;
        let column_metadata_end = (footer.global_buff_offsets_start - schema_start) as usize;
        let column_metadata_bytes =
            all_metadata_bytes.slice(column_metadata_start..column_metadata_end);
        let column_metadatas = Self::read_all_column_metadata(column_metadata_bytes, &footer)?;

        let num_global_buffer_bytes = gbo_table.iter().map(|buf| buf.size).sum::<u64>();
        let num_data_bytes = footer.column_meta_start - num_global_buffer_bytes;
        let num_column_metadata_bytes = footer.global_buff_offsets_start - footer.column_meta_start;

        let column_infos = Self::meta_to_col_infos(column_metadatas.as_slice(), file_version);

        Ok(CachedFileMetadata {
            file_schema: Arc::new(schema),
            column_metadatas,
            column_infos,
            num_rows,
            num_data_bytes,
            num_column_metadata_bytes,
            num_global_buffer_bytes,
            num_footer_bytes,
            file_buffers: gbo_table,
            major_version: footer.major_version,
            minor_version: footer.minor_version,
        })
    }

    fn fetch_encoding<M: Default + Name + Sized>(encoding: &pbfile::Encoding) -> M {
        match &encoding.location {
            Some(pbfile::encoding::Location::Indirect(_)) => todo!(),
            Some(pbfile::encoding::Location::Direct(encoding)) => {
                let encoding_buf = Bytes::from(encoding.encoding.clone());
                let encoding_any = prost_types::Any::decode(encoding_buf).unwrap();
                encoding_any.to_msg::<M>().unwrap()
            }
            Some(pbfile::encoding::Location::None(_)) => panic!(),
            None => panic!(),
        }
    }

    fn meta_to_col_infos(
        column_metadatas: &[pbfile::ColumnMetadata],
        file_version: LanceFileVersion,
    ) -> Vec<Arc<ColumnInfo>> {
        column_metadatas
            .iter()
            .enumerate()
            .map(|(col_idx, col_meta)| {
                let page_infos = col_meta
                    .pages
                    .iter()
                    .map(|page| {
                        let num_rows = page.length;
                        let encoding = match file_version {
                            LanceFileVersion::V2_0 => {
                                PageEncoding::Legacy(Self::fetch_encoding::<pbenc::ArrayEncoding>(
                                    page.encoding.as_ref().unwrap(),
                                ))
                            }
                            _ => {
                                PageEncoding::Structural(Self::fetch_encoding::<pbenc::PageLayout>(
                                    page.encoding.as_ref().unwrap(),
                                ))
                            }
                        };
                        let buffer_offsets_and_sizes = Arc::from(
                            page.buffer_offsets
                                .iter()
                                .zip(page.buffer_sizes.iter())
                                .map(|(offset, size)| {
                                    // Starting with version 2.1 we can assert that page buffers are aligned
                                    assert!(
                                        file_version < LanceFileVersion::V2_1
                                            || offset % PAGE_BUFFER_ALIGNMENT as u64 == 0
                                    );
                                    (*offset, *size)
                                })
                                .collect::<Vec<_>>(),
                        );
                        PageInfo {
                            buffer_offsets_and_sizes,
                            encoding,
                            num_rows,
                            priority: page.priority,
                        }
                    })
                    .collect::<Vec<_>>();
                let buffer_offsets_and_sizes = Arc::from(
                    col_meta
                        .buffer_offsets
                        .iter()
                        .zip(col_meta.buffer_sizes.iter())
                        .map(|(offset, size)| (*offset, *size))
                        .collect::<Vec<_>>(),
                );
                Arc::new(ColumnInfo {
                    index: col_idx as u32,
                    page_infos: Arc::from(page_infos),
                    buffer_offsets_and_sizes,
                    encoding: Self::fetch_encoding(col_meta.encoding.as_ref().unwrap()),
                })
            })
            .collect::<Vec<_>>()
    }

    fn validate_projection(
        projection: &ReaderProjection,
        metadata: &CachedFileMetadata,
    ) -> Result<()> {
        if projection.schema.fields.is_empty() {
            return Err(Error::invalid_input(
                "Attempt to read zero columns from the file, at least one column must be specified"
                    .to_string(),
                location!(),
            ));
        }
        let mut column_indices_seen = BTreeSet::new();
        for column_index in &projection.column_indices {
            if !column_indices_seen.insert(*column_index) {
                return Err(Error::invalid_input(
                    format!(
                        "The projection specified the column index {} more than once",
                        column_index
                    ),
                    location!(),
                ));
            }
            if *column_index >= metadata.column_infos.len() as u32 {
                return Err(Error::invalid_input(format!("The projection specified the column index {} but there are only {} columns in the file", column_index, metadata.column_infos.len()), location!()));
            }
        }
        Ok(())
    }

    /// Opens a new file reader without any pre-existing knowledge
    ///
    /// This will read the file schema from the file itself and thus requires a bit more I/O
    ///
    /// A `base_projection` can also be provided.  If provided, then the projection will apply
    /// to all reads from the file that do not specify their own projection.
    pub async fn try_open(
        scheduler: FileScheduler,
        base_projection: Option<ReaderProjection>,
        decoder_strategy: Arc<DecoderPlugins>,
        cache: &FileMetadataCache,
        options: FileReaderOptions,
    ) -> Result<Self> {
        let file_metadata = Arc::new(Self::read_all_metadata(&scheduler).await?);
        Self::try_open_with_file_metadata(
            scheduler,
            base_projection,
            decoder_strategy,
            file_metadata,
            cache,
            options,
        )
        .await
    }

    /// Same as `try_open` but with the file metadata already loaded.
    pub async fn try_open_with_file_metadata(
        scheduler: FileScheduler,
        base_projection: Option<ReaderProjection>,
        decoder_plugins: Arc<DecoderPlugins>,
        file_metadata: Arc<CachedFileMetadata>,
        cache: &FileMetadataCache,
        options: FileReaderOptions,
    ) -> Result<Self> {
        let cache = Arc::new(cache.with_base_path(scheduler.reader().path().clone()));

        if let Some(base_projection) = base_projection.as_ref() {
            Self::validate_projection(base_projection, &file_metadata)?;
        }
        let num_rows = file_metadata.num_rows;
        Ok(Self {
            scheduler: Arc::new(LanceEncodingsIo(scheduler)),
            base_projection: base_projection.unwrap_or(ReaderProjection::from_whole_schema(
                file_metadata.file_schema.as_ref(),
                file_metadata.version(),
            )),
            num_rows,
            metadata: file_metadata,
            decoder_plugins,
            cache,
            options,
        })
    }

    // The actual decoder needs all the column infos that make up a type.  In other words, if
    // the first type in the schema is Struct<i32, i32> then the decoder will need 3 column infos.
    //
    // This is a file reader concern because the file reader needs to support late projection of columns
    // and so it will need to figure this out anyways.
    //
    // It's a bit of a tricky process though because the number of column infos may depend on the
    // encoding.  Considering the above example, if we wrote it with a packed encoding, then there would
    // only be a single column in the file (and not 3).
    //
    // At the moment this method words because our rules are simple and we just repeat them here.  See
    // Self::default_projection for a similar problem.  In the future this is something the encodings
    // registry will need to figure out.
    fn collect_columns_from_projection(
        &self,
        _projection: &ReaderProjection,
    ) -> Result<Vec<Arc<ColumnInfo>>> {
        Ok(self.metadata.column_infos.to_vec())
    }

    #[allow(clippy::too_many_arguments)]
    fn do_read_range(
        column_infos: Vec<Arc<ColumnInfo>>,
        io: Arc<dyn EncodingsIo>,
        cache: Arc<FileMetadataCache>,
        num_rows: u64,
        decoder_plugins: Arc<DecoderPlugins>,
        range: Range<u64>,
        batch_size: u32,
        projection: ReaderProjection,
        filter: FilterExpression,
        should_validate: bool,
    ) -> Result<BoxStream<'static, ReadBatchTask>> {
        debug!(
            "Reading range {:?} with batch_size {} from file with {} rows and {} columns into schema with {} columns",
            range,
            batch_size,
            num_rows,
            column_infos.len(),
            projection.schema.fields.len(),
        );

        let config = SchedulerDecoderConfig {
            batch_size,
            cache,
            decoder_plugins,
            io,
            should_validate,
        };

        let requested_rows = RequestedRows::Ranges(vec![range]);

        Ok(schedule_and_decode(
            column_infos,
            requested_rows,
            filter,
            projection.column_indices,
            projection.schema,
            config,
        ))
    }

    fn read_range(
        &self,
        range: Range<u64>,
        batch_size: u32,
        projection: ReaderProjection,
        filter: FilterExpression,
    ) -> Result<BoxStream<'static, ReadBatchTask>> {
        // Create and initialize the stream
        Self::do_read_range(
            self.collect_columns_from_projection(&projection)?,
            self.scheduler.clone(),
            self.cache.clone(),
            self.num_rows,
            self.decoder_plugins.clone(),
            range,
            batch_size,
            projection,
            filter,
            self.options.validate_on_decode,
        )
    }

    #[allow(clippy::too_many_arguments)]
    fn do_take_rows(
        column_infos: Vec<Arc<ColumnInfo>>,
        io: Arc<dyn EncodingsIo>,
        cache: Arc<FileMetadataCache>,
        decoder_plugins: Arc<DecoderPlugins>,
        indices: Vec<u64>,
        batch_size: u32,
        projection: ReaderProjection,
        filter: FilterExpression,
        should_validate: bool,
    ) -> Result<BoxStream<'static, ReadBatchTask>> {
        debug!(
            "Taking {} rows spread across range {}..{} with batch_size {} from columns {:?}",
            indices.len(),
            indices[0],
            indices[indices.len() - 1],
            batch_size,
            column_infos.iter().map(|ci| ci.index).collect::<Vec<_>>()
        );

        let config = SchedulerDecoderConfig {
            batch_size,
            cache,
            decoder_plugins,
            io,
            should_validate,
        };

        let requested_rows = RequestedRows::Indices(indices);

        Ok(schedule_and_decode(
            column_infos,
            requested_rows,
            filter,
            projection.column_indices,
            projection.schema,
            config,
        ))
    }

    fn take_rows(
        &self,
        indices: Vec<u64>,
        batch_size: u32,
        projection: ReaderProjection,
    ) -> Result<BoxStream<'static, ReadBatchTask>> {
        // Create and initialize the stream
        Self::do_take_rows(
            self.collect_columns_from_projection(&projection)?,
            self.scheduler.clone(),
            self.cache.clone(),
            self.decoder_plugins.clone(),
            indices,
            batch_size,
            projection,
            FilterExpression::no_filter(),
            self.options.validate_on_decode,
        )
    }

    /// Creates a stream of "read tasks" to read the data from the file
    ///
    /// The arguments are similar to [`Self::read_stream_projected`] but instead of returning a stream
    /// of record batches it returns a stream of "read tasks".
    ///
    /// The tasks should be consumed with some kind of `buffered` argument if CPU parallelism is desired.
    ///
    /// Note that "read task" is probably a bit imprecise.  The tasks are actually "decode tasks".  The
    /// reading happens asynchronously in the background.  In other words, a single read task may map to
    /// multiple I/O operations or a single I/O operation may map to multiple read tasks.
    pub fn read_tasks(
        &self,
        params: ReadBatchParams,
        batch_size: u32,
        projection: Option<ReaderProjection>,
        filter: FilterExpression,
    ) -> Result<Pin<Box<dyn Stream<Item = ReadBatchTask> + Send>>> {
        let projection = projection.unwrap_or_else(|| self.base_projection.clone());
        Self::validate_projection(&projection, &self.metadata)?;
        let verify_bound = |params: &ReadBatchParams, bound: u64, inclusive: bool| {
            if bound > self.num_rows || bound == self.num_rows && inclusive {
                Err(Error::invalid_input(
                    format!(
                        "cannot read {:?} from file with {} rows",
                        params, self.num_rows
                    ),
                    location!(),
                ))
            } else {
                Ok(())
            }
        };
        match &params {
            ReadBatchParams::Indices(indices) => {
                for idx in indices {
                    match idx {
                        None => {
                            return Err(Error::invalid_input(
                                "Null value in indices array",
                                location!(),
                            ));
                        }
                        Some(idx) => {
                            verify_bound(&params, idx as u64, true)?;
                        }
                    }
                }
                let indices = indices.iter().map(|idx| idx.unwrap() as u64).collect();
                self.take_rows(indices, batch_size, projection)
            }
            ReadBatchParams::Range(range) => {
                verify_bound(&params, range.end as u64, false)?;
                self.read_range(
                    range.start as u64..range.end as u64,
                    batch_size,
                    projection,
                    filter,
                )
            }
            ReadBatchParams::RangeFrom(range) => {
                verify_bound(&params, range.start as u64, true)?;
                self.read_range(
                    range.start as u64..self.num_rows,
                    batch_size,
                    projection,
                    filter,
                )
            }
            ReadBatchParams::RangeTo(range) => {
                verify_bound(&params, range.end as u64, false)?;
                self.read_range(0..range.end as u64, batch_size, projection, filter)
            }
            ReadBatchParams::RangeFull => {
                self.read_range(0..self.num_rows, batch_size, projection, filter)
            }
        }
    }

    /// Reads data from the file as a stream of record batches
    ///
    /// * `params` - Specifies the range (or indices) of data to read
    /// * `batch_size` - The maximum size of a single batch.  A batch may be smaller
    ///   if it is the last batch or if it is not possible to create a batch of the
    ///   requested size.
    ///
    ///   For example, if the batch size is 1024 and one of the columns is a string
    ///   column then there may be some ranges of 1024 rows that contain more than
    ///   2^31 bytes of string data (which is the maximum size of a string column
    ///   in Arrow).  In this case smaller batches may be emitted.
    /// * `batch_readahead` - The number of batches to read ahead.  This controls the
    ///   amount of CPU parallelism of the read.  In other words it controls how many
    ///   batches will be decoded in parallel.  It has no effect on the I/O parallelism
    ///   of the read (how many I/O requests are in flight at once).
    ///
    ///   This parameter also is also related to backpressure.  If the consumer of the
    ///   stream is slow then the reader will build up RAM.
    /// * `projection` - A projection to apply to the read.  This controls which columns
    ///   are read from the file.  The projection is NOT applied on top of the base
    ///   projection.  The projection is applied directly to the file schema.
    pub fn read_stream_projected(
        &self,
        params: ReadBatchParams,
        batch_size: u32,
        batch_readahead: u32,
        projection: ReaderProjection,
        filter: FilterExpression,
    ) -> Result<Pin<Box<dyn RecordBatchStream>>> {
        let arrow_schema = Arc::new(ArrowSchema::from(projection.schema.as_ref()));
        let tasks_stream = self.read_tasks(params, batch_size, Some(projection), filter)?;
        let batch_stream = tasks_stream
            .map(|task| task.task)
            .buffered(batch_readahead as usize)
            .boxed();
        Ok(Box::pin(RecordBatchStreamAdapter::new(
            arrow_schema,
            batch_stream,
        )))
    }

    /// Reads data from the file as a stream of record batches
    ///
    /// This is similar to [`Self::read_stream_projected`] but uses the base projection
    /// provided when the file was opened (or reads all columns if the file was
    /// opened without a base projection)
    pub fn read_stream(
        &self,
        params: ReadBatchParams,
        batch_size: u32,
        batch_readahead: u32,
        filter: FilterExpression,
    ) -> Result<Pin<Box<dyn RecordBatchStream>>> {
        self.read_stream_projected(
            params,
            batch_size,
            batch_readahead,
            self.base_projection.clone(),
            filter,
        )
    }

    pub fn schema(&self) -> &Arc<Schema> {
        &self.metadata.file_schema
    }
}

/// Inspects a page and returns a String describing the page's encoding
pub fn describe_encoding(page: &pbfile::column_metadata::Page) -> String {
    if let Some(encoding) = &page.encoding {
        if let Some(style) = &encoding.location {
            match style {
                pbfile::encoding::Location::Indirect(indirect) => {
                    format!(
                        "IndirectEncoding(pos={},size={})",
                        indirect.buffer_location, indirect.buffer_length
                    )
                }
                pbfile::encoding::Location::Direct(direct) => {
                    let encoding_any =
                        prost_types::Any::decode(Bytes::from(direct.encoding.clone()))
                            .expect("failed to deserialize encoding as protobuf");
                    if encoding_any.type_url == "/lance.encodings.ArrayEncoding" {
                        let encoding = encoding_any.to_msg::<pbenc::ArrayEncoding>();
                        match encoding {
                            Ok(encoding) => {
                                format!("{:#?}", encoding)
                            }
                            Err(err) => {
                                format!("Unsupported(decode_err={})", err)
                            }
                        }
                    } else {
                        format!("Unrecognized(type_url={})", encoding_any.type_url)
                    }
                }
                pbfile::encoding::Location::None(_) => "NoEncodingDescription".to_string(),
            }
        } else {
            "MISSING STYLE".to_string()
        }
    } else {
        "MISSING".to_string()
    }
}

pub trait EncodedBatchReaderExt {
    fn try_from_mini_lance(
        bytes: Bytes,
        schema: &Schema,
        version: LanceFileVersion,
    ) -> Result<Self>
    where
        Self: Sized;
    fn try_from_self_described_lance(bytes: Bytes) -> Result<Self>
    where
        Self: Sized;
}

impl EncodedBatchReaderExt for EncodedBatch {
    fn try_from_mini_lance(
        bytes: Bytes,
        schema: &Schema,
        file_version: LanceFileVersion,
    ) -> Result<Self>
    where
        Self: Sized,
    {
        let projection = ReaderProjection::from_whole_schema(schema, file_version);
        let footer = FileReader::decode_footer(&bytes)?;

        // Next, read the metadata for the columns
        // This is both the column metadata and the CMO table
        let column_metadata_start = footer.column_meta_start as usize;
        let column_metadata_end = footer.global_buff_offsets_start as usize;
        let column_metadata_bytes = bytes.slice(column_metadata_start..column_metadata_end);
        let column_metadatas =
            FileReader::read_all_column_metadata(column_metadata_bytes, &footer)?;

        let file_version = LanceFileVersion::try_from_major_minor(
            footer.major_version as u32,
            footer.minor_version as u32,
        )?;

        let page_table = FileReader::meta_to_col_infos(&column_metadatas, file_version);

        Ok(Self {
            data: bytes,
            num_rows: page_table
                .first()
                .map(|col| col.page_infos.iter().map(|page| page.num_rows).sum::<u64>())
                .unwrap_or(0),
            page_table,
            top_level_columns: projection.column_indices,
            schema: Arc::new(schema.clone()),
        })
    }

    fn try_from_self_described_lance(bytes: Bytes) -> Result<Self>
    where
        Self: Sized,
    {
        let footer = FileReader::decode_footer(&bytes)?;
        let file_version = LanceFileVersion::try_from_major_minor(
            footer.major_version as u32,
            footer.minor_version as u32,
        )?;

        let gbo_table = FileReader::do_decode_gbo_table(
            &bytes.slice(footer.global_buff_offsets_start as usize..),
            &footer,
            file_version,
        )?;
        if gbo_table.is_empty() {
            return Err(Error::Internal {
                message: "File did not contain any global buffers, schema expected".to_string(),
                location: location!(),
            });
        }
        let schema_start = gbo_table[0].position as usize;
        let schema_size = gbo_table[0].size as usize;

        let schema_bytes = bytes.slice(schema_start..(schema_start + schema_size));
        let (_, schema) = FileReader::decode_schema(schema_bytes)?;
        let projection = ReaderProjection::from_whole_schema(&schema, file_version);

        // Next, read the metadata for the columns
        // This is both the column metadata and the CMO table
        let column_metadata_start = footer.column_meta_start as usize;
        let column_metadata_end = footer.global_buff_offsets_start as usize;
        let column_metadata_bytes = bytes.slice(column_metadata_start..column_metadata_end);
        let column_metadatas =
            FileReader::read_all_column_metadata(column_metadata_bytes, &footer)?;

        let page_table = FileReader::meta_to_col_infos(&column_metadatas, file_version);

        Ok(Self {
            data: bytes,
            num_rows: page_table
                .first()
                .map(|col| col.page_infos.iter().map(|page| page.num_rows).sum::<u64>())
                .unwrap_or(0),
            page_table,
            top_level_columns: projection.column_indices,
            schema: Arc::new(schema),
        })
    }
}

#[cfg(test)]
pub mod tests {
    use std::{collections::BTreeMap, pin::Pin, sync::Arc};

    use arrow_array::{
        types::{Float64Type, Int32Type},
        RecordBatch,
    };
    use arrow_schema::{DataType, Field, Fields, Schema as ArrowSchema};
    use bytes::Bytes;
    use futures::{prelude::stream::TryStreamExt, StreamExt};
    use lance_arrow::RecordBatchExt;
    use lance_core::datatypes::Schema;
    use lance_datagen::{array, gen, BatchCount, ByteCount, RowCount};
    use lance_encoding::{
        decoder::{decode_batch, DecodeBatchScheduler, DecoderPlugins, FilterExpression},
        encoder::{encode_batch, CoreFieldEncodingStrategy, EncodedBatch, EncodingOptions},
        version::LanceFileVersion,
    };
    use lance_io::stream::RecordBatchStream;
    use log::debug;
    use tokio::sync::mpsc;

    use crate::v2::{
        reader::{EncodedBatchReaderExt, FileReader, FileReaderOptions, ReaderProjection},
        testing::{test_cache, write_lance_file, FsFixture, WrittenFile},
        writer::{EncodedBatchWriteExt, FileWriter, FileWriterOptions},
    };

    async fn create_some_file(fs: &FsFixture) -> WrittenFile {
        let location_type = DataType::Struct(Fields::from(vec![
            Field::new("x", DataType::Float64, true),
            Field::new("y", DataType::Float64, true),
        ]));
        let categories_type = DataType::List(Arc::new(Field::new("item", DataType::Utf8, true)));

        let reader = gen()
            .col("score", array::rand::<Float64Type>())
            .col("location", array::rand_type(&location_type))
            .col("categories", array::rand_type(&categories_type))
            .col("binary", array::rand_type(&DataType::Binary))
            .col("large_bin", array::rand_type(&DataType::LargeBinary))
            .into_reader_rows(RowCount::from(1000), BatchCount::from(100));

        write_lance_file(reader, fs, FileWriterOptions::default()).await
    }

    type Transformer = Box<dyn Fn(&RecordBatch) -> RecordBatch>;

    async fn verify_expected(
        expected: &[RecordBatch],
        mut actual: Pin<Box<dyn RecordBatchStream>>,
        read_size: u32,
        transform: Option<Transformer>,
    ) {
        let mut remaining = expected.iter().map(|batch| batch.num_rows()).sum::<usize>() as u32;
        let mut expected_iter = expected.iter().map(|batch| {
            if let Some(transform) = &transform {
                transform(batch)
            } else {
                batch.clone()
            }
        });
        let mut next_expected = expected_iter.next().unwrap().clone();
        while let Some(actual) = actual.next().await {
            let mut actual = actual.unwrap();
            let mut rows_to_verify = actual.num_rows() as u32;
            let expected_length = remaining.min(read_size);
            assert_eq!(expected_length, rows_to_verify);

            while rows_to_verify > 0 {
                let next_slice_len = (next_expected.num_rows() as u32).min(rows_to_verify);
                assert_eq!(
                    next_expected.slice(0, next_slice_len as usize),
                    actual.slice(0, next_slice_len as usize)
                );
                remaining -= next_slice_len;
                rows_to_verify -= next_slice_len;
                if remaining > 0 {
                    if next_slice_len == next_expected.num_rows() as u32 {
                        next_expected = expected_iter.next().unwrap().clone();
                    } else {
                        next_expected = next_expected.slice(
                            next_slice_len as usize,
                            next_expected.num_rows() - next_slice_len as usize,
                        );
                    }
                }
                if rows_to_verify > 0 {
                    actual = actual.slice(
                        next_slice_len as usize,
                        actual.num_rows() - next_slice_len as usize,
                    );
                }
            }
        }
        assert_eq!(remaining, 0);
    }

    #[tokio::test]
    async fn test_round_trip() {
        let fs = FsFixture::default();

        let WrittenFile { data, .. } = create_some_file(&fs).await;

        for read_size in [32, 1024, 1024 * 1024] {
            let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
            let file_reader = FileReader::try_open(
                file_scheduler,
                None,
                Arc::<DecoderPlugins>::default(),
                &test_cache(),
                FileReaderOptions::default(),
            )
            .await
            .unwrap();

            let schema = file_reader.schema();
            assert_eq!(schema.metadata.get("foo").unwrap(), "bar");

            let batch_stream = file_reader
                .read_stream(
                    lance_io::ReadBatchParams::RangeFull,
                    read_size,
                    16,
                    FilterExpression::no_filter(),
                )
                .unwrap();

            verify_expected(&data, batch_stream, read_size, None).await;
        }
    }

    #[test_log::test(tokio::test)]
    async fn test_encoded_batch_round_trip() {
        let data = gen()
            .col("x", array::rand::<Int32Type>())
            .col("y", array::rand_utf8(ByteCount::from(16), false))
            .into_batch_rows(RowCount::from(10000))
            .unwrap();

        let lance_schema = Arc::new(Schema::try_from(data.schema().as_ref()).unwrap());

        let encoding_options = EncodingOptions {
            cache_bytes_per_column: 4096,
            max_page_bytes: 32 * 1024 * 1024,
            keep_original_array: true,
            buffer_alignment: 64,
        };
        let encoded_batch = encode_batch(
            &data,
            lance_schema.clone(),
            &CoreFieldEncodingStrategy::default(),
            &encoding_options,
        )
        .await
        .unwrap();

        // Test self described
        let bytes = encoded_batch.try_to_self_described_lance().unwrap();

        let decoded_batch = EncodedBatch::try_from_self_described_lance(bytes).unwrap();

        let decoded = decode_batch(
            &decoded_batch,
            &FilterExpression::no_filter(),
            Arc::<DecoderPlugins>::default(),
            false,
        )
        .await
        .unwrap();

        assert_eq!(data, decoded);

        // Test mini
        let bytes = encoded_batch.try_to_mini_lance().unwrap();
        let decoded_batch =
            EncodedBatch::try_from_mini_lance(bytes, lance_schema.as_ref(), LanceFileVersion::V2_0)
                .unwrap();
        let decoded = decode_batch(
            &decoded_batch,
            &FilterExpression::no_filter(),
            Arc::<DecoderPlugins>::default(),
            false,
        )
        .await
        .unwrap();

        assert_eq!(data, decoded);
    }

    #[test_log::test(tokio::test)]
    async fn test_projection() {
        let fs = FsFixture::default();

        let written_file = create_some_file(&fs).await;
        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();

        let field_id_mapping = written_file
            .field_id_mapping
            .iter()
            .copied()
            .collect::<BTreeMap<_, _>>();

        for columns in [
            vec!["score"],
            vec!["location"],
            vec!["categories"],
            vec!["score.x"],
            vec!["score", "categories"],
            vec!["score", "location"],
            vec!["location", "categories"],
            vec!["score.y", "location", "categories"],
        ] {
            debug!("Testing round trip with projection {:?}", columns);
            // We can specify the projection as part of the read operation via read_stream_projected
            let file_reader = FileReader::try_open(
                file_scheduler.clone(),
                None,
                Arc::<DecoderPlugins>::default(),
                &test_cache(),
                FileReaderOptions::default(),
            )
            .await
            .unwrap();

            let projected_schema = written_file.schema.project(&columns).unwrap();
            let projection = ReaderProjection::from_field_ids(
                &file_reader,
                &projected_schema,
                &field_id_mapping,
            )
            .unwrap();

            let batch_stream = file_reader
                .read_stream_projected(
                    lance_io::ReadBatchParams::RangeFull,
                    1024,
                    16,
                    projection.clone(),
                    FilterExpression::no_filter(),
                )
                .unwrap();

            let projection_arrow = ArrowSchema::from(projection.schema.as_ref());
            verify_expected(
                &written_file.data,
                batch_stream,
                1024,
                Some(Box::new(move |batch: &RecordBatch| {
                    batch.project_by_schema(&projection_arrow).unwrap()
                })),
            )
            .await;

            // We can also specify the projection as a base projection when we open the file
            let file_reader = FileReader::try_open(
                file_scheduler.clone(),
                Some(projection.clone()),
                Arc::<DecoderPlugins>::default(),
                &test_cache(),
                FileReaderOptions::default(),
            )
            .await
            .unwrap();

            let batch_stream = file_reader
                .read_stream(
                    lance_io::ReadBatchParams::RangeFull,
                    1024,
                    16,
                    FilterExpression::no_filter(),
                )
                .unwrap();

            let projection_arrow = ArrowSchema::from(projection.schema.as_ref());
            verify_expected(
                &written_file.data,
                batch_stream,
                1024,
                Some(Box::new(move |batch: &RecordBatch| {
                    batch.project_by_schema(&projection_arrow).unwrap()
                })),
            )
            .await;
        }

        let empty_projection = ReaderProjection {
            column_indices: Vec::default(),
            schema: Arc::new(Schema::default()),
        };

        assert!(FileReader::try_open(
            file_scheduler.clone(),
            Some(empty_projection),
            Arc::<DecoderPlugins>::default(),
            &test_cache(),
            FileReaderOptions::default(),
        )
        .await
        .is_err());

        let arrow_schema = ArrowSchema::new(vec![
            Field::new("x", DataType::Int32, true),
            Field::new("y", DataType::Int32, true),
        ]);
        let schema = Schema::try_from(&arrow_schema).unwrap();

        let projection_with_dupes = ReaderProjection {
            column_indices: vec![0, 0],
            schema: Arc::new(schema),
        };

        assert!(FileReader::try_open(
            file_scheduler.clone(),
            Some(projection_with_dupes),
            Arc::<DecoderPlugins>::default(),
            &test_cache(),
            FileReaderOptions::default(),
        )
        .await
        .is_err());
    }

    #[test_log::test(tokio::test)]
    async fn test_compressing_buffer() {
        let fs = FsFixture::default();

        let written_file = create_some_file(&fs).await;
        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();

        // We can specify the projection as part of the read operation via read_stream_projected
        let file_reader = FileReader::try_open(
            file_scheduler.clone(),
            None,
            Arc::<DecoderPlugins>::default(),
            &test_cache(),
            FileReaderOptions::default(),
        )
        .await
        .unwrap();

        let mut projection = written_file.schema.project(&["score"]).unwrap();
        for field in projection.fields.iter_mut() {
            field
                .metadata
                .insert("lance:compression".to_string(), "zstd".to_string());
        }
        let projection = ReaderProjection {
            column_indices: projection.fields.iter().map(|f| f.id as u32).collect(),
            schema: Arc::new(projection),
        };

        let batch_stream = file_reader
            .read_stream_projected(
                lance_io::ReadBatchParams::RangeFull,
                1024,
                16,
                projection.clone(),
                FilterExpression::no_filter(),
            )
            .unwrap();

        let projection_arrow = Arc::new(ArrowSchema::from(projection.schema.as_ref()));
        verify_expected(
            &written_file.data,
            batch_stream,
            1024,
            Some(Box::new(move |batch: &RecordBatch| {
                batch.project_by_schema(&projection_arrow).unwrap()
            })),
        )
        .await;
    }

    #[tokio::test]
    async fn test_read_all() {
        let fs = FsFixture::default();
        let WrittenFile { data, .. } = create_some_file(&fs).await;
        let total_rows = data.iter().map(|batch| batch.num_rows()).sum::<usize>();

        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
        let file_reader = FileReader::try_open(
            file_scheduler.clone(),
            None,
            Arc::<DecoderPlugins>::default(),
            &test_cache(),
            FileReaderOptions::default(),
        )
        .await
        .unwrap();

        let batches = file_reader
            .read_stream(
                lance_io::ReadBatchParams::RangeFull,
                total_rows as u32,
                16,
                FilterExpression::no_filter(),
            )
            .unwrap()
            .try_collect::<Vec<_>>()
            .await
            .unwrap();
        assert_eq!(batches.len(), 1);
        assert_eq!(batches[0].num_rows(), total_rows);
    }

    #[tokio::test(flavor = "multi_thread")]
    async fn test_drop_in_progress() {
        let fs = FsFixture::default();
        let WrittenFile { data, .. } = create_some_file(&fs).await;
        let total_rows = data.iter().map(|batch| batch.num_rows()).sum::<usize>();

        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
        let file_reader = FileReader::try_open(
            file_scheduler.clone(),
            None,
            Arc::<DecoderPlugins>::default(),
            &test_cache(),
            FileReaderOptions::default(),
        )
        .await
        .unwrap();

        let mut batches = file_reader
            .read_stream(
                lance_io::ReadBatchParams::RangeFull,
                (total_rows / 10) as u32,
                16,
                FilterExpression::no_filter(),
            )
            .unwrap();

        drop(file_reader);

        let batch = batches.next().await.unwrap().unwrap();
        assert!(batch.num_rows() > 0);

        // Drop in-progress scan
        drop(batches);
    }

    #[tokio::test]
    async fn drop_while_scheduling() {
        // This is a bit of a white-box test, pokes at the internals.  We want to
        // test the case where the read stream is dropped before the scheduling
        // thread finishes.  We can't do that in a black-box fashion because the
        // scheduling thread runs in the background and there is no easy way to
        // pause / gate it.

        // It's a regression for a bug where the scheduling thread would panic
        // if the stream was dropped before it finished.

        let fs = FsFixture::default();
        let written_file = create_some_file(&fs).await;
        let total_rows = written_file
            .data
            .iter()
            .map(|batch| batch.num_rows())
            .sum::<usize>();

        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
        let file_reader = FileReader::try_open(
            file_scheduler.clone(),
            None,
            Arc::<DecoderPlugins>::default(),
            &test_cache(),
            FileReaderOptions::default(),
        )
        .await
        .unwrap();

        let projection =
            ReaderProjection::from_whole_schema(&written_file.schema, LanceFileVersion::V2_0);
        let column_infos = file_reader
            .collect_columns_from_projection(&projection)
            .unwrap();
        let mut decode_scheduler = DecodeBatchScheduler::try_new(
            &projection.schema,
            &projection.column_indices,
            &column_infos,
            &vec![],
            total_rows as u64,
            Arc::<DecoderPlugins>::default(),
            file_reader.scheduler.clone(),
            test_cache(),
            &FilterExpression::no_filter(),
        )
        .await
        .unwrap();

        let range = 0..total_rows as u64;

        let (tx, rx) = mpsc::unbounded_channel();

        // Simulate the stream / decoder being dropped
        drop(rx);

        // Scheduling should not panic
        decode_scheduler.schedule_range(
            range,
            &FilterExpression::no_filter(),
            tx,
            file_reader.scheduler.clone(),
        )
    }

    #[tokio::test]
    async fn test_global_buffers() {
        let fs = FsFixture::default();

        let lance_schema =
            lance_core::datatypes::Schema::try_from(&ArrowSchema::new(vec![Field::new(
                "foo",
                DataType::Int32,
                true,
            )]))
            .unwrap();

        let mut file_writer = FileWriter::try_new(
            fs.object_store.create(&fs.tmp_path).await.unwrap(),
            lance_schema.clone(),
            FileWriterOptions::default(),
        )
        .unwrap();

        let test_bytes = Bytes::from_static(b"hello");

        let buf_index = file_writer
            .add_global_buffer(test_bytes.clone())
            .await
            .unwrap();

        assert_eq!(buf_index, 1);

        file_writer.finish().await.unwrap();

        let file_scheduler = fs.scheduler.open_file(&fs.tmp_path).await.unwrap();
        let file_reader = FileReader::try_open(
            file_scheduler.clone(),
            None,
            Arc::<DecoderPlugins>::default(),
            &test_cache(),
            FileReaderOptions::default(),
        )
        .await
        .unwrap();

        let buf = file_reader.read_global_buffer(1).await.unwrap();
        assert_eq!(buf, test_bytes);
    }
}