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

[adag] Avoid deserialization during CompiledDAGRef's deallocation #47614

Open
jeffreyjeffreywang opened this issue Sep 11, 2024 · 11 comments
Open
Labels
accelerated-dag bug Something that is supposed to be working; but isn't core Issues that should be addressed in Ray Core triage Needs triage (eg: priority, bug/not-bug, and owning component)

Comments

@jeffreyjeffreywang
Copy link
Contributor

What happened + What you expected to happen

Although we don't call ray.get, ray.get is called and deserialization still happens when the dag ref is deallocated because of the following code.

if not self._ray_get_called:

Versions / Dependencies

ray master

Reproduction script

a = Actor.remote(0)
with InputNode() as inp:
    dag = a.inc.bind(inp)

compiled_dag = dag.experimental_compile()
ref = compiled_dag.execute(1)
compiled_dag.teardown()

# ref.get() is called upon ref's deallocation -- deserialization still happens

Issue Severity

None

@jeffreyjeffreywang jeffreyjeffreywang added bug Something that is supposed to be working; but isn't triage Needs triage (eg: priority, bug/not-bug, and owning component) labels Sep 11, 2024
@rkooo567
Copy link
Contributor

@jeffreyjeffreywang would you be interested in taking this?

@jeffreyjeffreywang
Copy link
Contributor Author

jeffreyjeffreywang commented Sep 12, 2024

@rkooo567 Yup, definitely!

Hey @stephanie-wang @ruisearch42, I just wanted to clarify a few things before I proceed with this issue. You've suggested to release the value of a CompiledDAGRef if ray.get() isn't called by the user (#45951 (comment)). Could you please help me understand how the following code avoids execution result leak?

# If not yet, get the result and discard to avoid execution result leak.
if not self._ray_get_called:
self.get()

With this code, another issue arises -- attempting to load a python library during program exit (when CompiledDAGRef is destructed) will fail. Please refer to #47305 (comment) for more context.

foo = Foo.remote()
bar = Bar.remote()

with InputNode() as inp:
    dag = MultiOutputNode([foo.increment.bind(inp), bar.decrement.bind(inp)])

dag = dag.experimental_compile()

ref1 = dag.execute(1)
ref2 = dag.execute(1)

assert ref1.get() == [1, -1]

dag.teardown()
# Upon destruction, the DAG will be executed until the latest index.
# However, we attempt to import DAGContext during program exit. 

I'm thinking about removing the custom destructor entirely but wanted to understand the implications before doing so.

@rkooo567
Copy link
Contributor

With this code, another issue arises -- attempting to load a python library during program exit (when CompiledDAGRef is destructed) will fail. Please refer to #47305 (comment) for more context.

I think this is because python cannot guarantee all modules exist when __del__ is called. So I agreed doing clean up this way in a destructor is a bad idea, and it is something we should clean up.

@ruisearch42
Copy link
Contributor

Could you please help me understand how the following code avoids execution result leak?

With this code, the python object is retrieved and then immediately goes out of scope. If there are any native buffers underneath, they will also be released.

@jeffreyjeffreywang
Copy link
Contributor Author

With this code, the python object is retrieved and then immediately goes out of scope. If there are any native buffers underneath, they will also be released.

Thank you @ruisearch42, could you give me an example/repro when native buffers are used and therefore this destruction is necessary? I'd like to measure whether the deserialization is necessary.

If deserialization is necessary, we still need to solve the module import issue. We might want to move the deserialization (stepping through the remaining steps in the DAG) to teardown().

@jeffreyjeffreywang
Copy link
Contributor Author

This is a duplicate of #46909. Will close both bugs once this issue is addressed.

@stephanie-wang
Copy link
Contributor

Thanks, @jeffreyjeffreywang for the great questions! The deserialization is necessary because the native buffer is reused for future data. If the reader does not explicitly read and release the buffer, then the buffer cannot be reused for future values. You can reproduce it by returning a numpy array as the DAG output; since numpy arrays are zero-copy, the buffer will be held until the np array in python goes out of scope.

Note that you do not need to deserialize the data in order to release the buffer. We just need to make sure to call the ReadAcquire and ReadRelease methods on the buffer (but skip the python-based deserialization).

We do a similar custom destructor for when ObjectRefs and actors go out of scope, so I think you can reuse a similar codepath to avoid the destruction ordering problem, see here.

@jeffreyjeffreywang
Copy link
Contributor Author

Thank you, Stephanie, for the thorough explanation. I'll dig a bit deeper and publish a PR.

@rkooo567
Copy link
Contributor

Thanks @jeffreyjeffreywang ! Btw, are you in OSS ray slack? We have regular sync up, and you are more than welcome to join!

@jeffreyjeffreywang
Copy link
Contributor Author

@rkooo567 Yeah, I just joined couple days ago. Thanks for inviting, I'll keep an eye on the sync up next time and hopefully I'll be able to join! 😄

@rkooo567
Copy link
Contributor

@anyscalesam can you make sure @jeffreyjeffreywang is invited to next sync?! Thank you!

@anyscalesam anyscalesam added the core Issues that should be addressed in Ray Core label Sep 16, 2024
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 core Issues that should be addressed in Ray Core triage Needs triage (eg: priority, bug/not-bug, and owning component)
Projects
None yet
Development

No branches or pull requests

5 participants