Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[core][experimental] Calling ray.get() on CompiledDAGRef after dag.teardown() or actor failure hangs #46284

Closed
stephanie-wang opened this issue Jun 26, 2024 · 6 comments · Fixed by #46320
Assignees
Labels
accelerated-dag bug Something that is supposed to be working; but isn't P0 Issues that should be fixed in short order

Comments

@stephanie-wang
Copy link
Contributor

What happened + What you expected to happen

We should throw an error if ray.get() is called on a CompiledDAGRef after the DAG has already been torn down. Instead, it seems that ray.get() returns an infinite string of 0s and hangs.

Versions / Dependencies

3.0dev

Reproduction script

import ray
from ray.dag import InputNode

@ray.remote
class Actor:
    def foo(self, arg):
        return arg
        
a = Actor.remote()
with InputNode() as inp:
    dag = a.foo.bind(inp)
    
dag = dag.experimental_compile()
x = dag.execute(1)
ray.kill(a)
# This hangs and returns infinite string of 0s.
ray.get(x)

Issue Severity

None

@stephanie-wang stephanie-wang added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) P0 Issues that should be fixed in short order accelerated-dag and removed triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Jun 26, 2024
@stephanie-wang stephanie-wang added this to the ADAG Developer Preview milestone Jun 26, 2024
@stephanie-wang stephanie-wang changed the title [core][experimental] Calling ray.get() on CompiledDAGRef after dag.teardown() hangs [core][experimental] Calling ray.get() on CompiledDAGRef after dag.teardown() or actor failure hangs Jun 26, 2024
@anyscalesam
Copy link
Collaborator

possible root cause for #46253 ... jack and kai-hsun to look into it.

@jackhumphries
Copy link
Contributor

I reproduced this with the following script:

import ray
from ray.dag import InputNode

@ray.remote
class Actor:
    def foo(self, arg):
        return arg
        
a = Actor.remote()
with InputNode() as inp:
    dag = a.foo.bind(inp)
    
dag = dag.experimental_compile()
x = dag.execute(1)
dag.teardown()
# This hangs and returns infinite string of 0s.
print(ray.get(x))

@stephanie-wang
Copy link
Contributor Author

Yes, we want it to work in both cases, whether the DAG has been explicitly torn down or if an actor in the DAG failed. The only difference is that the latter case should also print out an ActorDiedError.

@jackhumphries
Copy link
Contributor

The issue seems to be that an IOError (channel closed) is being properly returned by CoreWorker::Get(), but the Python code is ignoring this error status and doing an invalid memory access to the buffer.

@kevin85421
Copy link
Member

I used the same example as #46284 (comment).

I added a print function, print("check_status: ", message, status.ok(), status.IsChannelError()), to check_status.

ray/python/ray/_raylet.pyx

Lines 560 to 561 in 755a49b

with gil:
message = status.message().decode()

I have two questions:

  • I can see (Actor pid=2054169) check_status: Channel closed. False True on my console. I expected that the driver process would also print similar logs for the RayChannelError, but it didn't.
  • I think check_status should raise a RayChannelError because status.IsChannelError() is True in the above log, and get_objects (code) doesn't have any try ... except logic, so I expected to see some logs about RayChannelError, but I didn't.
    raise RayChannelError(message)

@jackhumphries
Copy link
Contributor

I used the same example as #46284 (comment).

I added a print function, print("check_status: ", message, status.ok(), status.IsChannelError()), to check_status.

ray/python/ray/_raylet.pyx

Lines 560 to 561 in 755a49b

with gil:
message = status.message().decode()

I have two questions:

  • I can see (Actor pid=2054169) check_status: Channel closed. False True on my console. I expected that the driver process would also print similar logs for the RayChannelError, but it didn't.
  • I think check_status should raise a RayChannelError because status.IsChannelError() is True in the above log, and get_objects (code) doesn't have any try ... except logic, so I expected to see some logs about RayChannelError, but I didn't.
    raise RayChannelError(message)

I wrote a response for this in the PR: #46320 (comment)

can-anyscale pushed a commit that referenced this issue Jul 1, 2024
#46320)

Prior to this PR (described in #46284), calling `ray.get()` on a
`CompiledDAGRef` (i.e., a channel) after DAG teardown would return a
large series of zeroes. This issue could be reproduced with this script:
```
import ray
from ray.dag import InputNode

@ray.remote
class Actor:
    def foo(self, arg):
        return arg
        
a = Actor.remote()
with InputNode() as inp:
    dag = a.foo.bind(inp)
    
dag = dag.experimental_compile()
x = dag.execute(1)
dag.teardown()
# `ray.get(x)` returns a large series of zeroes.
print(ray.get(x))
```

This issue happened because the channel was unregistered with the
mutable object manager on DAG teardown, and thus on a subsequent access
to the channel, the core worker thought the channel reference was for a
normal immutable Ray object rather than for a channel mutable object.
Thus, the core worker was returning the raw underlying memory for the
mutable object, and the memory buffers were sized equal to the total
size of the underlying memory, not the amount of data in the mutable
object.

This PR fixes this issue by properly checking that a channel is either
currently registered or previously registered, rather than just checking
only that the channel is currently registered.

Signed-off-by: Jack Humphries <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
accelerated-dag bug Something that is supposed to be working; but isn't P0 Issues that should be fixed in short order
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants