Merge pull request #2408 from alexcrichton/fix-use-after-free-trampoline

Fix a use-after-free of trampoline code
This commit is contained in:
Nick Fitzgerald
2020-11-16 16:35:02 -08:00
committed by GitHub
15 changed files with 299 additions and 246 deletions

View File

@@ -46,7 +46,7 @@ pub mod trampoline;
pub use crate::code_memory::CodeMemory;
pub use crate::compiler::{Compilation, CompilationStrategy, Compiler};
pub use crate::instantiate::{CompilationArtifacts, CompiledModule, SetupError};
pub use crate::instantiate::{CompilationArtifacts, CompiledModule, ModuleCode, SetupError};
pub use crate::link::link_module;
/// Version number of this crate.

View File

@@ -43,7 +43,8 @@ pub use crate::memory::{RuntimeLinearMemory, RuntimeMemoryCreator};
pub use crate::mmap::Mmap;
pub use crate::table::{Table, TableElement};
pub use crate::traphandlers::{
catch_traps, init_traps, raise_lib_trap, raise_user_trap, resume_panic, SignalHandler, Trap,
catch_traps, init_traps, raise_lib_trap, raise_user_trap, resume_panic, with_last_info,
SignalHandler, Trap, TrapInfo,
};
pub use crate::vmcontext::{
VMCallerCheckedAnyfunc, VMContext, VMFunctionBody, VMFunctionImport, VMGlobalDefinition,

View File

@@ -370,9 +370,7 @@ impl Trap {
/// Highly unsafe since `closure` won't have any dtors run.
pub unsafe fn catch_traps<F>(
vmctx: *mut VMContext,
max_wasm_stack: usize,
is_wasm_code: impl Fn(usize) -> bool,
signal_handler: Option<&SignalHandler>,
trap_info: &impl TrapInfo,
mut closure: F,
) -> Result<(), Trap>
where
@@ -382,7 +380,7 @@ where
#[cfg(unix)]
setup_unix_sigaltstack()?;
return CallThreadState::new(vmctx, &is_wasm_code, signal_handler).with(max_wasm_stack, |cx| {
return CallThreadState::new(vmctx, trap_info).with(|cx| {
RegisterSetjmp(
cx.jmp_buf.as_ptr(),
call_closure::<F>,
@@ -398,6 +396,14 @@ where
}
}
/// Runs `func` with the last `trap_info` object registered by `catch_traps`.
///
/// Calls `func` with `None` if `catch_traps` wasn't previously called from this
/// stack frame.
pub fn with_last_info<R>(func: impl FnOnce(Option<&dyn Any>) -> R) -> R {
tls::with(|state| func(state.map(|s| s.trap_info.as_any())))
}
/// Temporary state stored on the stack which is registered in the `tls` module
/// below for calls into wasm.
pub struct CallThreadState<'a> {
@@ -405,8 +411,31 @@ pub struct CallThreadState<'a> {
jmp_buf: Cell<*const u8>,
vmctx: *mut VMContext,
handling_trap: Cell<bool>,
is_wasm_code: &'a (dyn Fn(usize) -> bool + 'a),
signal_handler: Option<&'a SignalHandler<'a>>,
trap_info: &'a (dyn TrapInfo + 'a),
}
/// A package of functionality needed by `catch_traps` to figure out what to do
/// when handling a trap.
///
/// Note that this is an `unsafe` trait at least because it's being run in the
/// context of a synchronous signal handler, so it needs to be careful to not
/// access too much state in answering these queries.
pub unsafe trait TrapInfo {
/// Converts this object into an `Any` to dynamically check its type.
fn as_any(&self) -> &dyn Any;
/// Returns whether the given program counter lies within wasm code,
/// indicating whether we should handle a trap or not.
fn is_wasm_code(&self, pc: usize) -> bool;
/// Uses `call` to call a custom signal handler, if one is specified.
///
/// Returns `true` if `call` returns true, otherwise returns `false`.
fn custom_signal_handler(&self, call: &dyn Fn(&SignalHandler) -> bool) -> bool;
/// Returns the maximum size, in bytes, the wasm native stack is allowed to
/// grow to.
fn max_wasm_stack(&self) -> usize;
}
enum UnwindReason {
@@ -418,27 +447,18 @@ enum UnwindReason {
}
impl<'a> CallThreadState<'a> {
fn new(
vmctx: *mut VMContext,
is_wasm_code: &'a (dyn Fn(usize) -> bool + 'a),
signal_handler: Option<&'a SignalHandler<'a>>,
) -> CallThreadState<'a> {
fn new(vmctx: *mut VMContext, trap_info: &'a (dyn TrapInfo + 'a)) -> CallThreadState<'a> {
CallThreadState {
unwind: Cell::new(UnwindReason::None),
vmctx,
jmp_buf: Cell::new(ptr::null()),
handling_trap: Cell::new(false),
is_wasm_code,
signal_handler,
trap_info,
}
}
fn with(
self,
max_wasm_stack: usize,
closure: impl FnOnce(&CallThreadState) -> i32,
) -> Result<(), Trap> {
let _reset = self.update_stack_limit(max_wasm_stack)?;
fn with(self, closure: impl FnOnce(&CallThreadState) -> i32) -> Result<(), Trap> {
let _reset = self.update_stack_limit()?;
let ret = tls::set(&self, || closure(&self));
match self.unwind.replace(UnwindReason::None) {
UnwindReason::None => {
@@ -498,7 +518,7 @@ impl<'a> CallThreadState<'a> {
///
/// Note that this function must be called with `self` on the stack, not the
/// heap/etc.
fn update_stack_limit(&self, max_wasm_stack: usize) -> Result<impl Drop + '_, Trap> {
fn update_stack_limit(&self) -> Result<impl Drop + '_, Trap> {
// Determine the stack pointer where, after which, any wasm code will
// immediately trap. This is checked on the entry to all wasm functions.
//
@@ -510,7 +530,7 @@ impl<'a> CallThreadState<'a> {
// to it). In any case it's expected to be at most a few hundred bytes
// of slop one way or another. When wasm is typically given a MB or so
// (a million bytes) the slop shouldn't matter too much.
let wasm_stack_limit = psm::stack_pointer() as usize - max_wasm_stack;
let wasm_stack_limit = psm::stack_pointer() as usize - self.trap_info.max_wasm_stack();
let interrupts = unsafe { &**(&*self.vmctx).instance().interrupts() };
let reset_stack_limit = match interrupts.stack_limit.compare_exchange(
@@ -604,14 +624,12 @@ impl<'a> CallThreadState<'a> {
// First up see if any instance registered has a custom trap handler,
// in which case run them all. If anything handles the trap then we
// return that the trap was handled.
if let Some(handler) = self.signal_handler {
if call_handler(handler) {
return 1 as *const _;
}
if self.trap_info.custom_signal_handler(&call_handler) {
return 1 as *const _;
}
// If this fault wasn't in wasm code, then it's not our problem
if !(self.is_wasm_code)(pc as usize) {
if !self.trap_info.is_wasm_code(pc as usize) {
return ptr::null();
}

View File

@@ -23,7 +23,6 @@ libc = "0.2"
cfg-if = "1.0"
backtrace = "0.3.42"
rustc-demangle = "0.1.16"
lazy_static = "1.4"
log = "0.4.8"
wat = { version = "1.0.18", optional = true }
smallvec = "1.4.0"

View File

@@ -492,13 +492,13 @@ impl Table {
// come from different modules.
let dst_table_index = dst_table.wasmtime_table_index();
let dst_table = dst_table.instance.get_defined_table(dst_table_index);
let dst_table_index = dst_table.instance.get_defined_table(dst_table_index);
let src_table_index = src_table.wasmtime_table_index();
let src_table = src_table.instance.get_defined_table(src_table_index);
let src_table_index = src_table.instance.get_defined_table(src_table_index);
runtime::Table::copy(dst_table, src_table, dst_index, src_index, len)
.map_err(Trap::from_runtime)?;
runtime::Table::copy(dst_table_index, src_table_index, dst_index, src_index, len)
.map_err(|e| Trap::from_runtime(&dst_table.instance.store, e))?;
Ok(())
}
@@ -523,7 +523,7 @@ impl Table {
self.instance
.handle
.defined_table_fill(table_index, dst, val.into_table_element()?, len)
.map_err(Trap::from_runtime)?;
.map_err(|e| Trap::from_runtime(&self.instance.store, e))?;
Ok(())
}

View File

@@ -1,23 +1,14 @@
use std::cmp;
use std::collections::BTreeMap;
use std::sync::{Arc, RwLock};
use std::sync::Arc;
use wasmtime_environ::entity::EntityRef;
use wasmtime_environ::ir;
use wasmtime_environ::wasm::FuncIndex;
use wasmtime_environ::{FunctionAddressMap, Module, TrapInformation};
use wasmtime_jit::CompiledModule;
lazy_static::lazy_static! {
/// This is a global cache of backtrace frame information for all active
///
/// This global cache is used during `Trap` creation to symbolicate frames.
/// This is populated on module compilation, and it is cleared out whenever
/// all references to a module are dropped.
pub static ref FRAME_INFO: RwLock<GlobalFrameInfo> = Default::default();
}
#[derive(Default)]
pub struct GlobalFrameInfo {
pub struct StoreFrameInfo {
/// An internal map that keeps track of backtrace frame information for
/// each module.
///
@@ -30,20 +21,10 @@ pub struct GlobalFrameInfo {
ranges: BTreeMap<usize, ModuleFrameInfo>,
}
/// An RAII structure used to unregister a module's frame information when the
/// module is destroyed.
pub struct GlobalFrameInfoRegistration {
/// The key that will be removed from the global `ranges` map when this is
/// dropped.
key: usize,
}
struct ModuleFrameInfo {
start: usize,
functions: BTreeMap<usize, FunctionInfo>,
module: Arc<Module>,
#[allow(dead_code)]
module_code: Arc<dyn std::any::Any + Send + Sync>,
}
struct FunctionInfo {
@@ -53,7 +34,7 @@ struct FunctionInfo {
instr_map: FunctionAddressMap,
}
impl GlobalFrameInfo {
impl StoreFrameInfo {
/// Fetches frame information about a program counter in a backtrace.
///
/// Returns an object if this `pc` is known to some previously registered
@@ -100,6 +81,12 @@ impl GlobalFrameInfo {
})
}
/// Returns whether the `pc` specified is contaained within some module's
/// function.
pub fn contains_pc(&self, pc: usize) -> bool {
self.func(pc).is_some()
}
/// Fetches trap information about a program counter in a backtrace.
pub fn lookup_trap_info(&self, pc: usize) -> Option<&TrapInformation> {
let (_module, func) = self.func(pc)?;
@@ -121,73 +108,62 @@ impl GlobalFrameInfo {
}
Some((info, func))
}
}
impl Drop for GlobalFrameInfoRegistration {
fn drop(&mut self) {
if let Ok(mut info) = FRAME_INFO.write() {
info.ranges.remove(&self.key);
/// Registers a new compiled module's frame information.
///
/// This function will register the `names` information for all of the
/// compiled functions within `module`. If the `module` has no functions
/// then `None` will be returned. Otherwise the returned object, when
/// dropped, will be used to unregister all name information from this map.
pub fn register(&mut self, module: &CompiledModule) {
let mut min = usize::max_value();
let mut max = 0;
let mut functions = BTreeMap::new();
for (i, allocated, traps, address_map) in module.trap_information() {
let (start, end) = unsafe {
let ptr = (*allocated).as_ptr();
let len = (*allocated).len();
// First and last byte of the function text.
(ptr as usize, ptr as usize + len - 1)
};
// Skip empty functions.
if end < start {
continue;
}
min = cmp::min(min, start);
max = cmp::max(max, end);
let func = FunctionInfo {
start,
index: module.module().func_index(i),
traps: traps.to_vec(),
instr_map: address_map.clone(),
};
assert!(functions.insert(end, func).is_none());
}
}
}
/// Registers a new compiled module's frame information.
///
/// This function will register the `names` information for all of the
/// compiled functions within `module`. If the `module` has no functions
/// then `None` will be returned. Otherwise the returned object, when
/// dropped, will be used to unregister all name information from this map.
pub fn register(module: &CompiledModule) -> Option<GlobalFrameInfoRegistration> {
let mut min = usize::max_value();
let mut max = 0;
let mut functions = BTreeMap::new();
for (i, allocated, traps, address_map) in module.trap_information() {
let (start, end) = unsafe {
let ptr = (*allocated).as_ptr();
let len = (*allocated).len();
// First and last byte of the function text.
(ptr as usize, ptr as usize + len - 1)
};
// Skip empty functions.
if end < start {
continue;
if functions.len() == 0 {
return;
}
min = cmp::min(min, start);
max = cmp::max(max, end);
let func = FunctionInfo {
start,
index: module.module().func_index(i),
traps: traps.to_vec(),
instr_map: address_map.clone(),
};
assert!(functions.insert(end, func).is_none());
}
if functions.len() == 0 {
return None;
}
let mut info = FRAME_INFO.write().unwrap();
// First up assert that our chunk of jit functions doesn't collide with
// any other known chunks of jit functions...
if let Some((_, prev)) = info.ranges.range(max..).next() {
assert!(prev.start > max);
}
if let Some((prev_end, _)) = info.ranges.range(..=min).next_back() {
assert!(*prev_end < min);
}
// First up assert that our chunk of jit functions doesn't collide with
// any other known chunks of jit functions...
if let Some((_, prev)) = self.ranges.range(max..).next() {
assert!(prev.start > max);
}
if let Some((prev_end, _)) = self.ranges.range(..=min).next_back() {
assert!(*prev_end < min);
}
// ... then insert our range and assert nothing was there previously
let prev = info.ranges.insert(
max,
ModuleFrameInfo {
start: min,
functions,
module: module.module().clone(),
module_code: module.code().clone(),
},
);
assert!(prev.is_none());
Some(GlobalFrameInfoRegistration { key: max })
// ... then insert our range and assert nothing was there previously
let prev = self.ranges.insert(
max,
ModuleFrameInfo {
start: min,
functions,
module: module.module().clone(),
},
);
assert!(prev.is_none());
}
}
/// Description of a frame in a backtrace for a [`Trap`].
@@ -285,8 +261,8 @@ fn test_frame_info() -> Result<(), anyhow::Error> {
"#,
)?;
// Create an instance to ensure the frame information is registered.
let _ = Instance::new(&store, &module, &[])?;
let info = FRAME_INFO.read().unwrap();
Instance::new(&store, &module, &[])?;
let info = store.frame_info().borrow();
for (i, alloc) in module.compiled_module().finished_functions() {
let (start, end) = unsafe {
let ptr = (**alloc).as_ptr();

View File

@@ -819,21 +819,14 @@ pub(crate) fn invoke_wasm_and_catch_traps(
store: &Store,
closure: impl FnMut(),
) -> Result<(), Trap> {
let signalhandler = store.signal_handler();
unsafe {
let canary = 0;
let _auto_reset_canary = store
.externref_activations_table()
.set_stack_canary(&canary);
wasmtime_runtime::catch_traps(
vmctx,
store.engine().config().max_wasm_stack,
|addr| store.is_in_jit_code(addr),
signalhandler.as_deref(),
closure,
)
.map_err(Trap::from_runtime)
wasmtime_runtime::catch_traps(vmctx, store, closure)
.map_err(|e| Trap::from_runtime(store, e))
}
}

View File

@@ -16,6 +16,11 @@ fn instantiate(
imports: Imports<'_>,
host: Box<dyn Any>,
) -> Result<StoreInstanceHandle, Error> {
// Register the module just before instantiation to ensure we have a
// trampoline registered for every signature and to preserve the module's
// compiled JIT code within the `Store`.
store.register_module(compiled_module);
let config = store.engine().config();
let instance = unsafe {
let instance = compiled_module.instantiate(
@@ -43,7 +48,7 @@ fn instantiate(
)
.map_err(|e| -> Error {
match e {
InstantiationError::Trap(trap) => Trap::from_runtime(trap).into(),
InstantiationError::Trap(trap) => Trap::from_runtime(store, trap).into(),
other => other.into(),
}
})?;
@@ -98,11 +103,6 @@ fn instantiate(
#[derive(Clone)]
pub struct Instance {
pub(crate) handle: StoreInstanceHandle,
// Note that this is required to keep the module's code memory alive while
// we have a handle to this `Instance`. We may eventually want to shrink
// this to only hold onto the bare minimum each instance needs to allow
// deallocating some `Module` resources early, but until then we just hold
// on to everything.
module: Module,
}
@@ -165,11 +165,8 @@ impl Instance {
bail!("cross-`Engine` instantiation is not currently supported");
}
store.register_module(&module);
let host_info = Box::new(module.register_frame_info());
let handle = with_imports(store, module.compiled_module(), imports, |imports| {
instantiate(store, module.compiled_module(), imports, host_info)
instantiate(store, module.compiled_module(), imports, Box::new(()))
})?;
Ok(Instance {

View File

@@ -1,11 +1,10 @@
use crate::frame_info::GlobalFrameInfoRegistration;
use crate::types::{EntityType, ExportType, ExternType, ImportType};
use crate::Engine;
use anyhow::{bail, Context, Result};
use bincode::Options;
use std::hash::Hash;
use std::path::Path;
use std::sync::{Arc, Mutex};
use std::sync::Arc;
use wasmparser::Validator;
#[cfg(feature = "cache")]
use wasmtime_cache::ModuleCacheEntry;
@@ -84,7 +83,6 @@ pub struct Module {
engine: Engine,
compiled: Arc<[CompiledModule]>,
index: usize,
frame_info_registration: Arc<Mutex<Option<Option<Arc<GlobalFrameInfoRegistration>>>>>,
}
impl Module {
@@ -259,7 +257,6 @@ impl Module {
engine: engine.clone(),
index: compiled.len() - 1,
compiled: compiled.into(),
frame_info_registration: Arc::new(Mutex::new(None)),
})
}
@@ -333,7 +330,6 @@ impl Module {
engine: engine.clone(),
index,
compiled: compiled.into(),
frame_info_registration: Arc::new(Mutex::new(None)),
})
}
@@ -548,19 +544,6 @@ impl Module {
pub fn engine(&self) -> &Engine {
&self.engine
}
/// Register this module's stack frame information into the global scope.
///
/// This is required to ensure that any traps can be properly symbolicated.
pub(crate) fn register_frame_info(&self) -> Option<Arc<GlobalFrameInfoRegistration>> {
let mut info = self.frame_info_registration.lock().unwrap();
if let Some(info) = &*info {
return info.clone();
}
let ret = super::frame_info::register(self.compiled_module()).map(Arc::new);
*info = Some(ret.clone());
return ret;
}
}
fn bincode_options() -> impl Options {

View File

@@ -1,16 +1,19 @@
use crate::frame_info::StoreFrameInfo;
use crate::sig_registry::SignatureRegistry;
use crate::trampoline::StoreInstanceHandle;
use crate::Engine;
use crate::Module;
use anyhow::{bail, Result};
use std::any::Any;
use std::cell::RefCell;
use std::collections::HashSet;
use std::fmt;
use std::hash::{Hash, Hasher};
use std::rc::{Rc, Weak};
use std::sync::Arc;
use wasmtime_environ::wasm;
use wasmtime_jit::{CompiledModule, ModuleCode};
use wasmtime_runtime::{
InstanceHandle, RuntimeMemoryCreator, SignalHandler, StackMapRegistry, VMExternRef,
InstanceHandle, RuntimeMemoryCreator, SignalHandler, StackMapRegistry, TrapInfo, VMExternRef,
VMExternRefActivationsTable, VMInterrupts, VMSharedSignatureIndex,
};
@@ -18,8 +21,8 @@ use wasmtime_runtime::{
///
/// All WebAssembly instances and items will be attached to and refer to a
/// `Store`. For example instances, functions, globals, and tables are all
/// attached to a `Store`. Instances are created by instantiating a [`Module`]
/// within a `Store`.
/// attached to a `Store`. Instances are created by instantiating a
/// [`Module`](crate::Module) within a `Store`.
///
/// `Store` is not thread-safe and cannot be sent to other threads. All items
/// which refer to a `Store` additionally are not threadsafe and can only be
@@ -56,9 +59,14 @@ pub(crate) struct StoreInner {
signatures: RefCell<SignatureRegistry>,
instances: RefCell<Vec<InstanceHandle>>,
signal_handler: RefCell<Option<Box<SignalHandler<'static>>>>,
jit_code_ranges: RefCell<Vec<(usize, usize)>>,
externref_activations_table: VMExternRefActivationsTable,
stack_map_registry: StackMapRegistry,
/// Information about JIT code which allows us to test if a program counter
/// is in JIT code, lookup trap information, etc.
frame_info: RefCell<StoreFrameInfo>,
/// Set of all compiled modules that we're holding a strong reference to
/// the module's code for. This includes JIT functions, trampolines, etc.
modules: RefCell<HashSet<ArcModuleCode>>,
}
struct HostInfoKey(VMExternRef);
@@ -97,9 +105,10 @@ impl Store {
signatures: RefCell::new(Default::default()),
instances: RefCell::new(Vec::new()),
signal_handler: RefCell::new(None),
jit_code_ranges: RefCell::new(Vec::new()),
externref_activations_table: VMExternRefActivationsTable::new(),
stack_map_registry: StackMapRegistry::default(),
frame_info: Default::default(),
modules: Default::default(),
}),
}
}
@@ -138,17 +147,7 @@ impl Store {
}
}
/// Returns whether or not the given address falls within the JIT code
/// managed by the compiler
pub(crate) fn is_in_jit_code(&self, addr: usize) -> bool {
self.inner
.jit_code_ranges
.borrow()
.iter()
.any(|(start, end)| *start <= addr && addr < *end)
}
pub(crate) fn register_module(&self, module: &Module) {
pub(crate) fn register_module(&self, module: &CompiledModule) {
// All modules register their JIT code in a store for two reasons
// currently:
//
@@ -171,28 +170,30 @@ impl Store {
// a `Func` wrapper for any function in the module, which requires that
// we know about the signature and trampoline for all instances.
self.register_signatures(module);
// And finally with a module being instantiated into this `Store` we
// need to preserve its jit-code. References to this module's code and
// trampolines are not owning-references so it's our responsibility to
// keep it all alive within the `Store`.
self.inner
.modules
.borrow_mut()
.insert(ArcModuleCode(module.code().clone()));
}
fn register_jit_code(&self, module: &Module) {
let mut ranges = module.compiled_module().jit_code_ranges();
// Checking of we already registered JIT code ranges by searching
// first range start.
match ranges.next() {
None => (),
Some(first) => {
if !self.is_in_jit_code(first.0) {
// The range is not registered -- add all ranges (including
// first one) to the jit_code_ranges.
let mut jit_code_ranges = self.inner.jit_code_ranges.borrow_mut();
jit_code_ranges.push(first);
jit_code_ranges.extend(ranges);
}
}
fn register_jit_code(&self, module: &CompiledModule) {
let functions = module.finished_functions();
let first_pc = match functions.values().next() {
Some(f) => unsafe { (**f).as_ptr() as usize },
None => return,
};
// Only register this module if it hasn't already been registered.
if !self.is_wasm_code(first_pc) {
self.inner.frame_info.borrow_mut().register(module);
}
}
fn register_stack_maps(&self, module: &Module) {
let module = &module.compiled_module();
fn register_stack_maps(&self, module: &CompiledModule) {
self.stack_map_registry()
.register_stack_maps(module.stack_maps().map(|(func, stack_maps)| unsafe {
let ptr = (*func).as_ptr();
@@ -204,9 +205,9 @@ impl Store {
}));
}
fn register_signatures(&self, module: &Module) {
let trampolines = module.compiled_module().trampolines();
let module = module.compiled_module().module();
fn register_signatures(&self, module: &CompiledModule) {
let trampolines = module.trampolines();
let module = module.module();
let mut signatures = self.signatures().borrow_mut();
for (index, wasm) in module.signatures.iter() {
signatures.register(wasm, trampolines[index]);
@@ -243,14 +244,8 @@ impl Store {
Some(Self { inner })
}
pub(crate) fn signal_handler(&self) -> std::cell::Ref<'_, Option<Box<SignalHandler<'static>>>> {
self.inner.signal_handler.borrow()
}
pub(crate) fn signal_handler_mut(
&self,
) -> std::cell::RefMut<'_, Option<Box<SignalHandler<'static>>>> {
self.inner.signal_handler.borrow_mut()
pub(crate) fn set_signal_handler(&self, handler: Option<Box<SignalHandler<'static>>>) {
*self.inner.signal_handler.borrow_mut() = handler;
}
pub(crate) fn interrupts(&self) -> &VMInterrupts {
@@ -367,6 +362,10 @@ impl Store {
&self.inner.stack_map_registry
}
pub(crate) fn frame_info(&self) -> &RefCell<StoreFrameInfo> {
&self.inner.frame_info
}
/// Perform garbage collection of `ExternRef`s.
pub fn gc(&self) {
// For this crate's API, we ensure that `set_stack_canary` invariants
@@ -381,6 +380,27 @@ impl Store {
}
}
unsafe impl TrapInfo for Store {
fn as_any(&self) -> &dyn Any {
self
}
fn is_wasm_code(&self, addr: usize) -> bool {
self.frame_info().borrow().contains_pc(addr)
}
fn custom_signal_handler(&self, call: &dyn Fn(&SignalHandler) -> bool) -> bool {
if let Some(handler) = &*self.inner.signal_handler.borrow() {
return call(handler);
}
false
}
fn max_wasm_stack(&self) -> usize {
self.engine().config().max_wasm_stack
}
}
impl Default for Store {
fn default() -> Store {
Store::new(&Engine::default())
@@ -424,3 +444,21 @@ impl InterruptHandle {
self.interrupts.interrupt()
}
}
// Wrapper struct to implement hash/equality based on the pointer value of the
// `Arc` in question.
struct ArcModuleCode(Arc<ModuleCode>);
impl PartialEq for ArcModuleCode {
fn eq(&self, other: &ArcModuleCode) -> bool {
Arc::ptr_eq(&self.0, &other.0)
}
}
impl Eq for ArcModuleCode {}
impl Hash for ArcModuleCode {
fn hash<H: Hasher>(&self, hasher: &mut H) {
Arc::as_ptr(&self.0).hash(hasher)
}
}

View File

@@ -1,5 +1,4 @@
use crate::frame_info::{GlobalFrameInfo, FRAME_INFO};
use crate::FrameInfo;
use crate::{FrameInfo, Store};
use backtrace::Backtrace;
use std::fmt;
use std::sync::Arc;
@@ -137,25 +136,22 @@ impl Trap {
/// assert!(trap.to_string().contains("unexpected error"));
/// ```
pub fn new<I: Into<String>>(message: I) -> Self {
let info = FRAME_INFO.read().unwrap();
let reason = TrapReason::Message(message.into());
Trap::new_with_trace(&info, None, reason, Backtrace::new_unresolved())
Trap::new_with_trace(None, None, reason, Backtrace::new_unresolved())
}
/// Creates a new `Trap` representing an explicit program exit with a classic `i32`
/// exit status value.
pub fn i32_exit(status: i32) -> Self {
Trap {
inner: Arc::new(TrapInner {
reason: TrapReason::I32Exit(status),
wasm_trace: Vec::new(),
native_trace: Backtrace::from(Vec::new()),
}),
}
Trap::new_with_trace(
None,
None,
TrapReason::I32Exit(status),
Backtrace::new_unresolved(),
)
}
pub(crate) fn from_runtime(runtime_trap: wasmtime_runtime::Trap) -> Self {
let info = FRAME_INFO.read().unwrap();
pub(crate) fn from_runtime(store: &Store, runtime_trap: wasmtime_runtime::Trap) -> Self {
match runtime_trap {
wasmtime_runtime::Trap::User(error) => Trap::from(error),
wasmtime_runtime::Trap::Jit {
@@ -163,63 +159,90 @@ impl Trap {
backtrace,
maybe_interrupted,
} => {
let mut code = info
let mut code = store
.frame_info()
.borrow()
.lookup_trap_info(pc)
.map(|info| info.trap_code)
.unwrap_or(ir::TrapCode::StackOverflow);
if maybe_interrupted && code == ir::TrapCode::StackOverflow {
code = ir::TrapCode::Interrupt;
}
Trap::new_wasm(&info, Some(pc), code, backtrace)
Trap::new_wasm(Some(store), Some(pc), code, backtrace)
}
wasmtime_runtime::Trap::Wasm {
trap_code,
backtrace,
} => Trap::new_wasm(&info, None, trap_code, backtrace),
} => Trap::new_wasm(Some(store), None, trap_code, backtrace),
wasmtime_runtime::Trap::OOM { backtrace } => {
let reason = TrapReason::Message("out of memory".to_string());
Trap::new_with_trace(&info, None, reason, backtrace)
Trap::new_with_trace(Some(store), None, reason, backtrace)
}
}
}
fn new_wasm(
info: &GlobalFrameInfo,
store: Option<&Store>,
trap_pc: Option<usize>,
code: ir::TrapCode,
backtrace: Backtrace,
) -> Self {
let code = TrapCode::from_non_user(code);
Trap::new_with_trace(info, trap_pc, TrapReason::InstructionTrap(code), backtrace)
Trap::new_with_trace(store, trap_pc, TrapReason::InstructionTrap(code), backtrace)
}
/// Creates a new `Trap`.
///
/// * `store` - this is optionally provided, if available. If `None` we'll
/// look up the last store, if available, used to call wasm code on the
/// stack.
///
/// * `trap_pc` - this is the precise program counter, if available, that
/// wasm trapped at. This is used when learning about the wasm stack trace
/// to ensure we assign the correct source to every frame.
///
/// * `reason` - this is the wasmtime-internal reason for why this trap is
/// being created.
///
/// * `native_trace` - this is a captured backtrace from when the trap
/// occurred, and this will iterate over the frames to find frames that
/// lie in wasm jit code.
fn new_with_trace(
info: &GlobalFrameInfo,
store: Option<&Store>,
trap_pc: Option<usize>,
reason: TrapReason,
native_trace: Backtrace,
) -> Self {
let mut wasm_trace = Vec::new();
for frame in native_trace.frames() {
let pc = frame.ip() as usize;
if pc == 0 {
continue;
wasmtime_runtime::with_last_info(|last| {
// If the `store` passed in is `None` then we look at the `last`
// store configured to call wasm, and if that's a `Store` we use
// that. If that all fails then we just don't generate any
// `wasm_trace` information.
if let Some(store) = store.or_else(|| last?.downcast_ref()) {
for frame in native_trace.frames() {
let pc = frame.ip() as usize;
if pc == 0 {
continue;
}
// Note that we need to be careful about the pc we pass in
// here to lookup frame information. This program counter is
// used to translate back to an original source location in
// the origin wasm module. If this pc is the exact pc that
// the trap happened at, then we look up that pc precisely.
// Otherwise backtrace information typically points at the
// pc *after* the call instruction (because otherwise it's
// likely a call instruction on the stack). In that case we
// want to lookup information for the previous instruction
// (the call instruction) so we subtract one as the lookup.
let pc_to_lookup = if Some(pc) == trap_pc { pc } else { pc - 1 };
if let Some(info) = store.frame_info().borrow().lookup_frame_info(pc_to_lookup)
{
wasm_trace.push(info);
}
}
}
// Note that we need to be careful about the pc we pass in here to
// lookup frame information. This program counter is used to
// translate back to an original source location in the origin wasm
// module. If this pc is the exact pc that the trap happened at,
// then we look up that pc precisely. Otherwise backtrace
// information typically points at the pc *after* the call
// instruction (because otherwise it's likely a call instruction on
// the stack). In that case we want to lookup information for the
// previous instruction (the call instruction) so we subtract one as
// the lookup.
let pc_to_lookup = if Some(pc) == trap_pc { pc } else { pc - 1 };
if let Some(info) = info.lookup_frame_info(pc_to_lookup) {
wasm_trace.push(info);
}
}
});
Trap {
inner: Arc::new(TrapInner {
reason,
@@ -311,9 +334,8 @@ impl From<Box<dyn std::error::Error + Send + Sync>> for Trap {
if let Some(trap) = e.downcast_ref::<Trap>() {
trap.clone()
} else {
let info = FRAME_INFO.read().unwrap();
let reason = TrapReason::Error(e.into());
Trap::new_with_trace(&info, None, reason, Backtrace::new_unresolved())
Trap::new_with_trace(None, None, reason, Backtrace::new_unresolved())
}
}
}

View File

@@ -26,6 +26,6 @@ impl StoreExt for Store {
where
H: 'static + Fn(libc::c_int, *const libc::siginfo_t, *const libc::c_void) -> bool,
{
*self.signal_handler_mut() = Some(Box::new(handler));
self.set_signal_handler(Some(Box::new(handler)));
}
}

View File

@@ -26,6 +26,6 @@ impl StoreExt for Store {
where
H: 'static + Fn(winapi::um::winnt::PEXCEPTION_POINTERS) -> bool,
{
*self.signal_handler_mut() = Some(Box::new(handler));
self.set_signal_handler(Some(Box::new(handler)));
}
}