Skip to content

[core] Spill at least the object fusion size instead of at most#22750

Merged
stephanie-wang merged 23 commits intoray-project:masterfrom
stephanie-wang:fix-object-fusion
Apr 5, 2022
Merged

[core] Spill at least the object fusion size instead of at most#22750
stephanie-wang merged 23 commits intoray-project:masterfrom
stephanie-wang:fix-object-fusion

Conversation

@stephanie-wang
Copy link
Copy Markdown
Contributor

@stephanie-wang stephanie-wang commented Mar 2, 2022

Why are these changes needed?

Copied from #22571:

Whenever we spill, we try to spill all spillable objects. We also try to fuse small objects together to reduce total IOPS. If there aren't enough objects in the object store to meet the fusion threshold, we spill the objects anyway to avoid liveness issues. However, currently we spill at most the object fusion size when instead we should be spilling at least the fusion size. Then we use the max number of fused objects as a cap.

This PR fixes the fusion behavior so that we always spill at minimum the fusion size. If we reach the end of the spillable objects, and we are under the fusion threshold, we'll only spill it if we don't have other spills pending too. This gives the pending spills time to finish, and then we can re-evaluate whether it's necessary to spill the remaining objects. Liveness is also preserved.

Increases some test timeouts to allow tests to pass.

Copy link
Copy Markdown
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

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

Didn't review the PR, but I assume the bug was fixed..

@kfstorm
Copy link
Copy Markdown
Member

kfstorm commented Mar 13, 2022

‼️ ACTION REQUIRED ‼️

We've updated our formatting configuration for C++ code. (see #22725)

This PR includes C++ code change. To prevent issues with merging your code, here's what you'll need to do:

  1. Merge the latest changes from upstream/master branch into your branch.
git pull upstream master
git merge upstream/master
  1. Resolve merge conflicts (if necessary).

After running these steps, you'll have the updated C++ formatting configuration.

  1. Format changed files.
scripts/format.sh
  1. Commit your changes.
git add --all
git commit -m "Format C++ code"

@stephanie-wang
Copy link
Copy Markdown
Contributor Author

@rkooo567 I updated this PR a bit to address another issue I found where we're spilling at most the fusion size instead of at least. Can you take a look at the C++ changes and re-approve?

@stephanie-wang stephanie-wang changed the title [core] Fix bug in fusion of spilled objects [core] Spill at least the object fusion size instead of at most Apr 1, 2022
Copy link
Copy Markdown
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

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

Temporarily request changed when I review the new change (will do this sun)

Copy link
Copy Markdown
Contributor

@rkooo567 rkooo567 left a comment

Choose a reason for hiding this comment

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

I think we should update the docstring, but other than that, it lgtm

int64_t counts = 0;
while (bytes_to_spill <= num_bytes_to_spill && it != pinned_objects_.end() &&
counts < max_fused_object_count_) {
while (it != pinned_objects_.end() && counts < max_fused_object_count_) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Can you update the docstring of SpillObjectsOfSize to explain semantic changes?

@stephanie-wang stephanie-wang merged commit 1c972d5 into ray-project:master Apr 5, 2022
@stephanie-wang stephanie-wang deleted the fix-object-fusion branch April 5, 2022 17:57
edoakes pushed a commit to edoakes/ray that referenced this pull request Apr 7, 2022
…project#22750)

Copied from ray-project#22571:

Whenever we spill, we try to spill all spillable objects. We also try to fuse small objects together to reduce total IOPS. If there aren't enough objects in the object store to meet the fusion threshold, we spill the objects anyway to avoid liveness issues. However, currently we spill at most the object fusion size when instead we should be spilling at least the fusion size. Then we use the max number of fused objects as a cap.

This PR fixes the fusion behavior so that we always spill at minimum the fusion size. If we reach the end of the spillable objects, and we are under the fusion threshold, we'll only spill it if we don't have other spills pending too. This gives the pending spills time to finish, and then we can re-evaluate whether it's necessary to spill the remaining objects. Liveness is also preserved.

Increases some test timeouts to allow tests to pass.
stephanie-wang added a commit to stephanie-wang/ray that referenced this pull request May 11, 2022
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.

3 participants