-
Notifications
You must be signed in to change notification settings - Fork 1.2k
Expand file tree
/
Copy pathcoalesce.rs
More file actions
2459 lines (2187 loc) · 94.8 KB
/
coalesce.rs
File metadata and controls
2459 lines (2187 loc) · 94.8 KB
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
// 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::{
FilterBuilder, FilterPredicate, IndexIterator, IterationStrategy, SlicesIterator,
filter_record_batch,
};
use crate::take::take_record_batch;
use arrow_array::cast::AsArray;
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> {
if supports_fused_inline_binary_view_filter(&batch) {
if filter.len() > batch.num_rows() {
return Err(ArrowError::InvalidArgumentError(format!(
"Filter predicate of length {} is larger than target array of length {}",
filter.len(),
batch.num_rows()
)));
}
let mut filter_builder = FilterBuilder::new(filter);
if batch.num_columns() > 1 {
filter_builder = filter_builder.optimize();
}
let predicate = filter_builder.build();
let selected_count = predicate.count();
if selected_count == 0 {
return Ok(());
}
if selected_count == batch.num_rows() && filter.len() == batch.num_rows() {
return self.push_batch(batch);
}
if let Some(limit) = self.biggest_coalesce_batch_size {
if selected_count > limit {
let filtered_batch = predicate.filter_record_batch(&batch)?;
return self.push_batch(filtered_batch);
}
}
// For dense inline filters, the existing filter kernel remains faster.
if selected_count.saturating_mul(4) > filter.len() {
let filtered_batch = predicate.filter_record_batch(&batch)?;
return self.push_batch(filtered_batch);
}
let space_in_batch = self.target_batch_size - self.buffered_rows;
if selected_count <= space_in_batch {
let (_schema, arrays, _num_rows) = batch.into_parts();
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));
});
let result = (|| {
for in_progress in self.in_progress_arrays.iter_mut() {
in_progress.copy_rows_by_filter(&predicate)?;
}
self.buffered_rows += selected_count;
if self.buffered_rows >= self.target_batch_size {
self.finish_buffered_batch()?;
}
Ok(())
})();
for in_progress in self.in_progress_arrays.iter_mut() {
in_progress.set_source(None);
}
return result;
}
}
// 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()),
}
}
fn supports_fused_inline_binary_view_filter(batch: &RecordBatch) -> bool {
let mut has_inline_binary_view = false;
let supported = batch
.schema()
.fields()
.iter()
.zip(batch.columns())
.all(|(field, array)| {
if field.data_type().is_primitive() {
return true;
}
let Some(binary_view) = array.as_binary_view_opt() else {
return false;
};
let inline = binary_view.data_buffers().is_empty();
has_inline_binary_view |= inline;
inline
});
supported && has_inline_binary_view
}
/// 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>;
/// Copy rows selected by `filter` from the current source array.
fn copy_rows_by_filter(&mut self, filter: &FilterPredicate) -> Result<(), ArrowError> {
match filter.strategy() {
IterationStrategy::None => Ok(()),
IterationStrategy::All => self.copy_rows(0, filter.count()),
IterationStrategy::Slices(slices) => {
for &(start, end) in slices {
self.copy_rows(start, end - start)?;
}
Ok(())
}
IterationStrategy::SlicesIterator => {
for (start, end) in SlicesIterator::new(filter.filter_array()) {
self.copy_rows(start, end - start)?;
}
Ok(())
}
IterationStrategy::Indices(indices) => self.copy_rows_by_indices(indices),
IterationStrategy::IndexIterator => {
let indices = IndexIterator::new(filter.filter_array(), filter.count()).collect();
self.copy_rows_by_indices(&indices)
}
}
}
/// Copy rows at the specified indices from the current source array.
fn copy_rows_by_indices(&mut self, indices: &[usize]) -> Result<(), ArrowError> {
for &idx in indices {
self.copy_rows(idx, 1)?;
}
Ok(())
}
/// 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]