Skip to content

Commit

Permalink
[core] Attach call site to ObjectRefs, print on error (#17971)
Browse files Browse the repository at this point in the history
* Attach call site to ObjectRef

* flag

* Fix build

* build

* build

* build

* x

* x

* skip on windows

* lint
  • Loading branch information
stephanie-wang authored Sep 1, 2021
1 parent d470e67 commit d43d297
Show file tree
Hide file tree
Showing 33 changed files with 366 additions and 239 deletions.
3 changes: 2 additions & 1 deletion cpp/src/ray/runtime/abstract_ray_runtime.cc
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,8 @@ std::vector<std::unique_ptr<::ray::TaskArg>> TransformArgs(
} else {
RAY_CHECK(arg.id);
ray_arg = absl::make_unique<ray::TaskArgByReference>(ObjectID::FromBinary(*arg.id),
ray::rpc::Address{});
ray::rpc::Address{},
/*call_site=*/"");
}
ray_args.push_back(std::move(ray_arg));
}
Expand Down
16 changes: 10 additions & 6 deletions cpp/src/ray/runtime/task/native_task_submitter.cc
Original file line number Diff line number Diff line change
Expand Up @@ -36,14 +36,18 @@ ObjectID NativeTaskSubmitter::Submit(InvocationSpec &invocation,
TaskOptions options{};
options.name = call_options.name;
options.resources = call_options.resources;
std::vector<ObjectID> return_ids;
std::vector<rpc::ObjectReference> return_refs;
if (invocation.task_type == TaskType::ACTOR_TASK) {
core_worker.SubmitActorTask(invocation.actor_id, BuildRayFunction(invocation),
invocation.args, options, &return_ids);
return_refs = core_worker.SubmitActorTask(
invocation.actor_id, BuildRayFunction(invocation), invocation.args, options);
} else {
core_worker.SubmitTask(BuildRayFunction(invocation), invocation.args, options,
&return_ids, 1, false,
std::make_pair(PlacementGroupID::Nil(), -1), true, "");
return_refs = core_worker.SubmitTask(
BuildRayFunction(invocation), invocation.args, options, 1, false,
std::make_pair(PlacementGroupID::Nil(), -1), true, "");
}
std::vector<ObjectID> return_ids;
for (const auto &ref : return_refs) {
return_ids.push_back(ObjectID::FromBinary(ref.object_id()));
}
return return_ids[0];
}
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/ray/runtime/task/task_executor.cc
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ Status TaskExecutor::ExecuteTask(
ray::TaskType task_type, const std::string task_name, const RayFunction &ray_function,
const std::unordered_map<std::string, double> &required_resources,
const std::vector<std::shared_ptr<ray::RayObject>> &args_buffer,
const std::vector<ObjectID> &arg_reference_ids,
const std::vector<rpc::ObjectReference> &arg_refs,
const std::vector<ObjectID> &return_ids, const std::string &debugger_breakpoint,
std::vector<std::shared_ptr<ray::RayObject>> *results,
std::shared_ptr<ray::LocalMemoryBuffer> &creation_task_exception_pb_bytes,
Expand Down
2 changes: 1 addition & 1 deletion cpp/src/ray/runtime/task/task_executor.h
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ class TaskExecutor {
const RayFunction &ray_function,
const std::unordered_map<std::string, double> &required_resources,
const std::vector<std::shared_ptr<ray::RayObject>> &args,
const std::vector<ObjectID> &arg_reference_ids,
const std::vector<rpc::ObjectReference> &arg_refs,
const std::vector<ObjectID> &return_ids, const std::string &debugger_breakpoint,
std::vector<std::shared_ptr<ray::RayObject>> *results,
std::shared_ptr<ray::LocalMemoryBuffer> &creation_task_exception_pb_bytes,
Expand Down
1 change: 1 addition & 0 deletions python/ray/_raylet.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@ cdef class ObjectRef(BaseID):
# of active IDs in the core worker so we know whether we should clean
# it up.
c_bool in_core_worker
c_string call_site_data

cdef CObjectID native(self)

Expand Down
61 changes: 37 additions & 24 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ from cython.operator import dereference, postincrement
from ray.includes.common cimport (
CBuffer,
CAddress,
CObjectReference,
CLanguage,
CObjectReference,
CRayObject,
Expand Down Expand Up @@ -182,10 +183,11 @@ cdef RayObjectsToDataMetadataPairs(
return data_metadata_pairs


cdef VectorToObjectRefs(const c_vector[CObjectID] &object_refs):
cdef VectorToObjectRefs(const c_vector[CObjectReference] &object_refs):
result = []
for i in range(object_refs.size()):
result.append(ObjectRef(object_refs[i].Binary()))
result.append(ObjectRef(object_refs[i].object_id(),
object_refs[i].call_site()))
return result


Expand Down Expand Up @@ -328,6 +330,7 @@ cdef prepare_args(
int64_t total_inlined
shared_ptr[CBuffer] arg_data
c_vector[CObjectID] inlined_ids
c_string put_arg_call_site
c_vector[CObjectReference] inlined_refs

worker = ray.worker.global_worker
Expand All @@ -341,7 +344,8 @@ cdef prepare_args(
unique_ptr[CTaskArg](new CTaskArgByReference(
c_arg,
CCoreWorkerProcess.GetCoreWorker().GetOwnerAddress(
c_arg))))
c_arg),
arg.call_site())))

else:
serialized_arg = worker.get_serialization_context().serialize(arg)
Expand All @@ -356,6 +360,8 @@ cdef prepare_args(
metadata_fields[0], language))
size = serialized_arg.total_bytes

if RayConfig.instance().record_ref_creation_sites():
get_py_stack(&put_arg_call_site)
# TODO(edoakes): any objects containing ObjectRefs are spilled to
# plasma here. This is inefficient for small objects, but inlined
# arguments aren't associated ObjectRefs right now so this is a
Expand Down Expand Up @@ -383,7 +389,8 @@ cdef prepare_args(
new CTaskArgByReference(CObjectID.FromBinary(
core_worker.put_serialized_object(
serialized_arg, inline_small_object=False)),
CCoreWorkerProcess.GetCoreWorker().GetRpcAddress())))
CCoreWorkerProcess.GetCoreWorker().GetRpcAddress(),
put_arg_call_site)))


cdef raise_if_dependency_failed(arg):
Expand All @@ -403,7 +410,7 @@ cdef execute_task(
const CRayFunction &ray_function,
const unordered_map[c_string, double] &c_resources,
const c_vector[shared_ptr[CRayObject]] &c_args,
const c_vector[CObjectID] &c_arg_reference_ids,
const c_vector[CObjectReference] &c_arg_refs,
const c_vector[CObjectID] &c_return_ids,
const c_string debugger_breakpoint,
c_vector[shared_ptr[CRayObject]] *returns,
Expand Down Expand Up @@ -526,7 +533,7 @@ cdef execute_task(
args, kwargs = [], {}
else:
metadata_pairs = RayObjectsToDataMetadataPairs(c_args)
object_refs = VectorToObjectRefs(c_arg_reference_ids)
object_refs = VectorToObjectRefs(c_arg_refs)

if core_worker.current_actor_is_asyncio():
# We deserialize objects in event loop thread to
Expand Down Expand Up @@ -658,7 +665,7 @@ cdef CRayStatus task_execution_handler(
const CRayFunction &ray_function,
const unordered_map[c_string, double] &c_resources,
const c_vector[shared_ptr[CRayObject]] &c_args,
const c_vector[CObjectID] &c_arg_reference_ids,
const c_vector[CObjectReference] &c_arg_refs,
const c_vector[CObjectID] &c_return_ids,
const c_string debugger_breakpoint,
c_vector[shared_ptr[CRayObject]] *returns,
Expand All @@ -670,7 +677,7 @@ cdef CRayStatus task_execution_handler(
# The call to execute_task should never raise an exception. If
# it does, that indicates that there was an internal error.
execute_task(task_type, task_name, ray_function, c_resources,
c_args, c_arg_reference_ids, c_return_ids,
c_args, c_arg_refs, c_return_ids,
debugger_breakpoint, returns,
is_application_level_error)
except Exception as e:
Expand Down Expand Up @@ -747,11 +754,17 @@ cdef void run_on_util_worker_handler(


cdef c_vector[c_string] spill_objects_handler(
const c_vector[CObjectID]& object_ids_to_spill,
const c_vector[c_string]& owner_addresses) nogil:
cdef c_vector[c_string] return_urls
const c_vector[CObjectReference]& object_refs_to_spill) nogil:
cdef:
c_vector[c_string] return_urls
c_vector[c_string] owner_addresses

with gil:
object_refs = VectorToObjectRefs(object_ids_to_spill)
object_refs = VectorToObjectRefs(object_refs_to_spill)
for i in range(object_refs_to_spill.size()):
owner_addresses.push_back(
object_refs_to_spill[i].owner_address()
.SerializeAsString())
try:
with ray.worker._changeproctitle(
ray_constants.WORKER_PROCESS_TYPE_SPILL_WORKER,
Expand All @@ -774,7 +787,7 @@ cdef c_vector[c_string] spill_objects_handler(


cdef int64_t restore_spilled_objects_handler(
const c_vector[CObjectID]& object_ids_to_restore,
const c_vector[CObjectReference]& object_refs_to_restore,
const c_vector[c_string]& object_urls) nogil:
cdef:
int64_t bytes_restored = 0
Expand All @@ -783,7 +796,7 @@ cdef int64_t restore_spilled_objects_handler(
size = object_urls.size()
for i in range(size):
urls.append(object_urls[i])
object_refs = VectorToObjectRefs(object_ids_to_restore)
object_refs = VectorToObjectRefs(object_refs_to_restore)
try:
with ray.worker._changeproctitle(
ray_constants.WORKER_PROCESS_TYPE_RESTORE_WORKER,
Expand Down Expand Up @@ -897,7 +910,8 @@ cdef void get_py_stack(c_string* stack_out) nogil:
frame.f_code.co_filename, frame.f_code.co_name,
frame.f_lineno))
frame = frame.f_back
stack_out[0] = " | ".join(msg_frames).encode("ascii")
stack_out[0] = (ray_constants.CALL_STACK_LINE_DELIMITER
.join(msg_frames).encode("ascii"))

cdef shared_ptr[CBuffer] string_to_buffer(c_string& c_str):
cdef shared_ptr[CBuffer] empty_metadata
Expand Down Expand Up @@ -1338,13 +1352,13 @@ cdef class CoreWorker:
unordered_map[c_string, double] c_resources
CRayFunction ray_function
c_vector[unique_ptr[CTaskArg]] args_vector
c_vector[CObjectID] return_ids
CPlacementGroupID c_placement_group_id = \
placement_group_id.native()
c_string c_serialized_runtime_env
unordered_map[c_string, c_string] \
c_override_environment_variables = \
override_environment_variables
c_vector[CObjectReference] return_refs

with self.profile_event(b"submit_task"):
c_serialized_runtime_env = \
Expand All @@ -1357,19 +1371,19 @@ cdef class CoreWorker:
# NOTE(edoakes): releasing the GIL while calling this method causes
# segfaults. See relevant issue for details:
# https://github.com/ray-project/ray/pull/12803
CCoreWorkerProcess.GetCoreWorker().SubmitTask(
return_refs = CCoreWorkerProcess.GetCoreWorker().SubmitTask(
ray_function, args_vector, CTaskOptions(
name, num_returns, c_resources,
b"",
c_serialized_runtime_env,
c_override_environment_variables),
&return_ids, max_retries, retry_exceptions,
max_retries, retry_exceptions,
c_pair[CPlacementGroupID, int64_t](
c_placement_group_id, placement_group_bundle_index),
placement_group_capture_child_tasks,
debugger_breakpoint)

return VectorToObjectRefs(return_ids)
return VectorToObjectRefs(return_refs)

def create_actor(self,
Language language,
Expand Down Expand Up @@ -1509,7 +1523,7 @@ cdef class CoreWorker:
unordered_map[c_string, double] c_resources
CRayFunction ray_function
c_vector[unique_ptr[CTaskArg]] args_vector
c_vector[CObjectID] return_ids
c_vector[CObjectReference] return_refs

with self.profile_event(b"submit_task"):
if num_method_cpus > 0:
Expand All @@ -1521,13 +1535,12 @@ cdef class CoreWorker:
# NOTE(edoakes): releasing the GIL while calling this method causes
# segfaults. See relevant issue for details:
# https://github.com/ray-project/ray/pull/12803
CCoreWorkerProcess.GetCoreWorker().SubmitActorTask(
return_refs = CCoreWorkerProcess.GetCoreWorker().SubmitActorTask(
c_actor_id,
ray_function,
args_vector, CTaskOptions(name, num_returns, c_resources),
&return_ids)
args_vector, CTaskOptions(name, num_returns, c_resources))

return VectorToObjectRefs(return_ids)
return VectorToObjectRefs(return_refs)

def kill_actor(self, ActorID actor_id, c_bool no_restart):
cdef:
Expand Down
17 changes: 15 additions & 2 deletions python/ray/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@

import ray.cloudpickle as pickle
from ray.core.generated.common_pb2 import RayException, Language, PYTHON
import ray.ray_constants as ray_constants
import colorama
import setproctitle

Expand Down Expand Up @@ -285,11 +286,23 @@ class ObjectLostError(RayError):
object_ref_hex: Hex ID of the object.
"""

def __init__(self, object_ref_hex):
def __init__(self, object_ref_hex, call_site):
self.object_ref_hex = object_ref_hex
self.call_site = call_site.replace(
ray_constants.CALL_STACK_LINE_DELIMITER, "\n ")

def __str__(self):
return (f"Object {self.object_ref_hex} is lost due to node failure.")
msg = (f"Object {self.object_ref_hex} cannot be retrieved due to node "
"failure or system error.")
if self.call_site:
msg += (f" The ObjectRef was created at: {self.call_site}")
else:
msg += (
" To see information about where this ObjectRef was created "
"in Python, set the environment variable "
"RAY_record_ref_creation_sites=1 during `ray start` and "
"`ray.init()`.")
return msg


class GetTimeoutError(RayError):
Expand Down
8 changes: 6 additions & 2 deletions python/ray/includes/common.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -152,12 +152,15 @@ cdef extern from "src/ray/protobuf/common.pb.h" nogil:
pass
cdef cppclass CAddress "ray::rpc::Address":
CAddress()
const c_string &SerializeAsString()
const c_string &SerializeAsString() const
void ParseFromString(const c_string &serialized)
void CopyFrom(const CAddress& address)
const c_string &worker_id()
cdef cppclass CObjectReference "ray::rpc::ObjectReference":
CObjectReference()
CAddress owner_address() const
const c_string &object_id() const
const c_string &call_site() const

# This is a workaround for C++ enum class since Cython has no corresponding
# representation.
Expand Down Expand Up @@ -243,7 +246,8 @@ cdef extern from "ray/core_worker/common.h" nogil:

cdef cppclass CTaskArgByReference "ray::TaskArgByReference":
CTaskArgByReference(const CObjectID &object_id,
const CAddress &owner_address)
const CAddress &owner_address,
const c_string &call_site)

cdef cppclass CTaskArgByValue "ray::TaskArgByValue":
CTaskArgByValue(const shared_ptr[CRayObject] &data)
Expand Down
17 changes: 8 additions & 9 deletions python/ray/includes/libcoreworker.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ from ray.includes.gcs_client cimport CGcsClient

from ray.includes.common cimport (
CAddress,
CObjectReference,
CActorCreationOptions,
CBuffer,
CPlacementGroupCreationOptions,
Expand Down Expand Up @@ -99,10 +100,10 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
CWorkerType GetWorkerType()
CLanguage GetLanguage()

void SubmitTask(
c_vector[CObjectReference] SubmitTask(
const CRayFunction &function,
const c_vector[unique_ptr[CTaskArg]] &args,
const CTaskOptions &options, c_vector[CObjectID] *return_ids,
const CTaskOptions &options,
int max_retries,
c_bool retry_exceptions,
c_pair[CPlacementGroupID, int64_t] placement_options,
Expand All @@ -120,11 +121,10 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
const CPlacementGroupID &placement_group_id)
CRayStatus WaitPlacementGroupReady(
const CPlacementGroupID &placement_group_id, int timeout_ms)
void SubmitActorTask(
c_vector[CObjectReference] SubmitActorTask(
const CActorID &actor_id, const CRayFunction &function,
const c_vector[unique_ptr[CTaskArg]] &args,
const CTaskOptions &options,
c_vector[CObjectID] *return_ids)
const CTaskOptions &options)
CRayStatus KillActor(
const CActorID &actor_id, c_bool force_kill,
c_bool no_restart)
Expand Down Expand Up @@ -276,7 +276,7 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
const CRayFunction &ray_function,
const unordered_map[c_string, double] &resources,
const c_vector[shared_ptr[CRayObject]] &args,
const c_vector[CObjectID] &arg_reference_ids,
const c_vector[CObjectReference] &arg_refs,
const c_vector[CObjectID] &return_ids,
const c_string debugger_breakpoint,
c_vector[shared_ptr[CRayObject]] *returns,
Expand All @@ -288,10 +288,9 @@ cdef extern from "ray/core_worker/core_worker.h" nogil:
(CRayStatus() nogil) check_signals
(void() nogil) gc_collect
(c_vector[c_string](
const c_vector[CObjectID] &,
const c_vector[c_string] &) nogil) spill_objects
const c_vector[CObjectReference] &) nogil) spill_objects
(int64_t(
const c_vector[CObjectID] &,
const c_vector[CObjectReference] &,
const c_vector[c_string] &) nogil) restore_spilled_objects
(void(
const c_vector[c_string]&,
Expand Down
Loading

0 comments on commit d43d297

Please sign in to comment.