GCS-Based actor management implementation (#6763)

* add gcs actor manager

* fix test_metrics.py

* fix TestTaskInfo

* fix comment

* fix comment

* fix comment

* fix comment

* fix comment

* fix comment

* fix compile error

* fix merge error

Co-authored-by: senlin.zsl <senlin.zsl@antfin.com>
This commit is contained in:
ZhuSenlin 2020-04-14 00:48:48 +08:00 committed by GitHub
parent 1b0f6fd558
commit 4a81793ba5
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
38 changed files with 2636 additions and 116 deletions

View file

@ -325,6 +325,9 @@ cc_library(
deps = [ deps = [
":gcs", ":gcs",
":gcs_service_rpc", ":gcs_service_rpc",
":node_manager_rpc",
":raylet_lib",
":worker_rpc",
], ],
) )
@ -725,6 +728,45 @@ cc_test(
], ],
) )
cc_test(
name = "gcs_node_manager_test",
srcs = [
"src/ray/gcs/gcs_server/test/gcs_node_manager_test.cc",
"src/ray/gcs/gcs_server/test/gcs_test_util.h",
],
copts = COPTS,
deps = [
":gcs_server_lib",
"@com_google_googletest//:gtest_main",
],
)
cc_test(
name = "gcs_actor_scheduler_test",
srcs = [
"src/ray/gcs/gcs_server/test/gcs_actor_scheduler_test.cc",
"src/ray/gcs/gcs_server/test/gcs_test_util.h",
],
copts = COPTS,
deps = [
":gcs_server_lib",
"@com_google_googletest//:gtest_main",
],
)
cc_test(
name = "gcs_actor_manager_test",
srcs = [
"src/ray/gcs/gcs_server/test/gcs_actor_manager_test.cc",
"src/ray/gcs/gcs_server/test/gcs_test_util.h",
],
copts = COPTS,
deps = [
":gcs_server_lib",
"@com_google_googletest//:gtest_main",
],
)
cc_library( cc_library(
name = "service_based_gcs_client_lib", name = "service_based_gcs_client_lib",
srcs = glob( srcs = glob(
@ -739,7 +781,8 @@ cc_library(
), ),
copts = COPTS, copts = COPTS,
deps = [ deps = [
":gcs_server_lib", ":gcs",
":gcs_service_rpc",
], ],
) )
@ -754,6 +797,7 @@ cc_test(
"//:redis-server", "//:redis-server",
], ],
deps = [ deps = [
":gcs_server_lib",
":service_based_gcs_client_lib", ":service_based_gcs_client_lib",
"@com_google_googletest//:gtest_main", "@com_google_googletest//:gtest_main",
], ],

View file

@ -40,7 +40,7 @@ public class RuntimeContextImpl implements RuntimeContext {
return false; return false;
} }
return runtime.getGcsClient().actorExists(getCurrentActorId()); return runtime.getGcsClient().wasCurrentActorReconstructed(getCurrentActorId());
} }
@Override @Override

View file

@ -9,6 +9,7 @@ import io.ray.api.id.JobId;
import io.ray.api.id.TaskId; import io.ray.api.id.TaskId;
import io.ray.api.id.UniqueId; import io.ray.api.id.UniqueId;
import io.ray.api.runtimecontext.NodeInfo; import io.ray.api.runtimecontext.NodeInfo;
import io.ray.runtime.config.RayConfig;
import io.ray.runtime.generated.Gcs; import io.ray.runtime.generated.Gcs;
import io.ray.runtime.generated.Gcs.ActorCheckpointIdData; import io.ray.runtime.generated.Gcs.ActorCheckpointIdData;
import io.ray.runtime.generated.Gcs.GcsNodeInfo; import io.ray.runtime.generated.Gcs.GcsNodeInfo;
@ -27,9 +28,7 @@ import org.slf4j.LoggerFactory;
* An implementation of GcsClient. * An implementation of GcsClient.
*/ */
public class GcsClient { public class GcsClient {
private static Logger LOGGER = LoggerFactory.getLogger(GcsClient.class); private static Logger LOGGER = LoggerFactory.getLogger(GcsClient.class);
private RedisClient primary; private RedisClient primary;
private List<RedisClient> shards; private List<RedisClient> shards;
@ -126,6 +125,29 @@ public class GcsClient {
return primary.exists(key); return primary.exists(key);
} }
public boolean wasCurrentActorReconstructed(ActorId actorId) {
byte[] key = ArrayUtils.addAll(TablePrefix.ACTOR.toString().getBytes(), actorId.getBytes());
if (!RayConfig.getInstance().gcsServiceEnabled) {
return primary.exists(key);
}
// TODO(ZhuSenlin): Get the actor table data from CoreWorker later.
byte[] value = primary.get(key);
if (value == null) {
return false;
}
Gcs.ActorTableData actorTableData = null;
try {
actorTableData = Gcs.ActorTableData.parseFrom(value);
} catch (InvalidProtocolBufferException e) {
throw new RuntimeException("Received invalid protobuf data from GCS.");
}
long maxReconstructions = actorTableData.getMaxReconstructions();
long remainingReconstructions = actorTableData.getRemainingReconstructions();
return maxReconstructions - remainingReconstructions != 0;
}
/** /**
* Query whether the raylet task exists in Gcs. * Query whether the raylet task exists in Gcs.
*/ */

View file

@ -227,7 +227,7 @@ def test_raylet_info_endpoint(shutdown_only):
if child_actor_info["state"] == -1: if child_actor_info["state"] == -1:
assert child_actor_info["requiredResources"]["CustomResource"] == 1 assert child_actor_info["requiredResources"]["CustomResource"] == 1
else: else:
assert child_actor_info["state"] == 0 assert child_actor_info["state"] == 1
assert len(child_actor_info["children"]) == 0 assert len(child_actor_info["children"]) == 0
assert child_actor_info["usedResources"]["CPU"] == 1 assert child_actor_info["usedResources"]["CPU"] == 1

View file

@ -263,6 +263,10 @@ RAY_CONFIG(int64_t, internal_gcs_service_connect_wait_milliseconds, 100)
/// The interval at which the gcs server will check if redis has gone down. /// The interval at which the gcs server will check if redis has gone down.
/// When this happens, gcs server will kill itself. /// When this happens, gcs server will kill itself.
RAY_CONFIG(int64_t, gcs_redis_heartbeat_interval_milliseconds, 100) RAY_CONFIG(int64_t, gcs_redis_heartbeat_interval_milliseconds, 100)
/// Duration to wait between retries for leasing worker in gcs server.
RAY_CONFIG(uint32_t, gcs_lease_worker_retry_interval_ms, 200)
/// Duration to wait between retries for creating actor in gcs server.
RAY_CONFIG(uint32_t, gcs_create_actor_retry_interval_ms, 200)
/// Maximum number of times to retry putting an object when the plasma store is full. /// Maximum number of times to retry putting an object when the plasma store is full.
/// Can be set to -1 to enable unlimited retries. /// Can be set to -1 to enable unlimited retries.

View file

@ -410,6 +410,16 @@ CoreWorker::CoreWorker(const CoreWorkerOptions &options, const WorkerID &worker_
return std::shared_ptr<raylet::RayletClient>( return std::shared_ptr<raylet::RayletClient>(
new raylet::RayletClient(std::move(grpc_client))); new raylet::RayletClient(std::move(grpc_client)));
}; };
std::function<Status(const TaskSpecification &, const gcs::StatusCallback &)>
actor_create_callback = nullptr;
if (RayConfig::instance().gcs_service_enabled()) {
actor_create_callback = [this](const TaskSpecification &task_spec,
const gcs::StatusCallback &callback) {
return gcs_client_->Actors().AsyncCreateActor(task_spec, callback);
};
}
direct_actor_submitter_ = std::unique_ptr<CoreWorkerDirectActorTaskSubmitter>( direct_actor_submitter_ = std::unique_ptr<CoreWorkerDirectActorTaskSubmitter>(
new CoreWorkerDirectActorTaskSubmitter(rpc_address_, client_factory, memory_store_, new CoreWorkerDirectActorTaskSubmitter(rpc_address_, client_factory, memory_store_,
task_manager_)); task_manager_));
@ -418,7 +428,8 @@ CoreWorker::CoreWorker(const CoreWorkerOptions &options, const WorkerID &worker_
std::unique_ptr<CoreWorkerDirectTaskSubmitter>(new CoreWorkerDirectTaskSubmitter( std::unique_ptr<CoreWorkerDirectTaskSubmitter>(new CoreWorkerDirectTaskSubmitter(
rpc_address_, local_raylet_client_, client_factory, raylet_client_factory, rpc_address_, local_raylet_client_, client_factory, raylet_client_factory,
memory_store_, task_manager_, local_raylet_id, memory_store_, task_manager_, local_raylet_id,
RayConfig::instance().worker_lease_timeout_milliseconds())); RayConfig::instance().worker_lease_timeout_milliseconds(),
std::move(actor_create_callback)));
future_resolver_.reset(new FutureResolver(memory_store_, client_factory)); future_resolver_.reset(new FutureResolver(memory_store_, client_factory));
// Unfortunately the raylet client has to be constructed after the receivers. // Unfortunately the raylet client has to be constructed after the receivers.
if (direct_task_receiver_ != nullptr) { if (direct_task_receiver_ != nullptr) {
@ -624,7 +635,6 @@ void CoreWorker::RegisterToGcs() {
RAY_CHECK_OK(gcs_client_->Workers().AsyncRegisterWorker(options_.worker_type, worker_id, RAY_CHECK_OK(gcs_client_->Workers().AsyncRegisterWorker(options_.worker_type, worker_id,
worker_info, nullptr)); worker_info, nullptr));
} }
void CoreWorker::CheckForRayletFailure(const boost::system::error_code &error) { void CoreWorker::CheckForRayletFailure(const boost::system::error_code &error) {
if (error == boost::asio::error::operation_aborted) { if (error == boost::asio::error::operation_aborted) {
return; return;
@ -1244,7 +1254,9 @@ bool CoreWorker::AddActorHandle(std::unique_ptr<ActorHandle> actor_handle,
// Register a callback to handle actor notifications. // Register a callback to handle actor notifications.
auto actor_notification_callback = [this](const ActorID &actor_id, auto actor_notification_callback = [this](const ActorID &actor_id,
const gcs::ActorTableData &actor_data) { const gcs::ActorTableData &actor_data) {
if (actor_data.state() == gcs::ActorTableData::RECONSTRUCTING) { if (actor_data.state() == gcs::ActorTableData::PENDING) {
// The actor is being created and not yet ready, just ignore!
} else if (actor_data.state() == gcs::ActorTableData::RECONSTRUCTING) {
absl::MutexLock lock(&actor_handles_mutex_); absl::MutexLock lock(&actor_handles_mutex_);
auto it = actor_handles_.find(actor_id); auto it = actor_handles_.find(actor_id);
RAY_CHECK(it != actor_handles_.end()); RAY_CHECK(it != actor_handles_.end());
@ -1265,8 +1277,9 @@ bool CoreWorker::AddActorHandle(std::unique_ptr<ActorHandle> actor_handle,
direct_actor_submitter_->ConnectActor(actor_id, actor_data.address()); direct_actor_submitter_->ConnectActor(actor_id, actor_data.address());
} }
RAY_LOG(INFO) << "received notification on actor, state=" const auto &actor_state = gcs::ActorTableData::ActorState_Name(actor_data.state());
<< static_cast<int>(actor_data.state()) << ", actor_id: " << actor_id RAY_LOG(INFO) << "received notification on actor, state: " << actor_state
<< ", actor_id: " << actor_id
<< ", ip address: " << actor_data.address().ip_address() << ", ip address: " << actor_data.address().ip_address()
<< ", port: " << actor_data.address().port() << ", worker_id: " << ", port: " << actor_data.address().port() << ", worker_id: "
<< WorkerID::FromBinary(actor_data.address().worker_id()) << WorkerID::FromBinary(actor_data.address().worker_id())

View file

@ -15,7 +15,6 @@
#include "ray/core_worker/transport/direct_task_transport.h" #include "ray/core_worker/transport/direct_task_transport.h"
#include "ray/core_worker/transport/dependency_resolver.h" #include "ray/core_worker/transport/dependency_resolver.h"
#include "ray/core_worker/transport/direct_actor_transport.h"
namespace ray { namespace ray {
@ -23,6 +22,30 @@ Status CoreWorkerDirectTaskSubmitter::SubmitTask(TaskSpecification task_spec) {
RAY_LOG(DEBUG) << "Submit task " << task_spec.TaskId(); RAY_LOG(DEBUG) << "Submit task " << task_spec.TaskId();
resolver_.ResolveDependencies(task_spec, [this, task_spec]() { resolver_.ResolveDependencies(task_spec, [this, task_spec]() {
RAY_LOG(DEBUG) << "Task dependencies resolved " << task_spec.TaskId(); 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;
}
absl::MutexLock lock(&mu_); absl::MutexLock lock(&mu_);
// Note that the dependencies in the task spec are mutated to only contain // Note that the dependencies in the task spec are mutated to only contain
// plasma dependencies after ResolveDependencies finishes. // plasma dependencies after ResolveDependencies finishes.

View file

@ -49,21 +49,23 @@ using SchedulingKey = std::tuple<SchedulingClass, std::vector<ObjectID>, ActorID
// This class is thread-safe. // This class is thread-safe.
class CoreWorkerDirectTaskSubmitter { class CoreWorkerDirectTaskSubmitter {
public: public:
CoreWorkerDirectTaskSubmitter(rpc::Address rpc_address, explicit CoreWorkerDirectTaskSubmitter(
std::shared_ptr<WorkerLeaseInterface> lease_client, rpc::Address rpc_address, std::shared_ptr<WorkerLeaseInterface> lease_client,
rpc::ClientFactoryFn client_factory, rpc::ClientFactoryFn client_factory, LeaseClientFactoryFn lease_client_factory,
LeaseClientFactoryFn lease_client_factory, std::shared_ptr<CoreWorkerMemoryStore> store,
std::shared_ptr<CoreWorkerMemoryStore> store, std::shared_ptr<TaskFinisherInterface> task_finisher, ClientID local_raylet_id,
std::shared_ptr<TaskFinisherInterface> task_finisher, int64_t lease_timeout_ms,
ClientID local_raylet_id, int64_t lease_timeout_ms) std::function<Status(const TaskSpecification &, const gcs::StatusCallback &)>
actor_create_callback = nullptr)
: rpc_address_(rpc_address), : rpc_address_(rpc_address),
local_lease_client_(lease_client), local_lease_client_(lease_client),
client_factory_(client_factory), client_factory_(client_factory),
lease_client_factory_(lease_client_factory), lease_client_factory_(lease_client_factory),
resolver_(store, task_finisher), resolver_(store, task_finisher),
task_finisher_(task_finisher), task_finisher_(task_finisher),
lease_timeout_ms_(lease_timeout_ms),
local_raylet_id_(local_raylet_id), local_raylet_id_(local_raylet_id),
lease_timeout_ms_(lease_timeout_ms) {} actor_create_callback_(std::move(actor_create_callback)) {}
/// Schedule a task for direct submission to a worker. /// Schedule a task for direct submission to a worker.
/// ///
@ -148,6 +150,13 @@ class CoreWorkerDirectTaskSubmitter {
/// if a remote raylet tells us to spill the task back to the local raylet. /// if a remote raylet tells us to spill the task back to the local raylet.
const ClientID local_raylet_id_; const ClientID local_raylet_id_;
/// A function to override actor creation. The callback will be called once the actor
/// creation task has been accepted for submission, but the actor may not be created
/// yet.
std::function<Status(const TaskSpecification &task_spec,
const gcs::StatusCallback &callback)>
actor_create_callback_;
// Protects task submission state below. // Protects task submission state below.
absl::Mutex mu_; absl::Mutex mu_;

View file

@ -16,6 +16,7 @@
#define RAY_GCS_ACCESSOR_H #define RAY_GCS_ACCESSOR_H
#include "ray/common/id.h" #include "ray/common/id.h"
#include "ray/common/task/task_spec.h"
#include "ray/gcs/callback.h" #include "ray/gcs/callback.h"
#include "ray/gcs/entry_change_notification.h" #include "ray/gcs/entry_change_notification.h"
#include "ray/protobuf/gcs.pb.h" #include "ray/protobuf/gcs.pb.h"
@ -46,6 +47,14 @@ class ActorInfoAccessor {
virtual Status AsyncGet(const ActorID &actor_id, virtual Status AsyncGet(const ActorID &actor_id,
const OptionalItemCallback<rpc::ActorTableData> &callback) = 0; const OptionalItemCallback<rpc::ActorTableData> &callback) = 0;
/// Create an actor to GCS asynchronously.
///
/// \param task_spec The specification for the actor creation task.
/// \param callback Callback that will be called after the actor info is written to GCS.
/// \return Status
virtual Status AsyncCreateActor(const TaskSpecification &task_spec,
const StatusCallback &callback) = 0;
/// Register an actor to GCS asynchronously. /// Register an actor to GCS asynchronously.
/// ///
/// \param data_ptr The actor that will be registered to the GCS. /// \param data_ptr The actor that will be registered to the GCS.

View file

@ -79,7 +79,7 @@ ServiceBasedActorInfoAccessor::ServiceBasedActorInfoAccessor(
ServiceBasedGcsClient *client_impl) ServiceBasedGcsClient *client_impl)
: subscribe_id_(ClientID::FromRandom()), : subscribe_id_(ClientID::FromRandom()),
client_impl_(client_impl), 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( Status ServiceBasedActorInfoAccessor::GetAll(
std::vector<ActorTableData> *actor_table_data_list) { std::vector<ActorTableData> *actor_table_data_list) {
@ -106,6 +106,22 @@ Status ServiceBasedActorInfoAccessor::AsyncGet(
return Status::OK(); 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 =
reply.status().code() == (int)StatusCode::OK
? Status()
: Status(StatusCode(reply.status().code()), reply.status().message());
callback(status);
});
return Status::OK();
}
Status ServiceBasedActorInfoAccessor::AsyncRegister( Status ServiceBasedActorInfoAccessor::AsyncRegister(
const std::shared_ptr<rpc::ActorTableData> &data_ptr, const std::shared_ptr<rpc::ActorTableData> &data_ptr,
const StatusCallback &callback) { const StatusCallback &callback) {

View file

@ -15,6 +15,7 @@
#ifndef RAY_GCS_SERVICE_BASED_ACCESSOR_H #ifndef RAY_GCS_SERVICE_BASED_ACCESSOR_H
#define RAY_GCS_SERVICE_BASED_ACCESSOR_H #define RAY_GCS_SERVICE_BASED_ACCESSOR_H
#include <ray/common/task/task_spec.h>
#include "ray/gcs/accessor.h" #include "ray/gcs/accessor.h"
#include "ray/gcs/subscription_executor.h" #include "ray/gcs/subscription_executor.h"
#include "ray/util/sequencer.h" #include "ray/util/sequencer.h"
@ -63,6 +64,9 @@ class ServiceBasedActorInfoAccessor : public ActorInfoAccessor {
Status AsyncGet(const ActorID &actor_id, Status AsyncGet(const ActorID &actor_id,
const OptionalItemCallback<rpc::ActorTableData> &callback) override; const OptionalItemCallback<rpc::ActorTableData> &callback) override;
Status AsyncCreateActor(const TaskSpecification &task_spec,
const StatusCallback &callback) override;
Status AsyncRegister(const std::shared_ptr<rpc::ActorTableData> &data_ptr, Status AsyncRegister(const std::shared_ptr<rpc::ActorTableData> &data_ptr,
const StatusCallback &callback) override; const StatusCallback &callback) override;
@ -97,7 +101,7 @@ class ServiceBasedActorInfoAccessor : public ActorInfoAccessor {
private: private:
ServiceBasedGcsClient *client_impl_; ServiceBasedGcsClient *client_impl_;
typedef SubscriptionExecutor<ActorID, ActorTableData, LogBasedActorTable> typedef SubscriptionExecutor<ActorID, ActorTableData, ActorTable>
ActorSubscriptionExecutor; ActorSubscriptionExecutor;
ActorSubscriptionExecutor actor_sub_executor_; ActorSubscriptionExecutor actor_sub_executor_;

View file

@ -18,6 +18,21 @@
namespace ray { namespace ray {
namespace rpc { namespace rpc {
void DefaultActorInfoHandler::HandleCreateActor(
const ray::rpc::CreateActorRequest &request, ray::rpc::CreateActorReply *reply,
ray::rpc::SendReplyCallback send_reply_callback) {
RAY_CHECK(request.task_spec().type() == TaskType::ACTOR_CREATION_TASK);
auto actor_id =
ActorID::FromBinary(request.task_spec().actor_creation_task_spec().actor_id());
RAY_LOG(INFO) << "Registering actor, actor id = " << actor_id;
gcs_actor_manager_.RegisterActor(request, [reply, send_reply_callback, actor_id](
std::shared_ptr<gcs::GcsActor> actor) {
RAY_LOG(INFO) << "Registered actor, actor id = " << actor_id;
GCS_RPC_SEND_REPLY(send_reply_callback, reply, Status::OK());
});
}
void DefaultActorInfoHandler::HandleGetActorInfo( void DefaultActorInfoHandler::HandleGetActorInfo(
const rpc::GetActorInfoRequest &request, rpc::GetActorInfoReply *reply, const rpc::GetActorInfoRequest &request, rpc::GetActorInfoReply *reply,
rpc::SendReplyCallback send_reply_callback) { rpc::SendReplyCallback send_reply_callback) {

View file

@ -15,17 +15,22 @@
#ifndef RAY_GCS_ACTOR_INFO_HANDLER_IMPL_H #ifndef RAY_GCS_ACTOR_INFO_HANDLER_IMPL_H
#define RAY_GCS_ACTOR_INFO_HANDLER_IMPL_H #define RAY_GCS_ACTOR_INFO_HANDLER_IMPL_H
#include "gcs_actor_manager.h"
#include "ray/gcs/redis_gcs_client.h" #include "ray/gcs/redis_gcs_client.h"
#include "ray/rpc/gcs_server/gcs_rpc_server.h" #include "ray/rpc/gcs_server/gcs_rpc_server.h"
namespace ray { namespace ray {
namespace rpc {
namespace rpc {
/// This implementation class of `ActorInfoHandler`. /// This implementation class of `ActorInfoHandler`.
class DefaultActorInfoHandler : public rpc::ActorInfoHandler { class DefaultActorInfoHandler : public rpc::ActorInfoHandler {
public: public:
explicit DefaultActorInfoHandler(gcs::RedisGcsClient &gcs_client) explicit DefaultActorInfoHandler(gcs::RedisGcsClient &gcs_client,
: gcs_client_(gcs_client) {} gcs::GcsActorManager &gcs_actor_manager)
: gcs_client_(gcs_client), gcs_actor_manager_(gcs_actor_manager) {}
void HandleCreateActor(const CreateActorRequest &request, CreateActorReply *reply,
SendReplyCallback send_reply_callback) override;
void HandleGetActorInfo(const GetActorInfoRequest &request, GetActorInfoReply *reply, void HandleGetActorInfo(const GetActorInfoRequest &request, GetActorInfoReply *reply,
SendReplyCallback send_reply_callback) override; SendReplyCallback send_reply_callback) override;
@ -52,6 +57,7 @@ class DefaultActorInfoHandler : public rpc::ActorInfoHandler {
private: private:
gcs::RedisGcsClient &gcs_client_; gcs::RedisGcsClient &gcs_client_;
gcs::GcsActorManager &gcs_actor_manager_;
}; };
} // namespace rpc } // namespace rpc

View file

@ -0,0 +1,283 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "gcs_actor_manager.h"
#include <ray/common/ray_config.h>
#include <utility>
namespace ray {
namespace gcs {
ClientID GcsActor::GetNodeID() const {
const auto &raylet_id_binary = actor_table_data_.address().raylet_id();
if (raylet_id_binary.empty()) {
return ClientID::Nil();
}
return ClientID::FromBinary(raylet_id_binary);
}
void GcsActor::UpdateAddress(const rpc::Address &address) {
actor_table_data_.mutable_address()->CopyFrom(address);
}
const rpc::Address &GcsActor::GetAddress() const { return actor_table_data_.address(); }
WorkerID GcsActor::GetWorkerID() const {
const auto &address = actor_table_data_.address();
if (address.worker_id().empty()) {
return WorkerID::Nil();
}
return WorkerID::FromBinary(address.worker_id());
}
void GcsActor::UpdateState(rpc::ActorTableData::ActorState state) {
actor_table_data_.set_state(state);
}
rpc::ActorTableData::ActorState GcsActor::GetState() const {
return actor_table_data_.state();
}
ActorID GcsActor::GetActorID() const {
return ActorID::FromBinary(actor_table_data_.actor_id());
}
TaskSpecification GcsActor::GetCreationTaskSpecification() const {
const auto &task_spec = actor_table_data_.task_spec();
return TaskSpecification(task_spec);
}
const rpc::ActorTableData &GcsActor::GetActorTableData() const {
return actor_table_data_;
}
rpc::ActorTableData *GcsActor::GetMutableActorTableData() { return &actor_table_data_; }
/////////////////////////////////////////////////////////////////////////////////////////
GcsActorManager::GcsActorManager(boost::asio::io_context &io_context,
gcs::ActorInfoAccessor &actor_info_accessor,
gcs::GcsNodeManager &gcs_node_manager,
LeaseClientFactoryFn lease_client_factory,
rpc::ClientFactoryFn client_factory)
: actor_info_accessor_(actor_info_accessor),
gcs_actor_scheduler_(new gcs::GcsActorScheduler(
io_context, actor_info_accessor, gcs_node_manager,
/*schedule_failure_handler=*/
[this](std::shared_ptr<GcsActor> actor) {
// When there are no available nodes to schedule the actor the
// gcs_actor_scheduler will treat it as failed and invoke this handler. In
// this case, the actor should be appended to the `pending_actors_` and wait
// for the registration of new node.
pending_actors_.emplace_back(std::move(actor));
},
/*schedule_success_handler=*/
[this](std::shared_ptr<GcsActor> actor) {
OnActorCreateSuccess(std::move(actor));
},
std::move(lease_client_factory), std::move(client_factory))) {
RAY_LOG(INFO) << "Initializing GcsActorManager.";
gcs_node_manager.AddNodeAddedListener(
[this](const std::shared_ptr<rpc::GcsNodeInfo> &) {
// Because a new node has been added, we need to try to schedule the pending
// actors.
SchedulePendingActors();
});
gcs_node_manager.AddNodeRemovedListener([this](std::shared_ptr<rpc::GcsNodeInfo> node) {
// All of the related actors should be reconstructed when a node is removed from the
// GCS.
ReconstructActorsOnNode(ClientID::FromBinary(node->node_id()));
});
RAY_LOG(INFO) << "Finished initialing GcsActorManager.";
}
void GcsActorManager::RegisterActor(
const ray::rpc::CreateActorRequest &request,
std::function<void(std::shared_ptr<GcsActor>)> callback) {
RAY_CHECK(callback);
const auto &actor_creation_task_spec = request.task_spec().actor_creation_task_spec();
auto actor_id = ActorID::FromBinary(actor_creation_task_spec.actor_id());
auto iter = registered_actors_.find(actor_id);
if (iter != registered_actors_.end()) {
// When the network fails, Driver/Worker is not sure whether GcsServer has received
// the request of actor creation task, so Driver/Worker will try again and again until
// receiving the reply from GcsServer. If the actor is already records on the GCS
// Server side, the GCS Server will be responsible for creating or reconstructing the
// actor regardless of whether the Driver/Worker sends the request to create the actor
// again, so we just need fast reply OK to the Driver/Worker that the actor is already
// recorded by GCS Server.
callback(iter->second);
return;
}
auto pending_register_iter = actor_to_register_callbacks_.find(actor_id);
if (pending_register_iter != actor_to_register_callbacks_.end()) {
// It is a duplicate message, just mark the callback as pending and invoke it after
// the related actor is flushed.
pending_register_iter->second.emplace_back(std::move(callback));
return;
}
// Mark the callback as pending and invoke it after the related actor is flushed.
actor_to_register_callbacks_[actor_id].emplace_back(std::move(callback));
auto actor = std::make_shared<GcsActor>(request);
auto actor_table_data =
std::make_shared<rpc::ActorTableData>(actor->GetActorTableData());
// The backend storage is reliable in the future, so the status must be ok.
RAY_CHECK_OK(actor_info_accessor_.AsyncUpdate(
actor_id, actor_table_data, [this, actor](Status status) {
RAY_CHECK_OK(status);
RAY_CHECK(registered_actors_.emplace(actor->GetActorID(), actor).second);
// Invoke all callbacks for all registration requests of this actor (duplicated
// requests are included) and remove all of them from
// actor_to_register_callbacks_.
auto iter = actor_to_register_callbacks_.find(actor->GetActorID());
RAY_CHECK(iter != actor_to_register_callbacks_.end() && !iter->second.empty());
for (auto &callback : iter->second) {
callback(actor);
}
actor_to_register_callbacks_.erase(iter);
gcs_actor_scheduler_->Schedule(actor);
}));
}
void GcsActorManager::ReconstructActorOnWorker(const ray::ClientID &node_id,
const ray::WorkerID &worker_id,
bool need_reschedule) {
std::shared_ptr<GcsActor> actor;
// Cancel the scheduling of the related actor.
auto actor_id = gcs_actor_scheduler_->CancelOnWorker(node_id, worker_id);
if (!actor_id.IsNil()) {
auto iter = registered_actors_.find(actor_id);
RAY_CHECK(iter != registered_actors_.end());
actor = iter->second;
} else {
// Find from worker_to_created_actor_.
auto iter = worker_to_created_actor_.find(worker_id);
if (iter != worker_to_created_actor_.end()) {
actor = std::move(iter->second);
// Remove the created actor from worker_to_created_actor_.
worker_to_created_actor_.erase(iter);
// remove the created actor from node_to_created_actors_.
auto node_iter = node_to_created_actors_.find(node_id);
RAY_CHECK(node_iter != node_to_created_actors_.end());
RAY_CHECK(node_iter->second.erase(actor->GetActorID()) != 0);
if (node_iter->second.empty()) {
node_to_created_actors_.erase(node_iter);
}
}
}
if (actor != nullptr) {
// Reconstruct the actor.
ReconstructActor(actor, need_reschedule);
}
}
void GcsActorManager::ReconstructActorsOnNode(const ClientID &node_id) {
// Cancel the scheduling of all related actors.
auto scheduling_actor_ids = gcs_actor_scheduler_->CancelOnNode(node_id);
for (auto &actor_id : scheduling_actor_ids) {
auto iter = registered_actors_.find(actor_id);
if (iter != registered_actors_.end()) {
// Reconstruct the canceled actor.
ReconstructActor(iter->second);
}
}
// Find all actors that were created on this node.
auto iter = node_to_created_actors_.find(node_id);
if (iter != node_to_created_actors_.end()) {
auto created_actors = std::move(iter->second);
// Remove all created actors from node_to_created_actors_.
node_to_created_actors_.erase(iter);
for (auto &entry : created_actors) {
// Remove the actor from worker_to_created_actor_.
RAY_CHECK(worker_to_created_actor_.erase(entry.second->GetWorkerID()) != 0);
// Reconstruct the removed actor.
ReconstructActor(std::move(entry.second));
}
}
}
void GcsActorManager::ReconstructActor(std::shared_ptr<GcsActor> actor,
bool need_reschedule) {
RAY_CHECK(actor != nullptr);
auto node_id = actor->GetNodeID();
auto worker_id = actor->GetWorkerID();
actor->UpdateAddress(rpc::Address());
auto mutable_actor_table_data = actor->GetMutableActorTableData();
// If the need_reschedule is set to false, then set the `remaining_reconstructions` to 0
// so that the actor will never be rescheduled.
auto remaining_reconstructions =
need_reschedule ? mutable_actor_table_data->remaining_reconstructions() : 0;
RAY_LOG(WARNING) << "Actor is failed " << actor->GetActorID() << " on worker "
<< worker_id << " at node " << node_id
<< ", need_reschedule = " << need_reschedule
<< ", remaining_reconstructions = " << remaining_reconstructions;
if (remaining_reconstructions > 0) {
mutable_actor_table_data->set_remaining_reconstructions(--remaining_reconstructions);
mutable_actor_table_data->set_state(rpc::ActorTableData::RECONSTRUCTING);
auto actor_table_data =
std::make_shared<rpc::ActorTableData>(*mutable_actor_table_data);
// The backend storage is reliable in the future, so the status must be ok.
RAY_CHECK_OK(actor_info_accessor_.AsyncUpdate(actor->GetActorID(), actor_table_data,
[this, actor](Status status) {
RAY_CHECK_OK(status);
gcs_actor_scheduler_->Schedule(actor);
}));
} else {
mutable_actor_table_data->set_state(rpc::ActorTableData::DEAD);
auto actor_table_data =
std::make_shared<rpc::ActorTableData>(*mutable_actor_table_data);
// The backend storage is reliable in the future, so the status must be ok.
RAY_CHECK_OK(
actor_info_accessor_.AsyncUpdate(actor->GetActorID(), actor_table_data, nullptr));
}
}
void GcsActorManager::OnActorCreateSuccess(std::shared_ptr<GcsActor> actor) {
auto worker_id = actor->GetWorkerID();
RAY_CHECK(!worker_id.IsNil());
RAY_CHECK(worker_to_created_actor_.emplace(worker_id, actor).second);
auto actor_id = actor->GetActorID();
auto node_id = actor->GetNodeID();
RAY_CHECK(!node_id.IsNil());
RAY_CHECK(node_to_created_actors_[node_id].emplace(actor_id, actor).second);
actor->UpdateState(rpc::ActorTableData::ALIVE);
auto actor_table_data =
std::make_shared<rpc::ActorTableData>(actor->GetActorTableData());
// The backend storage is reliable in the future, so the status must be ok.
RAY_CHECK_OK(actor_info_accessor_.AsyncUpdate(actor_id, actor_table_data, nullptr));
}
void GcsActorManager::SchedulePendingActors() {
if (pending_actors_.empty()) {
return;
}
RAY_LOG(DEBUG) << "Scheduling actor creation tasks, size = " << pending_actors_.size();
auto actors = std::move(pending_actors_);
for (auto &actor : actors) {
gcs_actor_scheduler_->Schedule(std::move(actor));
}
}
} // namespace gcs
} // namespace ray

View file

@ -0,0 +1,193 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef RAY_GCS_ACTOR_MANAGER_H
#define RAY_GCS_ACTOR_MANAGER_H
#include <ray/common/id.h>
#include <ray/common/task/task_execution_spec.h>
#include <ray/common/task/task_spec.h>
#include <ray/protobuf/gcs_service.pb.h>
#include <utility>
#include "absl/container/flat_hash_map.h"
#include "gcs_actor_scheduler.h"
namespace ray {
namespace gcs {
/// GcsActor just wraps `ActorTableData` and provides some convenient interfaces to access
/// the fields inside `ActorTableData`.
/// This class is not thread-safe.
class GcsActor {
public:
/// Create a GcsActor by actor_table_data.
///
/// \param actor_table_data Data of the actor (see gcs.proto).
explicit GcsActor(rpc::ActorTableData actor_table_data)
: actor_table_data_(std::move(actor_table_data)) {}
/// Create a GcsActor by CreateActorRequest.
///
/// \param request Contains the actor creation task specification.
explicit GcsActor(const ray::rpc::CreateActorRequest &request) {
RAY_CHECK(request.task_spec().type() == TaskType::ACTOR_CREATION_TASK);
const auto &actor_creation_task_spec = request.task_spec().actor_creation_task_spec();
actor_table_data_.set_actor_id(actor_creation_task_spec.actor_id());
actor_table_data_.set_job_id(request.task_spec().job_id());
actor_table_data_.set_max_reconstructions(
actor_creation_task_spec.max_actor_reconstructions());
actor_table_data_.set_remaining_reconstructions(
actor_creation_task_spec.max_actor_reconstructions());
auto dummy_object =
TaskSpecification(request.task_spec()).ActorDummyObject().Binary();
actor_table_data_.set_actor_creation_dummy_object_id(dummy_object);
actor_table_data_.set_is_detached(actor_creation_task_spec.is_detached());
actor_table_data_.mutable_owner_address()->CopyFrom(
request.task_spec().caller_address());
actor_table_data_.set_state(rpc::ActorTableData::PENDING);
actor_table_data_.mutable_task_spec()->CopyFrom(request.task_spec());
actor_table_data_.mutable_address()->set_raylet_id(ClientID::Nil().Binary());
actor_table_data_.mutable_address()->set_worker_id(WorkerID::Nil().Binary());
}
/// Get the node id on which this actor is created.
ClientID GetNodeID() const;
/// Get the id of the worker on which this actor is created.
WorkerID GetWorkerID() const;
/// Update the `Address` of this actor (see gcs.proto).
void UpdateAddress(const rpc::Address &address);
/// Get the `Address` of this actor.
const rpc::Address &GetAddress() const;
/// Update the state of this actor.
void UpdateState(rpc::ActorTableData::ActorState state);
/// Get the state of this gcs actor.
rpc::ActorTableData::ActorState GetState() const;
/// Get the id of this actor.
ActorID GetActorID() const;
/// Get the task specification of this actor.
TaskSpecification GetCreationTaskSpecification() const;
/// Get the immutable ActorTableData of this actor.
const rpc::ActorTableData &GetActorTableData() const;
/// Get the mutable ActorTableData of this actor.
rpc::ActorTableData *GetMutableActorTableData();
private:
/// The actor meta data which contains the task specification as well as the state of
/// the gcs actor and so on (see gcs.proto).
rpc::ActorTableData actor_table_data_;
};
using RegisterActorCallback = std::function<void(std::shared_ptr<GcsActor>)>;
/// GcsActorManager is responsible for managing the lifecycle of all actors.
/// This class is not thread-safe.
class GcsActorManager {
public:
/// Create a GcsActorManager
///
/// \param io_context The main event loop.
/// \param actor_info_accessor Used to flush actor data to storage.
/// \param gcs_node_manager The actor manager needs to listen to the node change events
/// inside gcs_node_manager.
/// \param lease_client_factory Factory to create remote lease client, it will be passed
/// through to the constructor of gcs_actor_scheduler, the gcs_actor_scheduler will use
/// default factory inside itself if it is not set.
/// \param client_factory Factory to create remote core worker client, it will be passed
/// through to the constructor of gcs_actor_scheduler, the gcs_actor_scheduler will use
/// default factory inside itself if it is not set.
explicit GcsActorManager(boost::asio::io_context &io_context,
gcs::ActorInfoAccessor &actor_info_accessor,
gcs::GcsNodeManager &gcs_node_manager,
LeaseClientFactoryFn lease_client_factory = nullptr,
rpc::ClientFactoryFn client_factory = nullptr);
virtual ~GcsActorManager() = default;
/// Register actor asynchronously.
///
/// \param request Contains the meta info to create the actor.
/// \param callback Will be invoked after the meta info is flushed to the storage or be
/// invoked immediately if the meta info already exists.
void RegisterActor(const rpc::CreateActorRequest &request,
RegisterActorCallback callback);
/// Reconstruct all actors associated with the specified node id, including actors which
/// are scheduled or have been created on this node. Triggered when the given node goes
/// down.
///
/// \param node_id The specified node id.
void ReconstructActorsOnNode(const ClientID &node_id);
/// Reconstruct actor associated with the specified node_id and worker_id.
/// The actor may be pending or already created.
///
/// \param node_id ID of the node where the worker is located
/// \param worker_id ID of the worker that the actor is creating/created on
/// \param need_reschedule Whether to reschedule the actor creation task, sometimes
/// users want to kill an actor intentionally and don't want it to be rescheduled
/// again.
void ReconstructActorOnWorker(const ClientID &node_id, const WorkerID &worker_id,
bool need_reschedule = true);
protected:
/// Schedule actors in the `pending_actors_` queue.
/// This method is triggered when new nodes are registered or resources change.
void SchedulePendingActors();
/// Reconstruct the specified actor.
///
/// \param actor The target actor to be reconstructed.
/// \param need_reschedule Whether to reschedule the actor creation task, sometimes
/// users want to kill an actor intentionally and don't want it to be reconstructed
/// again.
void ReconstructActor(std::shared_ptr<GcsActor> actor, bool need_reschedule = true);
/// This method is a callback of gcs_actor_scheduler when actor is created successfully.
/// It will update the state of actor as well as the worker_to_created_actor_ and
/// node_to_created_actors_ and flush the actor data to the storage.
void OnActorCreateSuccess(std::shared_ptr<GcsActor> actor);
protected:
/// Callbacks of actor registration requests that are not yet flushed.
/// This map is used to filter duplicated messages from a Driver/Worker caused by some
/// network problems.
absl::flat_hash_map<ActorID, std::vector<RegisterActorCallback>>
actor_to_register_callbacks_;
/// All registered actors (pending actors are also included).
absl::flat_hash_map<ActorID, std::shared_ptr<GcsActor>> registered_actors_;
/// The pending actors which will not be scheduled until there's a resource change.
std::vector<std::shared_ptr<GcsActor>> pending_actors_;
/// Map contains the relationship of worker and created actor.
absl::flat_hash_map<WorkerID, std::shared_ptr<GcsActor>> worker_to_created_actor_;
/// Map contains the relationship of node and created actors.
absl::flat_hash_map<ClientID, absl::flat_hash_map<ActorID, std::shared_ptr<GcsActor>>>
node_to_created_actors_;
/// The access info accessor.
gcs::ActorInfoAccessor &actor_info_accessor_;
/// The scheduler to schedule all registered actors.
std::unique_ptr<gcs::GcsActorScheduler> gcs_actor_scheduler_;
};
} // namespace gcs
} // namespace ray
#endif // RAY_GCS_ACTOR_MANAGER_H

View file

@ -0,0 +1,382 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "gcs_actor_scheduler.h"
#include <ray/common/ray_config.h>
#include <ray/protobuf/node_manager.pb.h>
#include <ray/util/asio_util.h>
#include "gcs_actor_manager.h"
namespace ray {
namespace gcs {
GcsActorScheduler::GcsActorScheduler(
boost::asio::io_context &io_context, gcs::ActorInfoAccessor &actor_info_accessor,
const gcs::GcsNodeManager &gcs_node_manager,
std::function<void(std::shared_ptr<GcsActor>)> schedule_failure_handler,
std::function<void(std::shared_ptr<GcsActor>)> schedule_success_handler,
LeaseClientFactoryFn lease_client_factory, rpc::ClientFactoryFn client_factory)
: io_context_(io_context),
client_call_manager_(io_context_),
actor_info_accessor_(actor_info_accessor),
gcs_node_manager_(gcs_node_manager),
schedule_failure_handler_(std::move(schedule_failure_handler)),
schedule_success_handler_(std::move(schedule_success_handler)),
lease_client_factory_(std::move(lease_client_factory)),
client_factory_(std::move(client_factory)) {
RAY_CHECK(schedule_failure_handler_ != nullptr && schedule_success_handler_ != nullptr);
if (lease_client_factory_ == nullptr) {
lease_client_factory_ = [this](const rpc::Address &address) {
auto node_manager_worker_client = rpc::NodeManagerWorkerClient::make(
address.ip_address(), address.port(), client_call_manager_);
return std::make_shared<raylet::RayletClient>(
std::move(node_manager_worker_client));
};
}
if (client_factory_ == nullptr) {
client_factory_ = [this](const rpc::Address &address) {
return std::make_shared<rpc::CoreWorkerClient>(address, client_call_manager_);
};
}
}
void GcsActorScheduler::Schedule(std::shared_ptr<GcsActor> actor) {
auto node_id = actor->GetNodeID();
if (!node_id.IsNil()) {
if (auto node = gcs_node_manager_.GetNode(node_id)) {
// If the actor is already tied to a node and the node is available, then record
// the relationship of the node and actor and then lease worker directly from the
// node.
RAY_CHECK(node_to_actors_when_leasing_[actor->GetNodeID()]
.emplace(actor->GetActorID())
.second);
LeaseWorkerFromNode(actor, node);
return;
}
// The actor is already tied to a node which is unavailable now, so we should reset
// the address.
actor->UpdateAddress(rpc::Address());
}
// Select a node to lease worker for the actor.
auto node = SelectNodeRandomly();
if (node == nullptr) {
// There are no available nodes to schedule the actor, so just trigger the failed
// handler.
schedule_failure_handler_(std::move(actor));
return;
}
// Update the address of the actor as it is tied to a new node.
rpc::Address address;
address.set_raylet_id(node->node_id());
actor->UpdateAddress(address);
auto actor_table_data =
std::make_shared<rpc::ActorTableData>(actor->GetActorTableData());
// The backend storage is reliable in the future, so the status must be ok.
RAY_CHECK_OK(actor_info_accessor_.AsyncUpdate(actor->GetActorID(), actor_table_data,
[this, actor](Status status) {
RAY_CHECK_OK(status);
// There is no promise that the node the
// actor tied to is still alive as the
// flush is asynchronously, so just
// invoke `Schedule` which will lease
// worker directly if the node is still
// available or select a new one if not.
Schedule(actor);
}));
}
std::vector<ActorID> GcsActorScheduler::CancelOnNode(const ClientID &node_id) {
// Remove all the actors from the map associated with this node, and return them as they
// will be reconstructed later.
std::vector<ActorID> actor_ids;
// Remove all actors in phase of leasing.
{
auto iter = node_to_actors_when_leasing_.find(node_id);
if (iter != node_to_actors_when_leasing_.end()) {
actor_ids.insert(actor_ids.end(), iter->second.begin(), iter->second.end());
node_to_actors_when_leasing_.erase(iter);
}
}
// Remove all actors in phase of creating.
{
auto iter = node_to_workers_when_creating_.find(node_id);
if (iter != node_to_workers_when_creating_.end()) {
for (auto &entry : iter->second) {
actor_ids.emplace_back(entry.second->GetAssignedActorID());
// Remove core worker client.
RAY_CHECK(core_worker_clients_.erase(entry.first) != 0);
}
node_to_workers_when_creating_.erase(iter);
}
}
// Remove the related remote lease client from remote_lease_clients_.
// There is no need to check in this place, because it is possible that there are no
// workers leased on this node.
remote_lease_clients_.erase(node_id);
return actor_ids;
}
ActorID GcsActorScheduler::CancelOnWorker(const ClientID &node_id,
const WorkerID &worker_id) {
// Remove the worker from creating map and return ID of the actor associated with the
// removed worker if exist, else return NilID.
ActorID assigned_actor_id;
auto iter = node_to_workers_when_creating_.find(node_id);
if (iter != node_to_workers_when_creating_.end()) {
auto actor_iter = iter->second.find(worker_id);
if (actor_iter != iter->second.end()) {
assigned_actor_id = actor_iter->second->GetAssignedActorID();
// Remove core worker client.
RAY_CHECK(core_worker_clients_.erase(worker_id) != 0);
iter->second.erase(actor_iter);
if (iter->second.empty()) {
node_to_workers_when_creating_.erase(iter);
}
}
}
return assigned_actor_id;
}
void GcsActorScheduler::LeaseWorkerFromNode(std::shared_ptr<GcsActor> actor,
std::shared_ptr<rpc::GcsNodeInfo> node) {
RAY_CHECK(actor && node);
auto node_id = ClientID::FromBinary(node->node_id());
RAY_LOG(INFO) << "Start leasing worker from node " << node_id << " for actor "
<< actor->GetActorID();
rpc::Address remote_address;
remote_address.set_raylet_id(node->node_id());
remote_address.set_ip_address(node->node_manager_address());
remote_address.set_port(node->node_manager_port());
auto lease_client = GetOrConnectLeaseClient(remote_address);
auto status = lease_client->RequestWorkerLease(
actor->GetCreationTaskSpecification(),
[this, node_id, actor, node](const Status &status,
const rpc::RequestWorkerLeaseReply &reply) {
// If the actor is still in the leasing map and the status is ok, remove the actor
// from the leasing map and handle the reply. Otherwise, lease again, because it
// may be a network exception.
// If the actor is not in the leasing map, it means that the actor has been
// cancelled as the node is dead, just do nothing in this case because the
// gcs_actor_manager will reconstruct it again.
auto iter = node_to_actors_when_leasing_.find(node_id);
if (iter != node_to_actors_when_leasing_.end()) {
// If the node is still available, the actor must be still in the leasing map as
// it is erased from leasing map only when `CancelOnNode` or the
// `RequestWorkerLeaseReply` is received from the node, so try lease again.
auto actor_iter = iter->second.find(actor->GetActorID());
RAY_CHECK(actor_iter != iter->second.end());
if (status.ok()) {
// Remove the actor from the leasing map as the reply is returned from the
// remote node.
iter->second.erase(actor_iter);
if (iter->second.empty()) {
node_to_actors_when_leasing_.erase(iter);
}
RAY_LOG(INFO) << "Finished leasing worker from " << node_id << " for actor "
<< actor->GetActorID();
HandleWorkerLeasedReply(actor, reply);
} else {
RetryLeasingWorkerFromNode(actor, node);
}
}
});
if (!status.ok()) {
RetryLeasingWorkerFromNode(actor, node);
}
}
void GcsActorScheduler::RetryLeasingWorkerFromNode(
std::shared_ptr<GcsActor> actor, std::shared_ptr<rpc::GcsNodeInfo> node) {
execute_after(io_context_,
[this, node, actor] { DoRetryLeasingWorkerFromNode(actor, node); },
RayConfig::instance().gcs_lease_worker_retry_interval_ms());
}
void GcsActorScheduler::DoRetryLeasingWorkerFromNode(
std::shared_ptr<GcsActor> actor, std::shared_ptr<rpc::GcsNodeInfo> node) {
auto iter = node_to_actors_when_leasing_.find(actor->GetNodeID());
if (iter != node_to_actors_when_leasing_.end()) {
// If the node is still available, the actor must be still in the
// leasing map as it is erased from leasing map only when
// `CancelOnNode` or the `RequestWorkerLeaseReply` is received from
// the node, so try leasing again.
RAY_CHECK(iter->second.count(actor->GetActorID()) != 0);
RAY_LOG(INFO) << "Retry leasing worker from " << actor->GetNodeID() << " for actor "
<< actor->GetActorID();
LeaseWorkerFromNode(actor, node);
}
}
void GcsActorScheduler::HandleWorkerLeasedReply(
std::shared_ptr<GcsActor> actor, const ray::rpc::RequestWorkerLeaseReply &reply) {
const auto &retry_at_raylet_address = reply.retry_at_raylet_address();
const auto &worker_address = reply.worker_address();
if (worker_address.raylet_id().empty()) {
// The worker did not succeed in the lease, but the specified node returned a new
// node, and then try again on the new node.
RAY_CHECK(!retry_at_raylet_address.raylet_id().empty());
actor->UpdateAddress(retry_at_raylet_address);
auto actor_table_data =
std::make_shared<rpc::ActorTableData>(actor->GetActorTableData());
// The backend storage is reliable in the future, so the status must be ok.
RAY_CHECK_OK(actor_info_accessor_.AsyncUpdate(actor->GetActorID(), actor_table_data,
[this, actor](Status status) {
RAY_CHECK_OK(status);
Schedule(actor);
}));
} else {
// The worker is leased successfully from the specified node.
std::vector<rpc::ResourceMapEntry> resources;
for (auto &resource : reply.resource_mapping()) {
resources.emplace_back(resource);
}
auto leased_worker = std::make_shared<GcsLeasedWorker>(
worker_address, std::move(resources), actor->GetActorID());
auto node_id = leased_worker->GetNodeID();
RAY_CHECK(node_to_workers_when_creating_[node_id]
.emplace(leased_worker->GetWorkerID(), leased_worker)
.second);
actor->UpdateAddress(leased_worker->GetAddress());
CreateActorOnWorker(actor, leased_worker);
}
}
void GcsActorScheduler::CreateActorOnWorker(std::shared_ptr<GcsActor> actor,
std::shared_ptr<GcsLeasedWorker> worker) {
RAY_CHECK(actor && worker);
RAY_LOG(INFO) << "Start creating actor " << actor->GetActorID() << " on worker "
<< worker->GetWorkerID() << " at node " << actor->GetNodeID();
std::unique_ptr<rpc::PushTaskRequest> request(new rpc::PushTaskRequest());
request->set_intended_worker_id(worker->GetWorkerID().Binary());
request->mutable_task_spec()->CopyFrom(
actor->GetCreationTaskSpecification().GetMessage());
google::protobuf::RepeatedPtrField<rpc::ResourceMapEntry> resources;
for (auto resource : worker->GetLeasedResources()) {
resources.Add(std::move(resource));
}
request->mutable_resource_mapping()->CopyFrom(resources);
auto client = GetOrConnectCoreWorkerClient(worker->GetAddress());
auto status = client->PushNormalTask(
std::move(request),
[this, actor, worker](Status status, const rpc::PushTaskReply &reply) {
RAY_UNUSED(reply);
// If the actor is still in the creating map and the status is ok, remove the
// actor from the creating map and invoke the schedule_success_handler_.
// Otherwise, create again, because it may be a network exception.
// If the actor is not in the creating map, it means that the actor has been
// cancelled as the worker or node is dead, just do nothing in this case because
// the gcs_actor_manager will reconstruct it again.
auto iter = node_to_workers_when_creating_.find(actor->GetNodeID());
if (iter != node_to_workers_when_creating_.end()) {
auto worker_iter = iter->second.find(actor->GetWorkerID());
if (worker_iter != iter->second.end()) {
// The worker is still in the creating map.
if (status.ok()) {
// Remove related core worker client.
RAY_CHECK(core_worker_clients_.erase(actor->GetWorkerID()) != 0);
// Remove related worker in phase of creating.
iter->second.erase(worker_iter);
if (iter->second.empty()) {
node_to_workers_when_creating_.erase(iter);
}
RAY_LOG(INFO) << "Succeeded in creating actor " << actor->GetActorID()
<< " on worker " << worker->GetWorkerID() << " at node "
<< actor->GetNodeID();
schedule_success_handler_(actor);
} else {
RetryCreatingActorOnWorker(actor, worker);
}
}
}
});
if (!status.ok()) {
RetryCreatingActorOnWorker(actor, worker);
}
}
void GcsActorScheduler::RetryCreatingActorOnWorker(
std::shared_ptr<GcsActor> actor, std::shared_ptr<GcsLeasedWorker> worker) {
execute_after(io_context_,
[this, actor, worker] { DoRetryCreatingActorOnWorker(actor, worker); },
RayConfig::instance().gcs_create_actor_retry_interval_ms());
}
void GcsActorScheduler::DoRetryCreatingActorOnWorker(
std::shared_ptr<GcsActor> actor, std::shared_ptr<GcsLeasedWorker> worker) {
auto iter = node_to_workers_when_creating_.find(actor->GetNodeID());
if (iter != node_to_workers_when_creating_.end()) {
auto worker_iter = iter->second.find(actor->GetWorkerID());
if (worker_iter != iter->second.end()) {
// The worker is still in the creating map, try create again.
// The worker is erased from creating map only when `CancelOnNode`
// or `CancelOnWorker` or the actor is created successfully.
RAY_LOG(INFO) << "Retry creating actor " << actor->GetActorID() << " on worker "
<< worker->GetWorkerID() << " at node " << actor->GetNodeID();
CreateActorOnWorker(actor, worker);
}
}
}
std::shared_ptr<rpc::GcsNodeInfo> GcsActorScheduler::SelectNodeRandomly() const {
auto &alive_nodes = gcs_node_manager_.GetAllAliveNodes();
if (alive_nodes.empty()) {
return nullptr;
}
static std::mt19937_64 gen_(
std::chrono::high_resolution_clock::now().time_since_epoch().count());
std::uniform_int_distribution<int> distribution(0, alive_nodes.size() - 1);
int key_index = distribution(gen_);
int index = 0;
auto iter = alive_nodes.begin();
for (; index != key_index && iter != alive_nodes.end(); ++index, ++iter)
;
return iter->second;
}
std::shared_ptr<WorkerLeaseInterface> GcsActorScheduler::GetOrConnectLeaseClient(
const rpc::Address &raylet_address) {
auto node_id = ClientID::FromBinary(raylet_address.raylet_id());
auto iter = remote_lease_clients_.find(node_id);
if (iter == remote_lease_clients_.end()) {
auto lease_client = lease_client_factory_(raylet_address);
iter = remote_lease_clients_.emplace(node_id, std::move(lease_client)).first;
}
return iter->second;
}
std::shared_ptr<rpc::CoreWorkerClientInterface>
GcsActorScheduler::GetOrConnectCoreWorkerClient(const rpc::Address &worker_address) {
auto worker_id = WorkerID::FromBinary(worker_address.worker_id());
auto iter = core_worker_clients_.find(worker_id);
if (iter == core_worker_clients_.end()) {
iter = core_worker_clients_.emplace(worker_id, client_factory_(worker_address)).first;
}
return iter->second;
}
} // namespace gcs
} // namespace ray

View file

@ -0,0 +1,243 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef RAY_GCS_ACTOR_SCHEDULER_H
#define RAY_GCS_ACTOR_SCHEDULER_H
#include <ray/common/id.h>
#include <ray/common/task/task_execution_spec.h>
#include <ray/common/task/task_spec.h>
#include <ray/gcs/accessor.h>
#include <ray/protobuf/gcs_service.pb.h>
#include <ray/raylet/raylet_client.h>
#include <ray/rpc/node_manager/node_manager_client.h>
#include <ray/rpc/worker/core_worker_client.h>
#include <queue>
#include "absl/container/flat_hash_map.h"
#include "absl/container/flat_hash_set.h"
#include "gcs_node_manager.h"
namespace ray {
namespace gcs {
using LeaseClientFactoryFn =
std::function<std::shared_ptr<WorkerLeaseInterface>(const rpc::Address &address)>;
class GcsActor;
/// GcsActorScheduler is responsible for scheduling actors registered to GcsActorManager.
/// This class is not thread-safe.
class GcsActorScheduler {
public:
/// Create a GcsActorScheduler
///
/// \param io_context The main event loop.
/// \param actor_info_accessor Used to flush actor info to storage.
/// \param gcs_node_manager The node manager which is used when scheduling.
/// \param schedule_failure_handler Invoked when there are no available nodes to
/// schedule actors.
/// \param schedule_success_handler Invoked when actors are created on the worker
/// successfully.
/// \param lease_client_factory Factory to create remote lease client, default factor
/// will be used if not set.
/// \param client_factory Factory to create remote core worker client, default factor
/// will be used if not set.
explicit GcsActorScheduler(
boost::asio::io_context &io_context, gcs::ActorInfoAccessor &actor_info_accessor,
const GcsNodeManager &gcs_node_manager,
std::function<void(std::shared_ptr<GcsActor>)> schedule_failure_handler,
std::function<void(std::shared_ptr<GcsActor>)> schedule_success_handler,
LeaseClientFactoryFn lease_client_factory = nullptr,
rpc::ClientFactoryFn client_factory = nullptr);
virtual ~GcsActorScheduler() = default;
/// Schedule the specified actor.
/// If there is no available nodes then the `schedule_failed_handler_` will be
/// triggered, otherwise the actor will be scheduled until succeed or canceled.
///
/// \param actor to be scheduled.
void Schedule(std::shared_ptr<GcsActor> actor);
/// Cancel all actors that are being scheduled to the specified node.
///
/// \param node_id ID of the node where the worker is located.
/// \return ID list of actors associated with the specified node id.
std::vector<ActorID> CancelOnNode(const ClientID &node_id);
/// Cancel the actor that is being scheduled to the specified worker.
///
/// \param node_id ID of the node where the worker is located.
/// \param worker_id ID of the worker that the actor is creating on.
/// \return ID of actor associated with the specified node id and worker id.
ActorID CancelOnWorker(const ClientID &node_id, const WorkerID &worker_id);
protected:
/// The GcsLeasedWorker is kind of abstraction of remote leased worker inside raylet. It
/// contains the address of remote leased worker as well as the leased resources and the
/// ID of the actor associated with this worker. Through this class, we can easily get
/// the WorkerID, Endpoint, NodeID and the associated ActorID of the remote worker.
class GcsLeasedWorker {
public:
/// Create a GcsLeasedWorker
///
/// \param address the Address of the remote leased worker.
/// \param resources the resources that leased from the remote node(raylet).
/// \param actor_id ID of the actor associated with this leased worker.
explicit GcsLeasedWorker(rpc::Address address,
std::vector<rpc::ResourceMapEntry> resources,
const ActorID &actor_id)
: address_(std::move(address)),
resources_(std::move(resources)),
assigned_actor_id_(actor_id) {}
virtual ~GcsLeasedWorker() = default;
/// Get the Address of this leased worker.
const rpc::Address &GetAddress() const { return address_; }
/// Get the ip address of this leased worker.
const std::string &GetIpAddress() const { return address_.ip_address(); }
/// Get the listening port of the leased worker at remote side.
uint16_t GetPort() const { return address_.port(); }
/// Get the WorkerID of this leased worker.
WorkerID GetWorkerID() const { return WorkerID::FromBinary(address_.worker_id()); }
/// Get the NodeID of this leased worker.
ClientID GetNodeID() const { return ClientID::FromBinary(address_.raylet_id()); }
/// Get the id of the actor which is assigned to this leased worker.
ActorID GetAssignedActorID() const { return assigned_actor_id_; }
/// Get the leased resources.
const std::vector<rpc::ResourceMapEntry> &GetLeasedResources() const {
return resources_;
}
protected:
/// The address of the remote leased worker.
rpc::Address address_;
/// The resources leased from remote node.
std::vector<rpc::ResourceMapEntry> resources_;
/// Id of the actor assigned to this worker.
ActorID assigned_actor_id_;
};
/// Lease a worker from the specified node for the specified actor.
///
/// \param actor A description of the actor to create. This object has the resource
/// specification needed to lease workers from the specified node.
/// \param node The node that the worker will be leased from.
void LeaseWorkerFromNode(std::shared_ptr<GcsActor> actor,
std::shared_ptr<rpc::GcsNodeInfo> node);
/// Retry leasing a worker from the specified node for the specified actor.
/// Make it a virtual method so that the io_context_ could be mocked out.
///
/// \param actor A description of the actor to create. This object has the resource
/// specification needed to lease workers from the specified node.
/// \param node The node that the worker will be leased from.
virtual void RetryLeasingWorkerFromNode(std::shared_ptr<GcsActor> actor,
std::shared_ptr<rpc::GcsNodeInfo> node);
/// This method is only invoked inside `RetryLeasingWorkerFromNode`, the purpose of this
/// is to make it easy to write unit tests.
///
/// \param actor A description of the actor to create. This object has the resource
/// specification needed to lease workers from the specified node.
/// \param node The node that the worker will be leased from.
void DoRetryLeasingWorkerFromNode(std::shared_ptr<GcsActor> actor,
std::shared_ptr<rpc::GcsNodeInfo> node);
/// Handler to process a granted lease.
///
/// \param actor Contains the resources needed to lease workers from the specified node.
/// \param reply The reply of `RequestWorkerLeaseRequest`.
void HandleWorkerLeasedReply(std::shared_ptr<GcsActor> actor,
const rpc::RequestWorkerLeaseReply &reply);
/// Create the specified actor on the specified worker.
///
/// \param actor The actor to be created.
/// \param worker The worker that the actor will created on.
void CreateActorOnWorker(std::shared_ptr<GcsActor> actor,
std::shared_ptr<GcsLeasedWorker> worker);
/// Retry creating the specified actor on the specified worker asynchoronously.
/// Make it a virtual method so that the io_context_ could be mocked out.
///
/// \param actor The actor to be created.
/// \param worker The worker that the actor will created on.
virtual void RetryCreatingActorOnWorker(std::shared_ptr<GcsActor> actor,
std::shared_ptr<GcsLeasedWorker> worker);
/// This method is only invoked inside `RetryCreatingActorOnWorker`, the purpose of this
/// is to make it easy to write unit tests.
///
/// \param actor The actor to be created.
/// \param worker The worker that the actor will created on.
void DoRetryCreatingActorOnWorker(std::shared_ptr<GcsActor> actor,
std::shared_ptr<GcsLeasedWorker> worker);
/// Select a node from alive nodes randomly.
std::shared_ptr<rpc::GcsNodeInfo> SelectNodeRandomly() const;
/// Get an existing lease client or connect a new one.
std::shared_ptr<WorkerLeaseInterface> GetOrConnectLeaseClient(
const rpc::Address &raylet_address);
/// Get or create CoreWorkerClient to communicate with the remote leased worker.
std::shared_ptr<rpc::CoreWorkerClientInterface> GetOrConnectCoreWorkerClient(
const rpc::Address &worker_address);
protected:
/// The io loop which is used to construct `client_call_manager_` and delay execution of
/// tasks(e.g. execute_after).
boost::asio::io_context &io_context_;
/// The `ClientCallManager` object that is shared by all `NodeManagerWorkerClient`s.
rpc::ClientCallManager client_call_manager_;
/// The actor info accessor.
gcs::ActorInfoAccessor &actor_info_accessor_;
/// Map from node ID to the set of actors for whom we are trying to acquire a lease from
/// that node. This is needed so that we can retry lease requests from the node until we
/// receive a reply or the node is removed.
absl::flat_hash_map<ClientID, absl::flat_hash_set<ActorID>>
node_to_actors_when_leasing_;
/// Map from node ID to the workers on which we are trying to create actors. This is
/// needed so that we can cancel actor creation requests if the worker is removed.
absl::flat_hash_map<ClientID,
absl::flat_hash_map<WorkerID, std::shared_ptr<GcsLeasedWorker>>>
node_to_workers_when_creating_;
/// The cached node clients which are used to communicate with raylet to lease workers.
absl::flat_hash_map<ClientID, std::shared_ptr<WorkerLeaseInterface>>
remote_lease_clients_;
/// The cached core worker clients which are used to communicate with leased worker.
absl::flat_hash_map<WorkerID, std::shared_ptr<rpc::CoreWorkerClientInterface>>
core_worker_clients_;
/// Reference of GcsNodeManager.
const GcsNodeManager &gcs_node_manager_;
/// The handler to handle the scheduling failures.
std::function<void(std::shared_ptr<GcsActor>)> schedule_failure_handler_;
/// The handler to handle the successful scheduling.
std::function<void(std::shared_ptr<GcsActor>)> schedule_success_handler_;
/// Factory for producing new clients to request leases from remote nodes.
LeaseClientFactoryFn lease_client_factory_;
/// Factory for producing new core worker clients.
rpc::ClientFactoryFn client_factory_;
};
} // namespace gcs
} // namespace ray
#endif // RAY_GCS_ACTOR_SCHEDULER_H

View file

@ -1,16 +1,29 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "gcs_node_manager.h" #include "gcs_node_manager.h"
#include <ray/common/ray_config.h> #include <ray/common/ray_config.h>
#include <ray/gcs/pb_util.h> #include <ray/gcs/pb_util.h>
#include <ray/rpc/node_manager/node_manager_client.h> #include <ray/protobuf/gcs.pb.h>
#include "ray/gcs/redis_gcs_client.h"
namespace ray { namespace ray {
namespace gcs { namespace gcs {
GcsNodeManager::GcsNodeManager(boost::asio::io_service &io_service, GcsNodeManager::GcsNodeManager(boost::asio::io_service &io_service,
std::shared_ptr<gcs::RedisGcsClient> gcs_client) gcs::NodeInfoAccessor &node_info_accessor,
: client_call_manager_(io_service), gcs::ErrorInfoAccessor &error_info_accessor)
gcs_client_(std::move(gcs_client)), : node_info_accessor_(node_info_accessor),
error_info_accessor_(error_info_accessor),
num_heartbeats_timeout_(RayConfig::instance().num_heartbeats_timeout()), num_heartbeats_timeout_(RayConfig::instance().num_heartbeats_timeout()),
heartbeat_timer_(io_service) { heartbeat_timer_(io_service) {
Start(); Start();
@ -24,26 +37,26 @@ void GcsNodeManager::HandleHeartbeat(const ClientID &node_id,
void GcsNodeManager::Start() { void GcsNodeManager::Start() {
RAY_LOG(INFO) << "Starting gcs node manager."; RAY_LOG(INFO) << "Starting gcs node manager.";
const auto lookup_callback = [this](Status status, const auto lookup_callback =
const std::vector<GcsNodeInfo> &node_info_list) { [this](Status status, const std::vector<rpc::GcsNodeInfo> &node_info_list) {
for (const auto &node_info : node_info_list) { for (const auto &node_info : node_info_list) {
if (node_info.state() != rpc::GcsNodeInfo::DEAD) { if (node_info.state() != rpc::GcsNodeInfo::DEAD) {
// If there're any existing alive clients in client table, add them to // If there're any existing alive clients in client table, add them to
// our `heartbeats_` cache. Thus, if they died before monitor starts, // our `heartbeats_` cache. Thus, if they died before monitor starts,
// we can also detect their death. // we can also detect their death.
// Use `emplace` instead of `operator []` because we just want to add this // Use `emplace` instead of `operator []` because we just want to add this
// client to `heartbeats_` only if it has not yet received heartbeat event. // client to `heartbeats_` only if it has not yet received heartbeat event.
// Besides, it is not necessary to add an empty `HeartbeatTableData` // Besides, it is not necessary to add an empty `HeartbeatTableData`
// to `heartbeat_buffer_` as it doesn't make sense to broadcast an empty // to `heartbeat_buffer_` as it doesn't make sense to broadcast an empty
// message to the cluster and it's ok to add it when actually receive // message to the cluster and it's ok to add it when actually receive
// its heartbeat event. // its heartbeat event.
heartbeats_.emplace(ClientID::FromBinary(node_info.node_id()), heartbeats_.emplace(ClientID::FromBinary(node_info.node_id()),
num_heartbeats_timeout_); num_heartbeats_timeout_);
} }
} }
Tick(); Tick();
}; };
RAY_CHECK_OK(gcs_client_->Nodes().AsyncGetAll(lookup_callback)); RAY_CHECK_OK(node_info_accessor_.AsyncGetAll(lookup_callback));
} }
/// A periodic timer that checks for timed out clients. /// A periodic timer that checks for timed out clients.
@ -55,55 +68,57 @@ void GcsNodeManager::Tick() {
void GcsNodeManager::DetectDeadNodes() { void GcsNodeManager::DetectDeadNodes() {
for (auto it = heartbeats_.begin(); it != heartbeats_.end();) { for (auto it = heartbeats_.begin(); it != heartbeats_.end();) {
it->second = it->second - 1; auto current = it++;
if (it->second == 0) { current->second = current->second - 1;
if (dead_nodes_.count(it->first) == 0) { if (current->second == 0) {
auto node_id = it->first; if (dead_nodes_.count(current->first) == 0) {
auto node_id = current->first;
RAY_LOG(WARNING) << "Node timed out: " << node_id; RAY_LOG(WARNING) << "Node timed out: " << node_id;
auto lookup_callback = [this, node_id](Status status, auto lookup_callback = [this, node_id](
const std::vector<GcsNodeInfo> &all_node) { Status status,
const std::vector<rpc::GcsNodeInfo> &all_node) {
RAY_CHECK_OK(status); RAY_CHECK_OK(status);
bool marked = false; bool marked = false;
for (const auto &node : all_node) { for (const auto &node : all_node) {
if (node_id.Binary() == node.node_id() && node.state() == GcsNodeInfo::DEAD) { if (node_id.Binary() == node.node_id() &&
node.state() == rpc::GcsNodeInfo::DEAD) {
// The node has been marked dead by itself. // The node has been marked dead by itself.
marked = true; marked = true;
break; break;
} }
} }
if (!marked) { if (!marked) {
RAY_CHECK_OK(gcs_client_->Nodes().AsyncUnregister(node_id, nullptr)); RemoveNode(node_id);
RAY_CHECK_OK(node_info_accessor_.AsyncUnregister(node_id, nullptr));
// Broadcast a warning to all of the drivers indicating that the node // Broadcast a warning to all of the drivers indicating that the node
// has been marked as dead. // has been marked as dead.
// TODO(rkn): Define this constant somewhere else. // TODO(rkn): Define this constant somewhere else.
std::string type = "node_removed"; std::string type = "node_removed";
std::ostringstream error_message; std::ostringstream error_message;
error_message << "The node with node ID " << node_id error_message << "The node with node id " << node_id
<< " has been marked dead because the monitor" << " has been marked dead because the monitor"
<< " has missed too many heartbeats from it."; << " has missed too many heartbeats from it.";
auto error_data_ptr = auto error_data_ptr =
gcs::CreateErrorTableData(type, error_message.str(), current_time_ms()); gcs::CreateErrorTableData(type, error_message.str(), current_time_ms());
RAY_CHECK_OK( RAY_CHECK_OK(
gcs_client_->Errors().AsyncReportJobError(error_data_ptr, nullptr)); error_info_accessor_.AsyncReportJobError(error_data_ptr, nullptr));
} }
}; };
RAY_CHECK_OK(gcs_client_->Nodes().AsyncGetAll(lookup_callback)); RAY_CHECK_OK(node_info_accessor_.AsyncGetAll(lookup_callback));
dead_nodes_.insert(node_id); dead_nodes_.insert(node_id);
} }
it = heartbeats_.erase(it); heartbeats_.erase(current);
} else {
it++;
} }
} }
} }
void GcsNodeManager::SendBatchedHeartbeat() { void GcsNodeManager::SendBatchedHeartbeat() {
if (!heartbeat_buffer_.empty()) { if (!heartbeat_buffer_.empty()) {
auto batch = std::make_shared<HeartbeatBatchTableData>(); auto batch = std::make_shared<rpc::HeartbeatBatchTableData>();
for (const auto &heartbeat : heartbeat_buffer_) { for (const auto &heartbeat : heartbeat_buffer_) {
batch->add_batch()->CopyFrom(heartbeat.second); batch->add_batch()->CopyFrom(heartbeat.second);
} }
RAY_CHECK_OK(gcs_client_->Nodes().AsyncReportBatchHeartbeat(batch, nullptr)); RAY_CHECK_OK(node_info_accessor_.AsyncReportBatchHeartbeat(batch, nullptr));
heartbeat_buffer_.clear(); heartbeat_buffer_.clear();
} }
} }
@ -123,5 +138,42 @@ void GcsNodeManager::ScheduleTick() {
}); });
} }
std::shared_ptr<rpc::GcsNodeInfo> GcsNodeManager::GetNode(
const ray::ClientID &node_id) const {
auto iter = alive_nodes_.find(node_id);
if (iter == alive_nodes_.end()) {
return nullptr;
}
return iter->second;
}
const absl::flat_hash_map<ClientID, std::shared_ptr<rpc::GcsNodeInfo>>
&GcsNodeManager::GetAllAliveNodes() const {
return alive_nodes_;
}
void GcsNodeManager::AddNode(std::shared_ptr<rpc::GcsNodeInfo> node) {
auto node_id = ClientID::FromBinary(node->node_id());
auto iter = alive_nodes_.find(node_id);
if (iter == alive_nodes_.end()) {
alive_nodes_.emplace(node_id, node);
for (auto &listener : node_added_listeners_) {
listener(node);
}
}
}
void GcsNodeManager::RemoveNode(const ray::ClientID &node_id) {
auto iter = alive_nodes_.find(node_id);
if (iter != alive_nodes_.end()) {
auto node = std::move(iter->second);
alive_nodes_.erase(iter);
for (auto &listener : node_removed_listeners_) {
listener(node);
}
}
}
} // namespace gcs } // namespace gcs
} // namespace ray } // namespace ray

View file

@ -1,23 +1,82 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef RAY_GCS_NODE_MANAGER_H #ifndef RAY_GCS_NODE_MANAGER_H
#define RAY_GCS_NODE_MANAGER_H #define RAY_GCS_NODE_MANAGER_H
#include <ray/common/id.h> #include <ray/common/id.h>
#include <ray/gcs/accessor.h>
#include <ray/protobuf/gcs.pb.h> #include <ray/protobuf/gcs.pb.h>
#include <ray/rpc/client_call.h> #include <ray/rpc/client_call.h>
#include "absl/container/flat_hash_map.h"
#include "absl/container/flat_hash_set.h"
namespace ray { namespace ray {
namespace gcs { namespace gcs {
class RedisGcsClient;
/// GcsNodeManager is responsible for managing and monitoring nodes. /// GcsNodeManager is responsible for managing and monitoring nodes.
/// This class is not thread-safe.
class GcsNodeManager { class GcsNodeManager {
public: public:
/// Create a GcsNodeManager. /// Create a GcsNodeManager.
/// ///
/// \param io_service The event loop to run the monitor on. /// \param io_service The event loop to run the monitor on.
/// \param gcs_client The client of gcs to access/pub/sub data. /// \param node_info_accessor The node info accessor.
/// \param error_info_accessor The error info accessor, which is used to report error
/// when detecting the death of nodes.
explicit GcsNodeManager(boost::asio::io_service &io_service, explicit GcsNodeManager(boost::asio::io_service &io_service,
std::shared_ptr<gcs::RedisGcsClient> gcs_client); gcs::NodeInfoAccessor &node_info_accessor,
gcs::ErrorInfoAccessor &error_info_accessor);
/// Add an alive node.
///
/// \param node The info of the node to be added.
void AddNode(std::shared_ptr<rpc::GcsNodeInfo> node);
/// Remove from alive nodes.
///
/// \param node_id The ID of the node to be removed.
void RemoveNode(const ClientID &node_id);
/// Get alive node by ID.
///
/// \param node_id The id of the node.
/// \return the node if it is alive else return nullptr.
std::shared_ptr<rpc::GcsNodeInfo> GetNode(const ClientID &node_id) const;
/// Get all alive nodes.
///
/// \return all alive nodes.
const absl::flat_hash_map<ClientID, std::shared_ptr<rpc::GcsNodeInfo>>
&GetAllAliveNodes() const;
/// Add listener to monitor the remove action of nodes.
///
/// \param listener The handler which process the remove of nodes.
void AddNodeRemovedListener(
std::function<void(std::shared_ptr<rpc::GcsNodeInfo>)> listener) {
RAY_CHECK(listener);
node_removed_listeners_.emplace_back(std::move(listener));
}
/// Add listener to monitor the add action of nodes.
///
/// \param listener The handler which process the add of nodes.
void AddNodeAddedListener(
std::function<void(std::shared_ptr<rpc::GcsNodeInfo>)> listener) {
RAY_CHECK(listener);
node_added_listeners_.emplace_back(std::move(listener));
}
/// Handle a heartbeat from a Raylet. /// Handle a heartbeat from a Raylet.
/// ///
@ -47,20 +106,29 @@ class GcsNodeManager {
void ScheduleTick(); void ScheduleTick();
private: private:
rpc::ClientCallManager client_call_manager_; /// Alive nodes.
/// A client to the GCS, through which heartbeats are received. absl::flat_hash_map<ClientID, std::shared_ptr<rpc::GcsNodeInfo>> alive_nodes_;
std::shared_ptr<gcs::RedisGcsClient> gcs_client_; /// Node info accessor.
gcs::NodeInfoAccessor &node_info_accessor_;
/// Error info accessor.
gcs::ErrorInfoAccessor &error_info_accessor_;
/// The number of heartbeats that can be missed before a node is removed. /// The number of heartbeats that can be missed before a node is removed.
int64_t num_heartbeats_timeout_; int64_t num_heartbeats_timeout_;
/// A timer that ticks every heartbeat_timeout_ms_ milliseconds. /// A timer that ticks every heartbeat_timeout_ms_ milliseconds.
boost::asio::deadline_timer heartbeat_timer_; boost::asio::deadline_timer heartbeat_timer_;
/// For each Raylet that we receive a heartbeat from, the number of ticks /// For each Raylet that we receive a heartbeat from, the number of ticks
/// that may pass before the Raylet will be declared dead. /// that may pass before the Raylet will be declared dead.
std::unordered_map<ClientID, int64_t> heartbeats_; absl::flat_hash_map<ClientID, int64_t> heartbeats_;
/// The Raylets that have been marked as dead in gcs. /// The Raylets that have been marked as dead in gcs.
std::unordered_set<ClientID> dead_nodes_; absl::flat_hash_set<ClientID> dead_nodes_;
/// A buffer containing heartbeats received from node managers in the last tick. /// A buffer containing heartbeats received from node managers in the last tick.
std::unordered_map<ClientID, rpc::HeartbeatTableData> heartbeat_buffer_; absl::flat_hash_map<ClientID, rpc::HeartbeatTableData> heartbeat_buffer_;
/// Listeners which monitors the addition of nodes.
std::vector<std::function<void(std::shared_ptr<rpc::GcsNodeInfo>)>>
node_added_listeners_;
/// Listeners which monitors the removal of nodes.
std::vector<std::function<void(std::shared_ptr<rpc::GcsNodeInfo>)>>
node_removed_listeners_;
}; };
} // namespace gcs } // namespace gcs

View file

@ -15,6 +15,7 @@
#include "gcs_server.h" #include "gcs_server.h"
#include "actor_info_handler_impl.h" #include "actor_info_handler_impl.h"
#include "error_info_handler_impl.h" #include "error_info_handler_impl.h"
#include "gcs_actor_manager.h"
#include "gcs_node_manager.h" #include "gcs_node_manager.h"
#include "job_info_handler_impl.h" #include "job_info_handler_impl.h"
#include "node_info_handler_impl.h" #include "node_info_handler_impl.h"
@ -45,6 +46,9 @@ void GcsServer::Start() {
main_service_, redis_gcs_client_->primary_context(), [this]() { Stop(); }); main_service_, redis_gcs_client_->primary_context(), [this]() { Stop(); });
gcs_redis_failure_detector_->Start(); gcs_redis_failure_detector_->Start();
// Init gcs actor manager
InitGcsActorManager();
// Register rpc service. // Register rpc service.
job_info_handler_ = InitJobInfoHandler(); job_info_handler_ = InitJobInfoHandler();
job_info_service_.reset(new rpc::JobInfoGrpcService(main_service_, *job_info_handler_)); job_info_service_.reset(new rpc::JobInfoGrpcService(main_service_, *job_info_handler_));
@ -119,7 +123,15 @@ void GcsServer::InitBackendClient() {
} }
void GcsServer::InitGcsNodeManager() { void GcsServer::InitGcsNodeManager() {
gcs_node_manager_ = std::make_shared<GcsNodeManager>(main_service_, redis_gcs_client_); RAY_CHECK(redis_gcs_client_ != nullptr);
gcs_node_manager_ = std::make_shared<GcsNodeManager>(
main_service_, redis_gcs_client_->Nodes(), redis_gcs_client_->Errors());
}
void GcsServer::InitGcsActorManager() {
RAY_CHECK(redis_gcs_client_ != nullptr && gcs_node_manager_ != nullptr);
gcs_actor_manager_ = std::make_shared<GcsActorManager>(
main_service_, redis_gcs_client_->Actors(), *gcs_node_manager_);
} }
std::unique_ptr<rpc::JobInfoHandler> GcsServer::InitJobInfoHandler() { std::unique_ptr<rpc::JobInfoHandler> GcsServer::InitJobInfoHandler() {
@ -129,7 +141,7 @@ std::unique_ptr<rpc::JobInfoHandler> GcsServer::InitJobInfoHandler() {
std::unique_ptr<rpc::ActorInfoHandler> GcsServer::InitActorInfoHandler() { std::unique_ptr<rpc::ActorInfoHandler> GcsServer::InitActorInfoHandler() {
return std::unique_ptr<rpc::DefaultActorInfoHandler>( return std::unique_ptr<rpc::DefaultActorInfoHandler>(
new rpc::DefaultActorInfoHandler(*redis_gcs_client_)); new rpc::DefaultActorInfoHandler(*redis_gcs_client_, *gcs_actor_manager_));
} }
std::unique_ptr<rpc::NodeInfoHandler> GcsServer::InitNodeInfoHandler() { std::unique_ptr<rpc::NodeInfoHandler> GcsServer::InitNodeInfoHandler() {
@ -198,7 +210,7 @@ std::unique_ptr<rpc::ErrorInfoHandler> GcsServer::InitErrorInfoHandler() {
std::unique_ptr<rpc::WorkerInfoHandler> GcsServer::InitWorkerInfoHandler() { std::unique_ptr<rpc::WorkerInfoHandler> GcsServer::InitWorkerInfoHandler() {
return std::unique_ptr<rpc::DefaultWorkerInfoHandler>( return std::unique_ptr<rpc::DefaultWorkerInfoHandler>(
new rpc::DefaultWorkerInfoHandler(*redis_gcs_client_)); new rpc::DefaultWorkerInfoHandler(*redis_gcs_client_, *gcs_actor_manager_));
} }
} // namespace gcs } // namespace gcs

View file

@ -34,6 +34,7 @@ struct GcsServerConfig {
}; };
class GcsNodeManager; class GcsNodeManager;
class GcsActorManager;
/// The GcsServer will take over all requests from ServiceBasedGcsClient and transparent /// The GcsServer will take over all requests from ServiceBasedGcsClient and transparent
/// transmit the command to the backend reliable storage for the time being. /// transmit the command to the backend reliable storage for the time being.
@ -72,6 +73,9 @@ class GcsServer {
/// cluster. /// cluster.
virtual void InitGcsNodeManager(); virtual void InitGcsNodeManager();
/// Initialize the gcs node manager.
virtual void InitGcsActorManager();
/// The job info handler /// The job info handler
virtual std::unique_ptr<rpc::JobInfoHandler> InitJobInfoHandler(); virtual std::unique_ptr<rpc::JobInfoHandler> InitJobInfoHandler();
@ -114,6 +118,8 @@ class GcsServer {
std::shared_ptr<GcsNodeManager> gcs_node_manager_; std::shared_ptr<GcsNodeManager> gcs_node_manager_;
/// The gcs redis failure detector. /// The gcs redis failure detector.
std::shared_ptr<GcsRedisFailureDetector> gcs_redis_failure_detector_; std::shared_ptr<GcsRedisFailureDetector> gcs_redis_failure_detector_;
/// The gcs actor manager
std::shared_ptr<GcsActorManager> gcs_actor_manager_;
/// Job info handler and service /// Job info handler and service
std::unique_ptr<rpc::JobInfoHandler> job_info_handler_; std::unique_ptr<rpc::JobInfoHandler> job_info_handler_;
std::unique_ptr<rpc::JobInfoGrpcService> job_info_service_; std::unique_ptr<rpc::JobInfoGrpcService> job_info_service_;

View file

@ -23,7 +23,7 @@ void DefaultNodeInfoHandler::HandleRegisterNode(
rpc::SendReplyCallback send_reply_callback) { rpc::SendReplyCallback send_reply_callback) {
ClientID node_id = ClientID::FromBinary(request.node_info().node_id()); ClientID node_id = ClientID::FromBinary(request.node_info().node_id());
RAY_LOG(DEBUG) << "Registering node info, node id = " << node_id; RAY_LOG(DEBUG) << "Registering node info, node id = " << node_id;
gcs_node_manager_.AddNode(std::make_shared<rpc::GcsNodeInfo>(request.node_info()));
auto on_done = [node_id, reply, send_reply_callback](Status status) { auto on_done = [node_id, reply, send_reply_callback](Status status) {
if (!status.ok()) { if (!status.ok()) {
RAY_LOG(ERROR) << "Failed to register node info: " << status.ToString() RAY_LOG(ERROR) << "Failed to register node info: " << status.ToString()
@ -44,6 +44,7 @@ void DefaultNodeInfoHandler::HandleUnregisterNode(
rpc::SendReplyCallback send_reply_callback) { rpc::SendReplyCallback send_reply_callback) {
ClientID node_id = ClientID::FromBinary(request.node_id()); ClientID node_id = ClientID::FromBinary(request.node_id());
RAY_LOG(DEBUG) << "Unregistering node info, node id = " << node_id; RAY_LOG(DEBUG) << "Unregistering node info, node id = " << node_id;
gcs_node_manager_.RemoveNode(node_id);
auto on_done = [node_id, reply, send_reply_callback](Status status) { auto on_done = [node_id, reply, send_reply_callback](Status status) {
if (!status.ok()) { if (!status.ok()) {

View file

@ -0,0 +1,173 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <ray/gcs/gcs_server/test/gcs_test_util.h>
#include <memory>
#include "gtest/gtest.h"
namespace ray {
class MockedGcsActorManager : public gcs::GcsActorManager {
public:
explicit MockedGcsActorManager(boost::asio::io_context &io_context,
gcs::ActorInfoAccessor &actor_info_accessor,
gcs::GcsNodeManager &gcs_node_manager,
gcs::LeaseClientFactoryFn lease_client_factory = nullptr,
rpc::ClientFactoryFn client_factory = nullptr)
: gcs::GcsActorManager(io_context, actor_info_accessor, gcs_node_manager,
lease_client_factory, client_factory) {
gcs_actor_scheduler_.reset(new Mocker::MockedGcsActorScheduler(
io_context, actor_info_accessor, gcs_node_manager,
/*schedule_failure_handler=*/
[this](std::shared_ptr<gcs::GcsActor> actor) {
// When there are no available nodes to schedule the actor the
// gcs_actor_scheduler will treat it as failed and invoke this handler. In
// this case, the actor should be appended to the `pending_actors_` and wait
// for the registration of new node.
pending_actors_.emplace_back(std::move(actor));
},
/*schedule_success_handler=*/
[this](std::shared_ptr<gcs::GcsActor> actor) {
OnActorCreateSuccess(std::move(actor));
},
std::move(lease_client_factory), std::move(client_factory)));
}
public:
void ResetLeaseClientFactory(gcs::LeaseClientFactoryFn lease_client_factory) {
auto gcs_actor_scheduler =
dynamic_cast<Mocker::MockedGcsActorScheduler *>(gcs_actor_scheduler_.get());
gcs_actor_scheduler->ResetLeaseClientFactory(std::move(lease_client_factory));
}
void ResetClientFactory(rpc::ClientFactoryFn client_factory) {
auto gcs_actor_scheduler =
dynamic_cast<Mocker::MockedGcsActorScheduler *>(gcs_actor_scheduler_.get());
gcs_actor_scheduler->ResetClientFactory(std::move(client_factory));
}
const absl::flat_hash_map<ActorID, std::shared_ptr<gcs::GcsActor>>
&GetAllRegisteredActors() const {
return registered_actors_;
}
const std::vector<std::shared_ptr<gcs::GcsActor>> &GetAllPendingActors() const {
return pending_actors_;
}
};
class GcsActorManagerTest : public ::testing::Test {
public:
void SetUp() override {
raylet_client_ = std::make_shared<Mocker::MockRayletClient>();
worker_client_ = std::make_shared<Mocker::MockWorkerClient>();
gcs_node_manager_ = std::make_shared<gcs::GcsNodeManager>(
io_service_, node_info_accessor_, error_info_accessor_);
gcs_actor_manager_ = std::make_shared<MockedGcsActorManager>(
io_service_, actor_info_accessor_, *gcs_node_manager_,
/*lease_client_factory=*/
[this](const rpc::Address &address) { return raylet_client_; },
/*client_factory=*/
[this](const rpc::Address &address) { return worker_client_; });
}
protected:
boost::asio::io_service io_service_;
Mocker::MockedActorInfoAccessor actor_info_accessor_;
Mocker::MockedNodeInfoAccessor node_info_accessor_;
Mocker::MockedErrorInfoAccessor error_info_accessor_;
std::shared_ptr<Mocker::MockRayletClient> raylet_client_;
std::shared_ptr<Mocker::MockWorkerClient> worker_client_;
std::shared_ptr<gcs::GcsNodeManager> gcs_node_manager_;
std::shared_ptr<MockedGcsActorManager> gcs_actor_manager_;
};
TEST_F(GcsActorManagerTest, TestNormalFlow) {
gcs_actor_manager_->ResetLeaseClientFactory([this](const rpc::Address &address) {
raylet_client_->auto_grant_node_id = ClientID::FromBinary(address.raylet_id());
return raylet_client_;
});
gcs_actor_manager_->ResetClientFactory([this](const rpc::Address &address) {
worker_client_->enable_auto_reply = true;
return worker_client_;
});
auto job_id = JobID::FromInt(1);
auto create_actor_request =
Mocker::GenCreateActorRequest(job_id, /*max_reconstructions=*/2);
std::vector<std::shared_ptr<gcs::GcsActor>> registered_actors;
gcs_actor_manager_->RegisterActor(
create_actor_request, [&registered_actors](std::shared_ptr<gcs::GcsActor> actor) {
registered_actors.emplace_back(actor);
});
ASSERT_EQ(1, registered_actors.size());
ASSERT_EQ(1, gcs_actor_manager_->GetAllRegisteredActors().size());
ASSERT_EQ(1, gcs_actor_manager_->GetAllPendingActors().size());
auto actor = registered_actors.front();
ASSERT_EQ(rpc::ActorTableData::PENDING, actor->GetState());
// Add node_1 and then check if the actor is in state `ALIVE`
auto node_1 = Mocker::GenNodeInfo();
auto node_id_1 = ClientID::FromBinary(node_1->node_id());
gcs_node_manager_->AddNode(node_1);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
ASSERT_EQ(0, gcs_actor_manager_->GetAllPendingActors().size());
ASSERT_EQ(rpc::ActorTableData::ALIVE, actor->GetState());
ASSERT_EQ(node_id_1, actor->GetNodeID());
// Remove node_1 and then check if the actor is in state `RECONSTRUCTING`
gcs_node_manager_->RemoveNode(node_id_1);
ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size());
ASSERT_EQ(1, gcs_actor_manager_->GetAllPendingActors().size());
ASSERT_EQ(rpc::ActorTableData::RECONSTRUCTING, actor->GetState());
// Add node_2 and then check if the actor is alive again.
auto node_2 = Mocker::GenNodeInfo();
auto node_id_2 = ClientID::FromBinary(node_2->node_id());
gcs_node_manager_->AddNode(node_2);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
ASSERT_EQ(0, gcs_actor_manager_->GetAllPendingActors().size());
ASSERT_EQ(rpc::ActorTableData::ALIVE, actor->GetState());
ASSERT_EQ(node_id_2, actor->GetNodeID());
// Add node_3.
auto node_3 = Mocker::GenNodeInfo();
auto node_id_3 = ClientID::FromBinary(node_3->node_id());
gcs_node_manager_->AddNode(node_3);
ASSERT_EQ(2, gcs_node_manager_->GetAllAliveNodes().size());
// Remove node_2 and then check if the actor drift to node_3.
gcs_node_manager_->RemoveNode(node_id_2);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
ASSERT_EQ(0, gcs_actor_manager_->GetAllPendingActors().size());
ASSERT_EQ(rpc::ActorTableData::ALIVE, actor->GetState());
ASSERT_EQ(node_id_3, actor->GetNodeID());
// Remove node_3 and then check if the actor is dead.
gcs_node_manager_->RemoveNode(node_id_3);
ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size());
ASSERT_EQ(0, gcs_actor_manager_->GetAllPendingActors().size());
ASSERT_EQ(rpc::ActorTableData::DEAD, actor->GetState());
}
} // namespace ray
int main(int argc, char **argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View file

@ -0,0 +1,364 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <ray/gcs/gcs_server/test/gcs_test_util.h>
#include <memory>
#include "gmock/gmock.h"
#include "gtest/gtest.h"
namespace ray {
class GcsActorSchedulerTest : public ::testing::Test {
public:
void SetUp() override {
raylet_client_ = std::make_shared<Mocker::MockRayletClient>();
worker_client_ = std::make_shared<Mocker::MockWorkerClient>();
gcs_node_manager_ = std::make_shared<gcs::GcsNodeManager>(
io_service_, node_info_accessor_, error_info_accessor_);
gcs_actor_scheduler_ = std::make_shared<Mocker::MockedGcsActorScheduler>(
io_service_, actor_info_accessor_, *gcs_node_manager_,
/*schedule_failure_handler=*/
[this](std::shared_ptr<gcs::GcsActor> actor) {
failure_actors_.emplace_back(std::move(actor));
},
/*schedule_success_handler=*/
[this](std::shared_ptr<gcs::GcsActor> actor) {
success_actors_.emplace_back(std::move(actor));
},
/*lease_client_factory=*/
[this](const rpc::Address &address) { return raylet_client_; },
/*client_factory=*/
[this](const rpc::Address &address) { return worker_client_; });
}
protected:
boost::asio::io_service io_service_;
Mocker::MockedActorInfoAccessor actor_info_accessor_;
Mocker::MockedNodeInfoAccessor node_info_accessor_;
Mocker::MockedErrorInfoAccessor error_info_accessor_;
std::shared_ptr<Mocker::MockRayletClient> raylet_client_;
std::shared_ptr<Mocker::MockWorkerClient> worker_client_;
std::shared_ptr<gcs::GcsNodeManager> gcs_node_manager_;
std::shared_ptr<Mocker::MockedGcsActorScheduler> gcs_actor_scheduler_;
std::vector<std::shared_ptr<gcs::GcsActor>> success_actors_;
std::vector<std::shared_ptr<gcs::GcsActor>> failure_actors_;
};
TEST_F(GcsActorSchedulerTest, TestScheduleFailedWithZeroNode) {
ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with zero node.
gcs_actor_scheduler_->Schedule(actor);
// The lease request should not be send and the scheduling of actor should fail as there
// are no available nodes.
ASSERT_EQ(raylet_client_->num_workers_requested, 0);
ASSERT_EQ(0, success_actors_.size());
ASSERT_EQ(1, failure_actors_.size());
ASSERT_EQ(actor, failure_actors_.front());
}
TEST_F(GcsActorSchedulerTest, TestScheduleActorSuccess) {
auto node = Mocker::GenNodeInfo();
auto node_id = ClientID::FromBinary(node->node_id());
gcs_node_manager_->AddNode(node);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with 1 available node, and the lease request should be send to the
// node.
gcs_actor_scheduler_->Schedule(actor);
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
// Grant a worker, then the actor creation request should be send to the worker.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(
node->node_manager_address(), node->node_manager_port(), WorkerID::FromRandom(),
node_id, ClientID::Nil()));
ASSERT_EQ(0, raylet_client_->callbacks.size());
ASSERT_EQ(1, worker_client_->callbacks.size());
// Reply the actor creation request, then the actor should be scheduled successfully.
ASSERT_TRUE(worker_client_->ReplyPushTask());
ASSERT_EQ(0, worker_client_->callbacks.size());
ASSERT_EQ(0, failure_actors_.size());
ASSERT_EQ(1, success_actors_.size());
ASSERT_EQ(actor, success_actors_.front());
}
TEST_F(GcsActorSchedulerTest, TestScheduleRetryWhenLeasing) {
auto node = Mocker::GenNodeInfo();
auto node_id = ClientID::FromBinary(node->node_id());
gcs_node_manager_->AddNode(node);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with 1 available node, and the lease request should be send to the
// node.
gcs_actor_scheduler_->Schedule(actor);
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
ASSERT_EQ(0, gcs_actor_scheduler_->num_retry_leasing_count_);
// Mock a IOError reply, then the lease request will retry again.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(
node->node_manager_address(), node->node_manager_port(), WorkerID::FromRandom(),
node_id, ClientID::Nil(), Status::IOError("")));
ASSERT_EQ(1, gcs_actor_scheduler_->num_retry_leasing_count_);
ASSERT_EQ(2, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
// Grant a worker, then the actor creation request should be send to the worker.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(
node->node_manager_address(), node->node_manager_port(), WorkerID::FromRandom(),
node_id, ClientID::Nil()));
ASSERT_EQ(0, raylet_client_->callbacks.size());
ASSERT_EQ(1, worker_client_->callbacks.size());
// Reply the actor creation request, then the actor should be scheduled successfully.
ASSERT_TRUE(worker_client_->ReplyPushTask());
ASSERT_EQ(0, worker_client_->callbacks.size());
ASSERT_EQ(0, failure_actors_.size());
ASSERT_EQ(1, success_actors_.size());
ASSERT_EQ(actor, success_actors_.front());
}
TEST_F(GcsActorSchedulerTest, TestScheduleRetryWhenCreating) {
auto node = Mocker::GenNodeInfo();
auto node_id = ClientID::FromBinary(node->node_id());
gcs_node_manager_->AddNode(node);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with 1 available node, and the lease request should be send to the
// node.
gcs_actor_scheduler_->Schedule(actor);
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
// Grant a worker, then the actor creation request should be send to the worker.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(
node->node_manager_address(), node->node_manager_port(), WorkerID::FromRandom(),
node_id, ClientID::Nil()));
ASSERT_EQ(0, raylet_client_->callbacks.size());
ASSERT_EQ(1, worker_client_->callbacks.size());
ASSERT_EQ(0, gcs_actor_scheduler_->num_retry_creating_count_);
// Reply a IOError, then the actor creation request will retry again.
ASSERT_TRUE(worker_client_->ReplyPushTask(Status::IOError("")));
ASSERT_EQ(1, gcs_actor_scheduler_->num_retry_creating_count_);
ASSERT_EQ(1, worker_client_->callbacks.size());
// Reply the actor creation request, then the actor should be scheduled successfully.
ASSERT_TRUE(worker_client_->ReplyPushTask());
ASSERT_EQ(0, worker_client_->callbacks.size());
ASSERT_EQ(0, failure_actors_.size());
ASSERT_EQ(1, success_actors_.size());
ASSERT_EQ(actor, success_actors_.front());
}
TEST_F(GcsActorSchedulerTest, TestNodeFailedWhenLeasing) {
auto node = Mocker::GenNodeInfo();
auto node_id = ClientID::FromBinary(node->node_id());
gcs_node_manager_->AddNode(node);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with 1 available node, and the lease request should be send to the
// node.
gcs_actor_scheduler_->Schedule(actor);
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
// Remove the node and cancel the scheduling on this node, the scheduling should be
// interrupted.
gcs_node_manager_->RemoveNode(node_id);
ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size());
auto actor_ids = gcs_actor_scheduler_->CancelOnNode(node_id);
ASSERT_EQ(1, actor_ids.size());
ASSERT_EQ(actor->GetActorID(), actor_ids.front());
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
// Grant a worker, which will influence nothing.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(
node->node_manager_address(), node->node_manager_port(), WorkerID::FromRandom(),
node_id, ClientID::Nil()));
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(0, raylet_client_->callbacks.size());
ASSERT_EQ(0, gcs_actor_scheduler_->num_retry_leasing_count_);
ASSERT_EQ(0, success_actors_.size());
ASSERT_EQ(0, failure_actors_.size());
}
TEST_F(GcsActorSchedulerTest, TestNodeFailedWhenCreating) {
auto node = Mocker::GenNodeInfo();
auto node_id = ClientID::FromBinary(node->node_id());
gcs_node_manager_->AddNode(node);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with 1 available node, and the lease request should be send to the
// node.
gcs_actor_scheduler_->Schedule(actor);
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
// Grant a worker, then the actor creation request should be send to the worker.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(
node->node_manager_address(), node->node_manager_port(), WorkerID::FromRandom(),
node_id, ClientID::Nil()));
ASSERT_EQ(0, raylet_client_->callbacks.size());
ASSERT_EQ(1, worker_client_->callbacks.size());
// Remove the node and cancel the scheduling on this node, the scheduling should be
// interrupted.
gcs_node_manager_->RemoveNode(node_id);
ASSERT_EQ(0, gcs_node_manager_->GetAllAliveNodes().size());
auto actor_ids = gcs_actor_scheduler_->CancelOnNode(node_id);
ASSERT_EQ(1, actor_ids.size());
ASSERT_EQ(actor->GetActorID(), actor_ids.front());
ASSERT_EQ(1, worker_client_->callbacks.size());
// Reply the actor creation request, which will influence nothing.
ASSERT_TRUE(worker_client_->ReplyPushTask());
ASSERT_EQ(0, worker_client_->callbacks.size());
ASSERT_EQ(0, gcs_actor_scheduler_->num_retry_creating_count_);
ASSERT_EQ(0, success_actors_.size());
ASSERT_EQ(0, failure_actors_.size());
}
TEST_F(GcsActorSchedulerTest, TestWorkerFailedWhenCreating) {
auto node = Mocker::GenNodeInfo();
auto node_id = ClientID::FromBinary(node->node_id());
gcs_node_manager_->AddNode(node);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with 1 available node, and the lease request should be send to the
// node.
gcs_actor_scheduler_->Schedule(actor);
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
// Grant a worker, then the actor creation request should be send to the worker.
auto worker_id = WorkerID::FromRandom();
ASSERT_TRUE(raylet_client_->GrantWorkerLease(node->node_manager_address(),
node->node_manager_port(), worker_id,
node_id, ClientID::Nil()));
ASSERT_EQ(0, raylet_client_->callbacks.size());
ASSERT_EQ(1, worker_client_->callbacks.size());
// Cancel the scheduling on this node, the scheduling should be interrupted.
ASSERT_EQ(actor->GetActorID(),
gcs_actor_scheduler_->CancelOnWorker(node_id, worker_id));
ASSERT_EQ(1, worker_client_->callbacks.size());
// Reply the actor creation request, which will influence nothing.
ASSERT_TRUE(worker_client_->ReplyPushTask());
ASSERT_EQ(0, worker_client_->callbacks.size());
ASSERT_EQ(0, gcs_actor_scheduler_->num_retry_creating_count_);
ASSERT_EQ(0, success_actors_.size());
ASSERT_EQ(0, failure_actors_.size());
}
TEST_F(GcsActorSchedulerTest, TestSpillback) {
auto node1 = Mocker::GenNodeInfo();
auto node_id_1 = ClientID::FromBinary(node1->node_id());
gcs_node_manager_->AddNode(node1);
ASSERT_EQ(1, gcs_node_manager_->GetAllAliveNodes().size());
auto job_id = JobID::FromInt(1);
auto create_actor_request = Mocker::GenCreateActorRequest(job_id);
auto actor = std::make_shared<gcs::GcsActor>(create_actor_request);
// Schedule the actor with 1 available node, and the lease request should be send to the
// node.
gcs_actor_scheduler_->Schedule(actor);
ASSERT_EQ(1, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
// Add another node.
auto node2 = Mocker::GenNodeInfo();
auto node_id_2 = ClientID::FromBinary(node2->node_id());
gcs_node_manager_->AddNode(node2);
ASSERT_EQ(2, gcs_node_manager_->GetAllAliveNodes().size());
// Grant with a spillback node(node2), and the lease request should be send to the
// node2.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(node2->node_manager_address(),
node2->node_manager_port(),
WorkerID::Nil(), node_id_1, node_id_2));
ASSERT_EQ(2, raylet_client_->num_workers_requested);
ASSERT_EQ(1, raylet_client_->callbacks.size());
ASSERT_EQ(0, worker_client_->callbacks.size());
// Grant a worker, then the actor creation request should be send to the worker.
ASSERT_TRUE(raylet_client_->GrantWorkerLease(
node2->node_manager_address(), node2->node_manager_port(), WorkerID::FromRandom(),
node_id_2, ClientID::Nil()));
ASSERT_EQ(0, raylet_client_->callbacks.size());
ASSERT_EQ(1, worker_client_->callbacks.size());
// Reply the actor creation request, then the actor should be scheduled successfully.
ASSERT_TRUE(worker_client_->ReplyPushTask());
ASSERT_EQ(0, worker_client_->callbacks.size());
ASSERT_EQ(node_id_2, actor->GetNodeID());
ASSERT_EQ(0, failure_actors_.size());
ASSERT_EQ(1, success_actors_.size());
ASSERT_EQ(actor, success_actors_.front());
}
} // namespace ray
int main(int argc, char **argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View file

@ -0,0 +1,85 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <ray/gcs/gcs_server/test/gcs_test_util.h>
#include <memory>
#include "gtest/gtest.h"
namespace ray {
class GcsNodeManagerTest : public ::testing::Test {};
TEST_F(GcsNodeManagerTest, TestManagement) {
boost::asio::io_service io_service;
auto node_info_accessor = Mocker::MockedNodeInfoAccessor();
auto error_info_accessor = Mocker::MockedErrorInfoAccessor();
gcs::GcsNodeManager node_manager(io_service, node_info_accessor, error_info_accessor);
// Test Add/Get/Remove functionality.
auto node = Mocker::GenNodeInfo();
auto node_id = ClientID::FromBinary(node->node_id());
node_manager.AddNode(node);
ASSERT_EQ(node, node_manager.GetNode(node_id));
node_manager.RemoveNode(node_id);
ASSERT_EQ(nullptr, node_manager.GetNode(node_id));
}
TEST_F(GcsNodeManagerTest, TestListener) {
boost::asio::io_service io_service;
auto node_info_accessor = Mocker::MockedNodeInfoAccessor();
auto error_info_accessor = Mocker::MockedErrorInfoAccessor();
gcs::GcsNodeManager node_manager(io_service, node_info_accessor, error_info_accessor);
// Test AddNodeAddedListener.
int node_count = 1000;
std::vector<std::shared_ptr<rpc::GcsNodeInfo>> added_nodes;
node_manager.AddNodeAddedListener(
[&added_nodes](std::shared_ptr<rpc::GcsNodeInfo> node) {
added_nodes.emplace_back(std::move(node));
});
for (int i = 0; i < node_count; ++i) {
auto node = Mocker::GenNodeInfo();
node_manager.AddNode(node);
}
ASSERT_EQ(node_count, added_nodes.size());
// Test GetAllAliveNodes.
auto &alive_nodes = node_manager.GetAllAliveNodes();
ASSERT_EQ(added_nodes.size(), alive_nodes.size());
for (const auto &node : added_nodes) {
ASSERT_EQ(1, alive_nodes.count(ClientID::FromBinary(node->node_id())));
}
// Test AddNodeRemovedListener.
std::vector<std::shared_ptr<rpc::GcsNodeInfo>> removed_nodes;
node_manager.AddNodeRemovedListener(
[&removed_nodes](std::shared_ptr<rpc::GcsNodeInfo> node) {
removed_nodes.emplace_back(std::move(node));
});
for (int i = 0; i < node_count; ++i) {
node_manager.RemoveNode(ClientID::FromBinary(added_nodes[i]->node_id()));
}
ASSERT_EQ(node_count, removed_nodes.size());
ASSERT_TRUE(node_manager.GetAllAliveNodes().empty());
for (int i = 0; i < node_count; ++i) {
ASSERT_EQ(added_nodes[i], removed_nodes[i]);
}
}
} // namespace ray
int main(int argc, char **argv) {
::testing::InitGoogleTest(&argc, argv);
return RUN_ALL_TESTS();
}

View file

@ -0,0 +1,392 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef RAY_GCS_TEST_UTIL_H
#define RAY_GCS_TEST_UTIL_H
#include <ray/common/task/task.h>
#include <ray/common/task/task_util.h>
#include <ray/common/test_util.h>
#include <ray/gcs/gcs_server/gcs_actor_manager.h>
#include <ray/gcs/gcs_server/gcs_actor_scheduler.h>
#include <ray/gcs/gcs_server/gcs_node_manager.h>
#include <ray/util/asio_util.h>
#include <memory>
#include <utility>
namespace ray {
struct Mocker {
static TaskSpecification GenActorCreationTask(const JobID &job_id,
int max_reconstructions = 100) {
TaskSpecBuilder builder;
rpc::Address empty_address;
ray::FunctionDescriptor empty_descriptor =
ray::FunctionDescriptorBuilder::BuildPython("", "", "", "");
auto actor_id = ActorID::Of(job_id, RandomTaskId(), 0);
auto task_id = TaskID::ForActorCreationTask(actor_id);
builder.SetCommonTaskSpec(task_id, Language::PYTHON, empty_descriptor, job_id,
TaskID::Nil(), 0, TaskID::Nil(), empty_address, 1, {}, {});
builder.SetActorCreationTaskSpec(actor_id, max_reconstructions);
return builder.Build();
}
static rpc::CreateActorRequest GenCreateActorRequest(const JobID &job_id,
int max_reconstructions = 100) {
rpc::CreateActorRequest request;
auto actor_creation_task_spec = GenActorCreationTask(job_id, max_reconstructions);
request.mutable_task_spec()->CopyFrom(actor_creation_task_spec.GetMessage());
return request;
}
static std::shared_ptr<rpc::GcsNodeInfo> GenNodeInfo(uint16_t port = 0) {
auto node = std::make_shared<rpc::GcsNodeInfo>();
node->set_node_id(ClientID::FromRandom().Binary());
node->set_node_manager_port(port);
node->set_node_manager_address("127.0.0.1");
return node;
}
class MockWorkerClient : public rpc::CoreWorkerClientInterface {
public:
ray::Status PushNormalTask(
std::unique_ptr<rpc::PushTaskRequest> request,
const rpc::ClientCallback<rpc::PushTaskReply> &callback) override {
callbacks.push_back(callback);
if (enable_auto_reply) {
ReplyPushTask();
}
return Status::OK();
}
bool ReplyPushTask(Status status = Status::OK(), bool exit = false) {
if (callbacks.size() == 0) {
return false;
}
auto callback = callbacks.front();
auto reply = rpc::PushTaskReply();
if (exit) {
reply.set_worker_exiting(true);
}
callback(status, reply);
callbacks.pop_front();
return true;
}
bool enable_auto_reply = false;
std::list<rpc::ClientCallback<rpc::PushTaskReply>> callbacks;
};
class MockRayletClient : public WorkerLeaseInterface {
public:
ray::Status ReturnWorker(int worker_port, const WorkerID &worker_id,
bool disconnect_worker) override {
if (disconnect_worker) {
num_workers_disconnected++;
} else {
num_workers_returned++;
}
return Status::OK();
}
ray::Status RequestWorkerLease(
const ray::TaskSpecification &resource_spec,
const rpc::ClientCallback<rpc::RequestWorkerLeaseReply> &callback) override {
num_workers_requested += 1;
callbacks.push_back(callback);
if (!auto_grant_node_id.IsNil()) {
GrantWorkerLease("", 0, WorkerID::FromRandom(), auto_grant_node_id,
ClientID::Nil());
}
return Status::OK();
}
ray::Status CancelWorkerLease(
const TaskID &task_id,
const rpc::ClientCallback<rpc::CancelWorkerLeaseReply> &callback) override {
num_leases_canceled += 1;
cancel_callbacks.push_back(callback);
return Status::OK();
}
// Trigger reply to RequestWorkerLease.
bool GrantWorkerLease(const std::string &address, int port, const WorkerID &worker_id,
const ClientID &raylet_id, const ClientID &retry_at_raylet_id,
Status status = Status::OK()) {
rpc::RequestWorkerLeaseReply reply;
if (!retry_at_raylet_id.IsNil()) {
reply.mutable_retry_at_raylet_address()->set_ip_address(address);
reply.mutable_retry_at_raylet_address()->set_port(port);
reply.mutable_retry_at_raylet_address()->set_raylet_id(
retry_at_raylet_id.Binary());
} else {
reply.mutable_worker_address()->set_ip_address(address);
reply.mutable_worker_address()->set_port(port);
reply.mutable_worker_address()->set_raylet_id(raylet_id.Binary());
reply.mutable_worker_address()->set_worker_id(worker_id.Binary());
}
if (callbacks.size() == 0) {
return false;
} else {
auto callback = callbacks.front();
callback(status, reply);
callbacks.pop_front();
return true;
}
}
bool ReplyCancelWorkerLease(bool success = true) {
rpc::CancelWorkerLeaseReply reply;
reply.set_success(success);
if (cancel_callbacks.size() == 0) {
return false;
} else {
auto callback = cancel_callbacks.front();
callback(Status::OK(), reply);
cancel_callbacks.pop_front();
return true;
}
}
~MockRayletClient() {}
int num_workers_requested = 0;
int num_workers_returned = 0;
int num_workers_disconnected = 0;
int num_leases_canceled = 0;
ClientID auto_grant_node_id;
std::list<rpc::ClientCallback<rpc::RequestWorkerLeaseReply>> callbacks = {};
std::list<rpc::ClientCallback<rpc::CancelWorkerLeaseReply>> cancel_callbacks = {};
};
class MockedGcsActorScheduler : public gcs::GcsActorScheduler {
public:
using gcs::GcsActorScheduler::GcsActorScheduler;
void ResetLeaseClientFactory(gcs::LeaseClientFactoryFn lease_client_factory) {
lease_client_factory_ = std::move(lease_client_factory);
}
void ResetClientFactory(rpc::ClientFactoryFn client_factory) {
client_factory_ = std::move(client_factory);
}
protected:
void RetryLeasingWorkerFromNode(std::shared_ptr<gcs::GcsActor> actor,
std::shared_ptr<rpc::GcsNodeInfo> node) override {
++num_retry_leasing_count_;
DoRetryLeasingWorkerFromNode(actor, node);
}
void RetryCreatingActorOnWorker(std::shared_ptr<gcs::GcsActor> actor,
std::shared_ptr<GcsLeasedWorker> worker) override {
++num_retry_creating_count_;
DoRetryCreatingActorOnWorker(actor, worker);
}
public:
int num_retry_leasing_count_ = 0;
int num_retry_creating_count_ = 0;
};
class MockedActorInfoAccessor : public gcs::ActorInfoAccessor {
public:
Status GetAll(std::vector<rpc::ActorTableData> *actor_table_data_list) override {
return Status::NotImplemented("");
}
Status AsyncGet(
const ActorID &actor_id,
const gcs::OptionalItemCallback<rpc::ActorTableData> &callback) override {
return Status::NotImplemented("");
}
Status AsyncCreateActor(const TaskSpecification &task_spec,
const gcs::StatusCallback &callback) override {
return Status::NotImplemented("");
}
Status AsyncRegister(const std::shared_ptr<rpc::ActorTableData> &data_ptr,
const gcs::StatusCallback &callback) override {
return Status::NotImplemented("");
}
Status AsyncUpdate(const ActorID &actor_id,
const std::shared_ptr<rpc::ActorTableData> &data_ptr,
const gcs::StatusCallback &callback) override {
if (callback) {
callback(Status::OK());
}
return Status::OK();
}
Status AsyncSubscribeAll(
const gcs::SubscribeCallback<ActorID, rpc::ActorTableData> &subscribe,
const gcs::StatusCallback &done) override {
return Status::NotImplemented("");
}
Status AsyncSubscribe(
const ActorID &actor_id,
const gcs::SubscribeCallback<ActorID, rpc::ActorTableData> &subscribe,
const gcs::StatusCallback &done) override {
return Status::NotImplemented("");
}
Status AsyncUnsubscribe(const ActorID &actor_id,
const gcs::StatusCallback &done) override {
return Status::NotImplemented("");
}
Status AsyncAddCheckpoint(const std::shared_ptr<rpc::ActorCheckpointData> &data_ptr,
const gcs::StatusCallback &callback) override {
return Status::NotImplemented("");
}
Status AsyncGetCheckpoint(
const ActorCheckpointID &checkpoint_id, const ActorID &actor_id,
const gcs::OptionalItemCallback<rpc::ActorCheckpointData> &callback) override {
return Status::NotImplemented("");
}
Status AsyncGetCheckpointID(
const ActorID &actor_id,
const gcs::OptionalItemCallback<rpc::ActorCheckpointIdData> &callback) override {
return Status::NotImplemented("");
}
};
class MockedNodeInfoAccessor : public gcs::NodeInfoAccessor {
public:
Status RegisterSelf(const rpc::GcsNodeInfo &local_node_info) override {
return Status::NotImplemented("");
}
Status UnregisterSelf() override { return Status::NotImplemented(""); }
const ClientID &GetSelfId() const override {
static ClientID node_id;
return node_id;
}
const rpc::GcsNodeInfo &GetSelfInfo() const override {
static rpc::GcsNodeInfo node_info;
return node_info;
}
Status AsyncRegister(const rpc::GcsNodeInfo &node_info,
const gcs::StatusCallback &callback) override {
return Status::NotImplemented("");
}
Status AsyncUnregister(const ClientID &node_id,
const gcs::StatusCallback &callback) override {
if (callback) {
callback(Status::OK());
}
return Status::OK();
}
Status AsyncGetAll(
const gcs::MultiItemCallback<rpc::GcsNodeInfo> &callback) override {
if (callback) {
callback(Status::OK(), {});
}
return Status::OK();
}
Status AsyncSubscribeToNodeChange(
const gcs::SubscribeCallback<ClientID, rpc::GcsNodeInfo> &subscribe,
const gcs::StatusCallback &done) override {
return Status::NotImplemented("");
}
boost::optional<rpc::GcsNodeInfo> Get(const ClientID &node_id) const override {
return boost::none;
}
const std::unordered_map<ClientID, rpc::GcsNodeInfo> &GetAll() const override {
static std::unordered_map<ClientID, rpc::GcsNodeInfo> node_info_list;
return node_info_list;
}
bool IsRemoved(const ClientID &node_id) const override { return false; }
Status AsyncGetResources(
const ClientID &node_id,
const gcs::OptionalItemCallback<ResourceMap> &callback) override {
return Status::NotImplemented("");
}
Status AsyncUpdateResources(const ClientID &node_id, const ResourceMap &resources,
const gcs::StatusCallback &callback) override {
return Status::NotImplemented("");
}
Status AsyncDeleteResources(const ClientID &node_id,
const std::vector<std::string> &resource_names,
const gcs::StatusCallback &callback) override {
return Status::NotImplemented("");
}
Status AsyncSubscribeToResources(
const gcs::SubscribeCallback<ClientID, gcs::ResourceChangeNotification>
&subscribe,
const gcs::StatusCallback &done) override {
return Status::NotImplemented("");
}
Status AsyncReportHeartbeat(const std::shared_ptr<rpc::HeartbeatTableData> &data_ptr,
const gcs::StatusCallback &callback) override {
return Status::NotImplemented("");
}
Status AsyncSubscribeHeartbeat(
const gcs::SubscribeCallback<ClientID, rpc::HeartbeatTableData> &subscribe,
const gcs::StatusCallback &done) override {
return Status::NotImplemented("");
}
Status AsyncReportBatchHeartbeat(
const std::shared_ptr<rpc::HeartbeatBatchTableData> &data_ptr,
const gcs::StatusCallback &callback) override {
if (callback) {
callback(Status::OK());
}
return Status::OK();
}
Status AsyncSubscribeBatchHeartbeat(
const gcs::ItemCallback<rpc::HeartbeatBatchTableData> &subscribe,
const gcs::StatusCallback &done) override {
return Status::NotImplemented("");
}
};
class MockedErrorInfoAccessor : public gcs::ErrorInfoAccessor {
public:
Status AsyncReportJobError(const std::shared_ptr<rpc::ErrorTableData> &data_ptr,
const gcs::StatusCallback &callback) override {
if (callback) {
callback(Status::OK());
}
return Status::OK();
}
};
};
} // namespace ray
#endif // RAY_GCS_TEST_UTIL_H

View file

@ -24,6 +24,11 @@ void DefaultWorkerInfoHandler::HandleReportWorkerFailure(
RAY_LOG(DEBUG) << "Reporting worker failure, " << worker_address.DebugString(); RAY_LOG(DEBUG) << "Reporting worker failure, " << worker_address.DebugString();
auto worker_failure_data = std::make_shared<WorkerFailureData>(); auto worker_failure_data = std::make_shared<WorkerFailureData>();
worker_failure_data->CopyFrom(request.worker_failure()); worker_failure_data->CopyFrom(request.worker_failure());
auto need_reschedule = !worker_failure_data->intentional_disconnect();
auto node_id = ClientID::FromBinary(worker_address.raylet_id());
auto worker_id = WorkerID::FromBinary(worker_address.worker_id());
gcs_actor_manager_.ReconstructActorOnWorker(node_id, worker_id, need_reschedule);
auto on_done = [worker_address, reply, send_reply_callback](Status status) { auto on_done = [worker_address, reply, send_reply_callback](Status status) {
if (!status.ok()) { if (!status.ok()) {
RAY_LOG(ERROR) << "Failed to report worker failure, " RAY_LOG(ERROR) << "Failed to report worker failure, "

View file

@ -15,6 +15,7 @@
#ifndef RAY_GCS_WORKER_INFO_HANDLER_IMPL_H #ifndef RAY_GCS_WORKER_INFO_HANDLER_IMPL_H
#define RAY_GCS_WORKER_INFO_HANDLER_IMPL_H #define RAY_GCS_WORKER_INFO_HANDLER_IMPL_H
#include "gcs_actor_manager.h"
#include "ray/gcs/redis_gcs_client.h" #include "ray/gcs/redis_gcs_client.h"
#include "ray/rpc/gcs_server/gcs_rpc_server.h" #include "ray/rpc/gcs_server/gcs_rpc_server.h"
@ -24,8 +25,9 @@ namespace rpc {
/// This implementation class of `WorkerInfoHandler`. /// This implementation class of `WorkerInfoHandler`.
class DefaultWorkerInfoHandler : public rpc::WorkerInfoHandler { class DefaultWorkerInfoHandler : public rpc::WorkerInfoHandler {
public: public:
explicit DefaultWorkerInfoHandler(gcs::RedisGcsClient &gcs_client) explicit DefaultWorkerInfoHandler(gcs::RedisGcsClient &gcs_client,
: gcs_client_(gcs_client) {} gcs::GcsActorManager &gcs_actor_manager)
: gcs_client_(gcs_client), gcs_actor_manager_(gcs_actor_manager) {}
void HandleReportWorkerFailure(const ReportWorkerFailureRequest &request, void HandleReportWorkerFailure(const ReportWorkerFailureRequest &request,
ReportWorkerFailureReply *reply, ReportWorkerFailureReply *reply,
@ -37,6 +39,7 @@ class DefaultWorkerInfoHandler : public rpc::WorkerInfoHandler {
private: private:
gcs::RedisGcsClient &gcs_client_; gcs::RedisGcsClient &gcs_client_;
gcs::GcsActorManager &gcs_actor_manager_;
}; };
} // namespace rpc } // namespace rpc

View file

@ -84,13 +84,15 @@ inline std::shared_ptr<ray::rpc::ActorTableData> CreateActorTableData(
/// Helper function to produce worker failure data. /// Helper function to produce worker failure data.
inline std::shared_ptr<ray::rpc::WorkerFailureData> CreateWorkerFailureData( inline std::shared_ptr<ray::rpc::WorkerFailureData> CreateWorkerFailureData(
const ClientID &raylet_id, const WorkerID &worker_id, const std::string &address, const ClientID &raylet_id, const WorkerID &worker_id, const std::string &address,
int32_t port, int64_t timestamp = std::time(nullptr)) { int32_t port, int64_t timestamp = std::time(nullptr),
bool intentional_disconnect = false) {
auto worker_failure_info_ptr = std::make_shared<ray::rpc::WorkerFailureData>(); auto worker_failure_info_ptr = std::make_shared<ray::rpc::WorkerFailureData>();
worker_failure_info_ptr->mutable_worker_address()->set_raylet_id(raylet_id.Binary()); worker_failure_info_ptr->mutable_worker_address()->set_raylet_id(raylet_id.Binary());
worker_failure_info_ptr->mutable_worker_address()->set_worker_id(worker_id.Binary()); worker_failure_info_ptr->mutable_worker_address()->set_worker_id(worker_id.Binary());
worker_failure_info_ptr->mutable_worker_address()->set_ip_address(address); worker_failure_info_ptr->mutable_worker_address()->set_ip_address(address);
worker_failure_info_ptr->mutable_worker_address()->set_port(port); worker_failure_info_ptr->mutable_worker_address()->set_port(port);
worker_failure_info_ptr->set_timestamp(timestamp); worker_failure_info_ptr->set_timestamp(timestamp);
worker_failure_info_ptr->set_intentional_disconnect(intentional_disconnect);
return worker_failure_info_ptr; return worker_failure_info_ptr;
} }

View file

@ -63,6 +63,14 @@ Status RedisLogBasedActorInfoAccessor::AsyncGet(
on_done); on_done);
} }
Status RedisLogBasedActorInfoAccessor::AsyncCreateActor(
const ray::TaskSpecification &task_spec, const ray::gcs::StatusCallback &callback) {
const std::string error_msg =
"Unsupported method of AsyncCreateActor in RedisLogBasedActorInfoAccessor.";
RAY_LOG(FATAL) << error_msg;
return Status::Invalid(error_msg);
}
Status RedisLogBasedActorInfoAccessor::AsyncRegister( Status RedisLogBasedActorInfoAccessor::AsyncRegister(
const std::shared_ptr<ActorTableData> &data_ptr, const StatusCallback &callback) { const std::shared_ptr<ActorTableData> &data_ptr, const StatusCallback &callback) {
auto on_success = [callback](RedisGcsClient *client, const ActorID &actor_id, auto on_success = [callback](RedisGcsClient *client, const ActorID &actor_id,

View file

@ -15,6 +15,7 @@
#ifndef RAY_GCS_REDIS_ACCESSOR_H #ifndef RAY_GCS_REDIS_ACCESSOR_H
#define RAY_GCS_REDIS_ACCESSOR_H #define RAY_GCS_REDIS_ACCESSOR_H
#include <ray/common/task/task_spec.h>
#include "ray/common/id.h" #include "ray/common/id.h"
#include "ray/gcs/accessor.h" #include "ray/gcs/accessor.h"
#include "ray/gcs/callback.h" #include "ray/gcs/callback.h"
@ -41,6 +42,9 @@ class RedisLogBasedActorInfoAccessor : public ActorInfoAccessor {
Status AsyncGet(const ActorID &actor_id, Status AsyncGet(const ActorID &actor_id,
const OptionalItemCallback<ActorTableData> &callback) override; const OptionalItemCallback<ActorTableData> &callback) override;
Status AsyncCreateActor(const TaskSpecification &task_spec,
const StatusCallback &callback) override;
Status AsyncRegister(const std::shared_ptr<ActorTableData> &data_ptr, Status AsyncRegister(const std::shared_ptr<ActorTableData> &data_ptr,
const StatusCallback &callback) override; const StatusCallback &callback) override;

View file

@ -70,7 +70,11 @@ Status RedisGcsClient::Connect(boost::asio::io_service &io_service) {
resource_table_.reset(new DynamicResourceTable({primary_context}, this)); resource_table_.reset(new DynamicResourceTable({primary_context}, this));
worker_failure_table_.reset(new WorkerFailureTable(shard_contexts, this)); worker_failure_table_.reset(new WorkerFailureTable(shard_contexts, this));
actor_accessor_.reset(new RedisLogBasedActorInfoAccessor(this)); if (RayConfig::instance().gcs_service_enabled()) {
actor_accessor_.reset(new RedisActorInfoAccessor(this));
} else {
actor_accessor_.reset(new RedisLogBasedActorInfoAccessor(this));
}
job_accessor_.reset(new RedisJobInfoAccessor(this)); job_accessor_.reset(new RedisJobInfoAccessor(this));
object_accessor_.reset(new RedisObjectInfoAccessor(this)); object_accessor_.reset(new RedisObjectInfoAccessor(this));
node_accessor_.reset(new RedisNodeInfoAccessor(this)); node_accessor_.reset(new RedisNodeInfoAccessor(this));

View file

@ -99,13 +99,15 @@ message TaskTableData {
message ActorTableData { message ActorTableData {
// State of an actor. // State of an actor.
enum ActorState { enum ActorState {
// Actor is pending.
PENDING = 0;
// Actor is alive. // Actor is alive.
ALIVE = 0; ALIVE = 1;
// Actor is dead, now being reconstructed. // Actor is dead, now being reconstructed.
// After reconstruction finishes, the state will become alive again. // After reconstruction finishes, the state will become alive again.
RECONSTRUCTING = 1; RECONSTRUCTING = 2;
// Actor is already dead and won't be reconstructed. // Actor is already dead and won't be reconstructed.
DEAD = 2; DEAD = 3;
} }
// The ID of the actor that was created. // The ID of the actor that was created.
bytes actor_id = 1; bytes actor_id = 1;
@ -131,6 +133,8 @@ message ActorTableData {
bool is_detached = 11; bool is_detached = 11;
// Timestamp that the actor is created or reconstructed. // Timestamp that the actor is created or reconstructed.
double timestamp = 12; double timestamp = 12;
// The task specification of this actor's creation task.
TaskSpec task_spec = 13;
} }
message ErrorTableData { message ErrorTableData {
@ -295,6 +299,8 @@ message WorkerFailureData {
Address worker_address = 1; Address worker_address = 1;
// The UNIX timestamp at which the worker failed. // The UNIX timestamp at which the worker failed.
int64 timestamp = 3; int64 timestamp = 3;
// Is intentional disconnect
bool intentional_disconnect = 4;
} }
// This enum type is used as object's metadata to indicate the object's creating // This enum type is used as object's metadata to indicate the object's creating

View file

@ -108,6 +108,8 @@ message GetActorCheckpointIDReply {
// Service for actor info access. // Service for actor info access.
service ActorInfoGcsService { service ActorInfoGcsService {
// Create actor via gcs service
rpc CreateActor(CreateActorRequest) returns (CreateActorReply);
// Get actor data from GCS Service. // Get actor data from GCS Service.
rpc GetActorInfo(GetActorInfoRequest) returns (GetActorInfoReply); rpc GetActorInfo(GetActorInfoRequest) returns (GetActorInfoReply);
// Register an actor to GCS Service. // Register an actor to GCS Service.
@ -366,3 +368,11 @@ service WorkerInfoGcsService {
// Register a worker to GCS Service. // Register a worker to GCS Service.
rpc RegisterWorker(RegisterWorkerRequest) returns (RegisterWorkerReply); rpc RegisterWorker(RegisterWorkerRequest) returns (RegisterWorkerReply);
} }
message CreateActorRequest {
TaskSpec task_spec = 1;
}
message CreateActorReply {
GcsStatus status = 1;
}

View file

@ -805,21 +805,25 @@ void NodeManager::HandleActorStateTransition(const ActorID &actor_id,
if (it == actor_registry_.end()) { if (it == actor_registry_.end()) {
it = actor_registry_.emplace(actor_id, actor_registration).first; it = actor_registry_.emplace(actor_id, actor_registration).first;
} else { } else {
// Only process the state transition if it is to a later state than ours. if (RayConfig::instance().gcs_service_enabled()) {
if (actor_registration.GetState() > it->second.GetState() &&
actor_registration.GetRemainingReconstructions() ==
it->second.GetRemainingReconstructions()) {
// The new state is later than ours if it is about the same lifetime, but
// a greater state.
it->second = actor_registration;
} else if (actor_registration.GetRemainingReconstructions() <
it->second.GetRemainingReconstructions()) {
// The new state is also later than ours it is about a later lifetime of
// the actor.
it->second = actor_registration; it->second = actor_registration;
} else { } else {
// Our state is already at or past the update, so skip the update. // Only process the state transition if it is to a later state than ours.
return; if (actor_registration.GetState() > it->second.GetState() &&
actor_registration.GetRemainingReconstructions() ==
it->second.GetRemainingReconstructions()) {
// The new state is later than ours if it is about the same lifetime, but
// a greater state.
it->second = actor_registration;
} else if (actor_registration.GetRemainingReconstructions() <
it->second.GetRemainingReconstructions()) {
// The new state is also later than ours it is about a later lifetime of
// the actor.
it->second = actor_registration;
} else {
// Our state is already at or past the update, so skip the update.
return;
}
} }
} }
RAY_LOG(DEBUG) << "Actor notification received: actor_id = " << actor_id RAY_LOG(DEBUG) << "Actor notification received: actor_id = " << actor_id
@ -868,13 +872,14 @@ void NodeManager::HandleActorStateTransition(const ActorID &actor_id,
for (auto const &task : removed_tasks) { for (auto const &task : removed_tasks) {
TreatTaskAsFailed(task, ErrorType::ACTOR_DIED); TreatTaskAsFailed(task, ErrorType::ACTOR_DIED);
} }
} else { } else if (actor_registration.GetState() == ActorTableData::RECONSTRUCTING) {
RAY_CHECK(actor_registration.GetState() == ActorTableData::RECONSTRUCTING);
RAY_LOG(DEBUG) << "Actor is being reconstructed: " << actor_id; RAY_LOG(DEBUG) << "Actor is being reconstructed: " << actor_id;
// The actor is dead and needs reconstruction. Attempting to reconstruct its if (!RayConfig::instance().gcs_service_enabled()) {
// creation task. // The actor is dead and needs reconstruction. Attempting to reconstruct its
reconstruction_policy_.ListenAndMaybeReconstruct( // creation task.
actor_registration.GetActorCreationDependency()); reconstruction_policy_.ListenAndMaybeReconstruct(
actor_registration.GetActorCreationDependency());
}
// When an actor fails but can be reconstructed, resubmit all of the queued // When an actor fails but can be reconstructed, resubmit all of the queued
// tasks for that actor. This will mark the tasks as waiting for actor // tasks for that actor. This will mark the tasks as waiting for actor
@ -884,6 +889,9 @@ void NodeManager::HandleActorStateTransition(const ActorID &actor_id,
for (auto const &task : removed_tasks) { for (auto const &task : removed_tasks) {
SubmitTask(task, Lineage()); SubmitTask(task, Lineage());
} }
} else {
RAY_CHECK(actor_registration.GetState() == ActorTableData::PENDING);
// Do nothing.
} }
} }
@ -1119,6 +1127,11 @@ void NodeManager::ProcessRegisterClientRequestMessage(
void NodeManager::HandleDisconnectedActor(const ActorID &actor_id, bool was_local, void NodeManager::HandleDisconnectedActor(const ActorID &actor_id, bool was_local,
bool intentional_disconnect) { bool intentional_disconnect) {
if (RayConfig::instance().gcs_service_enabled()) {
// If gcs actor management is enabled, the gcs will take over the status change of all
// actors.
return;
}
auto actor_entry = actor_registry_.find(actor_id); auto actor_entry = actor_registry_.find(actor_id);
RAY_CHECK(actor_entry != actor_registry_.end()); RAY_CHECK(actor_entry != actor_registry_.end());
auto &actor_registration = actor_entry->second; auto &actor_registration = actor_entry->second;
@ -1224,6 +1237,9 @@ void NodeManager::ProcessDisconnectClientMessage(
// particular, we are no longer waiting for their dependencies. // particular, we are no longer waiting for their dependencies.
if (worker) { if (worker) {
if (is_worker && worker->IsDead()) { if (is_worker && worker->IsDead()) {
// If the worker was killed by us because the driver exited,
// treat it as intentionally disconnected.
intentional_disconnect = true;
// Don't need to unblock the client if it's a worker and is already dead. // Don't need to unblock the client if it's a worker and is already dead.
// Because in this case, its task is already cleaned up. // Because in this case, its task is already cleaned up.
RAY_LOG(DEBUG) << "Skip unblocking worker because it's already dead."; RAY_LOG(DEBUG) << "Skip unblocking worker because it's already dead.";
@ -1245,19 +1261,12 @@ void NodeManager::ProcessDisconnectClientMessage(
// Publish the worker failure. // Publish the worker failure.
auto worker_failure_data_ptr = gcs::CreateWorkerFailureData( auto worker_failure_data_ptr = gcs::CreateWorkerFailureData(
self_node_id_, worker->WorkerId(), initial_config_.node_manager_address, self_node_id_, worker->WorkerId(), initial_config_.node_manager_address,
worker->Port()); worker->Port(), time(nullptr), intentional_disconnect);
RAY_CHECK_OK(gcs_client_->Workers().AsyncReportWorkerFailure(worker_failure_data_ptr, RAY_CHECK_OK(gcs_client_->Workers().AsyncReportWorkerFailure(worker_failure_data_ptr,
nullptr)); nullptr));
} }
if (is_worker) { if (is_worker) {
// The client is a worker.
if (worker->IsDead()) {
// If the worker was killed by us because the driver exited,
// treat it as intentionally disconnected.
intentional_disconnect = true;
}
const ActorID &actor_id = worker->GetActorId(); const ActorID &actor_id = worker->GetActorId();
if (!actor_id.IsNil()) { if (!actor_id.IsNil()) {
// If the worker was an actor, update actor state, reconstruct the actor if needed, // If the worker was an actor, update actor state, reconstruct the actor if needed,
@ -1569,6 +1578,8 @@ void NodeManager::DispatchScheduledTasksToWorkers() {
} }
worker->SetOwnerAddress(spec.CallerAddress()); worker->SetOwnerAddress(spec.CallerAddress());
if (spec.IsActorCreationTask()) { if (spec.IsActorCreationTask()) {
// The actor belongs to this worker now.
worker->AssignActorId(spec.ActorCreationId());
worker->SetLifetimeAllocatedInstances(allocated_instances); worker->SetLifetimeAllocatedInstances(allocated_instances);
} else { } else {
worker->SetAllocatedInstances(allocated_instances); worker->SetAllocatedInstances(allocated_instances);
@ -1647,7 +1658,7 @@ void NodeManager::WaitForTaskArgsRequests(std::pair<ScheduleFn, Task> &work) {
} else { } else {
tasks_to_dispatch_.push_back(work); tasks_to_dispatch_.push_back(work);
} }
}; }
void NodeManager::HandleRequestWorkerLease(const rpc::RequestWorkerLeaseRequest &request, void NodeManager::HandleRequestWorkerLease(const rpc::RequestWorkerLeaseRequest &request,
rpc::RequestWorkerLeaseReply *reply, rpc::RequestWorkerLeaseReply *reply,
@ -2667,6 +2678,15 @@ void NodeManager::FinishAssignedActorTask(Worker &worker, const Task &task) {
worker.MarkDetachedActor(); worker.MarkDetachedActor();
} }
if (RayConfig::instance().gcs_service_enabled()) {
// Gcs server is responsible for notifying other nodes of the changes of actor
// status, and thus raylet doesn't need to handle this anymore.
// And if `new_scheduler_enabled_` is true, this function `FinishAssignedActorTask`
// will not be called because raylet is not aware of the actual task when receiving
// a worker lease request.
return;
}
// Lookup the parent actor id. // Lookup the parent actor id.
auto parent_task_id = task_spec.ParentTaskId(); auto parent_task_id = task_spec.ParentTaskId();
int port = worker.Port(); int port = worker.Port();

View file

@ -96,6 +96,9 @@ class GcsRpcClient {
/// Mark job as finished to gcs server. /// Mark job as finished to gcs server.
VOID_GCS_RPC_CLIENT_METHOD(JobInfoGcsService, MarkJobFinished, job_info_grpc_client_, ) VOID_GCS_RPC_CLIENT_METHOD(JobInfoGcsService, MarkJobFinished, job_info_grpc_client_, )
/// Create actor via GCS Service.
VOID_RPC_CLIENT_METHOD(ActorInfoGcsService, CreateActor, actor_info_grpc_client_, )
/// Get actor data from GCS Service. /// Get actor data from GCS Service.
VOID_GCS_RPC_CLIENT_METHOD(ActorInfoGcsService, GetActorInfo, actor_info_grpc_client_, ) VOID_GCS_RPC_CLIENT_METHOD(ActorInfoGcsService, GetActorInfo, actor_info_grpc_client_, )

View file

@ -94,6 +94,10 @@ class ActorInfoGcsServiceHandler {
public: public:
virtual ~ActorInfoGcsServiceHandler() = default; virtual ~ActorInfoGcsServiceHandler() = default;
virtual void HandleCreateActor(const CreateActorRequest &request,
CreateActorReply *reply,
SendReplyCallback send_reply_callback) = 0;
virtual void HandleGetActorInfo(const GetActorInfoRequest &request, virtual void HandleGetActorInfo(const GetActorInfoRequest &request,
GetActorInfoReply *reply, GetActorInfoReply *reply,
SendReplyCallback send_reply_callback) = 0; SendReplyCallback send_reply_callback) = 0;
@ -135,6 +139,7 @@ class ActorInfoGrpcService : public GrpcService {
void InitServerCallFactories( void InitServerCallFactories(
const std::unique_ptr<grpc::ServerCompletionQueue> &cq, const std::unique_ptr<grpc::ServerCompletionQueue> &cq,
std::vector<std::unique_ptr<ServerCallFactory>> *server_call_factories) override { std::vector<std::unique_ptr<ServerCallFactory>> *server_call_factories) override {
ACTOR_INFO_SERVICE_RPC_HANDLER(CreateActor);
ACTOR_INFO_SERVICE_RPC_HANDLER(GetActorInfo); ACTOR_INFO_SERVICE_RPC_HANDLER(GetActorInfo);
ACTOR_INFO_SERVICE_RPC_HANDLER(RegisterActorInfo); ACTOR_INFO_SERVICE_RPC_HANDLER(RegisterActorInfo);
ACTOR_INFO_SERVICE_RPC_HANDLER(UpdateActorInfo); ACTOR_INFO_SERVICE_RPC_HANDLER(UpdateActorInfo);

31
src/ray/util/asio_util.h Normal file
View file

@ -0,0 +1,31 @@
// Copyright 2017 The Ray Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef RAY_ASIO_UTIL_H
#define RAY_ASIO_UTIL_H
#include <boost/asio.hpp>
inline void execute_after(boost::asio::io_context &io_context,
const std::function<void()> &fn, uint32_t delay_milliseconds) {
auto timer = std::make_shared<boost::asio::deadline_timer>(io_context);
timer->expires_from_now(boost::posix_time::milliseconds(delay_milliseconds));
timer->async_wait([timer, fn](const boost::system::error_code &error) {
if (error != boost::system::errc::operation_canceled && fn) {
fn();
}
});
}
#endif // RAY_ASIO_UTIL_H