Skip to content

Improve to_arrow_batch_reader performance + use to_arrow_batch_reader in upsert to lower memory pressure #1995

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 17 commits into
base: main
Choose a base branch
from
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
76 changes: 32 additions & 44 deletions pyiceberg/io/pyarrow.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@

from __future__ import annotations

import concurrent.futures
import fnmatch
import functools
import itertools
Expand All @@ -36,7 +35,6 @@
import uuid
import warnings
from abc import ABC, abstractmethod
from concurrent.futures import Future
from copy import copy
from dataclasses import dataclass
from enum import Enum
Expand Down Expand Up @@ -71,7 +69,6 @@
FileType,
FSSpecHandler,
)
from sortedcontainers import SortedList

from pyiceberg.conversions import to_bytes
from pyiceberg.exceptions import ResolveError
Expand Down Expand Up @@ -1570,47 +1567,16 @@ def to_table(self, tasks: Iterable[FileScanTask]) -> pa.Table:
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
"""
deletes_per_file = _read_all_delete_files(self._io, tasks)
executor = ExecutorFactory.get_or_create()

def _table_from_scan_task(task: FileScanTask) -> pa.Table:
batches = list(self._record_batches_from_scan_tasks_and_deletes([task], deletes_per_file))
if len(batches) > 0:
return pa.Table.from_batches(batches)
else:
return None

futures = [
executor.submit(
_table_from_scan_task,
task,
)
for task in tasks
]
total_row_count = 0
# for consistent ordering, we need to maintain future order
futures_index = {f: i for i, f in enumerate(futures)}
completed_futures: SortedList[Future[pa.Table]] = SortedList(iterable=[], key=lambda f: futures_index[f])
for future in concurrent.futures.as_completed(futures):
completed_futures.add(future)
if table_result := future.result():
total_row_count += len(table_result)
# stop early if limit is satisfied
if self._limit is not None and total_row_count >= self._limit:
break

# by now, we've either completed all tasks or satisfied the limit
if self._limit is not None:
_ = [f.cancel() for f in futures if not f.done()]

tables = [f.result() for f in completed_futures if f.result()]

arrow_schema = schema_to_pyarrow(self._projected_schema, include_field_ids=False)

if len(tables) < 1:
batches = self.to_record_batches(tasks)
try:
first_batch = next(batches)
except StopIteration:
# Empty
return pa.Table.from_batches([], schema=arrow_schema)

result = pa.concat_tables(tables, promote_options="permissive")
result = pa.Table.from_batches(itertools.chain([first_batch], batches))

if property_as_bool(self._io.properties, PYARROW_USE_LARGE_TYPES_ON_READ, False):
deprecation_message(
Expand All @@ -1620,9 +1586,6 @@ def _table_from_scan_task(task: FileScanTask) -> pa.Table:
)
result = result.cast(arrow_schema)

if self._limit is not None:
return result.slice(0, self._limit)

return result

def to_record_batches(self, tasks: Iterable[FileScanTask]) -> Iterator[pa.RecordBatch]:
Expand All @@ -1644,7 +1607,32 @@ def to_record_batches(self, tasks: Iterable[FileScanTask]) -> Iterator[pa.Record
ValueError: When a field type in the file cannot be projected to the schema type
"""
deletes_per_file = _read_all_delete_files(self._io, tasks)
return self._record_batches_from_scan_tasks_and_deletes(tasks, deletes_per_file)

total_row_count = 0
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)

limit_reached = True
break

yield batch
total_row_count += current_batch_size

if limit_reached:
# This break will also cancel all running tasks in the executor
break

def _record_batches_from_scan_tasks_and_deletes(
self, tasks: Iterable[FileScanTask], deletes_per_file: Dict[str, List[ChunkedArray]]
Expand Down
58 changes: 37 additions & 21 deletions pyiceberg/table/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -774,39 +774,55 @@ def upsert(
matched_predicate = upsert_util.create_match_filter(df, join_cols)

# We must use Transaction.table_metadata for the scan. This includes all uncommitted - but relevant - changes.
matched_iceberg_table = DataScan(
matched_iceberg_record_batches = DataScan(
table_metadata=self.table_metadata,
io=self._table.io,
row_filter=matched_predicate,
case_sensitive=case_sensitive,
).to_arrow()
).to_arrow_batch_reader()

update_row_cnt = 0
insert_row_cnt = 0
batches_to_overwrite = []
overwrite_predicates = []
rows_to_insert = df

if when_matched_update_all:
# function get_rows_to_update is doing a check on non-key columns to see if any of the values have actually changed
# we don't want to do just a blanket overwrite for matched rows if the actual non-key column data hasn't changed
# this extra step avoids unnecessary IO and writes
rows_to_update = upsert_util.get_rows_to_update(df, matched_iceberg_table, join_cols)
for batch in matched_iceberg_record_batches:
rows = pa.Table.from_batches([batch])

update_row_cnt = len(rows_to_update)
if when_matched_update_all:
# function get_rows_to_update is doing a check on non-key columns to see if any of the values have actually changed
# we don't want to do just a blanket overwrite for matched rows if the actual non-key column data hasn't changed
# this extra step avoids unnecessary IO and writes
rows_to_update = upsert_util.get_rows_to_update(df, rows, join_cols)

if len(rows_to_update) > 0:
# build the match predicate filter
overwrite_mask_predicate = upsert_util.create_match_filter(rows_to_update, join_cols)
if len(rows_to_update) > 0:
# build the match predicate filter
overwrite_mask_predicate = upsert_util.create_match_filter(rows_to_update, join_cols)

self.overwrite(rows_to_update, overwrite_filter=overwrite_mask_predicate)
batches_to_overwrite.append(rows_to_update)
overwrite_predicates.append(overwrite_mask_predicate)

if when_not_matched_insert_all:
expr_match = upsert_util.create_match_filter(matched_iceberg_table, join_cols)
expr_match_bound = bind(self.table_metadata.schema(), expr_match, case_sensitive=case_sensitive)
expr_match_arrow = expression_to_pyarrow(expr_match_bound)
rows_to_insert = df.filter(~expr_match_arrow)
if when_not_matched_insert_all:
expr_match = upsert_util.create_match_filter(rows, join_cols)
expr_match_bound = bind(self.table_metadata.schema(), expr_match, case_sensitive=case_sensitive)
expr_match_arrow = expression_to_pyarrow(expr_match_bound)

insert_row_cnt = len(rows_to_insert)
# Filter rows per batch.
rows_to_insert = rows_to_insert.filter(~expr_match_arrow)

if insert_row_cnt > 0:
update_row_cnt = 0
insert_row_cnt = 0

if batches_to_overwrite:
rows_to_update = pa.concat_tables(batches_to_overwrite)
update_row_cnt = len(rows_to_update)
self.overwrite(
rows_to_update,
overwrite_filter=Or(*overwrite_predicates) if len(overwrite_predicates) > 1 else overwrite_predicates[0],
)

if when_not_matched_insert_all:
insert_row_cnt = len(rows_to_insert)
if rows_to_insert:
self.append(rows_to_insert)

return UpsertResult(rows_updated=update_row_cnt, rows_inserted=insert_row_cnt)
Expand Down