-
Notifications
You must be signed in to change notification settings - Fork 477
feat: Add ArrivalOrder to ArrowScan for bounded-memory concurrent reads #3046
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
Changes from 20 commits
5ab0fd1
c1ece14
70af67f
2474b12
48b332a
b360ae8
4186713
7c415d4
70d5a99
2e044ea
8dcd240
4a0a430
2efdcba
09aad7a
b2ae725
afb244c
03bda3d
19841dc
e06c01a
c38bc76
2d4a67a
de9f3c2
ac8add8
b5cfb78
d93526e
432cd81
84adcfa
1c73ea4
caa079e
a882dd2
039e91b
75ba28b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -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 | ||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -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, TaskOrder, ArrivalOrder, 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 | ||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -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: | ||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -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() | ||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -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_streams: int, | ||||||||||||||||||||||||||||||||||||||||||||
| max_buffered_batches: int = 16, | ||||||||||||||||||||||||||||||||||||||||||||
|
sumedhsakdeo marked this conversation as resolved.
|
||||||||||||||||||||||||||||||||||||||||||||
| ) -> 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) | ||||||||||||||||||||||||||||||||||||||||||||
|
sumedhsakdeo marked this conversation as resolved.
|
||||||||||||||||||||||||||||||||||||||||||||
| 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_streams) 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(): | ||||||||||||||||||||||||||||||||||||||||||||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This could cause a worker to hang when Starting state max_buffered_batches=1, concurrent_files=3. Queue is full (1 item). Workers A, B, and C are all blocked on batch_queue.put(). Timeline
Fix
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good catch. I did have timeout in the previous version of the code, but it was causing performance regression. Exploring few other alternatives like condition variables, more complex but does not result in the bug.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We have a 15% regression if timeout is added. Will need to find another way.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Fixed, PTAL. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't see any issues with the logic, but am still a little worried about leaving an indefinite block in case of bugs. What put timeout did you use for testing? I'm surprised it would cause such a large regression.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I used
The root cause: Instead I went with a drain-until-sentinel approach — the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm assuming when the timeout is hit it looped and called put again? I think the short timeout is the cause the performance regression. Each loop is blocking other threads due to the GIL only executing one thread at time. I don't expect a timeout like 10sec to have the same issue. |
||||||||||||||||||||||||||||||||||||||||||||
| try: | ||||||||||||||||||||||||||||||||||||||||||||
| batch_queue.get_nowait() | ||||||||||||||||||||||||||||||||||||||||||||
| except queue.Empty: | ||||||||||||||||||||||||||||||||||||||||||||
| break | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| class ArrowScan: | ||||||||||||||||||||||||||||||||||||||||||||
| _table_metadata: TableMetadata | ||||||||||||||||||||||||||||||||||||||||||||
| _io: FileIO | ||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -1756,54 +1834,109 @@ 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 = TaskOrder(), | ||||||||||||||||||||||||||||||||||||||||||||
| ) -> 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. | ||||||||||||||||||||||||||||||||||||||||||||
| 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. | ||||||||||||||||||||||||||||||||||||||||||||
| TaskOrder() (default) yields batches one file at a time in task order. | ||||||||||||||||||||||||||||||||||||||||||||
| ArrivalOrder(concurrent_streams=N, max_buffered_batches=M) yields batches | ||||||||||||||||||||||||||||||||||||||||||||
| as they are produced without materializing entire files into memory. | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| 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_streams < 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 instance (TaskOrder() or ArrivalOrder()).") | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| total_row_count = 0 | ||||||||||||||||||||||||||||||||||||||||||||
| 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, batch_size, order.concurrent_streams, order.max_buffered_batches)) | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
| 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_streams: int, | ||||||||||||||||||||||||||||||||||||||||||||
| max_buffered_batches: int = 16, | ||||||||||||||||||||||||||||||||||||||||||||
|
sumedhsakdeo marked this conversation as resolved.
|
||||||||||||||||||||||||||||||||||||||||||||
| ) -> 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]], | ||||||||||||||||||||||||||||||||||||||||||||
| 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]: | ||||||||||||||||||||||||||||||||||||||||||||
|
sumedhsakdeo marked this conversation as resolved.
|
||||||||||||||||||||||||||||||||||||||||||||
| """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: | ||||||||||||||||||||||||||||||||||||||||||||
|
|
@@ -1822,6 +1955,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: | ||||||||||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||||||||||
Uh oh!
There was an error while loading. Please reload this page.