Merge pull request #2408 from alexcrichton/fix-use-after-free-trampoline
Fix a use-after-free of trampoline code
This commit is contained in:
@@ -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.
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
|
||||
@@ -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"
|
||||
|
||||
@@ -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(())
|
||||
}
|
||||
|
||||
@@ -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();
|
||||
|
||||
@@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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)));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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)));
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user