Skip to content
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

[core][state][dashboard] Use main threads's task id or actor creation task id for parent's task id in state API #32157

Merged
merged 5 commits into from
Feb 1, 2023

Conversation

rickyyx
Copy link
Contributor

@rickyyx rickyyx commented Feb 1, 2023

Signed-off-by: rickyyx rickyx@anyscale.com

Why are these changes needed?

Problems

Right now, if a new thread (or async actor's event loop executing thread) runs some ray code (e.g. submitting a task, calling runtime context), the thread will have a WorkerThreadContext that has a random task id.

This causes issues in state API since the task tree will have wrong structures, i.e. some tasks might have parent_task_id that doesn't match any existing tasks:

@ray.remote
class Actor:
    def actor_method(self):
        ... 
        # if below code runs in a separate thread (e.g. the actor manages its own thread pool) 
        # the task `some_ray_task` will have a parent task id that doesn't match any task. 
        ray.get(some_ray_task.remote())
        ...

Solutions

This PR adds a main_thread_or_actor_creation_task_id so that this will be used as the parent task id info for State API related features so that:

  1. Actors:
    • Tasks submitted in the anonymous thread will have the actor's creation task as the parent task.
  2. For normal tasks
    • Task submitted in the anonymous thread will have the main thread's task as the parent task.

Not fixed in this PR

This PR doesn't address the actual runtime context of tasks in anonymous threads: calling ray.get_runtime_context() is still not defined for async actors or threaded actors that start new threads.

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Signed-off-by: rickyyx <rickyx@anyscale.com>
@rickyyx rickyyx changed the title init [core][state][dashboard] Use main threads's parent task id for parent's task id in state API Feb 1, 2023
@rickyyx rickyyx marked this pull request as ready for review February 1, 2023 05:05
Copy link
Contributor

@ericl ericl left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, any reason to special case the concurrent actor case? If it's an implemention issue we can probably use an atomic reference or something like that for safety.

Copy link
Contributor

@ericl ericl left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe what we really want is an inheritable thread local variable.

@rickyyx
Copy link
Contributor Author

rickyyx commented Feb 1, 2023

Hmm, any reason to special case the concurrent actor case? If it's an implemention issue we can probably use an atomic reference or something like that for safety.

Guess it's special in what the main thread runs:

  • For async actors and concurrent threaded actors, the main thread runs the creation actor task, while actor methods could be run in other threads. So we will use the actor creation task as the parent task.
  • For non-concurrent actors, the main thread runs both the actor creation task and the actor tasks, so we could associate the parent task id to the actor task.

Or to make things consistent across all actors, we could probably always use the actor creation task's task id?

@rickyyx
Copy link
Contributor Author

rickyyx commented Feb 1, 2023

Maybe what we really want is an inheritable thread local variable.

For async actors, since multiple tasks could be run in the same thread concurrently, a thread-local variable might still not be able to solve the problem? Or maybe I am not understanding what you mean by "inheritable" correctly.

@ericl
Copy link
Contributor

ericl commented Feb 1, 2023

That's true, the management for an async actor would be more complex. I guess using the constructor task is probably good enough for now in these more unusual cases, and we can always revisit later...

@rickyyx rickyyx added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 1, 2023
Signed-off-by: rickyyx <rickyx@anyscale.com>
@rickyyx rickyyx removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 1, 2023
@rickyyx rickyyx changed the title [core][state][dashboard] Use main threads's parent task id for parent's task id in state API [core][state][dashboard] Use main threads's task id or actor creation task id for parent's task id in state API Feb 1, 2023
Copy link
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM for the short-term fix.

I think we need a lock?

Q: Is actor creation task semantic necessary? If not, let's just not do it. If so, can you modify the docstring of parent_task_id of the state API to explain this semantics?

src/ray/core_worker/context.h Outdated Show resolved Hide resolved
src/ray/core_worker/context.cc Outdated Show resolved Hide resolved
src/ray/core_worker/task_manager.cc Show resolved Hide resolved
@rickyyx
Copy link
Contributor Author

rickyyx commented Feb 1, 2023

Q: Is actor creation task semantic necessary? If not, let's just not do it. If so, can you modify the docstring of parent_task_id of the state API to explain this semantics?

Yeah, it's needed because we might have use cases where the actor manages a thread pool, and submits tasks from the thread pool while running actor task/actor creation task. Without proxying the actor creation task's id to those submitted tasks' parent task id, the parent task id of those tasks will be random.

Signed-off-by: rickyyx <rickyx@anyscale.com>
@rkooo567 rkooo567 added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 1, 2023
Signed-off-by: rickyyx <rickyx@anyscale.com>
@rickyyx rickyyx removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 1, 2023
@rickyyx
Copy link
Contributor Author

rickyyx commented Feb 1, 2023

Synced offline with Sang - so I guess I misunderstood Eric's original comment. We will revert to using the actor creation tasks for the tricky cases (concurrent actors), but use the main thread's task id for others.

Signed-off-by: rickyyx <rickyx@anyscale.com>
@rickyyx rickyyx requested a review from rkooo567 February 1, 2023 08:38
@rkooo567
Copy link
Contributor

rkooo567 commented Feb 1, 2023

Talked offline. The final behavior is;

  1. For normal single threaded task/actor, we will use the main thread's task id (correct hehavior).
  2. For unusual cases (threaded/async actors), we will use the actor creation task's task id. This means from the advanced visualization, all the remote tasks created from actor tasks will be under the constructor of threaded/async actors

Copy link
Contributor

@scv119 scv119 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

proto change lgtm

@rkooo567 rkooo567 merged commit 10c46dc into ray-project:master Feb 1, 2023
edoakes pushed a commit to edoakes/ray that referenced this pull request Mar 22, 2023
… task id for parent's task id in state API (ray-project#32157)

Right now, if a new thread (or async actor's event loop executing thread) runs some ray code (e.g. submitting a task, calling runtime context), the thread will have a WorkerThreadContext that has a random task id.

This causes issues in state API since the task tree will have wrong structures, i.e. some tasks might have parent_task_id that doesn't match any existing tasks:

For normal single threaded task/actor, we will use the main thread's task id (correct hehavior).
For unusual cases (threaded/async actors), we will use the actor creation task's task id. This means from the advanced visualization, all the remote tasks created from actor tasks will be under the constructor of threaded/async actors

Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants