merge vm_refactor_trap_handling into vm_refactor

This commit is contained in:
Lachlan Sneff
2019-01-21 13:53:33 -08:00
19 changed files with 563 additions and 194 deletions

View File

@ -0,0 +1,138 @@
mod recovery;
mod sighandler;
pub use self::recovery::HandlerData;
use crate::call::recovery::call_protected;
use hashbrown::HashSet;
use libffi::high::{arg as libffi_arg, call as libffi_call, CodePtr};
use std::iter;
use wasmer_runtime::{
backend::{ProtectedCaller, Token},
error::RuntimeResult,
export::Context,
module::{ExportIndex, ModuleInner},
types::{FuncIndex, FuncSig, LocalOrImport, Type, Value},
vm::{self, ImportBacking},
};
pub struct Caller {
func_export_set: HashSet<FuncIndex>,
handler_data: HandlerData,
}
impl Caller {
pub fn new(module: &ModuleInner, handler_data: HandlerData) -> Self {
let mut func_export_set = HashSet::new();
for export_index in module.exports.values() {
if let ExportIndex::Func(func_index) = export_index {
func_export_set.insert(*func_index);
}
}
if let Some(start_func_index) = module.start_func {
func_export_set.insert(start_func_index);
}
Self {
func_export_set,
handler_data,
}
}
}
impl ProtectedCaller for Caller {
fn call(
&self,
module: &ModuleInner,
func_index: FuncIndex,
params: &[Value],
returns: &mut [Value],
import_backing: &ImportBacking,
vmctx: *mut vm::Ctx,
_: Token,
) -> RuntimeResult<()> {
let (func_ptr, ctx, signature) = get_func_from_index(&module, import_backing, func_index);
let vmctx_ptr = match ctx {
Context::External(external_vmctx) => external_vmctx,
Context::Internal => vmctx,
};
assert!(self.func_export_set.contains(&func_index));
assert!(
returns.len() == signature.returns.len() && signature.returns.len() <= 1,
"multi-value returns not yet supported"
);
assert!(signature.check_sig(params), "incorrect signature");
let libffi_args: Vec<_> = params
.iter()
.map(|val| match val {
Value::I32(ref x) => libffi_arg(x),
Value::I64(ref x) => libffi_arg(x),
Value::F32(ref x) => libffi_arg(x),
Value::F64(ref x) => libffi_arg(x),
})
.chain(iter::once(libffi_arg(&vmctx_ptr)))
.collect();
let code_ptr = CodePtr::from_ptr(func_ptr as _);
call_protected(&self.handler_data, || {
// Only supports zero or one return values for now.
// To support multiple returns, we will have to
// generate trampolines instead of using libffi.
match signature.returns.first() {
Some(ty) => {
let val = match ty {
Type::I32 => Value::I32(unsafe { libffi_call(code_ptr, &libffi_args) }),
Type::I64 => Value::I64(unsafe { libffi_call(code_ptr, &libffi_args) }),
Type::F32 => Value::F32(unsafe { libffi_call(code_ptr, &libffi_args) }),
Type::F64 => Value::F64(unsafe { libffi_call(code_ptr, &libffi_args) }),
};
returns[0] = val;
}
// call with no returns
None => unsafe {
libffi_call::<()>(code_ptr, &libffi_args);
},
}
})
}
}
fn get_func_from_index<'a>(
module: &'a ModuleInner,
import_backing: &ImportBacking,
func_index: FuncIndex,
) -> (*const vm::Func, Context, &'a FuncSig) {
let sig_index = *module
.func_assoc
.get(func_index)
.expect("broken invariant, incorrect func index");
let (func_ptr, ctx) = match func_index.local_or_import(module) {
LocalOrImport::Local(local_func_index) => (
module
.func_resolver
.get(&module, local_func_index)
.expect("broken invariant, func resolver not synced with module.exports")
.cast()
.as_ptr() as *const _,
Context::Internal,
),
LocalOrImport::Import(imported_func_index) => {
let imported_func = import_backing.imported_func(imported_func_index);
(
imported_func.func as *const _,
Context::External(imported_func.vmctx),
)
}
};
let signature = module.sig_registry.lookup_func_sig(sig_index);
(func_ptr, ctx, signature)
}

View File

@ -0,0 +1,189 @@
//! When a WebAssembly module triggers any traps, we perform recovery here.
//!
//! This module uses TLS (thread-local storage) to track recovery information. Since the four signals we're handling
//! are very special, the async signal unsafety of Rust's TLS implementation generally does not affect the correctness here
//! unless you have memory unsafety elsewhere in your code.
use crate::call::sighandler::install_sighandler;
use crate::relocation::{TrapData, TrapSink};
use cranelift_codegen::ir::TrapCode;
use nix::libc::{c_void, siginfo_t};
use nix::sys::signal::{Signal, SIGBUS, SIGFPE, SIGILL, SIGSEGV};
use std::cell::{Cell, UnsafeCell};
use std::ptr;
use std::sync::Once;
use wasmer_runtime::{
error::{RuntimeError, RuntimeResult},
structures::TypedIndex,
types::{MemoryIndex, TableIndex},
};
extern "C" {
pub fn setjmp(env: *mut ::nix::libc::c_void) -> ::nix::libc::c_int;
fn longjmp(env: *mut ::nix::libc::c_void, val: ::nix::libc::c_int) -> !;
}
const SETJMP_BUFFER_LEN: usize = 27;
pub static SIGHANDLER_INIT: Once = Once::new();
thread_local! {
pub static SETJMP_BUFFER: UnsafeCell<[::nix::libc::c_int; SETJMP_BUFFER_LEN]> = UnsafeCell::new([0; SETJMP_BUFFER_LEN]);
pub static CAUGHT_ADDRESSES: Cell<(*const c_void, *const c_void)> = Cell::new((ptr::null(), ptr::null()));
pub static CURRENT_EXECUTABLE_BUFFER: Cell<*const c_void> = Cell::new(ptr::null());
}
pub struct HandlerData {
trap_data: TrapSink,
buffer_ptr: *const c_void,
buffer_size: usize,
}
impl HandlerData {
pub fn new(trap_data: TrapSink, buffer_ptr: *const c_void, buffer_size: usize) -> Self {
Self {
trap_data,
buffer_ptr,
buffer_size,
}
}
pub fn lookup(&self, ip: *const c_void) -> Option<TrapData> {
let ip = ip as usize;
let buffer_ptr = self.buffer_ptr as usize;
if buffer_ptr <= ip && ip < buffer_ptr + self.buffer_size {
let offset = ip - buffer_ptr;
self.trap_data.lookup(offset)
} else {
None
}
}
}
pub fn call_protected<T>(handler_data: &HandlerData, f: impl FnOnce() -> T) -> RuntimeResult<T> {
unsafe {
let jmp_buf = SETJMP_BUFFER.with(|buf| buf.get());
let prev_jmp_buf = *jmp_buf;
SIGHANDLER_INIT.call_once(|| {
install_sighandler();
});
let signum = setjmp(jmp_buf as *mut ::nix::libc::c_void);
if signum != 0 {
*jmp_buf = prev_jmp_buf;
let (faulting_addr, _) = CAUGHT_ADDRESSES.with(|cell| cell.get());
if let Some(TrapData {
trapcode,
srcloc: _,
}) = handler_data.lookup(faulting_addr)
{
Err(match Signal::from_c_int(signum) {
Ok(SIGILL) => match trapcode {
TrapCode::BadSignature => RuntimeError::IndirectCallSignature {
table: TableIndex::new(0),
},
TrapCode::IndirectCallToNull => RuntimeError::IndirectCallToNull {
table: TableIndex::new(0),
},
TrapCode::HeapOutOfBounds => {
let addr =
(faulting_addr as usize) - (handler_data.buffer_ptr as usize);
if addr <= handler_data.buffer_size {
// in the memory
RuntimeError::OutOfBoundsAccess {
memory: MemoryIndex::new(0),
addr: addr as u32,
}
} else {
// if there's an invalid access outside of the memory, including guard pages
// just kill the process.
panic!("invalid memory access, way out of bounds")
}
}
TrapCode::TableOutOfBounds => RuntimeError::TableOutOfBounds {
table: TableIndex::new(0),
},
_ => RuntimeError::Unknown {
msg: "unknown trap".to_string(),
},
},
Ok(SIGSEGV) | Ok(SIGBUS) => {
let addr = (faulting_addr as usize) - (handler_data.buffer_ptr as usize);
if addr <= handler_data.buffer_size {
// in the memory
RuntimeError::OutOfBoundsAccess {
memory: MemoryIndex::new(0),
addr: addr as u32,
}
} else {
// if there's an invalid access outside of the memory, including guard pages
// just kill the process.
panic!("invalid memory access, way out of bounds")
}
}
Ok(SIGFPE) => RuntimeError::IllegalArithmeticOperation,
_ => unimplemented!(),
}
.into())
} else {
let signal = match Signal::from_c_int(signum) {
Ok(SIGFPE) => "floating-point exception",
Ok(SIGILL) => "illegal instruction",
Ok(SIGSEGV) => "segmentation violation",
Ok(SIGBUS) => "bus error",
Err(_) => "error while getting the Signal",
_ => "unkown trapped signal",
};
// When the trap-handler is fully implemented, this will return more information.
Err(RuntimeError::Unknown {
msg: format!("trap at {:p} - {}", faulting_addr, signal),
}
.into())
}
} else {
let ret = f(); // TODO: Switch stack?
*jmp_buf = prev_jmp_buf;
Ok(ret)
}
}
}
/// Unwinds to last protected_call.
pub unsafe fn do_unwind(signum: i32, siginfo: *mut siginfo_t, ucontext: *const c_void) -> ! {
// Since do_unwind is only expected to get called from WebAssembly code which doesn't hold any host resources (locks etc.)
// itself, accessing TLS here is safe. In case any other code calls this, it often indicates a memory safety bug and you should
// temporarily disable the signal handlers to debug it.
let jmp_buf = SETJMP_BUFFER.with(|buf| buf.get());
if *jmp_buf == [0; SETJMP_BUFFER_LEN] {
::std::process::abort();
}
CAUGHT_ADDRESSES.with(|cell| cell.set(get_faulting_addr_and_ip(siginfo, ucontext)));
longjmp(jmp_buf as *mut ::nix::libc::c_void, signum)
}
#[cfg(all(target_os = "linux", target_arch = "x86_64"))]
unsafe fn get_faulting_addr_and_ip(
siginfo: *mut siginfo_t,
_ucontext: *const c_void,
) -> (*const c_void, *const c_void) {
(ptr::null(), ptr::null())
}
#[cfg(all(target_os = "macos", target_arch = "x86_64"))]
unsafe fn get_faulting_addr_and_ip(
siginfo: *mut siginfo_t,
_ucontext: *const c_void,
) -> (*const c_void, *const c_void) {
((*siginfo).si_addr, ptr::null())
}
#[cfg(not(any(
all(target_os = "macos", target_arch = "x86_64"),
all(target_os = "linux", target_arch = "x86_64"),
)))]
compile_error!("This crate doesn't yet support compiling on operating systems other than linux and macos and architectures other than x86_64");

View File

@ -0,0 +1,31 @@
//! Installing signal handlers allows us to handle traps and out-of-bounds memory
//! accesses that occur when runniing webassembly.
//!
//! This code is inspired by: https://github.com/pepyakin/wasmtime/commit/625a2b6c0815b21996e111da51b9664feb174622
use crate::call::recovery;
use nix::libc::{c_void, siginfo_t};
use nix::sys::signal::{
sigaction, SaFlags, SigAction, SigHandler, SigSet, SIGBUS, SIGFPE, SIGILL, SIGSEGV,
};
pub unsafe fn install_sighandler() {
let sa = SigAction::new(
SigHandler::SigAction(signal_trap_handler),
SaFlags::SA_ONSTACK,
SigSet::empty(),
);
sigaction(SIGFPE, &sa).unwrap();
sigaction(SIGILL, &sa).unwrap();
sigaction(SIGSEGV, &sa).unwrap();
sigaction(SIGBUS, &sa).unwrap();
}
extern "C" fn signal_trap_handler(
signum: ::nix::libc::c_int,
siginfo: *mut siginfo_t,
ucontext: *mut c_void,
) {
unsafe {
recovery::do_unwind(signum, siginfo, ucontext);
}
}

View File

@ -1,3 +1,5 @@
// pub mod codegen;
mod call;
mod func_env;
mod libcalls;
mod module;
@ -11,7 +13,7 @@ use cranelift_codegen::{
};
use target_lexicon::Triple;
use wasmer_runtime::{
backend::Compiler,
backend::{Compiler, Token},
error::{CompileError, CompileResult},
module::ModuleInner,
};
@ -27,7 +29,7 @@ impl CraneliftCompiler {
impl Compiler for CraneliftCompiler {
// Compiles wasm binary to a wasmer module.
fn compile(&self, wasm: &[u8]) -> CompileResult<ModuleInner> {
fn compile(&self, wasm: &[u8], _: Token) -> CompileResult<ModuleInner> {
validate(wasm)?;
let isa = get_isa();

View File

@ -1,4 +1,4 @@
use crate::resolver::FuncResolverBuilder;
use crate::{call::Caller, resolver::FuncResolverBuilder};
use cranelift_codegen::{ir, isa};
use cranelift_entity::EntityRef;
use cranelift_wasm;
@ -8,20 +8,21 @@ use std::{
ptr::NonNull,
};
use wasmer_runtime::{
backend::FuncResolver,
backend::SigRegistry,
error::CompileResult,
backend::{FuncResolver, ProtectedCaller, Token},
error::{CompileResult, RuntimeResult},
module::ModuleInner,
structures::{Map, TypedIndex},
types::{
FuncIndex, FuncSig, GlobalIndex, LocalFuncIndex, MemoryIndex, SigIndex, TableIndex, Type,
Value,
},
vm,
vm::{self, ImportBacking},
};
struct PlaceholderFuncResolver;
struct Placeholder;
impl FuncResolver for PlaceholderFuncResolver {
impl FuncResolver for Placeholder {
fn get(
&self,
_module: &ModuleInner,
@ -31,6 +32,21 @@ impl FuncResolver for PlaceholderFuncResolver {
}
}
impl ProtectedCaller for Placeholder {
fn call(
&self,
_module: &ModuleInner,
_func_index: FuncIndex,
_params: &[Value],
_returns: &mut [Value],
_import_backing: &ImportBacking,
_vmctx: *mut vm::Ctx,
_: Token,
) -> RuntimeResult<()> {
Ok(())
}
}
/// This contains all of the items in a `ModuleInner` except the `func_resolver`.
pub struct Module {
pub module: ModuleInner,
@ -41,7 +57,9 @@ impl Module {
Self {
module: ModuleInner {
// this is a placeholder
func_resolver: Box::new(PlaceholderFuncResolver),
func_resolver: Box::new(Placeholder),
protected_caller: Box::new(Placeholder),
memories: Map::new(),
globals: Map::new(),
tables: Map::new(),
@ -76,8 +94,11 @@ impl Module {
*sig_index = sig_registry.lookup_deduplicated_sigindex(*sig_index);
});
let func_resolver_builder = FuncResolverBuilder::new(isa, functions)?;
let (func_resolver_builder, handler_data) = FuncResolverBuilder::new(isa, functions)?;
self.module.func_resolver = Box::new(func_resolver_builder.finalize()?);
self.module.protected_caller = Box::new(Caller::new(&self.module, handler_data));
Ok(self.module)
}
}

View File

@ -3,10 +3,10 @@
//! any other calls that this function is doing, so we can "patch" the
//! function addrs in runtime with the functions we need.
use cranelift_codegen::binemit;
use cranelift_codegen::ir::{self, ExternalName, LibCall, SourceLoc, TrapCode};
use wasmer_runtime::{structures::TypedIndex, types::LocalFuncIndex};
pub use cranelift_codegen::binemit::Reloc;
use cranelift_codegen::ir::{self, ExternalName, LibCall, SourceLoc, TrapCode};
use hashbrown::HashMap;
use wasmer_runtime::{structures::TypedIndex, types::LocalFuncIndex};
#[derive(Debug, Clone)]
pub struct Relocation {
@ -133,30 +133,50 @@ impl RelocSink {
}
}
#[derive(Debug, Clone, Copy)]
pub struct TrapData {
pub offset: usize,
pub code: TrapCode,
pub trapcode: TrapCode,
pub srcloc: SourceLoc,
}
/// Simple implementation of a TrapSink
/// that saves the info for later.
pub struct TrapSink {
trap_datas: Vec<TrapData>,
trap_datas: HashMap<usize, TrapData>,
}
impl TrapSink {
pub fn new() -> TrapSink {
TrapSink {
trap_datas: Vec::new(),
trap_datas: HashMap::new(),
}
}
}
impl binemit::TrapSink for TrapSink {
fn trap(&mut self, offset: u32, _: SourceLoc, code: TrapCode) {
self.trap_datas.push(TrapData {
offset: offset as usize,
code,
pub fn lookup(&self, offset: usize) -> Option<TrapData> {
self.trap_datas.get(&offset).cloned()
}
pub fn drain_local(&mut self, current_func_offset: usize, local: &mut LocalTrapSink) {
local.trap_datas.drain(..).for_each(|(offset, trap_data)| {
self.trap_datas
.insert(current_func_offset + offset, trap_data);
});
}
}
pub struct LocalTrapSink {
trap_datas: Vec<(usize, TrapData)>,
}
impl LocalTrapSink {
pub fn new() -> Self {
LocalTrapSink { trap_datas: vec![] }
}
}
impl binemit::TrapSink for LocalTrapSink {
fn trap(&mut self, offset: u32, srcloc: SourceLoc, trapcode: TrapCode) {
self.trap_datas
.push((offset as usize, TrapData { trapcode, srcloc }));
}
}

View File

@ -1,5 +1,8 @@
use crate::call::HandlerData;
use crate::libcalls;
use crate::relocation::{Reloc, RelocSink, Relocation, RelocationType, TrapSink, VmCall};
use crate::relocation::{
LocalTrapSink, Reloc, RelocSink, Relocation, RelocationType, TrapSink, VmCall,
};
use byteorder::{ByteOrder, LittleEndian};
use cranelift_codegen::{ir, isa, Context};
use std::mem;
@ -20,17 +23,18 @@ use wasmer_runtime::{
pub struct FuncResolverBuilder {
resolver: FuncResolver,
relocations: Map<LocalFuncIndex, Vec<Relocation>>,
trap_sinks: Map<LocalFuncIndex, TrapSink>,
}
impl FuncResolverBuilder {
pub fn new(
isa: &isa::TargetIsa,
function_bodies: Map<LocalFuncIndex, ir::Function>,
) -> CompileResult<Self> {
) -> CompileResult<(Self, HandlerData)> {
let mut compiled_functions: Vec<Vec<u8>> = Vec::with_capacity(function_bodies.len());
let mut relocations = Map::with_capacity(function_bodies.len());
let mut trap_sinks = Map::with_capacity(function_bodies.len());
let mut trap_sink = TrapSink::new();
let mut local_trap_sink = LocalTrapSink::new();
let mut ctx = Context::new();
let mut total_size = 0;
@ -39,17 +43,20 @@ impl FuncResolverBuilder {
ctx.func = func;
let mut code_buf = Vec::new();
let mut reloc_sink = RelocSink::new();
let mut trap_sink = TrapSink::new();
ctx.compile_and_emit(isa, &mut code_buf, &mut reloc_sink, &mut trap_sink)
ctx.compile_and_emit(isa, &mut code_buf, &mut reloc_sink, &mut local_trap_sink)
.map_err(|e| CompileError::InternalError { msg: e.to_string() })?;
ctx.clear();
// Clear the local trap sink and consolidate all trap info
// into a single location.
trap_sink.drain_local(total_size, &mut local_trap_sink);
// Round up each function's size to pointer alignment.
total_size += round_up(code_buf.len(), mem::size_of::<usize>());
compiled_functions.push(code_buf);
relocations.push(reloc_sink.func_relocs);
trap_sinks.push(trap_sink);
}
let mut memory = Memory::with_size(total_size)
@ -87,11 +94,15 @@ impl FuncResolverBuilder {
previous_end = new_end;
}
Ok(Self {
resolver: FuncResolver { map, memory },
relocations,
trap_sinks,
})
let handler_data = HandlerData::new(trap_sink, memory.as_ptr() as _, memory.size());
Ok((
Self {
resolver: FuncResolver { map, memory },
relocations,
},
handler_data,
))
}
pub fn finalize(mut self) -> CompileResult<FuncResolver> {