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

ARROW-4296: [Plasma] Use one mmap file by default, prevent crash with -f #3490

Closed
wants to merge 4 commits into from

Conversation

pcmoritz
Copy link
Contributor

This PR is similar to #3434 but also makes sure we only have one well-tested code path to go through.

// Set system memory, potentially rounding it to a page size
// Also make it so dlmalloc fails if we try to request more memory than
// is available.
system_memory = plasma::dlmalloc_set_footprint_limit((size_t)system_memory);
Copy link
Contributor

Choose a reason for hiding this comment

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

c style cast

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

@@ -905,21 +905,20 @@ class PlasmaStoreRunner {
PlasmaStoreRunner() {}

void Start(char* socket_name, int64_t system_memory, std::string directory,
bool hugepages_enabled, bool use_one_memory_mapped_file) {
bool hugepages_enabled) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove the command line flag also.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed

@@ -471,22 +472,26 @@ def assert_create_raises_plasma_full(unit_test, size):
memory_buffers.append(memory_buffer)
# Remaining space is 50%. Make sure that we can't create an
# object of size 50% + 1, but we can create one of size 20%.
assert_create_raises_plasma_full(self, 50 * PERCENT + 1)
assert_create_raises_plasma_full(
self, 50 * PERCENT + SMALL_OBJECT_SIZE)
Copy link
Contributor

Choose a reason for hiding this comment

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

Why replace 1 with SMALL_OBJECT_SIZE?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

because right now, the memory capacity can (and will!) be off by some amount at the moment (due to the way that dlmalloc is computing the footprint limit).

// bookkeeping.
void* pointer = plasma::dlmemalign(kBlockSize, system_memory - 128 * sizeof(size_t));
ARROW_CHECK(pointer != nullptr);
plasma::dlfree(pointer);
Copy link
Contributor

Choose a reason for hiding this comment

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

We know that this won't cause the page to get unmapped?

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 tried it out and it will, but upon remapping, dlmalloc will use the large granularity size, so this has the desired effect of using one large mmap file.

Copy link
Contributor

@robertnishihara robertnishihara left a comment

Choose a reason for hiding this comment

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

If we're truly using one huge page now then we should just send the relevant file descriptor to the clients as soon as they connect and never again.

We should do it in a different PR, but maybe file an issue?

ARROW_LOG(INFO) << "Allowing the Plasma store to use up to "
<< static_cast<double>(system_memory) / 1000000000
<< "GB of memory.";
break;
}
case 'f':
Copy link
Contributor

Choose a reason for hiding this comment

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

Remove f from getopt above

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

// We are using a single memory-mapped file by mallocing and freeing a single
// large amount of space up front. According to the documentation,
// dlmalloc might need up to 128*sizeof(size_t) bytes for internal
// bookkeeping.
Copy link
Contributor

Choose a reason for hiding this comment

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

Add note saying that this relies on implementation details of dlmalloc (that after the initial memory mapped file is unmapped, subsequent memory-mapped files will use the same granularity as the first page) and that if we switch to using jemalloc, this may need to change.

Alteratively, you could do

void* pointer_big = plasma::dlmemalign(kBlockSize, system_memory - 128 * sizeof(size_t));
// We do not deallocate this small object so that the memory mapped file never gets unmapped.
void* pointer_small = plasma::dlmalloc(1);
plasma::dlfree(pointer_big);

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

// Set system memory, potentially rounding it to a page size
// Also make it so dlmalloc fails if we try to request more memory than
// is available.
system_memory =
Copy link
Contributor

Choose a reason for hiding this comment

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

While you're looking at this code, any idea about this ray-project/ray#3670

Somehow the way the object store computes the total system memory is different from the way we do it in Python (e.g., using psutil).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

not sure what is going on but the only way to guarantee that the two sizes are consistent is to use the same method in both (either psutil or the system call we use here)

Copy link
Contributor

Choose a reason for hiding this comment

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

Well we can't really use psutil here because it is in C++, right?

@pcmoritz
Copy link
Contributor Author

This does not guarantee to use one memory mapped file, it just makes it so that there is at least one that is large enough to fit all the objects.

Copy link
Contributor

@robertnishihara robertnishihara left a comment

Choose a reason for hiding this comment

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

LGTM once tests pass. Note that valgrind might complain about pointer_small getting leaked.

@pcmoritz pcmoritz force-pushed the one-mmap-file branch 2 times, most recently from 1e33c7c to 55c0f27 Compare January 28, 2019 02:25
@pcmoritz
Copy link
Contributor Author

This is strange. It looks like the plasma client_tests are hanging. They are running in valgrind and get to the point where

<all the output from the test here up to>
2: ==14032=2: [       OK ] TestPlasmaStore.ManyObjectTest (414 ms)
2: [----------] 11 tests from TestPlasmaStore (2754 ms total)
2: 
2: [----------] Global test environment tear-down
2: [==========] 11 tests from 1 test case ran. (2828 ms total)
2: [  PASSED  ] 11 tests. 
2: ==14032== HEAP SUMMARY:
2: ==14032==     in use at exit: 9,448 bytes in 173 blocks
2: ==14032==   total heap usage: 4,294 allocs, 4,121 frees, 1,131,536 bytes allocated
2: ==14032== 

and then the rest of the valgrind output is never printed.

On EC2 on ubuntu 14.04, running the tests in valgrind with

valgrind --tool=memcheck --gen-suppressions=all                  --leak-check=full --leak-check-heuristics=stdstring --error-exitcode=1 ~/arrow/cpp/build/debug//plasma-client_tests

works however :|

@wesm
Copy link
Member

wesm commented Jan 29, 2019

We recently switched our CI to Ubuntu 16.04 after the conda-forge compiler migration, so it may be specific to 16.04

@pcmoritz
Copy link
Contributor Author

pcmoritz commented Jan 29, 2019

Thanks for the pointer. Wow, I can actually reproduce in 16.04!

EDIT: It just takes a long time to finish, like about 30 sec on the EC2 machine.

@pcmoritz
Copy link
Contributor Author

I understand what is going on now. In verbose mode, valgrind prints

==22169== Searching for pointers to 1 not-freed blocks

where it is hanging. The memory consumption of the process goes up during that time (up to ~10GB), so probably on travis it started swapping and hanging).

Reducing the plasma store memory in the test seems to fix it. However I'd still like to understand where this PR is introducing the 1 non-freed block (there seem to be no such pointers found).

@pcmoritz
Copy link
Contributor Author

pcmoritz commented Jan 29, 2019

So apparently valgrind already did the search before this PR, but is now searching more memory because of the initial dlmemalign. So I'd say this is ready to merge if the tests pass now.

@pcmoritz
Copy link
Contributor Author

the failure is unrelated: /home/travis/build/apache/arrow/cpp/src/arrow/compute/kernel.h:170: error: The following parameters of arrow::compute::UnaryKernel::Call(FunctionContext *ctx, const Datum &input, Datum *out)=0 are not documented:

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