lance-arrow 6.0.0

Arrow Extension for Lance
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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

//! Utilities for working with streams of [`RecordBatch`].

use arrow_array::RecordBatch;
use arrow_schema::{ArrowError, SchemaRef};
use futures::stream::{self, Stream, StreamExt};
use std::pin::Pin;

use crate::deepcopy::deep_copy_batch_sliced;

/// Rechunks a stream of [`RecordBatch`] so that each output batch has
/// approximately `target_bytes` of array data.
///
/// Small input batches are accumulated (by concatenation) until at least
/// `min_bytes` of data has been collected. If the resulting batch exceeds
/// `max_bytes`, it is sliced into roughly equal pieces of ~`max_bytes`
/// (assuming uniform row sizes).
pub fn rechunk_stream_by_size<S, E>(
    input: S,
    input_schema: SchemaRef,
    min_bytes: usize,
    max_bytes: usize,
) -> impl Stream<Item = Result<RecordBatch, E>>
where
    S: Stream<Item = Result<RecordBatch, E>>,
    E: From<ArrowError>,
{
    rechunk_stream_by_size_inner(input, input_schema, min_bytes, max_bytes, false)
}

/// Like [`rechunk_stream_by_size`] but deep-copies slices so that
/// `get_array_memory_size` reflects the true size of each output batch.
///
/// After a normal `RecordBatch::slice`, the backing buffers are shared with
/// the original batch, so `get_array_memory_size` still reports the full
/// parent size.  This variant deep-copies every slice produced during the
/// splitting phase, which allows the stream to detect and re-split slices
/// that still exceed `max_bytes` (e.g. because a single row is much larger
/// than average).
///
/// The deep copy is a last resort and potentially expensive for large
/// batches.  However, it is only performed when a batch actually needs to be
/// sliced — batches that are already within the target range pass through at
/// zero cost.  Use this only when the hard cap on `max_bytes` is a
/// correctness requirement, not merely a performance hint.
pub fn rechunk_stream_by_size_deep_copy<S, E>(
    input: S,
    input_schema: SchemaRef,
    min_bytes: usize,
    max_bytes: usize,
) -> impl Stream<Item = Result<RecordBatch, E>>
where
    S: Stream<Item = Result<RecordBatch, E>>,
    E: From<ArrowError>,
{
    rechunk_stream_by_size_inner(input, input_schema, min_bytes, max_bytes, true)
}

fn rechunk_stream_by_size_inner<S, E>(
    input: S,
    input_schema: SchemaRef,
    min_bytes: usize,
    max_bytes: usize,
    deep_copy: bool,
) -> impl Stream<Item = Result<RecordBatch, E>>
where
    S: Stream<Item = Result<RecordBatch, E>>,
    E: From<ArrowError>,
{
    stream::try_unfold(
        RechunkState {
            input: Box::pin(input),
            accumulated: Vec::new(),
            acc_bytes: 0,
            done: false,
            input_schema,
            min_bytes,
            max_bytes,
            deep_copy,
        },
        |mut state| async move {
            if state.done && state.accumulated.is_empty() {
                return Ok(None);
            }

            // Pull batches until we reach the byte target or exhaust input.
            // Always pull at least one batch so that min_bytes=0 works.
            while !state.done && (state.accumulated.is_empty() || state.acc_bytes < state.min_bytes)
            {
                match state.input.next().await {
                    Some(Ok(batch)) => {
                        state.acc_bytes += batch.get_array_memory_size();
                        state.accumulated.push(batch);
                    }
                    Some(Err(e)) => return Err(e),
                    None => {
                        state.done = true;
                    }
                }
            }

            if state.accumulated.is_empty() {
                return Ok(None);
            }

            // Fast path: if the first accumulated batch already meets the
            // byte threshold, deliver it directly instead of concatenating
            // everything together (which would just get sliced back apart).
            if state.accumulated.len() > 1
                && state.accumulated[0].get_array_memory_size() >= state.min_bytes
            {
                let b = state.accumulated.remove(0);
                state.acc_bytes -= b.get_array_memory_size();
                return Ok(Some((b, state)));
            }

            let batch = if state.accumulated.len() == 1 {
                state.accumulated.pop().unwrap()
            } else {
                let b =
                    arrow_select::concat::concat_batches(&state.input_schema, &state.accumulated)
                        .map_err(E::from)?;
                state.accumulated.clear();
                b
            };
            state.acc_bytes = 0;

            // Slice the batch into ~max_bytes pieces assuming uniform row sizes.
            let mut slices =
                slice_batch(batch, state.max_bytes, state.deep_copy).map_err(E::from)?;

            if slices.len() == 1 {
                Ok(Some((slices.pop().unwrap(), state)))
            } else {
                let first = slices.remove(0);

                // Stash leftover slices for subsequent iterations.
                for a in &slices {
                    state.acc_bytes += a.get_array_memory_size();
                }
                state.accumulated = slices;

                Ok(Some((first, state)))
            }
        },
    )
}

/// Slice a batch into pieces of at most `max_bytes`.
///
/// When `deep_copy` is false, slices share buffers with the original batch
/// and `get_array_memory_size` will still report the parent buffer size.
/// This is fine when the caller only needs approximate sizing.
///
/// When `deep_copy` is true, each slice is deep-copied so that
/// `get_array_memory_size` reflects the true size.  If a deep-copied slice
/// still exceeds `max_bytes` (due to non-uniform row sizes), it is
/// recursively split until every piece is within budget or contains only a
/// single row.
fn slice_batch(
    batch: RecordBatch,
    max_bytes: usize,
    deep_copy: bool,
) -> Result<Vec<RecordBatch>, ArrowError> {
    let batch_bytes = batch.get_array_memory_size();
    let num_rows = batch.num_rows();

    if batch_bytes <= max_bytes || num_rows <= 1 {
        return Ok(vec![batch]);
    }

    let rows_per_chunk = (max_bytes as u64 * num_rows as u64 / batch_bytes as u64).max(1) as usize;

    let mut result = Vec::new();
    let mut offset = 0;
    while offset < num_rows {
        let len = rows_per_chunk.min(num_rows - offset);
        let slice = batch.slice(offset, len);
        if deep_copy {
            let copied = deep_copy_batch_sliced(&slice)?;
            // Recurse: the deep-copied slice has accurate sizes, so if it
            // still exceeds max_bytes we can split further.
            result.extend(slice_batch(copied, max_bytes, true)?);
        } else {
            result.push(slice);
        }
        offset += len;
    }

    Ok(result)
}

/// Internal state for [`rechunk_stream`].
///
/// Kept as a named struct so the `try_unfold` closure stays readable.
struct RechunkState<S> {
    input: Pin<Box<S>>,
    accumulated: Vec<RecordBatch>,
    acc_bytes: usize,
    done: bool,
    input_schema: SchemaRef,
    min_bytes: usize,
    max_bytes: usize,
    deep_copy: bool,
}

#[cfg(test)]
mod tests {
    use super::*;

    use std::sync::Arc;

    use arrow_array::Int32Array;
    use arrow_schema::{DataType, Field, Schema};
    use futures::executor::block_on;

    fn make_batch(num_rows: usize) -> RecordBatch {
        let schema = test_schema();
        let values: Vec<i32> = (0..num_rows as i32).collect();
        RecordBatch::try_new(schema, vec![Arc::new(Int32Array::from(values))]).unwrap()
    }

    fn test_schema() -> SchemaRef {
        Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, false)]))
    }

    fn collect_rechunked(
        batches: Vec<RecordBatch>,
        min_bytes: usize,
        max_bytes: usize,
    ) -> Vec<RecordBatch> {
        let input = stream::iter(batches.into_iter().map(Ok::<_, ArrowError>));
        let rechunked = rechunk_stream_by_size(input, test_schema(), min_bytes, max_bytes);
        block_on(rechunked.collect::<Vec<_>>())
            .into_iter()
            .map(|r| r.unwrap())
            .collect()
    }

    fn total_rows(batches: &[RecordBatch]) -> usize {
        batches.iter().map(|b| b.num_rows()).sum()
    }

    #[test]
    fn test_empty_stream() {
        let result = collect_rechunked(vec![], 100, 200);
        assert!(result.is_empty());
    }

    #[test]
    fn test_single_batch_passthrough() {
        let batch = make_batch(100);
        let bytes = batch.get_array_memory_size();
        // Batch is between min and max — should pass through as-is.
        let result = collect_rechunked(vec![batch], bytes / 2, bytes * 2);
        assert_eq!(result.len(), 1);
        assert_eq!(result[0].num_rows(), 100);
    }

    #[test]
    fn test_small_batches_concatenated() {
        let one_batch_bytes = make_batch(10).get_array_memory_size();
        let batches: Vec<_> = (0..8).map(|_| make_batch(10)).collect();
        // min = 5 batches worth, max = 10 batches worth.
        let result = collect_rechunked(batches, one_batch_bytes * 5, one_batch_bytes * 10);
        assert_eq!(total_rows(&result), 80);
        // Should have been concatenated into fewer batches than the 8 inputs.
        assert!(
            result.len() < 8,
            "expected fewer output batches, got {}",
            result.len()
        );
    }

    #[test]
    fn test_large_batch_sliced() {
        let batch = make_batch(1000);
        let bytes = batch.get_array_memory_size();
        let result = collect_rechunked(vec![batch], bytes / 8, bytes / 4);
        assert_eq!(total_rows(&result), 1000);
        assert!(
            result.len() >= 4,
            "expected at least 4 slices, got {}",
            result.len()
        );
    }

    #[test]
    fn test_sliced_leftovers_are_not_recombined() {
        // Key test for the fast-path optimisation. When a large batch is
        // sliced, leftover slices should be delivered one-at-a-time without
        // being concatenated back together.  We verify this by checking that
        // every output buffer pointer falls inside the original batch's
        // allocation (i.e. they are all zero-copy slices, not fresh copies).
        let batch = make_batch(1000);
        let bytes = batch.get_array_memory_size();
        let orig_data = batch.column(0).to_data();
        let orig_buf = &orig_data.buffers()[0];
        let orig_start = orig_buf.as_ptr() as usize;
        let orig_end = orig_start + orig_buf.len();

        let result = collect_rechunked(vec![batch], bytes / 8, bytes / 4);

        assert_eq!(total_rows(&result), 1000);
        assert!(result.len() >= 4);

        for (i, b) in result.iter().enumerate() {
            let ptr = b.column(0).to_data().buffers()[0].as_ptr() as usize;
            assert!(
                ptr >= orig_start && ptr < orig_end,
                "slice {i} buffer at {ptr:#x} is outside the original allocation \
                 [{orig_start:#x}, {orig_end:#x}) — it was re-concatenated"
            );
        }
    }

    #[test]
    fn test_flush_remainder_on_stream_end() {
        // Data below min_bytes should still be flushed when the stream ends.
        let batch = make_batch(10);
        let bytes = batch.get_array_memory_size();
        let result = collect_rechunked(vec![batch], bytes * 100, bytes * 200);
        assert_eq!(result.len(), 1);
        assert_eq!(result[0].num_rows(), 10);
    }

    #[test]
    fn test_large_then_small_batches() {
        // After a large batch is fully drained, subsequent small batches
        // should be accumulated normally.
        let large = make_batch(1000);
        let small_bytes = make_batch(10).get_array_memory_size();
        let batches = vec![
            large,
            make_batch(10),
            make_batch(10),
            make_batch(10),
            make_batch(10),
            make_batch(10),
        ];
        let result = collect_rechunked(batches, small_bytes * 3, small_bytes * 100);
        assert_eq!(total_rows(&result), 1050);
        // The large batch should appear (possibly sliced) followed by
        // concatenated small batches, so we should have fewer output batches
        // than the 6 inputs.
        assert!(result.len() < 6);
    }

    #[test]
    fn test_row_preservation_across_slicing() {
        // Verify that every input row appears exactly once in the output
        // and in the correct order after slicing.
        let batch = make_batch(237); // odd count to exercise remainder slice
        let bytes = batch.get_array_memory_size();
        let result = collect_rechunked(vec![batch], bytes / 8, bytes / 5);

        assert_eq!(total_rows(&result), 237);

        let values: Vec<i32> = result
            .iter()
            .flat_map(|b| {
                b.column(0)
                    .as_any()
                    .downcast_ref::<Int32Array>()
                    .unwrap()
                    .values()
                    .iter()
                    .copied()
            })
            .collect();
        let expected: Vec<i32> = (0..237).collect();
        assert_eq!(values, expected);
    }

    #[test]
    fn test_min_bytes_zero_still_yields_all_rows() {
        // When min_bytes=0, the stream should still yield every batch.
        // This is the "chop only, don't coalesce" use case.
        let batches: Vec<_> = (0..5).map(|_| make_batch(100)).collect();
        let batch_bytes = batches[0].get_array_memory_size();
        let result = collect_rechunked(batches, 0, batch_bytes * 2);
        assert_eq!(total_rows(&result), 500);
    }

    #[test]
    fn test_min_bytes_zero_slices_oversized() {
        // min_bytes=0 with a small max_bytes should still slice large batches.
        let batch = make_batch(1000);
        let bytes = batch.get_array_memory_size();
        let result = collect_rechunked(vec![batch], 0, bytes / 4);
        assert_eq!(total_rows(&result), 1000);
        assert!(
            result.len() >= 4,
            "expected at least 4 slices, got {}",
            result.len()
        );
    }

    /// Build a batch with one variable-length string column.
    /// Every row is `small_size` bytes except the row at index `big_row_idx`
    /// which is `big_size` bytes.
    fn make_variable_batch(
        num_rows: usize,
        small_size: usize,
        big_row_idx: usize,
        big_size: usize,
    ) -> RecordBatch {
        let schema = Arc::new(Schema::new(vec![Field::new("s", DataType::Utf8, false)]));
        let values: Vec<String> = (0..num_rows)
            .map(|i| {
                if i == big_row_idx {
                    "X".repeat(big_size)
                } else {
                    "x".repeat(small_size)
                }
            })
            .collect();
        let array = arrow_array::StringArray::from(values);
        RecordBatch::try_new(schema, vec![Arc::new(array)]).unwrap()
    }

    fn variable_schema() -> SchemaRef {
        Arc::new(Schema::new(vec![Field::new("s", DataType::Utf8, false)]))
    }

    fn collect_rechunked_variable(
        batches: Vec<RecordBatch>,
        min_bytes: usize,
        max_bytes: usize,
    ) -> Vec<RecordBatch> {
        let input = stream::iter(batches.into_iter().map(Ok::<_, ArrowError>));
        let rechunked =
            rechunk_stream_by_size_deep_copy(input, variable_schema(), min_bytes, max_bytes);
        block_on(rechunked.collect::<Vec<_>>())
            .into_iter()
            .map(|r| r.unwrap())
            .collect()
    }

    #[test]
    fn test_oversized_row_at_end() {
        // 100 rows: 99 small (64 bytes each) + 1 large (100KiB) at the end.
        let batch = make_variable_batch(100, 64, 99, 100 * 1024);
        let max_bytes = 64 * 1024;
        let result = collect_rechunked_variable(vec![batch], 0, max_bytes);
        assert_eq!(total_rows(&result), 100);
        for (i, b) in result.iter().enumerate() {
            let size = b.get_array_memory_size();
            assert!(
                size <= max_bytes || b.num_rows() == 1,
                "batch {i} has {size} bytes (max {max_bytes}) and {} rows",
                b.num_rows()
            );
        }
    }

    #[test]
    fn test_oversized_row_at_start() {
        // 100 rows: 1 large (100KiB) at the start + 99 small (64 bytes each).
        let batch = make_variable_batch(100, 64, 0, 100 * 1024);
        let max_bytes = 64 * 1024;
        let result = collect_rechunked_variable(vec![batch], 0, max_bytes);
        assert_eq!(total_rows(&result), 100);
        for (i, b) in result.iter().enumerate() {
            let size = b.get_array_memory_size();
            assert!(
                size <= max_bytes || b.num_rows() == 1,
                "batch {i} has {size} bytes (max {max_bytes}) and {} rows",
                b.num_rows()
            );
        }
    }

    #[test]
    fn test_oversized_row_in_middle() {
        // 100 rows: 1 large (100KiB) in the middle + 99 small (64 bytes each).
        let batch = make_variable_batch(100, 64, 50, 100 * 1024);
        let max_bytes = 64 * 1024;
        let result = collect_rechunked_variable(vec![batch], 0, max_bytes);
        assert_eq!(total_rows(&result), 100);
        for (i, b) in result.iter().enumerate() {
            let size = b.get_array_memory_size();
            assert!(
                size <= max_bytes || b.num_rows() == 1,
                "batch {i} has {size} bytes (max {max_bytes}) and {} rows",
                b.num_rows()
            );
        }
    }

    #[test]
    fn test_error_propagation() {
        let input = stream::iter(vec![
            Ok(make_batch(10)),
            Err(ArrowError::ComputeError("boom".into())),
            Ok(make_batch(10)),
        ]);
        let rechunked = rechunk_stream_by_size(input, test_schema(), 1, usize::MAX);
        let results: Vec<Result<RecordBatch, ArrowError>> = block_on(rechunked.collect());
        assert!(results.iter().any(|r| r.is_err()));
    }
}