feat: add batch_size_bytes option to file reader#6388
Draft
westonpace wants to merge 4 commits intolance-format:mainfrom
Draft
feat: add batch_size_bytes option to file reader#6388westonpace wants to merge 4 commits intolance-format:mainfrom
westonpace wants to merge 4 commits intolance-format:mainfrom
Conversation
Thread a new `batch_size_bytes: Option<u64>` option from `SchedulerDecoderConfig` through `create_decode_stream` into `StructuralBatchDecodeStream`. All existing call sites pass `None`, so there is no behavioral change. For legacy v2.0 files the option is ignored with a warning. Part of lance-format#6387 Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
When `batch_size_bytes` is `Some`, compute the number of rows to drain per batch from an estimated bytes-per-row instead of using `rows_per_batch`. The estimate is computed once from the schema using `estimate_bytes_per_row()`, which is exact for fixed-width types and uses rough defaults for variable-width types. Part of lance-format#6387 Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
After each batch is decoded, measure the actual data bytes per row and feed it back so that the next `next_batch_task()` call uses the measured value instead of the schema-based estimate. This corrects for inaccurate initial estimates on variable-width data (strings, binary) where the schema default of 64 bytes may be far off. The measurement uses `batch_data_size()`, a new helper that computes the actual data contribution of a batch by walking column types and reading offsets for variable-width arrays. This avoids the over-counting from `get_array_memory_size()` which reports full shared page-buffer capacity rather than per-batch data. Part of lance-format#6387 Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
Co-Authored-By: Claude Opus 4.6 <noreply@anthropic.com>
westonpace
commented
Apr 2, 2026
Comment on lines
+1686
to
+1696
| /// Compute the actual data size (in bytes) of a record batch, | ||
| /// accounting only for the portion of buffers that belongs to the | ||
| /// batch's row range. Unlike `get_array_memory_size()`, this does | ||
| /// not over-count when arrays share a larger underlying page buffer. | ||
| fn batch_data_size(batch: &RecordBatch) -> u64 { | ||
| batch | ||
| .columns() | ||
| .iter() | ||
| .map(|c| array_data_size(c.as_ref())) | ||
| .sum() | ||
| } |
Member
Author
There was a problem hiding this comment.
I don't like this. I'm going to make a prequel PR to address getting the size of decoded batches
Codecov Report❌ Patch coverage is
📢 Thoughts on this report? Let us know! |
4 tasks
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Summary
batch_size_bytes: Option<u64>toSchedulerDecoderConfigand thread it through the structural v2.1 decode pathbatch_sizerow countStructuralBatchDecodeStreamis modified; legacy v2.0BatchDecodeStreamis unchanged (logs a warning if the option is set)Test plan
test_estimate_bytes_per_row— unit test for the schema-based byte estimatortest_byte_sized_batches_fixed_width— 1000 rows × 4 Int32 columns,batch_size_bytes=1600→ 10 batches of exactly 100 rows, roundtrip verifiedtest_byte_sized_batches_none_unchanged—batch_size_bytes=Nonestill usesrows_per_batch(no behavioral change)test_byte_sized_batches_feedback_convergence— 100-byte strings with 64-byte schema estimate; verifies second/third batches converge to ~50 rows after feedbackcargo clippy -p lance-encoding --tests -p lance-file -- -D warningscleancargo fmt --all -- --checkcleanCloses #6387
🤖 Generated with Claude Code