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

//! Lance Data File Reader

// Standard
use std::ops::{Range, RangeTo};
use std::sync::Arc;

use arrow_arith::numeric::sub;
use arrow_array::{
    builder::PrimitiveBuilder,
    cast::AsArray,
    types::{Int32Type, Int64Type},
    ArrayRef, ArrowNativeTypeOp, ArrowNumericType, NullArray, OffsetSizeTrait, PrimitiveArray,
    RecordBatch, StructArray, UInt32Array,
};
use arrow_buffer::ArrowNativeType;
use arrow_schema::{DataType, FieldRef, Schema as ArrowSchema};
use arrow_select::concat::{self, concat_batches};
use async_recursion::async_recursion;
use deepsize::DeepSizeOf;
use futures::{stream, Future, FutureExt, StreamExt, TryStreamExt};
use lance_arrow::*;
use lance_core::cache::FileMetadataCache;
use lance_core::datatypes::{Field, Schema};
use lance_core::{Error, Result};
use lance_io::encodings::dictionary::DictionaryDecoder;
use lance_io::encodings::AsyncIndex;
use lance_io::stream::{RecordBatchStream, RecordBatchStreamAdapter};
use lance_io::traits::Reader;
use lance_io::utils::{
    read_fixed_stride_array, read_metadata_offset, read_struct, read_struct_from_buf,
};
use lance_io::{object_store::ObjectStore, ReadBatchParams};

use object_store::path::Path;
use snafu::{location, Location};
use tracing::instrument;

use crate::format::metadata::Metadata;
use crate::page_table::{PageInfo, PageTable};

/// Lance File Reader.
///
/// It reads arrow data from one data file.
#[derive(Clone, DeepSizeOf)]
pub struct FileReader {
    pub object_reader: Arc<dyn Reader>,
    metadata: Arc<Metadata>,
    page_table: Arc<PageTable>,
    schema: Schema,

    /// The id of the fragment which this file belong to.
    /// For simple file access, this can just be zero.
    fragment_id: u64,

    /// Page table for statistics
    stats_page_table: Arc<Option<PageTable>>,
}

impl std::fmt::Debug for FileReader {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("FileReader")
            .field("fragment", &self.fragment_id)
            .field("path", &self.object_reader.path())
            .finish()
    }
}

impl FileReader {
    /// Open file reader
    ///
    /// Open the file at the given path using the provided object store.
    ///
    /// The passed fragment ID determines the first 32-bits of the row IDs.
    ///
    /// If a manifest is passed in, it will be used to load the schema and dictionary.
    /// This is typically done if the file is part of a dataset fragment. If no manifest
    /// is passed in, then it is read from the file itself.
    ///
    /// The session passed in is used to cache metadata about the file. If no session
    /// is passed in, there will be no caching.
    #[instrument(level = "debug", skip(object_store, schema, session))]
    pub async fn try_new_with_fragment_id(
        object_store: &ObjectStore,
        path: &Path,
        schema: Schema,
        fragment_id: u32,
        field_id_offset: i32,
        max_field_id: i32,
        session: Option<&FileMetadataCache>,
    ) -> Result<Self> {
        let object_reader = object_store.open(path).await?;

        let metadata = Self::read_metadata(object_reader.as_ref(), session).await?;

        Self::try_new_from_reader(
            path,
            object_reader.into(),
            Some(metadata),
            schema,
            fragment_id,
            field_id_offset,
            max_field_id,
            session,
        )
        .await
    }

    #[allow(clippy::too_many_arguments)]
    pub async fn try_new_from_reader(
        path: &Path,
        object_reader: Arc<dyn Reader>,
        metadata: Option<Arc<Metadata>>,
        schema: Schema,
        fragment_id: u32,
        field_id_offset: i32,
        max_field_id: i32,
        session: Option<&FileMetadataCache>,
    ) -> Result<Self> {
        let metadata = match metadata {
            Some(metadata) => metadata,
            None => Self::read_metadata(object_reader.as_ref(), session).await?,
        };

        let page_table = async {
            Self::load_from_cache(session, path, |_| async {
                PageTable::load(
                    object_reader.as_ref(),
                    metadata.page_table_position,
                    field_id_offset,
                    max_field_id,
                    metadata.num_batches() as i32,
                )
                .await
            })
            .await
        };

        let stats_page_table = Self::read_stats_page_table(object_reader.as_ref(), session);

        // Can concurrently load page tables
        let (page_table, stats_page_table) = futures::try_join!(page_table, stats_page_table)?;

        Ok(Self {
            object_reader,
            metadata,
            schema,
            page_table,
            fragment_id: fragment_id as u64,
            stats_page_table,
        })
    }

    pub async fn read_metadata(
        object_reader: &dyn Reader,
        cache: Option<&FileMetadataCache>,
    ) -> Result<Arc<Metadata>> {
        Self::load_from_cache(cache, object_reader.path(), |_| async {
            let file_size = object_reader.size().await?;
            let begin = if file_size < object_reader.block_size() {
                0
            } else {
                file_size - object_reader.block_size()
            };
            let tail_bytes = object_reader.get_range(begin..file_size).await?;
            let metadata_pos = read_metadata_offset(&tail_bytes)?;

            let metadata: Metadata = if metadata_pos < file_size - tail_bytes.len() {
                // We have not read the metadata bytes yet.
                read_struct(object_reader, metadata_pos).await?
            } else {
                let offset = tail_bytes.len() - (file_size - metadata_pos);
                read_struct_from_buf(&tail_bytes.slice(offset..))?
            };
            Ok(metadata)
        })
        .await
    }

    /// Get the statistics page table. This will read the metadata if it is not cached.
    ///
    /// The page table is cached.
    async fn read_stats_page_table(
        reader: &dyn Reader,
        cache: Option<&FileMetadataCache>,
    ) -> Result<Arc<Option<PageTable>>> {
        // To prevent collisions, we cache this at a child path
        Self::load_from_cache(cache, &reader.path().child("stats"), |_| async {
            let metadata = Self::read_metadata(reader, cache).await?;

            if let Some(stats_meta) = metadata.stats_metadata.as_ref() {
                Ok(Some(
                    PageTable::load(
                        reader,
                        stats_meta.page_table_position,
                        /*min_field_id=*/ 0,
                        /*max_field_id=*/ *stats_meta.leaf_field_ids.iter().max().unwrap(),
                        /*num_batches=*/ 1,
                    )
                    .await?,
                ))
            } else {
                Ok(None)
            }
        })
        .await
    }

    /// Load some metadata about the fragment from the cache, if there is one.
    async fn load_from_cache<T: DeepSizeOf + Send + Sync + 'static, F, Fut>(
        cache: Option<&FileMetadataCache>,
        path: &Path,
        loader: F,
    ) -> Result<Arc<T>>
    where
        F: Fn(&Path) -> Fut,
        Fut: Future<Output = Result<T>>,
    {
        if let Some(cache) = cache {
            cache.get_or_insert(path, loader).await
        } else {
            Ok(Arc::new(loader(path).await?))
        }
    }

    /// Open one Lance data file for read.
    pub async fn try_new(object_store: &ObjectStore, path: &Path, schema: Schema) -> Result<Self> {
        // If just reading a lance data file we assume the schema is the schema of the data file
        let max_field_id = schema.max_field_id().unwrap_or_default();
        Self::try_new_with_fragment_id(object_store, path, schema, 0, 0, max_field_id, None).await
    }

    fn io_parallelism(&self) -> usize {
        self.object_reader.io_parallelism()
    }

    /// Requested projection of the data in this file, excluding the row id column.
    pub fn schema(&self) -> &Schema {
        &self.schema
    }

    pub fn num_batches(&self) -> usize {
        self.metadata.num_batches()
    }

    /// Get the number of rows in this batch
    pub fn num_rows_in_batch(&self, batch_id: i32) -> usize {
        self.metadata.get_batch_length(batch_id).unwrap_or_default() as usize
    }

    /// Count the number of rows in this file.
    pub fn len(&self) -> usize {
        self.metadata.len()
    }

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

    /// Read a batch of data from the file.
    ///
    /// The schema of the returned [RecordBatch] is set by [`FileReader::schema()`].
    #[instrument(level = "debug", skip(self, params, projection))]
    pub async fn read_batch(
        &self,
        batch_id: i32,
        params: impl Into<ReadBatchParams>,
        projection: &Schema,
    ) -> Result<RecordBatch> {
        read_batch(self, &params.into(), projection, batch_id).await
    }

    /// Read a range of records into one batch.
    ///
    /// Note that it might call concat if the range is crossing multiple batches, which
    /// makes it less efficient than [`FileReader::read_batch()`].
    #[instrument(level = "debug", skip(self, projection))]
    pub async fn read_range(
        &self,
        range: Range<usize>,
        projection: &Schema,
    ) -> Result<RecordBatch> {
        if range.is_empty() {
            return Ok(RecordBatch::new_empty(Arc::new(projection.into())));
        }
        let range_in_batches = self.metadata.range_to_batches(range)?;
        let batches =
            stream::iter(range_in_batches)
                .map(|(batch_id, range)| async move {
                    self.read_batch(batch_id, range, projection).await
                })
                .buffered(self.io_parallelism())
                .try_collect::<Vec<_>>()
                .await?;
        if batches.len() == 1 {
            return Ok(batches[0].clone());
        }
        let schema = batches[0].schema();
        Ok(tokio::task::spawn_blocking(move || concat_batches(&schema, &batches)).await??)
    }

    /// Take by records by indices within the file.
    ///
    /// The indices must be sorted.
    #[instrument(level = "debug", skip_all)]
    pub async fn take(&self, indices: &[u32], projection: &Schema) -> Result<RecordBatch> {
        let num_batches = self.num_batches();
        let num_rows = self.len() as u32;
        let indices_in_batches = self.metadata.group_indices_to_batches(indices);
        let batches = stream::iter(indices_in_batches)
            .map(|batch| async move {
                if batch.batch_id >= num_batches as i32 {
                    Err(Error::InvalidInput {
                        source: format!("batch_id: {} out of bounds", batch.batch_id).into(),
                        location: location!(),
                    })
                } else if *batch.offsets.last().expect("got empty batch") > num_rows {
                    Err(Error::InvalidInput {
                        source: format!("indices: {:?} out of bounds", batch.offsets).into(),
                        location: location!(),
                    })
                } else {
                    self.read_batch(batch.batch_id, batch.offsets.as_slice(), projection)
                        .await
                }
            })
            .buffered(self.io_parallelism())
            .try_collect::<Vec<_>>()
            .await?;

        let schema = Arc::new(ArrowSchema::from(projection));

        Ok(tokio::task::spawn_blocking(move || concat_batches(&schema, &batches)).await??)
    }

    /// Get the schema of the statistics page table, for the given data field ids.
    pub fn page_stats_schema(&self, field_ids: &[i32]) -> Option<Schema> {
        self.metadata.stats_metadata.as_ref().map(|meta| {
            let mut stats_field_ids = vec![];
            for stats_field in &meta.schema.fields {
                if let Ok(stats_field_id) = stats_field.name.parse::<i32>() {
                    if field_ids.contains(&stats_field_id) {
                        stats_field_ids.push(stats_field.id);
                        for child in &stats_field.children {
                            stats_field_ids.push(child.id);
                        }
                    }
                }
            }
            meta.schema.project_by_ids(&stats_field_ids, true)
        })
    }

    /// Get the page statistics for the given data field ids.
    pub async fn read_page_stats(&self, field_ids: &[i32]) -> Result<Option<RecordBatch>> {
        if let Some(stats_page_table) = self.stats_page_table.as_ref() {
            let projection = self.page_stats_schema(field_ids).unwrap();
            // It's possible none of the requested fields have stats.
            if projection.fields.is_empty() {
                return Ok(None);
            }
            let arrays = futures::stream::iter(projection.fields.iter().cloned())
                .map(|field| async move {
                    read_array(
                        self,
                        &field,
                        0,
                        stats_page_table,
                        &ReadBatchParams::RangeFull,
                    )
                    .await
                })
                .buffered(self.io_parallelism())
                .try_collect::<Vec<_>>()
                .await?;

            let schema = ArrowSchema::from(&projection);
            let batch = RecordBatch::try_new(Arc::new(schema), arrays)?;
            Ok(Some(batch))
        } else {
            Ok(None)
        }
    }
}

/// Stream desired full batches from the file.
///
/// Parameters:
/// - **reader**: An opened file reader.
/// - **projection**: The schema of the returning [RecordBatch].
/// - **predicate**: A function that takes a batch ID and returns true if the batch should be
///                  returned.
///
/// Returns:
/// - A stream of [RecordBatch]s, each one corresponding to one full batch in the file.
pub fn batches_stream(
    reader: FileReader,
    projection: Schema,
    predicate: impl FnMut(&i32) -> bool + Send + Sync + 'static,
) -> impl RecordBatchStream {
    // Make projection an Arc so we can clone it and pass between threads.
    let projection = Arc::new(projection);
    let arrow_schema = ArrowSchema::from(projection.as_ref());

    let total_batches = reader.num_batches() as i32;
    let batches = (0..total_batches).filter(predicate);
    // Make another copy of self so we can clone it and pass between threads.
    let this = Arc::new(reader);
    let inner = stream::iter(batches)
        .zip(stream::repeat_with(move || {
            (this.clone(), projection.clone())
        }))
        .map(move |(batch_id, (reader, projection))| async move {
            reader
                .read_batch(batch_id, ReadBatchParams::RangeFull, &projection)
                .await
        })
        .buffered(2)
        .boxed();
    RecordBatchStreamAdapter::new(Arc::new(arrow_schema), inner)
}

/// Read a batch.
///
/// `schema` may only be empty if `with_row_id` is also true. This function
/// panics otherwise.
pub async fn read_batch(
    reader: &FileReader,
    params: &ReadBatchParams,
    schema: &Schema,
    batch_id: i32,
) -> Result<RecordBatch> {
    if !schema.fields.is_empty() {
        // We box this because otherwise we get a higher-order lifetime error.
        let arrs = stream::iter(&schema.fields)
            .map(|f| async { read_array(reader, f, batch_id, &reader.page_table, params).await })
            .buffered(reader.io_parallelism())
            .try_collect::<Vec<_>>()
            .boxed();
        let arrs = arrs.await?;
        Ok(RecordBatch::try_new(Arc::new(schema.into()), arrs)?)
    } else {
        Err(Error::invalid_input("no fields requested", location!()))
    }
}

#[async_recursion]
async fn read_array(
    reader: &FileReader,
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    params: &ReadBatchParams,
) -> Result<ArrayRef> {
    let data_type = field.data_type();

    use DataType::*;

    if data_type.is_fixed_stride() {
        _read_fixed_stride_array(reader, field, batch_id, page_table, params).await
    } else {
        match data_type {
            Null => read_null_array(field, batch_id, page_table, params),
            Utf8 | LargeUtf8 | Binary | LargeBinary => {
                read_binary_array(reader, field, batch_id, page_table, params).await
            }
            Struct(_) => read_struct_array(reader, field, batch_id, page_table, params).await,
            Dictionary(_, _) => {
                read_dictionary_array(reader, field, batch_id, page_table, params).await
            }
            List(_) => {
                read_list_array::<Int32Type>(reader, field, batch_id, page_table, params).await
            }
            LargeList(_) => {
                read_list_array::<Int64Type>(reader, field, batch_id, page_table, params).await
            }
            _ => {
                unimplemented!("{}", format!("No support for {data_type} yet"));
            }
        }
    }
}

fn get_page_info<'a>(
    page_table: &'a PageTable,
    field: &'a Field,
    batch_id: i32,
) -> Result<&'a PageInfo> {
    page_table.get(field.id, batch_id).ok_or_else(|| {
        Error::io(
            format!(
                "No page info found for field: {}, field_id={} batch={}",
                field.name, field.id, batch_id
            ),
            location!(),
        )
    })
}

/// Read primitive array for batch `batch_idx`.
async fn _read_fixed_stride_array(
    reader: &FileReader,
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    params: &ReadBatchParams,
) -> Result<ArrayRef> {
    let page_info = get_page_info(page_table, field, batch_id)?;
    read_fixed_stride_array(
        reader.object_reader.as_ref(),
        &field.data_type(),
        page_info.position,
        page_info.length,
        params.clone(),
    )
    .await
}

fn read_null_array(
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    params: &ReadBatchParams,
) -> Result<ArrayRef> {
    let page_info = get_page_info(page_table, field, batch_id)?;

    let length_output = match params {
        ReadBatchParams::Indices(indices) => {
            if indices.is_empty() {
                0
            } else {
                let idx_max = *indices.values().iter().max().unwrap() as u64;
                if idx_max >= page_info.length.try_into().unwrap() {
                    return Err(Error::io(
                        format!(
                            "NullArray Reader: request([{}]) out of range: [0..{}]",
                            idx_max, page_info.length
                        ),
                        location!(),
                    ));
                }
                indices.len()
            }
        }
        _ => {
            let (idx_start, idx_end) = match params {
                ReadBatchParams::Range(r) => (r.start, r.end),
                ReadBatchParams::RangeFull => (0, page_info.length),
                ReadBatchParams::RangeTo(r) => (0, r.end),
                ReadBatchParams::RangeFrom(r) => (r.start, page_info.length),
                _ => unreachable!(),
            };
            if idx_end > page_info.length {
                return Err(Error::io(
                    format!(
                        "NullArray Reader: request([{}..{}]) out of range: [0..{}]",
                        // and wrap it in here.
                        idx_start,
                        idx_end,
                        page_info.length
                    ),
                    location!(),
                ));
            }
            idx_end - idx_start
        }
    };

    Ok(Arc::new(NullArray::new(length_output)))
}

async fn read_binary_array(
    reader: &FileReader,
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    params: &ReadBatchParams,
) -> Result<ArrayRef> {
    let page_info = get_page_info(page_table, field, batch_id)?;

    lance_io::utils::read_binary_array(
        reader.object_reader.as_ref(),
        &field.data_type(),
        field.nullable,
        page_info.position,
        page_info.length,
        params,
    )
    .await
}

async fn read_dictionary_array(
    reader: &FileReader,
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    params: &ReadBatchParams,
) -> Result<ArrayRef> {
    let page_info = get_page_info(page_table, field, batch_id)?;
    let data_type = field.data_type();
    let decoder = DictionaryDecoder::new(
        reader.object_reader.as_ref(),
        page_info.position,
        page_info.length,
        &data_type,
        field
            .dictionary
            .as_ref()
            .unwrap()
            .values
            .as_ref()
            .unwrap()
            .clone(),
    );
    decoder.get(params.clone()).await
}

async fn read_struct_array(
    reader: &FileReader,
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    params: &ReadBatchParams,
) -> Result<ArrayRef> {
    // TODO: use tokio to make the reads in parallel.
    let mut sub_arrays: Vec<(FieldRef, ArrayRef)> = vec![];

    for child in field.children.as_slice() {
        let arr = read_array(reader, child, batch_id, page_table, params).await?;
        sub_arrays.push((Arc::new(child.into()), arr));
    }

    Ok(Arc::new(StructArray::from(sub_arrays)))
}

async fn take_list_array<T: ArrowNumericType>(
    reader: &FileReader,
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    positions: &PrimitiveArray<T>,
    indices: &UInt32Array,
) -> Result<ArrayRef>
where
    T::Native: ArrowNativeTypeOp + OffsetSizeTrait,
{
    let first_idx = indices.value(0);
    // Range of values for each index
    let ranges = indices
        .values()
        .iter()
        .map(|i| (*i - first_idx).as_usize())
        .map(|idx| positions.value(idx).as_usize()..positions.value(idx + 1).as_usize())
        .collect::<Vec<_>>();
    let field = field.clone();
    let mut list_values: Vec<ArrayRef> = vec![];
    // TODO: read them in parallel.
    for range in ranges.iter() {
        list_values.push(
            read_array(
                reader,
                &field.children[0],
                batch_id,
                page_table,
                &(range.clone()).into(),
            )
            .await?,
        );
    }

    let value_refs = list_values
        .iter()
        .map(|arr| arr.as_ref())
        .collect::<Vec<_>>();
    let mut offsets_builder = PrimitiveBuilder::<T>::new();
    offsets_builder.append_value(T::Native::usize_as(0));
    let mut off = 0_usize;
    for range in ranges {
        off += range.len();
        offsets_builder.append_value(T::Native::usize_as(off));
    }
    let all_values = concat::concat(value_refs.as_slice())?;
    let offset_arr = offsets_builder.finish();
    let arr = try_new_generic_list_array(all_values, &offset_arr)?;
    Ok(Arc::new(arr) as ArrayRef)
}

async fn read_list_array<T: ArrowNumericType>(
    reader: &FileReader,
    field: &Field,
    batch_id: i32,
    page_table: &PageTable,
    params: &ReadBatchParams,
) -> Result<ArrayRef>
where
    T::Native: ArrowNativeTypeOp + OffsetSizeTrait,
{
    // Offset the position array by 1 in order to include the upper bound of the last element
    let positions_params = match params {
        ReadBatchParams::Range(range) => ReadBatchParams::from(range.start..(range.end + 1)),
        ReadBatchParams::RangeTo(range) => ReadBatchParams::from(..range.end + 1),
        ReadBatchParams::Indices(indices) => {
            (indices.value(0).as_usize()..indices.value(indices.len() - 1).as_usize() + 2).into()
        }
        p => p.clone(),
    };

    let page_info = get_page_info(&reader.page_table, field, batch_id)?;
    let position_arr = read_fixed_stride_array(
        reader.object_reader.as_ref(),
        &T::DATA_TYPE,
        page_info.position,
        page_info.length,
        positions_params,
    )
    .await?;

    let positions: &PrimitiveArray<T> = position_arr.as_primitive();

    // Recompute params so they align with the offset array
    let value_params = match params {
        ReadBatchParams::Range(range) => ReadBatchParams::from(
            positions.value(0).as_usize()..positions.value(range.end - range.start).as_usize(),
        ),
        ReadBatchParams::RangeTo(RangeTo { end }) => {
            ReadBatchParams::from(..positions.value(*end).as_usize())
        }
        ReadBatchParams::RangeFrom(_) => ReadBatchParams::from(positions.value(0).as_usize()..),
        ReadBatchParams::RangeFull => ReadBatchParams::from(
            positions.value(0).as_usize()..positions.value(positions.len() - 1).as_usize(),
        ),
        ReadBatchParams::Indices(indices) => {
            return take_list_array(reader, field, batch_id, page_table, positions, indices).await;
        }
    };

    let start_position = PrimitiveArray::<T>::new_scalar(positions.value(0));
    let offset_arr = sub(positions, &start_position)?;
    let offset_arr_ref = offset_arr.as_primitive::<T>();
    let value_arrs = read_array(
        reader,
        &field.children[0],
        batch_id,
        page_table,
        &value_params,
    )
    .await?;
    let arr = try_new_generic_list_array(value_arrs, offset_arr_ref)?;
    Ok(Arc::new(arr) as ArrayRef)
}

#[cfg(test)]
mod tests {
    use crate::writer::{FileWriter, NotSelfDescribing};

    use super::*;

    use arrow_array::{
        builder::{Int32Builder, LargeListBuilder, ListBuilder, StringBuilder},
        cast::{as_string_array, as_struct_array},
        types::UInt8Type,
        Array, DictionaryArray, Float32Array, Int64Array, LargeListArray, ListArray, StringArray,
        UInt8Array,
    };
    use arrow_array::{BooleanArray, Int32Array};
    use arrow_schema::{Field as ArrowField, Fields as ArrowFields, Schema as ArrowSchema};

    #[tokio::test]
    async fn test_take() {
        let arrow_schema = ArrowSchema::new(vec![
            ArrowField::new("i", DataType::Int64, true),
            ArrowField::new("f", DataType::Float32, false),
            ArrowField::new("s", DataType::Utf8, false),
            ArrowField::new(
                "d",
                DataType::Dictionary(Box::new(DataType::UInt8), Box::new(DataType::Utf8)),
                false,
            ),
        ]);
        let mut schema = Schema::try_from(&arrow_schema).unwrap();

        let store = ObjectStore::memory();
        let path = Path::from("/take_test");

        // Write 10 batches.
        let values = StringArray::from_iter_values(["a", "b", "c", "d", "e", "f", "g"]);
        let values_ref = Arc::new(values);
        let mut batches = vec![];
        for batch_id in 0..10 {
            let value_range: Range<i64> = batch_id * 10..batch_id * 10 + 10;
            let keys = UInt8Array::from_iter_values(value_range.clone().map(|v| (v % 7) as u8));
            let columns: Vec<ArrayRef> = vec![
                Arc::new(Int64Array::from_iter(
                    value_range.clone().collect::<Vec<_>>(),
                )),
                Arc::new(Float32Array::from_iter(
                    value_range.clone().map(|n| n as f32).collect::<Vec<_>>(),
                )),
                Arc::new(StringArray::from_iter_values(
                    value_range.clone().map(|n| format!("str-{}", n)),
                )),
                Arc::new(DictionaryArray::<UInt8Type>::try_new(keys, values_ref.clone()).unwrap()),
            ];
            batches.push(RecordBatch::try_new(Arc::new(arrow_schema.clone()), columns).unwrap());
        }
        schema.set_dictionary(&batches[0]).unwrap();

        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        for batch in batches.iter() {
            file_writer.write(&[batch.clone()]).await.unwrap();
        }
        file_writer.finish().await.unwrap();

        let reader = FileReader::try_new(&store, &path, schema).await.unwrap();
        let batch = reader
            .take(&[1, 15, 20, 25, 30, 48, 90], reader.schema())
            .await
            .unwrap();
        let dict_keys = UInt8Array::from_iter_values([1, 1, 6, 4, 2, 6, 6]);
        assert_eq!(
            batch,
            RecordBatch::try_new(
                batch.schema(),
                vec![
                    Arc::new(Int64Array::from_iter_values([1, 15, 20, 25, 30, 48, 90])),
                    Arc::new(Float32Array::from_iter_values([
                        1.0, 15.0, 20.0, 25.0, 30.0, 48.0, 90.0
                    ])),
                    Arc::new(StringArray::from_iter_values([
                        "str-1", "str-15", "str-20", "str-25", "str-30", "str-48", "str-90"
                    ])),
                    Arc::new(DictionaryArray::try_new(dict_keys, values_ref.clone()).unwrap()),
                ]
            )
            .unwrap()
        );
    }

    async fn test_write_null_string_in_struct(field_nullable: bool) {
        let arrow_schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "parent",
            DataType::Struct(ArrowFields::from(vec![ArrowField::new(
                "str",
                DataType::Utf8,
                field_nullable,
            )])),
            true,
        )]));

        let schema = Schema::try_from(arrow_schema.as_ref()).unwrap();

        let store = ObjectStore::memory();
        let path = Path::from("/null_strings");

        let string_arr = Arc::new(StringArray::from_iter([Some("a"), Some(""), Some("b")]));
        let struct_arr = Arc::new(StructArray::from(vec![(
            Arc::new(ArrowField::new("str", DataType::Utf8, field_nullable)),
            string_arr.clone() as ArrayRef,
        )]));
        let batch = RecordBatch::try_new(arrow_schema.clone(), vec![struct_arr]).unwrap();

        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        file_writer.write(&[batch.clone()]).await.unwrap();
        file_writer.finish().await.unwrap();

        let reader = FileReader::try_new(&store, &path, schema).await.unwrap();
        let actual_batch = reader.read_batch(0, .., reader.schema()).await.unwrap();

        if field_nullable {
            assert_eq!(
                &StringArray::from_iter(vec![Some("a"), None, Some("b")]),
                as_string_array(
                    as_struct_array(actual_batch.column_by_name("parent").unwrap().as_ref())
                        .column_by_name("str")
                        .unwrap()
                        .as_ref()
                )
            );
        } else {
            assert_eq!(actual_batch, batch);
        }
    }

    #[tokio::test]
    async fn read_nullable_string_in_struct() {
        test_write_null_string_in_struct(true).await;
        test_write_null_string_in_struct(false).await;
    }

    #[tokio::test]
    async fn test_read_struct_of_list_arrays() {
        let store = ObjectStore::memory();
        let path = Path::from("/null_strings");

        let arrow_schema = make_schema_of_list_array();
        let schema: Schema = Schema::try_from(arrow_schema.as_ref()).unwrap();

        let batches = (0..3)
            .map(|_| {
                let struct_array = make_struct_of_list_array(10, 10);
                RecordBatch::try_new(arrow_schema.clone(), vec![struct_array]).unwrap()
            })
            .collect::<Vec<_>>();
        let batches_ref = batches.iter().collect::<Vec<_>>();

        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        file_writer.write(&batches).await.unwrap();
        file_writer.finish().await.unwrap();

        let reader = FileReader::try_new(&store, &path, schema).await.unwrap();
        let actual_batch = reader.read_batch(0, .., reader.schema()).await.unwrap();
        let expected = concat_batches(&arrow_schema, batches_ref).unwrap();
        assert_eq!(expected, actual_batch);
    }

    #[tokio::test]
    async fn test_scan_struct_of_list_arrays() {
        let store = ObjectStore::memory();
        let path = Path::from("/null_strings");

        let arrow_schema = make_schema_of_list_array();
        let struct_array = make_struct_of_list_array(3, 10);
        let schema: Schema = Schema::try_from(arrow_schema.as_ref()).unwrap();
        let batch = RecordBatch::try_new(arrow_schema.clone(), vec![struct_array.clone()]).unwrap();

        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        file_writer.write(&[batch]).await.unwrap();
        file_writer.finish().await.unwrap();

        let mut expected_columns: Vec<ArrayRef> = Vec::new();
        for c in struct_array.columns().iter() {
            expected_columns.push(c.slice(1, 1));
        }

        let expected_struct = match arrow_schema.fields[0].data_type() {
            DataType::Struct(subfields) => subfields
                .iter()
                .zip(expected_columns)
                .map(|(f, d)| (f.clone(), d))
                .collect::<Vec<_>>(),
            _ => panic!("unexpected field"),
        };

        let expected_struct_array = StructArray::from(expected_struct);
        let expected_batch = RecordBatch::from(&StructArray::from(vec![(
            Arc::new(arrow_schema.fields[0].as_ref().clone()),
            Arc::new(expected_struct_array) as ArrayRef,
        )]));

        let reader = FileReader::try_new(&store, &path, schema).await.unwrap();
        let params = ReadBatchParams::Range(1..2);
        let slice_of_batch = reader.read_batch(0, params, reader.schema()).await.unwrap();
        assert_eq!(expected_batch, slice_of_batch);
    }

    fn make_schema_of_list_array() -> Arc<arrow_schema::Schema> {
        Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "s",
            DataType::Struct(ArrowFields::from(vec![
                ArrowField::new(
                    "li",
                    DataType::List(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                    true,
                ),
                ArrowField::new(
                    "ls",
                    DataType::List(Arc::new(ArrowField::new("item", DataType::Utf8, true))),
                    true,
                ),
                ArrowField::new(
                    "ll",
                    DataType::LargeList(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                    false,
                ),
            ])),
            true,
        )]))
    }

    fn make_struct_of_list_array(rows: i32, num_items: i32) -> Arc<StructArray> {
        let mut li_builder = ListBuilder::new(Int32Builder::new());
        let mut ls_builder = ListBuilder::new(StringBuilder::new());
        let ll_value_builder = Int32Builder::new();
        let mut large_list_builder = LargeListBuilder::new(ll_value_builder);
        for i in 0..rows {
            for j in 0..num_items {
                li_builder.values().append_value(i * 10 + j);
                ls_builder
                    .values()
                    .append_value(format!("str-{}", i * 10 + j));
                large_list_builder.values().append_value(i * 10 + j);
            }
            li_builder.append(true);
            ls_builder.append(true);
            large_list_builder.append(true);
        }
        Arc::new(StructArray::from(vec![
            (
                Arc::new(ArrowField::new(
                    "li",
                    DataType::List(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                    true,
                )),
                Arc::new(li_builder.finish()) as ArrayRef,
            ),
            (
                Arc::new(ArrowField::new(
                    "ls",
                    DataType::List(Arc::new(ArrowField::new("item", DataType::Utf8, true))),
                    true,
                )),
                Arc::new(ls_builder.finish()) as ArrayRef,
            ),
            (
                Arc::new(ArrowField::new(
                    "ll",
                    DataType::LargeList(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                    false,
                )),
                Arc::new(large_list_builder.finish()) as ArrayRef,
            ),
        ]))
    }

    #[tokio::test]
    async fn test_read_nullable_arrays() {
        use arrow_array::Array;

        // create a record batch with a null array column
        let arrow_schema = ArrowSchema::new(vec![
            ArrowField::new("i", DataType::Int64, false),
            ArrowField::new("n", DataType::Null, true),
        ]);
        let schema = Schema::try_from(&arrow_schema).unwrap();
        let columns: Vec<ArrayRef> = vec![
            Arc::new(Int64Array::from_iter_values(0..100)),
            Arc::new(NullArray::new(100)),
        ];
        let batch = RecordBatch::try_new(Arc::new(arrow_schema), columns).unwrap();

        // write to a lance file
        let store = ObjectStore::memory();
        let path = Path::from("/takes");
        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        file_writer.write(&[batch]).await.unwrap();
        file_writer.finish().await.unwrap();

        // read the file back
        let reader = FileReader::try_new(&store, &path, schema.clone())
            .await
            .unwrap();

        async fn read_array_w_params(
            reader: &FileReader,
            field: &Field,
            params: ReadBatchParams,
        ) -> ArrayRef {
            read_array(reader, field, 0, reader.page_table.as_ref(), &params)
                .await
                .expect("Error reading back the null array from file") as _
        }

        let arr = read_array_w_params(&reader, &schema.fields[1], ReadBatchParams::RangeFull).await;
        assert_eq!(100, arr.len());
        assert_eq!(arr.data_type(), &DataType::Null);

        let arr =
            read_array_w_params(&reader, &schema.fields[1], ReadBatchParams::Range(10..25)).await;
        assert_eq!(15, arr.len());
        assert_eq!(arr.data_type(), &DataType::Null);

        let arr =
            read_array_w_params(&reader, &schema.fields[1], ReadBatchParams::RangeFrom(60..)).await;
        assert_eq!(40, arr.len());
        assert_eq!(arr.data_type(), &DataType::Null);

        let arr =
            read_array_w_params(&reader, &schema.fields[1], ReadBatchParams::RangeTo(..25)).await;
        assert_eq!(25, arr.len());
        assert_eq!(arr.data_type(), &DataType::Null);

        let arr = read_array_w_params(
            &reader,
            &schema.fields[1],
            ReadBatchParams::Indices(UInt32Array::from(vec![1, 9, 30, 72])),
        )
        .await;
        assert_eq!(4, arr.len());
        assert_eq!(arr.data_type(), &DataType::Null);

        // raise error if take indices are out of bounds
        let params = ReadBatchParams::Indices(UInt32Array::from(vec![1, 9, 30, 72, 100]));
        let arr = read_array(
            &reader,
            &schema.fields[1],
            0,
            reader.page_table.as_ref(),
            &params,
        );
        assert!(arr.await.is_err());

        // raise error if range indices are out of bounds
        let params = ReadBatchParams::RangeTo(..107);
        let arr = read_array(
            &reader,
            &schema.fields[1],
            0,
            reader.page_table.as_ref(),
            &params,
        );
        assert!(arr.await.is_err());
    }

    #[tokio::test]
    async fn test_take_lists() {
        let arrow_schema = ArrowSchema::new(vec![
            ArrowField::new(
                "l",
                DataType::List(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                false,
            ),
            ArrowField::new(
                "ll",
                DataType::LargeList(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                false,
            ),
        ]);

        let value_builder = Int32Builder::new();
        let mut list_builder = ListBuilder::new(value_builder);
        let ll_value_builder = Int32Builder::new();
        let mut large_list_builder = LargeListBuilder::new(ll_value_builder);
        for i in 0..100 {
            list_builder.values().append_value(i);
            large_list_builder.values().append_value(i);
            if (i + 1) % 10 == 0 {
                list_builder.append(true);
                large_list_builder.append(true);
            }
        }
        let list_arr = Arc::new(list_builder.finish());
        let large_list_arr = Arc::new(large_list_builder.finish());

        let batch = RecordBatch::try_new(
            Arc::new(arrow_schema.clone()),
            vec![list_arr as ArrayRef, large_list_arr as ArrayRef],
        )
        .unwrap();

        // write to a lance file
        let store = ObjectStore::memory();
        let path = Path::from("/take_list");
        let schema: Schema = (&arrow_schema).try_into().unwrap();
        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        file_writer.write(&[batch]).await.unwrap();
        file_writer.finish().await.unwrap();

        // read the file back
        let reader = FileReader::try_new(&store, &path, schema.clone())
            .await
            .unwrap();
        let actual = reader.take(&[1, 3, 5, 9], &schema).await.unwrap();

        let value_builder = Int32Builder::new();
        let mut list_builder = ListBuilder::new(value_builder);
        let ll_value_builder = Int32Builder::new();
        let mut large_list_builder = LargeListBuilder::new(ll_value_builder);
        for i in [1, 3, 5, 9] {
            for j in 0..10 {
                list_builder.values().append_value(i * 10 + j);
                large_list_builder.values().append_value(i * 10 + j);
            }
            list_builder.append(true);
            large_list_builder.append(true);
        }
        let expected_list = list_builder.finish();
        let expected_large_list = large_list_builder.finish();

        assert_eq!(actual.column_by_name("l").unwrap().as_ref(), &expected_list);
        assert_eq!(
            actual.column_by_name("ll").unwrap().as_ref(),
            &expected_large_list
        );
    }

    #[tokio::test]
    async fn test_list_array_with_offsets() {
        let arrow_schema = ArrowSchema::new(vec![
            ArrowField::new(
                "l",
                DataType::List(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                false,
            ),
            ArrowField::new(
                "ll",
                DataType::LargeList(Arc::new(ArrowField::new("item", DataType::Int32, true))),
                false,
            ),
        ]);

        let store = ObjectStore::memory();
        let path = Path::from("/lists");

        let list_array = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
            Some(vec![Some(1), Some(2)]),
            Some(vec![Some(3), Some(4)]),
            Some((0..2_000).map(Some).collect::<Vec<_>>()),
        ])
        .slice(1, 1);
        let large_list_array = LargeListArray::from_iter_primitive::<Int32Type, _, _>(vec![
            Some(vec![Some(10), Some(11)]),
            Some(vec![Some(12), Some(13)]),
            Some((0..2_000).map(Some).collect::<Vec<_>>()),
        ])
        .slice(1, 1);

        let batch = RecordBatch::try_new(
            Arc::new(arrow_schema.clone()),
            vec![Arc::new(list_array), Arc::new(large_list_array)],
        )
        .unwrap();

        let schema: Schema = (&arrow_schema).try_into().unwrap();
        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        file_writer.write(&[batch.clone()]).await.unwrap();
        file_writer.finish().await.unwrap();

        // Make sure the big array was not written to the file
        let file_size_bytes = store.size(&path).await.unwrap();
        assert!(file_size_bytes < 1_000);

        let reader = FileReader::try_new(&store, &path, schema).await.unwrap();
        let actual_batch = reader.read_batch(0, .., reader.schema()).await.unwrap();
        assert_eq!(batch, actual_batch);
    }

    #[tokio::test]
    async fn test_read_ranges() {
        // create a record batch with a null array column
        let arrow_schema = ArrowSchema::new(vec![ArrowField::new("i", DataType::Int64, false)]);
        let schema = Schema::try_from(&arrow_schema).unwrap();
        let columns: Vec<ArrayRef> = vec![Arc::new(Int64Array::from_iter_values(0..100))];
        let batch = RecordBatch::try_new(Arc::new(arrow_schema), columns).unwrap();

        // write to a lance file
        let store = ObjectStore::memory();
        let path = Path::from("/read_range");
        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        file_writer.write(&[batch]).await.unwrap();
        file_writer.finish().await.unwrap();

        let reader = FileReader::try_new(&store, &path, schema).await.unwrap();
        let actual_batch = reader.read_range(7..25, reader.schema()).await.unwrap();

        assert_eq!(
            actual_batch.column_by_name("i").unwrap().as_ref(),
            &Int64Array::from_iter_values(7..25)
        );
    }

    #[tokio::test]
    async fn test_batches_stream() {
        let store = ObjectStore::memory();
        let path = Path::from("/batch_stream");

        let arrow_schema = ArrowSchema::new(vec![ArrowField::new("i", DataType::Int32, true)]);
        let schema = Schema::try_from(&arrow_schema).unwrap();
        let mut writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();
        for i in 0..10 {
            let batch = RecordBatch::try_new(
                Arc::new(arrow_schema.clone()),
                vec![Arc::new(Int32Array::from_iter_values(i * 10..(i + 1) * 10))],
            )
            .unwrap();
            writer.write(&[batch]).await.unwrap();
        }
        writer.finish().await.unwrap();

        let reader = FileReader::try_new(&store, &path, schema.clone())
            .await
            .unwrap();
        let stream = batches_stream(reader, schema, |id| id % 2 == 0);
        let batches = stream.try_collect::<Vec<_>>().await.unwrap();

        assert_eq!(batches.len(), 5);
        for (i, batch) in batches.iter().enumerate() {
            assert_eq!(
                batch,
                &RecordBatch::try_new(
                    Arc::new(arrow_schema.clone()),
                    vec![Arc::new(Int32Array::from_iter_values(
                        i as i32 * 2 * 10..(i as i32 * 2 + 1) * 10
                    ))],
                )
                .unwrap()
            )
        }
    }

    #[tokio::test]
    async fn test_take_boolean_beyond_chunk() {
        let mut store = ObjectStore::memory();
        store.set_block_size(256);
        let path = Path::from("/take_bools");

        let arrow_schema = Arc::new(ArrowSchema::new(vec![ArrowField::new(
            "b",
            DataType::Boolean,
            false,
        )]));
        let schema = Schema::try_from(arrow_schema.as_ref()).unwrap();
        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();

        let array = BooleanArray::from((0..5000).map(|v| v % 5 == 0).collect::<Vec<_>>());
        let batch =
            RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new(array.clone())]).unwrap();
        file_writer.write(&[batch]).await.unwrap();
        file_writer.finish().await.unwrap();

        let reader = FileReader::try_new(&store, &path, schema.clone())
            .await
            .unwrap();
        let actual = reader.take(&[2, 4, 5, 8, 4555], &schema).await.unwrap();

        assert_eq!(
            actual.column_by_name("b").unwrap().as_ref(),
            &BooleanArray::from(vec![false, false, true, false, true])
        );
    }

    #[tokio::test]
    async fn test_read_projection() {
        // The dataset schema may be very large.  The file reader should support reading
        // a small projection of that schema (this just tests the field_offset / num_fields
        // parameters)
        let store = ObjectStore::memory();
        let path = Path::from("/partial_read");

        // Create a large schema
        let mut fields = vec![];
        for i in 0..100 {
            fields.push(ArrowField::new(format!("f{}", i), DataType::Int32, false));
        }
        let arrow_schema = ArrowSchema::new(fields);
        let schema = Schema::try_from(&arrow_schema).unwrap();

        let partial_schema = schema.project(&["f50"]).unwrap();
        let partial_arrow: ArrowSchema = (&partial_schema).into();

        let mut file_writer = FileWriter::<NotSelfDescribing>::try_new(
            &store,
            &path,
            partial_schema.clone(),
            &Default::default(),
        )
        .await
        .unwrap();

        let array = Int32Array::from(vec![0; 15]);
        let batch =
            RecordBatch::try_new(Arc::new(partial_arrow), vec![Arc::new(array.clone())]).unwrap();
        file_writer.write(&[batch.clone()]).await.unwrap();
        file_writer.finish().await.unwrap();

        let field_id = partial_schema.fields.first().unwrap().id;
        let reader = FileReader::try_new_with_fragment_id(
            &store,
            &path,
            schema.clone(),
            0,
            /*min_field_id=*/ field_id,
            /*max_field_id=*/ field_id,
            None,
        )
        .await
        .unwrap();
        let actual = reader
            .read_batch(0, ReadBatchParams::RangeFull, &partial_schema)
            .await
            .unwrap();

        assert_eq!(actual, batch);
    }
}