arrow-select 58.1.0

Selection kernels for arrow arrays
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
// 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.

//! [`BatchCoalescer`]  concatenates multiple [`RecordBatch`]es after
//! operations such as [`filter`] and [`take`].
//!
//! [`filter`]: crate::filter::filter
//! [`take`]: crate::take::take
use crate::filter::filter_record_batch;
use crate::take::take_record_batch;
use arrow_array::types::{BinaryViewType, StringViewType};
use arrow_array::{Array, ArrayRef, BooleanArray, RecordBatch, downcast_primitive};
use arrow_schema::{ArrowError, DataType, SchemaRef};
use std::collections::VecDeque;
use std::sync::Arc;
// Originally From DataFusion's coalesce module:
// https://github.com/apache/datafusion/blob/9d2f04996604e709ee440b65f41e7b882f50b788/datafusion/physical-plan/src/coalesce/mod.rs#L26-L25

mod byte_view;
mod generic;
mod primitive;

use byte_view::InProgressByteViewArray;
use generic::GenericInProgressArray;
use primitive::InProgressPrimitiveArray;

/// Concatenate multiple [`RecordBatch`]es
///
/// Implements the common pattern of incrementally creating output
/// [`RecordBatch`]es of a specific size from an input stream of
/// [`RecordBatch`]es.
///
/// This is useful after operations such as [`filter`] and [`take`] that produce
/// smaller batches, and we want to coalesce them into larger batches for
/// further processing.
///
/// # Motivation
///
/// If we use [`concat_batches`] to implement the same functionality, there are 2 potential issues:
/// 1. At least 2x peak memory (holding the input and output of concat)
/// 2. 2 copies of the data (to create the output of filter and then create the output of concat)
///
/// See: <https://github.com/apache/arrow-rs/issues/6692> for more discussions
/// about the motivation.
///
/// [`filter`]: crate::filter::filter
/// [`take`]: crate::take::take
/// [`concat_batches`]: crate::concat::concat_batches
///
/// # Example
/// ```
/// use arrow_array::record_batch;
/// use arrow_select::coalesce::{BatchCoalescer};
/// let batch1 = record_batch!(("a", Int32, [1, 2, 3])).unwrap();
/// let batch2 = record_batch!(("a", Int32, [4, 5])).unwrap();
///
/// // Create a `BatchCoalescer` that will produce batches with at least 4 rows
/// let target_batch_size = 4;
/// let mut coalescer = BatchCoalescer::new(batch1.schema(), 4);
///
/// // push the batches
/// coalescer.push_batch(batch1).unwrap();
/// // only pushed 3 rows (not yet 4, enough to produce a batch)
/// assert!(coalescer.next_completed_batch().is_none());
/// coalescer.push_batch(batch2).unwrap();
/// // now we have 5 rows, so we can produce a batch
/// let finished = coalescer.next_completed_batch().unwrap();
/// // 4 rows came out (target batch size is 4)
/// let expected = record_batch!(("a", Int32, [1, 2, 3, 4])).unwrap();
/// assert_eq!(finished, expected);
///
/// // Have no more input, but still have an in-progress batch
/// assert!(coalescer.next_completed_batch().is_none());
/// // We can finish the batch, which will produce the remaining rows
/// coalescer.finish_buffered_batch().unwrap();
/// let expected = record_batch!(("a", Int32, [5])).unwrap();
/// assert_eq!(coalescer.next_completed_batch().unwrap(), expected);
///
/// // The coalescer is now empty
/// assert!(coalescer.next_completed_batch().is_none());
/// ```
///
/// # Background
///
/// Generally speaking, larger [`RecordBatch`]es are more efficient to process
/// than smaller [`RecordBatch`]es (until the CPU cache is exceeded) because
/// there is fixed processing overhead per batch. This coalescer builds up these
/// larger batches incrementally.
///
/// ```text
/// ┌────────────────────┐
/// │    RecordBatch     │
/// │   num_rows = 100   │
/// └────────────────────┘                 ┌────────────────────┐
///                                        │                    │
/// ┌────────────────────┐     Coalesce    │                    │
/// │                    │      Batches    │                    │
/// │    RecordBatch     │                 │                    │
/// │   num_rows = 200   │  ─ ─ ─ ─ ─ ─ ▶  │                    │
/// │                    │                 │    RecordBatch     │
/// │                    │                 │   num_rows = 400   │
/// └────────────────────┘                 │                    │
///                                        │                    │
/// ┌────────────────────┐                 │                    │
/// │                    │                 │                    │
/// │    RecordBatch     │                 │                    │
/// │   num_rows = 100   │                 └────────────────────┘
/// │                    │
/// └────────────────────┘
/// ```
///
/// # Notes:
///
/// 1. Output rows are produced in the same order as the input rows
///
/// 2. The output is a sequence of batches, with all but the last being at exactly
///    `target_batch_size` rows.
#[derive(Debug)]
pub struct BatchCoalescer {
    /// The input schema
    schema: SchemaRef,
    /// The target batch size (and thus size for views allocation). This is a
    /// hard limit: the output batch will be exactly `target_batch_size`,
    /// rather than possibly being slightly above.
    target_batch_size: usize,
    /// In-progress arrays
    in_progress_arrays: Vec<Box<dyn InProgressArray>>,
    /// Buffered row count. Always less than `batch_size`
    buffered_rows: usize,
    /// Completed batches
    completed: VecDeque<RecordBatch>,
    /// Biggest coalesce batch size. See [`Self::with_biggest_coalesce_batch_size`]
    biggest_coalesce_batch_size: Option<usize>,
}

impl BatchCoalescer {
    /// Create a new `BatchCoalescer`
    ///
    /// # Arguments
    /// - `schema` - the schema of the output batches
    /// - `target_batch_size` - the number of rows in each output batch.
    ///   Typical values are `4096` or `8192` rows.
    ///
    pub fn new(schema: SchemaRef, target_batch_size: usize) -> Self {
        let in_progress_arrays = schema
            .fields()
            .iter()
            .map(|field| create_in_progress_array(field.data_type(), target_batch_size))
            .collect::<Vec<_>>();

        Self {
            schema,
            target_batch_size,
            in_progress_arrays,
            // We will for sure store at least one completed batch
            completed: VecDeque::with_capacity(1),
            buffered_rows: 0,
            biggest_coalesce_batch_size: None,
        }
    }

    /// Set the coalesce batch size limit (default `None`)
    ///
    /// This limit determine when batches should bypass coalescing. Intuitively,
    /// batches that are already large are costly to coalesce and are efficient
    /// enough to process directly without coalescing.
    ///
    /// If `Some(limit)`, batches larger than this limit will bypass coalescing
    /// when there is no buffered data, or when the previously buffered data
    /// already exceeds this limit.
    ///
    /// If `None`, all batches will be coalesced according to the
    /// target_batch_size.
    pub fn with_biggest_coalesce_batch_size(mut self, limit: Option<usize>) -> Self {
        self.biggest_coalesce_batch_size = limit;
        self
    }

    /// Get the current biggest coalesce batch size limit
    ///
    /// See [`Self::with_biggest_coalesce_batch_size`] for details
    pub fn biggest_coalesce_batch_size(&self) -> Option<usize> {
        self.biggest_coalesce_batch_size
    }

    /// Set the biggest coalesce batch size limit
    ///
    /// See [`Self::with_biggest_coalesce_batch_size`] for details
    pub fn set_biggest_coalesce_batch_size(&mut self, limit: Option<usize>) {
        self.biggest_coalesce_batch_size = limit;
    }

    /// Return the schema of the output batches
    pub fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }

    /// Push a batch into the Coalescer after applying a filter
    ///
    /// This is semantically equivalent of calling [`Self::push_batch`]
    /// with the results from  [`filter_record_batch`]
    ///
    /// # Example
    /// ```
    /// # use arrow_array::{record_batch, BooleanArray};
    /// # use arrow_select::coalesce::BatchCoalescer;
    /// let batch1 = record_batch!(("a", Int32, [1, 2, 3])).unwrap();
    /// let batch2 = record_batch!(("a", Int32, [4, 5, 6])).unwrap();
    /// // Apply a filter to each batch to pick the first and last row
    /// let filter = BooleanArray::from(vec![true, false, true]);
    /// // create a new Coalescer that targets creating 1000 row batches
    /// let mut coalescer = BatchCoalescer::new(batch1.schema(), 1000);
    /// coalescer.push_batch_with_filter(batch1, &filter);
    /// coalescer.push_batch_with_filter(batch2, &filter);
    /// // finsh and retrieve the created batch
    /// coalescer.finish_buffered_batch().unwrap();
    /// let completed_batch = coalescer.next_completed_batch().unwrap();
    /// // filtered out 2 and 5:
    /// let expected_batch = record_batch!(("a", Int32, [1, 3, 4, 6])).unwrap();
    /// assert_eq!(completed_batch, expected_batch);
    /// ```
    pub fn push_batch_with_filter(
        &mut self,
        batch: RecordBatch,
        filter: &BooleanArray,
    ) -> Result<(), ArrowError> {
        // TODO: optimize this to avoid materializing (copying the results
        // of filter to a new batch)
        let filtered_batch = filter_record_batch(&batch, filter)?;
        self.push_batch(filtered_batch)
    }

    /// Push a batch into the Coalescer after applying a set of indices
    /// This is semantically equivalent of calling [`Self::push_batch`]
    /// with the results from  [`take_record_batch`]
    ///
    /// # Example
    /// ```
    /// # use arrow_array::{record_batch, UInt64Array};
    /// # use arrow_select::coalesce::BatchCoalescer;
    /// let batch1 = record_batch!(("a", Int32, [0, 0, 0])).unwrap();
    /// let batch2 = record_batch!(("a", Int32, [1, 1, 4, 5, 1, 4])).unwrap();
    /// // Sorted indices to create a sorted output, this can be obtained with
    /// // `arrow-ord`'s sort_to_indices operation
    /// let indices = UInt64Array::from(vec![0, 1, 4, 2, 5, 3]);
    /// // create a new Coalescer that targets creating 1000 row batches
    /// let mut coalescer = BatchCoalescer::new(batch1.schema(), 1000);
    /// coalescer.push_batch(batch1);
    /// coalescer.push_batch_with_indices(batch2, &indices);
    /// // finsh and retrieve the created batch
    /// coalescer.finish_buffered_batch().unwrap();
    /// let completed_batch = coalescer.next_completed_batch().unwrap();
    /// let expected_batch = record_batch!(("a", Int32, [0, 0, 0, 1, 1, 1, 4, 4, 5])).unwrap();
    /// assert_eq!(completed_batch, expected_batch);
    /// ```
    pub fn push_batch_with_indices(
        &mut self,
        batch: RecordBatch,
        indices: &dyn Array,
    ) -> Result<(), ArrowError> {
        // todo: optimize this to avoid materializing (copying the results of take indices to a new batch)
        let taken_batch = take_record_batch(&batch, indices)?;
        self.push_batch(taken_batch)
    }

    /// Push all the rows from `batch` into the Coalescer
    ///
    /// When buffered data plus incoming rows reach `target_batch_size` ,
    /// completed batches are generated eagerly and can be retrieved via
    /// [`Self::next_completed_batch()`].
    /// Output batches contain exactly `target_batch_size` rows, so the tail of
    /// the input batch may remain buffered.
    /// Remaining partial data either waits for future input batches or can be
    /// materialized immediately by calling [`Self::finish_buffered_batch()`].
    ///
    /// # Example
    /// ```
    /// # use arrow_array::record_batch;
    /// # use arrow_select::coalesce::BatchCoalescer;
    /// let batch1 = record_batch!(("a", Int32, [1, 2, 3])).unwrap();
    /// let batch2 = record_batch!(("a", Int32, [4, 5, 6])).unwrap();
    /// // create a new Coalescer that targets creating 1000 row batches
    /// let mut coalescer = BatchCoalescer::new(batch1.schema(), 1000);
    /// coalescer.push_batch(batch1);
    /// coalescer.push_batch(batch2);
    /// // finsh and retrieve the created batch
    /// coalescer.finish_buffered_batch().unwrap();
    /// let completed_batch = coalescer.next_completed_batch().unwrap();
    /// let expected_batch = record_batch!(("a", Int32, [1, 2, 3, 4, 5, 6])).unwrap();
    /// assert_eq!(completed_batch, expected_batch);
    /// ```
    pub fn push_batch(&mut self, batch: RecordBatch) -> Result<(), ArrowError> {
        // Large batch bypass optimization:
        // When biggest_coalesce_batch_size is configured and a batch exceeds this limit,
        // we can avoid expensive split-and-merge operations by passing it through directly.
        //
        // IMPORTANT: This optimization is OPTIONAL and only active when biggest_coalesce_batch_size
        // is explicitly set via with_biggest_coalesce_batch_size(Some(limit)).
        // If not set (None), ALL batches follow normal coalescing behavior regardless of size.

        // =============================================================================
        // CASE 1: No buffer + large batch → Direct bypass
        // =============================================================================
        // Example scenario (target_batch_size=1000, biggest_coalesce_batch_size=Some(500)):
        // Input sequence: [600, 1200, 300]
        //
        // With biggest_coalesce_batch_size=Some(500) (optimization enabled):
        //   600 → large batch detected! buffered_rows=0 → Case 1: direct bypass
        //        → output: [600] (bypass, preserves large batch)
        //   1200 → large batch detected! buffered_rows=0 → Case 1: direct bypass
        //         → output: [1200] (bypass, preserves large batch)
        //   300 → normal batch, buffer: [300]
        //   Result: [600], [1200], [300] - large batches preserved, mixed sizes

        // =============================================================================
        // CASE 2: Buffer too large + large batch → Flush first, then bypass
        // =============================================================================
        // This case prevents creating extremely large merged batches that would
        // significantly exceed both target_batch_size and biggest_coalesce_batch_size.
        //
        // Example 1: Buffer exceeds limit before large batch arrives
        // target_batch_size=1000, biggest_coalesce_batch_size=Some(400)
        // Input: [350, 200, 800]
        //
        // Step 1: push_batch([350])
        //   → batch_size=350 <= 400, normal path
        //   → buffer: [350], buffered_rows=350
        //
        // Step 2: push_batch([200])
        //   → batch_size=200 <= 400, normal path
        //   → buffer: [350, 200], buffered_rows=550
        //
        // Step 3: push_batch([800])
        //   → batch_size=800 > 400, large batch path
        //   → buffered_rows=550 > 400 → Case 2: flush first
        //   → flush: output [550] (combined [350, 200])
        //   → then bypass: output [800]
        //   Result: [550], [800] - buffer flushed to prevent oversized merge
        //
        // Example 2: Multiple small batches accumulate before large batch
        // target_batch_size=1000, biggest_coalesce_batch_size=Some(300)
        // Input: [150, 100, 80, 900]
        //
        // Step 1-3: Accumulate small batches
        //   150 → buffer: [150], buffered_rows=150
        //   100 → buffer: [150, 100], buffered_rows=250
        //   80  → buffer: [150, 100, 80], buffered_rows=330
        //
        // Step 4: push_batch([900])
        //   → batch_size=900 > 300, large batch path
        //   → buffered_rows=330 > 300 → Case 2: flush first
        //   → flush: output [330] (combined [150, 100, 80])
        //   → then bypass: output [900]
        //   Result: [330], [900] - prevents merge into [1230] which would be too large

        // =============================================================================
        // CASE 3: Small buffer + large batch → Normal coalescing (no bypass)
        // =============================================================================
        // When buffer is small enough, we still merge to maintain efficiency
        // Example: target_batch_size=1000, biggest_coalesce_batch_size=Some(500)
        // Input: [300, 1200]
        //
        // Step 1: push_batch([300])
        //   → batch_size=300 <= 500, normal path
        //   → buffer: [300], buffered_rows=300
        //
        // Step 2: push_batch([1200])
        //   → batch_size=1200 > 500, large batch path
        //   → buffered_rows=300 <= 500 → Case 3: normal merge
        //   → buffer: [300, 1200] (1500 total)
        //   → 1500 > target_batch_size → split: output [1000], buffer [500]
        //   Result: [1000], [500] - normal split/merge behavior maintained

        // =============================================================================
        // Comparison: Default vs Optimized Behavior
        // =============================================================================
        // target_batch_size=1000, biggest_coalesce_batch_size=Some(500)
        // Input: [600, 1200, 300]
        //
        // DEFAULT BEHAVIOR (biggest_coalesce_batch_size=None):
        //   600 → buffer: [600]
        //   1200 → buffer: [600, 1200] (1800 rows total)
        //         → split: output [1000 rows], buffer [800 rows remaining]
        //   300 → buffer: [800, 300] (1100 rows total)
        //        → split: output [1000 rows], buffer [100 rows remaining]
        //   Result: [1000], [1000], [100] - all outputs respect target_batch_size
        //
        // OPTIMIZED BEHAVIOR (biggest_coalesce_batch_size=Some(500)):
        //   600 → Case 1: direct bypass → output: [600]
        //   1200 → Case 1: direct bypass → output: [1200]
        //   300 → normal path → buffer: [300]
        //   Result: [600], [1200], [300] - large batches preserved

        // =============================================================================
        // Benefits and Trade-offs
        // =============================================================================
        // Benefits of the optimization:
        // - Large batches stay intact (better for downstream vectorized processing)
        // - Fewer split/merge operations (better CPU performance)
        // - More predictable memory usage patterns
        // - Maintains streaming efficiency while preserving batch boundaries
        //
        // Trade-offs:
        // - Output batch sizes become variable (not always target_batch_size)
        // - May produce smaller partial batches when flushing before large batches
        // - Requires tuning biggest_coalesce_batch_size parameter for optimal performance

        // TODO, for unsorted batches, we may can filter all large batches, and coalesce all
        // small batches together?

        let batch_size = batch.num_rows();

        // Fast path: skip empty batches
        if batch_size == 0 {
            return Ok(());
        }

        // Large batch optimization: bypass coalescing for oversized batches
        if let Some(limit) = self.biggest_coalesce_batch_size {
            if batch_size > limit {
                // Case 1: No buffered data - emit large batch directly
                // Example: [] + [1200] → output [1200], buffer []
                if self.buffered_rows == 0 {
                    self.completed.push_back(batch);
                    return Ok(());
                }

                // Case 2: Buffer too large - flush then emit to avoid oversized merge
                // Example: [850] + [1200] → output [850], then output [1200]
                // This prevents creating batches much larger than both target_batch_size
                // and biggest_coalesce_batch_size, which could cause memory issues
                if self.buffered_rows > limit {
                    self.finish_buffered_batch()?;
                    self.completed.push_back(batch);
                    return Ok(());
                }

                // Case 3: Small buffer - proceed with normal coalescing
                // Example: [300] + [1200] → split and merge normally
                // This ensures small batches still get properly coalesced
                // while allowing some controlled growth beyond the limit
            }
        }

        let (_schema, arrays, mut num_rows) = batch.into_parts();

        // Validate column count matches the expected schema
        if arrays.len() != self.in_progress_arrays.len() {
            return Err(ArrowError::InvalidArgumentError(format!(
                "Batch has {} columns but BatchCoalescer expects {}",
                arrays.len(),
                self.in_progress_arrays.len()
            )));
        }
        self.in_progress_arrays
            .iter_mut()
            .zip(arrays)
            .for_each(|(in_progress, array)| {
                in_progress.set_source(Some(array));
            });

        // If pushing this batch would exceed the target batch size,
        // finish the current batch and start a new one
        let mut offset = 0;
        while num_rows > (self.target_batch_size - self.buffered_rows) {
            let remaining_rows = self.target_batch_size - self.buffered_rows;
            debug_assert!(remaining_rows > 0);

            // Copy remaining_rows from each array
            for in_progress in self.in_progress_arrays.iter_mut() {
                in_progress.copy_rows(offset, remaining_rows)?;
            }

            self.buffered_rows += remaining_rows;
            offset += remaining_rows;
            num_rows -= remaining_rows;

            self.finish_buffered_batch()?;
        }

        // Add any the remaining rows to the buffer
        self.buffered_rows += num_rows;
        if num_rows > 0 {
            for in_progress in self.in_progress_arrays.iter_mut() {
                in_progress.copy_rows(offset, num_rows)?;
            }
        }

        // If we have reached the target batch size, finalize the buffered batch
        if self.buffered_rows >= self.target_batch_size {
            self.finish_buffered_batch()?;
        }

        // clear in progress sources (to allow the memory to be freed)
        for in_progress in self.in_progress_arrays.iter_mut() {
            in_progress.set_source(None);
        }

        Ok(())
    }

    /// Returns the number of buffered rows
    pub fn get_buffered_rows(&self) -> usize {
        self.buffered_rows
    }

    /// Concatenates any buffered batches into a single `RecordBatch` and
    /// clears any output buffers
    ///
    /// Normally this is called when the input stream is exhausted, and
    /// we want to finalize the last batch of rows.
    ///
    /// See [`Self::next_completed_batch()`] for the completed batches.
    pub fn finish_buffered_batch(&mut self) -> Result<(), ArrowError> {
        if self.buffered_rows == 0 {
            return Ok(());
        }
        let new_arrays = self
            .in_progress_arrays
            .iter_mut()
            .map(|array| array.finish())
            .collect::<Result<Vec<_>, ArrowError>>()?;

        for (array, field) in new_arrays.iter().zip(self.schema.fields().iter()) {
            debug_assert_eq!(array.data_type(), field.data_type());
            debug_assert_eq!(array.len(), self.buffered_rows);
        }

        // SAFETY: each array was created of the correct type and length.
        let batch = unsafe {
            RecordBatch::new_unchecked(Arc::clone(&self.schema), new_arrays, self.buffered_rows)
        };

        self.buffered_rows = 0;
        self.completed.push_back(batch);
        Ok(())
    }

    /// Returns true if there is any buffered data
    pub fn is_empty(&self) -> bool {
        self.buffered_rows == 0 && self.completed.is_empty()
    }

    /// Returns true if there are any completed batches
    pub fn has_completed_batch(&self) -> bool {
        !self.completed.is_empty()
    }

    /// Removes and returns the next completed batch, if any.
    pub fn next_completed_batch(&mut self) -> Option<RecordBatch> {
        self.completed.pop_front()
    }
}

/// Return a new `InProgressArray` for the given data type
fn create_in_progress_array(data_type: &DataType, batch_size: usize) -> Box<dyn InProgressArray> {
    macro_rules! instantiate_primitive {
        ($t:ty) => {
            Box::new(InProgressPrimitiveArray::<$t>::new(
                batch_size,
                data_type.clone(),
            ))
        };
    }

    downcast_primitive! {
        // Instantiate InProgressPrimitiveArray for each primitive type
        data_type => (instantiate_primitive),
        DataType::Utf8View => Box::new(InProgressByteViewArray::<StringViewType>::new(batch_size)),
        DataType::BinaryView => {
            Box::new(InProgressByteViewArray::<BinaryViewType>::new(batch_size))
        }
        _ => Box::new(GenericInProgressArray::new()),
    }
}

/// Incrementally builds up arrays
///
/// [`GenericInProgressArray`] is the default implementation that buffers
/// arrays and uses other kernels concatenates them when finished.
///
/// Some types have specialized implementations for this array types (e.g.,
/// [`StringViewArray`], etc.).
///
/// [`StringViewArray`]: arrow_array::StringViewArray
trait InProgressArray: std::fmt::Debug + Send + Sync {
    /// Set the source array.
    ///
    /// Calls to [`Self::copy_rows`] will copy rows from this array into the
    /// current in-progress array
    fn set_source(&mut self, source: Option<ArrayRef>);

    /// Copy rows from the current source array into the in-progress array
    ///
    /// The source array is set by [`Self::set_source`].
    ///
    /// Return an error if the source array is not set
    fn copy_rows(&mut self, offset: usize, len: usize) -> Result<(), ArrowError>;

    /// Finish the currently in-progress array and return it as an `ArrayRef`
    fn finish(&mut self) -> Result<ArrayRef, ArrowError>;
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::concat::concat_batches;
    use arrow_array::builder::StringViewBuilder;
    use arrow_array::cast::AsArray;
    use arrow_array::types::Int32Type;
    use arrow_array::{
        BinaryViewArray, Int32Array, Int64Array, RecordBatchOptions, StringArray, StringViewArray,
        TimestampNanosecondArray, UInt32Array, UInt64Array, make_array,
    };
    use arrow_buffer::BooleanBufferBuilder;
    use arrow_schema::{DataType, Field, Schema};
    use rand::{Rng, SeedableRng};
    use std::ops::Range;

    #[test]
    fn test_coalesce() {
        let batch = uint32_batch(0..8);
        Test::new("coalesce")
            .with_batches(std::iter::repeat_n(batch, 10))
            // expected output is exactly 21 rows (except for the final batch)
            .with_batch_size(21)
            .with_expected_output_sizes(vec![21, 21, 21, 17])
            .run();
    }

    #[test]
    fn test_coalesce_one_by_one() {
        let batch = uint32_batch(0..1); // single row input
        Test::new("coalesce_one_by_one")
            .with_batches(std::iter::repeat_n(batch, 97))
            // expected output is exactly 20 rows (except for the final batch)
            .with_batch_size(20)
            .with_expected_output_sizes(vec![20, 20, 20, 20, 17])
            .run();
    }

    #[test]
    fn test_coalesce_empty() {
        let schema = Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)]));

        Test::new("coalesce_empty")
            .with_batches(vec![])
            .with_schema(schema)
            .with_batch_size(21)
            .with_expected_output_sizes(vec![])
            .run();
    }

    #[test]
    fn test_single_large_batch_greater_than_target() {
        // test a single large batch
        let batch = uint32_batch(0..4096);
        Test::new("coalesce_single_large_batch_greater_than_target")
            .with_batch(batch)
            .with_batch_size(1000)
            .with_expected_output_sizes(vec![1000, 1000, 1000, 1000, 96])
            .run();
    }

    #[test]
    fn test_single_large_batch_smaller_than_target() {
        // test a single large batch
        let batch = uint32_batch(0..4096);
        Test::new("coalesce_single_large_batch_smaller_than_target")
            .with_batch(batch)
            .with_batch_size(8192)
            .with_expected_output_sizes(vec![4096])
            .run();
    }

    #[test]
    fn test_single_large_batch_equal_to_target() {
        // test a single large batch
        let batch = uint32_batch(0..4096);
        Test::new("coalesce_single_large_batch_equal_to_target")
            .with_batch(batch)
            .with_batch_size(4096)
            .with_expected_output_sizes(vec![4096])
            .run();
    }

    #[test]
    fn test_single_large_batch_equally_divisible_in_target() {
        // test a single large batch
        let batch = uint32_batch(0..4096);
        Test::new("coalesce_single_large_batch_equally_divisible_in_target")
            .with_batch(batch)
            .with_batch_size(1024)
            .with_expected_output_sizes(vec![1024, 1024, 1024, 1024])
            .run();
    }

    #[test]
    fn test_empty_schema() {
        let schema = Schema::empty();
        let batch = RecordBatch::new_empty(schema.into());
        Test::new("coalesce_empty_schema")
            .with_batch(batch)
            .with_expected_output_sizes(vec![])
            .run();
    }

    /// Coalesce multiple batches, 80k rows, with a 0.1% selectivity filter
    #[test]
    fn test_coalesce_filtered_001() {
        let mut filter_builder = RandomFilterBuilder {
            num_rows: 8000,
            selectivity: 0.001,
            seed: 0,
        };

        // add 10 batches of 8000 rows each
        // 80k rows, selecting 0.1% means 80 rows
        // not exactly 80 as the rows are random;
        let mut test = Test::new("coalesce_filtered_001");
        for _ in 0..10 {
            test = test
                .with_batch(multi_column_batch(0..8000))
                .with_filter(filter_builder.next_filter())
        }
        test.with_batch_size(15)
            .with_expected_output_sizes(vec![15, 15, 15, 13])
            .run();
    }

    /// Coalesce multiple batches, 80k rows, with a 1% selectivity filter
    #[test]
    fn test_coalesce_filtered_01() {
        let mut filter_builder = RandomFilterBuilder {
            num_rows: 8000,
            selectivity: 0.01,
            seed: 0,
        };

        // add 10 batches of 8000 rows each
        // 80k rows, selecting 1% means 800 rows
        // not exactly 800 as the rows are random;
        let mut test = Test::new("coalesce_filtered_01");
        for _ in 0..10 {
            test = test
                .with_batch(multi_column_batch(0..8000))
                .with_filter(filter_builder.next_filter())
        }
        test.with_batch_size(128)
            .with_expected_output_sizes(vec![128, 128, 128, 128, 128, 128, 15])
            .run();
    }

    /// Coalesce multiple batches, 80k rows, with a 10% selectivity filter
    #[test]
    fn test_coalesce_filtered_10() {
        let mut filter_builder = RandomFilterBuilder {
            num_rows: 8000,
            selectivity: 0.1,
            seed: 0,
        };

        // add 10 batches of 8000 rows each
        // 80k rows, selecting 10% means 8000 rows
        // not exactly 800 as the rows are random;
        let mut test = Test::new("coalesce_filtered_10");
        for _ in 0..10 {
            test = test
                .with_batch(multi_column_batch(0..8000))
                .with_filter(filter_builder.next_filter())
        }
        test.with_batch_size(1024)
            .with_expected_output_sizes(vec![1024, 1024, 1024, 1024, 1024, 1024, 1024, 840])
            .run();
    }

    /// Coalesce multiple batches, 8k rows, with a 90% selectivity filter
    #[test]
    fn test_coalesce_filtered_90() {
        let mut filter_builder = RandomFilterBuilder {
            num_rows: 800,
            selectivity: 0.90,
            seed: 0,
        };

        // add 10 batches of 800 rows each
        // 8k rows, selecting 99% means 7200 rows
        // not exactly 7200 as the rows are random;
        let mut test = Test::new("coalesce_filtered_90");
        for _ in 0..10 {
            test = test
                .with_batch(multi_column_batch(0..800))
                .with_filter(filter_builder.next_filter())
        }
        test.with_batch_size(1024)
            .with_expected_output_sizes(vec![1024, 1024, 1024, 1024, 1024, 1024, 1024, 13])
            .run();
    }

    /// Coalesce multiple batches, 8k rows, with mixed filers, including 100%
    #[test]
    fn test_coalesce_filtered_mixed() {
        let mut filter_builder = RandomFilterBuilder {
            num_rows: 800,
            selectivity: 0.90,
            seed: 0,
        };

        let mut test = Test::new("coalesce_filtered_mixed");
        for _ in 0..3 {
            // also add in a batch that selects almost all rows and when
            // sliced will have some batches that are entirely used
            let mut all_filter_builder = BooleanBufferBuilder::new(1000);
            all_filter_builder.append_n(500, true);
            all_filter_builder.append_n(1, false);
            all_filter_builder.append_n(499, false);
            let all_filter = all_filter_builder.build();

            test = test
                .with_batch(multi_column_batch(0..1000))
                .with_filter(BooleanArray::from(all_filter))
                .with_batch(multi_column_batch(0..800))
                .with_filter(filter_builder.next_filter());
            // decrease selectivity
            filter_builder.selectivity *= 0.6;
        }

        // use a small batch size to ensure the filter is appended in slices
        // and some of those slides will select the entire thing.
        test.with_batch_size(250)
            .with_expected_output_sizes(vec![
                250, 250, 250, 250, 250, 250, 250, 250, 250, 250, 250, 179,
            ])
            .run();
    }

    #[test]
    fn test_coalesce_non_null() {
        Test::new("coalesce_non_null")
            // 4040 rows of unit32
            .with_batch(uint32_batch_non_null(0..3000))
            .with_batch(uint32_batch_non_null(0..1040))
            .with_batch_size(1024)
            .with_expected_output_sizes(vec![1024, 1024, 1024, 968])
            .run();
    }
    #[test]
    fn test_utf8_split() {
        Test::new("coalesce_utf8")
            // 4040 rows of utf8 strings in total, split into batches of 1024
            .with_batch(utf8_batch(0..3000))
            .with_batch(utf8_batch(0..1040))
            .with_batch_size(1024)
            .with_expected_output_sizes(vec![1024, 1024, 1024, 968])
            .run();
    }

    #[test]
    fn test_string_view_no_views() {
        let output_batches = Test::new("coalesce_string_view_no_views")
            // both input batches have no views, so no need to compact
            .with_batch(stringview_batch([Some("foo"), Some("bar")]))
            .with_batch(stringview_batch([Some("baz"), Some("qux")]))
            .with_expected_output_sizes(vec![4])
            .run();

        expect_buffer_layout(
            col_as_string_view("c0", output_batches.first().unwrap()),
            vec![],
        );
    }

    #[test]
    fn test_string_view_batch_small_no_compact() {
        // view with only short strings (no buffers) --> no need to compact
        let batch = stringview_batch_repeated(1000, [Some("a"), Some("b"), Some("c")]);
        let output_batches = Test::new("coalesce_string_view_batch_small_no_compact")
            .with_batch(batch.clone())
            .with_expected_output_sizes(vec![1000])
            .run();

        let array = col_as_string_view("c0", &batch);
        let gc_array = col_as_string_view("c0", output_batches.first().unwrap());
        assert_eq!(array.data_buffers().len(), 0);
        assert_eq!(array.data_buffers().len(), gc_array.data_buffers().len()); // no compaction

        expect_buffer_layout(gc_array, vec![]);
    }

    #[test]
    fn test_string_view_batch_large_no_compact() {
        // view with large strings (has buffers) but full --> no need to compact
        let batch = stringview_batch_repeated(1000, [Some("This string is longer than 12 bytes")]);
        let output_batches = Test::new("coalesce_string_view_batch_large_no_compact")
            .with_batch(batch.clone())
            .with_batch_size(1000)
            .with_expected_output_sizes(vec![1000])
            .run();

        let array = col_as_string_view("c0", &batch);
        let gc_array = col_as_string_view("c0", output_batches.first().unwrap());
        assert_eq!(array.data_buffers().len(), 5);
        assert_eq!(array.data_buffers().len(), gc_array.data_buffers().len()); // no compaction

        expect_buffer_layout(
            gc_array,
            vec![
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 2240,
                    capacity: 8192,
                },
            ],
        );
    }

    #[test]
    fn test_string_view_batch_small_with_buffers_no_compact() {
        // view with buffers but only short views
        let short_strings = std::iter::repeat(Some("SmallString"));
        let long_strings = std::iter::once(Some("This string is longer than 12 bytes"));
        // 20 short strings, then a long ones
        let values = short_strings.take(20).chain(long_strings);
        let batch = stringview_batch_repeated(1000, values)
            // take only 10 short strings (no long ones)
            .slice(5, 10);
        let output_batches = Test::new("coalesce_string_view_batch_small_with_buffers_no_compact")
            .with_batch(batch.clone())
            .with_batch_size(1000)
            .with_expected_output_sizes(vec![10])
            .run();

        let array = col_as_string_view("c0", &batch);
        let gc_array = col_as_string_view("c0", output_batches.first().unwrap());
        assert_eq!(array.data_buffers().len(), 1); // input has one buffer
        assert_eq!(gc_array.data_buffers().len(), 0); // output has no buffers as only short strings
    }

    #[test]
    fn test_string_view_batch_large_slice_compact() {
        // view with large strings (has buffers) and only partially used  --> no need to compact
        let batch = stringview_batch_repeated(1000, [Some("This string is longer than 12 bytes")])
            // slice only 22 rows, so most of the buffer is not used
            .slice(11, 22);

        let output_batches = Test::new("coalesce_string_view_batch_large_slice_compact")
            .with_batch(batch.clone())
            .with_batch_size(1000)
            .with_expected_output_sizes(vec![22])
            .run();

        let array = col_as_string_view("c0", &batch);
        let gc_array = col_as_string_view("c0", output_batches.first().unwrap());
        assert_eq!(array.data_buffers().len(), 5);

        expect_buffer_layout(
            gc_array,
            vec![ExpectedLayout {
                len: 770,
                capacity: 8192,
            }],
        );
    }

    #[test]
    fn test_string_view_mixed() {
        let large_view_batch =
            stringview_batch_repeated(1000, [Some("This string is longer than 12 bytes")]);
        let small_view_batch = stringview_batch_repeated(1000, [Some("SmallString")]);
        let mixed_batch = stringview_batch_repeated(
            1000,
            [Some("This string is longer than 12 bytes"), Some("Small")],
        );
        let mixed_batch_nulls = stringview_batch_repeated(
            1000,
            [
                Some("This string is longer than 12 bytes"),
                Some("Small"),
                None,
            ],
        );

        // Several batches with mixed inline / non inline
        // 4k rows in
        let output_batches = Test::new("coalesce_string_view_mixed")
            .with_batch(large_view_batch.clone())
            .with_batch(small_view_batch)
            // this batch needs to be compacted (less than 1/2 full)
            .with_batch(large_view_batch.slice(10, 20))
            .with_batch(mixed_batch_nulls)
            // this batch needs to be compacted (less than 1/2 full)
            .with_batch(large_view_batch.slice(10, 20))
            .with_batch(mixed_batch)
            .with_expected_output_sizes(vec![1024, 1024, 1024, 968])
            .run();

        expect_buffer_layout(
            col_as_string_view("c0", output_batches.first().unwrap()),
            vec![
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 2240,
                    capacity: 8192,
                },
            ],
        );
    }

    #[test]
    fn test_string_view_many_small_compact() {
        // 200 rows alternating long (28) and short (≤12) strings.
        // Only the 100 long strings go into data buffers: 100 × 28 = 2800.
        let batch = stringview_batch_repeated(
            200,
            [Some("This string is 28 bytes long"), Some("small string")],
        );
        let output_batches = Test::new("coalesce_string_view_many_small_compact")
            // First allocated buffer is 8kb.
            // Appending 10 batches of 2800 bytes will use 2800 * 10 = 14kb (8kb, an 16kb and 32kbkb)
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch_size(8000)
            .with_expected_output_sizes(vec![2000]) // only 1000 rows total
            .run();

        // expect a nice even distribution of buffers
        expect_buffer_layout(
            col_as_string_view("c0", output_batches.first().unwrap()),
            vec![
                ExpectedLayout {
                    len: 8176,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 16380,
                    capacity: 16384,
                },
                ExpectedLayout {
                    len: 3444,
                    capacity: 32768,
                },
            ],
        );
    }

    #[test]
    fn test_string_view_many_small_boundary() {
        // The strings are designed to exactly fit into buffers that are powers of 2 long
        let batch = stringview_batch_repeated(100, [Some("This string is a power of two=32")]);
        let output_batches = Test::new("coalesce_string_view_many_small_boundary")
            .with_batches(std::iter::repeat_n(batch, 20))
            .with_batch_size(900)
            .with_expected_output_sizes(vec![900, 900, 200])
            .run();

        // expect each buffer to be entirely full except the last one
        expect_buffer_layout(
            col_as_string_view("c0", output_batches.first().unwrap()),
            vec![
                ExpectedLayout {
                    len: 8192,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 16384,
                    capacity: 16384,
                },
                ExpectedLayout {
                    len: 4224,
                    capacity: 32768,
                },
            ],
        );
    }

    #[test]
    fn test_string_view_large_small() {
        // The strings are 37 bytes long, so each batch has 100 * 28 = 2800 bytes
        let mixed_batch = stringview_batch_repeated(
            200,
            [Some("This string is 28 bytes long"), Some("small string")],
        );
        // These strings aren't copied, this array has an 8k buffer
        let all_large = stringview_batch_repeated(
            50,
            [Some(
                "This buffer has only large strings in it so there are no buffer copies",
            )],
        );

        let output_batches = Test::new("coalesce_string_view_large_small")
            // First allocated buffer is 8kb.
            // Appending five batches of 2800 bytes will use 2800 * 10 = 28kb (8kb, an 16kb and 32kbkb)
            .with_batch(mixed_batch.clone())
            .with_batch(mixed_batch.clone())
            .with_batch(all_large.clone())
            .with_batch(mixed_batch.clone())
            .with_batch(all_large.clone())
            .with_batch(mixed_batch.clone())
            .with_batch(mixed_batch.clone())
            .with_batch(all_large.clone())
            .with_batch(mixed_batch.clone())
            .with_batch(all_large.clone())
            .with_batch_size(8000)
            .with_expected_output_sizes(vec![1400])
            .run();

        expect_buffer_layout(
            col_as_string_view("c0", output_batches.first().unwrap()),
            vec![
                ExpectedLayout {
                    len: 8190,
                    capacity: 8192,
                },
                ExpectedLayout {
                    len: 16366,
                    capacity: 16384,
                },
                ExpectedLayout {
                    len: 6244,
                    capacity: 32768,
                },
            ],
        );
    }

    #[test]
    fn test_binary_view() {
        let values: Vec<Option<&[u8]>> = vec![
            Some(b"foo"),
            None,
            Some(b"A longer string that is more than 12 bytes"),
        ];

        let binary_view =
            BinaryViewArray::from_iter(std::iter::repeat(values.iter()).flatten().take(1000));
        let batch =
            RecordBatch::try_from_iter(vec![("c0", Arc::new(binary_view) as ArrayRef)]).unwrap();

        Test::new("coalesce_binary_view")
            .with_batch(batch.clone())
            .with_batch(batch.clone())
            .with_batch_size(512)
            .with_expected_output_sizes(vec![512, 512, 512, 464])
            .run();
    }

    #[derive(Debug, Clone, PartialEq)]
    struct ExpectedLayout {
        len: usize,
        capacity: usize,
    }

    /// Asserts that the buffer layout of the specified StringViewArray matches the expected layout
    fn expect_buffer_layout(array: &StringViewArray, expected: Vec<ExpectedLayout>) {
        let actual = array
            .data_buffers()
            .iter()
            .map(|b| ExpectedLayout {
                len: b.len(),
                capacity: b.capacity(),
            })
            .collect::<Vec<_>>();

        assert_eq!(
            actual, expected,
            "Expected buffer layout {expected:#?} but got {actual:#?}"
        );
    }

    /// Test for [`BatchCoalescer`]
    ///
    /// Pushes the input batches to the coalescer and verifies that the resulting
    /// batches have the
    /// 1. expected number of rows
    /// 2. The same results when the batches are filtered using the filter kernel
    #[derive(Debug, Clone)]
    struct Test {
        /// A human readable name to assist in debugging
        name: String,
        /// Batches to feed to the coalescer.
        input_batches: Vec<RecordBatch>,
        /// Filters to apply to the corresponding input batches.
        ///
        /// If there are no filters for the input batches, the batch will be
        /// pushed as is.
        filters: Vec<BooleanArray>,
        /// The schema. If not provided, the first batch's schema is used.
        schema: Option<SchemaRef>,
        /// Expected output sizes of the resulting batches
        expected_output_sizes: Vec<usize>,
        /// target batch size (default to 1024)
        target_batch_size: usize,
    }

    impl Default for Test {
        fn default() -> Self {
            Self {
                name: "".to_string(),
                input_batches: vec![],
                filters: vec![],
                schema: None,
                expected_output_sizes: vec![],
                target_batch_size: 1024,
            }
        }
    }

    impl Test {
        fn new(name: impl Into<String>) -> Self {
            Self {
                name: name.into(),
                ..Self::default()
            }
        }

        /// Append the description to the test name
        fn with_description(mut self, description: &str) -> Self {
            self.name.push_str(": ");
            self.name.push_str(description);
            self
        }

        /// Set the target batch size
        fn with_batch_size(mut self, target_batch_size: usize) -> Self {
            self.target_batch_size = target_batch_size;
            self
        }

        /// Extend the input batches with `batch`
        fn with_batch(mut self, batch: RecordBatch) -> Self {
            self.input_batches.push(batch);
            self
        }

        /// Extend the filters with `filter`
        fn with_filter(mut self, filter: BooleanArray) -> Self {
            self.filters.push(filter);
            self
        }

        /// Replaces the input batches with `batches`
        fn with_batches(mut self, batches: impl IntoIterator<Item = RecordBatch>) -> Self {
            self.input_batches = batches.into_iter().collect();
            self
        }

        /// Specifies the schema for the test
        fn with_schema(mut self, schema: SchemaRef) -> Self {
            self.schema = Some(schema);
            self
        }

        /// Extends `sizes` to expected output sizes
        fn with_expected_output_sizes(mut self, sizes: impl IntoIterator<Item = usize>) -> Self {
            self.expected_output_sizes.extend(sizes);
            self
        }

        /// Runs the test -- see documentation on [`Test`] for details
        ///
        /// Returns the resulting output batches
        fn run(self) -> Vec<RecordBatch> {
            // Test several permutations of input batches:
            // 1. Removing nulls from some batches (test non-null fast paths)
            // 2. Empty batches
            // 3. One column (from the batch)
            let mut extra_tests = vec![];
            extra_tests.push(self.clone().make_half_non_nullable());
            extra_tests.push(self.clone().insert_empty_batches());
            let single_column_tests = self.make_single_column_tests();
            for test in single_column_tests {
                extra_tests.push(test.clone().make_half_non_nullable());
                extra_tests.push(test);
            }

            // Run original test case first, so any obvious errors are caught
            // by an easier to understand test case
            let results = self.run_inner();
            // Run the extra cases to expand coverage
            for extra in extra_tests {
                extra.run_inner();
            }

            results
        }

        /// Runs the current test instance
        fn run_inner(self) -> Vec<RecordBatch> {
            let expected_output = self.expected_output();
            let schema = self.schema();

            let Self {
                name,
                input_batches,
                filters,
                schema: _,
                target_batch_size,
                expected_output_sizes,
            } = self;

            println!("Running test '{name}'");

            let had_input = input_batches.iter().any(|b| b.num_rows() > 0);

            let mut coalescer = BatchCoalescer::new(Arc::clone(&schema), target_batch_size);

            // feed input batches and filters to the coalescer
            let mut filters = filters.into_iter();
            for batch in input_batches {
                if let Some(filter) = filters.next() {
                    coalescer.push_batch_with_filter(batch, &filter).unwrap();
                } else {
                    coalescer.push_batch(batch).unwrap();
                }
            }
            assert_eq!(schema, coalescer.schema());

            if had_input {
                assert!(!coalescer.is_empty(), "Coalescer should not be empty");
            } else {
                assert!(coalescer.is_empty(), "Coalescer should be empty");
            }

            coalescer.finish_buffered_batch().unwrap();
            if had_input {
                assert!(
                    coalescer.has_completed_batch(),
                    "Coalescer should have completed batches"
                );
            }

            let mut output_batches = vec![];
            while let Some(batch) = coalescer.next_completed_batch() {
                output_batches.push(batch);
            }

            // make sure we got the expected number of output batches and content
            let mut starting_idx = 0;
            let actual_output_sizes: Vec<usize> =
                output_batches.iter().map(|b| b.num_rows()).collect();
            assert_eq!(
                expected_output_sizes, actual_output_sizes,
                "Unexpected number of rows in output batches\n\
                Expected\n{expected_output_sizes:#?}\nActual:{actual_output_sizes:#?}"
            );
            let iter = expected_output_sizes
                .iter()
                .zip(output_batches.iter())
                .enumerate();

            // Verify that the actual contents of each output batch matches the expected output
            for (i, (expected_size, batch)) in iter {
                // compare the contents of the batch after normalization (using
                // `==` compares the underlying memory layout too)
                let expected_batch = expected_output.slice(starting_idx, *expected_size);
                let expected_batch = normalize_batch(expected_batch);
                let batch = normalize_batch(batch.clone());
                assert_eq!(
                    expected_batch, batch,
                    "Unexpected content in batch {i}:\
                    \n\nExpected:\n{expected_batch:#?}\n\nActual:\n{batch:#?}"
                );
                starting_idx += *expected_size;
            }
            output_batches
        }

        /// Return the expected output schema. If not overridden by `with_schema`, it
        /// returns the schema of the first input batch.
        fn schema(&self) -> SchemaRef {
            self.schema
                .clone()
                .unwrap_or_else(|| Arc::clone(&self.input_batches[0].schema()))
        }

        /// Returns the expected output as a single `RecordBatch`
        fn expected_output(&self) -> RecordBatch {
            let schema = self.schema();
            if self.filters.is_empty() {
                return concat_batches(&schema, &self.input_batches).unwrap();
            }

            let mut filters = self.filters.iter();
            let filtered_batches = self
                .input_batches
                .iter()
                .map(|batch| {
                    if let Some(filter) = filters.next() {
                        filter_record_batch(batch, filter).unwrap()
                    } else {
                        batch.clone()
                    }
                })
                .collect::<Vec<_>>();
            concat_batches(&schema, &filtered_batches).unwrap()
        }

        /// Return a copy of self where every other batch has had its nulls removed
        /// (there are often fast paths that are used when there are no nulls)
        fn make_half_non_nullable(mut self) -> Self {
            // remove the nulls from every other batch
            self.input_batches = self
                .input_batches
                .iter()
                .enumerate()
                .map(|(i, batch)| {
                    if i % 2 == 1 {
                        batch.clone()
                    } else {
                        Self::remove_nulls_from_batch(batch)
                    }
                })
                .collect();
            self.with_description("non-nullable")
        }

        /// Insert several empty batches into the input before each existing input
        fn insert_empty_batches(mut self) -> Self {
            let empty_batch = RecordBatch::new_empty(self.schema());
            self.input_batches = self
                .input_batches
                .into_iter()
                .flat_map(|batch| [empty_batch.clone(), batch])
                .collect();
            let empty_filters = BooleanArray::builder(0).finish();
            self.filters = self
                .filters
                .into_iter()
                .flat_map(|filter| [empty_filters.clone(), filter])
                .collect();
            self.with_description("empty batches inserted")
        }

        /// Sets one batch to be non-nullable by removing nulls from all columns
        fn remove_nulls_from_batch(batch: &RecordBatch) -> RecordBatch {
            let new_columns = batch
                .columns()
                .iter()
                .map(Self::remove_nulls_from_array)
                .collect::<Vec<_>>();
            let options = RecordBatchOptions::new().with_row_count(Some(batch.num_rows()));
            RecordBatch::try_new_with_options(batch.schema(), new_columns, &options).unwrap()
        }

        fn remove_nulls_from_array(array: &ArrayRef) -> ArrayRef {
            make_array(array.to_data().into_builder().nulls(None).build().unwrap())
        }

        /// Returns a set of tests where each test that is the sae as self, but
        /// has a single column from the original input batch
        ///
        /// This can be useful to single column optimizations, specifically
        /// filter optimization.
        fn make_single_column_tests(&self) -> Vec<Self> {
            let original_schema = self.schema();
            let mut new_tests = vec![];
            for column in original_schema.fields() {
                let single_column_schema = Arc::new(Schema::new(vec![column.clone()]));

                let single_column_batches = self.input_batches.iter().map(|batch| {
                    let single_column = batch.column_by_name(column.name()).unwrap();
                    RecordBatch::try_new(
                        Arc::clone(&single_column_schema),
                        vec![single_column.clone()],
                    )
                    .unwrap()
                });

                let single_column_test = self
                    .clone()
                    .with_schema(Arc::clone(&single_column_schema))
                    .with_batches(single_column_batches)
                    .with_description("single column")
                    .with_description(column.name());

                new_tests.push(single_column_test);
            }
            new_tests
        }
    }

    /// Return a RecordBatch with a UInt32Array with the specified range and
    /// every third value is null.
    fn uint32_batch<T: std::iter::Iterator<Item = u32>>(range: T) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, true)]));

        let array = UInt32Array::from_iter(range.map(|i| if i % 3 == 0 { None } else { Some(i) }));
        RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap()
    }

    /// Return a RecordBatch with a UInt32Array with no nulls specified range
    fn uint32_batch_non_null<T: std::iter::Iterator<Item = u32>>(range: T) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)]));

        let array = UInt32Array::from_iter_values(range);
        RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap()
    }

    /// Return a RecordBatch with a UInt64Array with no nulls specified range
    fn uint64_batch_non_null<T: std::iter::Iterator<Item = u64>>(range: T) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt64, false)]));

        let array = UInt64Array::from_iter_values(range);
        RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap()
    }

    /// Return a RecordBatch with a StringArrary with values `value0`, `value1`, ...
    /// and every third value is `None`.
    fn utf8_batch(range: Range<u32>) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new("c0", DataType::Utf8, true)]));

        let array = StringArray::from_iter(range.map(|i| {
            if i % 3 == 0 {
                None
            } else {
                Some(format!("value{i}"))
            }
        }));

        RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap()
    }

    /// Return a RecordBatch with a StringViewArray with (only) the specified values
    fn stringview_batch<'a>(values: impl IntoIterator<Item = Option<&'a str>>) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new(
            "c0",
            DataType::Utf8View,
            false,
        )]));

        let array = StringViewArray::from_iter(values);
        RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap()
    }

    /// Return a RecordBatch with a StringViewArray with num_rows by repeating
    /// values over and over.
    fn stringview_batch_repeated<'a>(
        num_rows: usize,
        values: impl IntoIterator<Item = Option<&'a str>>,
    ) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new(
            "c0",
            DataType::Utf8View,
            true,
        )]));

        // Repeat the values to a total of num_rows
        let values: Vec<_> = values.into_iter().collect();
        let values_iter = std::iter::repeat(values.iter())
            .flatten()
            .cloned()
            .take(num_rows);

        let mut builder = StringViewBuilder::with_capacity(100).with_fixed_block_size(8192);
        for val in values_iter {
            builder.append_option(val);
        }

        let array = builder.finish();
        RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap()
    }

    /// Return a RecordBatch of 100 rows
    fn multi_column_batch(range: Range<i32>) -> RecordBatch {
        let int64_array = Int64Array::from_iter(
            range
                .clone()
                .map(|v| if v % 5 == 0 { None } else { Some(v as i64) }),
        );
        let string_view_array = StringViewArray::from_iter(range.clone().map(|v| {
            if v % 5 == 0 {
                None
            } else if v % 7 == 0 {
                Some(format!("This is a string longer than 12 bytes{v}"))
            } else {
                Some(format!("Short {v}"))
            }
        }));
        let string_array = StringArray::from_iter(range.clone().map(|v| {
            if v % 11 == 0 {
                None
            } else {
                Some(format!("Value {v}"))
            }
        }));
        let timestamp_array = TimestampNanosecondArray::from_iter(range.map(|v| {
            if v % 3 == 0 {
                None
            } else {
                Some(v as i64 * 1000) // simulate a timestamp in milliseconds
            }
        }))
        .with_timezone("America/New_York");

        RecordBatch::try_from_iter(vec![
            ("int64", Arc::new(int64_array) as ArrayRef),
            ("stringview", Arc::new(string_view_array) as ArrayRef),
            ("string", Arc::new(string_array) as ArrayRef),
            ("timestamp", Arc::new(timestamp_array) as ArrayRef),
        ])
        .unwrap()
    }

    /// Return a boolean array that filters out randomly selected rows
    /// from the input batch with a `selectivity`.
    ///
    /// For example a `selectivity` of 0.1 will filter out
    /// 90% of the rows.
    #[derive(Debug)]
    struct RandomFilterBuilder {
        /// Number of rows to add to each filter
        num_rows: usize,
        /// selectivity of the filter (between 0.0 and 1.0)
        /// 0 selects no rows, 1.0 selects all rows
        selectivity: f64,
        /// seed for random number generator, increases by one each time
        /// `next_filter` is called
        seed: u64,
    }
    impl RandomFilterBuilder {
        /// Build the next filter with the current seed and increment the seed
        /// by one.
        fn next_filter(&mut self) -> BooleanArray {
            assert!(self.selectivity >= 0.0 && self.selectivity <= 1.0);
            let mut rng = rand::rngs::StdRng::seed_from_u64(self.seed);
            self.seed += 1;
            BooleanArray::from_iter(
                (0..self.num_rows)
                    .map(|_| rng.random_bool(self.selectivity))
                    .map(Some),
            )
        }
    }

    /// Returns the named column as a StringViewArray
    fn col_as_string_view<'b>(name: &str, batch: &'b RecordBatch) -> &'b StringViewArray {
        batch
            .column_by_name(name)
            .expect("column not found")
            .as_string_view_opt()
            .expect("column is not a string view")
    }

    /// Normalize the `RecordBatch` so that the memory layout is consistent
    /// (e.g. StringArray is compacted).
    fn normalize_batch(batch: RecordBatch) -> RecordBatch {
        // Only need to normalize StringViews (as == also tests for memory layout)
        let (schema, mut columns, row_count) = batch.into_parts();

        for column in columns.iter_mut() {
            let Some(string_view) = column.as_string_view_opt() else {
                continue;
            };

            // Re-create the StringViewArray to ensure memory layout is
            // consistent
            let mut builder = StringViewBuilder::new();
            for s in string_view.iter() {
                builder.append_option(s);
            }
            // Update the column with the new StringViewArray
            *column = Arc::new(builder.finish());
        }

        let options = RecordBatchOptions::new().with_row_count(Some(row_count));
        RecordBatch::try_new_with_options(schema, columns, &options).unwrap()
    }

    /// Helper function to create a test batch with specified number of rows
    fn create_test_batch(num_rows: usize) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)]));
        let array = Int32Array::from_iter_values(0..num_rows as i32);
        RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap()
    }
    #[test]
    fn test_biggest_coalesce_batch_size_none_default() {
        // Test that default behavior (None) coalesces all batches
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );

        // Push a large batch (1000 rows) - should be coalesced normally
        let large_batch = create_test_batch(1000);
        coalescer.push_batch(large_batch).unwrap();

        // Should produce multiple batches of target size (100)
        let mut output_batches = vec![];
        while let Some(batch) = coalescer.next_completed_batch() {
            output_batches.push(batch);
        }

        coalescer.finish_buffered_batch().unwrap();
        while let Some(batch) = coalescer.next_completed_batch() {
            output_batches.push(batch);
        }

        // Should have 10 batches of 100 rows each
        assert_eq!(output_batches.len(), 10);
        for batch in output_batches {
            assert_eq!(batch.num_rows(), 100);
        }
    }

    #[test]
    fn test_biggest_coalesce_batch_size_bypass_large_batch() {
        // Test that batches larger than biggest_coalesce_batch_size bypass coalescing
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(500));

        // Push a large batch (1000 rows) - should bypass coalescing
        let large_batch = create_test_batch(1000);
        coalescer.push_batch(large_batch.clone()).unwrap();

        // Should have one completed batch immediately (the original large batch)
        assert!(coalescer.has_completed_batch());
        let output_batch = coalescer.next_completed_batch().unwrap();
        assert_eq!(output_batch.num_rows(), 1000);

        // Should be no more completed batches
        assert!(!coalescer.has_completed_batch());
        assert_eq!(coalescer.get_buffered_rows(), 0);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_coalesce_small_batch() {
        // Test that batches smaller than biggest_coalesce_batch_size are coalesced normally
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        )
        .with_biggest_coalesce_batch_size(Some(500));

        // Push small batches that should be coalesced
        let small_batch = create_test_batch(50);
        coalescer.push_batch(small_batch.clone()).unwrap();

        // Should not have completed batch yet (only 50 rows, target is 100)
        assert!(!coalescer.has_completed_batch());
        assert_eq!(coalescer.get_buffered_rows(), 50);

        // Push another small batch
        coalescer.push_batch(small_batch).unwrap();

        // Now should have a completed batch (100 rows total)
        assert!(coalescer.has_completed_batch());
        let output_batch = coalescer.next_completed_batch().unwrap();
        let size = output_batch
            .column(0)
            .as_primitive::<Int32Type>()
            .get_buffer_memory_size();
        assert_eq!(size, 400); // 100 rows * 4 bytes each
        assert_eq!(output_batch.num_rows(), 100);

        assert_eq!(coalescer.get_buffered_rows(), 0);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_equal_boundary() {
        // Test behavior when batch size equals biggest_coalesce_batch_size
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(500));

        // Push a batch exactly equal to the limit
        let boundary_batch = create_test_batch(500);
        coalescer.push_batch(boundary_batch).unwrap();

        // Should be coalesced (not bypass) since it's equal, not greater
        let mut output_count = 0;
        while coalescer.next_completed_batch().is_some() {
            output_count += 1;
        }

        coalescer.finish_buffered_batch().unwrap();
        while coalescer.next_completed_batch().is_some() {
            output_count += 1;
        }

        // Should have 5 batches of 100 rows each
        assert_eq!(output_count, 5);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_first_large_then_consecutive_bypass() {
        // Test the new consecutive large batch bypass behavior
        // Pattern: small batches -> first large batch (coalesced) -> consecutive large batches (bypass)
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(200));

        let small_batch = create_test_batch(50);

        // Push small batch first to create buffered data
        coalescer.push_batch(small_batch).unwrap();
        assert_eq!(coalescer.get_buffered_rows(), 50);
        assert!(!coalescer.has_completed_batch());

        // Push first large batch - should go through normal coalescing due to buffered data
        let large_batch1 = create_test_batch(250);
        coalescer.push_batch(large_batch1).unwrap();

        // 50 + 250 = 300 -> 3 complete batches of 100, 0 rows buffered
        let mut completed_batches = vec![];
        while let Some(batch) = coalescer.next_completed_batch() {
            completed_batches.push(batch);
        }
        assert_eq!(completed_batches.len(), 3);
        assert_eq!(coalescer.get_buffered_rows(), 0);

        // Now push consecutive large batches - they should bypass
        let large_batch2 = create_test_batch(300);
        let large_batch3 = create_test_batch(400);

        // Push second large batch - should bypass since it's consecutive and buffer is empty
        coalescer.push_batch(large_batch2).unwrap();
        assert!(coalescer.has_completed_batch());
        let output = coalescer.next_completed_batch().unwrap();
        assert_eq!(output.num_rows(), 300); // bypassed with original size
        assert_eq!(coalescer.get_buffered_rows(), 0);

        // Push third large batch - should also bypass
        coalescer.push_batch(large_batch3).unwrap();
        assert!(coalescer.has_completed_batch());
        let output = coalescer.next_completed_batch().unwrap();
        assert_eq!(output.num_rows(), 400); // bypassed with original size
        assert_eq!(coalescer.get_buffered_rows(), 0);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_empty_batch() {
        // Test that empty batches don't trigger the bypass logic
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(50));

        let empty_batch = create_test_batch(0);
        coalescer.push_batch(empty_batch).unwrap();

        // Empty batch should be handled normally (no effect)
        assert!(!coalescer.has_completed_batch());
        assert_eq!(coalescer.get_buffered_rows(), 0);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_with_buffered_data_no_bypass() {
        // Test that when there is buffered data, large batches do NOT bypass (unless consecutive)
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(200));

        // Add some buffered data first
        let small_batch = create_test_batch(30);
        coalescer.push_batch(small_batch.clone()).unwrap();
        coalescer.push_batch(small_batch).unwrap();
        assert_eq!(coalescer.get_buffered_rows(), 60);

        // Push large batch that would normally bypass, but shouldn't because buffered_rows > 0
        let large_batch = create_test_batch(250);
        coalescer.push_batch(large_batch).unwrap();

        // The large batch should be processed through normal coalescing logic
        // Total: 60 (buffered) + 250 (new) = 310 rows
        // Output: 3 complete batches of 100 rows each, 10 rows remain buffered

        let mut completed_batches = vec![];
        while let Some(batch) = coalescer.next_completed_batch() {
            completed_batches.push(batch);
        }

        assert_eq!(completed_batches.len(), 3);
        for batch in &completed_batches {
            assert_eq!(batch.num_rows(), 100);
        }
        assert_eq!(coalescer.get_buffered_rows(), 10);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_zero_limit() {
        // Test edge case where limit is 0 (all batches bypass when no buffered data)
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(0));

        // Even a 1-row batch should bypass when there's no buffered data
        let tiny_batch = create_test_batch(1);
        coalescer.push_batch(tiny_batch).unwrap();

        assert!(coalescer.has_completed_batch());
        let output = coalescer.next_completed_batch().unwrap();
        assert_eq!(output.num_rows(), 1);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_bypass_only_when_no_buffer() {
        // Test that bypass only occurs when buffered_rows == 0
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(200));

        // First, push a large batch with no buffered data - should bypass
        let large_batch = create_test_batch(300);
        coalescer.push_batch(large_batch.clone()).unwrap();

        assert!(coalescer.has_completed_batch());
        let output = coalescer.next_completed_batch().unwrap();
        assert_eq!(output.num_rows(), 300); // bypassed
        assert_eq!(coalescer.get_buffered_rows(), 0);

        // Now add some buffered data
        let small_batch = create_test_batch(50);
        coalescer.push_batch(small_batch).unwrap();
        assert_eq!(coalescer.get_buffered_rows(), 50);

        // Push the same large batch again - should NOT bypass this time (not consecutive)
        coalescer.push_batch(large_batch).unwrap();

        // Should process through normal coalescing: 50 + 300 = 350 rows
        // Output: 3 complete batches of 100 rows, 50 rows buffered
        let mut completed_batches = vec![];
        while let Some(batch) = coalescer.next_completed_batch() {
            completed_batches.push(batch);
        }

        assert_eq!(completed_batches.len(), 3);
        for batch in &completed_batches {
            assert_eq!(batch.num_rows(), 100);
        }
        assert_eq!(coalescer.get_buffered_rows(), 50);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_consecutive_large_batches_scenario() {
        // Test your exact scenario: 20, 20, 30, 700, 600, 700, 900, 700, 600
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            1000,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(500));

        // Push small batches first
        coalescer.push_batch(create_test_batch(20)).unwrap();
        coalescer.push_batch(create_test_batch(20)).unwrap();
        coalescer.push_batch(create_test_batch(30)).unwrap();

        assert_eq!(coalescer.get_buffered_rows(), 70);
        assert!(!coalescer.has_completed_batch());

        // Push first large batch (700) - should coalesce due to buffered data
        coalescer.push_batch(create_test_batch(700)).unwrap();

        // 70 + 700 = 770 rows, not enough for 1000, so all stay buffered
        assert_eq!(coalescer.get_buffered_rows(), 770);
        assert!(!coalescer.has_completed_batch());

        // Push second large batch (600) - should bypass since previous was large
        coalescer.push_batch(create_test_batch(600)).unwrap();

        // Should flush buffer (770 rows) and bypass the 600
        let mut outputs = vec![];
        while let Some(batch) = coalescer.next_completed_batch() {
            outputs.push(batch);
        }
        assert_eq!(outputs.len(), 2); // one flushed buffer batch (770) + one bypassed (600)
        assert_eq!(outputs[0].num_rows(), 770);
        assert_eq!(outputs[1].num_rows(), 600);
        assert_eq!(coalescer.get_buffered_rows(), 0);

        // Push remaining large batches - should all bypass
        let remaining_batches = [700, 900, 700, 600];
        for &size in &remaining_batches {
            coalescer.push_batch(create_test_batch(size)).unwrap();

            assert!(coalescer.has_completed_batch());
            let output = coalescer.next_completed_batch().unwrap();
            assert_eq!(output.num_rows(), size);
            assert_eq!(coalescer.get_buffered_rows(), 0);
        }
    }

    #[test]
    fn test_biggest_coalesce_batch_size_truly_consecutive_large_bypass() {
        // Test truly consecutive large batches that should all bypass
        // This test ensures buffer is completely empty between large batches
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(200));

        // Push consecutive large batches with no prior buffered data
        let large_batches = vec![
            create_test_batch(300),
            create_test_batch(400),
            create_test_batch(350),
            create_test_batch(500),
        ];

        let mut all_outputs = vec![];

        for (i, large_batch) in large_batches.into_iter().enumerate() {
            let expected_size = large_batch.num_rows();

            // Buffer should be empty before each large batch
            assert_eq!(
                coalescer.get_buffered_rows(),
                0,
                "Buffer should be empty before batch {}",
                i
            );

            coalescer.push_batch(large_batch).unwrap();

            // Each large batch should bypass and produce exactly one output batch
            assert!(
                coalescer.has_completed_batch(),
                "Should have completed batch after pushing batch {}",
                i
            );

            let output = coalescer.next_completed_batch().unwrap();
            assert_eq!(
                output.num_rows(),
                expected_size,
                "Batch {} should have bypassed with original size",
                i
            );

            // Should be no more batches and buffer should be empty
            assert!(
                !coalescer.has_completed_batch(),
                "Should have no more completed batches after batch {}",
                i
            );
            assert_eq!(
                coalescer.get_buffered_rows(),
                0,
                "Buffer should be empty after batch {}",
                i
            );

            all_outputs.push(output);
        }

        // Verify we got exactly 4 output batches with original sizes
        assert_eq!(all_outputs.len(), 4);
        assert_eq!(all_outputs[0].num_rows(), 300);
        assert_eq!(all_outputs[1].num_rows(), 400);
        assert_eq!(all_outputs[2].num_rows(), 350);
        assert_eq!(all_outputs[3].num_rows(), 500);
    }

    #[test]
    fn test_biggest_coalesce_batch_size_reset_consecutive_on_small_batch() {
        // Test that small batches reset the consecutive large batch tracking
        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::Int32, false)])),
            100,
        );
        coalescer.set_biggest_coalesce_batch_size(Some(200));

        // Push first large batch - should bypass (no buffered data)
        coalescer.push_batch(create_test_batch(300)).unwrap();
        let output = coalescer.next_completed_batch().unwrap();
        assert_eq!(output.num_rows(), 300);

        // Push second large batch - should bypass (consecutive)
        coalescer.push_batch(create_test_batch(400)).unwrap();
        let output = coalescer.next_completed_batch().unwrap();
        assert_eq!(output.num_rows(), 400);

        // Push small batch - resets consecutive tracking
        coalescer.push_batch(create_test_batch(50)).unwrap();
        assert_eq!(coalescer.get_buffered_rows(), 50);

        // Push large batch again - should NOT bypass due to buffered data
        coalescer.push_batch(create_test_batch(350)).unwrap();

        // Should coalesce: 50 + 350 = 400 -> 4 complete batches of 100
        let mut outputs = vec![];
        while let Some(batch) = coalescer.next_completed_batch() {
            outputs.push(batch);
        }
        assert_eq!(outputs.len(), 4);
        for batch in outputs {
            assert_eq!(batch.num_rows(), 100);
        }
        assert_eq!(coalescer.get_buffered_rows(), 0);
    }

    #[test]
    fn test_coalasce_push_batch_with_indices() {
        const MID_POINT: u32 = 2333;
        const TOTAL_ROWS: u32 = 23333;
        let batch1 = uint32_batch_non_null(0..MID_POINT);
        let batch2 = uint32_batch_non_null((MID_POINT..TOTAL_ROWS).rev());

        let mut coalescer = BatchCoalescer::new(
            Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])),
            TOTAL_ROWS as usize,
        );
        coalescer.push_batch(batch1).unwrap();

        let rev_indices = (0..((TOTAL_ROWS - MID_POINT) as u64)).rev();
        let reversed_indices_batch = uint64_batch_non_null(rev_indices);

        let reverse_indices = UInt64Array::from(reversed_indices_batch.column(0).to_data());
        coalescer
            .push_batch_with_indices(batch2, &reverse_indices)
            .unwrap();

        coalescer.finish_buffered_batch().unwrap();
        let actual = coalescer.next_completed_batch().unwrap();

        let expected = uint32_batch_non_null(0..TOTAL_ROWS);

        assert_eq!(expected, actual);
    }

    #[test]
    fn test_push_batch_schema_mismatch_fewer_columns() {
        // Coalescer expects 0 columns, batch has 1
        let empty_schema = Arc::new(Schema::empty());
        let mut coalescer = BatchCoalescer::new(empty_schema, 100);
        let batch = uint32_batch(0..5);
        let result = coalescer.push_batch(batch);
        assert!(result.is_err());
        let err = result.unwrap_err().to_string();
        assert!(
            err.contains("Batch has 1 columns but BatchCoalescer expects 0"),
            "unexpected error: {err}"
        );
    }

    #[test]
    fn test_push_batch_schema_mismatch_more_columns() {
        // Coalescer expects 2 columns, batch has 1
        let schema = Arc::new(Schema::new(vec![
            Field::new("c0", DataType::UInt32, false),
            Field::new("c1", DataType::UInt32, false),
        ]));
        let mut coalescer = BatchCoalescer::new(schema, 100);
        let batch = uint32_batch(0..5);
        let result = coalescer.push_batch(batch);
        assert!(result.is_err());
        let err = result.unwrap_err().to_string();
        assert!(
            err.contains("Batch has 1 columns but BatchCoalescer expects 2"),
            "unexpected error: {err}"
        );
    }

    #[test]
    fn test_push_batch_schema_mismatch_two_vs_zero() {
        // Coalescer expects 0 columns, batch has 2
        let empty_schema = Arc::new(Schema::empty());
        let mut coalescer = BatchCoalescer::new(empty_schema, 100);
        let schema = Arc::new(Schema::new(vec![
            Field::new("c0", DataType::UInt32, false),
            Field::new("c1", DataType::UInt32, false),
        ]));
        let batch = RecordBatch::try_new(
            schema,
            vec![
                Arc::new(UInt32Array::from(vec![1, 2, 3])),
                Arc::new(UInt32Array::from(vec![4, 5, 6])),
            ],
        )
        .unwrap();
        let result = coalescer.push_batch(batch);
        assert!(result.is_err());
        let err = result.unwrap_err().to_string();
        assert!(
            err.contains("Batch has 2 columns but BatchCoalescer expects 0"),
            "unexpected error: {err}"
        );
    }
}