-
Notifications
You must be signed in to change notification settings - Fork 5.6k
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
Prevent object store from allocating over the specified limit even if there is memory fragmentation #15951
Conversation
(Just re-mention it here) I think you are doing it in the right way. The fundamental solution would be using another library that creates fewer fragments (dlmalloc is kind of too old). Another way is to change the alignment of dlmalloc. Currently it is forced to be aligned to 64 bytes, this would increase some internal fragments. |
@@ -49,7 +49,7 @@ def create_object(self, size): | |||
# Submit enough methods on the actor so that they exceed the size of the | |||
# object store. | |||
objects = [] | |||
num_objects = 20 | |||
num_objects = 40 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
20 doesn't work due to memory fragmentation (60MB chunks in 150MB => lots of fragmentation).
if (space_needed > 0) { | ||
// make room. NOTE(ekl) if we can't achieve this after a number of retries, | ||
// it's because memory fragmentation in dlmalloc prevents us from allocating | ||
// even if our footprint tracker here still says we have free space. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why don't we use a different error for better error messages here? (e.g., FragmentationError?) (if retries == 10?)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think we should be exposing these low-level details to the user.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm I agree with that. But would there be other good idea to see if this causes the OOM issue? I think this could help us debugging issues when it happens. Probably writing a log message?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think this causes any new OOM issues (previously, you'd get SIGBUS, now you'd get a proper error message or spilling).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nice!!!
Given that this will effectively decrease the object store capacity for a given object store allocation, are there any changes that we should make to the docs? Should we change the default object store allocation to e.g. 90% of /dev/shm rather than 30% of available system memory? Anything like that?
RAY_LOG(DEBUG) << "fake_mmap called once already, refusing to allocate: " << size; | ||
return MFAIL; | ||
} | ||
allocated_once = true; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't this only be set if RayConfig::instance().preallocate_plasma_memory()
is set? Otherwise, won't every object that's created after the first object fail if preallocation is turned off? Given that the tests are passing, I'm probably missing something here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Plasma is always allocating "object_store_size" memory on the first allocation, the preallocation flag only controls whether we are telling the OS to pre-allocate the pages of the file (vs leaving them as allocate-on-write pages).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, that's right, thanks!
Co-authored-by: Clark Zinzow <clarkzinzow@gmail.com>
Windows test failure was flaky (and succeeded on previous build). |
@@ -210,6 +210,7 @@ uint8_t *PlasmaStore::AllocateMemory(size_t size, MEMFD_TYPE *fd, int64_t *map_s | |||
|
|||
// Try to evict objects until there is enough space. | |||
uint8_t *pointer = nullptr; | |||
int num_tries = 0; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to retry here? We also retry at a higher level, at the CreateRequestQueue.
Why are these changes needed?
Currently, we enforce the size of the plasma store at the server. However, dlmalloc can under the hood try to allocate more memory due to internal fragmentation. This disables these additional allocations, which can lead to SIGBUS if we run out of backing pages in /dev/shm.
Related to
#14182