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][experimental] Add experimental compiled DAG #41769

Merged
merged 81 commits into from Dec 14, 2023

Conversation

stephanie-wang
Copy link
Contributor

@stephanie-wang stephanie-wang commented Dec 9, 2023

Why are these changes needed?

This adds an experimental interface that allows you to compile a normal ray.dag to use a faster execution path. The execution path uses ray.experimental.channel to pass data directly between actors instead of going through the normal Ray control plane. See the REP for more information.

Several limitations will be addressed in a follow-up PR:

  1. dag.execute returns a Channel instead of an ObjectRef. Reading from a Channel requires the caller to release the value as soon as they are done with it, to avoid blocking the next DAG execution.
  2. Single-node only.
  3. Actors can only execute one DAG. Once they begin executing, they cannot execute other tasks and must be killed to stop executing. An actor can have at most one task in a DAG.

Initial results show about 35x speedup compared to normal Ray core on simple single-node DAGs:

single-actor DAG calls per second 593.42 +- 32.8
compiled single-actor DAG calls per second 20931.29 +- 675.78
scatter-gather DAG calls per second 248.87 +- 4.06
compiled scatter-gather DAG calls, n=4 actors per second 8896.79 +- 92.42
chain DAG calls, n=4 actors per second 156.48 +- 14.11
compiled chain DAG calls, n=4 actors per second 5738.39 +- 219.31

SangBin Cho and others added 30 commits November 17, 2023 08:37
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
x
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
@stephanie-wang stephanie-wang mentioned this pull request Dec 9, 2023
3 tasks
def __init__(self, actor: ClassNode, method_name: str, options: dict):
# TODO(sang): Theoretically, We should use weakref cuz it is
# a circular dependency but when I used weakref, it fails
# because we cannot serialize the weakref.
Copy link
Contributor

Choose a reason for hiding this comment

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

Explain the concern about circular dependencies here?

Copy link
Contributor

@edoakes edoakes Dec 11, 2023

Choose a reason for hiding this comment

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

you can add a simple wrapper for weakref to address the serialization issue if it's needed. I did the same in a patch to the pydantic repo: pydantic/pydantic#7780

python/ray/dag/compiled_dag_node.py Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/compiled_dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node.py Outdated Show resolved Hide resolved
python/ray/dag/dag_node.py Outdated Show resolved Hide resolved
@ericl ericl added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Dec 10, 2023
@ericl ericl self-assigned this Dec 10, 2023
logger = logging.getLogger(__name__)


@ray.remote(concurrency_groups={"_ray_system": 1})
Copy link
Contributor

Choose a reason for hiding this comment

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

What is the _ray_system concurrency group and why's it required here?

Copy link
Contributor

Choose a reason for hiding this comment

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

We shouldn't need to declare this group, it's automatically created per #41605

Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
x
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
x
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
x
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
x
Signed-off-by: Stephanie Wang <swang@cs.berkeley.edu>
@can-anyscale can-anyscale merged commit 6eb4f44 into ray-project:master Dec 14, 2023
12 of 14 checks passed
@stephanie-wang stephanie-wang deleted the compiled-dag branch December 15, 2023 02:39
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
@author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

5 participants