-
Notifications
You must be signed in to change notification settings - Fork 653
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
FIX-#4507: Do not call 'ray.get()' inside of the kernel executing call queues #6633
Changes from all commits
9602752
67b3c98
0bb11b4
746fdbd
fbfc276
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 |
---|---|---|
|
@@ -18,7 +18,11 @@ | |
|
||
from modin.core.dataframe.pandas.partitioning.partition import PandasDataframePartition | ||
from modin.core.execution.ray.common import RayWrapper | ||
from modin.core.execution.ray.common.utils import ObjectIDType, deserialize | ||
dchigarev marked this conversation as resolved.
Show resolved
Hide resolved
|
||
from modin.core.execution.ray.common.utils import ( | ||
ObjectIDType, | ||
deconstruct_call_queue, | ||
reconstruct_call_queue, | ||
) | ||
from modin.logging import get_logger | ||
from modin.pandas.indexing import compute_sliced_len | ||
|
||
|
@@ -97,7 +101,9 @@ def apply(self, func, *args, **kwargs): | |
self._is_debug(log) and log.debug( | ||
f"SUBMIT::_apply_list_of_funcs::{self._identity}" | ||
) | ||
result, length, width, ip = _apply_list_of_funcs.remote(call_queue, data) | ||
result, length, width, ip = _apply_list_of_funcs.remote( | ||
data, *deconstruct_call_queue(call_queue) | ||
) | ||
else: | ||
# We handle `len(call_queue) == 1` in a different way because | ||
# this dramatically improves performance. | ||
|
@@ -128,7 +134,7 @@ def drain_call_queue(self): | |
new_length, | ||
new_width, | ||
self._ip_cache, | ||
) = _apply_list_of_funcs.remote(call_queue, data) | ||
) = _apply_list_of_funcs.remote(data, *deconstruct_call_queue(call_queue)) | ||
else: | ||
# We handle `len(call_queue) == 1` in a different way because | ||
# this dramatically improves performance. | ||
|
@@ -391,16 +397,29 @@ def _apply_func(partition, func, *args, **kwargs): # pragma: no cover | |
|
||
|
||
@ray.remote(num_returns=4) | ||
def _apply_list_of_funcs(call_queue, partition): # pragma: no cover | ||
def _apply_list_of_funcs( | ||
partition, num_funcs, arg_lengths, kw_key_lengths, kw_value_lengths, *futures | ||
): # pragma: no cover | ||
""" | ||
Execute all operations stored in the call queue on the partition in a worker process. | ||
|
||
Parameters | ||
---------- | ||
call_queue : list | ||
A call queue that needs to be executed on the partition. | ||
partition : pandas.DataFrame | ||
A pandas DataFrame the call queue needs to be executed on. | ||
num_funcs : int | ||
The number of functions in the call queue. | ||
arg_lengths : list of ints | ||
The number of positional arguments for each function in the call queue. | ||
kw_key_lengths : list of ints | ||
The number of key-word arguments for each function in the call queue. | ||
kw_value_lengths : 2D list of dict{"len": int, "was_iterable": bool} | ||
Description of keyword arguments for each function. For example, `kw_value_lengths[i][j]` | ||
describes the j-th keyword argument of the i-th function in the call queue. | ||
The describtion contains of the lengths of the argument and whether it's a list at all | ||
(for example, {"len": 1, "was_iterable": False} describes a non-list argument). | ||
*futures : list | ||
A 1D call queue that is result of the ``deconstruct_call_queue()`` function. | ||
|
||
Returns | ||
------- | ||
|
@@ -413,10 +432,10 @@ def _apply_list_of_funcs(call_queue, partition): # pragma: no cover | |
str | ||
The node IP address of the worker process. | ||
""" | ||
for func, f_args, f_kwargs in call_queue: | ||
func = deserialize(func) | ||
args = deserialize(f_args) | ||
kwargs = deserialize(f_kwargs) | ||
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 wonder if we are passing anything at all through this parameter that could lead to a hang? I believe objects created explicitly using 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 think that we're indeed not passing anything like that via |
||
call_queue = reconstruct_call_queue( | ||
dchigarev marked this conversation as resolved.
Show resolved
Hide resolved
|
||
num_funcs, arg_lengths, kw_key_lengths, kw_value_lengths, futures | ||
) | ||
for func, args, kwargs in call_queue: | ||
try: | ||
partition = func(partition, *args, **kwargs) | ||
# Sometimes Arrow forces us to make a copy of an object before we operate on it. We | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe using tuple
(len(value), was_iterable)
here will be faster. Consider only if unboxing/packing performance is not sufficient.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
well, it may be faster for a couple of nanoseconds, but I would prefer readability of the code to saving not sufficient amount of time here