-
Notifications
You must be signed in to change notification settings - Fork 6k
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
[core] Support generators to allow tasks to return a dynamic number of objects #28291
Changes from 10 commits
5636eb9
3457a46
031640b
e59dd65
775ff3e
b316b9f
2d0f5d4
faf64cb
5d92940
7c69b32
c1ecf26
422e4b6
62b427c
636954c
0b73319
450d428
a0bca43
6511a04
7ea1404
6e4663a
7bba5b7
c31b1c4
167c6be
44d9d85
39ba48f
805a088
3360d63
2ed252a
3a811dc
285d98d
7a86f19
b045d34
7b3cf27
92a8491
492b95e
07ca73d
c196c67
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 |
---|---|---|
|
@@ -159,6 +159,19 @@ OPTIMIZED = __OPTIMIZE__ | |
|
||
logger = logging.getLogger(__name__) | ||
|
||
|
||
class ObjectRefGenerator: | ||
def __init__(self, refs): | ||
# TODO(swang): As an optimization, can also store the generator | ||
# ObjectID so that we don't need to keep individual ref counts for the | ||
# inner ObjectRefs. | ||
self.refs = refs | ||
stephanie-wang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
||
def __iter__(self): | ||
while self.refs: | ||
yield self.refs.pop(0) | ||
|
||
|
||
cdef int check_status(const CRayStatus& status) nogil except -1: | ||
if status.ok(): | ||
return 0 | ||
|
@@ -531,6 +544,7 @@ cdef c_bool determine_if_retryable( | |
|
||
|
||
cdef execute_task( | ||
const CAddress &caller_address, | ||
CTaskType task_type, | ||
const c_string name, | ||
const CRayFunction &ray_function, | ||
|
@@ -541,6 +555,7 @@ cdef execute_task( | |
const c_string debugger_breakpoint, | ||
const c_string serialized_retry_exception_allowlist, | ||
c_vector[shared_ptr[CRayObject]] *returns, | ||
c_vector[c_pair[CObjectID, shared_ptr[CRayObject]]] *dynamic_returns, | ||
c_bool *is_retryable_error, | ||
# This parameter is only used for actor creation task to define | ||
# the concurrency groups of this actor. | ||
|
@@ -558,6 +573,9 @@ cdef execute_task( | |
JobID job_id = core_worker.get_current_job_id() | ||
TaskID task_id = core_worker.get_current_task_id() | ||
CFiberEvent task_done_event | ||
CObjectID dynamic_return_id | ||
c_vector[CObjectID] dynamic_return_ids | ||
stephanie-wang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
c_vector[shared_ptr[CRayObject]] dynamic_return_ptrs | ||
|
||
# Automatically restrict the GPUs available to this task. | ||
ray._private.utils.set_cuda_visible_devices(ray.get_gpu_ids()) | ||
|
@@ -752,7 +770,7 @@ cdef execute_task( | |
core_worker.get_current_task_id()), | ||
exc_info=True) | ||
raise e | ||
if c_return_ids.size() == 1: | ||
if c_return_ids.size() == 1 and not inspect.isgenerator(outputs): | ||
# If there is only one return specified, we should return | ||
# all return values as a single object. | ||
outputs = (outputs,) | ||
|
@@ -786,8 +804,33 @@ cdef execute_task( | |
|
||
# Store the outputs in the object store. | ||
with core_worker.profile_event(b"task:store_outputs"): | ||
num_returns = c_return_ids.size() | ||
if num_returns == 1 and inspect.isgenerator(outputs): | ||
# If the task has a single return value and it is a | ||
# generator, first store all outputs yielded by the | ||
# generator. We will assign their ObjectIDs dynamically. | ||
# TODO(swang): Fix reconstruction case where generator | ||
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. Will this be resolved in this PR? 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. Oh it might be fixed already... let me check :D |
||
# returned a single value. | ||
core_worker.store_task_outputs( | ||
caller_address, worker, outputs, NULL, | ||
NULL, | ||
c_return_ids[0], | ||
dynamic_returns) | ||
dynamic_refs = [] | ||
for idx in range(dynamic_returns.size()): | ||
dynamic_refs.append(ObjectRef( | ||
dynamic_returns[0][idx].first.Binary(), | ||
caller_address.SerializeAsString(), | ||
skip_adding_local_ref=True | ||
)) | ||
# Swap out the generator for an ObjectRef generator. | ||
outputs = (ObjectRefGenerator(dynamic_refs), ) | ||
|
||
core_worker.store_task_outputs( | ||
worker, outputs, c_return_ids, returns) | ||
caller_address, worker, outputs, &c_return_ids, | ||
returns, | ||
CObjectID.Nil(), | ||
NULL) | ||
except Exception as error: | ||
# If the debugger is enabled, drop into the remote pdb here. | ||
if "RAY_PDB" in os.environ: | ||
|
@@ -812,7 +855,21 @@ cdef execute_task( | |
for _ in range(c_return_ids.size()): | ||
errors.append(failure_object) | ||
core_worker.store_task_outputs( | ||
worker, errors, c_return_ids, returns) | ||
caller_address, worker, errors, &c_return_ids, | ||
returns, | ||
CObjectID.Nil(), | ||
NULL) | ||
if not dynamic_returns[0].empty(): | ||
stephanie-wang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
# We generated dynamic objects during the first execution. | ||
# Store the error for these objects too. | ||
dynamic_errors = [] | ||
for _ in range(dynamic_returns[0].size()): | ||
dynamic_errors.append(failure_object) | ||
core_worker.store_task_outputs( | ||
caller_address, worker, dynamic_errors, NULL, NULL, | ||
stephanie-wang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
c_return_ids[0], | ||
dynamic_returns) | ||
|
||
ray._private.utils.push_error_to_driver( | ||
worker, | ||
ray_constants.TASK_PUSH_ERROR, | ||
|
@@ -841,6 +898,7 @@ cdef shared_ptr[LocalMemoryBuffer] ray_error_to_memory_buf(ray_error): | |
<uint8_t*>py_bytes, len(py_bytes), True) | ||
|
||
cdef CRayStatus task_execution_handler( | ||
const CAddress &caller_address, | ||
CTaskType task_type, | ||
const c_string task_name, | ||
const CRayFunction &ray_function, | ||
|
@@ -851,6 +909,7 @@ cdef CRayStatus task_execution_handler( | |
const c_string debugger_breakpoint, | ||
const c_string serialized_retry_exception_allowlist, | ||
c_vector[shared_ptr[CRayObject]] *returns, | ||
c_vector[c_pair[CObjectID, shared_ptr[CRayObject]]] *dynamic_returns, | ||
shared_ptr[LocalMemoryBuffer] &creation_task_exception_pb_bytes, | ||
c_bool *is_retryable_error, | ||
const c_vector[CConcurrencyGroup] &defined_concurrency_groups, | ||
|
@@ -860,11 +919,13 @@ cdef CRayStatus task_execution_handler( | |
try: | ||
# The call to execute_task should never raise an exception. If | ||
# it does, that indicates that there was an internal error. | ||
execute_task(task_type, task_name, ray_function, c_resources, | ||
execute_task(caller_address, task_type, task_name, | ||
ray_function, c_resources, | ||
c_args, c_arg_refs, c_return_ids, | ||
debugger_breakpoint, | ||
serialized_retry_exception_allowlist, | ||
returns, | ||
dynamic_returns, | ||
is_retryable_error, | ||
defined_concurrency_groups, | ||
name_of_concurrency_group_to_execute) | ||
|
@@ -1401,6 +1462,7 @@ cdef class CoreWorker: | |
check_status( | ||
CCoreWorkerProcess.GetCoreWorker().SealExisting( | ||
c_object_id, pin_object=False, | ||
generator_id=CObjectID.Nil(), | ||
owner_address=c_owner_address)) | ||
|
||
def put_serialized_object_and_increment_local_ref(self, serialized_object, | ||
|
@@ -1459,6 +1521,7 @@ cdef class CoreWorker: | |
check_status( | ||
CCoreWorkerProcess.GetCoreWorker().SealExisting( | ||
c_object_id, pin_object=False, | ||
generator_id=CObjectID.Nil(), | ||
owner_address=move(c_owner_address))) | ||
|
||
return c_object_id.Binary() | ||
|
@@ -2060,6 +2123,7 @@ cdef class CoreWorker: | |
serialized_object_status)) | ||
|
||
cdef store_task_output(self, serialized_object, const CObjectID &return_id, | ||
const CObjectID &generator_id, | ||
size_t data_size, shared_ptr[CBuffer] &metadata, | ||
const c_vector[CObjectID] &contained_id, | ||
int64_t *task_output_inlined_bytes, | ||
|
@@ -2086,38 +2150,74 @@ cdef class CoreWorker: | |
with nogil: | ||
check_status( | ||
CCoreWorkerProcess.GetCoreWorker().SealReturnObject( | ||
return_id, return_ptr[0])) | ||
return_id, return_ptr[0], generator_id)) | ||
return True | ||
else: | ||
with nogil: | ||
success = (CCoreWorkerProcess.GetCoreWorker() | ||
.PinExistingReturnObject(return_id, return_ptr)) | ||
.PinExistingReturnObject( | ||
return_id, return_ptr, generator_id)) | ||
return success | ||
|
||
cdef store_task_outputs( | ||
self, worker, outputs, const c_vector[CObjectID] return_ids, | ||
c_vector[shared_ptr[CRayObject]] *returns): | ||
cdef store_task_outputs(self, | ||
stephanie-wang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
const CAddress &caller_address, | ||
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 didn't find where this is used. 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. It gets passed to 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.
Seems 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. Ah yes, you're right, I'm pulling it directly from CoreWorker state now. Will remove! |
||
worker, outputs, | ||
const c_vector[CObjectID] *static_return_ids, | ||
c_vector[shared_ptr[CRayObject]] *returns, | ||
const CObjectID &generator_id, | ||
c_vector[c_pair[CObjectID, shared_ptr[CRayObject]]] | ||
*dynamic_returns): | ||
cdef: | ||
CObjectID return_id | ||
size_t data_size | ||
shared_ptr[CBuffer] metadata | ||
c_vector[CObjectID] contained_id | ||
int64_t task_output_inlined_bytes | ||
|
||
if return_ids.size() == 0: | ||
int64_t num_returns = -1 | ||
shared_ptr[CRayObject] *return_ptr | ||
if static_return_ids is not NULL: | ||
num_returns = static_return_ids[0].size() | ||
elif dynamic_returns is not NULL and dynamic_returns.size() > 0: | ||
num_returns = dynamic_returns[0].size() | ||
|
||
if num_returns == 0: | ||
return | ||
|
||
n_returns = return_ids.size() | ||
returns.resize(n_returns) | ||
task_output_inlined_bytes = 0 | ||
i = -1 | ||
outputs = list(outputs) | ||
stephanie-wang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
for i, output in enumerate(outputs): | ||
if i >= n_returns: | ||
if num_returns >= 0 and i >= num_returns: | ||
raise ValueError( | ||
"Task returned more than num_returns={} objects.".format( | ||
n_returns)) | ||
num_returns)) | ||
if static_return_ids != NULL: | ||
return_id = static_return_ids[0][i] | ||
while returns[0].size() <= i: | ||
returns[0].push_back(shared_ptr[CRayObject]()) | ||
return_ptr = &returns[0][i] | ||
else: | ||
if num_returns == -1: | ||
return_id = (CCoreWorkerProcess.GetCoreWorker() | ||
.AllocateDynamicReturnId()) | ||
dynamic_returns[0].push_back( | ||
c_pair[CObjectID, shared_ptr[CRayObject]]( | ||
return_id, shared_ptr[CRayObject]())) | ||
else: | ||
return_id = dynamic_returns[0][i].first | ||
assert i < dynamic_returns[0].size() | ||
return_ptr = &dynamic_returns[0][i].second | ||
|
||
# TODO(swang): We should only try to create an existing return | ||
# value if there were multiple return values, and we errored while | ||
# trying to create one of them. We should try to delete the first | ||
# value and store the error instead here. | ||
# Skip return values that we already created. | ||
if return_ptr.get() != NULL: | ||
stephanie-wang marked this conversation as resolved.
Show resolved
Hide resolved
|
||
continue | ||
|
||
return_id = return_ids[i] | ||
context = worker.get_serialization_context() | ||
|
||
serialized_object = context.serialize(output) | ||
data_size = serialized_object.total_bytes | ||
metadata_str = serialized_object.metadata | ||
|
@@ -2135,21 +2235,24 @@ cdef class CoreWorker: | |
|
||
if not self.store_task_output( | ||
serialized_object, return_id, | ||
generator_id, | ||
data_size, metadata, contained_id, | ||
&task_output_inlined_bytes, &returns[0][i]): | ||
&task_output_inlined_bytes, return_ptr): | ||
# If the object already exists, but we fail to pin the copy, it | ||
# means the existing copy might've gotten evicted. Try to | ||
# create another copy. | ||
self.store_task_output( | ||
serialized_object, return_id, data_size, metadata, | ||
serialized_object, return_id, | ||
generator_id, | ||
data_size, metadata, | ||
contained_id, &task_output_inlined_bytes, | ||
&returns[0][i]) | ||
return_ptr) | ||
|
||
i += 1 | ||
if i < n_returns: | ||
if i < num_returns: | ||
raise ValueError( | ||
"Task returned {} objects, but num_returns={}.".format( | ||
i, n_returns)) | ||
i, num_returns)) | ||
|
||
cdef c_function_descriptors_to_python( | ||
self, | ||
|
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.
I'm not sure I understand this: don't we still need to ref count the inner ObjectRefs since Generator can be out of scope while we still have inner ObjectRefs?
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.
Yes, that's possible. The TODO is an optimization for cases where we never use the inner ObjectRefs separately from the outer one. Like if you pass the generator to another task, and that task calls ray.get on each inner ref but deletes the refs before it returns. You can sort of "batch" the ref counts in that case by passing the outer ref to the task instead of the deserialized ObjectRefGenerator.