ray-project / ray

Ray is an AI compute engine. Ray consists of a core distributed runtime and a set of AI Libraries for accelerating ML workloads.
https://ray.io
Apache License 2.0
33.87k stars 5.76k forks source link

[Core, RLlib] Multi GPU RLlib experiment is unable to be scheduled. #35409

Open avnishn opened 1 year ago

avnishn commented 1 year ago

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

======== Autoscaler status: 2023-05-16 15:20:23.259600 ========
GCS request time: 0.001424s
Node Provider non_terminated_nodes time: 0.003899s

Node status
---------------------------------------------------------------
Healthy:
 4 worker-node-type-1
 1 head-node-type
Pending:
 (no pending nodes)
Recent failures:
 (no failures)

Resources
---------------------------------------------------------------
Total Usage:
 0.0/32.0 CPU (0 used of 0.0 reserved in placement groups)
 0.0/4.0 GPU (0 used of 0.0 reserved in placement groups)
 0.0/4.0 accelerator_type:T4
 0B/74.07GiB memory
 0B/34.19GiB object_store_memory

Total Demands:
 (no resource demands)

Node: 10.0.60.35
 Usage:
  0.0/16.0 CPU
  0B/34.23GiB memory
  0B/17.11GiB object_store_memory

Node: 10.0.14.120
 Usage:
  0.0/4.0 CPU (0 used of 0.0 reserved in placement groups)
  0.0/1.0 GPU (0 used of 0.0 reserved in placement groups)
  0.0/1.0 accelerator_type:T4
  0B/9.97GiB memory
  0B/4.27GiB object_store_memory

Node: 10.0.52.205
 Usage:
  0.0/4.0 CPU (0 used of 0.0 reserved in placement groups)
  0.0/1.0 GPU (0 used of 0.0 reserved in placement groups)
  0.0/1.0 accelerator_type:T4
  0B/9.97GiB memory
  0B/4.27GiB object_store_memory

Node: 10.0.35.146
 Usage:
  0.0/4.0 CPU
  0.0/1.0 GPU
  0.0/1.0 accelerator_type:T4
  0B/9.97GiB memory
  0B/4.27GiB object_store_memory

Node: 10.0.39.211
 Usage:
  0.0/4.0 CPU
  0.0/1.0 GPU
  0.0/1.0 accelerator_type:T4
  0B/9.95GiB memory
  0B/4.26GiB object_store_memory

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:

======== Autoscaler status: 2023-05-16 15:25:29.495778 ========
GCS request time: 0.001564s
Node Provider non_terminated_nodes time: 0.004192s

Node status
---------------------------------------------------------------
Healthy:
 1 head-node-type
 4 worker-node-type-1
Pending:
 (no pending nodes)
Recent failures:
 (no failures)

Resources
---------------------------------------------------------------
Total Usage:
 3.0/32.0 CPU (3.0 used of 4.0 reserved in placement groups)
 1.0/4.0 GPU (1.0 used of 2.0 reserved in placement groups)
 0.0/4.0 accelerator_type:T4
 0B/74.07GiB memory
 0B/34.19GiB object_store_memory

Total Demands:
 {'CPU': 1.0, 'GPU': 1.0}: 1+ pending tasks/actors (1+ using placement groups)

Node: 10.0.60.35
 Usage:
  0.0/16.0 CPU
  0B/34.23GiB memory
  0B/17.11GiB object_store_memory

Node: 10.0.14.120
 Usage:
  1.0/4.0 CPU (1.0 used of 1.0 reserved in placement groups)
  0.0/1.0 GPU (0.0 used of 1.0 reserved in placement groups)
  0.0/1.0 accelerator_type:T4
  0B/9.97GiB memory
  0B/4.27GiB object_store_memory

Node: 10.0.52.205
 Usage:
  0.0/4.0 CPU (0 used of 0.0 reserved in placement groups)
  0.0/1.0 GPU (0 used of 0.0 reserved in placement groups)
  0.0/1.0 accelerator_type:T4
  0B/9.97GiB memory
  0B/4.27GiB object_store_memory

Node: 10.0.35.146
 Usage:
  2.0/4.0 CPU (2.0 used of 3.0 reserved in placement groups)
  1.0/1.0 GPU (1.0 used of 1.0 reserved in placement groups)
  0.0/1.0 accelerator_type:T4
  0B/9.97GiB memory
  0B/4.27GiB object_store_memory

Node: 10.0.39.211
 Usage:
  0.0/4.0 CPU
  0.0/1.0 GPU
  0.0/1.0 accelerator_type:T4
  0B/9.95GiB memory
  0B/4.26GiB object_store_memory

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

import argparse

import ray
from ray import air, tune
from ray.rllib.algorithms.ppo import PPOConfig
from ray.rllib.examples.env.dm_control_suite import cheetah_run

def run_with_tuner_1_rollout_worker(config):
    config = config.rollouts(num_rollout_workers=1)
    print("-" * 80)
    tuner = tune.Tuner(
        "PPO",
        param_space=config,
        run_config=air.RunConfig(
            stop={"timesteps_total": 128},
            failure_config=air.FailureConfig(max_failures=3),
            storage_path="/mnt/shared_storage/avnishn/ppo_multi_gpu_benchmarking",
            checkpoint_config=air.CheckpointConfig(checkpoint_frequency=1),
            sync_config=tune.SyncConfig(syncer=None)
        ),
    )
    tuner.fit()

def run_with_tuner_0_rollout_worker(config):
    print("-" * 80)
    config = config.rollouts(num_rollout_workers=0)
    tuner = tune.Tuner(
        "PPO",
        param_space=config,
        run_config=air.RunConfig(
            stop={"timesteps_total": 128},
            failure_config=air.FailureConfig(max_failures=3),
            storage_path="/mnt/shared_storage/avnishn/ppo_multi_gpu_benchmarking",
            checkpoint_config=air.CheckpointConfig(checkpoint_frequency=1),
            sync_config=tune.SyncConfig(syncer=None)
        ),
    )
    tuner.fit()

if __name__ == "__main__":

    # This experiment is run on a machine with a 8 cpu headnode, and 2, 1 gpu 4 cpu workernodes.
    # Note I couldn't reproduce this bug if I made my worker nodes 2, 4 gpu 32 cpu instances.

    ray.init()

    tune.registry.register_env("HalfCheetahDmc", lambda c: cheetah_run(from_pixels=False))

    config = (
        PPOConfig()
        .training(_enable_learner_api=True,
                  model={"fcnet_hiddens": [256, 256, 256], 
                         "fcnet_activation": "relu", 
                         "vf_share_layers": True},
                  train_batch_size=128)
        .rl_module(_enable_rl_module_api=True)
        .environment("HalfCheetahDmc")
        .resources(num_gpus_per_learner_worker=1, num_learner_workers=2)
        .rollouts(num_rollout_workers=1)
        .reporting(min_time_s_per_iteration=0,
                   min_sample_timesteps_per_iteration=10

        )
    )

    # run_with_tuner_0_rollout_worker(config)  # this works
    print("finished without tune")
    print("*" * 100)
    run_with_tuner_1_rollout_worker(config)  # this hangs

Issue Severity

High: It blocks me from completing my task.

jjyao commented 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.

jjyao commented 1 year ago

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

kouroshHakha commented 1 year ago

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?

kouroshHakha commented 1 year ago

Please tag me and @avnishn in the follow up conversations.

jjyao commented 1 year ago

@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.

can-anyscale commented 1 year ago

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!

avnishn commented 1 year ago

https://github.com/ray-project/ray/pull/35679

also need to merge this

ArturNiederfahrenhorst commented 1 year ago

@avnishn Merged on release, branch.

can-anyscale commented 1 year ago

@avnishn: is this already done? thanks