-
Notifications
You must be signed in to change notification settings - Fork 29.1k
[SPARK-48634][PYTHON][CONNECT] Avoid statically initialize threadpool at ExecutePlanResponseReattachableIterator #46993
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 all commits
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 |
|---|---|---|
|
|
@@ -58,28 +58,41 @@ class ExecutePlanResponseReattachableIterator(Generator): | |
|
|
||
| # Lock to manage the pool | ||
| _lock: ClassVar[RLock] = RLock() | ||
| _release_thread_pool: Optional[ThreadPool] = ThreadPool(os.cpu_count() if os.cpu_count() else 8) | ||
| _release_thread_pool_instance: Optional[ThreadPool] = None | ||
|
|
||
| @classmethod # type: ignore[misc] | ||
| @property | ||
| def _release_thread_pool(cls) -> ThreadPool: | ||
| # Perform a first check outside the critical path. | ||
| if cls._release_thread_pool_instance is not None: | ||
| return cls._release_thread_pool_instance | ||
| with cls._lock: | ||
| if cls._release_thread_pool_instance is None: | ||
| cls._release_thread_pool_instance = ThreadPool( | ||
| os.cpu_count() if os.cpu_count() else 8 | ||
| ) | ||
| return cls._release_thread_pool_instance | ||
|
|
||
| @classmethod | ||
| def shutdown(cls: Type["ExecutePlanResponseReattachableIterator"]) -> None: | ||
| def shutdown_threadpool(cls: Type["ExecutePlanResponseReattachableIterator"]) -> None: | ||
| """ | ||
| When the channel is closed, this method will be called before, to make sure all | ||
| outstanding calls are closed. | ||
| """ | ||
| with cls._lock: | ||
| if cls._release_thread_pool is not None: | ||
| cls._release_thread_pool.close() | ||
| cls._release_thread_pool.join() | ||
| cls._release_thread_pool = None | ||
| if cls._release_thread_pool_instance is not None: | ||
| cls._release_thread_pool.close() # type: ignore[attr-defined] | ||
| cls._release_thread_pool.join() # type: ignore[attr-defined] | ||
| cls._release_thread_pool_instance = None | ||
|
|
||
| @classmethod | ||
| def _initialize_pool_if_necessary(cls: Type["ExecutePlanResponseReattachableIterator"]) -> None: | ||
| def shutdown(self: "ExecutePlanResponseReattachableIterator") -> None: | ||
| """ | ||
| If the processing pool for the release calls is None, initialize the pool exactly once. | ||
| When the channel is closed, this method will be called before, to make sure all | ||
| outstanding calls are closed, and mark this iterator is shutdown. | ||
| """ | ||
| with cls._lock: | ||
| if cls._release_thread_pool is None: | ||
| cls._release_thread_pool = ThreadPool(os.cpu_count() if os.cpu_count() else 8) | ||
| with self._lock: | ||
| self.shutdown_threadpool() | ||
| self._is_shutdown = True | ||
|
|
||
| def __init__( | ||
| self, | ||
|
|
@@ -88,7 +101,7 @@ def __init__( | |
| retrying: Callable[[], Retrying], | ||
| metadata: Iterable[Tuple[str, str]], | ||
| ): | ||
| ExecutePlanResponseReattachableIterator._initialize_pool_if_necessary() | ||
| self._is_shutdown = False | ||
| self._request = request | ||
| self._retrying = retrying | ||
| if request.operation_id: | ||
|
|
@@ -206,8 +219,8 @@ def target() -> None: | |
| except Exception as e: | ||
| warnings.warn(f"ReleaseExecute failed with exception: {e}.") | ||
|
|
||
| if ExecutePlanResponseReattachableIterator._release_thread_pool is not None: | ||
| ExecutePlanResponseReattachableIterator._release_thread_pool.apply_async(target) | ||
| if not self._is_shutdown: | ||
| self._release_thread_pool.apply_async(target) | ||
|
Comment on lines
+222
to
+223
Contributor
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 code is a bit confusing to me, but it might simply because of the way Python works. My understanding was that self only gives me access to instance but not class variables, but _release_thread_pool is an actual class variable.
Member
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. Yeah It actually accesses to the class variable, and it is not either discouraged or encouraged in PEP 8 if I am correctly remembering this. I just used |
||
|
|
||
| def _release_all(self) -> None: | ||
| """ | ||
|
|
@@ -230,8 +243,8 @@ def target() -> None: | |
| except Exception as e: | ||
| warnings.warn(f"ReleaseExecute failed with exception: {e}.") | ||
|
|
||
| if ExecutePlanResponseReattachableIterator._release_thread_pool is not None: | ||
| ExecutePlanResponseReattachableIterator._release_thread_pool.apply_async(target) | ||
| if not self._is_shutdown: | ||
| self._release_thread_pool.apply_async(target) | ||
| self._result_complete = True | ||
|
|
||
| def _call_iter(self, iter_fun: Callable) -> Any: | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.