From 430ea70b202cf3e691e942f49bc1449de82e3b11 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sun, 20 Apr 2025 17:40:49 +0800 Subject: [PATCH 1/3] [core] add more context on logs Signed-off-by: fscnick --- src/ray/core_worker/actor_manager.cc | 17 +++++++++-------- src/ray/core_worker/common.cc | 4 ++-- src/ray/core_worker/context.cc | 2 +- 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/ray/core_worker/actor_manager.cc b/src/ray/core_worker/actor_manager.cc index b0a276606a29..6159c9ff0b5f 100644 --- a/src/ray/core_worker/actor_manager.cc +++ b/src/ray/core_worker/actor_manager.cc @@ -205,7 +205,7 @@ void ActorManager::WaitForActorRefDeleted( const auto actor_creation_return_id = ObjectID::ForActorHandle(actor_id); if (!reference_counter_.SetObjectRefDeletedCallback(actor_creation_return_id, callback)) { - RAY_LOG(DEBUG) << "ActorID reference already gone for " << actor_id; + RAY_LOG(DEBUG).WithField(actor_id) << "ActorID reference already gone"; callback(actor_creation_return_id); } } @@ -213,12 +213,12 @@ void ActorManager::WaitForActorRefDeleted( void ActorManager::HandleActorStateNotification(const ActorID &actor_id, const rpc::ActorTableData &actor_data) { const auto &actor_state = rpc::ActorTableData::ActorState_Name(actor_data.state()); - RAY_LOG(INFO) << "received notification on actor, state: " << actor_state - << ", actor_id: " << actor_id + const auto worker_id = WorkerID::FromBinary(actor_data.address().worker_id()); + const auto raylet_id = NodeID::FromBinary(actor_data.address().raylet_id()); + RAY_LOG(INFO).WithField(actor_id).WithField(worker_id).WithField(raylet_id) + << "received notification on actor, state: " << actor_state << ", ip address: " << actor_data.address().ip_address() - << ", port: " << actor_data.address().port() << ", worker_id: " - << WorkerID::FromBinary(actor_data.address().worker_id()) - << ", raylet_id: " << NodeID::FromBinary(actor_data.address().raylet_id()) + << ", port: " << actor_data.address().port() << ", num_restarts: " << actor_data.num_restarts() << ", death context type=" << gcs::GetActorDeathCauseString(actor_data.death_cause()); @@ -333,8 +333,9 @@ void ActorManager::MarkActorKilledOrOutOfScope( /// Invalidate named actor cache. if (!actor_name.empty()) { - RAY_LOG(DEBUG) << "Actor name cache is invalidated for the actor of name " - << actor_name << " namespace " << ray_namespace << " id " << actor_id; + RAY_LOG(DEBUG).WithField(actor_id) + << "Actor name cache is invalidated for the actor of name " + << actor_name << " namespace " << ray_namespace; cached_actor_name_to_ids_.erase(GenerateCachedActorName(ray_namespace, actor_name)); } } diff --git a/src/ray/core_worker/common.cc b/src/ray/core_worker/common.cc index 449fbc8a23ad..910ff5efa27f 100644 --- a/src/ray/core_worker/common.cc +++ b/src/ray/core_worker/common.cc @@ -62,8 +62,8 @@ void SerializeReturnObject(const ObjectID &object_id, if (!return_object) { // This should only happen if the local raylet died. Caller should // retry the task. - RAY_LOG(WARNING) << "Failed to create task return object " << object_id - << " in the object store, exiting."; + RAY_LOG(WARNING).WithField(object_id) + << "Failed to create task return object in the object store, exiting."; QuickExit(); } return_object_proto->set_size(return_object->GetSize()); diff --git a/src/ray/core_worker/context.cc b/src/ray/core_worker/context.cc index 68f8e26eb95e..682f16d63df4 100644 --- a/src/ray/core_worker/context.cc +++ b/src/ray/core_worker/context.cc @@ -450,7 +450,7 @@ ObjectID WorkerContext::GetGeneratorReturnId(const TaskID &task_id, // return values. auto max_generator_returns = GetThreadContext().GetMaxNumGeneratorReturnIndex(); if (put_index > max_generator_returns) { - RAY_LOG(FATAL) + RAY_LOG(FATAL).WithField("task_id", current_task_id) << "The generator returns " << current_put_index << " items, which exceed the maximum number of return values allowed, " << max_generator_returns; From 0b4ae9a3e055c246927853430873298de5482ff4 Mon Sep 17 00:00:00 2001 From: fscnick Date: Sun, 20 Apr 2025 17:41:56 +0800 Subject: [PATCH 2/3] [chore][core] Reformatted changed files Signed-off-by: fscnick --- src/ray/core_worker/actor_manager.cc | 15 +++++++-------- src/ray/core_worker/common.cc | 2 +- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/ray/core_worker/actor_manager.cc b/src/ray/core_worker/actor_manager.cc index 6159c9ff0b5f..cc08ee292918 100644 --- a/src/ray/core_worker/actor_manager.cc +++ b/src/ray/core_worker/actor_manager.cc @@ -216,12 +216,11 @@ void ActorManager::HandleActorStateNotification(const ActorID &actor_id, const auto worker_id = WorkerID::FromBinary(actor_data.address().worker_id()); const auto raylet_id = NodeID::FromBinary(actor_data.address().raylet_id()); RAY_LOG(INFO).WithField(actor_id).WithField(worker_id).WithField(raylet_id) - << "received notification on actor, state: " << actor_state - << ", ip address: " << actor_data.address().ip_address() - << ", port: " << actor_data.address().port() - << ", num_restarts: " << actor_data.num_restarts() - << ", death context type=" - << gcs::GetActorDeathCauseString(actor_data.death_cause()); + << "received notification on actor, state: " << actor_state + << ", ip address: " << actor_data.address().ip_address() + << ", port: " << actor_data.address().port() + << ", num_restarts: " << actor_data.num_restarts() << ", death context type=" + << gcs::GetActorDeathCauseString(actor_data.death_cause()); if (actor_data.preempted()) { actor_task_submitter_.SetPreempted(actor_id); } @@ -334,8 +333,8 @@ void ActorManager::MarkActorKilledOrOutOfScope( /// Invalidate named actor cache. if (!actor_name.empty()) { RAY_LOG(DEBUG).WithField(actor_id) - << "Actor name cache is invalidated for the actor of name " - << actor_name << " namespace " << ray_namespace; + << "Actor name cache is invalidated for the actor of name " << actor_name + << " namespace " << ray_namespace; cached_actor_name_to_ids_.erase(GenerateCachedActorName(ray_namespace, actor_name)); } } diff --git a/src/ray/core_worker/common.cc b/src/ray/core_worker/common.cc index 910ff5efa27f..e82ed8d8fbed 100644 --- a/src/ray/core_worker/common.cc +++ b/src/ray/core_worker/common.cc @@ -63,7 +63,7 @@ void SerializeReturnObject(const ObjectID &object_id, // This should only happen if the local raylet died. Caller should // retry the task. RAY_LOG(WARNING).WithField(object_id) - << "Failed to create task return object in the object store, exiting."; + << "Failed to create task return object in the object store, exiting."; QuickExit(); } return_object_proto->set_size(return_object->GetSize()); From 335c9046880c74853a3a6510b925d453ac55cd97 Mon Sep 17 00:00:00 2001 From: fscnick Date: Fri, 25 Apr 2025 00:26:02 +0800 Subject: [PATCH 3/3] [core] remove key task_id from WithField Signed-off-by: fscnick --- src/ray/core_worker/context.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/core_worker/context.cc b/src/ray/core_worker/context.cc index 682f16d63df4..0163d6a59187 100644 --- a/src/ray/core_worker/context.cc +++ b/src/ray/core_worker/context.cc @@ -450,7 +450,7 @@ ObjectID WorkerContext::GetGeneratorReturnId(const TaskID &task_id, // return values. auto max_generator_returns = GetThreadContext().GetMaxNumGeneratorReturnIndex(); if (put_index > max_generator_returns) { - RAY_LOG(FATAL).WithField("task_id", current_task_id) + RAY_LOG(FATAL).WithField(current_task_id) << "The generator returns " << current_put_index << " items, which exceed the maximum number of return values allowed, " << max_generator_returns;