15
15
// specific language governing permissions and limitations
16
16
// under the License.
17
17
18
- //! CoalesceBatchesExec combines small batches into larger batches for more efficient use of
19
- //! vectorized processing by upstream operators.
18
+ //! [`CoalesceBatchesExec`] combines small batches into larger batches.
20
19
21
20
use std:: any:: Any ;
22
21
use std:: pin:: Pin ;
23
22
use std:: sync:: Arc ;
24
- use std:: task:: { Context , Poll } ;
23
+ use std:: task:: { ready , Context , Poll } ;
25
24
26
25
use super :: metrics:: { BaselineMetrics , ExecutionPlanMetricsSet , MetricsSet } ;
27
26
use super :: { DisplayAs , ExecutionPlanProperties , PlanProperties , Statistics } ;
@@ -38,8 +37,35 @@ use datafusion_execution::TaskContext;
38
37
use futures:: stream:: { Stream , StreamExt } ;
39
38
use log:: trace;
40
39
41
- /// CoalesceBatchesExec combines small batches into larger batches for more efficient use of
42
- /// vectorized processing by upstream operators.
40
+ /// `CoalesceBatchesExec` combines small batches into larger batches for more
41
+ /// efficient use of vectorized processing by upstream operators.
42
+ ///
43
+ /// Generally speaking, larger RecordBatches are more efficient to process than
44
+ /// smaller record batches (until the CPU cache is exceeded) because there is
45
+ /// fixed processing overhead per batch. This code concatenates multiple small
46
+ /// record batches into larger ones to amortize this overhead.
47
+ ///
48
+ /// ```text
49
+ /// ┌────────────────────┐
50
+ /// │ RecordBatch │
51
+ /// │ num_rows = 23 │
52
+ /// └────────────────────┘ ┌────────────────────┐
53
+ /// │ │
54
+ /// ┌────────────────────┐ Coalesce │ │
55
+ /// │ │ Batches │ │
56
+ /// │ RecordBatch │ │ │
57
+ /// │ num_rows = 50 │ ─ ─ ─ ─ ─ ─ ▶ │ │
58
+ /// │ │ │ RecordBatch │
59
+ /// │ │ │ num_rows = 106 │
60
+ /// └────────────────────┘ │ │
61
+ /// │ │
62
+ /// ┌────────────────────┐ │ │
63
+ /// │ │ │ │
64
+ /// │ RecordBatch │ │ │
65
+ /// │ num_rows = 33 │ └────────────────────┘
66
+ /// │ │
67
+ /// └────────────────────┘
68
+ /// ```
43
69
#[ derive( Debug ) ]
44
70
pub struct CoalesceBatchesExec {
45
71
/// The input plan
@@ -146,10 +172,7 @@ impl ExecutionPlan for CoalesceBatchesExec {
146
172
) -> Result < SendableRecordBatchStream > {
147
173
Ok ( Box :: pin ( CoalesceBatchesStream {
148
174
input : self . input . execute ( partition, context) ?,
149
- schema : self . input . schema ( ) ,
150
- target_batch_size : self . target_batch_size ,
151
- buffer : Vec :: new ( ) ,
152
- buffered_rows : 0 ,
175
+ coalescer : BatchCoalescer :: new ( self . input . schema ( ) , self . target_batch_size ) ,
153
176
is_closed : false ,
154
177
baseline_metrics : BaselineMetrics :: new ( & self . metrics , partition) ,
155
178
} ) )
@@ -164,17 +187,12 @@ impl ExecutionPlan for CoalesceBatchesExec {
164
187
}
165
188
}
166
189
190
+ /// Stream for [`CoalesceBatchesExec`]. See [`CoalesceBatchesExec`] for more details.
167
191
struct CoalesceBatchesStream {
168
192
/// The input plan
169
193
input : SendableRecordBatchStream ,
170
- /// The input schema
171
- schema : SchemaRef ,
172
- /// Minimum number of rows for coalesces batches
173
- target_batch_size : usize ,
174
- /// Buffered batches
175
- buffer : Vec < RecordBatch > ,
176
- /// Buffered row count
177
- buffered_rows : usize ,
194
+ /// Buffer for combining batches
195
+ coalescer : BatchCoalescer ,
178
196
/// Whether the stream has finished returning all of its data or not
179
197
is_closed : bool ,
180
198
/// Execution metrics
@@ -213,66 +231,35 @@ impl CoalesceBatchesStream {
213
231
let input_batch = self . input . poll_next_unpin ( cx) ;
214
232
// records time on drop
215
233
let _timer = cloned_time. timer ( ) ;
216
- match input_batch {
217
- Poll :: Ready ( x) => match x {
218
- Some ( Ok ( batch) ) => {
219
- if batch. num_rows ( ) >= self . target_batch_size
220
- && self . buffer . is_empty ( )
221
- {
222
- return Poll :: Ready ( Some ( Ok ( batch) ) ) ;
223
- } else if batch. num_rows ( ) == 0 {
224
- // discard empty batches
225
- } else {
226
- // add to the buffered batches
227
- self . buffered_rows += batch. num_rows ( ) ;
228
- self . buffer . push ( batch) ;
229
- // check to see if we have enough batches yet
230
- if self . buffered_rows >= self . target_batch_size {
231
- // combine the batches and return
232
- let batch = concat_batches (
233
- & self . schema ,
234
- & self . buffer ,
235
- self . buffered_rows ,
236
- ) ?;
237
- // reset buffer state
238
- self . buffer . clear ( ) ;
239
- self . buffered_rows = 0 ;
240
- // return batch
241
- return Poll :: Ready ( Some ( Ok ( batch) ) ) ;
242
- }
243
- }
244
- }
245
- None => {
246
- self . is_closed = true ;
247
- // we have reached the end of the input stream but there could still
248
- // be buffered batches
249
- if self . buffer . is_empty ( ) {
250
- return Poll :: Ready ( None ) ;
251
- } else {
252
- // combine the batches and return
253
- let batch = concat_batches (
254
- & self . schema ,
255
- & self . buffer ,
256
- self . buffered_rows ,
257
- ) ?;
258
- // reset buffer state
259
- self . buffer . clear ( ) ;
260
- self . buffered_rows = 0 ;
261
- // return batch
262
- return Poll :: Ready ( Some ( Ok ( batch) ) ) ;
263
- }
234
+ match ready ! ( input_batch) {
235
+ Some ( result) => {
236
+ let Ok ( input_batch) = result else {
237
+ return Poll :: Ready ( Some ( result) ) ; // pass back error
238
+ } ;
239
+ // Buffer the batch and either get more input if not enough
240
+ // rows yet or output
241
+ match self . coalescer . push_batch ( input_batch) {
242
+ Ok ( None ) => continue ,
243
+ res => return Poll :: Ready ( res. transpose ( ) ) ,
264
244
}
265
- other => return Poll :: Ready ( other) ,
266
- } ,
267
- Poll :: Pending => return Poll :: Pending ,
245
+ }
246
+ None => {
247
+ self . is_closed = true ;
248
+ // we have reached the end of the input stream but there could still
249
+ // be buffered batches
250
+ return match self . coalescer . finish ( ) {
251
+ Ok ( None ) => Poll :: Ready ( None ) ,
252
+ res => Poll :: Ready ( res. transpose ( ) ) ,
253
+ } ;
254
+ }
268
255
}
269
256
}
270
257
}
271
258
}
272
259
273
260
impl RecordBatchStream for CoalesceBatchesStream {
274
261
fn schema ( & self ) -> SchemaRef {
275
- Arc :: clone ( & self . schema )
262
+ self . coalescer . schema ( )
276
263
}
277
264
}
278
265
@@ -290,26 +277,106 @@ pub fn concat_batches(
290
277
arrow:: compute:: concat_batches ( schema, batches)
291
278
}
292
279
280
+ /// Concatenate multiple record batches into larger batches
281
+ ///
282
+ /// See [`CoalesceBatchesExec`] for more details.
283
+ ///
284
+ /// Notes:
285
+ ///
286
+ /// 1. The output rows is the same order as the input rows
287
+ ///
288
+ /// 2. The output is a sequence of batches, with all but the last being at least
289
+ /// `target_batch_size` rows.
290
+ ///
291
+ /// 3. Eventually this may also be able to handle other optimizations such as a
292
+ /// combined filter/coalesce operation.
293
+ #[ derive( Debug ) ]
294
+ struct BatchCoalescer {
295
+ /// The input schema
296
+ schema : SchemaRef ,
297
+ /// Minimum number of rows for coalesces batches
298
+ target_batch_size : usize ,
299
+ /// Buffered batches
300
+ buffer : Vec < RecordBatch > ,
301
+ /// Buffered row count
302
+ buffered_rows : usize ,
303
+ }
304
+
305
+ impl BatchCoalescer {
306
+ /// Create a new BatchCoalescer that produces batches of at least `target_batch_size` rows
307
+ fn new ( schema : SchemaRef , target_batch_size : usize ) -> Self {
308
+ Self {
309
+ schema,
310
+ target_batch_size,
311
+ buffer : vec ! [ ] ,
312
+ buffered_rows : 0 ,
313
+ }
314
+ }
315
+
316
+ /// Return the schema of the output batches
317
+ fn schema ( & self ) -> SchemaRef {
318
+ Arc :: clone ( & self . schema )
319
+ }
320
+
321
+ /// Add a batch to the coalescer, returning a batch if the target batch size is reached
322
+ fn push_batch ( & mut self , batch : RecordBatch ) -> Result < Option < RecordBatch > > {
323
+ if batch. num_rows ( ) >= self . target_batch_size && self . buffer . is_empty ( ) {
324
+ return Ok ( Some ( batch) ) ;
325
+ }
326
+ // discard empty batches
327
+ if batch. num_rows ( ) == 0 {
328
+ return Ok ( None ) ;
329
+ }
330
+ // add to the buffered batches
331
+ self . buffered_rows += batch. num_rows ( ) ;
332
+ self . buffer . push ( batch) ;
333
+ // check to see if we have enough batches yet
334
+ let batch = if self . buffered_rows >= self . target_batch_size {
335
+ // combine the batches and return
336
+ let batch = concat_batches ( & self . schema , & self . buffer , self . buffered_rows ) ?;
337
+ // reset buffer state
338
+ self . buffer . clear ( ) ;
339
+ self . buffered_rows = 0 ;
340
+ // return batch
341
+ Some ( batch)
342
+ } else {
343
+ None
344
+ } ;
345
+ Ok ( batch)
346
+ }
347
+
348
+ /// Finish the coalescing process, returning all buffered data as a final,
349
+ /// single batch, if any
350
+ fn finish ( & mut self ) -> Result < Option < RecordBatch > > {
351
+ if self . buffer . is_empty ( ) {
352
+ Ok ( None )
353
+ } else {
354
+ // combine the batches and return
355
+ let batch = concat_batches ( & self . schema , & self . buffer , self . buffered_rows ) ?;
356
+ // reset buffer state
357
+ self . buffer . clear ( ) ;
358
+ self . buffered_rows = 0 ;
359
+ // return batch
360
+ Ok ( Some ( batch) )
361
+ }
362
+ }
363
+ }
364
+
293
365
#[ cfg( test) ]
294
366
mod tests {
295
367
use super :: * ;
296
- use crate :: { memory:: MemoryExec , repartition:: RepartitionExec , Partitioning } ;
297
-
298
368
use arrow:: datatypes:: { DataType , Field , Schema } ;
299
369
use arrow_array:: UInt32Array ;
300
370
301
371
#[ tokio:: test( flavor = "multi_thread" ) ]
302
372
async fn test_concat_batches ( ) -> Result < ( ) > {
303
- let schema = test_schema ( ) ;
304
- let partition = create_vec_batches ( & schema, 10 ) ;
305
- let partitions = vec ! [ partition] ;
306
-
307
- let output_partitions = coalesce_batches ( & schema, partitions, 21 ) . await ?;
308
- assert_eq ! ( 1 , output_partitions. len( ) ) ;
373
+ let Scenario { schema, batch } = uint32_scenario ( ) ;
309
374
310
375
// input is 10 batches x 8 rows (80 rows)
376
+ let input = std:: iter:: repeat ( batch) . take ( 10 ) ;
377
+
311
378
// expected output is batches of at least 20 rows (except for the final batch)
312
- let batches = & output_partitions [ 0 ] ;
379
+ let batches = do_coalesce_batches ( & schema , input , 21 ) ;
313
380
assert_eq ! ( 4 , batches. len( ) ) ;
314
381
assert_eq ! ( 24 , batches[ 0 ] . num_rows( ) ) ;
315
382
assert_eq ! ( 24 , batches[ 1 ] . num_rows( ) ) ;
@@ -319,54 +386,43 @@ mod tests {
319
386
Ok ( ( ) )
320
387
}
321
388
322
- fn test_schema ( ) -> Arc < Schema > {
323
- Arc :: new ( Schema :: new ( vec ! [ Field :: new( "c0" , DataType :: UInt32 , false ) ] ) )
324
- }
325
-
326
- async fn coalesce_batches (
389
+ // Coalesce the batches with a BatchCoalescer function with the given input
390
+ // and target batch size returning the resulting batches
391
+ fn do_coalesce_batches (
327
392
schema : & SchemaRef ,
328
- input_partitions : Vec < Vec < RecordBatch > > ,
393
+ input : impl IntoIterator < Item = RecordBatch > ,
329
394
target_batch_size : usize ,
330
- ) -> Result < Vec < Vec < RecordBatch > > > {
395
+ ) -> Vec < RecordBatch > {
331
396
// create physical plan
332
- let exec = MemoryExec :: try_new ( & input_partitions, Arc :: clone ( schema) , None ) ?;
333
- let exec =
334
- RepartitionExec :: try_new ( Arc :: new ( exec) , Partitioning :: RoundRobinBatch ( 1 ) ) ?;
335
- let exec: Arc < dyn ExecutionPlan > =
336
- Arc :: new ( CoalesceBatchesExec :: new ( Arc :: new ( exec) , target_batch_size) ) ;
337
-
338
- // execute and collect results
339
- let output_partition_count = exec. output_partitioning ( ) . partition_count ( ) ;
340
- let mut output_partitions = Vec :: with_capacity ( output_partition_count) ;
341
- for i in 0 ..output_partition_count {
342
- // execute this *output* partition and collect all batches
343
- let task_ctx = Arc :: new ( TaskContext :: default ( ) ) ;
344
- let mut stream = exec. execute ( i, Arc :: clone ( & task_ctx) ) ?;
345
- let mut batches = vec ! [ ] ;
346
- while let Some ( result) = stream. next ( ) . await {
347
- batches. push ( result?) ;
348
- }
349
- output_partitions. push ( batches) ;
397
+ let mut coalescer = BatchCoalescer :: new ( Arc :: clone ( schema) , target_batch_size) ;
398
+ let mut output_batches: Vec < _ > = input
399
+ . into_iter ( )
400
+ . filter_map ( |batch| coalescer. push_batch ( batch) . unwrap ( ) )
401
+ . collect ( ) ;
402
+ if let Some ( batch) = coalescer. finish ( ) . unwrap ( ) {
403
+ output_batches. push ( batch) ;
350
404
}
351
- Ok ( output_partitions )
405
+ output_batches
352
406
}
353
407
354
- /// Create vector batches
355
- fn create_vec_batches ( schema : & Schema , n : usize ) -> Vec < RecordBatch > {
356
- let batch = create_batch ( schema) ;
357
- let mut vec = Vec :: with_capacity ( n) ;
358
- for _ in 0 ..n {
359
- vec. push ( batch. clone ( ) ) ;
360
- }
361
- vec
408
+ /// Test scenario
409
+ #[ derive( Debug ) ]
410
+ struct Scenario {
411
+ schema : Arc < Schema > ,
412
+ batch : RecordBatch ,
362
413
}
363
414
364
- /// Create batch
365
- fn create_batch ( schema : & Schema ) -> RecordBatch {
366
- RecordBatch :: try_new (
367
- Arc :: new ( schema. clone ( ) ) ,
415
+ /// a batch of 8 rows of UInt32
416
+ fn uint32_scenario ( ) -> Scenario {
417
+ let schema =
418
+ Arc :: new ( Schema :: new ( vec ! [ Field :: new( "c0" , DataType :: UInt32 , false ) ] ) ) ;
419
+
420
+ let batch = RecordBatch :: try_new (
421
+ Arc :: clone ( & schema) ,
368
422
vec ! [ Arc :: new( UInt32Array :: from( vec![ 1 , 2 , 3 , 4 , 5 , 6 , 7 , 8 ] ) ) ] ,
369
423
)
370
- . unwrap ( )
424
+ . unwrap ( ) ;
425
+
426
+ Scenario { schema, batch }
371
427
}
372
428
}
0 commit comments