Skip to content

Commit

Permalink
wasm/allocator: introduce stack allocator
Browse files Browse the repository at this point in the history
By default wasmtime allocates stacks using `mmap`, but there is now an
API to override the stack allocator.

This is the custom allocator that we will plug into wasmtime. We
allocate stacks on demand since they will fit within our 128KB
allocation limit that we recommend. These stacks will also have a
guard page at the bottom of the stack to protect against stack overflow.

We cache stacks (indefinitely) because the call to `mprotect` will break
up any transparent huge pages (THP) that have been allocated by the
seastar allocator, in an effort to not breakup all these pages all over
memory we reuse them aggressively. Another option is to preallocate a
2MB THP on startup and chunk that up for wasmtime stacks, but then that
imposes a limit on the number of VMs we can run on a single core, to
attempt to prevent that limitation we will allocate them dynamically, as
the performance impact should be small anyways for breaking up those
pages.

There is more as comments in the allocator header as well.

Signed-off-by: Tyler Rockwood <rockwood@redpanda.com>
  • Loading branch information
rockwotj committed Oct 13, 2023
1 parent 6764502 commit 821bd83
Show file tree
Hide file tree
Showing 3 changed files with 266 additions and 0 deletions.
82 changes: 82 additions & 0 deletions src/v/wasm/allocator.cc
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,16 @@
*/
#include "wasm/allocator.h"

#include "vassert.h"
#include "vlog.h"
#include "wasm/logger.h"

#include <seastar/core/align.hh>
#include <seastar/core/aligned_buffer.hh>
#include <seastar/core/print.hh>

#include <sys/mman.h>

#include <stdexcept>
#include <unistd.h>

Expand Down Expand Up @@ -46,4 +52,80 @@ void heap_allocator::deallocate(heap_memory m) {
_memory_pool.push_back(std::move(m));
}

stack_memory::stack_memory(stack_bounds bounds, allocated_memory data)
: _bounds(bounds)
, _data(std::move(data)) {}

stack_memory::~stack_memory() {
if (!_data) {
// This can happen if the memory was moved.
return;
}
int r = ::mprotect(
_data.get(), _bounds.bottom - _data.get(), PROT_READ | PROT_WRITE);
vassert(r == 0, "stack memory must be able to unprotect on destruction");
}

size_t stack_memory::size() const { return _bounds.top - _bounds.bottom; }

stack_bounds stack_memory::bounds() const { return _bounds; }

stack_allocator::stack_allocator(config c)
: _tracking_enabled(c.tracking_enabled)
, _page_size(::getpagesize()) {}

stack_memory stack_allocator::allocate(size_t size) {
size = ss::align_up(size, _page_size);
stack_memory mem;
// Reuse a page
if (!_memory_pool.empty() && _memory_pool.front().size() == size) {
mem = std::move(_memory_pool.front());
_memory_pool.pop_front();
} else {
// Create a stack with a guard page and aligned to a page.
auto buffer = ss::allocate_aligned_buffer<uint8_t>(
size + _page_size, _page_size);
// Protect the guard page by making it read only.
::mprotect(buffer.get(), _page_size, PROT_NONE);
uint8_t* bottom = buffer.get() + _page_size;
// NOLINTNEXTLINE(cppcoreguidelines-pro-bounds-pointer-arithmetic)
stack_bounds bounds{.top = bottom + size, .bottom = bottom};
mem = {bounds, std::move(buffer)};
}
if (_tracking_enabled) {
_live_stacks.emplace(mem.bounds());
}
return mem;
}

bool stack_allocator::tracking_enabled() const { return _tracking_enabled; }

std::optional<stack_bounds>
stack_allocator::stack_bounds_for_address(uint8_t* address) const {
// We're looking for the first lexicographically ordered (top, bottom)
// pair that comes before address. In the case top == address, we need to
// pick a bottom that is lower than all others, thus we pick 0 (nullptr).
stack_bounds point{.top = address, .bottom = nullptr};
auto it = _live_stacks.lower_bound(point);
if (it == _live_stacks.end()) {
return std::nullopt;
}
return *it;
}

void stack_allocator::deallocate(stack_memory mem) {
if (_tracking_enabled) {
_live_stacks.erase(mem.bounds());
}
// Return this stack back to the pool so that it can be reused.
_memory_pool.push_back(std::move(mem));
}

std::ostream& operator<<(std::ostream& os, const stack_bounds& bounds) {
return os << ss::format(
"{{.top = {}, .bottom = {}}}",
static_cast<void*>(bounds.top),
static_cast<void*>(bounds.bottom));
}

} // namespace wasm
116 changes: 116 additions & 0 deletions src/v/wasm/allocator.h
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
#include <seastar/core/chunked_fifo.hh>
#include <seastar/util/optimized_optional.hh>

#include <absl/container/btree_set.h>

#include <type_traits>

namespace wasm {
Expand Down Expand Up @@ -84,4 +86,118 @@ class heap_allocator {
ss::chunked_fifo<heap_memory, items_per_chunk> _memory_pool;
};

/**
* The useable (i.e. excluding guard pages) bounds of an allocated stack.
*/
struct stack_bounds {
uint8_t* top;
uint8_t* bottom;

auto operator<=>(const stack_bounds&) const = default;
friend std::ostream& operator<<(std::ostream&, const stack_bounds&);
};

/**
* An owned bit of stack memory, which is aligned to page size and it's size is
* a multiple of the page size.
*
* Note that there is a guard page of protected memory "below" the stack to
* protect against stack overflow.
*/
class stack_memory {
// NOLINTNEXTLINE(*-avoid-c-arrays)
using allocated_memory = std::unique_ptr<uint8_t[], ss::free_deleter>;

public:
stack_memory() = default;
stack_memory(stack_bounds bounds, allocated_memory data);
stack_memory(const stack_memory&) = delete;
stack_memory& operator=(const stack_memory&) = delete;
stack_memory(stack_memory&&) = default;
stack_memory& operator=(stack_memory&&) = default;
~stack_memory();

stack_bounds bounds() const;
size_t size() const;

private:
friend class stack_allocator;
stack_bounds _bounds;
allocated_memory _data;
};

// The allocator for stack memory within a Wasm VM.
//
// We execute WebAssembly on a seperate stack because WebAssembly doesn't
// have a notion of "async" and we need to be able to both pause the VM and
// support
//
// Additionally memory must be page-aligned and a multiple of page size.
// Some architectures require this alignment for stacks. Additionally, we
// always allocate a guard page at the bottom, which is unprotected when memory
// is "deallocated".
//
// The allocator also supports the ability to query if the current stack being
// used has been allocated from this allocator and returns the bounds. This is
// used in tests to support ensuring that we won't ever overflow the stack by
// adjusting our host functions to run as if Wasm has taken up the maximum
// amount stack space the VM allows.
//
// There are penalties for allocating the guard page on these stacks - the
// kernel will have to break up transparent huge pages (THP) when just a single
// 4KiB page is allocated. To mitigate this we cache stacks that are allocated.
// Currently these are never freed. However there can still be THPs that are
// broken up due to these stacks being scattered around in memory. A future
// optimization may help with this by allocating larger chunks and protecting
// neighboring (small) pages so that it's less likely THPs need to be broken up.
//
// Instance on every core.
class stack_allocator {
public:
struct config {
// If true, enable tracking, otherwise `current_stack_bounds` always
// returns `std::nullopt`.
bool tracking_enabled;
};

explicit stack_allocator(config);

/**
* Allocate stack memory. This size will be allocated plus a single guard
* page at the "bottom" of the stack.
*/
stack_memory allocate(size_t size);

/**
* If tracking was enabled for this allocator.
*/
bool tracking_enabled() const;

/**
* Return the stack bounds if the stack that contains the address has been
* created by this allocator.
*
* Example usage:
* ```
* uint8_t* dummy = 0;
* auto my_stack = stack_allocator.stack_bounds_for_address(&dummy);
* // do something with my_stack
* ```
*/
std::optional<stack_bounds> stack_bounds_for_address(uint8_t*) const;

/**
* Deallocate stack memory.
*/
void deallocate(stack_memory);

private:
bool _tracking_enabled;
size_t _page_size;
absl::btree_set<stack_bounds> _live_stacks;
// We expect this list to be small, so override the chunk to be smaller too.
static constexpr size_t items_per_chunk = 16;
ss::chunked_fifo<stack_memory, items_per_chunk> _memory_pool;
};

} // namespace wasm
68 changes: 68 additions & 0 deletions src/v/wasm/tests/wasm_allocator_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -9,8 +9,10 @@
* by the Apache License, Version 2.0
*/

#include "gmock/gmock.h"
#include "wasm/allocator.h"

#include <gmock/gmock.h>
#include <gtest/gtest.h>

#include <limits>
Expand Down Expand Up @@ -94,4 +96,70 @@ TEST(HeapAllocatorTest, CanReturnMemoryToThePool) {
EXPECT_FALSE(mem.has_value());
}

TEST(StackAllocatorParamsTest, TrackingCanBeEnabled) {
stack_allocator allocator(stack_allocator::config{
.tracking_enabled = true,
});
EXPECT_TRUE(allocator.tracking_enabled());
allocator = stack_allocator(stack_allocator::config{
.tracking_enabled = false,
});
EXPECT_FALSE(allocator.tracking_enabled());
}

TEST(StackAllocatorTest, CanAllocateOne) {
stack_allocator allocator(stack_allocator::config{
.tracking_enabled = true,
});
size_t page_size = ::getpagesize();
auto stack = allocator.allocate(page_size * 4);
EXPECT_EQ(stack.size(), page_size * 4);
EXPECT_EQ(stack.bounds().top - stack.bounds().bottom, page_size * 4);
}

using ::testing::Optional;

TEST(StackAllocatorTest, CanLookupMemory) {
stack_allocator allocator(stack_allocator::config{
.tracking_enabled = true,
});
size_t page_size = ::getpagesize();
stack_memory stack = allocator.allocate(page_size * 4);
stack_bounds bounds = stack.bounds();

// NOLINTBEGIN(cppcoreguidelines-pro-bounds-pointer-arithmetic)
EXPECT_THAT(
allocator.stack_bounds_for_address(bounds.top), Optional(bounds));
EXPECT_THAT(
allocator.stack_bounds_for_address(bounds.top - page_size),
Optional(bounds));
EXPECT_THAT(
allocator.stack_bounds_for_address(bounds.bottom), Optional(bounds));
EXPECT_THAT(
allocator.stack_bounds_for_address(bounds.bottom + page_size),
Optional(bounds));

allocator.deallocate(std::move(stack));

EXPECT_EQ(allocator.stack_bounds_for_address(bounds.top), std::nullopt);
EXPECT_EQ(
allocator.stack_bounds_for_address(bounds.top - page_size), std::nullopt);
EXPECT_EQ(allocator.stack_bounds_for_address(bounds.bottom), std::nullopt);
EXPECT_EQ(
allocator.stack_bounds_for_address(bounds.bottom + page_size),
std::nullopt);
// NOLINTEND(cppcoreguidelines-pro-bounds-pointer-arithmetic)
}

TEST(StackAllocatorTest, CanReturnMemoryToThePool) {
stack_allocator allocator(stack_allocator::config{
.tracking_enabled = true,
});
size_t page_size = ::getpagesize();
auto stack = allocator.allocate(page_size * 4);
auto bounds = stack.bounds();
allocator.deallocate(std::move(stack));
EXPECT_EQ(stack.bounds(), bounds);
}

} // namespace wasm

0 comments on commit 821bd83

Please sign in to comment.