Skip to content

Commit

Permalink
test: assert that WASM allocations can fail without crashing
Browse files Browse the repository at this point in the history
The main source of big allocations in the WASM UDF implementation
is the WASM Linear Memory. We do not want Scylla to crash even if
a memory allocation for the WASM Memory fails, so we assert that
an exception is thrown instead.

The wasmtime runtime does not actually fail on an allocation failure
(assuming the memory allocator does not abort and returns nullptr
instead - which our seastar allocator does). What happens then
depends on the failed allocation handling of the code that was
compiled to WASM. If the original code threw an exception or aborted,
the resulting WASM code will trap. To make sure that we can handle
the trap, we need to allow wasmtime to handle SIGILL signals, because
that what is used to carry information about WASM traps.

The new test uses a special WASM Memory allocator that fails after
n allocations, and the allocations include both memory growth
instructions in WASM, as well as growing memory manually using the
wasmtime API.

Signed-off-by: Wojciech Mitros <wojciech.mitros@scylladb.com>
  • Loading branch information
wmitros committed Jan 6, 2023
1 parent f05d612 commit 996a942
Show file tree
Hide file tree
Showing 4 changed files with 230 additions and 0 deletions.
1 change: 1 addition & 0 deletions configure.py
Original file line number Diff line number Diff line change
Expand Up @@ -484,6 +484,7 @@ def find_headers(repodir, excluded_dirs):
'test/boost/virtual_table_mutation_source_test',
'test/boost/virtual_table_test',
'test/boost/wasm_test',
'test/boost/wasm_alloc_test',
'test/boost/bptree_test',
'test/boost/btree_test',
'test/boost/radix_tree_test',
Expand Down
26 changes: 26 additions & 0 deletions rust/wasmtime_bindings/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,8 @@ use core::task::Poll;
use futures::future::{BoxFuture, Future};
mod memory_creator;
use memory_creator::ScyllaMemoryCreator;
mod test_memory_creator;
use test_memory_creator::TestScyllaMemoryCreator;

#[cxx::bridge(namespace = "wasmtime")]
mod ffi {
Expand Down Expand Up @@ -61,6 +63,7 @@ mod ffi {

type Engine;
fn create_engine(max_size: u32) -> Result<Box<Engine>>;
fn create_test_engine(max_size: u32, fail_after: usize) -> Result<Box<Engine>>;

type Func;
fn create_func(
Expand Down Expand Up @@ -276,6 +279,29 @@ fn create_engine(max_size: u32) -> Result<Box<Engine>> {
}))
}

fn create_test_engine(max_size: u32, fail_after: usize) -> Result<Box<Engine>> {
let mut config = wasmtime::Config::new();
config.async_support(true);
config.consume_fuel(true);
// ScyllaMemoryCreator uses malloc (from seastar) to allocate linear memory
config.with_host_memory(std::sync::Arc::new(TestScyllaMemoryCreator::new(
max_size as usize,
fail_after,
)));
// The following configuration settings make wasmtime allocate only as much memory as it needs
config.static_memory_maximum_size(0);
config.dynamic_memory_reserved_for_growth(0);
config.dynamic_memory_guard_size(0);
config.max_wasm_stack(128 * 1024);
config.async_stack_size(256 * 1024);

let engine =
wasmtime::Engine::new(&config).map_err(|e| anyhow!("Failed to create engine: {:?}", e))?;
Ok(Box::new(Engine {
wasmtime_engine: engine,
}))
}

pub struct Func {
wasmtime_func: wasmtime::Func,
}
Expand Down
100 changes: 100 additions & 0 deletions rust/wasmtime_bindings/src/test_memory_creator.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
/*
* Copyright (C) 2022-present ScyllaDB
*/

/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/

use crate::memory_creator::ScyllaMemoryCreator;
use anyhow::{anyhow, Result};
use std::ptr;
use wasmtime::LinearMemory;

// The implementation below adds a conditional failure of an allocation
// to the original grow_to().
// It is used in testing instead of an alloc_failure_injector
// because we only assert that the big allocations of WASM pages
// can fail without aborting (as the allocations more likely to fail)
// TODO: remove this code when avoiding crashes on allocation failures
// becomes possible in Rust
pub struct TestScyllaLinearMemory {
memory: Box<dyn wasmtime::LinearMemory>,
allocs: usize,
fail_after: usize,
}

unsafe impl LinearMemory for TestScyllaLinearMemory {
fn byte_size(&self) -> usize {
self.memory.byte_size()
}
fn maximum_byte_size(&self) -> Option<usize> {
self.memory.maximum_byte_size()
}
fn grow_to(&mut self, new_size: usize) -> Result<()> {
let old_ptr = self.memory.as_ptr();
// This may change the value of self.memory.as_ptr() even if we decide
// that it should have failed, but we didn't actually guarantee that
// it would be unchanged in that case, and we're only using this
// in a test case where we finish the test immediately after the failure
self.memory.grow_to(new_size)?;
let new_ptr = self.memory.as_ptr();
if old_ptr != new_ptr && new_ptr != ptr::null_mut() {
// We needed to actually perform an allocation, so we could have failed
let failed = self.allocs == self.fail_after;
self.allocs += 1;
if failed {
return Err(anyhow!("Failed to grow WASM memory: allocation error"));
}
}
Ok(())
}
fn as_ptr(&self) -> *mut u8 {
self.memory.as_ptr()
}
}

// In order to use the Seastar memory allocator instead of mmap,
// create our own MemoryCreator which directly calls aligned_alloc
// and free, both of which came from Seastar
pub struct TestScyllaMemoryCreator {
memory_creator: ScyllaMemoryCreator,
fail_after: usize,
}

impl TestScyllaMemoryCreator {
pub fn new(max_scylla_size: usize, fail_after: usize) -> Self {
TestScyllaMemoryCreator {
memory_creator: ScyllaMemoryCreator::new(max_scylla_size),
fail_after: fail_after,
}
}
}

unsafe impl wasmtime::MemoryCreator for TestScyllaMemoryCreator {
fn new_memory(
&self,
ty: wasmtime::MemoryType,
minimum: usize,
maximum: Option<usize>,
reserved_size_in_bytes: Option<usize>,
guard_size_in_bytes: usize,
) -> Result<Box<dyn wasmtime::LinearMemory>, String> {
// Create standard linear memory with initial size 0 to avoid the initial allocation
let memory = self.memory_creator.new_memory(
ty,
0,
maximum,
reserved_size_in_bytes,
guard_size_in_bytes,
)?;
let mut mem = TestScyllaLinearMemory {
memory: memory,
allocs: 0,
fail_after: self.fail_after,
};
// Perform the initial allocation using grow_to of the test memory
mem.grow_to(minimum).map_err(|e| e.to_string())?;
Ok(Box::new(mem))
}
}
103 changes: 103 additions & 0 deletions test/boost/wasm_alloc_test.cc
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* Copyright (C) 2022-present ScyllaDB
*/

/*
* SPDX-License-Identifier: AGPL-3.0-or-later
*/

#include "lang/wasm.hh"
#include "lang/wasm_instance_cache.hh"
#include "rust/wasmtime_bindings.hh"
#include "seastar/core/reactor.hh"
#include <seastar/testing/test_case.hh>
#include <seastar/core/coroutine.hh>

// This test file can contain only a single test case which uses the wasmtime
// runtime. This is because the wasmtime runtime registers a signal handler
// only once in the lifetime of the process, while each boost test case
// registers its own signal handler.
// We need the signal handler registered by the wasm runtime not to be
// overwritten by the signal handlers registered by the boost test cases
// because the wasm runtime uses the SIGILL signal to detect illegal
// instructions in the wasm code - in particular, an illegal instruction is
// executed when an memory allocation fails in the WASM program.

static const char* grow_return = R"(
(module
(type (;0;) (func (param i64) (result i64)))
(func (;0;) (type 0) (param i64) (result i64)
i32.const 1
memory.grow
i32.const -1
i32.eq
if ;; label = @1
unreachable
end
i64.const 10)
(memory (;0;) 2)
(global (;0;) i32 (i32.const 1024))
(export "memory" (memory 0))
(export "grow_return" (func 0))
(export "_scylla_abi" (global 0))
(data (;0;) (i32.const 1024) "\01"))
)";

// This test injects allocation failure at every wasm memory growth
// and checks that the wasm function returns an error and does not abort.
// This would normally be done using an alloc_failure_injector, but
// we can't do that here because this test case uses Rust code, which
// may abort on allocation failure in general.
// TODO: test all allocation points when Rust enables OOM handling.
SEASTAR_TEST_CASE(test_allocation_failures) {
// First we register an empty signal handler to remove SIGILL from the set
// of signals blocked by pthread_sigmask. This signal is excluded from the
// mask in the app_template but not in the test_runner, which is why we need
// to do this in the test case and not in the main Scylla code.
// Other signals that may need to be unblocked in future test cases
// are SIGFPE and SIGBUS.
engine().handle_signal(SIGILL, [] {});
int errors_during_compilation = 0;
int errors_during_execution = 0;
for (size_t fail_after = 0;;fail_after++) {
auto wasm_engine = wasmtime::create_test_engine(1024 * 1024, fail_after);
auto wasm_cache = std::make_unique<wasm::instance_cache>(100 * 1024 * 1024, 1024 * 1024, std::chrono::seconds(1));
auto wasm_ctx = wasm::context(*wasm_engine, "grow_return", wasm_cache.get(), 1000, 1000000000);
try {
// Function that ignores the input, grows its memory by 1 page, and returns 10
wasm::precompile(wasm_ctx, {}, grow_return);
wasm_ctx.module.value()->compile(*wasm_engine);
} catch (const wasm::exception& e) {
errors_during_compilation++;
continue;
}
try {
auto store = wasmtime::create_store(wasm_ctx.engine_ptr, wasm_ctx.total_fuel, wasm_ctx.yield_fuel);
auto instance = wasmtime::create_instance(wasm_ctx.engine_ptr, **wasm_ctx.module, *store);
auto func = wasmtime::create_func(*instance, *store, wasm_ctx.function_name);
auto memory = wasmtime::get_memory(*instance, *store);

size_t mem_size = memory->grow(*store, 1) * 64 * 1024;
int32_t serialized_size = -1;
data_value arg{(int64_t)10};
auto arg_serialized = serialized(arg);

int64_t arg_combined = ((int64_t)serialized_size << 32) | mem_size;
auto argv = wasmtime::get_val_vec();
argv->push_i64(arg_combined);
auto rets = wasmtime::get_val_vec();
rets->push_i32(0);
auto fut = wasmtime::get_func_future(*store, *func, *argv, *rets);
while (!fut->resume());

BOOST_CHECK_EQUAL(rets->pop_val()->i64(), 10);
} catch (const rust::Error& e) {
errors_during_execution++;
continue;
}
// Check that we can actually throw errors both during compilation and during execution
BOOST_CHECK(errors_during_compilation > 0);
BOOST_CHECK(errors_during_execution > 0);
co_return;
}
}

0 comments on commit 996a942

Please sign in to comment.