Skip to content

[core][compiled graphs] Fix execution schedules with collective operations#53007

Merged
stephanie-wang merged 17 commits intoray-project:masterfrom
dengwxn:coll-sched-pr
May 20, 2025
Merged

[core][compiled graphs] Fix execution schedules with collective operations#53007
stephanie-wang merged 17 commits intoray-project:masterfrom
dengwxn:coll-sched-pr

Conversation

@dengwxn
Copy link
Copy Markdown

@dengwxn dengwxn commented May 14, 2025

Why are these changes needed?

Given an input DAG of SPMD training strategies such as DDP, after DAG compile, the first actor will generate different execution schedules than others. This is due to the current scheduling policy, when there are multiple ready operation nodes such as actor1.compute (non-NCCL) and actor4.collective (NCCL, for actor1-4, there's only one collective operation node that's eventually ready), the policy does not know actor1 has both the non-NCCL actor1.compute and the NCCL actor4.collective. This leads to actor1 scheduling the actor1.compute first, and actor1-4 scheduling the collective next.

We update the policy to push all the collective operations nodes into candidates when the last of them is ready. In the previous example, actor1 will have both actor1.compute and actor1.collective as candidates. In a DAG of SPMD strategies, all the actors pop either the compute or the collective together.

We also update the policy to simply prioritize the NCCL operation node over the non-NCCL. This will lead to NCCL operations to be scheduled as soon as possible. It is safe to do so under the current settings of CUDA streams in the system, because each NCCL read/write/collective stream only allows one outstanding NCCL kernel at a time.

We add a test test_collective_dag.py::test_exec_schedules_ddp to verify the generated schedules are identical across workers for the DDP stragegy. Other tests are updated to reflect the changes of prioritizing the NCCL operation node over the non-NCCL.

Related issue number

This PR is part of #48649 planning to be merged incrementally.

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 added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • 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 :(

Weixin Deng added 14 commits May 12, 2025 23:25
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
@dengwxn
Copy link
Copy Markdown
Author

dengwxn commented May 14, 2025

cc @stephanie-wang

@hainesmichaelc hainesmichaelc added the community-contribution Contributed by the community label May 15, 2025
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
@mascharkh mascharkh added the core Issues that should be addressed in Ray Core label May 15, 2025
Weixin Deng added 2 commits May 15, 2025 15:53
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Copy link
Copy Markdown
Contributor

@stephanie-wang stephanie-wang left a comment

Choose a reason for hiding this comment

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

Nice fix!

)
expected_schedule = actor_to_execution_schedule[0]
for schedule in actor_to_execution_schedule[1:]:
assert schedule == expected_schedule
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Nice test!

@stephanie-wang stephanie-wang self-assigned this May 16, 2025
@stephanie-wang stephanie-wang added the go add ONLY when ready to merge, run all tests label May 16, 2025
@stephanie-wang stephanie-wang merged commit 5f154c6 into ray-project:master May 20, 2025
6 checks passed
stephanie-wang pushed a commit that referenced this pull request May 29, 2025
…3111)

This PR unifies the scheduling implementation for the NCCL P2P and
collective operation nodes. The logic remains the same: (1) P2P case:
When a NCCL send node is selected, its downstream NCCL recv nodes are
also selected; (2) Collective case: When a NCCL collective node is
selected, its corresponding NCCL collective nodes are also selected.
Previously, the NCCL P2P case was implemented by selecting the recv
nodes if a send node is detected, and the NCCL collective case was
implemented by maintaining a set of pending collective nodes.

We unify the implementation for both cases. Concretely, they both
maintain a set of (pending) synchronous nodes named `sync_idxs` and
`pending_sync_idxs`. The synchronous nodes denote the P2P send/recv
nodes or the collective nodes. The NCCL P2P/collective operation is
ready when `sync_idxs == pending_sync_idxs`.

Test cases are updated to reflect the use of synchronous nodes for both
NCCL P2P and collective nodes.

This PR is a follow-up of #53007. They are parts of #48649 planning to
be merged incrementally.
---------

Signed-off-by: Weixin Deng <weixin@cs.washington.edu>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-backlog community-contribution Contributed by the community core Issues that should be addressed in Ray Core go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants