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

Prevent object store from allocating over the specified limit even if there is memory fragmentation #15951

Merged
merged 10 commits into from
May 25, 2021

Conversation

ericl
Copy link
Contributor

@ericl ericl commented May 20, 2021

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

@suquark
Copy link
Member

suquark commented May 20, 2021

(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
Copy link
Contributor Author

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).

@clarkzinzow clarkzinzow self-assigned this May 24, 2021
@ericl ericl changed the title [WIP] Force dlmalloc to only allocate once on startup Prevent object store from allocating over the specified limit even if there is memory fragmentation May 24, 2021
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.
Copy link
Contributor

@rkooo567 rkooo567 May 24, 2021

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?)

Copy link
Contributor Author

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.

Copy link
Contributor

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?

Copy link
Contributor Author

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).

Copy link
Contributor

@clarkzinzow clarkzinzow left a 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?

src/ray/object_manager/plasma/dlmalloc.cc Outdated Show resolved Hide resolved
RAY_LOG(DEBUG) << "fake_mmap called once already, refusing to allocate: " << size;
return MFAIL;
}
allocated_once = true;
Copy link
Contributor

@clarkzinzow clarkzinzow May 24, 2021

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.

Copy link
Contributor Author

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).

Copy link
Contributor

@clarkzinzow clarkzinzow May 24, 2021

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>
@ericl
Copy link
Contributor Author

ericl commented May 25, 2021

Windows test failure was flaky (and succeeded on previous build).

@ericl ericl merged commit ea6bdfb into ray-project:master May 25, 2021
@@ -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;
Copy link
Contributor

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.

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.

5 participants