Open avnishn opened 1 year ago
>>> ray._private.state.state._available_resources_per_node()
{'bee925af8b9709467925658d4bae618fbcda9c54690facf0a7d9bdd7': {'memory': 36752714958.0, 'node:10.0.60.35': 1.0, 'object_store_memory': 18376357478.0, 'CPU': 16.0}, '436333409ff6a802f346a84ade5f27973fd59612725cc9a68f917c80': {'memory': 10700826215.0, 'node:10.0.35.146': 1.0, 'CPU': 4.0, 'object_store_memory': 4586068377.0, 'accelerator_type:T4': 1.0, 'GPU': 1.0}, 'be7ba149913b7b994eeca34d043a14ea383ad26c8a900d25d1eb1900': {'CPU': 1.0, 'node:10.0.39.211': 1.0, 'bundle_group_66ea2b543e78f3f76a0e4d20173f0c000000': 3000.0, 'accelerator_type:T4': 1.0, 'bundle_group_2_66ea2b543e78f3f76a0e4d20173f0c000000': 1000.0, 'CPU_group_2_66ea2b543e78f3f76a0e4d20173f0c000000': 1.0, 'memory': 10678697165.0, 'CPU_group_1_66ea2b543e78f3f76a0e4d20173f0c000000': 1.0, 'bundle_group_0_66ea2b543e78f3f76a0e4d20173f0c000000': 1000.0, 'bundle_group_1_66ea2b543e78f3f76a0e4d20173f0c000000': 1000.0, 'GPU_group_2_66ea2b543e78f3f76a0e4d20173f0c000000': 1.0, 'object_store_memory': 4576584499.0, 'CPU_group_66ea2b543e78f3f76a0e4d20173f0c000000': 1.0}, 'b46155e307ea475886e9edbd708426f66d29b1986a3f28c01a662952': {'bundle_group_66ea2b543e78f3f76a0e4d20173f0c000000': 1000.0, 'accelerator_type:T4': 1.0, 'memory': 10700740199.0, 'bundle_group_3_66ea2b543e78f3f76a0e4d20173f0c000000': 1000.0, 'CPU': 3.0, 'GPU_group_66ea2b543e78f3f76a0e4d20173f0c000000': 1.0, 'CPU_group_3_66ea2b543e78f3f76a0e4d20173f0c000000': 1.0, 'node:10.0.52.205': 1.0, 'GPU_group_3_66ea2b543e78f3f76a0e4d20173f0c000000': 1.0, 'object_store_memory': 4586031513.0}, '8625f254c392267afc08438899f7481e2914acf6aea670c765076c04': {'memory': 10700705792.0, 'object_store_memory': 4586016768.0, 'CPU': 4.0, 'node:10.0.14.120': 1.0, 'accelerator_type:T4': 1.0, 'GPU': 1.0}}
I think the hang is due to the PG fragmentation:
The pg has 4 bundles [{"CPU": 1}, {"CPU": 1}, {"CPU": 1, "GPU": 1}, {"CPU": 1, "GPU": 1}]
. Bundles 0, 1, 2 are on node A and bundle 3 are on node B.
We already scheduled two {"CPU": 1}
task/actor and one {"CPU": 1, "GPU": 1}
task/actor in this PG. After that the PG has 1 free CPU on bundle 1or 2 and 1 free GPU on bundle 3 but they don't belong to the same bundle. As a result, {"CPU":1, "GPU": 1}
request cannot be satisified. The reason why is that when we schedule the first two {"CPU": 1}
task/actor, we didn't specify the bundle_index and it happened to use the CPU from bundle 3 due to bad luck.
The short-term fix now should be on the Tune side to always specify bundle_index during scheduling to avoid fragmentation.
In the long term, core can probably do a better job to reduce fragmentation automatically.
Also as suggested by @cadedaniel: core should provide some message on why the scheduling is pending, that way people are not left assuming it’s a bug in ray core
The reason why is that when we schedule the first two {"CPU": 1} task/actor, we didn't specify the bundle_index and it happened to use the CPU from bundle 3 due to bad luck.
@jjyao Why does that happen? The two task/actors that specify one CPU requirement and no-gpu requirements should not be assigned to the bundle that has GPU requirements. Isn't that the case?
Please tag me and @avnishn in the follow up conversations.
@kouroshHakha, for this particular case, yes, core can be smarter to only use the cpu-only bundle to avoid fragmentation. But in general, core doesn't have the complete view to fully solve the fragmentation issue since it doesn't know what requests will come in later and what resources they need.
Quote from https://docs.ray.io/en/latest/ray-core/scheduling/placement-group.html
If you don’t specify a bundle, the actor (or task) is scheduled on a random bundle that has unallocated reserved resources.
Currently, if you don't specify bundle_index, you cannot expect core will use a certain bundle.
Since this is a release-blocker issue, please close it only after the cherry pick fix is merged into 2.5 release branch.
Please add @ArturNiederfahrenhorst as one of the reviewer of the fix as well for tracking purpose. Thankks!
https://github.com/ray-project/ray/pull/35679
also need to merge this
@avnishn Merged on release, branch.
@avnishn: is this already done? thanks
What happened + What you expected to happen
Above is a script to reproduce the problem. I am running on a the following cluster: https://console.anyscale.com/o/anyscale-internal/workspaces/expwrk_rexsdhckwvn3wltbtxwce57a77/ses_qstkpd5ej9qjmle94esjcl6nyr
which has the following cluster compute layout
I'm trying to run a script that creates a placement group that looks like the following:
[{"CPU:1, "GPU: 0"}, {"CPU:1, "GPU: 0"}, {"CPU:1, "GPU: 1"}, {"CPU:1, "GPU: 1"}]
and when I run this one of my gpu actors is never created.When I run ray status I see the following:
If I run the same script, but remove the need for 1 of the actors, then it runs without hanging. The placement group for that script has 1 less bundle:
[{"CPU:1, "GPU: 0"}, {"CPU:1, "GPU: 1"}, {"CPU:1, "GPU: 1"}]
This issue blocks me from being able to run experiments for a blog post on multi gpu training with RLlib in ray 2.5. I cannot train across multiple nodes without this issue appearing.
Versions / Dependencies
ray 5197da2c388a731c05b54b6228fb9800b96982a5
Reproduction script
Issue Severity
High: It blocks me from completing my task.