Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
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
80 changes: 80 additions & 0 deletions mkdocs/docs/api.md
Original file line number Diff line number Diff line change
Expand Up @@ -355,6 +355,62 @@ for buf in tbl.scan().to_arrow_batch_reader():
print(f"Buffer contains {len(buf)} rows")
```

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

```python
from pyiceberg.table import ArrivalOrder

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

For maximum throughput, tune `concurrent_streams` 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 ArrivalOrder

for buf in tbl.scan().to_arrow_batch_reader(order=ArrivalOrder(concurrent_streams=4)):
print(f"Buffer contains {len(buf)} rows")
```

**Ordering semantics:**

| Configuration | File ordering | Within-file ordering |
|---|---|---|
| `TaskOrder()` (default) | Batches grouped by file, in task submission order | Row order |
| `ArrivalOrder(concurrent_streams=1)` | Sequential, one file at a time | Row order |
| `ArrivalOrder(concurrent_streams>1)` | Interleaved across files (no grouping guarantee) | Row order within each file |

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, maximum throughput with bounded memory | `order=ArrivalOrder(concurrent_streams=N)` — tune N to balance throughput vs memory |
| Fine-grained memory control | `order=ArrivalOrder(concurrent_streams=N, batch_size=M, max_buffered_batches=K)` — tune all parameters |

**Memory usage and performance characteristics:**

- **TaskOrder (default)**: Uses full file materialization. Each file is loaded entirely into memory before yielding batches. Memory usage depends on file sizes.
- **ArrivalOrder**: Uses streaming with controlled memory usage. Memory is bounded by the batch buffering mechanism.

**Memory formula for ArrivalOrder:**

```text
Peak memory ≈ concurrent_streams × batch_size × max_buffered_batches × (average row size in bytes)
```

Where:

- `concurrent_streams`: Number of files read in parallel (default: 8)
- `batch_size`: Number of rows per batch (default: 131,072, can be set via ArrivalOrder constructor)
- `max_buffered_batches`: Internal buffering parameter (default: 16, can be tuned for advanced use cases)
- Average row size depends on your schema and data; multiply the above by it to estimate bytes.

**Note:** `ArrivalOrder()` yields batches in arrival order (interleaved across files when `concurrent_streams > 1`). For deterministic file ordering, use the default `TaskOrder()` mode. The `batch_size` parameter in `ArrivalOrder` controls streaming memory usage, while `TaskOrder` uses full file materialization regardless of batch size.

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

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

To avoid materializing entire files in memory, use `ArrivalOrder` which yields batches as they are produced by PyArrow:

```python
from pyiceberg.table import ArrivalOrder

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

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

```python
from pyiceberg.table import ArrivalOrder

table.scan(
row_filter=GreaterThanOrEqual("trip_distance", 10.0),
selected_fields=("VendorID", "tpep_pickup_datetime", "tpep_dropoff_datetime"),
).to_arrow_batch_reader(order=ArrivalOrder(concurrent_streams=16))
```

When using `concurrent_streams > 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
207 changes: 177 additions & 30 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, ArrivalOrder, ScanOrder, TableProperties, TaskOrder
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,86 @@ 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_streams: 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_streams) 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_streams: Maximum number of concurrent read streams.
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
is_last = remaining == 0
if is_last:
batch_queue.put(_QUEUE_SENTINEL)

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

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

if item is _QUEUE_SENTINEL:
saw_sentinel = True
break

if isinstance(item, BaseException):
raise item

yield item
finally:
cancel.set()
if not saw_sentinel:
# Drain the queue to unblock workers stuck on put().
# Each get() wakes one waiting producer; that producer checks
# cancel and returns, eventually allowing the last worker to
# put the sentinel. We stop only when we see the sentinel,
# which guarantees all workers have finished.
while True:
item = batch_queue.get()
if item is _QUEUE_SENTINEL:
break


_DEFAULT_SCAN_ORDER: ScanOrder = TaskOrder()


class ArrowScan:
_table_metadata: TableMetadata
_io: FileIO
Expand Down Expand Up @@ -1756,54 +1844,112 @@ 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],
order: ScanOrder = _DEFAULT_SCAN_ORDER,
) -> 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:
- TaskOrder() (default): Yields batches one file at a time in task submission order.
- ArrivalOrder(): Batches may be interleaved across files as they arrive.
Within each file, batch ordering follows row order.

Args:
tasks: FileScanTasks representing the data files and delete files to read from.
order: Controls the order in which record batches are returned.
TaskOrder() (default) yields batches one file at a time in task order.
ArrivalOrder(concurrent_streams=N, batch_size=B, max_buffered_batches=M)
yields batches as they are produced without materializing entire files
into memory. Peak memory ≈ concurrent_streams × batch_size × max_buffered_batches
× (average row size in bytes). batch_size is the number of rows per batch.
For example (if average row size ≈ 32 bytes):
- ArrivalOrder(concurrent_streams=4, batch_size=32768, max_buffered_batches=8)
- Peak memory ≈ 4 × 32768 rows × 8 × 32 bytes ≈ ~32 MB (plus Arrow overhead)

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 concurrent_streams < 1.
"""
deletes_per_file = _read_all_delete_files(self._io, tasks)
task_list, deletes_per_file = self._prepare_tasks_and_deletes(tasks)

if isinstance(order, ArrivalOrder):
if order.concurrent_streams < 1:
raise ValueError(f"concurrent_streams must be >= 1, got {order.concurrent_streams}")
return self._apply_limit(
self._iter_batches_arrival(
task_list, deletes_per_file, order.batch_size, order.concurrent_streams, order.max_buffered_batches
)
)

total_row_count = 0
return self._apply_limit(self._iter_batches_materialized(task_list, deletes_per_file))

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_streams: int,
max_buffered_batches: int = 16,
) -> 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_streams, max_buffered_batches)

def _iter_batches_materialized(
self,
task_list: list[FileScanTask],
deletes_per_file: dict[str, list[ChunkedArray]],
) -> 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))

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 +1968,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
Loading