-
Notifications
You must be signed in to change notification settings - Fork 5.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[tracing] Fix issue where actor/task is defined before ray.init
is called
#38323
Changes from 9 commits
ff1c719
5cd3d7e
00f1a02
2807217
26b9691
3bfc690
38c4177
5cdaa81
3f8cc73
0bea033
0db796a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,6 +19,7 @@ | |
cast, | ||
) | ||
|
||
import ray | ||
import ray._private.worker | ||
from ray._private.inspect_util import ( | ||
is_class_method, | ||
|
@@ -84,7 +85,6 @@ def _try_import(self, module): | |
) | ||
|
||
|
||
_nameable = Union[str, Callable[..., Any]] | ||
_global_is_tracing_enabled = False | ||
_opentelemetry = None | ||
|
||
|
@@ -192,14 +192,14 @@ def _use_context( | |
_opentelemetry.context.detach(token) | ||
|
||
|
||
def _function_hydrate_span_args(func: Callable[..., Any]): | ||
def _function_hydrate_span_args(function_name: str): | ||
"""Get the Attributes of the function that will be reported as attributes | ||
in the trace.""" | ||
runtime_context = get_runtime_context() | ||
|
||
span_args = { | ||
"ray.remote": "function", | ||
"ray.function": func, | ||
"ray.function": function_name, | ||
"ray.pid": str(os.getpid()), | ||
"ray.job_id": runtime_context.get_job_id(), | ||
"ray.node_id": runtime_context.get_node_id(), | ||
|
@@ -220,21 +220,18 @@ def _function_hydrate_span_args(func: Callable[..., Any]): | |
|
||
def _function_span_producer_name(func: Callable[..., Any]) -> str: | ||
"""Returns the function span name that has span kind of producer.""" | ||
args = _function_hydrate_span_args(func) | ||
name = args["ray.function"] | ||
|
||
return f"{name} ray.remote" | ||
return f"{func} ray.remote" | ||
|
||
|
||
def _function_span_consumer_name(func: Callable[..., Any]) -> str: | ||
"""Returns the function span name that has span kind of consumer.""" | ||
args = _function_hydrate_span_args(func) | ||
name = args["ray.function"] | ||
|
||
return f"{name} ray.remote_worker" | ||
return f"{func} ray.remote_worker" | ||
|
||
|
||
def _actor_hydrate_span_args(class_: _nameable, method: _nameable): | ||
def _actor_hydrate_span_args( | ||
class_: Union[str, Callable[..., Any]], | ||
method: Union[str, Callable[..., Any]], | ||
): | ||
"""Get the Attributes of the actor that will be reported as attributes | ||
in the trace.""" | ||
if callable(class_): | ||
|
@@ -243,7 +240,6 @@ def _actor_hydrate_span_args(class_: _nameable, method: _nameable): | |
method = method.__name__ | ||
|
||
runtime_context = get_runtime_context() | ||
|
||
span_args = { | ||
"ray.remote": "actor", | ||
"ray.actor_class": class_, | ||
|
@@ -268,22 +264,30 @@ def _actor_hydrate_span_args(class_: _nameable, method: _nameable): | |
return span_args | ||
|
||
|
||
def _actor_span_producer_name(class_: _nameable, method: _nameable) -> str: | ||
def _actor_span_producer_name( | ||
class_: Union[str, Callable[..., Any]], | ||
method: Union[str, Callable[..., Any]], | ||
) -> str: | ||
"""Returns the actor span name that has span kind of producer.""" | ||
args = _actor_hydrate_span_args(class_, method) | ||
assert args is not None | ||
name = args["ray.function"] | ||
if not isinstance(class_, str): | ||
class_ = class_.__name__ | ||
if not isinstance(method, str): | ||
method = method.__name__ | ||
|
||
return f"{name} ray.remote" | ||
return f"{class_}.{method} ray.remote" | ||
|
||
|
||
def _actor_span_consumer_name(class_: _nameable, method: _nameable) -> str: | ||
def _actor_span_consumer_name( | ||
class_: Union[str, Callable[..., Any]], | ||
method: Union[str, Callable[..., Any]], | ||
) -> str: | ||
"""Returns the actor span name that has span kind of consumer.""" | ||
args = _actor_hydrate_span_args(class_, method) | ||
assert args is not None | ||
name = args["ray.function"] | ||
if not isinstance(class_, str): | ||
class_ = class_.__name__ | ||
if not isinstance(method, str): | ||
method = method.__name__ | ||
|
||
return f"{name} ray.remote_worker" | ||
return f"{class_}.{method} ray.remote_worker" | ||
|
||
|
||
def _tracing_task_invocation(method): | ||
|
@@ -325,9 +329,6 @@ def _inject_tracing_into_function(function): | |
Use the provided trace context from kwargs. | ||
""" | ||
# Add _ray_trace_ctx to function signature | ||
if not _is_tracing_enabled(): | ||
return function | ||
Comment on lines
-328
to
-329
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is the main change: if Need to always inject this Note that this is already how it worked for actor methods. |
||
|
||
setattr( | ||
function, | ||
"__signature__", | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
auto-init needs to happen before the tracing decorator runs because it calls
get_runtime_context