arrow-ipc 58.1.0

Support for the Arrow IPC format
Documentation
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
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
2407
2408
2409
2410
2411
2412
2413
2414
2415
2416
2417
2418
2419
2420
2421
2422
2423
2424
2425
2426
2427
2428
2429
2430
2431
2432
2433
2434
2435
2436
2437
2438
2439
2440
2441
2442
2443
2444
2445
2446
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489
2490
2491
2492
2493
2494
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509
2510
2511
2512
2513
2514
2515
2516
2517
2518
2519
2520
2521
2522
2523
2524
2525
2526
2527
2528
2529
2530
2531
2532
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557
2558
2559
2560
2561
2562
2563
2564
2565
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582
2583
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617
2618
2619
2620
2621
2622
2623
2624
2625
2626
2627
2628
2629
2630
2631
2632
2633
2634
2635
2636
2637
2638
2639
2640
2641
2642
2643
2644
2645
2646
2647
2648
2649
2650
2651
2652
2653
2654
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665
2666
2667
2668
2669
2670
2671
2672
2673
2674
2675
2676
2677
2678
2679
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690
2691
2692
2693
2694
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710
2711
2712
2713
2714
2715
2716
2717
2718
2719
2720
2721
2722
2723
2724
2725
2726
2727
2728
2729
2730
2731
2732
2733
2734
2735
2736
2737
2738
2739
2740
2741
2742
2743
2744
2745
2746
2747
2748
2749
2750
2751
2752
2753
2754
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799
2800
2801
2802
2803
2804
2805
2806
2807
2808
2809
2810
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820
2821
2822
2823
2824
2825
2826
2827
2828
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849
2850
2851
2852
2853
2854
2855
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865
2866
2867
2868
2869
2870
2871
2872
2873
2874
2875
2876
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887
2888
2889
2890
2891
2892
2893
2894
2895
2896
2897
2898
2899
2900
2901
2902
2903
2904
2905
2906
2907
2908
2909
2910
2911
2912
2913
2914
2915
2916
2917
2918
2919
2920
2921
2922
2923
2924
2925
2926
2927
2928
2929
2930
2931
2932
2933
2934
2935
2936
2937
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955
2956
2957
2958
2959
2960
2961
2962
2963
2964
2965
2966
2967
2968
2969
2970
2971
2972
2973
2974
2975
2976
2977
2978
2979
2980
2981
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040
3041
3042
3043
3044
3045
3046
3047
3048
3049
3050
3051
3052
3053
3054
3055
3056
3057
3058
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094
3095
3096
3097
3098
3099
3100
3101
3102
3103
3104
3105
3106
3107
3108
3109
3110
3111
3112
3113
3114
3115
3116
3117
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131
3132
3133
3134
3135
3136
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153
3154
3155
3156
3157
3158
3159
3160
3161
3162
3163
3164
3165
3166
3167
3168
3169
3170
3171
3172
3173
3174
3175
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195
3196
3197
3198
3199
3200
3201
3202
3203
3204
3205
3206
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216
3217
3218
3219
3220
3221
3222
3223
3224
3225
3226
3227
3228
3229
3230
3231
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248
3249
3250
3251
// 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.

//! Arrow IPC File and Stream Readers
//!
//! # Notes
//!
//! The [`FileReader`] and [`StreamReader`] have similar interfaces,
//! however the [`FileReader`] expects a reader that supports [`Seek`]ing
//!
//! [`Seek`]: std::io::Seek

mod stream;
pub use stream::*;

use arrow_select::concat;

use flatbuffers::{VectorIter, VerifierOptions};
use std::collections::{HashMap, VecDeque};
use std::fmt;
use std::io::{BufReader, Read, Seek, SeekFrom};
use std::sync::Arc;

use arrow_array::*;
use arrow_buffer::{
    ArrowNativeType, BooleanBuffer, Buffer, MutableBuffer, NullBuffer, ScalarBuffer,
};
use arrow_data::{ArrayData, ArrayDataBuilder, UnsafeFlag};
use arrow_schema::*;

use crate::compression::{CompressionCodec, DecompressionContext};
use crate::r#gen::Message::{self};
use crate::{Block, CONTINUATION_MARKER, FieldNode, MetadataVersion};
use DataType::*;

/// Read a buffer based on offset and length
/// From <https://github.com/apache/arrow/blob/6a936c4ff5007045e86f65f1a6b6c3c955ad5103/format/Message.fbs#L58>
/// Each constituent buffer is first compressed with the indicated
/// compressor, and then written with the uncompressed length in the first 8
/// bytes as a 64-bit little-endian signed integer followed by the compressed
/// buffer bytes (and then padding as required by the protocol). The
/// uncompressed length may be set to -1 to indicate that the data that
/// follows is not compressed, which can be useful for cases where
/// compression does not yield appreciable savings.
fn read_buffer(
    buf: &crate::Buffer,
    a_data: &Buffer,
    compression_codec: Option<CompressionCodec>,
    decompression_context: &mut DecompressionContext,
) -> Result<Buffer, ArrowError> {
    let start_offset = buf.offset() as usize;
    let buf_data = a_data.slice_with_length(start_offset, buf.length() as usize);
    // corner case: empty buffer
    match (buf_data.is_empty(), compression_codec) {
        (true, _) | (_, None) => Ok(buf_data),
        (false, Some(decompressor)) => {
            decompressor.decompress_to_buffer(&buf_data, decompression_context)
        }
    }
}
impl RecordBatchDecoder<'_> {
    /// Coordinates reading arrays based on data types.
    ///
    /// `variadic_counts` encodes the number of buffers to read for variadic types (e.g., Utf8View, BinaryView)
    /// When encounter such types, we pop from the front of the queue to get the number of buffers to read.
    ///
    /// Notes:
    /// * In the IPC format, null buffers are always set, but may be empty. We discard them if an array has 0 nulls
    /// * Numeric values inside list arrays are often stored as 64-bit values regardless of their data type size.
    ///   We thus:
    ///     - check if the bit width of non-64-bit numbers is 64, and
    ///     - read the buffer as 64-bit (signed integer or float), and
    ///     - cast the 64-bit array to the appropriate data type
    fn create_array(
        &mut self,
        field: &Field,
        variadic_counts: &mut VecDeque<i64>,
    ) -> Result<ArrayRef, ArrowError> {
        let data_type = field.data_type();
        match data_type {
            Utf8 | Binary | LargeBinary | LargeUtf8 => {
                let field_node = self.next_node(field)?;
                let buffers = [
                    self.next_buffer()?,
                    self.next_buffer()?,
                    self.next_buffer()?,
                ];
                self.create_primitive_array(field_node, data_type, &buffers)
            }
            BinaryView | Utf8View => {
                let count = variadic_counts
                    .pop_front()
                    .ok_or(ArrowError::IpcError(format!(
                        "Missing variadic count for {data_type} column"
                    )))?;
                let count = count + 2; // view and null buffer.
                let buffers = (0..count)
                    .map(|_| self.next_buffer())
                    .collect::<Result<Vec<_>, _>>()?;
                let field_node = self.next_node(field)?;
                self.create_primitive_array(field_node, data_type, &buffers)
            }
            FixedSizeBinary(_) => {
                let field_node = self.next_node(field)?;
                let buffers = [self.next_buffer()?, self.next_buffer()?];
                self.create_primitive_array(field_node, data_type, &buffers)
            }
            List(list_field) | LargeList(list_field) | Map(list_field, _) => {
                let list_node = self.next_node(field)?;
                let list_buffers = [self.next_buffer()?, self.next_buffer()?];
                let values = self.create_array(list_field, variadic_counts)?;
                self.create_list_array(list_node, data_type, &list_buffers, values)
            }
            ListView(list_field) | LargeListView(list_field) => {
                let list_node = self.next_node(field)?;
                let list_buffers = [
                    self.next_buffer()?, // null buffer
                    self.next_buffer()?, // offsets
                    self.next_buffer()?, // sizes
                ];
                let values = self.create_array(list_field, variadic_counts)?;
                self.create_list_view_array(list_node, data_type, &list_buffers, values)
            }
            FixedSizeList(list_field, _) => {
                let list_node = self.next_node(field)?;
                let list_buffers = [self.next_buffer()?];
                let values = self.create_array(list_field, variadic_counts)?;
                self.create_list_array(list_node, data_type, &list_buffers, values)
            }
            Struct(struct_fields) => {
                let struct_node = self.next_node(field)?;
                let null_buffer = self.next_buffer()?;

                // read the arrays for each field
                let mut struct_arrays = vec![];
                // TODO investigate whether just knowing the number of buffers could
                // still work
                for struct_field in struct_fields {
                    let child = self.create_array(struct_field, variadic_counts)?;
                    struct_arrays.push(child);
                }
                self.create_struct_array(struct_node, null_buffer, struct_fields, struct_arrays)
            }
            RunEndEncoded(run_ends_field, values_field) => {
                let run_node = self.next_node(field)?;
                let run_ends = self.create_array(run_ends_field, variadic_counts)?;
                let values = self.create_array(values_field, variadic_counts)?;

                let run_array_length = run_node.length() as usize;
                let builder = ArrayData::builder(data_type.clone())
                    .len(run_array_length)
                    .offset(0)
                    .add_child_data(run_ends.into_data())
                    .add_child_data(values.into_data())
                    .null_count(run_node.null_count() as usize);

                self.create_array_from_builder(builder)
            }
            // Create dictionary array from RecordBatch
            Dictionary(_, _) => {
                let index_node = self.next_node(field)?;
                let index_buffers = [self.next_buffer()?, self.next_buffer()?];

                #[allow(deprecated)]
                let dict_id = field.dict_id().ok_or_else(|| {
                    ArrowError::ParseError(format!("Field {field} does not have dict id"))
                })?;

                let value_array = self.dictionaries_by_id.get(&dict_id).ok_or_else(|| {
                    ArrowError::ParseError(format!(
                        "Cannot find a dictionary batch with dict id: {dict_id}"
                    ))
                })?;

                self.create_dictionary_array(
                    index_node,
                    data_type,
                    &index_buffers,
                    value_array.clone(),
                )
            }
            Union(fields, mode) => {
                let union_node = self.next_node(field)?;
                let len = union_node.length() as usize;

                // In V4, union types has validity bitmap
                // In V5 and later, union types have no validity bitmap
                if self.version < MetadataVersion::V5 {
                    self.next_buffer()?;
                }

                let type_ids: ScalarBuffer<i8> =
                    self.next_buffer()?.slice_with_length(0, len).into();

                let value_offsets = match mode {
                    UnionMode::Dense => {
                        let offsets: ScalarBuffer<i32> =
                            self.next_buffer()?.slice_with_length(0, len * 4).into();
                        Some(offsets)
                    }
                    UnionMode::Sparse => None,
                };

                let mut children = Vec::with_capacity(fields.len());

                for (_id, field) in fields.iter() {
                    let child = self.create_array(field, variadic_counts)?;
                    children.push(child);
                }

                let array = if self.skip_validation.get() {
                    // safety: flag can only be set via unsafe code
                    unsafe {
                        UnionArray::new_unchecked(fields.clone(), type_ids, value_offsets, children)
                    }
                } else {
                    UnionArray::try_new(fields.clone(), type_ids, value_offsets, children)?
                };
                Ok(Arc::new(array))
            }
            Null => {
                let node = self.next_node(field)?;
                let length = node.length();
                let null_count = node.null_count();

                if length != null_count {
                    return Err(ArrowError::SchemaError(format!(
                        "Field {field} of NullArray has unequal null_count {null_count} and len {length}"
                    )));
                }

                let builder = ArrayData::builder(data_type.clone())
                    .len(length as usize)
                    .offset(0);
                self.create_array_from_builder(builder)
            }
            _ => {
                let field_node = self.next_node(field)?;
                let buffers = [self.next_buffer()?, self.next_buffer()?];
                self.create_primitive_array(field_node, data_type, &buffers)
            }
        }
    }

    /// Reads the correct number of buffers based on data type and null_count, and creates a
    /// primitive array ref
    fn create_primitive_array(
        &self,
        field_node: &FieldNode,
        data_type: &DataType,
        buffers: &[Buffer],
    ) -> Result<ArrayRef, ArrowError> {
        let length = field_node.length() as usize;
        let null_buffer = (field_node.null_count() > 0).then_some(buffers[0].clone());
        let mut builder = match data_type {
            Utf8 | Binary | LargeBinary | LargeUtf8 => {
                // read 3 buffers: null buffer (optional), offsets buffer and data buffer
                ArrayData::builder(data_type.clone())
                    .len(length)
                    .buffers(buffers[1..3].to_vec())
                    .null_bit_buffer(null_buffer)
            }
            BinaryView | Utf8View => ArrayData::builder(data_type.clone())
                .len(length)
                .buffers(buffers[1..].to_vec())
                .null_bit_buffer(null_buffer),
            _ if data_type.is_primitive() || matches!(data_type, Boolean | FixedSizeBinary(_)) => {
                // read 2 buffers: null buffer (optional) and data buffer
                ArrayData::builder(data_type.clone())
                    .len(length)
                    .add_buffer(buffers[1].clone())
                    .null_bit_buffer(null_buffer)
            }
            t => unreachable!("Data type {:?} either unsupported or not primitive", t),
        };

        builder = builder.null_count(field_node.null_count() as usize);

        self.create_array_from_builder(builder)
    }

    /// Update the ArrayDataBuilder based on settings in this decoder
    fn create_array_from_builder(&self, builder: ArrayDataBuilder) -> Result<ArrayRef, ArrowError> {
        let mut builder = builder.align_buffers(!self.require_alignment);
        if self.skip_validation.get() {
            // SAFETY: flag can only be set via unsafe code
            unsafe { builder = builder.skip_validation(true) }
        };
        Ok(make_array(builder.build()?))
    }

    /// Reads the correct number of buffers based on list type and null_count, and creates a
    /// list array ref
    fn create_list_array(
        &self,
        field_node: &FieldNode,
        data_type: &DataType,
        buffers: &[Buffer],
        child_array: ArrayRef,
    ) -> Result<ArrayRef, ArrowError> {
        let null_buffer = (field_node.null_count() > 0).then_some(buffers[0].clone());
        let length = field_node.length() as usize;
        let child_data = child_array.into_data();
        let mut builder = match data_type {
            List(_) | LargeList(_) | Map(_, _) => ArrayData::builder(data_type.clone())
                .len(length)
                .add_buffer(buffers[1].clone())
                .add_child_data(child_data)
                .null_bit_buffer(null_buffer),

            FixedSizeList(_, _) => ArrayData::builder(data_type.clone())
                .len(length)
                .add_child_data(child_data)
                .null_bit_buffer(null_buffer),

            _ => unreachable!("Cannot create list or map array from {:?}", data_type),
        };

        builder = builder.null_count(field_node.null_count() as usize);

        self.create_array_from_builder(builder)
    }

    fn create_list_view_array(
        &self,
        field_node: &FieldNode,
        data_type: &DataType,
        buffers: &[Buffer],
        child_array: ArrayRef,
    ) -> Result<ArrayRef, ArrowError> {
        assert!(matches!(data_type, ListView(_) | LargeListView(_)));

        let null_buffer = (field_node.null_count() > 0).then_some(buffers[0].clone());
        let length = field_node.length() as usize;
        let child_data = child_array.into_data();

        self.create_array_from_builder(
            ArrayData::builder(data_type.clone())
                .len(length)
                .add_buffer(buffers[1].clone()) // offsets
                .add_buffer(buffers[2].clone()) // sizes
                .add_child_data(child_data)
                .null_bit_buffer(null_buffer)
                .null_count(field_node.null_count() as usize),
        )
    }

    fn create_struct_array(
        &self,
        struct_node: &FieldNode,
        null_buffer: Buffer,
        struct_fields: &Fields,
        struct_arrays: Vec<ArrayRef>,
    ) -> Result<ArrayRef, ArrowError> {
        let null_count = struct_node.null_count() as usize;
        let len = struct_node.length() as usize;
        let skip_validation = self.skip_validation.get();

        let nulls = if null_count > 0 {
            let validity_buffer = BooleanBuffer::new(null_buffer, 0, len);
            let null_buffer = if skip_validation {
                // safety: flag can only be set via unsafe code
                unsafe { NullBuffer::new_unchecked(validity_buffer, null_count) }
            } else {
                let null_buffer = NullBuffer::new(validity_buffer);

                if null_buffer.null_count() != null_count {
                    return Err(ArrowError::InvalidArgumentError(format!(
                        "null_count value ({}) doesn't match actual number of nulls in array ({})",
                        null_count,
                        null_buffer.null_count()
                    )));
                }

                null_buffer
            };

            Some(null_buffer)
        } else {
            None
        };
        if struct_arrays.is_empty() {
            // `StructArray::from` can't infer the correct row count
            // if we have zero fields
            return Ok(Arc::new(StructArray::new_empty_fields(len, nulls)));
        }

        let struct_array = if skip_validation {
            // safety: flag can only be set via unsafe code
            unsafe { StructArray::new_unchecked(struct_fields.clone(), struct_arrays, nulls) }
        } else {
            StructArray::try_new(struct_fields.clone(), struct_arrays, nulls)?
        };

        Ok(Arc::new(struct_array))
    }

    /// Reads the correct number of buffers based on list type and null_count, and creates a
    /// list array ref
    fn create_dictionary_array(
        &self,
        field_node: &FieldNode,
        data_type: &DataType,
        buffers: &[Buffer],
        value_array: ArrayRef,
    ) -> Result<ArrayRef, ArrowError> {
        if let Dictionary(_, _) = *data_type {
            let null_buffer = (field_node.null_count() > 0).then_some(buffers[0].clone());
            let builder = ArrayData::builder(data_type.clone())
                .len(field_node.length() as usize)
                .add_buffer(buffers[1].clone())
                .add_child_data(value_array.into_data())
                .null_bit_buffer(null_buffer)
                .null_count(field_node.null_count() as usize);
            self.create_array_from_builder(builder)
        } else {
            unreachable!("Cannot create dictionary array from {:?}", data_type)
        }
    }
}

/// State for decoding Arrow arrays from an [IPC RecordBatch] structure to
/// [`RecordBatch`]
///
/// [IPC RecordBatch]: crate::RecordBatch
///
pub struct RecordBatchDecoder<'a> {
    /// The flatbuffers encoded record batch
    batch: crate::RecordBatch<'a>,
    /// The output schema
    schema: SchemaRef,
    /// Decoded dictionaries indexed by dictionary id
    dictionaries_by_id: &'a HashMap<i64, ArrayRef>,
    /// Optional compression codec
    compression: Option<CompressionCodec>,
    /// Decompression context for reusing zstd decompressor state
    decompression_context: DecompressionContext,
    /// The format version
    version: MetadataVersion,
    /// The raw data buffer
    data: &'a Buffer,
    /// The fields comprising this array
    nodes: VectorIter<'a, FieldNode>,
    /// The buffers comprising this array
    buffers: VectorIter<'a, crate::Buffer>,
    /// Projection (subset of columns) to read, if any
    /// See [`RecordBatchDecoder::with_projection`] for details
    projection: Option<&'a [usize]>,
    /// Are buffers required to already be aligned? See
    /// [`RecordBatchDecoder::with_require_alignment`] for details
    require_alignment: bool,
    /// Should validation be skipped when reading data? Defaults to false.
    ///
    /// See [`FileDecoder::with_skip_validation`] for details.
    skip_validation: UnsafeFlag,
}

impl<'a> RecordBatchDecoder<'a> {
    /// Create a reader for decoding arrays from an encoded [`RecordBatch`]
    fn try_new(
        buf: &'a Buffer,
        batch: crate::RecordBatch<'a>,
        schema: SchemaRef,
        dictionaries_by_id: &'a HashMap<i64, ArrayRef>,
        metadata: &'a MetadataVersion,
    ) -> Result<Self, ArrowError> {
        let buffers = batch.buffers().ok_or_else(|| {
            ArrowError::IpcError("Unable to get buffers from IPC RecordBatch".to_string())
        })?;
        let field_nodes = batch.nodes().ok_or_else(|| {
            ArrowError::IpcError("Unable to get field nodes from IPC RecordBatch".to_string())
        })?;

        let batch_compression = batch.compression();
        let compression = batch_compression
            .map(|batch_compression| batch_compression.codec().try_into())
            .transpose()?;

        Ok(Self {
            batch,
            schema,
            dictionaries_by_id,
            compression,
            decompression_context: DecompressionContext::new(),
            version: *metadata,
            data: buf,
            nodes: field_nodes.iter(),
            buffers: buffers.iter(),
            projection: None,
            require_alignment: false,
            skip_validation: UnsafeFlag::new(),
        })
    }

    /// Set the projection (default: None)
    ///
    /// If set, the projection is the list  of column indices
    /// that will be read
    pub fn with_projection(mut self, projection: Option<&'a [usize]>) -> Self {
        self.projection = projection;
        self
    }

    /// Set require_alignment (default: false)
    ///
    /// If true, buffers must be aligned appropriately or error will
    /// result. If false, buffers will be copied to aligned buffers
    /// if necessary.
    pub fn with_require_alignment(mut self, require_alignment: bool) -> Self {
        self.require_alignment = require_alignment;
        self
    }

    /// Specifies if validation should be skipped when reading data (defaults to `false`)
    ///
    /// Note this API is somewhat "funky" as it allows the caller to skip validation
    /// without having to use `unsafe` code. If this is ever made public
    /// it should be made clearer that this is a potentially unsafe by
    /// using an `unsafe` function that takes a boolean flag.
    ///
    /// # Safety
    ///
    /// Relies on the caller only passing a flag with `true` value if they are
    /// certain that the data is valid
    pub(crate) fn with_skip_validation(mut self, skip_validation: UnsafeFlag) -> Self {
        self.skip_validation = skip_validation;
        self
    }

    /// Read the record batch, consuming the reader
    fn read_record_batch(mut self) -> Result<RecordBatch, ArrowError> {
        let mut variadic_counts: VecDeque<i64> = self
            .batch
            .variadicBufferCounts()
            .into_iter()
            .flatten()
            .collect();

        let options = RecordBatchOptions::new().with_row_count(Some(self.batch.length() as usize));

        let schema = Arc::clone(&self.schema);
        if let Some(projection) = self.projection {
            let mut arrays = vec![];
            // project fields
            for (idx, field) in schema.fields().iter().enumerate() {
                // Create array for projected field
                if let Some(proj_idx) = projection.iter().position(|p| p == &idx) {
                    let child = self.create_array(field, &mut variadic_counts)?;
                    arrays.push((proj_idx, child));
                } else {
                    self.skip_field(field, &mut variadic_counts)?;
                }
            }

            arrays.sort_by_key(|t| t.0);

            let schema = Arc::new(schema.project(projection)?);
            let columns = arrays.into_iter().map(|t| t.1).collect::<Vec<_>>();

            if self.skip_validation.get() {
                // Safety: setting `skip_validation` requires `unsafe`, user assures data is valid
                unsafe {
                    Ok(RecordBatch::new_unchecked(
                        schema,
                        columns,
                        self.batch.length() as usize,
                    ))
                }
            } else {
                assert!(variadic_counts.is_empty());
                RecordBatch::try_new_with_options(schema, columns, &options)
            }
        } else {
            let mut children = vec![];
            // keep track of index as lists require more than one node
            for field in schema.fields() {
                let child = self.create_array(field, &mut variadic_counts)?;
                children.push(child);
            }

            if self.skip_validation.get() {
                // Safety: setting `skip_validation` requires `unsafe`, user assures data is valid
                unsafe {
                    Ok(RecordBatch::new_unchecked(
                        schema,
                        children,
                        self.batch.length() as usize,
                    ))
                }
            } else {
                assert!(variadic_counts.is_empty());
                RecordBatch::try_new_with_options(schema, children, &options)
            }
        }
    }

    fn next_buffer(&mut self) -> Result<Buffer, ArrowError> {
        let buffer = self.buffers.next().ok_or_else(|| {
            ArrowError::IpcError("Buffer count mismatched with metadata".to_string())
        })?;
        read_buffer(
            buffer,
            self.data,
            self.compression,
            &mut self.decompression_context,
        )
    }

    fn skip_buffer(&mut self) {
        self.buffers.next().unwrap();
    }

    fn next_node(&mut self, field: &Field) -> Result<&'a FieldNode, ArrowError> {
        self.nodes.next().ok_or_else(|| {
            ArrowError::SchemaError(format!(
                "Invalid data for schema. {field} refers to node not found in schema",
            ))
        })
    }

    fn skip_field(
        &mut self,
        field: &Field,
        variadic_count: &mut VecDeque<i64>,
    ) -> Result<(), ArrowError> {
        self.next_node(field)?;

        match field.data_type() {
            Utf8 | Binary | LargeBinary | LargeUtf8 => {
                for _ in 0..3 {
                    self.skip_buffer()
                }
            }
            Utf8View | BinaryView => {
                let count = variadic_count
                    .pop_front()
                    .ok_or(ArrowError::IpcError(format!(
                        "Missing variadic count for {} column",
                        field.data_type()
                    )))?;
                let count = count + 2; // view and null buffer.
                for _i in 0..count {
                    self.skip_buffer()
                }
            }
            FixedSizeBinary(_) => {
                self.skip_buffer();
                self.skip_buffer();
            }
            List(list_field) | LargeList(list_field) | Map(list_field, _) => {
                self.skip_buffer();
                self.skip_buffer();
                self.skip_field(list_field, variadic_count)?;
            }
            FixedSizeList(list_field, _) => {
                self.skip_buffer();
                self.skip_field(list_field, variadic_count)?;
            }
            Struct(struct_fields) => {
                self.skip_buffer();

                // skip for each field
                for struct_field in struct_fields {
                    self.skip_field(struct_field, variadic_count)?
                }
            }
            RunEndEncoded(run_ends_field, values_field) => {
                self.skip_field(run_ends_field, variadic_count)?;
                self.skip_field(values_field, variadic_count)?;
            }
            Dictionary(_, _) => {
                self.skip_buffer(); // Nulls
                self.skip_buffer(); // Indices
            }
            Union(fields, mode) => {
                self.skip_buffer(); // Nulls

                match mode {
                    UnionMode::Dense => self.skip_buffer(),
                    UnionMode::Sparse => {}
                };

                for (_, field) in fields.iter() {
                    self.skip_field(field, variadic_count)?
                }
            }
            Null => {} // No buffer increases
            _ => {
                self.skip_buffer();
                self.skip_buffer();
            }
        };
        Ok(())
    }
}

/// Creates a record batch from binary data using the `crate::RecordBatch` indexes and the `Schema`.
///
/// If `require_alignment` is true, this function will return an error if any array data in the
/// input `buf` is not properly aligned.
/// Under the hood it will use [`arrow_data::ArrayDataBuilder::build`] to construct [`arrow_data::ArrayData`].
///
/// If `require_alignment` is false, this function will automatically allocate a new aligned buffer
/// and copy over the data if any array data in the input `buf` is not properly aligned.
/// (Properly aligned array data will remain zero-copy.)
/// Under the hood it will use [`arrow_data::ArrayDataBuilder::build_aligned`] to construct [`arrow_data::ArrayData`].
pub fn read_record_batch(
    buf: &Buffer,
    batch: crate::RecordBatch,
    schema: SchemaRef,
    dictionaries_by_id: &HashMap<i64, ArrayRef>,
    projection: Option<&[usize]>,
    metadata: &MetadataVersion,
) -> Result<RecordBatch, ArrowError> {
    RecordBatchDecoder::try_new(buf, batch, schema, dictionaries_by_id, metadata)?
        .with_projection(projection)
        .with_require_alignment(false)
        .read_record_batch()
}

/// Read the dictionary from the buffer and provided metadata,
/// updating the `dictionaries_by_id` with the resulting dictionary
pub fn read_dictionary(
    buf: &Buffer,
    batch: crate::DictionaryBatch,
    schema: &Schema,
    dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
    metadata: &MetadataVersion,
) -> Result<(), ArrowError> {
    read_dictionary_impl(
        buf,
        batch,
        schema,
        dictionaries_by_id,
        metadata,
        false,
        UnsafeFlag::new(),
    )
}

fn read_dictionary_impl(
    buf: &Buffer,
    batch: crate::DictionaryBatch,
    schema: &Schema,
    dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
    metadata: &MetadataVersion,
    require_alignment: bool,
    skip_validation: UnsafeFlag,
) -> Result<(), ArrowError> {
    let id = batch.id();

    let dictionary_values = get_dictionary_values(
        buf,
        batch,
        schema,
        dictionaries_by_id,
        metadata,
        require_alignment,
        skip_validation,
    )?;

    update_dictionaries(dictionaries_by_id, batch.isDelta(), id, dictionary_values)?;

    Ok(())
}

/// Updates the `dictionaries_by_id` with the provided dictionary values and id.
///
/// # Errors
/// - If `is_delta` is true and there is no existing dictionary for the given
///   `dict_id`
/// - If `is_delta` is true and the concatenation of the existing and new
///   dictionary fails. This usually signals a type mismatch between the old and
///   new values.
fn update_dictionaries(
    dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
    is_delta: bool,
    dict_id: i64,
    dict_values: ArrayRef,
) -> Result<(), ArrowError> {
    if !is_delta {
        // We don't currently record the isOrdered field. This could be general
        // attributes of arrays.
        // Add (possibly multiple) array refs to the dictionaries array.
        dictionaries_by_id.insert(dict_id, dict_values.clone());
        return Ok(());
    }

    let existing = dictionaries_by_id.get(&dict_id).ok_or_else(|| {
        ArrowError::InvalidArgumentError(format!(
            "No existing dictionary for delta dictionary with id '{dict_id}'"
        ))
    })?;

    let combined = concat::concat(&[existing, &dict_values]).map_err(|e| {
        ArrowError::InvalidArgumentError(format!("Failed to concat delta dictionary: {e}"))
    })?;

    dictionaries_by_id.insert(dict_id, combined);

    Ok(())
}

/// Given a dictionary batch IPC message/body along with the full state of a
/// stream including schema, dictionary cache, metadata, and other flags, this
/// function will parse the buffer into an array of dictionary values.
fn get_dictionary_values(
    buf: &Buffer,
    batch: crate::DictionaryBatch,
    schema: &Schema,
    dictionaries_by_id: &mut HashMap<i64, ArrayRef>,
    metadata: &MetadataVersion,
    require_alignment: bool,
    skip_validation: UnsafeFlag,
) -> Result<ArrayRef, ArrowError> {
    let id = batch.id();
    #[allow(deprecated)]
    let fields_using_this_dictionary = schema.fields_with_dict_id(id);
    let first_field = fields_using_this_dictionary.first().ok_or_else(|| {
        ArrowError::InvalidArgumentError(format!("dictionary id {id} not found in schema"))
    })?;

    // As the dictionary batch does not contain the type of the
    // values array, we need to retrieve this from the schema.
    // Get an array representing this dictionary's values.
    let dictionary_values: ArrayRef = match first_field.data_type() {
        DataType::Dictionary(_, value_type) => {
            // Make a fake schema for the dictionary batch.
            let value = value_type.as_ref().clone();
            let schema = Schema::new(vec![Field::new("", value, true)]);
            // Read a single column
            let record_batch = RecordBatchDecoder::try_new(
                buf,
                batch.data().unwrap(),
                Arc::new(schema),
                dictionaries_by_id,
                metadata,
            )?
            .with_require_alignment(require_alignment)
            .with_skip_validation(skip_validation)
            .read_record_batch()?;

            Some(record_batch.column(0).clone())
        }
        _ => None,
    }
    .ok_or_else(|| {
        ArrowError::InvalidArgumentError(format!("dictionary id {id} not found in schema"))
    })?;

    Ok(dictionary_values)
}

/// Read the data for a given block
fn read_block<R: Read + Seek>(mut reader: R, block: &Block) -> Result<Buffer, ArrowError> {
    reader.seek(SeekFrom::Start(block.offset() as u64))?;
    let body_len = block.bodyLength().to_usize().unwrap();
    let metadata_len = block.metaDataLength().to_usize().unwrap();
    let total_len = body_len.checked_add(metadata_len).unwrap();

    let mut buf = MutableBuffer::from_len_zeroed(total_len);
    reader.read_exact(&mut buf)?;
    Ok(buf.into())
}

/// Parse an encapsulated message
///
/// <https://arrow.apache.org/docs/format/Columnar.html#encapsulated-message-format>
fn parse_message(buf: &[u8]) -> Result<Message::Message<'_>, ArrowError> {
    let buf = match buf[..4] == CONTINUATION_MARKER {
        true => &buf[8..],
        false => &buf[4..],
    };
    crate::root_as_message(buf)
        .map_err(|err| ArrowError::ParseError(format!("Unable to get root as message: {err:?}")))
}

/// Read the footer length from the last 10 bytes of an Arrow IPC file
///
/// Expects a 4 byte footer length followed by `b"ARROW1"`
pub fn read_footer_length(buf: [u8; 10]) -> Result<usize, ArrowError> {
    if buf[4..] != super::ARROW_MAGIC {
        return Err(ArrowError::ParseError(
            "Arrow file does not contain correct footer".to_string(),
        ));
    }

    // read footer length
    let footer_len = i32::from_le_bytes(buf[..4].try_into().unwrap());
    footer_len
        .try_into()
        .map_err(|_| ArrowError::ParseError(format!("Invalid footer length: {footer_len}")))
}

/// A low-level, push-based interface for reading an IPC file
///
/// For a higher-level interface see [`FileReader`]
///
/// For an example of using this API with `mmap` see the [`zero_copy_ipc`] example.
///
/// [`zero_copy_ipc`]: https://github.com/apache/arrow-rs/blob/main/arrow/examples/zero_copy_ipc.rs
///
/// ```
/// # use std::sync::Arc;
/// # use arrow_array::*;
/// # use arrow_array::types::Int32Type;
/// # use arrow_buffer::Buffer;
/// # use arrow_ipc::convert::fb_to_schema;
/// # use arrow_ipc::reader::{FileDecoder, read_footer_length};
/// # use arrow_ipc::root_as_footer;
/// # use arrow_ipc::writer::FileWriter;
/// // Write an IPC file
///
/// let batch = RecordBatch::try_from_iter([
///     ("a", Arc::new(Int32Array::from(vec![1, 2, 3])) as _),
///     ("b", Arc::new(Int32Array::from(vec![1, 2, 3])) as _),
///     ("c", Arc::new(DictionaryArray::<Int32Type>::from_iter(["hello", "hello", "world"])) as _),
/// ]).unwrap();
///
/// let schema = batch.schema();
///
/// let mut out = Vec::with_capacity(1024);
/// let mut writer = FileWriter::try_new(&mut out, schema.as_ref()).unwrap();
/// writer.write(&batch).unwrap();
/// writer.finish().unwrap();
///
/// drop(writer);
///
/// // Read IPC file
///
/// let buffer = Buffer::from_vec(out);
/// let trailer_start = buffer.len() - 10;
/// let footer_len = read_footer_length(buffer[trailer_start..].try_into().unwrap()).unwrap();
/// let footer = root_as_footer(&buffer[trailer_start - footer_len..trailer_start]).unwrap();
///
/// let back = fb_to_schema(footer.schema().unwrap());
/// assert_eq!(&back, schema.as_ref());
///
/// let mut decoder = FileDecoder::new(schema, footer.version());
///
/// // Read dictionaries
/// for block in footer.dictionaries().iter().flatten() {
///     let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
///     let data = buffer.slice_with_length(block.offset() as _, block_len);
///     decoder.read_dictionary(&block, &data).unwrap();
/// }
///
/// // Read record batch
/// let batches = footer.recordBatches().unwrap();
/// assert_eq!(batches.len(), 1); // Only wrote a single batch
///
/// let block = batches.get(0);
/// let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
/// let data = buffer.slice_with_length(block.offset() as _, block_len);
/// let back = decoder.read_record_batch(block, &data).unwrap().unwrap();
///
/// assert_eq!(batch, back);
/// ```
#[derive(Debug)]
pub struct FileDecoder {
    schema: SchemaRef,
    dictionaries: HashMap<i64, ArrayRef>,
    version: MetadataVersion,
    projection: Option<Vec<usize>>,
    require_alignment: bool,
    skip_validation: UnsafeFlag,
}

impl FileDecoder {
    /// Create a new [`FileDecoder`] with the given schema and version
    pub fn new(schema: SchemaRef, version: MetadataVersion) -> Self {
        Self {
            schema,
            version,
            dictionaries: Default::default(),
            projection: None,
            require_alignment: false,
            skip_validation: UnsafeFlag::new(),
        }
    }

    /// Specify a projection
    pub fn with_projection(mut self, projection: Vec<usize>) -> Self {
        self.projection = Some(projection);
        self
    }

    /// Specifies if the array data in input buffers is required to be properly aligned.
    ///
    /// If `require_alignment` is true, this decoder will return an error if any array data in the
    /// input `buf` is not properly aligned.
    /// Under the hood it will use [`arrow_data::ArrayDataBuilder::build`] to construct
    /// [`arrow_data::ArrayData`].
    ///
    /// If `require_alignment` is false (the default), this decoder will automatically allocate a
    /// new aligned buffer and copy over the data if any array data in the input `buf` is not
    /// properly aligned. (Properly aligned array data will remain zero-copy.)
    /// Under the hood it will use [`arrow_data::ArrayDataBuilder::build_aligned`] to construct
    /// [`arrow_data::ArrayData`].
    pub fn with_require_alignment(mut self, require_alignment: bool) -> Self {
        self.require_alignment = require_alignment;
        self
    }

    /// Specifies if validation should be skipped when reading data (defaults to `false`)
    ///
    /// # Safety
    ///
    /// This flag must only be set to `true` when you trust the input data and are sure the data you are
    /// reading is a valid Arrow IPC file, otherwise undefined behavior may
    /// result.
    ///
    /// For example, some programs may wish to trust reading IPC files written
    /// by the same process that created the files.
    pub unsafe fn with_skip_validation(mut self, skip_validation: bool) -> Self {
        unsafe { self.skip_validation.set(skip_validation) };
        self
    }

    fn read_message<'a>(&self, buf: &'a [u8]) -> Result<Message::Message<'a>, ArrowError> {
        let message = parse_message(buf)?;

        // some old test data's footer metadata is not set, so we account for that
        if self.version != MetadataVersion::V1 && message.version() != self.version {
            return Err(ArrowError::IpcError(
                "Could not read IPC message as metadata versions mismatch".to_string(),
            ));
        }
        Ok(message)
    }

    /// Read the dictionary with the given block and data buffer
    pub fn read_dictionary(&mut self, block: &Block, buf: &Buffer) -> Result<(), ArrowError> {
        let message = self.read_message(buf)?;
        match message.header_type() {
            crate::MessageHeader::DictionaryBatch => {
                let batch = message.header_as_dictionary_batch().unwrap();
                read_dictionary_impl(
                    &buf.slice(block.metaDataLength() as _),
                    batch,
                    &self.schema,
                    &mut self.dictionaries,
                    &message.version(),
                    self.require_alignment,
                    self.skip_validation.clone(),
                )
            }
            t => Err(ArrowError::ParseError(format!(
                "Expecting DictionaryBatch in dictionary blocks, found {t:?}."
            ))),
        }
    }

    /// Read the RecordBatch with the given block and data buffer
    pub fn read_record_batch(
        &self,
        block: &Block,
        buf: &Buffer,
    ) -> Result<Option<RecordBatch>, ArrowError> {
        let message = self.read_message(buf)?;
        match message.header_type() {
            crate::MessageHeader::Schema => Err(ArrowError::IpcError(
                "Not expecting a schema when messages are read".to_string(),
            )),
            crate::MessageHeader::RecordBatch => {
                let batch = message.header_as_record_batch().ok_or_else(|| {
                    ArrowError::IpcError("Unable to read IPC message as record batch".to_string())
                })?;
                // read the block that makes up the record batch into a buffer
                RecordBatchDecoder::try_new(
                    &buf.slice(block.metaDataLength() as _),
                    batch,
                    self.schema.clone(),
                    &self.dictionaries,
                    &message.version(),
                )?
                .with_projection(self.projection.as_deref())
                .with_require_alignment(self.require_alignment)
                .with_skip_validation(self.skip_validation.clone())
                .read_record_batch()
                .map(Some)
            }
            crate::MessageHeader::NONE => Ok(None),
            t => Err(ArrowError::InvalidArgumentError(format!(
                "Reading types other than record batches not yet supported, unable to read {t:?}"
            ))),
        }
    }
}

/// Build an Arrow [`FileReader`] with custom options.
#[derive(Debug)]
pub struct FileReaderBuilder {
    /// Optional projection for which columns to load (zero-based column indices)
    projection: Option<Vec<usize>>,
    /// Passed through to construct [`VerifierOptions`]
    max_footer_fb_tables: usize,
    /// Passed through to construct [`VerifierOptions`]
    max_footer_fb_depth: usize,
}

impl Default for FileReaderBuilder {
    fn default() -> Self {
        let verifier_options = VerifierOptions::default();
        Self {
            max_footer_fb_tables: verifier_options.max_tables,
            max_footer_fb_depth: verifier_options.max_depth,
            projection: None,
        }
    }
}

impl FileReaderBuilder {
    /// Options for creating a new [`FileReader`].
    ///
    /// To convert a builder into a reader, call [`FileReaderBuilder::build`].
    pub fn new() -> Self {
        Self::default()
    }

    /// Optional projection for which columns to load (zero-based column indices).
    pub fn with_projection(mut self, projection: Vec<usize>) -> Self {
        self.projection = Some(projection);
        self
    }

    /// Flatbuffers option for parsing the footer. Controls the max number of fields and
    /// metadata key-value pairs that can be parsed from the schema of the footer.
    ///
    /// By default this is set to `1_000_000` which roughly translates to a schema with
    /// no metadata key-value pairs but 499,999 fields.
    ///
    /// This default limit is enforced to protect against malicious files with a massive
    /// amount of flatbuffer tables which could cause a denial of service attack.
    ///
    /// If you need to ingest a trusted file with a massive number of fields and/or
    /// metadata key-value pairs and are facing the error `"Unable to get root as
    /// footer: TooManyTables"` then increase this parameter as necessary.
    pub fn with_max_footer_fb_tables(mut self, max_footer_fb_tables: usize) -> Self {
        self.max_footer_fb_tables = max_footer_fb_tables;
        self
    }

    /// Flatbuffers option for parsing the footer. Controls the max depth for schemas with
    /// nested fields parsed from the footer.
    ///
    /// By default this is set to `64` which roughly translates to a schema with
    /// a field nested 60 levels down through other struct fields.
    ///
    /// This default limit is enforced to protect against malicious files with a extremely
    /// deep flatbuffer structure which could cause a denial of service attack.
    ///
    /// If you need to ingest a trusted file with a deeply nested field and are facing the
    /// error `"Unable to get root as footer: DepthLimitReached"` then increase this
    /// parameter as necessary.
    pub fn with_max_footer_fb_depth(mut self, max_footer_fb_depth: usize) -> Self {
        self.max_footer_fb_depth = max_footer_fb_depth;
        self
    }

    /// Build [`FileReader`] with given reader.
    pub fn build<R: Read + Seek>(self, mut reader: R) -> Result<FileReader<R>, ArrowError> {
        // Space for ARROW_MAGIC (6 bytes) and length (4 bytes)
        let mut buffer = [0; 10];
        reader.seek(SeekFrom::End(-10))?;
        reader.read_exact(&mut buffer)?;

        let footer_len = read_footer_length(buffer)?;

        // read footer
        let mut footer_data = vec![0; footer_len];
        reader.seek(SeekFrom::End(-10 - footer_len as i64))?;
        reader.read_exact(&mut footer_data)?;

        let verifier_options = VerifierOptions {
            max_tables: self.max_footer_fb_tables,
            max_depth: self.max_footer_fb_depth,
            ..Default::default()
        };
        let footer = crate::root_as_footer_with_opts(&verifier_options, &footer_data[..]).map_err(
            |err| ArrowError::ParseError(format!("Unable to get root as footer: {err:?}")),
        )?;

        let blocks = footer.recordBatches().ok_or_else(|| {
            ArrowError::ParseError("Unable to get record batches from IPC Footer".to_string())
        })?;

        let total_blocks = blocks.len();

        let ipc_schema = footer.schema().unwrap();
        if !ipc_schema.endianness().equals_to_target_endianness() {
            return Err(ArrowError::IpcError(
                "the endianness of the source system does not match the endianness of the target system.".to_owned()
            ));
        }

        let schema = crate::convert::fb_to_schema(ipc_schema);

        let mut custom_metadata = HashMap::new();
        if let Some(fb_custom_metadata) = footer.custom_metadata() {
            for kv in fb_custom_metadata.into_iter() {
                custom_metadata.insert(
                    kv.key().unwrap().to_string(),
                    kv.value().unwrap().to_string(),
                );
            }
        }

        let mut decoder = FileDecoder::new(Arc::new(schema), footer.version());
        if let Some(projection) = self.projection {
            decoder = decoder.with_projection(projection)
        }

        // Create an array of optional dictionary value arrays, one per field.
        if let Some(dictionaries) = footer.dictionaries() {
            for block in dictionaries {
                let buf = read_block(&mut reader, block)?;
                decoder.read_dictionary(block, &buf)?;
            }
        }

        Ok(FileReader {
            reader,
            blocks: blocks.iter().copied().collect(),
            current_block: 0,
            total_blocks,
            decoder,
            custom_metadata,
        })
    }
}

/// Arrow File Reader
///
/// Reads Arrow [`RecordBatch`]es from bytes in the [IPC File Format],
/// providing random access to the record batches.
///
/// # See Also
///
/// * [`Self::set_index`] for random access
/// * [`StreamReader`] for reading streaming data
///
/// # Example: Reading from a `File`
/// ```
/// # use std::io::Cursor;
/// use arrow_array::record_batch;
/// # use arrow_ipc::reader::FileReader;
/// # use arrow_ipc::writer::FileWriter;
/// # let batch = record_batch!(("a", Int32, [1, 2, 3])).unwrap();
/// # let mut file = vec![]; // mimic a stream for the example
/// # {
/// #  let mut writer = FileWriter::try_new(&mut file, &batch.schema()).unwrap();
/// #  writer.write(&batch).unwrap();
/// #  writer.write(&batch).unwrap();
/// #  writer.finish().unwrap();
/// # }
/// # let mut file = Cursor::new(&file);
/// let projection = None; // read all columns
/// let mut reader = FileReader::try_new(&mut file, projection).unwrap();
/// // Position the reader to the second batch
/// reader.set_index(1).unwrap();
/// // read batches from the reader using the Iterator trait
/// let mut num_rows = 0;
/// for batch in reader {
///    let batch = batch.unwrap();
///    num_rows += batch.num_rows();
/// }
/// assert_eq!(num_rows, 3);
/// ```
/// # Example: Reading from `mmap`ed file
///
/// For an example creating Arrays without copying using  memory mapped (`mmap`)
/// files see the [`zero_copy_ipc`] example.
///
/// [IPC File Format]: https://arrow.apache.org/docs/format/Columnar.html#ipc-file-format
/// [`zero_copy_ipc`]: https://github.com/apache/arrow-rs/blob/main/arrow/examples/zero_copy_ipc.rs
pub struct FileReader<R> {
    /// File reader that supports reading and seeking
    reader: R,

    /// The decoder
    decoder: FileDecoder,

    /// The blocks in the file
    ///
    /// A block indicates the regions in the file to read to get data
    blocks: Vec<Block>,

    /// A counter to keep track of the current block that should be read
    current_block: usize,

    /// The total number of blocks, which may contain record batches and other types
    total_blocks: usize,

    /// User defined metadata
    custom_metadata: HashMap<String, String>,
}

impl<R> fmt::Debug for FileReader<R> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> Result<(), fmt::Error> {
        f.debug_struct("FileReader<R>")
            .field("decoder", &self.decoder)
            .field("blocks", &self.blocks)
            .field("current_block", &self.current_block)
            .field("total_blocks", &self.total_blocks)
            .finish_non_exhaustive()
    }
}

impl<R: Read + Seek> FileReader<BufReader<R>> {
    /// Try to create a new file reader with the reader wrapped in a BufReader.
    ///
    /// See [`FileReader::try_new`] for an unbuffered version.
    pub fn try_new_buffered(reader: R, projection: Option<Vec<usize>>) -> Result<Self, ArrowError> {
        Self::try_new(BufReader::new(reader), projection)
    }
}

impl<R: Read + Seek> FileReader<R> {
    /// Try to create a new file reader.
    ///
    /// There is no internal buffering. If buffered reads are needed you likely want to use
    /// [`FileReader::try_new_buffered`] instead.
    ///
    /// # Errors
    ///
    /// An ['Err'](Result::Err) may be returned if:
    /// - the file does not meet the Arrow Format footer requirements, or
    /// - file endianness does not match the target endianness.
    pub fn try_new(reader: R, projection: Option<Vec<usize>>) -> Result<Self, ArrowError> {
        let builder = FileReaderBuilder {
            projection,
            ..Default::default()
        };
        builder.build(reader)
    }

    /// Return user defined customized metadata
    pub fn custom_metadata(&self) -> &HashMap<String, String> {
        &self.custom_metadata
    }

    /// Return the number of batches in the file
    pub fn num_batches(&self) -> usize {
        self.total_blocks
    }

    /// Return the schema of the file
    pub fn schema(&self) -> SchemaRef {
        self.decoder.schema.clone()
    }

    /// See to a specific [`RecordBatch`]
    ///
    /// Sets the current block to the index, allowing random reads
    pub fn set_index(&mut self, index: usize) -> Result<(), ArrowError> {
        if index >= self.total_blocks {
            Err(ArrowError::InvalidArgumentError(format!(
                "Cannot set batch to index {} from {} total batches",
                index, self.total_blocks
            )))
        } else {
            self.current_block = index;
            Ok(())
        }
    }

    fn maybe_next(&mut self) -> Result<Option<RecordBatch>, ArrowError> {
        let block = &self.blocks[self.current_block];
        self.current_block += 1;

        // read length
        let buffer = read_block(&mut self.reader, block)?;
        self.decoder.read_record_batch(block, &buffer)
    }

    /// Gets a reference to the underlying reader.
    ///
    /// It is inadvisable to directly read from the underlying reader.
    pub fn get_ref(&self) -> &R {
        &self.reader
    }

    /// Gets a mutable reference to the underlying reader.
    ///
    /// It is inadvisable to directly read from the underlying reader.
    pub fn get_mut(&mut self) -> &mut R {
        &mut self.reader
    }

    /// Specifies if validation should be skipped when reading data (defaults to `false`)
    ///
    /// # Safety
    ///
    /// See [`FileDecoder::with_skip_validation`]
    pub unsafe fn with_skip_validation(mut self, skip_validation: bool) -> Self {
        self.decoder = unsafe { self.decoder.with_skip_validation(skip_validation) };
        self
    }
}

impl<R: Read + Seek> Iterator for FileReader<R> {
    type Item = Result<RecordBatch, ArrowError>;

    fn next(&mut self) -> Option<Self::Item> {
        // get current block
        if self.current_block < self.total_blocks {
            self.maybe_next().transpose()
        } else {
            None
        }
    }
}

impl<R: Read + Seek> RecordBatchReader for FileReader<R> {
    fn schema(&self) -> SchemaRef {
        self.schema()
    }
}

/// Arrow Stream Reader
///
/// Reads Arrow [`RecordBatch`]es from bytes in the [IPC Streaming Format].
///
/// # See Also
///
/// * [`FileReader`] for random access.
///
/// # Example
/// ```
/// # use arrow_array::record_batch;
/// # use arrow_ipc::reader::StreamReader;
/// # use arrow_ipc::writer::StreamWriter;
/// # let batch = record_batch!(("a", Int32, [1, 2, 3])).unwrap();
/// # let mut stream = vec![]; // mimic a stream for the example
/// # {
/// #  let mut writer = StreamWriter::try_new(&mut stream, &batch.schema()).unwrap();
/// #  writer.write(&batch).unwrap();
/// #  writer.finish().unwrap();
/// # }
/// # let stream = stream.as_slice();
/// let projection = None; // read all columns
/// let mut reader = StreamReader::try_new(stream, projection).unwrap();
/// // read batches from the reader using the Iterator trait
/// let mut num_rows = 0;
/// for batch in reader {
///    let batch = batch.unwrap();
///    num_rows += batch.num_rows();
/// }
/// assert_eq!(num_rows, 3);
/// ```
///
/// [IPC Streaming Format]: https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format
pub struct StreamReader<R> {
    /// Stream reader
    reader: MessageReader<R>,

    /// The schema that is read from the stream's first message
    schema: SchemaRef,

    /// Optional dictionaries for each schema field.
    ///
    /// Dictionaries may be appended to in the streaming format.
    dictionaries_by_id: HashMap<i64, ArrayRef>,

    /// An indicator of whether the stream is complete.
    ///
    /// This value is set to `true` the first time the reader's `next()` returns `None`.
    finished: bool,

    /// Optional projection
    projection: Option<(Vec<usize>, Schema)>,

    /// Should validation be skipped when reading data? Defaults to false.
    ///
    /// See [`FileDecoder::with_skip_validation`] for details.
    skip_validation: UnsafeFlag,
}

impl<R> fmt::Debug for StreamReader<R> {
    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> std::result::Result<(), fmt::Error> {
        f.debug_struct("StreamReader<R>")
            .field("reader", &"R")
            .field("schema", &self.schema)
            .field("dictionaries_by_id", &self.dictionaries_by_id)
            .field("finished", &self.finished)
            .field("projection", &self.projection)
            .finish()
    }
}

impl<R: Read> StreamReader<BufReader<R>> {
    /// Try to create a new stream reader with the reader wrapped in a BufReader.
    ///
    /// See [`StreamReader::try_new`] for an unbuffered version.
    pub fn try_new_buffered(reader: R, projection: Option<Vec<usize>>) -> Result<Self, ArrowError> {
        Self::try_new(BufReader::new(reader), projection)
    }
}

impl<R: Read> StreamReader<R> {
    /// Try to create a new stream reader.
    ///
    /// To check if the reader is done, use [`is_finished(self)`](StreamReader::is_finished).
    ///
    /// There is no internal buffering. If buffered reads are needed you likely want to use
    /// [`StreamReader::try_new_buffered`] instead.
    ///
    /// # Errors
    ///
    /// An ['Err'](Result::Err) may be returned if the reader does not encounter a schema
    /// as the first message in the stream.
    pub fn try_new(
        reader: R,
        projection: Option<Vec<usize>>,
    ) -> Result<StreamReader<R>, ArrowError> {
        let mut msg_reader = MessageReader::new(reader);
        let message = msg_reader.maybe_next()?;
        let Some((message, _)) = message else {
            return Err(ArrowError::IpcError(
                "Expected schema message, found empty stream.".to_string(),
            ));
        };

        if message.header_type() != Message::MessageHeader::Schema {
            return Err(ArrowError::IpcError(format!(
                "Expected a schema as the first message in the stream, got: {:?}",
                message.header_type()
            )));
        }

        let schema = message.header_as_schema().ok_or_else(|| {
            ArrowError::ParseError("Failed to parse schema from message header".to_string())
        })?;
        let schema = crate::convert::fb_to_schema(schema);

        // Create an array of optional dictionary value arrays, one per field.
        let dictionaries_by_id = HashMap::new();

        let projection = match projection {
            Some(projection_indices) => {
                let schema = schema.project(&projection_indices)?;
                Some((projection_indices, schema))
            }
            _ => None,
        };

        Ok(Self {
            reader: msg_reader,
            schema: Arc::new(schema),
            finished: false,
            dictionaries_by_id,
            projection,
            skip_validation: UnsafeFlag::new(),
        })
    }

    /// Deprecated, use [`StreamReader::try_new`] instead.
    #[deprecated(since = "53.0.0", note = "use `try_new` instead")]
    pub fn try_new_unbuffered(
        reader: R,
        projection: Option<Vec<usize>>,
    ) -> Result<Self, ArrowError> {
        Self::try_new(reader, projection)
    }

    /// Return the schema of the stream
    pub fn schema(&self) -> SchemaRef {
        self.schema.clone()
    }

    /// Check if the stream is finished
    pub fn is_finished(&self) -> bool {
        self.finished
    }

    fn maybe_next(&mut self) -> Result<Option<RecordBatch>, ArrowError> {
        if self.finished {
            return Ok(None);
        }

        // Read messages until we get a record batch or end of stream
        loop {
            let message = self.next_ipc_message()?;
            let Some(message) = message else {
                // If the message is None, we have reached the end of the stream.
                self.finished = true;
                return Ok(None);
            };

            match message {
                IpcMessage::Schema(_) => {
                    return Err(ArrowError::IpcError(
                        "Expected a record batch, but found a schema".to_string(),
                    ));
                }
                IpcMessage::RecordBatch(record_batch) => {
                    return Ok(Some(record_batch));
                }
                IpcMessage::DictionaryBatch { .. } => {
                    continue;
                }
            };
        }
    }

    /// Reads and fully parses the next IPC message from the stream. Whereas
    /// [`Self::maybe_next`] is a higher level method focused on reading
    /// `RecordBatch`es, this method returns the individual fully parsed IPC
    /// messages from the underlying stream.
    ///
    /// This is useful primarily for testing reader/writer behaviors as it
    /// allows a full view into the messages that have been written to a stream.
    pub(crate) fn next_ipc_message(&mut self) -> Result<Option<IpcMessage>, ArrowError> {
        let message = self.reader.maybe_next()?;
        let Some((message, body)) = message else {
            // If the message is None, we have reached the end of the stream.
            return Ok(None);
        };

        let ipc_message = match message.header_type() {
            Message::MessageHeader::Schema => {
                let schema = message.header_as_schema().ok_or_else(|| {
                    ArrowError::ParseError("Failed to parse schema from message header".to_string())
                })?;
                let arrow_schema = crate::convert::fb_to_schema(schema);
                IpcMessage::Schema(arrow_schema)
            }
            Message::MessageHeader::RecordBatch => {
                let batch = message.header_as_record_batch().ok_or_else(|| {
                    ArrowError::IpcError("Unable to read IPC message as record batch".to_string())
                })?;

                let version = message.version();
                let schema = self.schema.clone();
                let record_batch = RecordBatchDecoder::try_new(
                    &body.into(),
                    batch,
                    schema,
                    &self.dictionaries_by_id,
                    &version,
                )?
                .with_projection(self.projection.as_ref().map(|x| x.0.as_ref()))
                .with_require_alignment(false)
                .with_skip_validation(self.skip_validation.clone())
                .read_record_batch()?;
                IpcMessage::RecordBatch(record_batch)
            }
            Message::MessageHeader::DictionaryBatch => {
                let dict = message.header_as_dictionary_batch().ok_or_else(|| {
                    ArrowError::ParseError(
                        "Failed to parse dictionary batch from message header".to_string(),
                    )
                })?;

                let version = message.version();
                let dict_values = get_dictionary_values(
                    &body.into(),
                    dict,
                    &self.schema,
                    &mut self.dictionaries_by_id,
                    &version,
                    false,
                    self.skip_validation.clone(),
                )?;

                update_dictionaries(
                    &mut self.dictionaries_by_id,
                    dict.isDelta(),
                    dict.id(),
                    dict_values.clone(),
                )?;

                IpcMessage::DictionaryBatch {
                    id: dict.id(),
                    is_delta: (dict.isDelta()),
                    values: (dict_values),
                }
            }
            x => {
                return Err(ArrowError::ParseError(format!(
                    "Unsupported message header type in IPC stream: '{x:?}'"
                )));
            }
        };

        Ok(Some(ipc_message))
    }

    /// Gets a reference to the underlying reader.
    ///
    /// It is inadvisable to directly read from the underlying reader.
    pub fn get_ref(&self) -> &R {
        self.reader.inner()
    }

    /// Gets a mutable reference to the underlying reader.
    ///
    /// It is inadvisable to directly read from the underlying reader.
    pub fn get_mut(&mut self) -> &mut R {
        self.reader.inner_mut()
    }

    /// Specifies if validation should be skipped when reading data (defaults to `false`)
    ///
    /// # Safety
    ///
    /// See [`FileDecoder::with_skip_validation`]
    pub unsafe fn with_skip_validation(mut self, skip_validation: bool) -> Self {
        unsafe { self.skip_validation.set(skip_validation) };
        self
    }
}

impl<R: Read> Iterator for StreamReader<R> {
    type Item = Result<RecordBatch, ArrowError>;

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

impl<R: Read> RecordBatchReader for StreamReader<R> {
    fn schema(&self) -> SchemaRef {
        self.schema.clone()
    }
}

/// Representation of a fully parsed IpcMessage from the underlying stream.
/// Parsing this kind of message is done by higher level constructs such as
/// [`StreamReader`], because fully interpreting the messages into a record
/// batch or dictionary batch requires access to stream state such as schema
/// and the full dictionary cache.
#[derive(Debug)]
#[allow(dead_code)]
pub(crate) enum IpcMessage {
    Schema(arrow_schema::Schema),
    RecordBatch(RecordBatch),
    DictionaryBatch {
        id: i64,
        is_delta: bool,
        values: ArrayRef,
    },
}

/// A low-level construct that reads [`Message::Message`]s from a reader while
/// re-using a buffer for metadata. This is composed into [`StreamReader`].
struct MessageReader<R> {
    reader: R,
    buf: Vec<u8>,
}

impl<R: Read> MessageReader<R> {
    fn new(reader: R) -> Self {
        Self {
            reader,
            buf: Vec::new(),
        }
    }

    /// Reads the entire next message from the underlying reader which includes
    /// the metadata length, the metadata, and the body.
    ///
    /// # Returns
    /// - `Ok(None)` if the the reader signals the end of stream with EOF on
    ///   the first read
    /// - `Err(_)` if the reader returns an error other than on the first
    ///   read, or if the metadata length is invalid
    /// - `Ok(Some(_))` with the Message and buffer containiner the
    ///   body bytes otherwise.
    fn maybe_next(&mut self) -> Result<Option<(Message::Message<'_>, MutableBuffer)>, ArrowError> {
        let meta_len = self.read_meta_len()?;
        let Some(meta_len) = meta_len else {
            return Ok(None);
        };

        self.buf.resize(meta_len, 0);
        self.reader.read_exact(&mut self.buf)?;

        let message = crate::root_as_message(self.buf.as_slice()).map_err(|err| {
            ArrowError::ParseError(format!("Unable to get root as message: {err:?}"))
        })?;

        let mut buf = MutableBuffer::from_len_zeroed(message.bodyLength() as usize);
        self.reader.read_exact(&mut buf)?;

        Ok(Some((message, buf)))
    }

    /// Get a mutable reference to the underlying reader.
    fn inner_mut(&mut self) -> &mut R {
        &mut self.reader
    }

    /// Get an immutable reference to the underlying reader.
    fn inner(&self) -> &R {
        &self.reader
    }

    /// Read the metadata length for the next message from the underlying stream.
    ///
    /// # Returns
    /// - `Ok(None)` if the the reader signals the end of stream with EOF on
    ///   the first read
    /// - `Err(_)` if the reader returns an error other than on the first
    ///   read, or if the metadata length is less than 0.
    /// - `Ok(Some(_))` with the length otherwise.
    pub fn read_meta_len(&mut self) -> Result<Option<usize>, ArrowError> {
        let mut meta_len: [u8; 4] = [0; 4];
        match self.reader.read_exact(&mut meta_len) {
            Ok(_) => {}
            Err(e) => {
                return if e.kind() == std::io::ErrorKind::UnexpectedEof {
                    // Handle EOF without the "0xFFFFFFFF 0x00000000"
                    // valid according to:
                    // https://arrow.apache.org/docs/format/Columnar.html#ipc-streaming-format
                    Ok(None)
                } else {
                    Err(ArrowError::from(e))
                };
            }
        };

        let meta_len = {
            // If a continuation marker is encountered, skip over it and read
            // the size from the next four bytes.
            if meta_len == CONTINUATION_MARKER {
                self.reader.read_exact(&mut meta_len)?;
            }

            i32::from_le_bytes(meta_len)
        };

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

        let meta_len = usize::try_from(meta_len)
            .map_err(|_| ArrowError::ParseError(format!("Invalid metadata length: {meta_len}")))?;

        Ok(Some(meta_len))
    }
}

#[cfg(test)]
mod tests {
    use std::io::Cursor;

    use crate::convert::fb_to_schema;
    use crate::writer::{
        DictionaryTracker, IpcDataGenerator, IpcWriteOptions, unslice_run_array, write_message,
    };

    use super::*;

    use crate::{root_as_footer, root_as_message, size_prefixed_root_as_message};
    use arrow_array::builder::{PrimitiveRunBuilder, UnionBuilder};
    use arrow_array::types::*;
    use arrow_buffer::{NullBuffer, OffsetBuffer};
    use arrow_data::ArrayDataBuilder;

    fn create_test_projection_schema() -> Schema {
        // define field types
        let list_data_type = DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true)));

        let fixed_size_list_data_type =
            DataType::FixedSizeList(Arc::new(Field::new_list_field(DataType::Int32, false)), 3);

        let union_fields = UnionFields::from_fields(vec![
            Field::new("a", DataType::Int32, false),
            Field::new("b", DataType::Float64, false),
        ]);

        let union_data_type = DataType::Union(union_fields, UnionMode::Dense);

        let struct_fields = Fields::from(vec![
            Field::new("id", DataType::Int32, false),
            Field::new_list("list", Field::new_list_field(DataType::Int8, true), false),
        ]);
        let struct_data_type = DataType::Struct(struct_fields);

        let run_encoded_data_type = DataType::RunEndEncoded(
            Arc::new(Field::new("run_ends", DataType::Int16, false)),
            Arc::new(Field::new("values", DataType::Int32, true)),
        );

        // define schema
        Schema::new(vec![
            Field::new("f0", DataType::UInt32, false),
            Field::new("f1", DataType::Utf8, false),
            Field::new("f2", DataType::Boolean, false),
            Field::new("f3", union_data_type, true),
            Field::new("f4", DataType::Null, true),
            Field::new("f5", DataType::Float64, true),
            Field::new("f6", list_data_type, false),
            Field::new("f7", DataType::FixedSizeBinary(3), true),
            Field::new("f8", fixed_size_list_data_type, false),
            Field::new("f9", struct_data_type, false),
            Field::new("f10", run_encoded_data_type, false),
            Field::new("f11", DataType::Boolean, false),
            Field::new_dictionary("f12", DataType::Int8, DataType::Utf8, false),
            Field::new("f13", DataType::Utf8, false),
        ])
    }

    fn create_test_projection_batch_data(schema: &Schema) -> RecordBatch {
        // set test data for each column
        let array0 = UInt32Array::from(vec![1, 2, 3]);
        let array1 = StringArray::from(vec!["foo", "bar", "baz"]);
        let array2 = BooleanArray::from(vec![true, false, true]);

        let mut union_builder = UnionBuilder::new_dense();
        union_builder.append::<Int32Type>("a", 1).unwrap();
        union_builder.append::<Float64Type>("b", 10.1).unwrap();
        union_builder.append_null::<Float64Type>("b").unwrap();
        let array3 = union_builder.build().unwrap();

        let array4 = NullArray::new(3);
        let array5 = Float64Array::from(vec![Some(1.1), None, Some(3.3)]);
        let array6_values = vec![
            Some(vec![Some(10), Some(10), Some(10)]),
            Some(vec![Some(20), Some(20), Some(20)]),
            Some(vec![Some(30), Some(30)]),
        ];
        let array6 = ListArray::from_iter_primitive::<Int32Type, _, _>(array6_values);
        let array7_values = vec![vec![11, 12, 13], vec![22, 23, 24], vec![33, 34, 35]];
        let array7 = FixedSizeBinaryArray::try_from_iter(array7_values.into_iter()).unwrap();

        let array8_values = ArrayData::builder(DataType::Int32)
            .len(9)
            .add_buffer(Buffer::from_slice_ref([40, 41, 42, 43, 44, 45, 46, 47, 48]))
            .build()
            .unwrap();
        let array8_data = ArrayData::builder(schema.field(8).data_type().clone())
            .len(3)
            .add_child_data(array8_values)
            .build()
            .unwrap();
        let array8 = FixedSizeListArray::from(array8_data);

        let array9_id: ArrayRef = Arc::new(Int32Array::from(vec![1001, 1002, 1003]));
        let array9_list: ArrayRef =
            Arc::new(ListArray::from_iter_primitive::<Int8Type, _, _>(vec![
                Some(vec![Some(-10)]),
                Some(vec![Some(-20), Some(-20), Some(-20)]),
                Some(vec![Some(-30)]),
            ]));
        let array9 = ArrayDataBuilder::new(schema.field(9).data_type().clone())
            .add_child_data(array9_id.into_data())
            .add_child_data(array9_list.into_data())
            .len(3)
            .build()
            .unwrap();
        let array9 = StructArray::from(array9);

        let array10_input = vec![Some(1_i32), None, None];
        let mut array10_builder = PrimitiveRunBuilder::<Int16Type, Int32Type>::new();
        array10_builder.extend(array10_input);
        let array10 = array10_builder.finish();

        let array11 = BooleanArray::from(vec![false, false, true]);

        let array12_values = StringArray::from(vec!["x", "yy", "zzz"]);
        let array12_keys = Int8Array::from_iter_values([1, 1, 2]);
        let array12 = DictionaryArray::new(array12_keys, Arc::new(array12_values));

        let array13 = StringArray::from(vec!["a", "bb", "ccc"]);

        // create record batch
        RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![
                Arc::new(array0),
                Arc::new(array1),
                Arc::new(array2),
                Arc::new(array3),
                Arc::new(array4),
                Arc::new(array5),
                Arc::new(array6),
                Arc::new(array7),
                Arc::new(array8),
                Arc::new(array9),
                Arc::new(array10),
                Arc::new(array11),
                Arc::new(array12),
                Arc::new(array13),
            ],
        )
        .unwrap()
    }

    #[test]
    fn test_negative_meta_len_start_stream() {
        let bytes = i32::to_le_bytes(-1);
        let mut buf = vec![];
        buf.extend(CONTINUATION_MARKER);
        buf.extend(bytes);

        let reader_err = StreamReader::try_new(Cursor::new(buf), None).err();
        assert!(reader_err.is_some());
        assert_eq!(
            reader_err.unwrap().to_string(),
            "Parser error: Invalid metadata length: -1"
        );
    }

    #[test]
    fn test_negative_meta_len_mid_stream() {
        let schema = Schema::new(vec![Field::new("a", DataType::Int32, false)]);
        let mut buf = Vec::new();
        {
            let mut writer = crate::writer::StreamWriter::try_new(&mut buf, &schema).unwrap();
            let batch =
                RecordBatch::try_new(Arc::new(schema), vec![Arc::new(Int32Array::from(vec![1]))])
                    .unwrap();
            writer.write(&batch).unwrap();
        }

        let bytes = i32::to_le_bytes(-1);
        buf.extend(CONTINUATION_MARKER);
        buf.extend(bytes);

        let mut reader = StreamReader::try_new(Cursor::new(buf), None).unwrap();
        // Read the valid value
        assert!(reader.maybe_next().is_ok());
        // Read the invalid meta len
        let batch_err = reader.maybe_next().err();
        assert!(batch_err.is_some());
        assert_eq!(
            batch_err.unwrap().to_string(),
            "Parser error: Invalid metadata length: -1"
        );
    }

    #[test]
    fn test_missing_buffer_metadata_error() {
        use crate::r#gen::Message::*;
        use flatbuffers::FlatBufferBuilder;

        let schema = Arc::new(Schema::new(vec![Field::new("col", DataType::Int32, true)]));

        // create RecordBatch buffer metadata with invalid buffer count
        // Int32Array needs 2 buffers (validity + data) but we provide only 1
        let mut fbb = FlatBufferBuilder::new();
        let nodes = fbb.create_vector(&[FieldNode::new(2, 0)]);
        let buffers = fbb.create_vector(&[crate::Buffer::new(0, 8)]);
        let batch_offset = RecordBatch::create(
            &mut fbb,
            &RecordBatchArgs {
                length: 2,
                nodes: Some(nodes),
                buffers: Some(buffers),
                compression: None,
                variadicBufferCounts: None,
            },
        );
        fbb.finish_minimal(batch_offset);
        let batch_bytes = fbb.finished_data().to_vec();
        let batch = flatbuffers::root::<RecordBatch>(&batch_bytes).unwrap();

        let data_buffer = Buffer::from(vec![0u8; 8]);
        let dictionaries: HashMap<i64, ArrayRef> = HashMap::new();
        let metadata = MetadataVersion::V5;

        let decoder = RecordBatchDecoder::try_new(
            &data_buffer,
            batch,
            schema.clone(),
            &dictionaries,
            &metadata,
        )
        .unwrap();

        let result = decoder.read_record_batch();

        match result {
            Err(ArrowError::IpcError(msg)) => {
                assert_eq!(msg, "Buffer count mismatched with metadata");
            }
            other => panic!("unexpected error: {other:?}"),
        }
    }

    #[test]
    fn test_projection_array_values() {
        // define schema
        let schema = create_test_projection_schema();

        // create record batch with test data
        let batch = create_test_projection_batch_data(&schema);

        // write record batch in IPC format
        let mut buf = Vec::new();
        {
            let mut writer = crate::writer::FileWriter::try_new(&mut buf, &schema).unwrap();
            writer.write(&batch).unwrap();
            writer.finish().unwrap();
        }

        // read record batch with projection
        for index in 0..12 {
            let projection = vec![index];
            let reader = FileReader::try_new(std::io::Cursor::new(buf.clone()), Some(projection));
            let read_batch = reader.unwrap().next().unwrap().unwrap();
            let projected_column = read_batch.column(0);
            let expected_column = batch.column(index);

            // check the projected column equals the expected column
            assert_eq!(projected_column.as_ref(), expected_column.as_ref());
        }

        {
            // read record batch with reversed projection
            let reader =
                FileReader::try_new(std::io::Cursor::new(buf.clone()), Some(vec![3, 2, 1]));
            let read_batch = reader.unwrap().next().unwrap().unwrap();
            let expected_batch = batch.project(&[3, 2, 1]).unwrap();
            assert_eq!(read_batch, expected_batch);
        }
    }

    #[test]
    fn test_arrow_single_float_row() {
        let schema = Schema::new(vec![
            Field::new("a", DataType::Float32, false),
            Field::new("b", DataType::Float32, false),
            Field::new("c", DataType::Int32, false),
            Field::new("d", DataType::Int32, false),
        ]);
        let arrays = vec![
            Arc::new(Float32Array::from(vec![1.23])) as ArrayRef,
            Arc::new(Float32Array::from(vec![-6.50])) as ArrayRef,
            Arc::new(Int32Array::from(vec![2])) as ArrayRef,
            Arc::new(Int32Array::from(vec![1])) as ArrayRef,
        ];
        let batch = RecordBatch::try_new(Arc::new(schema.clone()), arrays).unwrap();
        // create stream writer
        let mut file = tempfile::tempfile().unwrap();
        let mut stream_writer = crate::writer::StreamWriter::try_new(&mut file, &schema).unwrap();
        stream_writer.write(&batch).unwrap();
        stream_writer.finish().unwrap();

        drop(stream_writer);

        file.rewind().unwrap();

        // read stream back
        let reader = StreamReader::try_new(&mut file, None).unwrap();

        reader.for_each(|batch| {
            let batch = batch.unwrap();
            assert!(
                batch
                    .column(0)
                    .as_any()
                    .downcast_ref::<Float32Array>()
                    .unwrap()
                    .value(0)
                    != 0.0
            );
            assert!(
                batch
                    .column(1)
                    .as_any()
                    .downcast_ref::<Float32Array>()
                    .unwrap()
                    .value(0)
                    != 0.0
            );
        });

        file.rewind().unwrap();

        // Read with projection
        let reader = StreamReader::try_new(file, Some(vec![0, 3])).unwrap();

        reader.for_each(|batch| {
            let batch = batch.unwrap();
            assert_eq!(batch.schema().fields().len(), 2);
            assert_eq!(batch.schema().fields()[0].data_type(), &DataType::Float32);
            assert_eq!(batch.schema().fields()[1].data_type(), &DataType::Int32);
        });
    }

    /// Write the record batch to an in-memory buffer in IPC File format
    fn write_ipc(rb: &RecordBatch) -> Vec<u8> {
        let mut buf = Vec::new();
        let mut writer = crate::writer::FileWriter::try_new(&mut buf, rb.schema_ref()).unwrap();
        writer.write(rb).unwrap();
        writer.finish().unwrap();
        buf
    }

    /// Return the first record batch read from the IPC File buffer
    fn read_ipc(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
        let mut reader = FileReader::try_new(std::io::Cursor::new(buf), None)?;
        reader.next().unwrap()
    }

    /// Return the first record batch read from the IPC File buffer, disabling
    /// validation
    fn read_ipc_skip_validation(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
        let mut reader = unsafe {
            FileReader::try_new(std::io::Cursor::new(buf), None)?.with_skip_validation(true)
        };
        reader.next().unwrap()
    }

    fn roundtrip_ipc(rb: &RecordBatch) -> RecordBatch {
        let buf = write_ipc(rb);
        read_ipc(&buf).unwrap()
    }

    /// Return the first record batch read from the IPC File buffer
    /// using the FileDecoder API
    fn read_ipc_with_decoder(buf: Vec<u8>) -> Result<RecordBatch, ArrowError> {
        read_ipc_with_decoder_inner(buf, false)
    }

    /// Return the first record batch read from the IPC File buffer
    /// using the FileDecoder API, disabling validation
    fn read_ipc_with_decoder_skip_validation(buf: Vec<u8>) -> Result<RecordBatch, ArrowError> {
        read_ipc_with_decoder_inner(buf, true)
    }

    fn read_ipc_with_decoder_inner(
        buf: Vec<u8>,
        skip_validation: bool,
    ) -> Result<RecordBatch, ArrowError> {
        let buffer = Buffer::from_vec(buf);
        let trailer_start = buffer.len() - 10;
        let footer_len = read_footer_length(buffer[trailer_start..].try_into().unwrap())?;
        let footer = root_as_footer(&buffer[trailer_start - footer_len..trailer_start])
            .map_err(|e| ArrowError::InvalidArgumentError(format!("Invalid footer: {e}")))?;

        let schema = fb_to_schema(footer.schema().unwrap());

        let mut decoder = unsafe {
            FileDecoder::new(Arc::new(schema), footer.version())
                .with_skip_validation(skip_validation)
        };
        // Read dictionaries
        for block in footer.dictionaries().iter().flatten() {
            let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
            let data = buffer.slice_with_length(block.offset() as _, block_len);
            decoder.read_dictionary(block, &data)?
        }

        // Read record batch
        let batches = footer.recordBatches().unwrap();
        assert_eq!(batches.len(), 1); // Only wrote a single batch

        let block = batches.get(0);
        let block_len = block.bodyLength() as usize + block.metaDataLength() as usize;
        let data = buffer.slice_with_length(block.offset() as _, block_len);
        Ok(decoder.read_record_batch(block, &data)?.unwrap())
    }

    /// Write the record batch to an in-memory buffer in IPC Stream format
    fn write_stream(rb: &RecordBatch) -> Vec<u8> {
        let mut buf = Vec::new();
        let mut writer = crate::writer::StreamWriter::try_new(&mut buf, rb.schema_ref()).unwrap();
        writer.write(rb).unwrap();
        writer.finish().unwrap();
        buf
    }

    /// Return the first record batch read from the IPC Stream buffer
    fn read_stream(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
        let mut reader = StreamReader::try_new(std::io::Cursor::new(buf), None)?;
        reader.next().unwrap()
    }

    /// Return the first record batch read from the IPC Stream buffer,
    /// disabling validation
    fn read_stream_skip_validation(buf: &[u8]) -> Result<RecordBatch, ArrowError> {
        let mut reader = unsafe {
            StreamReader::try_new(std::io::Cursor::new(buf), None)?.with_skip_validation(true)
        };
        reader.next().unwrap()
    }

    fn roundtrip_ipc_stream(rb: &RecordBatch) -> RecordBatch {
        let buf = write_stream(rb);
        read_stream(&buf).unwrap()
    }

    #[test]
    fn test_roundtrip_with_custom_metadata() {
        let schema = Schema::new(vec![Field::new("dummy", DataType::Float64, false)]);
        let mut buf = Vec::new();
        let mut writer = crate::writer::FileWriter::try_new(&mut buf, &schema).unwrap();
        let mut test_metadata = HashMap::new();
        test_metadata.insert("abc".to_string(), "abc".to_string());
        test_metadata.insert("def".to_string(), "def".to_string());
        for (k, v) in &test_metadata {
            writer.write_metadata(k, v);
        }
        writer.finish().unwrap();
        drop(writer);

        let reader = crate::reader::FileReader::try_new(std::io::Cursor::new(buf), None).unwrap();
        assert_eq!(reader.custom_metadata(), &test_metadata);
    }

    #[test]
    fn test_roundtrip_nested_dict() {
        let inner: DictionaryArray<Int32Type> = vec!["a", "b", "a"].into_iter().collect();

        let array = Arc::new(inner) as ArrayRef;

        let dctfield = Arc::new(Field::new("dict", array.data_type().clone(), false));

        let s = StructArray::from(vec![(dctfield, array)]);
        let struct_array = Arc::new(s) as ArrayRef;

        let schema = Arc::new(Schema::new(vec![Field::new(
            "struct",
            struct_array.data_type().clone(),
            false,
        )]));

        let batch = RecordBatch::try_new(schema, vec![struct_array]).unwrap();

        assert_eq!(batch, roundtrip_ipc(&batch));
    }

    #[test]
    fn test_roundtrip_nested_dict_no_preserve_dict_id() {
        let inner: DictionaryArray<Int32Type> = vec!["a", "b", "a"].into_iter().collect();

        let array = Arc::new(inner) as ArrayRef;

        let dctfield = Arc::new(Field::new("dict", array.data_type().clone(), false));

        let s = StructArray::from(vec![(dctfield, array)]);
        let struct_array = Arc::new(s) as ArrayRef;

        let schema = Arc::new(Schema::new(vec![Field::new(
            "struct",
            struct_array.data_type().clone(),
            false,
        )]));

        let batch = RecordBatch::try_new(schema, vec![struct_array]).unwrap();

        let mut buf = Vec::new();
        let mut writer = crate::writer::FileWriter::try_new_with_options(
            &mut buf,
            batch.schema_ref(),
            IpcWriteOptions::default(),
        )
        .unwrap();
        writer.write(&batch).unwrap();
        writer.finish().unwrap();
        drop(writer);

        let mut reader = FileReader::try_new(std::io::Cursor::new(buf), None).unwrap();

        assert_eq!(batch, reader.next().unwrap().unwrap());
    }

    fn check_union_with_builder(mut builder: UnionBuilder) {
        builder.append::<Int32Type>("a", 1).unwrap();
        builder.append_null::<Int32Type>("a").unwrap();
        builder.append::<Float64Type>("c", 3.0).unwrap();
        builder.append::<Int32Type>("a", 4).unwrap();
        builder.append::<Int64Type>("d", 11).unwrap();
        let union = builder.build().unwrap();

        let schema = Arc::new(Schema::new(vec![Field::new(
            "union",
            union.data_type().clone(),
            false,
        )]));

        let union_array = Arc::new(union) as ArrayRef;

        let rb = RecordBatch::try_new(schema, vec![union_array]).unwrap();
        let rb2 = roundtrip_ipc(&rb);
        // TODO: equality not yet implemented for union, so we check that the length of the array is
        // the same and that all of the buffers are the same instead.
        assert_eq!(rb.schema(), rb2.schema());
        assert_eq!(rb.num_columns(), rb2.num_columns());
        assert_eq!(rb.num_rows(), rb2.num_rows());
        let union1 = rb.column(0);
        let union2 = rb2.column(0);

        assert_eq!(union1, union2);
    }

    #[test]
    fn test_roundtrip_dense_union() {
        check_union_with_builder(UnionBuilder::new_dense());
    }

    #[test]
    fn test_roundtrip_sparse_union() {
        check_union_with_builder(UnionBuilder::new_sparse());
    }

    #[test]
    fn test_roundtrip_struct_empty_fields() {
        let nulls = NullBuffer::from(&[true, true, false]);
        let rb = RecordBatch::try_from_iter([(
            "",
            Arc::new(StructArray::new_empty_fields(nulls.len(), Some(nulls))) as _,
        )])
        .unwrap();
        let rb2 = roundtrip_ipc(&rb);
        assert_eq!(rb, rb2);
    }

    #[test]
    fn test_roundtrip_stream_run_array_sliced() {
        let run_array_1: Int32RunArray = vec!["a", "a", "a", "b", "b", "c", "c", "c"]
            .into_iter()
            .collect();
        let run_array_1_sliced = run_array_1.slice(2, 5);

        let run_array_2_inupt = vec![Some(1_i32), None, None, Some(2), Some(2)];
        let mut run_array_2_builder = PrimitiveRunBuilder::<Int16Type, Int32Type>::new();
        run_array_2_builder.extend(run_array_2_inupt);
        let run_array_2 = run_array_2_builder.finish();

        let schema = Arc::new(Schema::new(vec![
            Field::new(
                "run_array_1_sliced",
                run_array_1_sliced.data_type().clone(),
                false,
            ),
            Field::new("run_array_2", run_array_2.data_type().clone(), false),
        ]));
        let input_batch = RecordBatch::try_new(
            schema,
            vec![Arc::new(run_array_1_sliced.clone()), Arc::new(run_array_2)],
        )
        .unwrap();
        let output_batch = roundtrip_ipc_stream(&input_batch);

        // As partial comparison not yet supported for run arrays, the sliced run array
        // has to be unsliced before comparing with the output. the second run array
        // can be compared as such.
        assert_eq!(input_batch.column(1), output_batch.column(1));

        let run_array_1_unsliced = unslice_run_array(run_array_1_sliced.into_data()).unwrap();
        assert_eq!(run_array_1_unsliced, output_batch.column(0).into_data());
    }

    #[test]
    fn test_roundtrip_stream_nested_dict() {
        let xs = vec!["AA", "BB", "AA", "CC", "BB"];
        let dict = Arc::new(
            xs.clone()
                .into_iter()
                .collect::<DictionaryArray<Int8Type>>(),
        );
        let string_array: ArrayRef = Arc::new(StringArray::from(xs.clone()));
        let struct_array = StructArray::from(vec![
            (
                Arc::new(Field::new("f2.1", DataType::Utf8, false)),
                string_array,
            ),
            (
                Arc::new(Field::new("f2.2_struct", dict.data_type().clone(), false)),
                dict.clone() as ArrayRef,
            ),
        ]);
        let schema = Arc::new(Schema::new(vec![
            Field::new("f1_string", DataType::Utf8, false),
            Field::new("f2_struct", struct_array.data_type().clone(), false),
        ]));
        let input_batch = RecordBatch::try_new(
            schema,
            vec![
                Arc::new(StringArray::from(xs.clone())),
                Arc::new(struct_array),
            ],
        )
        .unwrap();
        let output_batch = roundtrip_ipc_stream(&input_batch);
        assert_eq!(input_batch, output_batch);
    }

    #[test]
    fn test_roundtrip_stream_nested_dict_of_map_of_dict() {
        let values = StringArray::from(vec![Some("a"), None, Some("b"), Some("c")]);
        let values = Arc::new(values) as ArrayRef;
        let value_dict_keys = Int8Array::from_iter_values([0, 1, 1, 2, 3, 1]);
        let value_dict_array = DictionaryArray::new(value_dict_keys, values.clone());

        let key_dict_keys = Int8Array::from_iter_values([0, 0, 2, 1, 1, 3]);
        let key_dict_array = DictionaryArray::new(key_dict_keys, values);

        #[allow(deprecated)]
        let keys_field = Arc::new(Field::new_dict(
            "keys",
            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
            true, // It is technically not legal for this field to be null.
            1,
            false,
        ));
        #[allow(deprecated)]
        let values_field = Arc::new(Field::new_dict(
            "values",
            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
            true,
            2,
            false,
        ));
        let entry_struct = StructArray::from(vec![
            (keys_field, make_array(key_dict_array.into_data())),
            (values_field, make_array(value_dict_array.into_data())),
        ]);
        let map_data_type = DataType::Map(
            Arc::new(Field::new(
                "entries",
                entry_struct.data_type().clone(),
                false,
            )),
            false,
        );

        let entry_offsets = Buffer::from_slice_ref([0, 2, 4, 6]);
        let map_data = ArrayData::builder(map_data_type)
            .len(3)
            .add_buffer(entry_offsets)
            .add_child_data(entry_struct.into_data())
            .build()
            .unwrap();
        let map_array = MapArray::from(map_data);

        let dict_keys = Int8Array::from_iter_values([0, 1, 1, 2, 2, 1]);
        let dict_dict_array = DictionaryArray::new(dict_keys, Arc::new(map_array));

        let schema = Arc::new(Schema::new(vec![Field::new(
            "f1",
            dict_dict_array.data_type().clone(),
            false,
        )]));
        let input_batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
        let output_batch = roundtrip_ipc_stream(&input_batch);
        assert_eq!(input_batch, output_batch);
    }

    fn test_roundtrip_stream_dict_of_list_of_dict_impl<
        OffsetSize: OffsetSizeTrait,
        U: ArrowNativeType,
    >(
        list_data_type: DataType,
        offsets: &[U; 5],
    ) {
        let values = StringArray::from(vec![Some("a"), None, Some("c"), None]);
        let keys = Int8Array::from_iter_values([0, 0, 1, 2, 0, 1, 3]);
        let dict_array = DictionaryArray::new(keys, Arc::new(values));
        let dict_data = dict_array.to_data();

        let value_offsets = Buffer::from_slice_ref(offsets);

        let list_data = ArrayData::builder(list_data_type)
            .len(4)
            .add_buffer(value_offsets)
            .add_child_data(dict_data)
            .build()
            .unwrap();
        let list_array = GenericListArray::<OffsetSize>::from(list_data);

        let keys_for_dict = Int8Array::from_iter_values([0, 3, 0, 1, 1, 2, 0, 1, 3]);
        let dict_dict_array = DictionaryArray::new(keys_for_dict, Arc::new(list_array));

        let schema = Arc::new(Schema::new(vec![Field::new(
            "f1",
            dict_dict_array.data_type().clone(),
            false,
        )]));
        let input_batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
        let output_batch = roundtrip_ipc_stream(&input_batch);
        assert_eq!(input_batch, output_batch);
    }

    #[test]
    fn test_roundtrip_stream_dict_of_list_of_dict() {
        // list
        #[allow(deprecated)]
        let list_data_type = DataType::List(Arc::new(Field::new_dict(
            "item",
            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
            true,
            1,
            false,
        )));
        let offsets: &[i32; 5] = &[0, 2, 4, 4, 6];
        test_roundtrip_stream_dict_of_list_of_dict_impl::<i32, i32>(list_data_type, offsets);

        // large list
        #[allow(deprecated)]
        let list_data_type = DataType::LargeList(Arc::new(Field::new_dict(
            "item",
            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
            true,
            1,
            false,
        )));
        let offsets: &[i64; 5] = &[0, 2, 4, 4, 7];
        test_roundtrip_stream_dict_of_list_of_dict_impl::<i64, i64>(list_data_type, offsets);
    }

    #[test]
    fn test_roundtrip_stream_dict_of_fixed_size_list_of_dict() {
        let values = StringArray::from(vec![Some("a"), None, Some("c"), None]);
        let keys = Int8Array::from_iter_values([0, 0, 1, 2, 0, 1, 3, 1, 2]);
        let dict_array = DictionaryArray::new(keys, Arc::new(values));
        let dict_data = dict_array.into_data();

        #[allow(deprecated)]
        let list_data_type = DataType::FixedSizeList(
            Arc::new(Field::new_dict(
                "item",
                DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8)),
                true,
                1,
                false,
            )),
            3,
        );
        let list_data = ArrayData::builder(list_data_type)
            .len(3)
            .add_child_data(dict_data)
            .build()
            .unwrap();
        let list_array = FixedSizeListArray::from(list_data);

        let keys_for_dict = Int8Array::from_iter_values([0, 1, 0, 1, 1, 2, 0, 1, 2]);
        let dict_dict_array = DictionaryArray::new(keys_for_dict, Arc::new(list_array));

        let schema = Arc::new(Schema::new(vec![Field::new(
            "f1",
            dict_dict_array.data_type().clone(),
            false,
        )]));
        let input_batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
        let output_batch = roundtrip_ipc_stream(&input_batch);
        assert_eq!(input_batch, output_batch);
    }

    const LONG_TEST_STRING: &str =
        "This is a long string to make sure binary view array handles it";

    #[test]
    fn test_roundtrip_view_types() {
        let schema = Schema::new(vec![
            Field::new("field_1", DataType::BinaryView, true),
            Field::new("field_2", DataType::Utf8, true),
            Field::new("field_3", DataType::Utf8View, true),
        ]);
        let bin_values: Vec<Option<&[u8]>> = vec![
            Some(b"foo"),
            None,
            Some(b"bar"),
            Some(LONG_TEST_STRING.as_bytes()),
        ];
        let utf8_values: Vec<Option<&str>> =
            vec![Some("foo"), None, Some("bar"), Some(LONG_TEST_STRING)];
        let bin_view_array = BinaryViewArray::from_iter(bin_values);
        let utf8_array = StringArray::from_iter(utf8_values.iter());
        let utf8_view_array = StringViewArray::from_iter(utf8_values);
        let record_batch = RecordBatch::try_new(
            Arc::new(schema.clone()),
            vec![
                Arc::new(bin_view_array),
                Arc::new(utf8_array),
                Arc::new(utf8_view_array),
            ],
        )
        .unwrap();

        assert_eq!(record_batch, roundtrip_ipc(&record_batch));
        assert_eq!(record_batch, roundtrip_ipc_stream(&record_batch));

        let sliced_batch = record_batch.slice(1, 2);
        assert_eq!(sliced_batch, roundtrip_ipc(&sliced_batch));
        assert_eq!(sliced_batch, roundtrip_ipc_stream(&sliced_batch));
    }

    #[test]
    fn test_roundtrip_view_types_nested_dict() {
        let bin_values: Vec<Option<&[u8]>> = vec![
            Some(b"foo"),
            None,
            Some(b"bar"),
            Some(LONG_TEST_STRING.as_bytes()),
            Some(b"field"),
        ];
        let utf8_values: Vec<Option<&str>> = vec![
            Some("foo"),
            None,
            Some("bar"),
            Some(LONG_TEST_STRING),
            Some("field"),
        ];
        let bin_view_array = Arc::new(BinaryViewArray::from_iter(bin_values));
        let utf8_view_array = Arc::new(StringViewArray::from_iter(utf8_values));

        let key_dict_keys = Int8Array::from_iter_values([0, 0, 1, 2, 0, 1, 3]);
        let key_dict_array = DictionaryArray::new(key_dict_keys, utf8_view_array.clone());
        #[allow(deprecated)]
        let keys_field = Arc::new(Field::new_dict(
            "keys",
            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8View)),
            true,
            1,
            false,
        ));

        let value_dict_keys = Int8Array::from_iter_values([0, 3, 0, 1, 2, 0, 1]);
        let value_dict_array = DictionaryArray::new(value_dict_keys, bin_view_array);
        #[allow(deprecated)]
        let values_field = Arc::new(Field::new_dict(
            "values",
            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::BinaryView)),
            true,
            2,
            false,
        ));
        let entry_struct = StructArray::from(vec![
            (keys_field, make_array(key_dict_array.into_data())),
            (values_field, make_array(value_dict_array.into_data())),
        ]);

        let map_data_type = DataType::Map(
            Arc::new(Field::new(
                "entries",
                entry_struct.data_type().clone(),
                false,
            )),
            false,
        );
        let entry_offsets = Buffer::from_slice_ref([0, 2, 4, 7]);
        let map_data = ArrayData::builder(map_data_type)
            .len(3)
            .add_buffer(entry_offsets)
            .add_child_data(entry_struct.into_data())
            .build()
            .unwrap();
        let map_array = MapArray::from(map_data);

        let dict_keys = Int8Array::from_iter_values([0, 1, 0, 1, 1, 2, 0, 1, 2]);
        let dict_dict_array = DictionaryArray::new(dict_keys, Arc::new(map_array));
        let schema = Arc::new(Schema::new(vec![Field::new(
            "f1",
            dict_dict_array.data_type().clone(),
            false,
        )]));
        let batch = RecordBatch::try_new(schema, vec![Arc::new(dict_dict_array)]).unwrap();
        assert_eq!(batch, roundtrip_ipc(&batch));
        assert_eq!(batch, roundtrip_ipc_stream(&batch));

        let sliced_batch = batch.slice(1, 2);
        assert_eq!(sliced_batch, roundtrip_ipc(&sliced_batch));
        assert_eq!(sliced_batch, roundtrip_ipc_stream(&sliced_batch));
    }

    #[test]
    fn test_no_columns_batch() {
        let schema = Arc::new(Schema::empty());
        let options = RecordBatchOptions::new()
            .with_match_field_names(true)
            .with_row_count(Some(10));
        let input_batch = RecordBatch::try_new_with_options(schema, vec![], &options).unwrap();
        let output_batch = roundtrip_ipc_stream(&input_batch);
        assert_eq!(input_batch, output_batch);
    }

    #[test]
    fn test_unaligned() {
        let batch = RecordBatch::try_from_iter(vec![(
            "i32",
            Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _,
        )])
        .unwrap();

        let r#gen = IpcDataGenerator {};
        let mut dict_tracker = DictionaryTracker::new(false);
        let (_, encoded) = r#gen
            .encode(
                &batch,
                &mut dict_tracker,
                &Default::default(),
                &mut Default::default(),
            )
            .unwrap();

        let message = root_as_message(&encoded.ipc_message).unwrap();

        // Construct an unaligned buffer
        let mut buffer = MutableBuffer::with_capacity(encoded.arrow_data.len() + 1);
        buffer.push(0_u8);
        buffer.extend_from_slice(&encoded.arrow_data);
        let b = Buffer::from(buffer).slice(1);
        assert_ne!(b.as_ptr().align_offset(8), 0);

        let ipc_batch = message.header_as_record_batch().unwrap();
        let roundtrip = RecordBatchDecoder::try_new(
            &b,
            ipc_batch,
            batch.schema(),
            &Default::default(),
            &message.version(),
        )
        .unwrap()
        .with_require_alignment(false)
        .read_record_batch()
        .unwrap();
        assert_eq!(batch, roundtrip);
    }

    #[test]
    fn test_unaligned_throws_error_with_require_alignment() {
        let batch = RecordBatch::try_from_iter(vec![(
            "i32",
            Arc::new(Int32Array::from(vec![1, 2, 3, 4])) as _,
        )])
        .unwrap();

        let r#gen = IpcDataGenerator {};
        let mut dict_tracker = DictionaryTracker::new(false);
        let (_, encoded) = r#gen
            .encode(
                &batch,
                &mut dict_tracker,
                &Default::default(),
                &mut Default::default(),
            )
            .unwrap();

        let message = root_as_message(&encoded.ipc_message).unwrap();

        // Construct an unaligned buffer
        let mut buffer = MutableBuffer::with_capacity(encoded.arrow_data.len() + 1);
        buffer.push(0_u8);
        buffer.extend_from_slice(&encoded.arrow_data);
        let b = Buffer::from(buffer).slice(1);
        assert_ne!(b.as_ptr().align_offset(8), 0);

        let ipc_batch = message.header_as_record_batch().unwrap();
        let result = RecordBatchDecoder::try_new(
            &b,
            ipc_batch,
            batch.schema(),
            &Default::default(),
            &message.version(),
        )
        .unwrap()
        .with_require_alignment(true)
        .read_record_batch();

        let error = result.unwrap_err();
        assert_eq!(
            error.to_string(),
            "Invalid argument error: Misaligned buffers[0] in array of type Int32, \
             offset from expected alignment of 4 by 1"
        );
    }

    #[test]
    fn test_file_with_massive_column_count() {
        // 499_999 is upper limit for default settings (1_000_000)
        let limit = 600_000;

        let fields = (0..limit)
            .map(|i| Field::new(format!("{i}"), DataType::Boolean, false))
            .collect::<Vec<_>>();
        let schema = Arc::new(Schema::new(fields));
        let batch = RecordBatch::new_empty(schema);

        let mut buf = Vec::new();
        let mut writer = crate::writer::FileWriter::try_new(&mut buf, batch.schema_ref()).unwrap();
        writer.write(&batch).unwrap();
        writer.finish().unwrap();
        drop(writer);

        let mut reader = FileReaderBuilder::new()
            .with_max_footer_fb_tables(1_500_000)
            .build(std::io::Cursor::new(buf))
            .unwrap();
        let roundtrip_batch = reader.next().unwrap().unwrap();

        assert_eq!(batch, roundtrip_batch);
    }

    #[test]
    fn test_file_with_deeply_nested_columns() {
        // 60 is upper limit for default settings (64)
        let limit = 61;

        let fields = (0..limit).fold(
            vec![Field::new("leaf", DataType::Boolean, false)],
            |field, index| vec![Field::new_struct(format!("{index}"), field, false)],
        );
        let schema = Arc::new(Schema::new(fields));
        let batch = RecordBatch::new_empty(schema);

        let mut buf = Vec::new();
        let mut writer = crate::writer::FileWriter::try_new(&mut buf, batch.schema_ref()).unwrap();
        writer.write(&batch).unwrap();
        writer.finish().unwrap();
        drop(writer);

        let mut reader = FileReaderBuilder::new()
            .with_max_footer_fb_depth(65)
            .build(std::io::Cursor::new(buf))
            .unwrap();
        let roundtrip_batch = reader.next().unwrap().unwrap();

        assert_eq!(batch, roundtrip_batch);
    }

    #[test]
    fn test_invalid_struct_array_ipc_read_errors() {
        let a_field = Field::new("a", DataType::Int32, false);
        let b_field = Field::new("b", DataType::Int32, false);
        let struct_fields = Fields::from(vec![a_field.clone(), b_field.clone()]);

        let a_array_data = ArrayData::builder(a_field.data_type().clone())
            .len(4)
            .add_buffer(Buffer::from_slice_ref([1, 2, 3, 4]))
            .build()
            .unwrap();
        let b_array_data = ArrayData::builder(b_field.data_type().clone())
            .len(3)
            .add_buffer(Buffer::from_slice_ref([5, 6, 7]))
            .build()
            .unwrap();

        let invalid_struct_arr = unsafe {
            StructArray::new_unchecked(
                struct_fields,
                vec![make_array(a_array_data), make_array(b_array_data)],
                None,
            )
        };

        expect_ipc_validation_error(
            Arc::new(invalid_struct_arr),
            "Invalid argument error: Incorrect array length for StructArray field \"b\", expected 4 got 3",
        );
    }

    #[test]
    fn test_invalid_nested_array_ipc_read_errors() {
        // one of the nested arrays has invalid data
        let a_field = Field::new("a", DataType::Int32, false);
        let b_field = Field::new("b", DataType::Utf8, false);

        let schema = Arc::new(Schema::new(vec![Field::new_struct(
            "s",
            vec![a_field.clone(), b_field.clone()],
            false,
        )]));

        let a_array_data = ArrayData::builder(a_field.data_type().clone())
            .len(4)
            .add_buffer(Buffer::from_slice_ref([1, 2, 3, 4]))
            .build()
            .unwrap();
        // invalid nested child array -- length is correct, but has invalid utf8 data
        let b_array_data = {
            let valid: &[u8] = b"   ";
            let mut invalid = vec![];
            invalid.extend_from_slice(b"ValidString");
            invalid.extend_from_slice(INVALID_UTF8_FIRST_CHAR);
            let binary_array =
                BinaryArray::from_iter(vec![None, Some(valid), None, Some(&invalid)]);
            let array = unsafe {
                StringArray::new_unchecked(
                    binary_array.offsets().clone(),
                    binary_array.values().clone(),
                    binary_array.nulls().cloned(),
                )
            };
            array.into_data()
        };
        let struct_data_type = schema.field(0).data_type();

        let invalid_struct_arr = unsafe {
            make_array(
                ArrayData::builder(struct_data_type.clone())
                    .len(4)
                    .add_child_data(a_array_data)
                    .add_child_data(b_array_data)
                    .build_unchecked(),
            )
        };
        expect_ipc_validation_error(
            invalid_struct_arr,
            "Invalid argument error: Invalid UTF8 sequence at string index 3 (3..18): invalid utf-8 sequence of 1 bytes from index 11",
        );
    }

    #[test]
    fn test_same_dict_id_without_preserve() {
        let batch = RecordBatch::try_new(
            Arc::new(Schema::new(
                ["a", "b"]
                    .iter()
                    .map(|name| {
                        #[allow(deprecated)]
                        Field::new_dict(
                            name.to_string(),
                            DataType::Dictionary(
                                Box::new(DataType::Int32),
                                Box::new(DataType::Utf8),
                            ),
                            true,
                            0,
                            false,
                        )
                    })
                    .collect::<Vec<Field>>(),
            )),
            vec![
                Arc::new(
                    vec![Some("c"), Some("d")]
                        .into_iter()
                        .collect::<DictionaryArray<Int32Type>>(),
                ) as ArrayRef,
                Arc::new(
                    vec![Some("e"), Some("f")]
                        .into_iter()
                        .collect::<DictionaryArray<Int32Type>>(),
                ) as ArrayRef,
            ],
        )
        .expect("Failed to create RecordBatch");

        // serialize the record batch as an IPC stream
        let mut buf = vec![];
        {
            let mut writer = crate::writer::StreamWriter::try_new_with_options(
                &mut buf,
                batch.schema().as_ref(),
                crate::writer::IpcWriteOptions::default(),
            )
            .expect("Failed to create StreamWriter");
            writer.write(&batch).expect("Failed to write RecordBatch");
            writer.finish().expect("Failed to finish StreamWriter");
        }

        StreamReader::try_new(std::io::Cursor::new(buf), None)
            .expect("Failed to create StreamReader")
            .for_each(|decoded_batch| {
                assert_eq!(decoded_batch.expect("Failed to read RecordBatch"), batch);
            });
    }

    #[test]
    fn test_validation_of_invalid_list_array() {
        // ListArray with invalid offsets
        let array = unsafe {
            let values = Int32Array::from(vec![1, 2, 3]);
            let bad_offsets = ScalarBuffer::<i32>::from(vec![0, 2, 4, 2]); // offsets can't go backwards
            let offsets = OffsetBuffer::new_unchecked(bad_offsets); // INVALID array created
            let field = Field::new_list_field(DataType::Int32, true);
            let nulls = None;
            ListArray::new(Arc::new(field), offsets, Arc::new(values), nulls)
        };

        expect_ipc_validation_error(
            Arc::new(array),
            "Invalid argument error: Offset invariant failure: offset at position 2 out of bounds: 4 > 2",
        );
    }

    #[test]
    fn test_validation_of_invalid_string_array() {
        let valid: &[u8] = b"   ";
        let mut invalid = vec![];
        invalid.extend_from_slice(b"ThisStringIsCertainlyLongerThan12Bytes");
        invalid.extend_from_slice(INVALID_UTF8_FIRST_CHAR);
        let binary_array = BinaryArray::from_iter(vec![None, Some(valid), None, Some(&invalid)]);
        // data is not valid utf8 we can not construct a correct StringArray
        // safely, so purposely create an invalid StringArray
        let array = unsafe {
            StringArray::new_unchecked(
                binary_array.offsets().clone(),
                binary_array.values().clone(),
                binary_array.nulls().cloned(),
            )
        };
        expect_ipc_validation_error(
            Arc::new(array),
            "Invalid argument error: Invalid UTF8 sequence at string index 3 (3..45): invalid utf-8 sequence of 1 bytes from index 38",
        );
    }

    #[test]
    fn test_validation_of_invalid_string_view_array() {
        let valid: &[u8] = b"   ";
        let mut invalid = vec![];
        invalid.extend_from_slice(b"ThisStringIsCertainlyLongerThan12Bytes");
        invalid.extend_from_slice(INVALID_UTF8_FIRST_CHAR);
        let binary_view_array =
            BinaryViewArray::from_iter(vec![None, Some(valid), None, Some(&invalid)]);
        // data is not valid utf8 we can not construct a correct StringArray
        // safely, so purposely create an invalid StringArray
        let array = unsafe {
            StringViewArray::new_unchecked(
                binary_view_array.views().clone(),
                binary_view_array.data_buffers().to_vec(),
                binary_view_array.nulls().cloned(),
            )
        };
        expect_ipc_validation_error(
            Arc::new(array),
            "Invalid argument error: Encountered non-UTF-8 data at index 3: invalid utf-8 sequence of 1 bytes from index 38",
        );
    }

    /// return an invalid dictionary array (key is larger than values)
    /// ListArray with invalid offsets
    #[test]
    fn test_validation_of_invalid_dictionary_array() {
        let array = unsafe {
            let values = StringArray::from_iter_values(["a", "b", "c"]);
            let keys = Int32Array::from(vec![1, 200]); // keys are not valid for values
            DictionaryArray::new_unchecked(keys, Arc::new(values))
        };

        expect_ipc_validation_error(
            Arc::new(array),
            "Invalid argument error: Value at position 1 out of bounds: 200 (should be in [0, 2])",
        );
    }

    #[test]
    fn test_validation_of_invalid_union_array() {
        let array = unsafe {
            let fields = UnionFields::try_new(
                vec![1, 3], // typeids : type id 2 is not valid
                vec![
                    Field::new("a", DataType::Int32, false),
                    Field::new("b", DataType::Utf8, false),
                ],
            )
            .unwrap();
            let type_ids = ScalarBuffer::from(vec![1i8, 2, 3]); // 2 is invalid
            let offsets = None;
            let children: Vec<ArrayRef> = vec![
                Arc::new(Int32Array::from(vec![10, 20, 30])),
                Arc::new(StringArray::from(vec![Some("a"), Some("b"), Some("c")])),
            ];

            UnionArray::new_unchecked(fields, type_ids, offsets, children)
        };

        expect_ipc_validation_error(
            Arc::new(array),
            "Invalid argument error: Type Ids values must match one of the field type ids",
        );
    }

    /// Invalid Utf-8 sequence in the first character
    /// <https://stackoverflow.com/questions/1301402/example-invalid-utf8-string>
    const INVALID_UTF8_FIRST_CHAR: &[u8] = &[0xa0, 0xa1, 0x20, 0x20];

    /// Expect an error when reading the record batch using IPC or IPC Streams
    fn expect_ipc_validation_error(array: ArrayRef, expected_err: &str) {
        let rb = RecordBatch::try_from_iter([("a", array)]).unwrap();

        // IPC Stream format
        let buf = write_stream(&rb); // write is ok
        read_stream_skip_validation(&buf).unwrap();
        let err = read_stream(&buf).unwrap_err();
        assert_eq!(err.to_string(), expected_err);

        // IPC File format
        let buf = write_ipc(&rb); // write is ok
        read_ipc_skip_validation(&buf).unwrap();
        let err = read_ipc(&buf).unwrap_err();
        assert_eq!(err.to_string(), expected_err);

        // IPC Format with FileDecoder
        read_ipc_with_decoder_skip_validation(buf.clone()).unwrap();
        let err = read_ipc_with_decoder(buf).unwrap_err();
        assert_eq!(err.to_string(), expected_err);
    }

    #[test]
    fn test_roundtrip_schema() {
        let schema = Schema::new(vec![
            Field::new(
                "a",
                DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)),
                false,
            ),
            Field::new(
                "b",
                DataType::Dictionary(Box::new(DataType::UInt16), Box::new(DataType::Utf8)),
                false,
            ),
        ]);

        let options = IpcWriteOptions::default();
        let data_gen = IpcDataGenerator::default();
        let mut dict_tracker = DictionaryTracker::new(false);
        let encoded_data =
            data_gen.schema_to_bytes_with_dictionary_tracker(&schema, &mut dict_tracker, &options);
        let mut schema_bytes = vec![];
        write_message(&mut schema_bytes, encoded_data, &options).expect("write_message");

        let begin_offset: usize = if schema_bytes[0..4].eq(&CONTINUATION_MARKER) {
            4
        } else {
            0
        };

        size_prefixed_root_as_message(&schema_bytes[begin_offset..])
            .expect_err("size_prefixed_root_as_message");

        let msg = parse_message(&schema_bytes).expect("parse_message");
        let ipc_schema = msg.header_as_schema().expect("header_as_schema");
        let new_schema = fb_to_schema(ipc_schema);

        assert_eq!(schema, new_schema);
    }

    #[test]
    fn test_negative_meta_len() {
        let bytes = i32::to_le_bytes(-1);
        let mut buf = vec![];
        buf.extend(CONTINUATION_MARKER);
        buf.extend(bytes);

        let reader = StreamReader::try_new(Cursor::new(buf), None);
        assert!(reader.is_err());
    }
}