Skip to content

Commit

Permalink
wasm: add testing for host stack usage
Browse files Browse the repository at this point in the history
We want to ensure that our host functions don't blow the stack when
executing if a guest uses up a bunch of stack space within the Wasm VM.

We do this in our tests by allocating a variable amount on the stack so
that it looks to every host function that the guest has used the maximum
amount of the stack.

Additionally, we only run in this "strict stack mode" in release tests.
We don't want to do this in production builds, and in debug builds ASAN
throws a fit when doing this. Presumably ASAN complains because it
doesn't know we've switched the stack as that happens in Rust land which
isn't instrumented with ASAN checks. Honestly this is fine as stack
usage in debug mode is wildly different than release mode and we're
realistically only using debug mode internally in non production usage.

Signed-off-by: Tyler Rockwood <rockwood@redpanda.com>
  • Loading branch information
rockwotj committed Oct 13, 2023
1 parent 35d12ea commit c3d5994
Show file tree
Hide file tree
Showing 4 changed files with 218 additions and 43 deletions.
3 changes: 3 additions & 0 deletions src/v/redpanda/application.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2243,6 +2243,9 @@ void application::wire_up_and_start(::stop_signal& app_signal, bool test_mode) {
.per_core_pool_size_bytes = cluster.wasm_per_core_memory_reservation.value(),
.per_engine_memory_limit = cluster.wasm_per_function_memory_limit.value(),
},
.stack_memory = {
.debug_host_stack_usage = false,
},
};
_wasm_runtime->start(config).get();
_transform_service.invoke_on_all(&transform::service::start).get();
Expand Down
11 changes: 11 additions & 0 deletions src/v/wasm/api.h
Original file line number Diff line number Diff line change
Expand Up @@ -88,10 +88,21 @@ class runtime {

struct config {
struct heap_memory {
// per core how many bytes to reserve
size_t per_core_pool_size_bytes;
// per engine the max amount of memory
size_t per_engine_memory_limit;
};
heap_memory heap_memory;
struct stack_memory {
// Enable debugging of host function's stack usage.
// These host functions are called on the VM stack, so we need to
// ensure that we aren't susceptible to guests that use most of the
// stack and then our host function overflowing the rest of the
// stack.
bool debug_host_stack_usage;
};
stack_memory stack_memory;
};

virtual ss::future<> start(config) = 0;
Expand Down
12 changes: 12 additions & 0 deletions src/v/wasm/tests/wasm_fixture.cc
Original file line number Diff line number Diff line change
Expand Up @@ -138,6 +138,18 @@ void WasmTestFixture::SetUp() {
.per_core_pool_size_bytes = MAX_MEMORY * 4,
.per_engine_memory_limit = MAX_MEMORY,
},
.stack_memory = {
#ifdef NDEBUG
// Only turn this on if ASAN is off.
// With ASAN on, we get issues because we haven't told
// ASAN that the stack has switched (as this happens within
// wasmtime and we don't have the ability to instrument rust
// with ASAN checks).
.debug_host_stack_usage = true,
#else
.debug_host_stack_usage = false,
#endif
},
};
_runtime->start(wasm_runtime_config).get();
_meta = {
Expand Down
235 changes: 192 additions & 43 deletions src/v/wasm/wasmtime.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
#include "model/transform.h"
#include "ssx/thread_worker.h"
#include "storage/parser_utils.h"
#include "utils/human.h"
#include "utils/type_traits.h"
#include "vassert.h"
#include "wasm/allocator.h"
Expand All @@ -34,9 +35,12 @@
#include <seastar/coroutine/as_future.hh>
#include <seastar/coroutine/maybe_yield.hh>
#include <seastar/util/backtrace.hh>
#include <seastar/util/bool_class.hh>
#include <seastar/util/defer.hh>
#include <seastar/util/noncopyable_function.hh>
#include <seastar/util/optimized_optional.hh>

#include <alloca.h>
#include <cmath>
#include <csignal>
#include <memory>
Expand All @@ -49,8 +53,15 @@
namespace wasm::wasmtime {

namespace {

constexpr uint64_t wasmtime_fuel_amount = 1'000'000'000;
constexpr size_t vm_stack_size = 128_KiB;
constexpr size_t max_vm_guest_stack_usage = 64_KiB;
// We allow for half the stack for host functions,
// plus a little wiggle room to not get too close
// to the guard page.
constexpr size_t max_host_function_stack_usage = vm_stack_size
- max_vm_guest_stack_usage
- 4_KiB;
constexpr uint64_t fuel_amount = 1'000'000'000;

template<typename T, auto fn>
struct deleter {
Expand Down Expand Up @@ -347,7 +358,7 @@ class wasmtime_engine : public engine {
uint64_t fuel = 0;
wasmtime_context_fuel_remaining(ctx, &fuel);
handle<wasmtime_error_t, wasmtime_error_delete> error(
wasmtime_context_add_fuel(ctx, wasmtime_fuel_amount - fuel));
wasmtime_context_add_fuel(ctx, fuel_amount - fuel));
check_error(error.get());
}

Expand Down Expand Up @@ -523,6 +534,17 @@ class wasmtime_engine : public engine {
std::optional<ss::future<>> _pending_host_function;
};

// If strict stack checking is configured
//
// Strict stack checking ensures that our host functions don't use too much
// stack, even if in our tests they leave plenty of extra stack space (not
// all VM guest programs will be so nice).
struct strict_stack_config {
ss::sharded<stack_allocator>* allocator;

bool enabled() const { return allocator != nullptr; }
};

template<auto value>
struct host_function;

Expand All @@ -537,7 +559,10 @@ struct host_function<module_func> {
* Register a host function such that it can be invoked from the Wasmtime
* VM.
*/
static void reg(wasmtime_linker_t* linker, std::string_view function_name) {
static void reg(
wasmtime_linker_t* linker,
std::string_view function_name,
const strict_stack_config& ssc) {
std::vector<ffi::val_type> ffi_inputs;
ffi::transform_types<ArgTypes...>(ffi_inputs);
std::vector<ffi::val_type> ffi_outputs;
Expand All @@ -548,6 +573,37 @@ struct host_function<module_func> {
handle<wasm_functype_t, wasm_functype_delete> functype{
wasm_functype_new(&inputs, &outputs)};

if (ssc.enabled()) {
if constexpr (ss::is_future<ReturnType>::value) {
handle<wasmtime_error_t, wasmtime_error_delete> error(
wasmtime_linker_define_async_func(
linker,
Module::name.data(),
Module::name.size(),
function_name.data(),
function_name.size(),
functype.get(),
&invoke_async_host_fn_with_strict_stack_checking,
/*data=*/ssc.allocator,
/*finalizer=*/nullptr));
check_error(error.get());
} else {
handle<wasmtime_error_t, wasmtime_error_delete> error(
wasmtime_linker_define_func(
linker,
Module::name.data(),
Module::name.size(),
function_name.data(),
function_name.size(),
functype.get(),
&invoke_sync_host_fn_with_strict_stack_checking,
/*data=*/ssc.allocator,
/*finalizer=*/nullptr));
check_error(error.get());
}
return;
}

if constexpr (ss::is_future<ReturnType>::value) {
handle<wasmtime_error_t, wasmtime_error_delete> error(
wasmtime_linker_define_async_func(
Expand Down Expand Up @@ -760,12 +816,96 @@ struct host_function<module_func> {
mem->set_underlying_memory(&extern_item.of.memory);
return nullptr;
}

static wasm_trap_t* invoke_sync_host_fn_with_strict_stack_checking(
void* env,
wasmtime_caller_t* caller,
const wasmtime_val_t* args,
size_t nargs,
wasmtime_val_t* results,
size_t nresults) {
auto* allocator = static_cast<ss::sharded<stack_allocator>*>(env);
uint8_t dummy_stack_var = 0;
auto bounds = allocator->local().stack_bounds_for_address(
&dummy_stack_var);
if (!bounds) {
vlog(wasm_log.warn, "can't find vm stack!");
return invoke_sync_host_fn(
nullptr, caller, args, nargs, results, nresults);
}
// Ensure that we only use `max_host_function_stack_usage` by
// allocing enough to call the host function with only that much
// stack space left.
std::ptrdiff_t stack_left = (&dummy_stack_var) - bounds->bottom;
void* stack_ptr = ::alloca(stack_left - max_host_function_stack_usage);
// Prevent the alloca from being optimized away by logging the result.
vlog(
wasm_log.trace,
"alloca-ing {}, stack left: {}, alloca address: {}, stack bounds: {}",
human::bytes(stack_left - max_host_function_stack_usage),
human::bytes(stack_left),
stack_ptr,
bounds);
return invoke_sync_host_fn(
nullptr, caller, args, nargs, results, nresults);
}

static void invoke_async_host_fn_with_strict_stack_checking(
void* env,
wasmtime_caller_t* caller,
const wasmtime_val_t* args,
size_t nargs,
wasmtime_val_t* results,
size_t nresults,
wasm_trap_t** trap_ret,
wasmtime_async_continuation_t* continuation) {
auto* allocator = static_cast<ss::sharded<stack_allocator>*>(env);
uint8_t dummy_stack_var = 0;
auto bounds = allocator->local().stack_bounds_for_address(
&dummy_stack_var);
if (!bounds) {
vlog(wasm_log.warn, "can't find vm stack!");
invoke_async_host_fn(
nullptr,
caller,
args,
nargs,
results,
nresults,
trap_ret,
continuation);
return;
}
// Ensure that we only use `max_host_function_stack_usage` by
// allocing enough to call the host function with only that much
// stack space left.
std::ptrdiff_t stack_left = (&dummy_stack_var) - bounds->bottom;
void* stack_ptr = ::alloca(stack_left - max_host_function_stack_usage);
// Prevent the alloca from being optimized away by logging the result.
vlog(
wasm_log.trace,
"alloca-ing {}, stack left: {}, alloca address: {}, stack bounds: {}",
human::bytes(stack_left - max_host_function_stack_usage),
human::bytes(stack_left),
stack_ptr,
bounds);
invoke_async_host_fn(
nullptr,
caller,
args,
nargs,
results,
nresults,
trap_ret,
continuation);
}
};

void register_wasi_module(wasmtime_linker_t* linker) {
void register_wasi_module(
wasmtime_linker_t* linker, const strict_stack_config& ssc) {
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
#define REG_HOST_FN(name) \
host_function<&wasi::preview1_module::name>::reg(linker, #name)
host_function<&wasi::preview1_module::name>::reg(linker, #name, ssc)
REG_HOST_FN(args_get);
REG_HOST_FN(args_sizes_get);
REG_HOST_FN(environ_get);
Expand Down Expand Up @@ -813,19 +953,21 @@ void register_wasi_module(wasmtime_linker_t* linker) {
#undef REG_HOST_FN
}

void register_transform_module(wasmtime_linker_t* linker) {
void register_transform_module(
wasmtime_linker_t* linker, const strict_stack_config& ssc) {
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
#define REG_HOST_FN(name) \
host_function<&transform_module::name>::reg(linker, #name)
host_function<&transform_module::name>::reg(linker, #name, ssc)
REG_HOST_FN(read_batch_header);
REG_HOST_FN(read_record);
REG_HOST_FN(write_record);
#undef REG_HOST_FN
}
void register_sr_module(wasmtime_linker_t* linker) {
void register_sr_module(
wasmtime_linker_t* linker, const strict_stack_config& ssc) {
// NOLINTNEXTLINE(cppcoreguidelines-macro-usage)
#define REG_HOST_FN(name) \
host_function<&schema_registry_module::name>::reg(linker, #name)
host_function<&schema_registry_module::name>::reg(linker, #name, ssc)
REG_HOST_FN(get_schema_definition);
REG_HOST_FN(get_schema_definition_len);
REG_HOST_FN(get_subject_schema);
Expand Down Expand Up @@ -889,15 +1031,11 @@ class wasmtime_runtime : public runtime {
wasmtime_config_async_support_set(config, true);
// Set max stack size to generally be as big as a contiguous memory
// region we're willing to allocate in Redpanda.
//
// NOLINTNEXTLINE(cppcoreguidelines-avoid-magic-numbers)
wasmtime_config_async_stack_size_set(config, 128_KiB);
wasmtime_config_async_stack_size_set(config, vm_stack_size);
// The stack size needs to be less than the async stack size, and
// whatever is difference between the two is how much host functions can
// get, make sure to leave our own functions some room to execute.
//
// NOLINTNEXTLINE(cppcoreguidelines-avoid-magic-numbers)
wasmtime_config_max_wasm_stack_set(config, 64_KiB);
wasmtime_config_max_wasm_stack_set(config, max_vm_guest_stack_usage);
// This disables static memory, see:
// https://docs.wasmtime.dev/contributing-architecture.html#linear-memory
wasmtime_config_static_memory_maximum_size_set(config, 0_KiB);
Expand Down Expand Up @@ -955,7 +1093,7 @@ class wasmtime_runtime : public runtime {
.num_heaps = num_heaps,
});
co_await _stack_allocator.start(stack_allocator::config{
.tracking_enabled = false,
.tracking_enabled = c.stack_memory.debug_host_stack_usage,
});
co_await _alien_thread.start({.name = "wasm"});
co_await ss::smp::invoke_on_all([] {
Expand All @@ -979,33 +1117,44 @@ class wasmtime_runtime : public runtime {
ss::future<ss::shared_ptr<factory>> make_factory(
model::transform_metadata meta, iobuf buf, ss::logger* logger) override {
auto preinitialized = ss::make_lw_shared<preinitialized_instance>();
co_await _alien_thread.submit([this, &meta, &buf, &preinitialized] {
vlog(wasm_log.debug, "compiling wasm module {}", meta.name);
// This can be a large contiguous allocation, however it happens
// on an alien thread so it bypasses the seastar allocator.
bytes b = iobuf_to_bytes(buf);
wasmtime_module_t* user_module_ptr = nullptr;
handle<wasmtime_error_t, wasmtime_error_delete> error{
wasmtime_module_new(
_engine.get(), b.data(), b.size(), &user_module_ptr)};
check_error(error.get());
handle<wasmtime_module_t, wasmtime_module_delete> user_module{
user_module_ptr};
wasm_log.info("Finished compiling wasm module {}", meta.name);

handle<wasmtime_linker_t, wasmtime_linker_delete> linker{
wasmtime_linker_new(_engine.get())};

register_transform_module(linker.get());
register_sr_module(linker.get());
register_wasi_module(linker.get());

wasmtime_instance_pre_t* preinitialized_ptr = nullptr;
error.reset(wasmtime_linker_instantiate_pre(
linker.get(), user_module.get(), &preinitialized_ptr));
preinitialized->underlying.reset(preinitialized_ptr);
check_error(error.get());
});
// Enable strict stack checking only if tracking is enabled.
//
// (strict stack checking ensures our host functions don't use too much
// stack space, even when guests leave extra stack).
strict_stack_config ssc = {
.allocator = _stack_allocator.local().tracking_enabled()
? &_stack_allocator
: nullptr,
};
co_await _alien_thread.submit(
[this, &meta, &buf, &preinitialized, &ssc] {
vlog(wasm_log.debug, "compiling wasm module {}", meta.name);
// This can be a large contiguous allocation, however it happens
// on an alien thread so it bypasses the seastar allocator.
bytes b = iobuf_to_bytes(buf);
wasmtime_module_t* user_module_ptr = nullptr;
handle<wasmtime_error_t, wasmtime_error_delete> error{
wasmtime_module_new(
_engine.get(), b.data(), b.size(), &user_module_ptr)};
check_error(error.get());
handle<wasmtime_module_t, wasmtime_module_delete> user_module{
user_module_ptr};
wasm_log.info("Finished compiling wasm module {}", meta.name);

handle<wasmtime_linker_t, wasmtime_linker_delete> linker{
wasmtime_linker_new(_engine.get())};

register_transform_module(linker.get(), ssc);
register_sr_module(linker.get(), ssc);
register_wasi_module(linker.get(), ssc);

wasmtime_instance_pre_t* preinitialized_ptr = nullptr;
error.reset(wasmtime_linker_instantiate_pre(
linker.get(), user_module.get(), &preinitialized_ptr));
preinitialized->underlying.reset(preinitialized_ptr);
check_error(error.get());
});
co_return ss::make_shared<wasmtime_engine_factory>(
_engine.get(),
std::move(meta),
Expand Down

0 comments on commit c3d5994

Please sign in to comment.