Skip to content
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

Python: Support serverless environments #8015

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
47 changes: 33 additions & 14 deletions python/pyiceberg/io/pyarrow.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,20 +24,19 @@
"""
from __future__ import annotations

import multiprocessing
import os
from abc import ABC, abstractmethod
from concurrent.futures import Executor
from functools import lru_cache, singledispatch
from itertools import chain
from multiprocessing.pool import ThreadPool
from multiprocessing.sharedctypes import Synchronized
from typing import (
TYPE_CHECKING,
Any,
Callable,
Dict,
Generic,
Iterable,
Iterator,
List,
Optional,
Set,
Expand Down Expand Up @@ -124,6 +123,7 @@
TimeType,
UUIDType,
)
from pyiceberg.utils.concurrent import DynamicManagedExecutor, Synchronized
from pyiceberg.utils.singleton import Singleton

if TYPE_CHECKING:
Expand Down Expand Up @@ -521,6 +521,10 @@ def _construct_fragment(fs: FileSystem, data_file: DataFile, file_format_kwargs:
return _get_file_format(data_file.file_format, **file_format_kwargs).make_fragment(path, fs)


def _starmap_read_deletes(args: Tuple[FileSystem, DataFile]) -> Dict[str, pa.ChunkedArray]:
mr-brobot marked this conversation as resolved.
Show resolved Hide resolved
return _read_deletes(*args)


def _read_deletes(fs: FileSystem, data_file: DataFile) -> Dict[str, pa.ChunkedArray]:
delete_fragment = _construct_fragment(
fs, data_file, file_format_kwargs={"dictionary_columns": ("file_path",), "pre_buffer": True, "buffer_size": ONE_MEGABYTE}
Expand Down Expand Up @@ -725,6 +729,22 @@ def primitive(self, primitive: pa.DataType) -> IcebergType:
raise TypeError(f"Unsupported type: {primitive}")


def _starmap_task_to_table(
mr-brobot marked this conversation as resolved.
Show resolved Hide resolved
args: Tuple[
FileSystem,
FileScanTask,
BooleanExpression,
Schema,
Set[int],
Optional[List[ChunkedArray]],
bool,
Synchronized[int],
Optional[int],
]
) -> Optional[pa.Table]:
return _task_to_table(*args)


def _task_to_table(
fs: FileSystem,
task: FileScanTask,
Expand Down Expand Up @@ -799,7 +819,7 @@ def _task_to_table(
arrow_table = fragment_scanner.to_table()

if limit:
with rows_counter.get_lock():
with rows_counter:
if rows_counter.value >= limit:
return None
rows_counter.value += len(arrow_table)
Expand All @@ -811,12 +831,12 @@ def _task_to_table(
return None


def _read_all_delete_files(fs: FileSystem, pool: ThreadPool, tasks: Iterable[FileScanTask]) -> Dict[str, List[ChunkedArray]]:
def _read_all_delete_files(fs: FileSystem, executor: Executor, tasks: Iterable[FileScanTask]) -> Dict[str, List[ChunkedArray]]:
deletes_per_file: Dict[str, List[ChunkedArray]] = {}
unique_deletes = set(chain.from_iterable([task.delete_files for task in tasks]))
if len(unique_deletes) > 0:
deletes_per_files: List[Dict[str, ChunkedArray]] = pool.starmap(
func=_read_deletes, iterable=[(fs, delete) for delete in unique_deletes]
deletes_per_files: Iterator[Dict[str, ChunkedArray]] = executor.map(
_starmap_read_deletes, [(fs, delete) for delete in unique_deletes]
)
for delete in deletes_per_files:
for file, arr in delete.items():
Expand Down Expand Up @@ -870,15 +890,14 @@ def project_table(
id for id in projected_schema.field_ids if not isinstance(projected_schema.find_type(id), (MapType, ListType))
}.union(extract_field_ids(bound_row_filter))

rows_counter = multiprocessing.Value("i", 0)

with ThreadPool() as pool:
deletes_per_file = _read_all_delete_files(fs, pool, tasks)
with DynamicManagedExecutor() as executor:
rows_counter = executor.synchronized(0)
deletes_per_file = _read_all_delete_files(fs, executor, tasks)
tables = [
table
for table in pool.starmap(
func=_task_to_table,
iterable=[
for table in executor.map(
_starmap_task_to_table,
[
(
fs,
task,
Expand Down
16 changes: 11 additions & 5 deletions python/pyiceberg/table/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
from enum import Enum
from functools import cached_property
from itertools import chain
from multiprocessing.pool import ThreadPool
from typing import (
TYPE_CHECKING,
Any,
Expand Down Expand Up @@ -70,6 +69,7 @@
KeyDefaultDict,
Properties,
)
from pyiceberg.utils.concurrent import DynamicExecutor

if TYPE_CHECKING:
import pandas as pd
Expand Down Expand Up @@ -639,6 +639,12 @@ def __init__(
self.length = length or data_file.file_size_in_bytes


def _starmap_open_manifest(
mr-brobot marked this conversation as resolved.
Show resolved Hide resolved
args: Tuple[FileIO, ManifestFile, Callable[[DataFile], bool], Callable[[DataFile], bool]]
) -> List[ManifestEntry]:
return _open_manifest(*args)


def _open_manifest(
io: FileIO,
manifest: ManifestFile,
Expand Down Expand Up @@ -773,11 +779,11 @@ def plan_files(self) -> Iterable[FileScanTask]:
data_entries: List[ManifestEntry] = []
positional_delete_entries = SortedList(key=lambda entry: entry.data_sequence_number or INITIAL_SEQUENCE_NUMBER)

with ThreadPool() as pool:
with DynamicExecutor() as executor:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is awesome. I have one more suggestion. What do you think of being able to set the executor through the config?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure! What were you thinking for the config key and accepted values? Currently, this uses PYICEBERG_CONCURRENCY_MODE where possible values are None, thread, or process.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this works for me. Can you also add this to the docs under python/mkdocs/?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

None doesn't seem to work on my end:

export PYICEBERG_CONCURRENCY_MODE=None   
python3 /tmp/vo.py               
Traceback (most recent call last):
  File "/tmp/vo.py", line 1, in <module>
    from pyiceberg.catalog import load_catalog
  File "/Users/fokkodriesprong/Desktop/iceberg/python/pyiceberg/catalog/__init__.py", line 42, in <module>
    from pyiceberg.serializers import ToOutputFile
  File "/Users/fokkodriesprong/Desktop/iceberg/python/pyiceberg/serializers.py", line 26, in <module>
    from pyiceberg.table.metadata import TableMetadata, TableMetadataUtil
  File "/Users/fokkodriesprong/Desktop/iceberg/python/pyiceberg/table/__init__.py", line 72, in <module>
    from pyiceberg.utils.concurrent import DynamicExecutor
  File "/Users/fokkodriesprong/Desktop/iceberg/python/pyiceberg/utils/concurrent.py", line 146, in <module>
    concurrency_mode = _concurrency_mode()
  File "/Users/fokkodriesprong/Desktop/iceberg/python/pyiceberg/utils/concurrent.py", line 143, in _concurrency_mode
    raise ValueError(f"Invalid concurrency mode: {mode}")
ValueError: Invalid concurrency mode: None

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Apologies! I meant unset PYICEBERG_CONCURRENCY_MODE, which comes across as None in pyiceberg.utils.concurrent._concurrency_mode:

def _concurrency_mode() -> ConcurrencyMode:
    mode = Config().config.get("concurrency-mode")

    if mode is None:
        return "thread"

    if mode in ("thread", "process"):
        return mode  # type: ignore

    raise ValueError(f"Invalid concurrency mode: {mode}")

Will add some documentation!

for manifest_entry in chain(
*pool.starmap(
func=_open_manifest,
iterable=[
*executor.map(
_starmap_open_manifest,
[
(
io,
manifest,
Expand Down
153 changes: 153 additions & 0 deletions python/pyiceberg/utils/concurrent.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,153 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing,
# software distributed under the License is distributed on an
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
# pylint: disable=redefined-outer-name,arguments-renamed,fixme
"""Concurrency concepts that adapt to the shared memory support in the current runtime.

Performance-optimized concurrency in Python prefers `multiprocessing` to avoid the global
interpreter lock. However, this requires shared memory provided via mount at `/dev/shm`. This
is not provided in serverless runtimes. In environments where multiprocessing is not supported,
we fall back to multithreading.
mr-brobot marked this conversation as resolved.
Show resolved Hide resolved
"""
import logging
import multiprocessing
import multiprocessing.managers
import multiprocessing.synchronize
import os
import threading
from concurrent.futures import Executor, ProcessPoolExecutor, ThreadPoolExecutor
from contextlib import AbstractContextManager
from typing import (
Any,
Generic,
Type,
TypeVar,
Union,
)

from typing_extensions import Self

from pyiceberg.typedef import RecursiveDict
from pyiceberg.utils.config import Config

logger = logging.getLogger(__name__)

T = TypeVar("T")


class Synchronized(Generic[T], AbstractContextManager): # type: ignore
"""A context manager that provides concurrency-safe access to a value."""

value: T
lock: threading.Lock

def __init__(self, value: T, lock: threading.Lock):
super().__init__()
self.value = value
self.lock = lock

def __enter__(self) -> T:
"""Acquires a lock, allowing access to the wrapped value."""
self.lock.acquire()
return self.value

def __exit__(self, exc_type: Any, exc_value: Any, traceback: Any) -> None:
"""Releases the lock, allowing other threads to access the value."""
self.lock.release()


class ManagedExecutor(Executor):
"""An executor that provides synchronization."""

def synchronized(self, value: T) -> Synchronized[T]:
raise NotImplementedError


class ManagedThreadPoolExecutor(ThreadPoolExecutor, ManagedExecutor):
"""A thread pool executor that provides synchronization."""

def __enter__(self) -> Self:
"""Returns the executor itself as a context manager."""
super().__enter__()
return self

def synchronized(self, value: T) -> Synchronized[T]:
lock = threading.Lock()
return Synchronized(value, lock)


class ManagedProcessPoolExecutor(ProcessPoolExecutor, ManagedExecutor):
"""A process pool executor provides synchronization."""

manager: multiprocessing.managers.SyncManager

def __init__(self) -> None:
super().__init__()
self.manager = multiprocessing.Manager()

def __enter__(self) -> Self:
"""Returns the executor itself as a context manager."""
self.manager.__enter__()
super().__enter__()
return self

def __exit__(self, exc_type: Any, exc_value: Any, traceback: Any) -> None:
"""Exits the executor and the manager."""
super().__exit__(exc_type, exc_value, traceback)
self.manager.__exit__(exc_type, exc_value, traceback)

def synchronized(self, value: T) -> Synchronized[T]:
lock = self.manager.Lock()
return Synchronized(value, lock)


def _get_executor_class(concurrency_mode: Union[str, RecursiveDict, None], shm_avail: bool) -> Type[Executor]:
"""Returns the executor class for the given concurrency mode."""
if concurrency_mode == "process":
return ProcessPoolExecutor
if concurrency_mode == "thread":
return ThreadPoolExecutor
if concurrency_mode is None and shm_avail:
return ProcessPoolExecutor
if concurrency_mode is None and not shm_avail:
logger.debug("Falling back to thread pool executor")
return ThreadPoolExecutor

raise ValueError(f"Invalid concurrency mode: {concurrency_mode}")


def _get_managed_executor_class(concurrency_mode: Union[str, RecursiveDict, None], shm_avail: bool) -> Type[ManagedExecutor]:
"""Returns the managed executor class for the given concurrency mode."""
if concurrency_mode == "process":
return ManagedProcessPoolExecutor
if concurrency_mode == "thread":
return ManagedThreadPoolExecutor
if concurrency_mode is None and shm_avail:
return ManagedProcessPoolExecutor
if concurrency_mode is None and not shm_avail:
logger.debug("Falling back to managed thread pool executor")
return ManagedThreadPoolExecutor

raise ValueError(f"Invalid concurrency mode: {concurrency_mode}")


_shm_avail = os.path.exists("/dev/shm")
mr-brobot marked this conversation as resolved.
Show resolved Hide resolved

concurrency_mode: Union[str, RecursiveDict, None] = Config().config.get("concurrency-mode")

DynamicExecutor: Type[Executor] = _get_executor_class(concurrency_mode, _shm_avail)

DynamicManagedExecutor: Type[ManagedExecutor] = _get_managed_executor_class(concurrency_mode, _shm_avail)