datafusion_physical_plan/coalesce/mod.rs
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600
// 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.
use arrow::compute::concat_batches;
use arrow_array::builder::StringViewBuilder;
use arrow_array::cast::AsArray;
use arrow_array::{Array, ArrayRef, RecordBatch, RecordBatchOptions};
use arrow_schema::SchemaRef;
use std::sync::Arc;
/// Concatenate multiple [`RecordBatch`]es
///
/// `BatchCoalescer` concatenates multiple small [`RecordBatch`]es, produced by
/// operations such as `FilterExec` and `RepartitionExec`, into larger ones for
/// more efficient processing by subsequent operations.
///
/// # Background
///
/// Generally speaking, larger [`RecordBatch`]es are more efficient to process
/// than smaller record batches (until the CPU cache is exceeded) because there
/// is fixed processing overhead per batch. DataFusion tries to operate on
/// batches of `target_batch_size` rows to amortize this overhead
///
/// ```text
/// ┌────────────────────┐
/// │ RecordBatch │
/// │ num_rows = 23 │
/// └────────────────────┘ ┌────────────────────┐
/// │ │
/// ┌────────────────────┐ Coalesce │ │
/// │ │ Batches │ │
/// │ RecordBatch │ │ │
/// │ num_rows = 50 │ ─ ─ ─ ─ ─ ─ ▶ │ │
/// │ │ │ RecordBatch │
/// │ │ │ num_rows = 106 │
/// └────────────────────┘ │ │
/// │ │
/// ┌────────────────────┐ │ │
/// │ │ │ │
/// │ RecordBatch │ │ │
/// │ num_rows = 33 │ └────────────────────┘
/// │ │
/// └────────────────────┘
/// ```
///
/// # 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 least
/// `target_batch_size` rows.
///
/// 3. Eventually this may also be able to handle other optimizations such as a
/// combined filter/coalesce operation.
///
#[derive(Debug)]
pub struct BatchCoalescer {
/// The input schema
schema: SchemaRef,
/// Minimum number of rows for coalesces batches
target_batch_size: usize,
/// Total number of rows returned so far
total_rows: usize,
/// Buffered batches
buffer: Vec<RecordBatch>,
/// Buffered row count
buffered_rows: usize,
/// Limit: maximum number of rows to fetch, `None` means fetch all rows
fetch: Option<usize>,
}
impl BatchCoalescer {
/// Create a new `BatchCoalescer`
///
/// # Arguments
/// - `schema` - the schema of the output batches
/// - `target_batch_size` - the minimum number of rows for each
/// output batch (until limit reached)
/// - `fetch` - the maximum number of rows to fetch, `None` means fetch all rows
pub fn new(
schema: SchemaRef,
target_batch_size: usize,
fetch: Option<usize>,
) -> Self {
Self {
schema,
target_batch_size,
total_rows: 0,
buffer: vec![],
buffered_rows: 0,
fetch,
}
}
/// Return the schema of the output batches
pub fn schema(&self) -> SchemaRef {
Arc::clone(&self.schema)
}
/// Push next batch, and returns [`CoalescerState`] indicating the current
/// state of the buffer.
pub fn push_batch(&mut self, batch: RecordBatch) -> CoalescerState {
let batch = gc_string_view_batch(&batch);
if self.limit_reached(&batch) {
CoalescerState::LimitReached
} else if self.target_reached(batch) {
CoalescerState::TargetReached
} else {
CoalescerState::Continue
}
}
/// Return true if the there is no data buffered
pub fn is_empty(&self) -> bool {
self.buffer.is_empty()
}
/// Checks if the buffer will reach the specified limit after getting
/// `batch`.
///
/// If fetch would be exceeded, slices the received batch, updates the
/// buffer with it, and returns `true`.
///
/// Otherwise: does nothing and returns `false`.
fn limit_reached(&mut self, batch: &RecordBatch) -> bool {
match self.fetch {
Some(fetch) if self.total_rows + batch.num_rows() >= fetch => {
// Limit is reached
let remaining_rows = fetch - self.total_rows;
debug_assert!(remaining_rows > 0);
let batch = batch.slice(0, remaining_rows);
self.buffered_rows += batch.num_rows();
self.total_rows = fetch;
self.buffer.push(batch);
true
}
_ => false,
}
}
/// Updates the buffer with the given batch.
///
/// If the target batch size is reached, returns `true`. Otherwise, returns
/// `false`.
fn target_reached(&mut self, batch: RecordBatch) -> bool {
if batch.num_rows() == 0 {
false
} else {
self.total_rows += batch.num_rows();
self.buffered_rows += batch.num_rows();
self.buffer.push(batch);
self.buffered_rows >= self.target_batch_size
}
}
/// Concatenates and returns all buffered batches, and clears the buffer.
pub fn finish_batch(&mut self) -> datafusion_common::Result<RecordBatch> {
let batch = concat_batches(&self.schema, &self.buffer)?;
self.buffer.clear();
self.buffered_rows = 0;
Ok(batch)
}
}
/// Indicates the state of the [`BatchCoalescer`] buffer after the
/// [`BatchCoalescer::push_batch()`] operation.
///
/// The caller should take diferent actions, depending on the variant returned.
pub enum CoalescerState {
/// Neither the limit nor the target batch size is reached.
///
/// Action: continue pushing batches.
Continue,
/// The limit has been reached.
///
/// Action: call [`BatchCoalescer::finish_batch()`] to get the final
/// buffered results as a batch and finish the query.
LimitReached,
/// The specified minimum number of rows a batch should have is reached.
///
/// Action: call [`BatchCoalescer::finish_batch()`] to get the current
/// buffered results as a batch and then continue pushing batches.
TargetReached,
}
/// Heuristically compact `StringViewArray`s to reduce memory usage, if needed
///
/// Decides when to consolidate the StringView into a new buffer to reduce
/// memory usage and improve string locality for better performance.
///
/// This differs from `StringViewArray::gc` because:
/// 1. It may not compact the array depending on a heuristic.
/// 2. It uses a precise block size to reduce the number of buffers to track.
///
/// # Heuristic
///
/// If the average size of each view is larger than 32 bytes, we compact the array.
///
/// `StringViewArray` include pointers to buffer that hold the underlying data.
/// One of the great benefits of `StringViewArray` is that many operations
/// (e.g., `filter`) can be done without copying the underlying data.
///
/// However, after a while (e.g., after `FilterExec` or `HashJoinExec`) the
/// `StringViewArray` may only refer to a small portion of the buffer,
/// significantly increasing memory usage.
fn gc_string_view_batch(batch: &RecordBatch) -> RecordBatch {
let new_columns: Vec<ArrayRef> = batch
.columns()
.iter()
.map(|c| {
// Try to re-create the `StringViewArray` to prevent holding the underlying buffer too long.
let Some(s) = c.as_string_view_opt() else {
return Arc::clone(c);
};
let ideal_buffer_size: usize = s
.views()
.iter()
.map(|v| {
let len = (*v as u32) as usize;
if len > 12 {
len
} else {
0
}
})
.sum();
let actual_buffer_size = s.get_buffer_memory_size();
// Re-creating the array copies data and can be time consuming.
// We only do it if the array is sparse
if actual_buffer_size > (ideal_buffer_size * 2) {
// We set the block size to `ideal_buffer_size` so that the new StringViewArray only has one buffer, which accelerate later concat_batches.
// See https://github.com/apache/arrow-rs/issues/6094 for more details.
let mut builder = StringViewBuilder::with_capacity(s.len());
if ideal_buffer_size > 0 {
builder = builder.with_fixed_block_size(ideal_buffer_size as u32);
}
for v in s.iter() {
builder.append_option(v);
}
let gc_string = builder.finish();
debug_assert!(gc_string.data_buffers().len() <= 1); // buffer count can be 0 if the `ideal_buffer_size` is 0
Arc::new(gc_string)
} else {
Arc::clone(c)
}
})
.collect();
let mut options = RecordBatchOptions::new();
options = options.with_row_count(Some(batch.num_rows()));
RecordBatch::try_new_with_options(batch.schema(), new_columns, &options)
.expect("Failed to re-create the gc'ed record batch")
}
#[cfg(test)]
mod tests {
use std::ops::Range;
use super::*;
use arrow::datatypes::{DataType, Field, Schema};
use arrow_array::builder::ArrayBuilder;
use arrow_array::{StringViewArray, UInt32Array};
#[test]
fn test_coalesce() {
let batch = uint32_batch(0..8);
Test::new()
.with_batches(std::iter::repeat(batch).take(10))
// expected output is batches of at least 20 rows (except for the final batch)
.with_target_batch_size(21)
.with_expected_output_sizes(vec![24, 24, 24, 8])
.run()
}
#[test]
fn test_coalesce_with_fetch_larger_than_input_size() {
let batch = uint32_batch(0..8);
Test::new()
.with_batches(std::iter::repeat(batch).take(10))
// input is 10 batches x 8 rows (80 rows) with fetch limit of 100
// expected to behave the same as `test_concat_batches`
.with_target_batch_size(21)
.with_fetch(Some(100))
.with_expected_output_sizes(vec![24, 24, 24, 8])
.run();
}
#[test]
fn test_coalesce_with_fetch_less_than_input_size() {
let batch = uint32_batch(0..8);
Test::new()
.with_batches(std::iter::repeat(batch).take(10))
// input is 10 batches x 8 rows (80 rows) with fetch limit of 50
.with_target_batch_size(21)
.with_fetch(Some(50))
.with_expected_output_sizes(vec![24, 24, 2])
.run();
}
#[test]
fn test_coalesce_with_fetch_less_than_target_and_no_remaining_rows() {
let batch = uint32_batch(0..8);
Test::new()
.with_batches(std::iter::repeat(batch).take(10))
// input is 10 batches x 8 rows (80 rows) with fetch limit of 48
.with_target_batch_size(21)
.with_fetch(Some(48))
.with_expected_output_sizes(vec![24, 24])
.run();
}
#[test]
fn test_coalesce_with_fetch_less_target_batch_size() {
let batch = uint32_batch(0..8);
Test::new()
.with_batches(std::iter::repeat(batch).take(10))
// input is 10 batches x 8 rows (80 rows) with fetch limit of 10
.with_target_batch_size(21)
.with_fetch(Some(10))
.with_expected_output_sizes(vec![10])
.run();
}
#[test]
fn test_coalesce_single_large_batch_over_fetch() {
let large_batch = uint32_batch(0..100);
Test::new()
.with_batch(large_batch)
.with_target_batch_size(20)
.with_fetch(Some(7))
.with_expected_output_sizes(vec![7])
.run()
}
/// Test for [`BatchCoalescer`]
///
/// Pushes the input batches to the coalescer and verifies that the resulting
/// batches have the expected number of rows and contents.
#[derive(Debug, Clone, Default)]
struct Test {
/// Batches to feed to the coalescer. Tests must have at least one
/// schema
input_batches: Vec<RecordBatch>,
/// Expected output sizes of the resulting batches
expected_output_sizes: Vec<usize>,
/// target batch size
target_batch_size: usize,
/// Fetch (limit)
fetch: Option<usize>,
}
impl Test {
fn new() -> Self {
Self::default()
}
/// Set the target batch size
fn with_target_batch_size(mut self, target_batch_size: usize) -> Self {
self.target_batch_size = target_batch_size;
self
}
/// Set the fetch (limit)
fn with_fetch(mut self, fetch: Option<usize>) -> Self {
self.fetch = fetch;
self
}
/// Extend the input batches with `batch`
fn with_batch(mut self, batch: RecordBatch) -> Self {
self.input_batches.push(batch);
self
}
/// Extends the input batches with `batches`
fn with_batches(
mut self,
batches: impl IntoIterator<Item = RecordBatch>,
) -> Self {
self.input_batches.extend(batches);
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
fn run(self) {
let Self {
input_batches,
target_batch_size,
fetch,
expected_output_sizes,
} = self;
let schema = input_batches[0].schema();
// create a single large input batch for output comparison
let single_input_batch = concat_batches(&schema, &input_batches).unwrap();
let mut coalescer =
BatchCoalescer::new(Arc::clone(&schema), target_batch_size, fetch);
let mut output_batches = vec![];
for batch in input_batches {
match coalescer.push_batch(batch) {
CoalescerState::Continue => {}
CoalescerState::LimitReached => {
output_batches.push(coalescer.finish_batch().unwrap());
break;
}
CoalescerState::TargetReached => {
coalescer.buffered_rows = 0;
output_batches.push(coalescer.finish_batch().unwrap());
}
}
}
if coalescer.buffered_rows != 0 {
output_batches.extend(coalescer.buffer);
}
// make sure we got the expected number of output batches and content
let mut starting_idx = 0;
assert_eq!(expected_output_sizes.len(), output_batches.len());
for (i, (expected_size, batch)) in
expected_output_sizes.iter().zip(output_batches).enumerate()
{
assert_eq!(
*expected_size,
batch.num_rows(),
"Unexpected number of rows in Batch {i}"
);
// compare the contents of the batch (using `==` compares the
// underlying memory layout too)
let expected_batch =
single_input_batch.slice(starting_idx, *expected_size);
let batch_strings = batch_to_pretty_strings(&batch);
let expected_batch_strings = batch_to_pretty_strings(&expected_batch);
let batch_strings = batch_strings.lines().collect::<Vec<_>>();
let expected_batch_strings =
expected_batch_strings.lines().collect::<Vec<_>>();
assert_eq!(
expected_batch_strings, batch_strings,
"Unexpected content in Batch {i}:\
\n\nExpected:\n{expected_batch_strings:#?}\n\nActual:\n{batch_strings:#?}"
);
starting_idx += *expected_size;
}
}
}
/// Return a batch of UInt32 with the specified range
fn uint32_batch(range: Range<u32>) -> RecordBatch {
let schema =
Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)]));
RecordBatch::try_new(
Arc::clone(&schema),
vec![Arc::new(UInt32Array::from_iter_values(range))],
)
.unwrap()
}
#[test]
fn test_gc_string_view_batch_small_no_compact() {
// view with only short strings (no buffers) --> no need to compact
let array = StringViewTest {
rows: 1000,
strings: vec![Some("a"), Some("b"), Some("c")],
}
.build();
let gc_array = do_gc(array.clone());
compare_string_array_values(&array, &gc_array);
assert_eq!(array.data_buffers().len(), 0);
assert_eq!(array.data_buffers().len(), gc_array.data_buffers().len()); // no compaction
}
#[test]
fn test_gc_string_view_test_batch_empty() {
let schema = Schema::empty();
let batch = RecordBatch::new_empty(schema.into());
let output_batch = gc_string_view_batch(&batch);
assert_eq!(batch.num_columns(), output_batch.num_columns());
assert_eq!(batch.num_rows(), output_batch.num_rows());
}
#[test]
fn test_gc_string_view_batch_large_no_compact() {
// view with large strings (has buffers) but full --> no need to compact
let array = StringViewTest {
rows: 1000,
strings: vec![Some("This string is longer than 12 bytes")],
}
.build();
let gc_array = do_gc(array.clone());
compare_string_array_values(&array, &gc_array);
assert_eq!(array.data_buffers().len(), 5);
assert_eq!(array.data_buffers().len(), gc_array.data_buffers().len()); // no compaction
}
#[test]
fn test_gc_string_view_batch_large_slice_compact() {
// view with large strings (has buffers) and only partially used --> no need to compact
let array = StringViewTest {
rows: 1000,
strings: vec![Some("this string is longer than 12 bytes")],
}
.build();
// slice only 11 rows, so most of the buffer is not used
let array = array.slice(11, 22);
let gc_array = do_gc(array.clone());
compare_string_array_values(&array, &gc_array);
assert_eq!(array.data_buffers().len(), 5);
assert_eq!(gc_array.data_buffers().len(), 1); // compacted into a single buffer
}
/// Compares the values of two string view arrays
fn compare_string_array_values(arr1: &StringViewArray, arr2: &StringViewArray) {
assert_eq!(arr1.len(), arr2.len());
for (s1, s2) in arr1.iter().zip(arr2.iter()) {
assert_eq!(s1, s2);
}
}
/// runs garbage collection on string view array
/// and ensures the number of rows are the same
fn do_gc(array: StringViewArray) -> StringViewArray {
let batch =
RecordBatch::try_from_iter(vec![("a", Arc::new(array) as ArrayRef)]).unwrap();
let gc_batch = gc_string_view_batch(&batch);
assert_eq!(batch.num_rows(), gc_batch.num_rows());
assert_eq!(batch.schema(), gc_batch.schema());
gc_batch
.column(0)
.as_any()
.downcast_ref::<StringViewArray>()
.unwrap()
.clone()
}
/// Describes parameters for creating a `StringViewArray`
struct StringViewTest {
/// The number of rows in the array
rows: usize,
/// The strings to use in the array (repeated over and over
strings: Vec<Option<&'static str>>,
}
impl StringViewTest {
/// Create a `StringViewArray` with the parameters specified in this struct
fn build(self) -> StringViewArray {
let mut builder =
StringViewBuilder::with_capacity(100).with_fixed_block_size(8192);
loop {
for &v in self.strings.iter() {
builder.append_option(v);
if builder.len() >= self.rows {
return builder.finish();
}
}
}
}
}
fn batch_to_pretty_strings(batch: &RecordBatch) -> String {
arrow::util::pretty::pretty_format_batches(&[batch.clone()])
.unwrap()
.to_string()
}
}