[core] Spill at least the object fusion size instead of at most#22750
[core] Spill at least the object fusion size instead of at most#22750stephanie-wang merged 23 commits intoray-project:masterfrom
Conversation
rkooo567
left a comment
There was a problem hiding this comment.
Didn't review the PR, but I assume the bug was fixed..
80758d4 to
8507965
Compare
|
|
@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? |
rkooo567
left a comment
There was a problem hiding this comment.
Temporarily request changed when I review the new change (will do this sun)
rkooo567
left a comment
There was a problem hiding this comment.
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_) { |
There was a problem hiding this comment.
Can you update the docstring of SpillObjectsOfSize to explain semantic changes?
…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.
…st (ray-project#22750)" This reverts commit 1c972d5.
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.