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
GCS-Based actor management implementation #6763
Changes from all commits
7f5d748
a8b2313
b0d6c80
c354004
4e910c6
2d063b8
2e86e71
561a428
98184b7
adaea18
88b61c1
373ba07
a639ffa
13376ca
786c97e
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 |
---|---|---|
|
@@ -15,14 +15,37 @@ | |
#include "ray/core_worker/transport/direct_task_transport.h" | ||
|
||
#include "ray/core_worker/transport/dependency_resolver.h" | ||
#include "ray/core_worker/transport/direct_actor_transport.h" | ||
|
||
namespace ray { | ||
|
||
Status CoreWorkerDirectTaskSubmitter::SubmitTask(TaskSpecification task_spec) { | ||
RAY_LOG(DEBUG) << "Submit task " << task_spec.TaskId(); | ||
resolver_.ResolveDependencies(task_spec, [this, task_spec]() { | ||
RAY_LOG(DEBUG) << "Task dependencies resolved " << task_spec.TaskId(); | ||
if (actor_create_callback_ && task_spec.IsActorCreationTask()) { | ||
// If gcs actor management is enabled, the actor creation task will be sent to gcs | ||
// server directly after the in-memory dependent objects are resolved. | ||
// For more details please see the protocol of actor management based on gcs. | ||
// https://docs.google.com/document/d/1EAWide-jy05akJp6OMtDn58XOK7bUyruWMia4E-fV28/edit?usp=sharing | ||
auto actor_id = task_spec.ActorCreationId(); | ||
auto task_id = task_spec.TaskId(); | ||
RAY_LOG(INFO) << "Submitting actor creation task to GCS: " << actor_id; | ||
auto status = | ||
actor_create_callback_(task_spec, [this, actor_id, task_id](Status status) { | ||
// If GCS is failed, GcsRpcClient may receive IOError status but it will not | ||
// trigger this callback, because GcsRpcClient has retry logic at the | ||
// bottom. So if this callback is invoked with an error there must be | ||
// something wrong with the protocol of gcs-based actor management. | ||
// So just check `status.ok()` here. | ||
RAY_CHECK_OK(status); | ||
RAY_LOG(INFO) << "Actor creation task submitted to GCS: " << actor_id; | ||
task_finisher_->CompletePendingTask(task_id, rpc::PushTaskReply(), | ||
rpc::Address()); | ||
}); | ||
RAY_CHECK_OK(status); | ||
return; | ||
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 just remembered, we should probably also call back to the TaskManager here, as we do for the non-GCS service path. This is used to indicate that the actor creation task has finished. We can also call the failure handler if the status returned an error, instead of crashing here. There is a good example of what to do here. 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.
GcsRpcClient is a reliable client, it will retry at the bottom and ensure that the message must be delivered to the GCS Server, even if the GCS Server is restarted. So I assert here that status must be OK, if not then there must be a serious bug in the system. 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 understand, I meant that we need to notify the TaskManager that the actor creation task is done. Otherwise, the TaskManager will still think that the creation task is pending. You have to call something like 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.
Okey, I know what you mean, I will fix it. |
||
} | ||
|
||
absl::MutexLock lock(&mu_); | ||
// Note that the dependencies in the task spec are mutated to only contain | ||
// plasma dependencies after ResolveDependencies finishes. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -79,7 +79,7 @@ ServiceBasedActorInfoAccessor::ServiceBasedActorInfoAccessor( | |
ServiceBasedGcsClient *client_impl) | ||
: subscribe_id_(ClientID::FromRandom()), | ||
client_impl_(client_impl), | ||
actor_sub_executor_(client_impl->GetRedisGcsClient().log_based_actor_table()) {} | ||
actor_sub_executor_(client_impl->GetRedisGcsClient().actor_table()) {} | ||
|
||
Status ServiceBasedActorInfoAccessor::GetAll( | ||
std::vector<ActorTableData> *actor_table_data_list) { | ||
|
@@ -106,6 +106,22 @@ Status ServiceBasedActorInfoAccessor::AsyncGet( | |
return Status::OK(); | ||
} | ||
|
||
Status ServiceBasedActorInfoAccessor::AsyncCreateActor( | ||
const ray::TaskSpecification &task_spec, const ray::gcs::StatusCallback &callback) { | ||
RAY_CHECK(task_spec.IsActorCreationTask() && callback); | ||
rpc::CreateActorRequest request; | ||
request.mutable_task_spec()->CopyFrom(task_spec.GetMessage()); | ||
client_impl_->GetGcsRpcClient().CreateActor( | ||
request, [callback](const Status &, const rpc::CreateActorReply &reply) { | ||
auto status = | ||
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 logic to convert gRPC statuses to Ray statuses should probably be in a utility function somewhere (I think it already is?) 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've searched the code. Similar code only exists in gcs_rpc_client.h. I think these two pieces are not quite same. One is the logic of returning reply on the server side, and the other is the logic of converting after receiving reply on the client side. |
||
reply.status().code() == (int)StatusCode::OK | ||
? Status() | ||
: Status(StatusCode(reply.status().code()), reply.status().message()); | ||
callback(status); | ||
wumuzi520 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
}); | ||
return Status::OK(); | ||
} | ||
|
||
Status ServiceBasedActorInfoAccessor::AsyncRegister( | ||
const std::shared_ptr<rpc::ActorTableData> &data_ptr, | ||
const StatusCallback &callback) { | ||
|
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.
Why did this change? Was it wrong before?
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.
That is because a new enum value 'ActorTableData::::ActorState::PENDING' is added. The value 0 is refers to ActorTableData::::ActorState::PENDING while ActorTableData::::ActorState::ALIVE is expected in this unit test.