Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
16 commits
Select commit Hold shift + click to select a range
5ab0fd1
feat: forward batch_size parameter to PyArrow Scanner
sumedhsakdeo Feb 14, 2026
c1ece14
style: fix ruff formatting in residual_evaluator lambda
sumedhsakdeo Feb 15, 2026
70af67f
chore: remove unintended vendor directory changes
sumedhsakdeo Feb 15, 2026
2474b12
feat: add ScanOrder enum to ArrowScan.to_record_batches
sumedhsakdeo Feb 14, 2026
48b332a
feat: add concurrent_files flag for bounded concurrent streaming
sumedhsakdeo Feb 14, 2026
b360ae8
fix: remove unused imports in test_bounded_concurrent_batches
sumedhsakdeo Feb 15, 2026
4186713
refactor: simplify _bounded_concurrent_batches with per-scan executor
sumedhsakdeo Feb 17, 2026
7c415d4
refactor: replace streaming param with order=ScanOrder in concurrent …
sumedhsakdeo Feb 17, 2026
70d5a99
feat: add read throughput micro-benchmark for ArrowScan configurations
sumedhsakdeo Feb 15, 2026
2e044ea
fix: remove extraneous f-string prefix in benchmark
sumedhsakdeo Feb 15, 2026
8dcd240
fix: properly reset mock call_count in test_hive_wait_for_lock
sumedhsakdeo Feb 15, 2026
4a0a430
feat: add default-4threads benchmark and time-to-first-record metric
sumedhsakdeo Feb 15, 2026
2efdcba
chore: remove default-4threads benchmark configuration
sumedhsakdeo Feb 15, 2026
09aad7a
docs: add configuration guidance table to streaming API docs
sumedhsakdeo Feb 17, 2026
b2ae725
chore: remove benchmark marker so tests run in CI
sumedhsakdeo Feb 17, 2026
afb244c
refactor: replace streaming param with order=ScanOrder in benchmarks …
sumedhsakdeo Feb 17, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
78 changes: 78 additions & 0 deletions mkdocs/docs/api.md
Original file line number Diff line number Diff line change
Expand Up @@ -355,6 +355,51 @@ for buf in tbl.scan().to_arrow_batch_reader():
print(f"Buffer contains {len(buf)} rows")
```

You can control the number of rows per batch using the `batch_size` parameter:

```python
for buf in tbl.scan().to_arrow_batch_reader(batch_size=1000):
print(f"Buffer contains {len(buf)} rows")
```

By default, each file's batches are materialized in memory before being yielded (`order=ScanOrder.TASK`). For large files that may exceed available memory, use `order=ScanOrder.ARRIVAL` to yield batches as they are produced without materializing entire files:

```python
from pyiceberg.table import ScanOrder

for buf in tbl.scan().to_arrow_batch_reader(order=ScanOrder.ARRIVAL, batch_size=1000):
print(f"Buffer contains {len(buf)} rows")
```

For maximum throughput, use `concurrent_files` to read multiple files in parallel with arrival order. Batches are yielded as they arrive from any file — ordering across files is not guaranteed:

```python
from pyiceberg.table import ScanOrder

for buf in tbl.scan().to_arrow_batch_reader(order=ScanOrder.ARRIVAL, concurrent_files=4, batch_size=1000):
print(f"Buffer contains {len(buf)} rows")
```

**Ordering semantics:**

| Configuration | File ordering | Within-file ordering |
|---|---|---|
| `ScanOrder.TASK` (default) | Batches grouped by file, in task submission order | Row order |
| `ScanOrder.ARRIVAL` | Interleaved across files (no grouping guarantee) | Row order within each file |

Within each file, batch ordering always follows row order. The `limit` parameter is enforced correctly regardless of configuration.

**Which configuration should I use?**

| Use case | Recommended config |
|---|---|
| Small tables, simple queries | Default — no extra args needed |
| Large tables, memory-constrained | `order=ScanOrder.ARRIVAL` — one file at a time, minimal memory |
| Maximum throughput with bounded memory | `order=ScanOrder.ARRIVAL, concurrent_files=N` — tune N to balance throughput vs memory |
| Fine-grained batch control | Add `batch_size=N` to any of the above |

**Note:** `ScanOrder.ARRIVAL` yields batches in arrival order (interleaved across files when `concurrent_files > 1`). For deterministic file ordering, use the default `ScanOrder.TASK` mode. `batch_size` is usually an advanced tuning knob — the PyArrow default of 131,072 rows works well for most workloads.

To avoid any type inconsistencies during writing, you can convert the Iceberg table schema to Arrow:

```python
Expand Down Expand Up @@ -1619,6 +1664,39 @@ table.scan(
).to_arrow_batch_reader()
```

The `batch_size` parameter controls the maximum number of rows per RecordBatch (default is PyArrow's 131,072 rows):

```python
table.scan(
row_filter=GreaterThanOrEqual("trip_distance", 10.0),
selected_fields=("VendorID", "tpep_pickup_datetime", "tpep_dropoff_datetime"),
).to_arrow_batch_reader(batch_size=1000)
```

Use `order=ScanOrder.ARRIVAL` to avoid materializing entire files in memory. This yields batches as they are produced by PyArrow, one file at a time:

```python
from pyiceberg.table import ScanOrder

table.scan(
row_filter=GreaterThanOrEqual("trip_distance", 10.0),
selected_fields=("VendorID", "tpep_pickup_datetime", "tpep_dropoff_datetime"),
).to_arrow_batch_reader(order=ScanOrder.ARRIVAL)
```

For concurrent file reads with arrival order, use `concurrent_files`. Note that batch ordering across files is not guaranteed:

```python
from pyiceberg.table import ScanOrder

table.scan(
row_filter=GreaterThanOrEqual("trip_distance", 10.0),
selected_fields=("VendorID", "tpep_pickup_datetime", "tpep_dropoff_datetime"),
).to_arrow_batch_reader(order=ScanOrder.ARRIVAL, concurrent_files=4)
```

When using `concurrent_files > 1`, batches from different files may be interleaved. Within each file, batches are always in row order. See the ordering semantics table in the [Apache Arrow section](#apache-arrow) above for details.

### Pandas

<!-- prettier-ignore-start -->
Expand Down
201 changes: 170 additions & 31 deletions pyiceberg/io/pyarrow.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,11 +33,14 @@
import logging
import operator
import os
import queue
import re
import threading
import uuid
import warnings
from abc import ABC, abstractmethod
from collections.abc import Callable, Iterable, Iterator
from collections.abc import Callable, Generator, Iterable, Iterator
from concurrent.futures import ThreadPoolExecutor
from copy import copy
from dataclasses import dataclass
from enum import Enum
Expand Down Expand Up @@ -141,7 +144,7 @@
visit,
visit_with_partner,
)
from pyiceberg.table import DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE, TableProperties
from pyiceberg.table import DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE, ScanOrder, TableProperties
from pyiceberg.table.locations import load_location_provider
from pyiceberg.table.metadata import TableMetadata
from pyiceberg.table.name_mapping import NameMapping, apply_name_mapping
Expand Down Expand Up @@ -1581,6 +1584,7 @@ def _task_to_record_batches(
partition_spec: PartitionSpec | None = None,
format_version: TableVersion = TableProperties.DEFAULT_FORMAT_VERSION,
downcast_ns_timestamp_to_us: bool | None = None,
batch_size: int | None = None,
) -> Iterator[pa.RecordBatch]:
arrow_format = _get_file_format(task.file.file_format, pre_buffer=True, buffer_size=(ONE_MEGABYTE * 8))
with io.new_input(task.file.file_path).open() as fin:
Expand Down Expand Up @@ -1612,14 +1616,18 @@ def _task_to_record_batches(

file_project_schema = prune_columns(file_schema, projected_field_ids, select_full_types=False)

fragment_scanner = ds.Scanner.from_fragment(
fragment=fragment,
schema=physical_schema,
scanner_kwargs: dict[str, Any] = {
"fragment": fragment,
"schema": physical_schema,
# This will push down the query to Arrow.
# But in case there are positional deletes, we have to apply them first
filter=pyarrow_filter if not positional_deletes else None,
columns=[col.name for col in file_project_schema.columns],
)
"filter": pyarrow_filter if not positional_deletes else None,
"columns": [col.name for col in file_project_schema.columns],
}
if batch_size is not None:
scanner_kwargs["batch_size"] = batch_size

fragment_scanner = ds.Scanner.from_fragment(**scanner_kwargs)

next_index = 0
batches = fragment_scanner.to_batches()
Expand Down Expand Up @@ -1677,6 +1685,76 @@ def _read_all_delete_files(io: FileIO, tasks: Iterable[FileScanTask]) -> dict[st
return deletes_per_file


_QUEUE_SENTINEL = object()


def _bounded_concurrent_batches(
tasks: list[FileScanTask],
batch_fn: Callable[[FileScanTask], Iterator[pa.RecordBatch]],
concurrent_files: int,
max_buffered_batches: int = 16,
) -> Generator[pa.RecordBatch, None, None]:
"""Read batches from multiple files concurrently with bounded memory.

Uses a per-scan ThreadPoolExecutor(max_workers=concurrent_files) to naturally
bound concurrency. Workers push batches into a bounded queue which provides
backpressure when the consumer is slower than the producers.

Args:
tasks: The file scan tasks to process.
batch_fn: A callable that takes a FileScanTask and returns an iterator of RecordBatches.
concurrent_files: Maximum number of files to read concurrently.
max_buffered_batches: Maximum number of batches to buffer in the queue.
"""
if not tasks:
return

batch_queue: queue.Queue[pa.RecordBatch | BaseException | object] = queue.Queue(maxsize=max_buffered_batches)
cancel = threading.Event()
remaining = len(tasks)
remaining_lock = threading.Lock()

def worker(task: FileScanTask) -> None:
nonlocal remaining
try:
for batch in batch_fn(task):
if cancel.is_set():
return
batch_queue.put(batch)
except BaseException as e:
if not cancel.is_set():
batch_queue.put(e)
finally:
with remaining_lock:
remaining -= 1
if remaining == 0:
batch_queue.put(_QUEUE_SENTINEL)

with ThreadPoolExecutor(max_workers=concurrent_files) as executor:
for task in tasks:
executor.submit(worker, task)

try:
while True:
item = batch_queue.get()

if item is _QUEUE_SENTINEL:
break

if isinstance(item, BaseException):
raise item

yield item
finally:
cancel.set()
# Drain the queue to unblock any workers stuck on put()
while not batch_queue.empty():
try:
batch_queue.get_nowait()
except queue.Empty:
break


class ArrowScan:
_table_metadata: TableMetadata
_io: FileIO
Expand Down Expand Up @@ -1756,54 +1834,114 @@ def to_table(self, tasks: Iterable[FileScanTask]) -> pa.Table:

return result

def to_record_batches(self, tasks: Iterable[FileScanTask]) -> Iterator[pa.RecordBatch]:
def to_record_batches(
self,
tasks: Iterable[FileScanTask],
batch_size: int | None = None,
order: ScanOrder = ScanOrder.TASK,
concurrent_files: int = 1,
) -> Iterator[pa.RecordBatch]:
"""Scan the Iceberg table and return an Iterator[pa.RecordBatch].

Returns an Iterator of pa.RecordBatch with data from the Iceberg table
by resolving the right columns that match the current table schema.
Only data that matches the provided row_filter expression is returned.

Ordering semantics:
- ScanOrder.TASK (default): Batches are grouped by file in task submission order.
- ScanOrder.ARRIVAL: Batches may be interleaved across files. Within each file,
batch ordering follows row order.

Args:
tasks: FileScanTasks representing the data files and delete files to read from.
batch_size: The number of rows per batch. If None, PyArrow's default is used.
order: Controls the order in which record batches are returned.
ScanOrder.TASK (default) returns batches in task order, with each task
fully materialized before proceeding to the next. Allows parallel file
reads via executor. ScanOrder.ARRIVAL yields batches as they are
produced without materializing entire files into memory.
concurrent_files: Number of files to read concurrently when order=ScanOrder.ARRIVAL.
Must be >= 1. When > 1, batches may arrive interleaved across files.
Ignored when order=ScanOrder.TASK.

Returns:
An Iterator of PyArrow RecordBatches.
Total number of rows will be capped if specified.

Raises:
ResolveError: When a required field cannot be found in the file
ValueError: When a field type in the file cannot be projected to the schema type
ValueError: When a field type in the file cannot be projected to the schema type,
or when an invalid order value is provided, or when concurrent_files < 1.
"""
deletes_per_file = _read_all_delete_files(self._io, tasks)
if not isinstance(order, ScanOrder):
raise ValueError(f"Invalid order: {order!r}. Must be a ScanOrder enum value (ScanOrder.TASK or ScanOrder.ARRIVAL).")

total_row_count = 0
if concurrent_files < 1:
raise ValueError(f"concurrent_files must be >= 1, got {concurrent_files}")

task_list, deletes_per_file = self._prepare_tasks_and_deletes(tasks)

if order == ScanOrder.ARRIVAL:
return self._apply_limit(self._iter_batches_arrival(task_list, deletes_per_file, batch_size, concurrent_files))

return self._apply_limit(self._iter_batches_materialized(task_list, deletes_per_file, batch_size))

def _prepare_tasks_and_deletes(
self, tasks: Iterable[FileScanTask]
) -> tuple[list[FileScanTask], dict[str, list[ChunkedArray]]]:
"""Resolve delete files and return tasks as a list."""
task_list = list(tasks)
deletes_per_file = _read_all_delete_files(self._io, task_list)
return task_list, deletes_per_file

def _iter_batches_arrival(
self,
task_list: list[FileScanTask],
deletes_per_file: dict[str, list[ChunkedArray]],
batch_size: int | None,
concurrent_files: int,
) -> Iterator[pa.RecordBatch]:
"""Yield batches using bounded concurrent streaming in arrival order."""

def batch_fn(task: FileScanTask) -> Iterator[pa.RecordBatch]:
return self._record_batches_from_scan_tasks_and_deletes([task], deletes_per_file, batch_size)

yield from _bounded_concurrent_batches(task_list, batch_fn, concurrent_files)

def _iter_batches_materialized(
self,
task_list: list[FileScanTask],
deletes_per_file: dict[str, list[ChunkedArray]],
batch_size: int | None,
) -> Iterator[pa.RecordBatch]:
"""Yield batches using executor.map with full file materialization."""
executor = ExecutorFactory.get_or_create()

def batches_for_task(task: FileScanTask) -> list[pa.RecordBatch]:
# Materialize the iterator here to ensure execution happens within the executor.
# Otherwise, the iterator would be lazily consumed later (in the main thread),
# defeating the purpose of using executor.map.
return list(self._record_batches_from_scan_tasks_and_deletes([task], deletes_per_file))
return list(self._record_batches_from_scan_tasks_and_deletes([task], deletes_per_file, batch_size))

limit_reached = False
for batches in executor.map(batches_for_task, tasks):
for batch in batches:
current_batch_size = len(batch)
if self._limit is not None and total_row_count + current_batch_size >= self._limit:
yield batch.slice(0, self._limit - total_row_count)
for batches in executor.map(batches_for_task, task_list):
yield from batches

limit_reached = True
break
else:
yield batch
total_row_count += current_batch_size
def _apply_limit(self, batches: Iterator[pa.RecordBatch]) -> Iterator[pa.RecordBatch]:
"""Apply row limit across batches."""
if self._limit is None:
yield from batches
return

if limit_reached:
# This break will also cancel all running tasks in the executor
break
total_row_count = 0
for batch in batches:
remaining = self._limit - total_row_count
if remaining <= 0:
return
if len(batch) > remaining:
yield batch.slice(0, remaining)
return
yield batch
total_row_count += len(batch)

def _record_batches_from_scan_tasks_and_deletes(
self, tasks: Iterable[FileScanTask], deletes_per_file: dict[str, list[ChunkedArray]]
self, tasks: Iterable[FileScanTask], deletes_per_file: dict[str, list[ChunkedArray]], batch_size: int | None = None
) -> Iterator[pa.RecordBatch]:
total_row_count = 0
for task in tasks:
Expand All @@ -1822,6 +1960,7 @@ def _record_batches_from_scan_tasks_and_deletes(
self._table_metadata.specs().get(task.file.spec_id),
self._table_metadata.format_version,
self._downcast_ns_timestamp_to_us,
batch_size,
)
for batch in batches:
if self._limit is not None:
Expand Down
Loading