[Object spilling] Add policy to automatically spill objects on OutOfMemory#11673
[Object spilling] Add policy to automatically spill objects on OutOfMemory#11673ericl merged 18 commits intoray-project:masterfrom
Conversation
|
I think there are some missing BUILD files (doesn't compile). |
| // TODO: Only respond to the client with OutOfMemory if we could not | ||
| // make enough space through spilling. If we could make enough space, | ||
| // respond to the plasma client once spilling is complete. | ||
| static_cast<void>(spill_objects_callback_(space_needed)); |
There was a problem hiding this comment.
I might misunderstand here, but doesn't this mean raylet can try creating new objects or pulling new objects "before" all objects are actually spilled, and it can lead to OOM in some scenarios?
There was a problem hiding this comment.
Or is it handled from the object store side?
There was a problem hiding this comment.
Yes, it's possible that if the spilling is too slow, the client will still receive an OOM even though enough space will be made eventually.
| // See the License for the specific language governing permissions and | ||
| // limitations under the License. | ||
|
|
||
| namespace ray { |
|
This is pretty cool. One thing I noticed is that This kind of workload will hang because the retries are waiting not enough time. I guess this will be addressed with the async RPC, but in the interim can we return a special return code to allow indefinite retries with low delay (e.g., every 10ms retry). This would allow spilling to be used for real workloads with pretty good performance. |
Hmm so I tried this and it actually still has the same problem. I added a new error code for the object store being transiently out of memory, and the client will retry quickly after receiving this error code. This is to prevent indefinite retries in the case where two clients need to create an object and there are pending spills, but only one of the clients will be able to create its object after the spills complete. If we just did an infinite retry loop on the second client, we could hang and never throw OutOfMemory. There are two issues in the script you posted, both pretty subtle:
I'll push what I have, but it seems like there's a lot more to do here. |
| } | ||
|
|
||
| int64_t LocalObjectManager::SpillObjectsOfSize(int64_t num_bytes_required) { | ||
| if (RayConfig::instance().object_spilling_config().empty() || |
There was a problem hiding this comment.
Btw, isn't the plasma store and object store running in a separate thread? If so, don't we need to lock this method? (since it is called within store.cc)?
There was a problem hiding this comment.
Oh hmm I think you're right about that. I'll add a lock. It seems weird that this wasn't triggered in any of the tests yet.
|
@stephanie-wang I need more familiarity to the codebase, so it is probably not possible, but why don't we just queue up the creation requests and invoke them after object spilling requests are completed? Is it tricky to be implemented? |
It is possible, and I think we should do it, but it's a bit complicated to do right now without refactoring the plasma store. I thought it'd be better to leave it for a separate PR. |
Why are these changes needed?
This adds a callback when an object store node runs out of memory to choose objects to spill, after first attempting to make space through eviction of objects not currently referenced. Since spilling is asynchronous, the object store client must try to create the object again. Once the object spilling is complete, the object creation will succeed.
A TODO is to modify the object store to respond to the client asynchronously. This is so that, in the case that we can definitely make enough space by spilling other objects, the object store client does not have to retry the create call on a timer and we do not block the object store while the objects are being spilled.
This PR also introduces some changes to the way configs are passed around the cluster to accommodate passing around the object spilling config, which is a JSON string. Long-term, we should have a less brittle way to pass around arbitrary config values.
Related issue number
Closes #9849.
Checks
scripts/format.shto lint the changes in this PR.