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] Fix bug in dynamic generator tasks in object location handling #29082

Merged
merged 6 commits into from
Oct 6, 2022

Conversation

stephanie-wang
Copy link
Contributor

Why are these changes needed?

When dynamically generated objects are put into the object store, we need to notify the owner of the new location. However, currently this message can reach the owner before the owner knows about the object, since the task reply can arrive much later. Then, the owner will think that the object has already gone out of scope and will not add the location, leading to an ObjectFetchTimedOutError once any worker on a different node tries to read the object.

This PR fixes the bug by adding the object to the ref counter if it was returned by a dynamic generator task. The owner does this by checking whether the task spec had num_returns="dynamic" set.

Related issue number

Closes #28911.

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@stephanie-wang stephanie-wang marked this pull request as ready for review October 5, 2022 16:56
Signed-off-by: Stephanie Wang <[email protected]>
time.sleep(0.1)

@ray.remote
def read(gen):
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is not used?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah thanks, will remove.

def dynamic_generator(num_returns):
for i in range(num_returns):
yield np.ones(1_000_000, dtype=np.int8) * i
time.sleep(0.1)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how often does this test reproduce the bug? If low, would increase the sleep increase the likelyhood of reproducibility?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It reproduces reliably on my laptop! So I think it's probably safe enough on CI.

Copy link
Collaborator

@jjyao jjyao left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, could we add a c++ test as well?

@stephanie-wang
Copy link
Contributor Author

Actually, could we add a c++ test as well?

Not sure of a good way to do this since the relevant logic is all in the CoreWorker handler.

x
Signed-off-by: Stephanie Wang <[email protected]>
Signed-off-by: Stephanie Wang <[email protected]>
@jjyao
Copy link
Collaborator

jjyao commented Oct 5, 2022

Not sure of a good way to do this since the relevant logic is all in the CoreWorker handler.

Seems in the future, we should consider how to refactor the code so it's more unit-testable.

@stephanie-wang
Copy link
Contributor Author

Not sure of a good way to do this since the relevant logic is all in the CoreWorker handler.

Seems in the future, we should consider how to refactor the code so it's more unit-testable.

For sure, it's been a long-standing issue :(

@stephanie-wang stephanie-wang merged commit 1e616ef into ray-project:master Oct 6, 2022
@stephanie-wang stephanie-wang deleted the fix-generator-loc branch October 6, 2022 19:57
WeichenXu123 pushed a commit to WeichenXu123/ray that referenced this pull request Dec 19, 2022
…ray-project#29082)

When dynamically generated objects are put into the object store, we need to notify the owner of the new location. However, currently this message can reach the owner before the owner knows about the object, since the task reply can arrive much later. Then, the owner will think that the object has already gone out of scope and will not add the location, leading to an ObjectFetchTimedOutError once any worker on a different node tries to read the object.

This PR fixes the bug by adding the object to the ref counter if it was returned by a dynamic generator task. The owner does this by checking whether the task spec had num_returns="dynamic" set.
Related issue number

Closes ray-project#28911.

Signed-off-by: Stephanie Wang <[email protected]>
Signed-off-by: Weichen Xu <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Datasets] ObjectFetchTimedOutError with dynamic block splitting enabled
3 participants