-
Notifications
You must be signed in to change notification settings - Fork 7k
Labels
P0Issues that should be fixed in short orderIssues that should be fixed in short orderbugSomething that is supposed to be working; but isn'tSomething that is supposed to be working; but isn'tcoreIssues that should be addressed in Ray CoreIssues that should be addressed in Ray Corestability
Description
What happened + What you expected to happen
In actor tasks, the completion of ray.get by any thread can cause all pull requests generated by ray.get in the process (in the pull manager) to be canceled. At this point, other gets will retrigger pull requests, and the progress of the previously pulled objects will be completely reset to zero. This can cause gets to hang directly under the following circumstances.
Versions / Dependencies
ray master: 385e000
Reproduction script
- test.py:
from threading import Thread
import ray; ray.init("auto")
@ray.remote(resources={"worker": 0.1})
class A:
def run(self, refs):
ref_largs = refs[0]
ref_small = ray.put("1")
def get_small():
while True:
ray.get(ref_small)
import time
time.sleep(0.1)
t = Thread(target=get_small, daemon=True)
t.start()
print("start get")
ray.get(ref_largs)
print("done.")
@ray.remote(resources={"head": 0.1})
def main():
refs = [ray.put(b"0" * 10 * 1024 ** 3)]
a = A.remote()
ray.get(a.run.remote(refs))
ray.get(main.remote())- run:
- step1: export RAY_object_manager_max_bytes_in_flight=12000000
- step2: ray start --head --resources '{"head": 10}'
- step3: ray start --address='$YOUR_IP:6379' --resources '{"worker": 10}'
- step4: python test.py
will run forever....
Issue Severity
None
Metadata
Metadata
Assignees
Labels
P0Issues that should be fixed in short orderIssues that should be fixed in short orderbugSomething that is supposed to be working; but isn'tSomething that is supposed to be working; but isn'tcoreIssues that should be addressed in Ray CoreIssues that should be addressed in Ray Corestability