Skip to content

[core] ray.wait doesn't return ready objects until they are local #12190

@stephanie-wang

Description

@stephanie-wang

What is the problem?

Ray version and other system information (Python version, TensorFlow version, OS): Ray 1.1dev

I believe ray.wait() is supposed to return objects once they are ready anywhere in the cluster. Right now, it seems that it only returns ready objects once they have been pulled to the local node.

At minimum, the docs should be updated to reflect this.

Probably, we should also add an option to support returning ready objects once they are ready anywhere. Now that distributed ref counting is implemented, it seems like the buggy raylet-based ray.wait implementation could actually be removed completely and the worker can just check if the value is available in its local memory store.

Reproduction (REQUIRED)

Please provide a script that can be run to reproduce the issue. The script should have no external library dependencies (i.e., use fake or mock data / environments):

This script creates a remote object, then a local object which takes up all the memory in the local node's object store. This causes ray.wait on the remote object to hang even though the remote object has been created, because the remote object cannot be fetched to the local node.

import numpy as np

import ray
from ray.cluster_utils import Cluster
import pytest


cluster = Cluster()
# Force task onto the second node.
cluster.add_node(num_cpus=0, object_store_memory=75 * 1024 * 1024)
cluster.add_node(object_store_memory=75 * 1024 * 1024)

ray.init(cluster.address)

@ray.remote
def put():
    return np.random.rand(5 * 1024 * 1024)  # 40 MB data

local_ref = ray.put(np.random.rand(5 * 1024 * 1024))
print("local", local_ref)
remote_ref = put.remote()
print("remote", remote_ref)

ray.wait([remote_ref], num_returns=1)
print("----")
with pytest.raises(ray.exceptions.GetTimeoutError):
    ray.get(remote_ref, timeout=1)
print("----")
del local_ref
ray.get(remote_ref)

If we cannot run your script, we cannot fix your issue.

  • I have verified my script runs in a clean environment and reproduces the issue.
  • I have verified the issue also occurs with the latest wheels.

Metadata

Metadata

Assignees

No one assigned

    Labels

    P1Issue that should be fixed within a few weeksenhancementRequest for new feature and/or capability

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions