More optimizations for calling into WebAssembly (#2759)

* Combine stack-based cleanups for faster wasm calls

This commit is an extension of #2757 where the goal is to optimize entry
into WebAssembly. Currently wasmtime has two stack-based cleanups when
entering wasm, one for the externref activation table and another for
stack limits getting reset. This commit fuses these two cleanups
together into one and moves some code around which enables less captures
for fewer closures and such to speed up calls in to wasm a bit more.
Overall this drops the execution time from 88ns to 80ns locally for me.

This also updates the atomic orderings when updating the stack limit
from `SeqCst` to `Relaxed`. While `SeqCst` is a reasonable starting
point the usage here should be safe to use `Relaxed` since we're not
using the atomics to actually protect any memory, it's simply receiving
signals from other threads.

* Determine whether a pc is wasm via a global map

The macOS implementation of traps recently changed to using mach ports
for handlers instead of signal handlers. This means that a previously
relied upon invariant, each thread fixes its own trap, was broken. The
macOS implementation worked around this by maintaining a global map from
thread id to thread local information, however, to solve the problem.

This global map is quite slow though. It involves taking a lock and
updating a hash map on all calls into WebAssembly. In my local testing
this accounts for >70% of the overhead of calling into WebAssembly on
macOS. Naturally it'd be great to remove this!

This commit fixes this issue and removes the global lock/map that is
updated on all calls into WebAssembly. The fix is to maintain a global
map of wasm modules and their trap addresses in the `wasmtime` crate.
Doing so is relatively simple since we're already tracking this
information at the `Store` level.

Once we've got a global map then the macOS implementation can use this
from a foreign thread and everything works out.

Locally this brings the overhead, on macOS specifically, of calling into
wasm from 80ns to ~20ns.

* Fix compiles

* Review comments
This commit is contained in:
Alex Crichton
2021-03-24 11:41:33 -05:00
committed by GitHub
parent 6b2da3d299
commit d4b54ee0a8
13 changed files with 324 additions and 309 deletions

3
Cargo.lock generated
View File

@@ -3177,9 +3177,11 @@ dependencies = [
"cfg-if 1.0.0",
"cpp_demangle",
"indexmap",
"lazy_static",
"libc",
"log",
"paste",
"psm",
"region",
"rustc-demangle",
"serde",
@@ -3478,7 +3480,6 @@ dependencies = [
"mach",
"memoffset",
"more-asserts",
"psm",
"rand 0.8.3",
"region",
"thiserror",

View File

@@ -24,7 +24,6 @@ more-asserts = "0.2.1"
cfg-if = "1.0"
backtrace = "0.3.55"
lazy_static = "1.3.0"
psm = "0.1.11"
rand = "0.8.3"
anyhow = "1.0.38"

View File

@@ -522,13 +522,13 @@ pub struct VMExternRefActivationsTable {
/// than create a new hash set every GC.
precise_stack_roots: RefCell<HashSet<VMExternRefWithTraits>>,
/// A pointer to a `u8` on the youngest host stack frame before we called
/// A pointer to the youngest host stack frame before we called
/// into Wasm for the first time. When walking the stack in garbage
/// collection, if we don't find this frame, then we failed to walk every
/// Wasm stack frame, which means we failed to find all on-stack,
/// inside-a-Wasm-frame roots, and doing a GC could lead to freeing one of
/// those missed roots, and use after free.
stack_canary: Cell<Option<NonNull<u8>>>,
stack_canary: Cell<Option<usize>>,
}
impl VMExternRefActivationsTable {
@@ -717,73 +717,29 @@ impl VMExternRefActivationsTable {
}
}
/// Set the stack canary around a call into Wasm.
/// Fetches the current value of this table's stack canary.
///
/// The return value should not be dropped until after the Wasm call has
/// returned.
/// This should only be used in conjunction with setting the stack canary
/// below if the return value is `None` typically. This is called from RAII
/// guards in `wasmtime::func::invoke_wasm_and_catch_traps`.
///
/// While this method is always safe to call (or not call), it is unsafe to
/// call the `wasmtime_runtime::gc` function unless this method is called at
/// the proper times and its return value properly outlives its Wasm call.
///
/// For `gc` to be safe, this is only *strictly required* to surround the
/// oldest host-->Wasm stack frame transition on this thread, but repeatedly
/// calling it is idempotent and cheap, so it is recommended to call this
/// for every host-->Wasm call.
///
/// # Example
///
/// ```no_run
/// use wasmtime_runtime::*;
///
/// # let get_table_from_somewhere = || unimplemented!();
/// let table: &VMExternRefActivationsTable = get_table_from_somewhere();
///
/// // Set the canary before a Wasm call. The canary should always be a
/// // local on the stack.
/// let canary = 0;
/// let auto_reset_canary = table.set_stack_canary(&canary);
///
/// // Do the call into Wasm.
/// # let call_into_wasm = || unimplemented!();
/// call_into_wasm();
///
/// // Only drop the value returned by `set_stack_canary` after the Wasm
/// // call has returned.
/// drop(auto_reset_canary);
/// ```
/// For more information on canaries see the gc functions below.
#[inline]
pub fn set_stack_canary<'a>(&'a self, canary: &u8) -> impl Drop + 'a {
let should_reset = if self.stack_canary.get().is_none() {
let canary = canary as *const u8 as *mut u8;
self.stack_canary.set(Some(unsafe {
debug_assert!(!canary.is_null());
NonNull::new_unchecked(canary)
}));
true
} else {
false
};
return AutoResetCanary {
table: self,
should_reset,
};
struct AutoResetCanary<'a> {
table: &'a VMExternRefActivationsTable,
should_reset: bool,
pub fn stack_canary(&self) -> Option<usize> {
self.stack_canary.get()
}
impl Drop for AutoResetCanary<'_> {
/// Sets the current value of the stack canary.
///
/// This is called from RAII guards in
/// `wasmtime::func::invoke_wasm_and_catch_traps`. This is used to update
/// the stack canary to a concrete value and then reset it back to `None`
/// when wasm is finished.
///
/// For more information on canaries see the gc functions below.
#[inline]
fn drop(&mut self) {
if self.should_reset {
debug_assert!(self.table.stack_canary.get().is_some());
self.table.stack_canary.set(None);
}
}
}
pub fn set_stack_canary(&self, canary: Option<usize>) {
self.stack_canary.set(canary);
}
}
@@ -1066,7 +1022,7 @@ pub unsafe fn gc(
log::debug!("end GC");
return;
}
Some(canary) => canary.as_ptr() as usize,
Some(canary) => canary,
};
// There is a stack canary, so there must be Wasm frames on the stack. The

View File

@@ -8,7 +8,7 @@ use std::cell::{Cell, UnsafeCell};
use std::error::Error;
use std::mem::MaybeUninit;
use std::ptr;
use std::sync::atomic::{AtomicUsize, Ordering::SeqCst};
use std::sync::atomic::Ordering::SeqCst;
use std::sync::Once;
use wasmtime_environ::ir;
@@ -38,19 +38,32 @@ cfg_if::cfg_if! {
pub use sys::SignalHandler;
/// This function performs the low-overhead platform-specific initialization
/// that we want to do eagerly to ensure a more-deterministic global process
/// state.
/// Globally-set callback to determine whether a program counter is actually a
/// wasm trap.
///
/// This is especially relevant for signal handlers since handler ordering
/// depends on installation order: the wasm signal handler must run *before*
/// the other crash handlers and since POSIX signal handlers work LIFO, this
/// function needs to be called at the end of the startup process, after other
/// handlers have been installed. This function can thus be called multiple
/// times, having no effect after the first call.
pub fn init_traps() -> Result<(), Trap> {
/// This is initialized during `init_traps` below. The definition lives within
/// `wasmtime` currently.
static mut IS_WASM_PC: fn(usize) -> bool = |_| false;
/// This function is required to be called before any WebAssembly is entered.
/// This will configure global state such as signal handlers to prepare the
/// process to receive wasm traps.
///
/// This function must not only be called globally once before entering
/// WebAssembly but it must also be called once-per-thread that enters
/// WebAssembly. Currently in wasmtime's integration this function is called on
/// creation of a `Store`.
///
/// The `is_wasm_pc` argument is used when a trap happens to determine if a
/// program counter is the pc of an actual wasm trap or not. This is then used
/// to disambiguate faults that happen due to wasm and faults that happen due to
/// bugs in Rust or elsewhere.
pub fn init_traps(is_wasm_pc: fn(usize) -> bool) -> Result<(), Trap> {
static INIT: Once = Once::new();
INIT.call_once(|| unsafe { sys::platform_init() });
INIT.call_once(|| unsafe {
IS_WASM_PC = is_wasm_pc;
sys::platform_init();
});
sys::lazy_per_thread_init()
}
@@ -208,19 +221,11 @@ 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_trap(&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;
/// Callback invoked whenever WebAssembly has entirely consumed the fuel
/// that it was allotted.
///
@@ -251,7 +256,6 @@ impl<'a> CallThreadState<'a> {
}
fn with(self, closure: impl FnOnce(&CallThreadState) -> i32) -> Result<(), Trap> {
let _reset = self.update_stack_limit()?;
let ret = tls::set(&self, || closure(&self));
if ret != 0 {
return Ok(());
@@ -273,98 +277,6 @@ impl<'a> CallThreadState<'a> {
}
}
/// Checks and/or initializes the wasm native call stack limit.
///
/// This function will inspect the current state of the stack and calling
/// context to determine which of three buckets we're in:
///
/// 1. We are the first wasm call on the stack. This means that we need to
/// set up a stack limit where beyond which if the native wasm stack
/// pointer goes beyond forces a trap. For now we simply reserve an
/// arbitrary chunk of bytes (1 MB from roughly the current native stack
/// pointer). This logic will likely get tweaked over time.
///
/// 2. We aren't the first wasm call on the stack. In this scenario the wasm
/// stack limit is already configured. This case of wasm -> host -> wasm
/// we assume that the native stack consumed by the host is accounted for
/// in the initial stack limit calculation. That means that in this
/// scenario we do nothing.
///
/// 3. We were previously interrupted. In this case we consume the interrupt
/// here and return a trap, clearing the interrupt and allowing the next
/// wasm call to proceed.
///
/// The return value here is a trap for case 3, a noop destructor in case 2,
/// and a meaningful destructor in case 1
///
/// For more information about interrupts and stack limits see
/// `crates/environ/src/cranelift.rs`.
///
/// Note that this function must be called with `self` on the stack, not the
/// heap/etc.
#[inline]
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.
//
// Note that this isn't 100% precise. We are requested to give wasm
// `max_wasm_stack` bytes, but what we're actually doing is giving wasm
// probably a little less than `max_wasm_stack` because we're
// calculating the limit relative to this function's approximate stack
// pointer. Wasm will be executed on a frame beneath this one (or next
// 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 - self.trap_info.max_wasm_stack();
let interrupts = self.trap_info.interrupts();
let reset_stack_limit = match interrupts.stack_limit.compare_exchange(
usize::max_value(),
wasm_stack_limit,
SeqCst,
SeqCst,
) {
Ok(_) => {
// We're the first wasm on the stack so we've now reserved the
// `max_wasm_stack` bytes of native stack space for wasm.
// Nothing left to do here now except reset back when we're
// done.
true
}
Err(n) if n == wasmtime_environ::INTERRUPTED => {
// This means that an interrupt happened before we actually
// called this function, which means that we're now
// considered interrupted. Be sure to consume this interrupt
// as part of this process too.
interrupts.stack_limit.store(usize::max_value(), SeqCst);
return Err(Trap::Wasm {
trap_code: ir::TrapCode::Interrupt,
backtrace: Backtrace::new_unresolved(),
});
}
Err(_) => {
// The stack limit was previously set by a previous wasm
// call on the stack. We leave the original stack limit for
// wasm in place in that case, and don't reset the stack
// limit when we're done.
false
}
};
struct Reset<'a>(bool, &'a AtomicUsize);
impl Drop for Reset<'_> {
#[inline]
fn drop(&mut self) {
if self.0 {
self.1.store(usize::max_value(), SeqCst);
}
}
}
Ok(Reset(reset_stack_limit, &interrupts.stack_limit))
}
fn unwind_with(&self, reason: UnwindReason) -> ! {
unsafe {
(*self.unwind.get()).as_mut_ptr().write(reason);
@@ -387,6 +299,7 @@ impl<'a> CallThreadState<'a> {
/// instance, and the trap handler should quickly return.
/// * a different pointer - a jmp_buf buffer to longjmp to, meaning that
/// the wasm trap was succesfully handled.
#[cfg_attr(target_os = "macos", allow(dead_code))] // macOS is more raw and doesn't use this
fn jmp_buf_if_trap(
&self,
pc: *const u8,
@@ -415,7 +328,7 @@ impl<'a> CallThreadState<'a> {
}
// If this fault wasn't in wasm code, then it's not our problem
if !self.trap_info.is_wasm_trap(pc as usize) {
if unsafe { !IS_WASM_PC(pc as usize) } {
return ptr::null();
}
@@ -480,10 +393,6 @@ mod tls {
#[inline(never)] // see module docs for why this is here
pub fn replace(val: Ptr) -> Ptr {
// Mark the current thread as handling interrupts for this specific
// CallThreadState: may clobber the previous entry.
super::super::sys::register_tls(val);
PTR.with(|p| p.replace(val))
}

View File

@@ -21,11 +21,10 @@
//! port. This means that, unlike signals, threads can't fix their own traps.
//! Instead a helper thread is spun up to service exception messages. This is
//! also in conflict with Wasmtime's exception handling currently which is to
//! use a thread-local to figure out whether a pc is a wasm pc or not on a
//! trap. To work around this we have a global map from mach thread numbers to
//! the state for that thread, updated on entry/exit from wasm. This is likely
//! slower than signals which do less updating on wasm entry/exit, but hopefully
//! by the time this is a problem we can figure out a better solution.
//! use a thread-local to store information about how to unwind. Additionally
//! this requires that the check of whether a pc is a wasm trap or not is a
//! global check rather than a per-thread check. This necessitates the existence
//! of `GlobalFrameInfo` in the `wasmtime` crate.
//!
//! Otherwise this file heavily uses the `mach` Rust crate for type and
//! function declarations. Many bits and pieces are copied or translated from
@@ -33,7 +32,7 @@
#![allow(non_snake_case)]
use crate::traphandlers::{tls, CallThreadState, Trap, Unwind};
use crate::traphandlers::{tls, Trap, Unwind};
use mach::exception_types::*;
use mach::kern_return::*;
use mach::mach_init::*;
@@ -43,10 +42,7 @@ use mach::port::*;
use mach::thread_act::*;
use mach::traps::*;
use std::cell::Cell;
use std::collections::HashMap;
use std::mem;
use std::ptr;
use std::sync::Mutex;
use std::thread;
/// Other `mach` declarations awaiting https://github.com/fitzgen/mach/pull/64 to be merged.
@@ -154,20 +150,10 @@ pub enum Void {}
/// Wasmtime on macOS.
pub type SignalHandler<'a> = dyn Fn(Void) -> bool + 'a;
/// Process-global map for mapping thread names to their state to figure out
/// whether a thread's trap is related to wasm or not. This is extremely
/// unsafe and caution must be used when accessing. Be sure to read
/// documentation below on this.
static mut MAP: *mut Mutex<HashMap<mach_port_name_t, *const CallThreadState<'static>>> =
ptr::null_mut();
/// Process-global port that we use to route thread-level exceptions to.
static mut WASMTIME_PORT: mach_port_name_t = MACH_PORT_NULL;
pub unsafe fn platform_init() {
// Initialize the process global map
MAP = Box::into_raw(Default::default());
// Allocate our WASMTIME_PORT and make sure that it can be sent to so we
// can receive exceptions.
let me = mach_task_self();
@@ -289,7 +275,7 @@ unsafe fn handle_exception(request: &mut ExceptionRequest) -> bool {
let get_pc = |state: &ThreadState| state.__rip as *const u8;
let resume = |state: &mut ThreadState, pc: usize, jmp_buf: usize| {
let resume = |state: &mut ThreadState, pc: usize| {
// The x86_64 ABI requires a 16-byte stack alignment for
// functions, so typically we'll be 16-byte aligned. In this
// case we simulate a `call` instruction by decrementing the
@@ -315,7 +301,6 @@ unsafe fn handle_exception(request: &mut ExceptionRequest) -> bool {
}
state.__rip = unwind as u64;
state.__rdi = pc as u64;
state.__rsi = jmp_buf as u64;
};
let mut thread_state = ThreadState::new();
} else if #[cfg(target_arch = "aarch64")] {
@@ -325,18 +310,17 @@ unsafe fn handle_exception(request: &mut ExceptionRequest) -> bool {
let get_pc = |state: &ThreadState| state.__pc as *const u8;
let resume = |state: &mut ThreadState, pc: usize, jmp_buf: usize| {
let resume = |state: &mut ThreadState, pc: usize| {
// Clobber LR with the faulting PC, so unwinding resumes at the
// faulting instruction. The previous value of LR has been saved
// by the callee (in Cranelift generated code), so no need to
// stash it.
state.__lr = pc as u64;
// Fill in the 2 arguments to unwind here, and set PC to it, so
// Fill in the argument to unwind here, and set PC to it, so
// it looks like a call to unwind.
state.__pc = unwind as u64;
state.__x[0] = pc as u64;
state.__x[1] = jmp_buf as u64;
state.__pc = unwind as u64;
};
let mut thread_state = mem::zeroed::<ThreadState>();
} else {
@@ -372,19 +356,7 @@ unsafe fn handle_exception(request: &mut ExceptionRequest) -> bool {
// pointer value and if `MAP` changes happen after we read our entry that's
// ok since they won't invalidate our entry.
let pc = get_pc(&thread_state);
let state = (*MAP)
.lock()
.unwrap_or_else(|e| e.into_inner())
.get(&origin_thread)
.copied();
let jmp_buf = match state {
Some(state) => (*state).jmp_buf_if_trap(pc, |_| false),
None => ptr::null(),
};
if jmp_buf.is_null() {
return false;
}
if jmp_buf as usize == 1 {
if !super::IS_WASM_PC(pc as usize) {
return false;
}
@@ -392,7 +364,7 @@ unsafe fn handle_exception(request: &mut ExceptionRequest) -> bool {
// force the thread itself to trap. The thread's register state is
// configured to resume in the `unwind` function below, we update the
// thread's register state, and then we're off to the races.
resume(&mut thread_state, pc as usize, jmp_buf as usize);
resume(&mut thread_state, pc as usize);
let kret = thread_set_state(
origin_thread,
thread_state_flavor,
@@ -409,13 +381,13 @@ unsafe fn handle_exception(request: &mut ExceptionRequest) -> bool {
/// a native backtrace once we've switched back to the thread itself. After
/// the backtrace is captured we can do the usual `longjmp` back to the source
/// of the wasm code.
unsafe extern "C" fn unwind(wasm_pc: *const u8, jmp_buf: *const u8) -> ! {
tls::with(|state| {
if let Some(state) = state {
unsafe extern "C" fn unwind(wasm_pc: *const u8) -> ! {
let jmp_buf = tls::with(|state| {
let state = state.unwrap();
state.capture_backtrace(wasm_pc);
}
state.jmp_buf.get()
});
debug_assert!(!jmp_buf.is_null());
Unwind(jmp_buf);
}
@@ -474,23 +446,3 @@ pub fn lazy_per_thread_init() -> Result<(), Trap> {
});
Ok(())
}
/// This hook is invoked whenever TLS state for the current thread is updated
/// to the `ptr` specified.
///
/// The purpose for hooking this on macOS is we register in a process-global map
/// that our mach thread's state is `ptr` at this time. This allows the
/// exception handling thread to lookup in this map later if our thread
/// generates an exception.
///
/// Note that in general this is quite unsafe since we're moving non-Send state
/// (`ptr`) which is also only valid for a short portion of the program (it
/// lives on the stack) into a global portion of the program. This needs to be
/// kept tightly in sync with `handle_exception` above where it's accessed in a
/// very limited fashion.
pub fn register_tls(ptr: *const CallThreadState<'static>) {
unsafe {
let me = MY_PORT.with(|p| p.0);
(*MAP).lock().unwrap().insert(me, ptr);
}
}

View File

@@ -1,4 +1,4 @@
use crate::traphandlers::{tls, CallThreadState, Trap, Unwind};
use crate::traphandlers::{tls, Trap, Unwind};
use std::cell::RefCell;
use std::convert::TryInto;
use std::io;
@@ -250,7 +250,3 @@ pub fn lazy_per_thread_init() -> Result<(), Trap> {
}
}
}
pub fn register_tls(_: *const CallThreadState<'static>) {
// Unused on unix
}

View File

@@ -1,4 +1,4 @@
use crate::traphandlers::{tls, CallThreadState, Trap, Unwind};
use crate::traphandlers::{tls, Trap, Unwind};
use std::io;
use winapi::um::errhandlingapi::*;
use winapi::um::minwinbase::*;
@@ -77,7 +77,3 @@ pub fn lazy_per_thread_init() -> Result<(), Trap> {
// Unused on Windows
Ok(())
}
pub fn register_tls(_: *const CallThreadState<'static>) {
// Unused on Windows
}

View File

@@ -35,6 +35,8 @@ serde = { version = "1.0.94", features = ["derive"] }
bincode = "1.2.1"
indexmap = "1.6"
paste = "1.0.3"
psm = "0.1.11"
lazy_static = "1.4"
[target.'cfg(target_os = "windows")'.dependencies]
winapi = "0.3.7"

View File

@@ -1,12 +1,22 @@
use std::cmp;
use std::collections::BTreeMap;
use std::sync::Arc;
use std::sync::Mutex;
use wasmtime_environ::entity::EntityRef;
use wasmtime_environ::ir;
use wasmtime_environ::wasm::FuncIndex;
use wasmtime_environ::{FunctionAddressMap, Module, TrapInformation};
use wasmtime_jit::{CompiledModule, SymbolizeContext};
/// This is a structure that lives within a `Store` and retains information
/// about all wasm code registered with the `Store` (e.g. modules that have
/// been instantiated into a store).
///
/// "frame information" here refers to things like determining whether a
/// program counter is a wasm program counter, and additionally mapping program
/// counters to wasm filenames, modules, line numbers, etc. This store of
/// information lives as long as a `Store` lives since modules are never
/// unloaded today.
#[derive(Default)]
pub struct StoreFrameInfo {
/// An internal map that keeps track of backtrace frame information for
@@ -21,14 +31,18 @@ pub struct StoreFrameInfo {
ranges: BTreeMap<usize, ModuleFrameInfo>,
}
/// This is a listing of information for each module registered with a store
/// which lives in `StoreFrameInfo`.
struct ModuleFrameInfo {
start: usize,
functions: BTreeMap<usize, FunctionInfo>,
functions: Arc<BTreeMap<usize, FunctionInfo>>,
module: Arc<Module>,
symbolize: Option<SymbolizeContext>,
has_unparsed_debuginfo: bool,
}
/// Information about a function, specifically information about individual
/// traps and such.
struct FunctionInfo {
start: usize,
index: FuncIndex,
@@ -45,26 +59,7 @@ impl StoreFrameInfo {
/// information due to the compiler's configuration.
pub fn lookup_frame_info(&self, pc: usize) -> Option<(FrameInfo, bool)> {
let (module, func) = self.func(pc)?;
// Use our relative position from the start of the function to find the
// machine instruction that corresponds to `pc`, which then allows us to
// map that to a wasm original source location.
let rel_pos = (pc - func.start) as u32;
let pos = match func
.instr_map
.instructions
.binary_search_by_key(&rel_pos, |map| map.code_offset)
{
// Exact hit!
Ok(pos) => Some(pos),
// This *would* be at the first slot in the array, so no
// instructions cover `pc`.
Err(0) => None,
// This would be at the `nth` slot, so we're at the `n-1`th slot.
Err(n) => Some(n - 1),
};
let pos = func.instr_pos(pc);
// In debug mode for now assert that we found a mapping for `pc` within
// the function, because otherwise something is buggy along the way and
@@ -138,15 +133,7 @@ impl StoreFrameInfo {
}
fn func(&self, pc: usize) -> Option<(&ModuleFrameInfo, &FunctionInfo)> {
let (end, info) = self.ranges.range(pc..).next()?;
if pc < info.start || *end < pc {
return None;
}
let (end, func) = info.functions.range(pc..).next()?;
if pc < func.start || *end < pc {
return None;
}
Some((info, func))
func(pc, &self.ranges, |t| (t.start, &t.functions))
}
/// Registers a new compiled module's frame information.
@@ -183,6 +170,7 @@ impl StoreFrameInfo {
if functions.len() == 0 {
return;
}
let functions = Arc::new(functions);
// First up assert that our chunk of jit functions doesn't collide with
// any other known chunks of jit functions...
@@ -194,6 +182,7 @@ impl StoreFrameInfo {
}
// ... then insert our range and assert nothing was there previously
GLOBAL_INFO.lock().unwrap().register(min, max, &functions);
let prev = self.ranges.insert(
max,
ModuleFrameInfo {
@@ -208,6 +197,138 @@ impl StoreFrameInfo {
}
}
impl FunctionInfo {
fn instr_pos(&self, pc: usize) -> Option<usize> {
// Use our relative position from the start of the function to find the
// machine instruction that corresponds to `pc`, which then allows us to
// map that to a wasm original source location.
let rel_pos = (pc - self.start) as u32;
match self
.instr_map
.instructions
.binary_search_by_key(&rel_pos, |map| map.code_offset)
{
// Exact hit!
Ok(pos) => Some(pos),
// This *would* be at the first slot in the array, so no
// instructions cover `pc`.
Err(0) => None,
// This would be at the `nth` slot, so we're at the `n-1`th slot.
Err(n) => Some(n - 1),
}
}
}
impl Drop for StoreFrameInfo {
fn drop(&mut self) {
let mut info = GLOBAL_INFO.lock().unwrap();
for end in self.ranges.keys() {
info.unregister(*end);
}
}
}
/// This is the dual of `StoreFrameInfo` and is stored globally (as the name
/// implies) rather than simply in one `Store`.
///
/// The purpose of this map is to be called from signal handlers to determine
/// whether a program counter is a wasm trap or not. Specifically macOS has
/// no contextual information about the thread available, hence the necessity
/// for global state rather than using thread local state.
///
/// This is similar to `StoreFrameInfo` except that it has less information and
/// supports removal. Any time anything is registered with a `StoreFrameInfo`
/// it is also automatically registered with the singleton global frame
/// information. When a `StoreFrameInfo` is destroyed then all of its entries
/// are removed from the global frame information.
#[derive(Default)]
pub(crate) struct GlobalFrameInfo {
// The map here behaves the same way as `StoreFrameInfo`.
ranges: BTreeMap<usize, GlobalModuleFrameInfo>,
}
/// This is the equivalent of `ModuleFrameInfo` except has less code and is
/// stored within `GlobalFrameInfo`.
struct GlobalModuleFrameInfo {
start: usize,
functions: Arc<BTreeMap<usize, FunctionInfo>>,
/// Note that modules can be instantiated in many stores, so the purpose of
/// this field is to keep track of how many stores have registered a
/// module. Information is only removed from the global store when this
/// reference count reaches 0.
references: usize,
}
lazy_static::lazy_static! {
static ref GLOBAL_INFO: Mutex<GlobalFrameInfo> = Default::default();
}
impl GlobalFrameInfo {
/// Returns whether the `pc`, according to globally registered information,
/// is a wasm trap or not.
pub(crate) fn is_wasm_pc(pc: usize) -> bool {
let info = GLOBAL_INFO.lock().unwrap();
match func(pc, &info.ranges, |i| (i.start, &i.functions)) {
Some((_, info)) => info.instr_pos(pc).is_some(),
None => false,
}
}
/// Registers a new region of code, described by `(start, end)` and with
/// the given function information, with the global information.
fn register(
&mut self,
start: usize,
end: usize,
functions: &Arc<BTreeMap<usize, FunctionInfo>>,
) {
let info = self
.ranges
.entry(end)
.or_insert_with(|| GlobalModuleFrameInfo {
start,
functions: functions.clone(),
references: 0,
});
// Note that ideally we'd debug_assert that the information previously
// stored, if any, matches the `functions` we were given, but for now we
// just do some simple checks to hope it's the same.
assert_eq!(info.start, start);
assert_eq!(info.functions.len(), functions.len());
info.references += 1;
}
/// Unregisters a region of code (keyed by the `end` address) from this
/// global information.
fn unregister(&mut self, end: usize) {
let val = self.ranges.get_mut(&end).unwrap();
val.references -= 1;
if val.references == 0 {
self.ranges.remove(&end);
}
}
}
fn func<T>(
pc: usize,
ranges: &BTreeMap<usize, T>,
get_start_and_functions: impl FnOnce(&T) -> (usize, &BTreeMap<usize, FunctionInfo>),
) -> Option<(&T, &FunctionInfo)> {
let (end, info) = ranges.range(pc..).next()?;
let (start, functions) = get_start_and_functions(info);
if pc < start || *end < pc {
return None;
}
let (end, func) = functions.range(pc..).next()?;
if pc < func.start || *end < pc {
return None;
}
Some((info, func))
}
/// Description of a frame in a backtrace for a [`Trap`].
///
/// Whenever a WebAssembly trap occurs an instance of [`Trap`] is created. Each

View File

@@ -10,6 +10,7 @@ use std::mem;
use std::panic::{self, AssertUnwindSafe};
use std::pin::Pin;
use std::ptr::{self, NonNull};
use std::sync::atomic::Ordering::Relaxed;
use wasmtime_environ::wasm::{EntityIndex, FuncIndex};
use wasmtime_runtime::{
raise_user_trap, ExportFunction, InstanceAllocator, InstanceHandle, OnDemandInstanceAllocator,
@@ -1149,20 +1150,109 @@ impl fmt::Debug for Func {
}
}
#[inline]
pub(crate) fn invoke_wasm_and_catch_traps(
store: &Store,
closure: impl FnMut(),
) -> Result<(), Trap> {
unsafe {
let canary = 0;
let _auto_reset_canary = store
.externref_activations_table()
.set_stack_canary(&canary);
let _reset = if store.externref_activations_table().stack_canary().is_some() {
None
} else {
Some(enter_wasm_init(store)?)
};
wasmtime_runtime::catch_traps(store, closure).map_err(|e| Trap::from_runtime(store, e))
}
}
/// This function is called to register state within `Store` whenever
/// WebAssembly is entered for the first time within the `Store`. This isn't
/// called when wasm is called recursively within the `Store`.
///
/// This function sets up various limits such as:
///
/// * The stack limit. This is what ensures that we limit the stack space
/// allocated by WebAssembly code and it's relative to the initial stack
/// pointer that called into wasm.
///
/// * Stack canaries for externref gc tracing. Currently the implementation
/// relies on walking frames but the stack walker isn't always 100% reliable,
/// so a canary is used to ensure that if the canary is seen then it's
/// guaranteed all wasm frames have been walked.
///
/// This function may fail if the the stack limit can't be set because an
/// interrupt already happened. Otherwise it returns a value that resets the
/// various limits on `Drop`.
#[inline]
fn enter_wasm_init<'a>(store: &'a Store) -> Result<impl Drop + 'a, Trap> {
let stack_pointer = psm::stack_pointer() as usize;
// Determine the stack pointer where, after which, any wasm code will
// immediately trap. This is checked on the entry to all wasm functions.
//
// Note that this isn't 100% precise. We are requested to give wasm
// `max_wasm_stack` bytes, but what we're actually doing is giving wasm
// probably a little less than `max_wasm_stack` because we're
// calculating the limit relative to this function's approximate stack
// pointer. Wasm will be executed on a frame beneath this one (or next
// 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.
//
// After we've got the stack limit then we store it into the `stack_limit`
// variable. Note that the store is an atomic swap to ensure that we can
// consume any previously-sent interrupt requests. If we found that wasm was
// previously interrupted then we immediately return a trap (after resetting
// the stack limit). Otherwise we're good to keep on going.
//
// Note the usage of `Relaxed` memory orderings here. This is specifically
// an optimization in the `Drop` below where a `Relaxed` store is speedier
// than a `SeqCst` store. The rationale for `Relaxed` here is that the
// atomic orderings here aren't actually protecting any memory, we're just
// trying to be atomic with respect to this one location in memory (for when
// `InterruptHandle` sends us a signal). Due to the lack of needing to
// synchronize with any other memory it's hoped that the choice of `Relaxed`
// here should be correct for our use case.
let wasm_stack_limit = stack_pointer - store.engine().config().max_wasm_stack;
let interrupts = store.interrupts();
match interrupts.stack_limit.swap(wasm_stack_limit, Relaxed) {
wasmtime_environ::INTERRUPTED => {
// This means that an interrupt happened before we actually
// called this function, which means that we're now
// considered interrupted.
interrupts.stack_limit.store(usize::max_value(), Relaxed);
return Err(Trap::new_wasm(
Some(store),
None,
wasmtime_environ::ir::TrapCode::Interrupt,
backtrace::Backtrace::new_unresolved(),
));
}
n => debug_assert_eq!(usize::max_value(), n),
}
store
.externref_activations_table()
.set_stack_canary(Some(stack_pointer));
return Ok(Reset(store));
struct Reset<'a>(&'a Store);
impl Drop for Reset<'_> {
#[inline]
fn drop(&mut self) {
self.0.externref_activations_table().set_stack_canary(None);
// see docs above for why this uses `Relaxed`
self.0
.interrupts()
.stack_limit
.store(usize::max_value(), Relaxed);
}
}
}
/// A trait implemented for types which can be returned from closures passed to
/// [`Func::wrap`] and friends.
///

View File

@@ -93,8 +93,6 @@ where
));
}
let anyfunc = self.func.export.anyfunc.as_ref();
let trampoline = self.func.trampoline;
let params = MaybeUninit::new(params);
let mut ret = MaybeUninit::uninit();
let mut called = false;
@@ -102,9 +100,10 @@ where
let result = invoke_wasm_and_catch_traps(&self.func.instance.store, || {
called = true;
let params = ptr::read(params.as_ptr());
let anyfunc = self.func.export.anyfunc.as_ref();
let result = params.invoke::<Results>(
&self.func.instance.store,
trampoline,
self.func.trampoline,
anyfunc.func_ptr.as_ptr(),
anyfunc.vmctx,
ptr::null_mut(),

View File

@@ -1,3 +1,4 @@
use crate::frame_info;
use crate::frame_info::StoreFrameInfo;
use crate::sig_registry::SignatureRegistry;
use crate::trampoline::StoreInstanceHandle;
@@ -136,7 +137,8 @@ impl Store {
// once-per-thread. Platforms like Unix, however, only require this
// once-per-program. In any case this is safe to call many times and
// each one that's not relevant just won't do anything.
wasmtime_runtime::init_traps().expect("failed to initialize trap handling");
wasmtime_runtime::init_traps(frame_info::GlobalFrameInfo::is_wasm_pc)
.expect("failed to initialize trap handling");
Store {
inner: Rc::new(StoreInner {
@@ -401,6 +403,7 @@ impl Store {
*self.inner.signal_handler.borrow_mut() = handler;
}
#[inline]
pub(crate) fn interrupts(&self) -> &VMInterrupts {
&self.inner.interrupts
}
@@ -924,10 +927,6 @@ unsafe impl TrapInfo for Store {
self
}
fn is_wasm_trap(&self, addr: usize) -> bool {
self.frame_info().borrow().lookup_trap_info(addr).is_some()
}
fn custom_signal_handler(&self, call: &dyn Fn(&SignalHandler) -> bool) -> bool {
if let Some(handler) = &*self.inner.signal_handler.borrow() {
return call(handler);
@@ -935,11 +934,6 @@ unsafe impl TrapInfo for Store {
false
}
#[inline]
fn max_wasm_stack(&self) -> usize {
self.engine().config().max_wasm_stack
}
fn out_of_gas(&self) {
match self.inner.out_of_gas_behavior.get() {
OutOfGas::Trap => self.out_of_gas_trap(),

View File

@@ -182,7 +182,7 @@ impl Trap {
}
}
fn new_wasm(
pub(crate) fn new_wasm(
store: Option<&Store>,
trap_pc: Option<usize>,
code: ir::TrapCode,