-
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
[1/N] Streaming Generator. Cpp interfaces and implementation #35291
Changes from 11 commits
452ed1f
3ebe327
c140a5c
b83af80
509b311
d0795e5
f8a90f6
0a9169d
05f468a
122b705
7a8fe2c
1942394
3e0212e
aaa0582
37c3bdd
d350b5d
9ed05d9
e2f1980
5dc6b98
1ff672a
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 |
---|---|---|
|
@@ -264,7 +264,7 @@ def read(gen): | |
gen = ray.get( | ||
remote_generator_fn.options(num_returns="dynamic").remote(0, store_in_plasma) | ||
) | ||
assert len(gen) == 0 | ||
assert len(list(gen)) == 0 | ||
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. you can ignore every change from this file. |
||
|
||
# Check that passing as task arg. | ||
gen = remote_generator_fn.options(num_returns="dynamic").remote(10, store_in_plasma) | ||
|
@@ -284,7 +284,9 @@ def static(num_returns): | |
return list(range(num_returns)) | ||
|
||
with pytest.raises(ray.exceptions.RayTaskError): | ||
ray.get(static.remote(3)) | ||
gen = ray.get(static.remote(3)) | ||
for ref in gen: | ||
ray.get(ref) | ||
|
||
|
||
def test_dynamic_generator_distributed(ray_start_cluster): | ||
|
@@ -535,7 +537,7 @@ def maybe_empty_generator(exec_counter): | |
|
||
@ray.remote | ||
def check(empty_generator): | ||
return len(empty_generator) == 0 | ||
return len(list(empty_generator)) == 0 | ||
|
||
exec_counter = ExecutionCounter.remote() | ||
gen = maybe_empty_generator.remote(exec_counter) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -49,5 +49,35 @@ std::string GenerateCachedActorName(const std::string &ns, | |
return ns + "-" + actor_name; | ||
} | ||
|
||
void SerializeReturnObject(const ObjectID &object_id, | ||
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 is just a copy/paste from other file. |
||
const std::shared_ptr<RayObject> &return_object, | ||
rpc::ReturnObject *return_object_proto) { | ||
return_object_proto->set_object_id(object_id.Binary()); | ||
|
||
if (!return_object) { | ||
// This should only happen if the local raylet died. Caller should | ||
// retry the task. | ||
RAY_LOG(WARNING) << "Failed to create task return object " << object_id | ||
<< " in the object store, exiting."; | ||
QuickExit(); | ||
} | ||
return_object_proto->set_size(return_object->GetSize()); | ||
if (return_object->GetData() != nullptr && return_object->GetData()->IsPlasmaBuffer()) { | ||
return_object_proto->set_in_plasma(true); | ||
} else { | ||
if (return_object->GetData() != nullptr) { | ||
return_object_proto->set_data(return_object->GetData()->Data(), | ||
return_object->GetData()->Size()); | ||
} | ||
if (return_object->GetMetadata() != nullptr) { | ||
return_object_proto->set_metadata(return_object->GetMetadata()->Data(), | ||
return_object->GetMetadata()->Size()); | ||
} | ||
} | ||
for (const auto &nested_ref : return_object->GetNestedRefs()) { | ||
return_object_proto->add_nested_inlined_refs()->CopyFrom(nested_ref); | ||
} | ||
} | ||
|
||
} // namespace core | ||
} // namespace ray |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2599,6 +2599,9 @@ Status CoreWorker::ExecuteTask( | |
dynamic_return_objects = NULL; | ||
} else if (task_spec.AttemptNumber() > 0) { | ||
for (const auto &dynamic_return_id : task_spec.DynamicReturnIds()) { | ||
// Increase the put index so that when the generator creates a new obj | ||
// the object id won't conflict. | ||
worker_context_.GetNextPutIndex(); | ||
dynamic_return_objects->push_back( | ||
std::make_pair<>(dynamic_return_id, std::shared_ptr<RayObject>())); | ||
RAY_LOG(DEBUG) << "Re-executed task " << task_spec.TaskId() | ||
|
@@ -2815,6 +2818,56 @@ ObjectID CoreWorker::AllocateDynamicReturnId() { | |
return return_id; | ||
} | ||
|
||
Status CoreWorker::ReportIntermediateTaskReturn( | ||
const std::pair<ObjectID, std::shared_ptr<RayObject>> &dynamic_return_object, | ||
const ObjectID &generator_id, | ||
const rpc::Address &caller_address, | ||
int64_t idx, | ||
bool finished) { | ||
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. Is it true that when finished is True, dynamic_return_object must be 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. Maybe we can remove finished flag and replying on dynamic_return_objects being Nil? 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. Discussed offline. I decided to just add else that checks finish == true when object id == nil. |
||
RAY_LOG(DEBUG) << "Write the object ref stream, index: " << idx | ||
<< " finished: " << finished << ", id: " << dynamic_return_object.first; | ||
rpc::ReportIntermediateTaskReturnRequest request; | ||
request.mutable_worker_addr()->CopyFrom(rpc_address_); | ||
request.set_idx(idx); | ||
request.set_finished(finished); | ||
request.set_generator_id(generator_id.Binary()); | ||
auto client = core_worker_client_pool_->GetOrConnect(caller_address); | ||
|
||
if (!dynamic_return_object.first.IsNil()) { | ||
RAY_CHECK_EQ(finished, false); | ||
auto return_object_proto = request.add_dynamic_return_objects(); | ||
SerializeReturnObject( | ||
dynamic_return_object.first, dynamic_return_object.second, return_object_proto); | ||
std::vector<ObjectID> deleted; | ||
// When we allocate a dynamic return ID (AllocateDynamicReturnId), | ||
// we borrow the object. When the object value is allocatd, the | ||
// memory store is updated. We should clear borrowers and memory store | ||
// here. | ||
ReferenceCounter::ReferenceTableProto borrowed_refs; | ||
reference_counter_->PopAndClearLocalBorrowers( | ||
{dynamic_return_object.first}, &borrowed_refs, &deleted); | ||
memory_store_->Delete(deleted); | ||
} | ||
|
||
client->ReportIntermediateTaskReturn( | ||
request, | ||
[](const Status &status, const rpc::ReportIntermediateTaskReturnReply &reply) { | ||
if (!status.ok()) { | ||
// TODO(sang): Handle network error more gracefully. | ||
RAY_LOG(ERROR) << "Failed to send the object ref."; | ||
} | ||
}); | ||
return Status::OK(); | ||
} | ||
|
||
void CoreWorker::HandleReportIntermediateTaskReturn( | ||
rpc::ReportIntermediateTaskReturnRequest request, | ||
rpc::ReportIntermediateTaskReturnReply *reply, | ||
rpc::SendReplyCallback send_reply_callback) { | ||
task_manager_->HandleReportIntermediateTaskReturn(request); | ||
send_reply_callback(Status::OK(), nullptr, nullptr); | ||
} | ||
|
||
std::vector<rpc::ObjectReference> CoreWorker::ExecuteTaskLocalMode( | ||
const TaskSpecification &task_spec, const ActorID &actor_id) { | ||
auto resource_ids = std::make_shared<ResourceMappingType>(); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -239,6 +239,36 @@ void ReferenceCounter::AddDynamicReturn(const ObjectID &object_id, | |
AddNestedObjectIdsInternal(generator_id, {object_id}, owner_address); | ||
} | ||
|
||
void ReferenceCounter::OwnDynamicStreamingTaskReturnRef(const ObjectID &object_id, | ||
rkooo567 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
const ObjectID &generator_id) { | ||
absl::MutexLock lock(&mutex_); | ||
// NOTE: The upper layer (the layer that manges the object ref stream) | ||
// should make sure the generator ref is not GC'ed when the | ||
rkooo567 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
// | ||
auto outer_it = object_id_refs_.find(generator_id); | ||
if (outer_it == object_id_refs_.end()) { | ||
// Generator object already went out of scope. | ||
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. add warning? 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. Actually I think we don't need a warning here. It can happen under normal circumstance (e.g., generator went out of scope before task finishes). I instead add a debug message! |
||
// It means the generator is already GC'ed. No need to | ||
// update the reference. | ||
return; | ||
} | ||
RAY_LOG(DEBUG) << "Adding dynamic return " << object_id | ||
<< " contained in generator object " << generator_id; | ||
RAY_CHECK(outer_it->second.owned_by_us); | ||
RAY_CHECK(outer_it->second.owner_address.has_value()); | ||
rpc::Address owner_address(outer_it->second.owner_address.value()); | ||
// We add a local reference here. The ref removal will be handled | ||
// by the ObjectRefStream. | ||
RAY_UNUSED(AddOwnedObjectInternal(object_id, | ||
{}, | ||
owner_address, | ||
outer_it->second.call_site, | ||
/*object_size=*/-1, | ||
outer_it->second.is_reconstructable, | ||
/*add_local_ref=*/true, | ||
absl::optional<NodeID>())); | ||
} | ||
|
||
bool ReferenceCounter::AddOwnedObjectInternal( | ||
const ObjectID &object_id, | ||
const std::vector<ObjectID> &inner_ids, | ||
|
@@ -382,7 +412,7 @@ void ReferenceCounter::UpdateSubmittedTaskReferences( | |
std::vector<ObjectID> *deleted) { | ||
absl::MutexLock lock(&mutex_); | ||
for (const auto &return_id : return_ids) { | ||
UpdateObjectPendingCreation(return_id, true); | ||
UpdateObjectPendingCreationInternal(return_id, true); | ||
} | ||
for (const ObjectID &argument_id : argument_ids_to_add) { | ||
RAY_LOG(DEBUG) << "Increment ref count for submitted task argument " << argument_id; | ||
|
@@ -411,7 +441,7 @@ void ReferenceCounter::UpdateResubmittedTaskReferences( | |
const std::vector<ObjectID> return_ids, const std::vector<ObjectID> &argument_ids) { | ||
absl::MutexLock lock(&mutex_); | ||
for (const auto &return_id : return_ids) { | ||
UpdateObjectPendingCreation(return_id, true); | ||
UpdateObjectPendingCreationInternal(return_id, true); | ||
} | ||
for (const ObjectID &argument_id : argument_ids) { | ||
auto it = object_id_refs_.find(argument_id); | ||
|
@@ -433,7 +463,7 @@ void ReferenceCounter::UpdateFinishedTaskReferences( | |
std::vector<ObjectID> *deleted) { | ||
absl::MutexLock lock(&mutex_); | ||
for (const auto &return_id : return_ids) { | ||
UpdateObjectPendingCreation(return_id, false); | ||
UpdateObjectPendingCreationInternal(return_id, false); | ||
} | ||
// Must merge the borrower refs before decrementing any ref counts. This is | ||
// to make sure that for serialized IDs, we increment the borrower count for | ||
|
@@ -1278,8 +1308,8 @@ void ReferenceCounter::RemoveObjectLocationInternal(ReferenceTable::iterator it, | |
PushToLocationSubscribers(it); | ||
} | ||
|
||
void ReferenceCounter::UpdateObjectPendingCreation(const ObjectID &object_id, | ||
bool pending_creation) { | ||
void ReferenceCounter::UpdateObjectPendingCreationInternal(const ObjectID &object_id, | ||
bool pending_creation) { | ||
auto it = object_id_refs_.find(object_id); | ||
bool push = false; | ||
if (it != object_id_refs_.end()) { | ||
|
@@ -1439,6 +1469,11 @@ bool ReferenceCounter::IsObjectReconstructable(const ObjectID &object_id, | |
return it->second.is_reconstructable; | ||
} | ||
|
||
void ReferenceCounter::UpdateObjectReady(const ObjectID &object_id) { | ||
absl::MutexLock lock(&mutex_); | ||
UpdateObjectPendingCreationInternal(object_id, /*pending_creation*/ false); | ||
} | ||
|
||
bool ReferenceCounter::IsObjectPendingCreation(const ObjectID &object_id) const { | ||
absl::MutexLock lock(&mutex_); | ||
auto it = object_id_refs_.find(object_id); | ||
|
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.
This will be used by execution path to report intermediate result later.