parquet/arrow/async_reader/
mod.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.

//! Provides `async` API for reading parquet files as
//! [`RecordBatch`]es
//!
//! ```
//! # #[tokio::main(flavor="current_thread")]
//! # async fn main() {
//! #
//! # use arrow_array::RecordBatch;
//! # use arrow::util::pretty::pretty_format_batches;
//! # use futures::TryStreamExt;
//! # use tokio::fs::File;
//! #
//! # use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask};
//! #
//! # fn assert_batches_eq(batches: &[RecordBatch], expected_lines: &[&str]) {
//! #     let formatted = pretty_format_batches(batches).unwrap().to_string();
//! #     let actual_lines: Vec<_> = formatted.trim().lines().collect();
//! #     assert_eq!(
//! #          &actual_lines, expected_lines,
//! #          "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n",
//! #          expected_lines, actual_lines
//! #      );
//! #  }
//! #
//! let testdata = arrow::util::test_util::parquet_test_data();
//! let path = format!("{}/alltypes_plain.parquet", testdata);
//! let file = File::open(path).await.unwrap();
//!
//! let builder = ParquetRecordBatchStreamBuilder::new(file)
//!     .await
//!     .unwrap()
//!     .with_batch_size(3);
//!
//! let file_metadata = builder.metadata().file_metadata();
//! let mask = ProjectionMask::roots(file_metadata.schema_descr(), [1, 2, 6]);
//!
//! let stream = builder.with_projection(mask).build().unwrap();
//! let results = stream.try_collect::<Vec<_>>().await.unwrap();
//! assert_eq!(results.len(), 3);
//!
//! assert_batches_eq(
//!     &results,
//!     &[
//!         "+----------+-------------+-----------+",
//!         "| bool_col | tinyint_col | float_col |",
//!         "+----------+-------------+-----------+",
//!         "| true     | 0           | 0.0       |",
//!         "| false    | 1           | 1.1       |",
//!         "| true     | 0           | 0.0       |",
//!         "| false    | 1           | 1.1       |",
//!         "| true     | 0           | 0.0       |",
//!         "| false    | 1           | 1.1       |",
//!         "| true     | 0           | 0.0       |",
//!         "| false    | 1           | 1.1       |",
//!         "+----------+-------------+-----------+",
//!      ],
//!  );
//! # }
//! ```

use std::collections::VecDeque;
use std::fmt::Formatter;
use std::io::SeekFrom;
use std::ops::Range;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};

use bytes::{Buf, Bytes};
use futures::future::{BoxFuture, FutureExt};
use futures::ready;
use futures::stream::Stream;
use tokio::io::{AsyncRead, AsyncReadExt, AsyncSeek, AsyncSeekExt};

use arrow_array::RecordBatch;
use arrow_schema::{DataType, Fields, Schema, SchemaRef};

use crate::arrow::array_reader::{build_array_reader, RowGroups};
use crate::arrow::arrow_reader::{
    apply_range, evaluate_predicate, selects_any, ArrowReaderBuilder, ArrowReaderMetadata,
    ArrowReaderOptions, ParquetRecordBatchReader, RowFilter, RowSelection,
};
use crate::arrow::ProjectionMask;

use crate::bloom_filter::{
    chunk_read_bloom_filter_header_and_offset, Sbbf, SBBF_HEADER_SIZE_ESTIMATE,
};
use crate::column::page::{PageIterator, PageReader};
use crate::errors::{ParquetError, Result};
use crate::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData};
use crate::file::page_index::offset_index::OffsetIndexMetaData;
use crate::file::reader::{ChunkReader, Length, SerializedPageReader};
use crate::file::FOOTER_SIZE;
use crate::format::{BloomFilterAlgorithm, BloomFilterCompression, BloomFilterHash};

mod metadata;
pub use metadata::*;

#[cfg(feature = "object_store")]
mod store;

use crate::arrow::schema::ParquetField;
#[cfg(feature = "object_store")]
pub use store::*;

/// The asynchronous interface used by [`ParquetRecordBatchStream`] to read parquet files
///
/// Notes:
///
/// 1. There is a default implementation for types that implement [`AsyncRead`]
///    and [`AsyncSeek`], for example [`tokio::fs::File`].
///
/// 2. [`ParquetObjectReader`], available when the `object_store` crate feature
///    is enabled, implements this interface for [`ObjectStore`].
///
/// [`ObjectStore`]: object_store::ObjectStore
///
/// [`tokio::fs::File`]: https://docs.rs/tokio/latest/tokio/fs/struct.File.html
pub trait AsyncFileReader: Send {
    /// Retrieve the bytes in `range`
    fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>>;

    /// Retrieve multiple byte ranges. The default implementation will call `get_bytes` sequentially
    fn get_byte_ranges(&mut self, ranges: Vec<Range<usize>>) -> BoxFuture<'_, Result<Vec<Bytes>>> {
        async move {
            let mut result = Vec::with_capacity(ranges.len());

            for range in ranges.into_iter() {
                let data = self.get_bytes(range).await?;
                result.push(data);
            }

            Ok(result)
        }
        .boxed()
    }

    /// Provides asynchronous access to the [`ParquetMetaData`] of a parquet file,
    /// allowing fine-grained control over how metadata is sourced, in particular allowing
    /// for caching, pre-fetching, catalog metadata, etc...
    fn get_metadata(&mut self) -> BoxFuture<'_, Result<Arc<ParquetMetaData>>>;
}

impl AsyncFileReader for Box<dyn AsyncFileReader> {
    fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> {
        self.as_mut().get_bytes(range)
    }

    fn get_byte_ranges(&mut self, ranges: Vec<Range<usize>>) -> BoxFuture<'_, Result<Vec<Bytes>>> {
        self.as_mut().get_byte_ranges(ranges)
    }

    fn get_metadata(&mut self) -> BoxFuture<'_, Result<Arc<ParquetMetaData>>> {
        self.as_mut().get_metadata()
    }
}

impl<T: AsyncRead + AsyncSeek + Unpin + Send> AsyncFileReader for T {
    fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> {
        async move {
            self.seek(SeekFrom::Start(range.start as u64)).await?;

            let to_read = range.end - range.start;
            let mut buffer = Vec::with_capacity(to_read);
            let read = self.take(to_read as u64).read_to_end(&mut buffer).await?;
            if read != to_read {
                return Err(eof_err!("expected to read {} bytes, got {}", to_read, read));
            }

            Ok(buffer.into())
        }
        .boxed()
    }

    fn get_metadata(&mut self) -> BoxFuture<'_, Result<Arc<ParquetMetaData>>> {
        const FOOTER_SIZE_I64: i64 = FOOTER_SIZE as i64;
        async move {
            self.seek(SeekFrom::End(-FOOTER_SIZE_I64)).await?;

            let mut buf = [0_u8; FOOTER_SIZE];
            self.read_exact(&mut buf).await?;

            let metadata_len = ParquetMetaDataReader::decode_footer(&buf)?;
            self.seek(SeekFrom::End(-FOOTER_SIZE_I64 - metadata_len as i64))
                .await?;

            let mut buf = Vec::with_capacity(metadata_len);
            self.take(metadata_len as _).read_to_end(&mut buf).await?;

            Ok(Arc::new(ParquetMetaDataReader::decode_metadata(&buf)?))
        }
        .boxed()
    }
}

impl ArrowReaderMetadata {
    /// Returns a new [`ArrowReaderMetadata`] for this builder
    ///
    /// See [`ParquetRecordBatchStreamBuilder::new_with_metadata`] for how this can be used
    ///
    /// # Notes
    ///
    /// If `options` has [`ArrowReaderOptions::with_page_index`] true, but
    /// `Self::metadata` is missing the page index, this function will attempt
    /// to load the page index by making an object store request.
    pub async fn load_async<T: AsyncFileReader>(
        input: &mut T,
        options: ArrowReaderOptions,
    ) -> Result<Self> {
        // TODO: this is all rather awkward. It would be nice if AsyncFileReader::get_metadata
        // took an argument to fetch the page indexes.
        let mut metadata = input.get_metadata().await?;

        if options.page_index
            && metadata.column_index().is_none()
            && metadata.offset_index().is_none()
        {
            let m = Arc::try_unwrap(metadata).unwrap_or_else(|e| e.as_ref().clone());
            let mut reader = ParquetMetaDataReader::new_with_metadata(m).with_page_indexes(true);
            reader.load_page_index(input).await?;
            metadata = Arc::new(reader.finish()?)
        }
        Self::try_new(metadata, options)
    }
}

#[doc(hidden)]
/// A newtype used within [`ReaderOptionsBuilder`] to distinguish sync readers from async
///
/// Allows sharing the same builder for both the sync and async versions, whilst also not
/// breaking the pre-existing ParquetRecordBatchStreamBuilder API
pub struct AsyncReader<T>(T);

/// A builder used to construct a [`ParquetRecordBatchStream`] for `async` reading of a parquet file
///
/// In particular, this handles reading the parquet file metadata, allowing consumers
/// to use this information to select what specific columns, row groups, etc...
/// they wish to be read by the resulting stream
///
/// See [`ArrowReaderBuilder`] for additional member functions
pub type ParquetRecordBatchStreamBuilder<T> = ArrowReaderBuilder<AsyncReader<T>>;

impl<T: AsyncFileReader + Send + 'static> ParquetRecordBatchStreamBuilder<T> {
    /// Create a new [`ParquetRecordBatchStreamBuilder`] with the provided parquet file
    ///
    /// # Example
    ///
    /// ```
    /// # use std::fs::metadata;
    /// # use std::sync::Arc;
    /// # use bytes::Bytes;
    /// # use arrow_array::{Int32Array, RecordBatch};
    /// # use arrow_schema::{DataType, Field, Schema};
    /// # use parquet::arrow::arrow_reader::ArrowReaderMetadata;
    /// # use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder};
    /// # use tempfile::tempfile;
    /// # use futures::StreamExt;
    /// # #[tokio::main(flavor="current_thread")]
    /// # async fn main() {
    /// #
    /// # let mut file = tempfile().unwrap();
    /// # let schema = Arc::new(Schema::new(vec![Field::new("i32", DataType::Int32, false)]));
    /// # let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap();
    /// # let batch = RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![1, 2, 3]))]).unwrap();
    /// # writer.write(&batch).unwrap();
    /// # writer.close().unwrap();
    /// // Open async file containing parquet data
    /// let mut file = tokio::fs::File::from_std(file);
    /// // construct the reader
    /// let mut reader = ParquetRecordBatchStreamBuilder::new(file)
    ///   .await.unwrap().build().unwrap();
    /// // Read batche
    /// let batch: RecordBatch = reader.next().await.unwrap().unwrap();
    /// # }
    /// ```
    pub async fn new(input: T) -> Result<Self> {
        Self::new_with_options(input, Default::default()).await
    }

    /// Create a new [`ParquetRecordBatchStreamBuilder`] with the provided parquet file
    /// and [`ArrowReaderOptions`]
    pub async fn new_with_options(mut input: T, options: ArrowReaderOptions) -> Result<Self> {
        let metadata = ArrowReaderMetadata::load_async(&mut input, options).await?;
        Ok(Self::new_with_metadata(input, metadata))
    }

    /// Create a [`ParquetRecordBatchStreamBuilder`] from the provided [`ArrowReaderMetadata`]
    ///
    /// This allows loading metadata once and using it to create multiple builders with
    /// potentially different settings, that can be read in parallel.
    ///
    /// # Example of reading from multiple streams in parallel
    ///
    /// ```
    /// # use std::fs::metadata;
    /// # use std::sync::Arc;
    /// # use bytes::Bytes;
    /// # use arrow_array::{Int32Array, RecordBatch};
    /// # use arrow_schema::{DataType, Field, Schema};
    /// # use parquet::arrow::arrow_reader::ArrowReaderMetadata;
    /// # use parquet::arrow::{ArrowWriter, ParquetRecordBatchStreamBuilder};
    /// # use tempfile::tempfile;
    /// # use futures::StreamExt;
    /// # #[tokio::main(flavor="current_thread")]
    /// # async fn main() {
    /// #
    /// # let mut file = tempfile().unwrap();
    /// # let schema = Arc::new(Schema::new(vec![Field::new("i32", DataType::Int32, false)]));
    /// # let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap();
    /// # let batch = RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(vec![1, 2, 3]))]).unwrap();
    /// # writer.write(&batch).unwrap();
    /// # writer.close().unwrap();
    /// // open file with parquet data
    /// let mut file = tokio::fs::File::from_std(file);
    /// // load metadata once
    /// let meta = ArrowReaderMetadata::load_async(&mut file, Default::default()).await.unwrap();
    /// // create two readers, a and b, from the same underlying file
    /// // without reading the metadata again
    /// let mut a = ParquetRecordBatchStreamBuilder::new_with_metadata(
    ///     file.try_clone().await.unwrap(),
    ///     meta.clone()
    /// ).build().unwrap();
    /// let mut b = ParquetRecordBatchStreamBuilder::new_with_metadata(file, meta).build().unwrap();
    ///
    /// // Can read batches from both readers in parallel
    /// assert_eq!(
    ///   a.next().await.unwrap().unwrap(),
    ///   b.next().await.unwrap().unwrap(),
    /// );
    /// # }
    /// ```
    pub fn new_with_metadata(input: T, metadata: ArrowReaderMetadata) -> Self {
        Self::new_builder(AsyncReader(input), metadata)
    }

    /// Read bloom filter for a column in a row group
    /// Returns `None` if the column does not have a bloom filter
    ///
    /// We should call this function after other forms pruning, such as projection and predicate pushdown.
    pub async fn get_row_group_column_bloom_filter(
        &mut self,
        row_group_idx: usize,
        column_idx: usize,
    ) -> Result<Option<Sbbf>> {
        let metadata = self.metadata.row_group(row_group_idx);
        let column_metadata = metadata.column(column_idx);

        let offset: usize = if let Some(offset) = column_metadata.bloom_filter_offset() {
            offset
                .try_into()
                .map_err(|_| ParquetError::General("Bloom filter offset is invalid".to_string()))?
        } else {
            return Ok(None);
        };

        let buffer = match column_metadata.bloom_filter_length() {
            Some(length) => self.input.0.get_bytes(offset..offset + length as usize),
            None => self
                .input
                .0
                .get_bytes(offset..offset + SBBF_HEADER_SIZE_ESTIMATE),
        }
        .await?;

        let (header, bitset_offset) =
            chunk_read_bloom_filter_header_and_offset(offset as u64, buffer.clone())?;

        match header.algorithm {
            BloomFilterAlgorithm::BLOCK(_) => {
                // this match exists to future proof the singleton algorithm enum
            }
        }
        match header.compression {
            BloomFilterCompression::UNCOMPRESSED(_) => {
                // this match exists to future proof the singleton compression enum
            }
        }
        match header.hash {
            BloomFilterHash::XXHASH(_) => {
                // this match exists to future proof the singleton hash enum
            }
        }

        let bitset = match column_metadata.bloom_filter_length() {
            Some(_) => buffer.slice((bitset_offset as usize - offset)..),
            None => {
                let bitset_length: usize = header.num_bytes.try_into().map_err(|_| {
                    ParquetError::General("Bloom filter length is invalid".to_string())
                })?;
                self.input
                    .0
                    .get_bytes(bitset_offset as usize..bitset_offset as usize + bitset_length)
                    .await?
            }
        };
        Ok(Some(Sbbf::new(&bitset)))
    }

    /// Build a new [`ParquetRecordBatchStream`]
    pub fn build(self) -> Result<ParquetRecordBatchStream<T>> {
        let num_row_groups = self.metadata.row_groups().len();

        let row_groups = match self.row_groups {
            Some(row_groups) => {
                if let Some(col) = row_groups.iter().find(|x| **x >= num_row_groups) {
                    return Err(general_err!(
                        "row group {} out of bounds 0..{}",
                        col,
                        num_row_groups
                    ));
                }
                row_groups.into()
            }
            None => (0..self.metadata.row_groups().len()).collect(),
        };

        // Try to avoid allocate large buffer
        let batch_size = self
            .batch_size
            .min(self.metadata.file_metadata().num_rows() as usize);
        let reader = ReaderFactory {
            input: self.input.0,
            filter: self.filter,
            metadata: self.metadata.clone(),
            fields: self.fields,
            limit: self.limit,
            offset: self.offset,
        };

        // Ensure schema of ParquetRecordBatchStream respects projection, and does
        // not store metadata (same as for ParquetRecordBatchReader and emitted RecordBatches)
        let projected_fields = match reader.fields.as_deref().map(|pf| &pf.arrow_type) {
            Some(DataType::Struct(fields)) => {
                fields.filter_leaves(|idx, _| self.projection.leaf_included(idx))
            }
            None => Fields::empty(),
            _ => unreachable!("Must be Struct for root type"),
        };
        let schema = Arc::new(Schema::new(projected_fields));

        Ok(ParquetRecordBatchStream {
            metadata: self.metadata,
            batch_size,
            row_groups,
            projection: self.projection,
            selection: self.selection,
            schema,
            reader: Some(reader),
            state: StreamState::Init,
        })
    }
}

type ReadResult<T> = Result<(ReaderFactory<T>, Option<ParquetRecordBatchReader>)>;

/// [`ReaderFactory`] is used by [`ParquetRecordBatchStream`] to create
/// [`ParquetRecordBatchReader`]
struct ReaderFactory<T> {
    metadata: Arc<ParquetMetaData>,

    fields: Option<Arc<ParquetField>>,

    input: T,

    filter: Option<RowFilter>,

    limit: Option<usize>,

    offset: Option<usize>,
}

impl<T> ReaderFactory<T>
where
    T: AsyncFileReader + Send,
{
    /// Reads the next row group with the provided `selection`, `projection` and `batch_size`
    ///
    /// Note: this captures self so that the resulting future has a static lifetime
    async fn read_row_group(
        mut self,
        row_group_idx: usize,
        mut selection: Option<RowSelection>,
        projection: ProjectionMask,
        batch_size: usize,
    ) -> ReadResult<T> {
        // TODO: calling build_array multiple times is wasteful

        let meta = self.metadata.row_group(row_group_idx);
        let offset_index = self
            .metadata
            .offset_index()
            // filter out empty offset indexes (old versions specified Some(vec![]) when no present)
            .filter(|index| !index.is_empty())
            .map(|x| x[row_group_idx].as_slice());

        let mut row_group = InMemoryRowGroup {
            metadata: meta,
            // schema: meta.schema_descr_ptr(),
            row_count: meta.num_rows() as usize,
            column_chunks: vec![None; meta.columns().len()],
            offset_index,
        };

        if let Some(filter) = self.filter.as_mut() {
            for predicate in filter.predicates.iter_mut() {
                if !selects_any(selection.as_ref()) {
                    return Ok((self, None));
                }

                let predicate_projection = predicate.projection();
                row_group
                    .fetch(&mut self.input, predicate_projection, selection.as_ref())
                    .await?;

                let array_reader =
                    build_array_reader(self.fields.as_deref(), predicate_projection, &row_group)?;

                selection = Some(evaluate_predicate(
                    batch_size,
                    array_reader,
                    selection,
                    predicate.as_mut(),
                )?);
            }
        }

        // Compute the number of rows in the selection before applying limit and offset
        let rows_before = selection
            .as_ref()
            .map(|s| s.row_count())
            .unwrap_or(row_group.row_count);

        if rows_before == 0 {
            return Ok((self, None));
        }

        selection = apply_range(selection, row_group.row_count, self.offset, self.limit);

        // Compute the number of rows in the selection after applying limit and offset
        let rows_after = selection
            .as_ref()
            .map(|s| s.row_count())
            .unwrap_or(row_group.row_count);

        // Update offset if necessary
        if let Some(offset) = &mut self.offset {
            // Reduction is either because of offset or limit, as limit is applied
            // after offset has been "exhausted" can just use saturating sub here
            *offset = offset.saturating_sub(rows_before - rows_after)
        }

        if rows_after == 0 {
            return Ok((self, None));
        }

        if let Some(limit) = &mut self.limit {
            *limit -= rows_after;
        }

        row_group
            .fetch(&mut self.input, &projection, selection.as_ref())
            .await?;

        let reader = ParquetRecordBatchReader::new(
            batch_size,
            build_array_reader(self.fields.as_deref(), &projection, &row_group)?,
            selection,
        );

        Ok((self, Some(reader)))
    }
}

enum StreamState<T> {
    /// At the start of a new row group, or the end of the parquet stream
    Init,
    /// Decoding a batch
    Decoding(ParquetRecordBatchReader),
    /// Reading data from input
    Reading(BoxFuture<'static, ReadResult<T>>),
    /// Error
    Error,
}

impl<T> std::fmt::Debug for StreamState<T> {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        match self {
            StreamState::Init => write!(f, "StreamState::Init"),
            StreamState::Decoding(_) => write!(f, "StreamState::Decoding"),
            StreamState::Reading(_) => write!(f, "StreamState::Reading"),
            StreamState::Error => write!(f, "StreamState::Error"),
        }
    }
}

/// An asynchronous [`Stream`](https://docs.rs/futures/latest/futures/stream/trait.Stream.html) of [`RecordBatch`]
/// for a parquet file that can be constructed using [`ParquetRecordBatchStreamBuilder`].
pub struct ParquetRecordBatchStream<T> {
    metadata: Arc<ParquetMetaData>,

    schema: SchemaRef,

    row_groups: VecDeque<usize>,

    projection: ProjectionMask,

    batch_size: usize,

    selection: Option<RowSelection>,

    /// This is an option so it can be moved into a future
    reader: Option<ReaderFactory<T>>,

    state: StreamState<T>,
}

impl<T> std::fmt::Debug for ParquetRecordBatchStream<T> {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ParquetRecordBatchStream")
            .field("metadata", &self.metadata)
            .field("schema", &self.schema)
            .field("batch_size", &self.batch_size)
            .field("projection", &self.projection)
            .field("state", &self.state)
            .finish()
    }
}

impl<T> ParquetRecordBatchStream<T> {
    /// Returns the projected [`SchemaRef`] for reading the parquet file.
    ///
    /// Note that the schema metadata will be stripped here. See
    /// [`ParquetRecordBatchStreamBuilder::schema`] if the metadata is desired.
    pub fn schema(&self) -> &SchemaRef {
        &self.schema
    }
}

impl<T> Stream for ParquetRecordBatchStream<T>
where
    T: AsyncFileReader + Unpin + Send + 'static,
{
    type Item = Result<RecordBatch>;

    fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
        loop {
            match &mut self.state {
                StreamState::Decoding(batch_reader) => match batch_reader.next() {
                    Some(Ok(batch)) => {
                        return Poll::Ready(Some(Ok(batch)));
                    }
                    Some(Err(e)) => {
                        self.state = StreamState::Error;
                        return Poll::Ready(Some(Err(ParquetError::ArrowError(e.to_string()))));
                    }
                    None => self.state = StreamState::Init,
                },
                StreamState::Init => {
                    let row_group_idx = match self.row_groups.pop_front() {
                        Some(idx) => idx,
                        None => return Poll::Ready(None),
                    };

                    let reader = self.reader.take().expect("lost reader");

                    let row_count = self.metadata.row_group(row_group_idx).num_rows() as usize;

                    let selection = self.selection.as_mut().map(|s| s.split_off(row_count));

                    let fut = reader
                        .read_row_group(
                            row_group_idx,
                            selection,
                            self.projection.clone(),
                            self.batch_size,
                        )
                        .boxed();

                    self.state = StreamState::Reading(fut)
                }
                StreamState::Reading(f) => match ready!(f.poll_unpin(cx)) {
                    Ok((reader_factory, maybe_reader)) => {
                        self.reader = Some(reader_factory);
                        match maybe_reader {
                            // Read records from [`ParquetRecordBatchReader`]
                            Some(reader) => self.state = StreamState::Decoding(reader),
                            // All rows skipped, read next row group
                            None => self.state = StreamState::Init,
                        }
                    }
                    Err(e) => {
                        self.state = StreamState::Error;
                        return Poll::Ready(Some(Err(e)));
                    }
                },
                StreamState::Error => return Poll::Ready(None), // Ends the stream as error happens.
            }
        }
    }
}

/// An in-memory collection of column chunks
struct InMemoryRowGroup<'a> {
    metadata: &'a RowGroupMetaData,
    offset_index: Option<&'a [OffsetIndexMetaData]>,
    column_chunks: Vec<Option<Arc<ColumnChunkData>>>,
    row_count: usize,
}

impl<'a> InMemoryRowGroup<'a> {
    /// Fetches the necessary column data into memory
    async fn fetch<T: AsyncFileReader + Send>(
        &mut self,
        input: &mut T,
        projection: &ProjectionMask,
        selection: Option<&RowSelection>,
    ) -> Result<()> {
        if let Some((selection, offset_index)) = selection.zip(self.offset_index) {
            // If we have a `RowSelection` and an `OffsetIndex` then only fetch pages required for the
            // `RowSelection`
            let mut page_start_offsets: Vec<Vec<usize>> = vec![];

            let fetch_ranges = self
                .column_chunks
                .iter()
                .zip(self.metadata.columns())
                .enumerate()
                .filter(|&(idx, (chunk, _chunk_meta))| {
                    chunk.is_none() && projection.leaf_included(idx)
                })
                .flat_map(|(idx, (_chunk, chunk_meta))| {
                    // If the first page does not start at the beginning of the column,
                    // then we need to also fetch a dictionary page.
                    let mut ranges = vec![];
                    let (start, _len) = chunk_meta.byte_range();
                    match offset_index[idx].page_locations.first() {
                        Some(first) if first.offset as u64 != start => {
                            ranges.push(start as usize..first.offset as usize);
                        }
                        _ => (),
                    }

                    ranges.extend(selection.scan_ranges(&offset_index[idx].page_locations));
                    page_start_offsets.push(ranges.iter().map(|range| range.start).collect());

                    ranges
                })
                .collect();

            let mut chunk_data = input.get_byte_ranges(fetch_ranges).await?.into_iter();
            let mut page_start_offsets = page_start_offsets.into_iter();

            for (idx, chunk) in self.column_chunks.iter_mut().enumerate() {
                if chunk.is_some() || !projection.leaf_included(idx) {
                    continue;
                }

                if let Some(offsets) = page_start_offsets.next() {
                    let mut chunks = Vec::with_capacity(offsets.len());
                    for _ in 0..offsets.len() {
                        chunks.push(chunk_data.next().unwrap());
                    }

                    *chunk = Some(Arc::new(ColumnChunkData::Sparse {
                        length: self.metadata.column(idx).byte_range().1 as usize,
                        data: offsets.into_iter().zip(chunks.into_iter()).collect(),
                    }))
                }
            }
        } else {
            let fetch_ranges = self
                .column_chunks
                .iter()
                .enumerate()
                .filter(|&(idx, chunk)| chunk.is_none() && projection.leaf_included(idx))
                .map(|(idx, _chunk)| {
                    let column = self.metadata.column(idx);
                    let (start, length) = column.byte_range();
                    start as usize..(start + length) as usize
                })
                .collect();

            let mut chunk_data = input.get_byte_ranges(fetch_ranges).await?.into_iter();

            for (idx, chunk) in self.column_chunks.iter_mut().enumerate() {
                if chunk.is_some() || !projection.leaf_included(idx) {
                    continue;
                }

                if let Some(data) = chunk_data.next() {
                    *chunk = Some(Arc::new(ColumnChunkData::Dense {
                        offset: self.metadata.column(idx).byte_range().0 as usize,
                        data,
                    }));
                }
            }
        }

        Ok(())
    }
}

impl RowGroups for InMemoryRowGroup<'_> {
    fn num_rows(&self) -> usize {
        self.row_count
    }

    fn column_chunks(&self, i: usize) -> Result<Box<dyn PageIterator>> {
        match &self.column_chunks[i] {
            None => Err(ParquetError::General(format!(
                "Invalid column index {i}, column was not fetched"
            ))),
            Some(data) => {
                let page_locations = self
                    .offset_index
                    // filter out empty offset indexes (old versions specified Some(vec![]) when no present)
                    .filter(|index| !index.is_empty())
                    .map(|index| index[i].page_locations.clone());
                let page_reader: Box<dyn PageReader> = Box::new(SerializedPageReader::new(
                    data.clone(),
                    self.metadata.column(i),
                    self.row_count,
                    page_locations,
                )?);

                Ok(Box::new(ColumnChunkIterator {
                    reader: Some(Ok(page_reader)),
                }))
            }
        }
    }
}

/// An in-memory column chunk
#[derive(Clone)]
enum ColumnChunkData {
    /// Column chunk data representing only a subset of data pages
    Sparse {
        /// Length of the full column chunk
        length: usize,
        /// Set of data pages included in this sparse chunk. Each element is a tuple
        /// of (page offset, page data)
        data: Vec<(usize, Bytes)>,
    },
    /// Full column chunk and its offset
    Dense { offset: usize, data: Bytes },
}

impl ColumnChunkData {
    fn get(&self, start: u64) -> Result<Bytes> {
        match &self {
            ColumnChunkData::Sparse { data, .. } => data
                .binary_search_by_key(&start, |(offset, _)| *offset as u64)
                .map(|idx| data[idx].1.clone())
                .map_err(|_| {
                    ParquetError::General(format!(
                        "Invalid offset in sparse column chunk data: {start}"
                    ))
                }),
            ColumnChunkData::Dense { offset, data } => {
                let start = start as usize - *offset;
                Ok(data.slice(start..))
            }
        }
    }
}

impl Length for ColumnChunkData {
    fn len(&self) -> u64 {
        match &self {
            ColumnChunkData::Sparse { length, .. } => *length as u64,
            ColumnChunkData::Dense { data, .. } => data.len() as u64,
        }
    }
}

impl ChunkReader for ColumnChunkData {
    type T = bytes::buf::Reader<Bytes>;

    fn get_read(&self, start: u64) -> Result<Self::T> {
        Ok(self.get(start)?.reader())
    }

    fn get_bytes(&self, start: u64, length: usize) -> Result<Bytes> {
        Ok(self.get(start)?.slice(..length))
    }
}

/// Implements [`PageIterator`] for a single column chunk, yielding a single [`PageReader`]
struct ColumnChunkIterator {
    reader: Option<Result<Box<dyn PageReader>>>,
}

impl Iterator for ColumnChunkIterator {
    type Item = Result<Box<dyn PageReader>>;

    fn next(&mut self) -> Option<Self::Item> {
        self.reader.take()
    }
}

impl PageIterator for ColumnChunkIterator {}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::arrow::arrow_reader::{
        ArrowPredicateFn, ParquetRecordBatchReaderBuilder, RowSelector,
    };
    use crate::arrow::schema::parquet_to_arrow_schema_and_fields;
    use crate::arrow::ArrowWriter;
    use crate::file::metadata::ParquetMetaDataReader;
    use crate::file::page_index::index_reader;
    use crate::file::properties::WriterProperties;
    use arrow::compute::kernels::cmp::eq;
    use arrow::error::Result as ArrowResult;
    use arrow_array::builder::{ListBuilder, StringBuilder};
    use arrow_array::cast::AsArray;
    use arrow_array::types::Int32Type;
    use arrow_array::{
        Array, ArrayRef, Int32Array, Int8Array, RecordBatchReader, Scalar, StringArray,
        StructArray, UInt64Array,
    };
    use arrow_schema::{DataType, Field, Schema};
    use futures::{StreamExt, TryStreamExt};
    use rand::{thread_rng, Rng};
    use std::collections::HashMap;
    use std::sync::{Arc, Mutex};
    use tempfile::tempfile;

    #[derive(Clone)]
    struct TestReader {
        data: Bytes,
        metadata: Arc<ParquetMetaData>,
        requests: Arc<Mutex<Vec<Range<usize>>>>,
    }

    impl AsyncFileReader for TestReader {
        fn get_bytes(&mut self, range: Range<usize>) -> BoxFuture<'_, Result<Bytes>> {
            self.requests.lock().unwrap().push(range.clone());
            futures::future::ready(Ok(self.data.slice(range))).boxed()
        }

        fn get_metadata(&mut self) -> BoxFuture<'_, Result<Arc<ParquetMetaData>>> {
            futures::future::ready(Ok(self.metadata.clone())).boxed()
        }
    }

    #[tokio::test]
    async fn test_async_reader() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let requests = async_reader.requests.clone();
        let builder = ParquetRecordBatchStreamBuilder::new(async_reader)
            .await
            .unwrap();

        let mask = ProjectionMask::leaves(builder.parquet_schema(), vec![1, 2]);
        let stream = builder
            .with_projection(mask.clone())
            .with_batch_size(1024)
            .build()
            .unwrap();

        let async_batches: Vec<_> = stream.try_collect().await.unwrap();

        let sync_batches = ParquetRecordBatchReaderBuilder::try_new(data)
            .unwrap()
            .with_projection(mask)
            .with_batch_size(104)
            .build()
            .unwrap()
            .collect::<ArrowResult<Vec<_>>>()
            .unwrap();

        assert_eq!(async_batches, sync_batches);

        let requests = requests.lock().unwrap();
        let (offset_1, length_1) = metadata.row_group(0).column(1).byte_range();
        let (offset_2, length_2) = metadata.row_group(0).column(2).byte_range();

        assert_eq!(
            &requests[..],
            &[
                offset_1 as usize..(offset_1 + length_1) as usize,
                offset_2 as usize..(offset_2 + length_2) as usize
            ]
        );
    }

    #[tokio::test]
    async fn test_async_reader_with_index() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let options = ArrowReaderOptions::new().with_page_index(true);
        let builder = ParquetRecordBatchStreamBuilder::new_with_options(async_reader, options)
            .await
            .unwrap();

        // The builder should have page and offset indexes loaded now
        let metadata_with_index = builder.metadata();

        // Check offset indexes are present for all columns
        let offset_index = metadata_with_index.offset_index().unwrap();
        let column_index = metadata_with_index.column_index().unwrap();

        assert_eq!(offset_index.len(), metadata_with_index.num_row_groups());
        assert_eq!(column_index.len(), metadata_with_index.num_row_groups());

        let num_columns = metadata_with_index
            .file_metadata()
            .schema_descr()
            .num_columns();

        // Check page indexes are present for all columns
        offset_index
            .iter()
            .for_each(|x| assert_eq!(x.len(), num_columns));
        column_index
            .iter()
            .for_each(|x| assert_eq!(x.len(), num_columns));

        let mask = ProjectionMask::leaves(builder.parquet_schema(), vec![1, 2]);
        let stream = builder
            .with_projection(mask.clone())
            .with_batch_size(1024)
            .build()
            .unwrap();

        let async_batches: Vec<_> = stream.try_collect().await.unwrap();

        let sync_batches = ParquetRecordBatchReaderBuilder::try_new(data)
            .unwrap()
            .with_projection(mask)
            .with_batch_size(1024)
            .build()
            .unwrap()
            .collect::<ArrowResult<Vec<_>>>()
            .unwrap();

        assert_eq!(async_batches, sync_batches);
    }

    #[tokio::test]
    async fn test_async_reader_with_limit() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let builder = ParquetRecordBatchStreamBuilder::new(async_reader)
            .await
            .unwrap();

        let mask = ProjectionMask::leaves(builder.parquet_schema(), vec![1, 2]);
        let stream = builder
            .with_projection(mask.clone())
            .with_batch_size(1024)
            .with_limit(1)
            .build()
            .unwrap();

        let async_batches: Vec<_> = stream.try_collect().await.unwrap();

        let sync_batches = ParquetRecordBatchReaderBuilder::try_new(data)
            .unwrap()
            .with_projection(mask)
            .with_batch_size(1024)
            .with_limit(1)
            .build()
            .unwrap()
            .collect::<ArrowResult<Vec<_>>>()
            .unwrap();

        assert_eq!(async_batches, sync_batches);
    }

    #[tokio::test]
    async fn test_async_reader_skip_pages() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let options = ArrowReaderOptions::new().with_page_index(true);
        let builder = ParquetRecordBatchStreamBuilder::new_with_options(async_reader, options)
            .await
            .unwrap();

        let selection = RowSelection::from(vec![
            RowSelector::skip(21),   // Skip first page
            RowSelector::select(21), // Select page to boundary
            RowSelector::skip(41),   // Skip multiple pages
            RowSelector::select(41), // Select multiple pages
            RowSelector::skip(25),   // Skip page across boundary
            RowSelector::select(25), // Select across page boundary
            RowSelector::skip(7116), // Skip to final page boundary
            RowSelector::select(10), // Select final page
        ]);

        let mask = ProjectionMask::leaves(builder.parquet_schema(), vec![9]);

        let stream = builder
            .with_projection(mask.clone())
            .with_row_selection(selection.clone())
            .build()
            .expect("building stream");

        let async_batches: Vec<_> = stream.try_collect().await.unwrap();

        let sync_batches = ParquetRecordBatchReaderBuilder::try_new(data)
            .unwrap()
            .with_projection(mask)
            .with_batch_size(1024)
            .with_row_selection(selection)
            .build()
            .unwrap()
            .collect::<ArrowResult<Vec<_>>>()
            .unwrap();

        assert_eq!(async_batches, sync_batches);
    }

    #[tokio::test]
    async fn test_fuzz_async_reader_selection() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);

        let mut rand = thread_rng();

        for _ in 0..100 {
            let mut expected_rows = 0;
            let mut total_rows = 0;
            let mut skip = false;
            let mut selectors = vec![];

            while total_rows < 7300 {
                let row_count: usize = rand.gen_range(1..100);

                let row_count = row_count.min(7300 - total_rows);

                selectors.push(RowSelector { row_count, skip });

                total_rows += row_count;
                if !skip {
                    expected_rows += row_count;
                }

                skip = !skip;
            }

            let selection = RowSelection::from(selectors);

            let async_reader = TestReader {
                data: data.clone(),
                metadata: metadata.clone(),
                requests: Default::default(),
            };

            let options = ArrowReaderOptions::new().with_page_index(true);
            let builder = ParquetRecordBatchStreamBuilder::new_with_options(async_reader, options)
                .await
                .unwrap();

            let col_idx: usize = rand.gen_range(0..13);
            let mask = ProjectionMask::leaves(builder.parquet_schema(), vec![col_idx]);

            let stream = builder
                .with_projection(mask.clone())
                .with_row_selection(selection.clone())
                .build()
                .expect("building stream");

            let async_batches: Vec<_> = stream.try_collect().await.unwrap();

            let actual_rows: usize = async_batches.into_iter().map(|b| b.num_rows()).sum();

            assert_eq!(actual_rows, expected_rows);
        }
    }

    #[tokio::test]
    async fn test_async_reader_zero_row_selector() {
        //See https://github.com/apache/arrow-rs/issues/2669
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);

        let mut rand = thread_rng();

        let mut expected_rows = 0;
        let mut total_rows = 0;
        let mut skip = false;
        let mut selectors = vec![];

        selectors.push(RowSelector {
            row_count: 0,
            skip: false,
        });

        while total_rows < 7300 {
            let row_count: usize = rand.gen_range(1..100);

            let row_count = row_count.min(7300 - total_rows);

            selectors.push(RowSelector { row_count, skip });

            total_rows += row_count;
            if !skip {
                expected_rows += row_count;
            }

            skip = !skip;
        }

        let selection = RowSelection::from(selectors);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let options = ArrowReaderOptions::new().with_page_index(true);
        let builder = ParquetRecordBatchStreamBuilder::new_with_options(async_reader, options)
            .await
            .unwrap();

        let col_idx: usize = rand.gen_range(0..13);
        let mask = ProjectionMask::leaves(builder.parquet_schema(), vec![col_idx]);

        let stream = builder
            .with_projection(mask.clone())
            .with_row_selection(selection.clone())
            .build()
            .expect("building stream");

        let async_batches: Vec<_> = stream.try_collect().await.unwrap();

        let actual_rows: usize = async_batches.into_iter().map(|b| b.num_rows()).sum();

        assert_eq!(actual_rows, expected_rows);
    }

    #[tokio::test]
    async fn test_row_filter() {
        let a = StringArray::from_iter_values(["a", "b", "b", "b", "c", "c"]);
        let b = StringArray::from_iter_values(["1", "2", "3", "4", "5", "6"]);
        let c = Int32Array::from_iter(0..6);
        let data = RecordBatch::try_from_iter([
            ("a", Arc::new(a) as ArrayRef),
            ("b", Arc::new(b) as ArrayRef),
            ("c", Arc::new(c) as ArrayRef),
        ])
        .unwrap();

        let mut buf = Vec::with_capacity(1024);
        let mut writer = ArrowWriter::try_new(&mut buf, data.schema(), None).unwrap();
        writer.write(&data).unwrap();
        writer.close().unwrap();

        let data: Bytes = buf.into();
        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let parquet_schema = metadata.file_metadata().schema_descr_ptr();

        let test = TestReader {
            data,
            metadata: Arc::new(metadata),
            requests: Default::default(),
        };
        let requests = test.requests.clone();

        let a_scalar = StringArray::from_iter_values(["b"]);
        let a_filter = ArrowPredicateFn::new(
            ProjectionMask::leaves(&parquet_schema, vec![0]),
            move |batch| eq(batch.column(0), &Scalar::new(&a_scalar)),
        );

        let b_scalar = StringArray::from_iter_values(["4"]);
        let b_filter = ArrowPredicateFn::new(
            ProjectionMask::leaves(&parquet_schema, vec![1]),
            move |batch| eq(batch.column(0), &Scalar::new(&b_scalar)),
        );

        let filter = RowFilter::new(vec![Box::new(a_filter), Box::new(b_filter)]);

        let mask = ProjectionMask::leaves(&parquet_schema, vec![0, 2]);
        let stream = ParquetRecordBatchStreamBuilder::new(test)
            .await
            .unwrap()
            .with_projection(mask.clone())
            .with_batch_size(1024)
            .with_row_filter(filter)
            .build()
            .unwrap();

        let batches: Vec<_> = stream.try_collect().await.unwrap();
        assert_eq!(batches.len(), 1);

        let batch = &batches[0];
        assert_eq!(batch.num_rows(), 1);
        assert_eq!(batch.num_columns(), 2);

        let col = batch.column(0);
        let val = col.as_any().downcast_ref::<StringArray>().unwrap().value(0);
        assert_eq!(val, "b");

        let col = batch.column(1);
        let val = col.as_any().downcast_ref::<Int32Array>().unwrap().value(0);
        assert_eq!(val, 3);

        // Should only have made 3 requests
        assert_eq!(requests.lock().unwrap().len(), 3);
    }

    #[tokio::test]
    async fn test_limit_multiple_row_groups() {
        let a = StringArray::from_iter_values(["a", "b", "b", "b", "c", "c"]);
        let b = StringArray::from_iter_values(["1", "2", "3", "4", "5", "6"]);
        let c = Int32Array::from_iter(0..6);
        let data = RecordBatch::try_from_iter([
            ("a", Arc::new(a) as ArrayRef),
            ("b", Arc::new(b) as ArrayRef),
            ("c", Arc::new(c) as ArrayRef),
        ])
        .unwrap();

        let mut buf = Vec::with_capacity(1024);
        let props = WriterProperties::builder()
            .set_max_row_group_size(3)
            .build();
        let mut writer = ArrowWriter::try_new(&mut buf, data.schema(), Some(props)).unwrap();
        writer.write(&data).unwrap();
        writer.close().unwrap();

        let data: Bytes = buf.into();
        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();

        assert_eq!(metadata.num_row_groups(), 2);

        let test = TestReader {
            data,
            metadata: Arc::new(metadata),
            requests: Default::default(),
        };

        let stream = ParquetRecordBatchStreamBuilder::new(test.clone())
            .await
            .unwrap()
            .with_batch_size(1024)
            .with_limit(4)
            .build()
            .unwrap();

        let batches: Vec<_> = stream.try_collect().await.unwrap();
        // Expect one batch for each row group
        assert_eq!(batches.len(), 2);

        let batch = &batches[0];
        // First batch should contain all rows
        assert_eq!(batch.num_rows(), 3);
        assert_eq!(batch.num_columns(), 3);
        let col2 = batch.column(2).as_primitive::<Int32Type>();
        assert_eq!(col2.values(), &[0, 1, 2]);

        let batch = &batches[1];
        // Second batch should trigger the limit and only have one row
        assert_eq!(batch.num_rows(), 1);
        assert_eq!(batch.num_columns(), 3);
        let col2 = batch.column(2).as_primitive::<Int32Type>();
        assert_eq!(col2.values(), &[3]);

        let stream = ParquetRecordBatchStreamBuilder::new(test.clone())
            .await
            .unwrap()
            .with_offset(2)
            .with_limit(3)
            .build()
            .unwrap();

        let batches: Vec<_> = stream.try_collect().await.unwrap();
        // Expect one batch for each row group
        assert_eq!(batches.len(), 2);

        let batch = &batches[0];
        // First batch should contain one row
        assert_eq!(batch.num_rows(), 1);
        assert_eq!(batch.num_columns(), 3);
        let col2 = batch.column(2).as_primitive::<Int32Type>();
        assert_eq!(col2.values(), &[2]);

        let batch = &batches[1];
        // Second batch should contain two rows
        assert_eq!(batch.num_rows(), 2);
        assert_eq!(batch.num_columns(), 3);
        let col2 = batch.column(2).as_primitive::<Int32Type>();
        assert_eq!(col2.values(), &[3, 4]);

        let stream = ParquetRecordBatchStreamBuilder::new(test.clone())
            .await
            .unwrap()
            .with_offset(4)
            .with_limit(20)
            .build()
            .unwrap();

        let batches: Vec<_> = stream.try_collect().await.unwrap();
        // Should skip first row group
        assert_eq!(batches.len(), 1);

        let batch = &batches[0];
        // First batch should contain two rows
        assert_eq!(batch.num_rows(), 2);
        assert_eq!(batch.num_columns(), 3);
        let col2 = batch.column(2).as_primitive::<Int32Type>();
        assert_eq!(col2.values(), &[4, 5]);
    }

    #[tokio::test]
    async fn test_row_filter_with_index() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let parquet_schema = metadata.file_metadata().schema_descr_ptr();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let a_filter =
            ArrowPredicateFn::new(ProjectionMask::leaves(&parquet_schema, vec![1]), |batch| {
                Ok(batch.column(0).as_boolean().clone())
            });

        let b_scalar = Int8Array::from(vec![2]);
        let b_filter = ArrowPredicateFn::new(
            ProjectionMask::leaves(&parquet_schema, vec![2]),
            move |batch| eq(batch.column(0), &Scalar::new(&b_scalar)),
        );

        let filter = RowFilter::new(vec![Box::new(a_filter), Box::new(b_filter)]);

        let mask = ProjectionMask::leaves(&parquet_schema, vec![0, 2]);

        let options = ArrowReaderOptions::new().with_page_index(true);
        let stream = ParquetRecordBatchStreamBuilder::new_with_options(async_reader, options)
            .await
            .unwrap()
            .with_projection(mask.clone())
            .with_batch_size(1024)
            .with_row_filter(filter)
            .build()
            .unwrap();

        let batches: Vec<RecordBatch> = stream.try_collect().await.unwrap();

        let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum();

        assert_eq!(total_rows, 730);
    }

    #[tokio::test]
    async fn test_in_memory_row_group_sparse() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();

        let offset_index =
            index_reader::read_offset_indexes(&data, metadata.row_group(0).columns())
                .expect("reading offset index");

        let mut metadata_builder = metadata.into_builder();
        let mut row_groups = metadata_builder.take_row_groups();
        row_groups.truncate(1);
        let row_group_meta = row_groups.pop().unwrap();

        let metadata = metadata_builder
            .add_row_group(row_group_meta)
            .set_column_index(None)
            .set_offset_index(Some(vec![offset_index.clone()]))
            .build();

        let metadata = Arc::new(metadata);

        let num_rows = metadata.row_group(0).num_rows();

        assert_eq!(metadata.num_row_groups(), 1);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let requests = async_reader.requests.clone();
        let (_, fields) = parquet_to_arrow_schema_and_fields(
            metadata.file_metadata().schema_descr(),
            ProjectionMask::all(),
            None,
        )
        .unwrap();

        let _schema_desc = metadata.file_metadata().schema_descr();

        let projection = ProjectionMask::leaves(metadata.file_metadata().schema_descr(), vec![0]);

        let reader_factory = ReaderFactory {
            metadata,
            fields: fields.map(Arc::new),
            input: async_reader,
            filter: None,
            limit: None,
            offset: None,
        };

        let mut skip = true;
        let mut pages = offset_index[0].page_locations.iter().peekable();

        // Setup `RowSelection` so that we can skip every other page, selecting the last page
        let mut selectors = vec![];
        let mut expected_page_requests: Vec<Range<usize>> = vec![];
        while let Some(page) = pages.next() {
            let num_rows = if let Some(next_page) = pages.peek() {
                next_page.first_row_index - page.first_row_index
            } else {
                num_rows - page.first_row_index
            };

            if skip {
                selectors.push(RowSelector::skip(num_rows as usize));
            } else {
                selectors.push(RowSelector::select(num_rows as usize));
                let start = page.offset as usize;
                let end = start + page.compressed_page_size as usize;
                expected_page_requests.push(start..end);
            }
            skip = !skip;
        }

        let selection = RowSelection::from(selectors);

        let (_factory, _reader) = reader_factory
            .read_row_group(0, Some(selection), projection.clone(), 48)
            .await
            .expect("reading row group");

        let requests = requests.lock().unwrap();

        assert_eq!(&requests[..], &expected_page_requests)
    }

    #[tokio::test]
    async fn test_batch_size_overallocate() {
        let testdata = arrow::util::test_util::parquet_test_data();
        // `alltypes_plain.parquet` only have 8 rows
        let path = format!("{testdata}/alltypes_plain.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());

        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let file_rows = metadata.file_metadata().num_rows() as usize;
        let metadata = Arc::new(metadata);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let builder = ParquetRecordBatchStreamBuilder::new(async_reader)
            .await
            .unwrap();

        let stream = builder
            .with_projection(ProjectionMask::all())
            .with_batch_size(1024)
            .build()
            .unwrap();
        assert_ne!(1024, file_rows);
        assert_eq!(stream.batch_size, file_rows);
    }

    #[tokio::test]
    async fn test_get_row_group_column_bloom_filter_without_length() {
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/data_index_bloom_encoding_stats.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());
        test_get_row_group_column_bloom_filter(data, false).await;
    }

    #[tokio::test]
    async fn test_parquet_record_batch_stream_schema() {
        fn get_all_field_names(schema: &Schema) -> Vec<&String> {
            schema.flattened_fields().iter().map(|f| f.name()).collect()
        }

        // ParquetRecordBatchReaderBuilder::schema differs from
        // ParquetRecordBatchReader::schema and RecordBatch::schema in the returned
        // schema contents (in terms of custom metadata attached to schema, and fields
        // returned). Test to ensure this remains consistent behaviour.
        //
        // Ensure same for asynchronous versions of the above.

        // Prep data, for a schema with nested fields, with custom metadata
        let mut metadata = HashMap::with_capacity(1);
        metadata.insert("key".to_string(), "value".to_string());

        let nested_struct_array = StructArray::from(vec![
            (
                Arc::new(Field::new("d", DataType::Utf8, true)),
                Arc::new(StringArray::from(vec!["a", "b"])) as ArrayRef,
            ),
            (
                Arc::new(Field::new("e", DataType::Utf8, true)),
                Arc::new(StringArray::from(vec!["c", "d"])) as ArrayRef,
            ),
        ]);
        let struct_array = StructArray::from(vec![
            (
                Arc::new(Field::new("a", DataType::Int32, true)),
                Arc::new(Int32Array::from(vec![-1, 1])) as ArrayRef,
            ),
            (
                Arc::new(Field::new("b", DataType::UInt64, true)),
                Arc::new(UInt64Array::from(vec![1, 2])) as ArrayRef,
            ),
            (
                Arc::new(Field::new(
                    "c",
                    nested_struct_array.data_type().clone(),
                    true,
                )),
                Arc::new(nested_struct_array) as ArrayRef,
            ),
        ]);

        let schema =
            Arc::new(Schema::new(struct_array.fields().clone()).with_metadata(metadata.clone()));
        let record_batch = RecordBatch::from(struct_array)
            .with_schema(schema.clone())
            .unwrap();

        // Write parquet with custom metadata in schema
        let mut file = tempfile().unwrap();
        let mut writer = ArrowWriter::try_new(&mut file, schema.clone(), None).unwrap();
        writer.write(&record_batch).unwrap();
        writer.close().unwrap();

        let all_fields = ["a", "b", "c", "d", "e"];
        // (leaf indices in mask, expected names in output schema all fields)
        let projections = [
            (vec![], vec![]),
            (vec![0], vec!["a"]),
            (vec![0, 1], vec!["a", "b"]),
            (vec![0, 1, 2], vec!["a", "b", "c", "d"]),
            (vec![0, 1, 2, 3], vec!["a", "b", "c", "d", "e"]),
        ];

        // Ensure we're consistent for each of these projections
        for (indices, expected_projected_names) in projections {
            let assert_schemas = |builder: SchemaRef, reader: SchemaRef, batch: SchemaRef| {
                // Builder schema should preserve all fields and metadata
                assert_eq!(get_all_field_names(&builder), all_fields);
                assert_eq!(builder.metadata, metadata);
                // Reader & batch schema should show only projected fields, and no metadata
                assert_eq!(get_all_field_names(&reader), expected_projected_names);
                assert_eq!(reader.metadata, HashMap::default());
                assert_eq!(get_all_field_names(&batch), expected_projected_names);
                assert_eq!(batch.metadata, HashMap::default());
            };

            let builder =
                ParquetRecordBatchReaderBuilder::try_new(file.try_clone().unwrap()).unwrap();
            let sync_builder_schema = builder.schema().clone();
            let mask = ProjectionMask::leaves(builder.parquet_schema(), indices.clone());
            let mut reader = builder.with_projection(mask).build().unwrap();
            let sync_reader_schema = reader.schema();
            let batch = reader.next().unwrap().unwrap();
            let sync_batch_schema = batch.schema();
            assert_schemas(sync_builder_schema, sync_reader_schema, sync_batch_schema);

            // asynchronous should be same
            let file = tokio::fs::File::from(file.try_clone().unwrap());
            let builder = ParquetRecordBatchStreamBuilder::new(file).await.unwrap();
            let async_builder_schema = builder.schema().clone();
            let mask = ProjectionMask::leaves(builder.parquet_schema(), indices);
            let mut reader = builder.with_projection(mask).build().unwrap();
            let async_reader_schema = reader.schema().clone();
            let batch = reader.next().await.unwrap().unwrap();
            let async_batch_schema = batch.schema();
            assert_schemas(
                async_builder_schema,
                async_reader_schema,
                async_batch_schema,
            );
        }
    }

    #[tokio::test]
    async fn test_get_row_group_column_bloom_filter_with_length() {
        // convert to new parquet file with bloom_filter_length
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/data_index_bloom_encoding_stats.parquet");
        let data = Bytes::from(std::fs::read(path).unwrap());
        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);
        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };
        let builder = ParquetRecordBatchStreamBuilder::new(async_reader)
            .await
            .unwrap();
        let schema = builder.schema().clone();
        let stream = builder.build().unwrap();
        let batches = stream.try_collect::<Vec<_>>().await.unwrap();

        let mut parquet_data = Vec::new();
        let props = WriterProperties::builder()
            .set_bloom_filter_enabled(true)
            .build();
        let mut writer = ArrowWriter::try_new(&mut parquet_data, schema, Some(props)).unwrap();
        for batch in batches {
            writer.write(&batch).unwrap();
        }
        writer.close().unwrap();

        // test the new parquet file
        test_get_row_group_column_bloom_filter(parquet_data.into(), true).await;
    }

    async fn test_get_row_group_column_bloom_filter(data: Bytes, with_length: bool) {
        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let metadata = Arc::new(metadata);

        assert_eq!(metadata.num_row_groups(), 1);
        let row_group = metadata.row_group(0);
        let column = row_group.column(0);
        assert_eq!(column.bloom_filter_length().is_some(), with_length);

        let async_reader = TestReader {
            data: data.clone(),
            metadata: metadata.clone(),
            requests: Default::default(),
        };

        let mut builder = ParquetRecordBatchStreamBuilder::new(async_reader)
            .await
            .unwrap();

        let sbbf = builder
            .get_row_group_column_bloom_filter(0, 0)
            .await
            .unwrap()
            .unwrap();
        assert!(sbbf.check(&"Hello"));
        assert!(!sbbf.check(&"Hello_Not_Exists"));
    }

    #[tokio::test]
    async fn test_nested_skip() {
        let schema = Arc::new(Schema::new(vec![
            Field::new("col_1", DataType::UInt64, false),
            Field::new_list("col_2", Field::new("item", DataType::Utf8, true), true),
        ]));

        // Default writer properties
        let props = WriterProperties::builder()
            .set_data_page_row_count_limit(256)
            .set_write_batch_size(256)
            .set_max_row_group_size(1024);

        // Write data
        let mut file = tempfile().unwrap();
        let mut writer =
            ArrowWriter::try_new(&mut file, schema.clone(), Some(props.build())).unwrap();

        let mut builder = ListBuilder::new(StringBuilder::new());
        for id in 0..1024 {
            match id % 3 {
                0 => builder.append_value([Some("val_1".to_string()), Some(format!("id_{id}"))]),
                1 => builder.append_value([Some(format!("id_{id}"))]),
                _ => builder.append_null(),
            }
        }
        let refs = vec![
            Arc::new(UInt64Array::from_iter_values(0..1024)) as ArrayRef,
            Arc::new(builder.finish()) as ArrayRef,
        ];

        let batch = RecordBatch::try_new(schema.clone(), refs).unwrap();
        writer.write(&batch).unwrap();
        writer.close().unwrap();

        let selections = [
            RowSelection::from(vec![
                RowSelector::skip(313),
                RowSelector::select(1),
                RowSelector::skip(709),
                RowSelector::select(1),
            ]),
            RowSelection::from(vec![
                RowSelector::skip(255),
                RowSelector::select(1),
                RowSelector::skip(767),
                RowSelector::select(1),
            ]),
            RowSelection::from(vec![
                RowSelector::select(255),
                RowSelector::skip(1),
                RowSelector::select(767),
                RowSelector::skip(1),
            ]),
            RowSelection::from(vec![
                RowSelector::skip(254),
                RowSelector::select(1),
                RowSelector::select(1),
                RowSelector::skip(767),
                RowSelector::select(1),
            ]),
        ];

        for selection in selections {
            let expected = selection.row_count();
            // Read data
            let mut reader = ParquetRecordBatchStreamBuilder::new_with_options(
                tokio::fs::File::from_std(file.try_clone().unwrap()),
                ArrowReaderOptions::new().with_page_index(true),
            )
            .await
            .unwrap();

            reader = reader.with_row_selection(selection);

            let mut stream = reader.build().unwrap();

            let mut total_rows = 0;
            while let Some(rb) = stream.next().await {
                let rb = rb.unwrap();
                total_rows += rb.num_rows();
            }
            assert_eq!(total_rows, expected);
        }
    }

    #[tokio::test]
    async fn test_row_filter_nested() {
        let a = StringArray::from_iter_values(["a", "b", "b", "b", "c", "c"]);
        let b = StructArray::from(vec![
            (
                Arc::new(Field::new("aa", DataType::Utf8, true)),
                Arc::new(StringArray::from(vec!["a", "b", "b", "b", "c", "c"])) as ArrayRef,
            ),
            (
                Arc::new(Field::new("bb", DataType::Utf8, true)),
                Arc::new(StringArray::from(vec!["1", "2", "3", "4", "5", "6"])) as ArrayRef,
            ),
        ]);
        let c = Int32Array::from_iter(0..6);
        let data = RecordBatch::try_from_iter([
            ("a", Arc::new(a) as ArrayRef),
            ("b", Arc::new(b) as ArrayRef),
            ("c", Arc::new(c) as ArrayRef),
        ])
        .unwrap();

        let mut buf = Vec::with_capacity(1024);
        let mut writer = ArrowWriter::try_new(&mut buf, data.schema(), None).unwrap();
        writer.write(&data).unwrap();
        writer.close().unwrap();

        let data: Bytes = buf.into();
        let metadata = ParquetMetaDataReader::new()
            .parse_and_finish(&data)
            .unwrap();
        let parquet_schema = metadata.file_metadata().schema_descr_ptr();

        let test = TestReader {
            data,
            metadata: Arc::new(metadata),
            requests: Default::default(),
        };
        let requests = test.requests.clone();

        let a_scalar = StringArray::from_iter_values(["b"]);
        let a_filter = ArrowPredicateFn::new(
            ProjectionMask::leaves(&parquet_schema, vec![0]),
            move |batch| eq(batch.column(0), &Scalar::new(&a_scalar)),
        );

        let b_scalar = StringArray::from_iter_values(["4"]);
        let b_filter = ArrowPredicateFn::new(
            ProjectionMask::leaves(&parquet_schema, vec![2]),
            move |batch| {
                // Filter on the second element of the struct.
                let struct_array = batch
                    .column(0)
                    .as_any()
                    .downcast_ref::<StructArray>()
                    .unwrap();
                eq(struct_array.column(0), &Scalar::new(&b_scalar))
            },
        );

        let filter = RowFilter::new(vec![Box::new(a_filter), Box::new(b_filter)]);

        let mask = ProjectionMask::leaves(&parquet_schema, vec![0, 3]);
        let stream = ParquetRecordBatchStreamBuilder::new(test)
            .await
            .unwrap()
            .with_projection(mask.clone())
            .with_batch_size(1024)
            .with_row_filter(filter)
            .build()
            .unwrap();

        let batches: Vec<_> = stream.try_collect().await.unwrap();
        assert_eq!(batches.len(), 1);

        let batch = &batches[0];
        assert_eq!(batch.num_rows(), 1);
        assert_eq!(batch.num_columns(), 2);

        let col = batch.column(0);
        let val = col.as_any().downcast_ref::<StringArray>().unwrap().value(0);
        assert_eq!(val, "b");

        let col = batch.column(1);
        let val = col.as_any().downcast_ref::<Int32Array>().unwrap().value(0);
        assert_eq!(val, 3);

        // Should only have made 3 requests
        assert_eq!(requests.lock().unwrap().len(), 3);
    }

    #[tokio::test]
    async fn empty_offset_index_doesnt_panic_in_read_row_group() {
        use tokio::fs::File;
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_plain.parquet");
        let mut file = File::open(&path).await.unwrap();
        let file_size = file.metadata().await.unwrap().len();
        let mut metadata = ParquetMetaDataReader::new()
            .with_page_indexes(true)
            .load_and_finish(&mut file, file_size as usize)
            .await
            .unwrap();

        metadata.set_offset_index(Some(vec![]));
        let options = ArrowReaderOptions::new().with_page_index(true);
        let arrow_reader_metadata = ArrowReaderMetadata::try_new(metadata.into(), options).unwrap();
        let reader =
            ParquetRecordBatchStreamBuilder::new_with_metadata(file, arrow_reader_metadata)
                .build()
                .unwrap();

        let result = reader.try_collect::<Vec<_>>().await.unwrap();
        assert_eq!(result.len(), 1);
    }

    #[tokio::test]
    async fn non_empty_offset_index_doesnt_panic_in_read_row_group() {
        use tokio::fs::File;
        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_tiny_pages.parquet");
        let mut file = File::open(&path).await.unwrap();
        let file_size = file.metadata().await.unwrap().len();
        let metadata = ParquetMetaDataReader::new()
            .with_page_indexes(true)
            .load_and_finish(&mut file, file_size as usize)
            .await
            .unwrap();

        let options = ArrowReaderOptions::new().with_page_index(true);
        let arrow_reader_metadata = ArrowReaderMetadata::try_new(metadata.into(), options).unwrap();
        let reader =
            ParquetRecordBatchStreamBuilder::new_with_metadata(file, arrow_reader_metadata)
                .build()
                .unwrap();

        let result = reader.try_collect::<Vec<_>>().await.unwrap();
        assert_eq!(result.len(), 8);
    }

    #[tokio::test]
    async fn empty_offset_index_doesnt_panic_in_column_chunks() {
        use tempfile::TempDir;
        use tokio::fs::File;
        fn write_metadata_to_local_file(
            metadata: ParquetMetaData,
            file: impl AsRef<std::path::Path>,
        ) {
            use crate::file::metadata::ParquetMetaDataWriter;
            use std::fs::File;
            let file = File::create(file).unwrap();
            ParquetMetaDataWriter::new(file, &metadata)
                .finish()
                .unwrap()
        }

        fn read_metadata_from_local_file(file: impl AsRef<std::path::Path>) -> ParquetMetaData {
            use std::fs::File;
            let file = File::open(file).unwrap();
            ParquetMetaDataReader::new()
                .with_page_indexes(true)
                .parse_and_finish(&file)
                .unwrap()
        }

        let testdata = arrow::util::test_util::parquet_test_data();
        let path = format!("{testdata}/alltypes_plain.parquet");
        let mut file = File::open(&path).await.unwrap();
        let file_size = file.metadata().await.unwrap().len();
        let metadata = ParquetMetaDataReader::new()
            .with_page_indexes(true)
            .load_and_finish(&mut file, file_size as usize)
            .await
            .unwrap();

        let tempdir = TempDir::new().unwrap();
        let metadata_path = tempdir.path().join("thrift_metadata.dat");
        write_metadata_to_local_file(metadata, &metadata_path);
        let metadata = read_metadata_from_local_file(&metadata_path);

        let options = ArrowReaderOptions::new().with_page_index(true);
        let arrow_reader_metadata = ArrowReaderMetadata::try_new(metadata.into(), options).unwrap();
        let reader =
            ParquetRecordBatchStreamBuilder::new_with_metadata(file, arrow_reader_metadata)
                .build()
                .unwrap();

        // Panics here
        let result = reader.try_collect::<Vec<_>>().await.unwrap();
        assert_eq!(result.len(), 1);
    }
}