From c3d5994ec0c21166b93e802afdc838b97549b284 Mon Sep 17 00:00:00 2001 From: Tyler Rockwood Date: Fri, 13 Oct 2023 10:57:25 -0500 Subject: [PATCH] wasm: add testing for host stack usage 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 --- src/v/redpanda/application.cc | 3 + src/v/wasm/api.h | 11 ++ src/v/wasm/tests/wasm_fixture.cc | 12 ++ src/v/wasm/wasmtime.cc | 235 +++++++++++++++++++++++++------ 4 files changed, 218 insertions(+), 43 deletions(-) diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index 70a8a1c2197c4..67faa4f894d7f 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -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(); diff --git a/src/v/wasm/api.h b/src/v/wasm/api.h index 4ad439d86d90b..fd78507fee538 100644 --- a/src/v/wasm/api.h +++ b/src/v/wasm/api.h @@ -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; diff --git a/src/v/wasm/tests/wasm_fixture.cc b/src/v/wasm/tests/wasm_fixture.cc index 07ffe6f2eea0c..e064ba645a5df 100644 --- a/src/v/wasm/tests/wasm_fixture.cc +++ b/src/v/wasm/tests/wasm_fixture.cc @@ -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 = { diff --git a/src/v/wasm/wasmtime.cc b/src/v/wasm/wasmtime.cc index d7d040bdda258..2888c02198478 100644 --- a/src/v/wasm/wasmtime.cc +++ b/src/v/wasm/wasmtime.cc @@ -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" @@ -34,9 +35,12 @@ #include #include #include +#include #include #include +#include +#include #include #include #include @@ -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 struct deleter { @@ -347,7 +358,7 @@ class wasmtime_engine : public engine { uint64_t fuel = 0; wasmtime_context_fuel_remaining(ctx, &fuel); handle error( - wasmtime_context_add_fuel(ctx, wasmtime_fuel_amount - fuel)); + wasmtime_context_add_fuel(ctx, fuel_amount - fuel)); check_error(error.get()); } @@ -523,6 +534,17 @@ class wasmtime_engine : public engine { std::optional> _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* allocator; + + bool enabled() const { return allocator != nullptr; } +}; + template struct host_function; @@ -537,7 +559,10 @@ struct host_function { * 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_inputs; ffi::transform_types(ffi_inputs); std::vector ffi_outputs; @@ -548,6 +573,37 @@ struct host_function { handle functype{ wasm_functype_new(&inputs, &outputs)}; + if (ssc.enabled()) { + if constexpr (ss::is_future::value) { + handle 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 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::value) { handle error( wasmtime_linker_define_async_func( @@ -760,12 +816,96 @@ struct host_function { 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*>(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*>(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); @@ -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); @@ -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); @@ -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([] { @@ -979,33 +1117,44 @@ class wasmtime_runtime : public runtime { ss::future> make_factory( model::transform_metadata meta, iobuf buf, ss::logger* logger) override { auto preinitialized = ss::make_lw_shared(); - 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 error{ - wasmtime_module_new( - _engine.get(), b.data(), b.size(), &user_module_ptr)}; - check_error(error.get()); - handle user_module{ - user_module_ptr}; - wasm_log.info("Finished compiling wasm module {}", meta.name); - - handle 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 error{ + wasmtime_module_new( + _engine.get(), b.data(), b.size(), &user_module_ptr)}; + check_error(error.get()); + handle user_module{ + user_module_ptr}; + wasm_log.info("Finished compiling wasm module {}", meta.name); + + handle 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( _engine.get(), std::move(meta),