Skip to content

Commit

Permalink
[observability][export-api] Write actor events (#47529)
Browse files Browse the repository at this point in the history
- Add back code changes from [observability][export-api] Write actor events #47303
- Separate out actor manager export event test into a separate file so we can skip on windows. Update BUILD rule so all tests in src/ray/gcs/gcs_server/test/export_api are skipped on windows

Signed-off-by: Nikita Vemuri <[email protected]>
Co-authored-by: Nikita Vemuri <[email protected]>
  • Loading branch information
nikitavemuri and Nikita Vemuri authored Sep 9, 2024
1 parent 1dd8d60 commit d3c0708
Show file tree
Hide file tree
Showing 7 changed files with 420 additions and 23 deletions.
38 changes: 19 additions & 19 deletions BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1637,25 +1637,6 @@ ray_cc_test(
],
)

ray_cc_test(
name = "gcs_node_manager_export_event_test",
size = "small",
srcs = [
"src/ray/gcs/gcs_server/test/gcs_node_manager_export_event_test.cc",
],
tags = [
"no_windows",
"team:core"
],
deps = [
":gcs_server_lib",
":gcs_server_test_util",
":gcs_test_util_lib",
":ray_mock",
"@com_google_googletest//:gtest_main",
],
)

ray_cc_test(
name = "gcs_job_manager_test",
size = "small",
Expand Down Expand Up @@ -2363,6 +2344,25 @@ ray_cc_test(
],
)

ray_cc_test(
name = "gcs_export_event_test",
size = "small",
srcs = glob([
"src/ray/gcs/gcs_server/test/export_api/*.cc",
]),
tags = [
"no_windows",
"team:core"
],
deps = [
":gcs_server_lib",
":gcs_server_test_util",
":gcs_test_util_lib",
":ray_mock",
"@com_google_googletest//:gtest_main",
],
)

flatbuffer_cc_library(
name = "node_manager_fbs",
srcs = ["src/ray/raylet/format/node_manager.fbs"],
Expand Down
40 changes: 38 additions & 2 deletions src/ray/gcs/gcs_server/gcs_actor_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,36 @@ const rpc::ActorTableData &GcsActor::GetActorTableData() const {

rpc::ActorTableData *GcsActor::GetMutableActorTableData() { return &actor_table_data_; }

void GcsActor::WriteActorExportEvent() const {
/// Write actor_table_data_ as a export actor event if
/// enable_export_api_write() is enabled.
if (!RayConfig::instance().enable_export_api_write()) {
return;
}
std::shared_ptr<rpc::ExportActorData> export_actor_data_ptr =
std::make_shared<rpc::ExportActorData>();

export_actor_data_ptr->set_actor_id(actor_table_data_.actor_id());
export_actor_data_ptr->set_job_id(actor_table_data_.job_id());
export_actor_data_ptr->set_state(ConvertActorStateToExport(actor_table_data_.state()));
export_actor_data_ptr->set_is_detached(actor_table_data_.is_detached());
export_actor_data_ptr->set_name(actor_table_data_.name());
export_actor_data_ptr->set_pid(actor_table_data_.pid());
export_actor_data_ptr->set_ray_namespace(actor_table_data_.ray_namespace());
export_actor_data_ptr->set_serialized_runtime_env(
actor_table_data_.serialized_runtime_env());
export_actor_data_ptr->set_class_name(actor_table_data_.class_name());
export_actor_data_ptr->mutable_death_cause()->CopyFrom(actor_table_data_.death_cause());
export_actor_data_ptr->mutable_required_resources()->insert(
actor_table_data_.required_resources().begin(),
actor_table_data_.required_resources().end());
export_actor_data_ptr->set_node_id(actor_table_data_.node_id());
export_actor_data_ptr->set_placement_group_id(actor_table_data_.placement_group_id());
export_actor_data_ptr->set_repr_name(actor_table_data_.repr_name());

RayExportEvent(export_actor_data_ptr).SendEvent();
}

rpc::TaskSpec *GcsActor::GetMutableTaskSpec() { return task_spec_.get(); }

const ResourceRequest &GcsActor::GetAcquiredResources() const {
Expand Down Expand Up @@ -660,6 +690,7 @@ Status GcsActorManager::RegisterActor(const ray::rpc::RegisterActorRequest &requ
[this, actor](const Status &status) {
// The backend storage is supposed to be reliable, so the status must be ok.
RAY_CHECK_OK(status);
actor->WriteActorExportEvent();
// If a creator dies before this callback is called, the actor could have
// been already destroyed. It is okay not to invoke a callback because we
// don't need to reply to the creator as it is already dead.
Expand Down Expand Up @@ -756,6 +787,7 @@ Status GcsActorManager::CreateActor(const ray::rpc::CreateActorRequest &request,

// Pub this state for dashboard showing.
RAY_CHECK_OK(gcs_publisher_->PublishActor(actor_id, actor_table_data, nullptr));
actor->WriteActorExportEvent();
RemoveUnresolvedActor(actor);

// Update the registered actor as its creation task specification may have changed due
Expand Down Expand Up @@ -947,10 +979,11 @@ void GcsActorManager::DestroyActor(const ActorID &actor_id,
RAY_CHECK_OK(gcs_table_storage_->ActorTable().Put(
actor->GetActorID(),
*actor_table_data,
[this, actor_id, actor_table_data](Status status) {
[this, actor, actor_id, actor_table_data](Status status) {
RAY_CHECK_OK(gcs_publisher_->PublishActor(
actor_id, *GenActorDataOnlyWithStates(*actor_table_data), nullptr));
RAY_CHECK_OK(gcs_table_storage_->ActorTaskSpecTable().Delete(actor_id, nullptr));
actor->WriteActorExportEvent();
// Destroy placement group owned by this actor.
destroy_owned_placement_group_if_needed_(actor_id);
}));
Expand Down Expand Up @@ -1234,9 +1267,10 @@ void GcsActorManager::ReconstructActor(const ActorID &actor_id,
RAY_CHECK_OK(gcs_table_storage_->ActorTable().Put(
actor_id,
*mutable_actor_table_data,
[this, actor_id, mutable_actor_table_data](Status status) {
[this, actor, actor_id, mutable_actor_table_data](Status status) {
RAY_CHECK_OK(gcs_publisher_->PublishActor(
actor_id, *GenActorDataOnlyWithStates(*mutable_actor_table_data), nullptr));
actor->WriteActorExportEvent();
}));
gcs_actor_scheduler_->Schedule(actor);
} else {
Expand All @@ -1262,6 +1296,7 @@ void GcsActorManager::ReconstructActor(const ActorID &actor_id,
actor_id, *GenActorDataOnlyWithStates(*mutable_actor_table_data), nullptr));
RAY_CHECK_OK(
gcs_table_storage_->ActorTaskSpecTable().Delete(actor_id, nullptr));
actor->WriteActorExportEvent();
}));
// The actor is dead, but we should not remove the entry from the
// registered actors yet. If the actor is owned, we will destroy the actor
Expand Down Expand Up @@ -1368,6 +1403,7 @@ void GcsActorManager::OnActorCreationSuccess(const std::shared_ptr<GcsActor> &ac
[this, actor_id, actor_table_data, actor, reply](Status status) {
RAY_CHECK_OK(gcs_publisher_->PublishActor(
actor_id, *GenActorDataOnlyWithStates(actor_table_data), nullptr));
actor->WriteActorExportEvent();
// Invoke all callbacks for all registration requests of this actor (duplicated
// requests are included) and remove all of them from
// actor_to_create_callbacks_.
Expand Down
25 changes: 25 additions & 0 deletions src/ray/gcs/gcs_server/gcs_actor_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
#include "ray/rpc/gcs_server/gcs_rpc_server.h"
#include "ray/rpc/worker/core_worker_client.h"
#include "ray/util/counter_map.h"
#include "ray/util/event.h"
#include "src/ray/protobuf/gcs_service.pb.h"

namespace ray {
Expand Down Expand Up @@ -187,6 +188,9 @@ class GcsActor {
/// Get the mutable ActorTableData of this actor.
rpc::ActorTableData *GetMutableActorTableData();
rpc::TaskSpec *GetMutableTaskSpec();
/// Write an event containing this actor's ActorTableData
/// to file for the Export API.
void WriteActorExportEvent() const;

const ResourceRequest &GetAcquiredResources() const;
void SetAcquiredResources(ResourceRequest &&resource_request);
Expand All @@ -213,6 +217,27 @@ class GcsActor {
last_metric_state_ = cur_state;
}

rpc::ExportActorData::ActorState ConvertActorStateToExport(
rpc::ActorTableData::ActorState actor_state) const {
switch (actor_state) {
case rpc::ActorTableData::DEPENDENCIES_UNREADY:
return rpc::ExportActorData::DEPENDENCIES_UNREADY;
case rpc::ActorTableData::PENDING_CREATION:
return rpc::ExportActorData::PENDING_CREATION;
case rpc::ActorTableData::ALIVE:
return rpc::ExportActorData::ALIVE;
case rpc::ActorTableData::RESTARTING:
return rpc::ExportActorData::RESTARTING;
case rpc::ActorTableData::DEAD:
return rpc::ExportActorData::DEAD;
default:
// Unknown rpc::ActorTableData::ActorState value
RAY_LOG(FATAL) << "Invalid value for rpc::ActorTableData::ActorState"
<< rpc::ActorTableData::ActorState_Name(actor_state);
return rpc::ExportActorData::DEAD;
}
}

/// 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_;
Expand Down
5 changes: 3 additions & 2 deletions src/ray/gcs/gcs_server/gcs_server_main.cc
Original file line number Diff line number Diff line change
Expand Up @@ -88,12 +88,13 @@ int main(int argc, char *argv[]) {

// Initialize event framework.
if (RayConfig::instance().event_log_reporter_enabled() && !log_dir.empty()) {
// This GCS server process emits GCS standard events, and Node export events
// This GCS server process emits GCS standard events, and Node and Actor export events
// so the various source types are passed to RayEventInit. The type of an
// event is determined by the schema of its event data.
const std::vector<ray::SourceTypeVariant> source_types = {
ray::rpc::Event_SourceType::Event_SourceType_GCS,
ray::rpc::ExportEvent_SourceType::ExportEvent_SourceType_EXPORT_NODE};
ray::rpc::ExportEvent_SourceType::ExportEvent_SourceType_EXPORT_NODE,
ray::rpc::ExportEvent_SourceType_EXPORT_ACTOR};
ray::RayEventInit(source_types,
absl::flat_hash_map<std::string, std::string>(),
log_dir,
Expand Down
Loading

0 comments on commit d3c0708

Please sign in to comment.