Code review feedback.
* Make `FunctionInfo` public and `CompiledModule::func_info` return it. * Make the `StackMapLookup` trait unsafe. * Add comments for the purpose of `EngineHostFuncs`. * Rework ownership model of shared signatures: `SignatureCollection` in conjunction with `SignatureRegistry` is now used so that the `Engine`, `Store`, and `Module` don't need to worry about unregistering shared signatures. * Implement `Func::param_arity` and `Func::result_arity` in terms of `Func::ty`. * Make looking up a trampoline with the module registry more efficient by doing a binary search on the function's starting PC value for the owning module and then looking up the trampoline with only that module. * Remove reference to the shared signatures from `GlobalRegisteredModule`.
This commit is contained in:
@@ -176,11 +176,13 @@ struct FinishedFunctions(PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>);
|
||||
unsafe impl Send for FinishedFunctions {}
|
||||
unsafe impl Sync for FinishedFunctions {}
|
||||
|
||||
/// Information about a function, such as trap information, address map,
|
||||
/// and stack maps.
|
||||
#[derive(Serialize, Deserialize, Clone)]
|
||||
struct FunctionInfo {
|
||||
traps: Vec<TrapInformation>,
|
||||
address_map: FunctionAddressMap,
|
||||
stack_maps: Vec<StackMapInformation>,
|
||||
pub struct FunctionInfo {
|
||||
pub traps: Vec<TrapInformation>,
|
||||
pub address_map: FunctionAddressMap,
|
||||
pub stack_maps: Vec<StackMapInformation>,
|
||||
}
|
||||
|
||||
/// This is intended to mirror the type tables in `wasmtime_environ`, except that
|
||||
@@ -362,18 +364,10 @@ impl CompiledModule {
|
||||
}
|
||||
|
||||
/// Gets the function information for a given function index.
|
||||
pub fn func_info(
|
||||
&self,
|
||||
index: DefinedFuncIndex,
|
||||
) -> (
|
||||
&FunctionAddressMap,
|
||||
&[TrapInformation],
|
||||
&[StackMapInformation],
|
||||
) {
|
||||
pub fn func_info(&self, index: DefinedFuncIndex) -> &FunctionInfo {
|
||||
self.artifacts
|
||||
.funcs
|
||||
.get(index)
|
||||
.map(|f| (&f.address_map, f.traps.as_ref(), f.stack_maps.as_ref()))
|
||||
.expect("defined function should be present")
|
||||
}
|
||||
|
||||
|
||||
@@ -742,14 +742,22 @@ impl VMExternRefActivationsTable {
|
||||
}
|
||||
|
||||
/// Used by the runtime to lookup a stack map from a PC value.
|
||||
pub trait StackMapLookup {
|
||||
///
|
||||
/// # Safety
|
||||
///
|
||||
/// This trait is unsafe as it returns pointers to a stack map without
|
||||
/// any clear ownership.
|
||||
///
|
||||
/// It is the responsibility of the caller to not have the pointer outlive
|
||||
/// the stack map lookup trait object.
|
||||
pub unsafe trait StackMapLookup {
|
||||
/// Lookup the stack map at a program counter (PC) value.
|
||||
fn lookup(&self, pc: usize) -> Option<*const StackMap>;
|
||||
}
|
||||
|
||||
pub(crate) struct EmptyStackMapLookup;
|
||||
|
||||
impl StackMapLookup for EmptyStackMapLookup {
|
||||
unsafe impl StackMapLookup for EmptyStackMapLookup {
|
||||
fn lookup(&self, _pc: usize) -> Option<*const StackMap> {
|
||||
None
|
||||
}
|
||||
|
||||
@@ -1,24 +1,34 @@
|
||||
use crate::signatures::{SharedSignatures, SignatureRegistry, TrampolineMap};
|
||||
use crate::signatures::{SignatureCollection, SignatureRegistry};
|
||||
use crate::Config;
|
||||
use anyhow::Result;
|
||||
use std::collections::HashMap;
|
||||
use std::sync::{Arc, RwLock};
|
||||
use std::sync::Arc;
|
||||
#[cfg(feature = "cache")]
|
||||
use wasmtime_cache::CacheConfig;
|
||||
use wasmtime_environ::{
|
||||
entity::PrimaryMap,
|
||||
wasm::{SignatureIndex, WasmFuncType},
|
||||
};
|
||||
use wasmtime_jit::Compiler;
|
||||
use wasmtime_runtime::{
|
||||
debug_builtins, InstanceAllocator, InstanceHandle, VMCallerCheckedAnyfunc,
|
||||
VMSharedSignatureIndex, VMTrampoline,
|
||||
};
|
||||
use wasmtime_runtime::{debug_builtins, InstanceAllocator, InstanceHandle, VMCallerCheckedAnyfunc};
|
||||
|
||||
#[derive(Default)]
|
||||
/// This is used as a Send+Sync wrapper around two data structures relating to
|
||||
/// host functions defined on `Config`:
|
||||
///
|
||||
/// * `anyfuncs` - this stores a mapping between the host function instance and
|
||||
/// a `VMCallerCheckedAnyfunc` that can be used as the function's value in Wasmtime's ABI.
|
||||
/// The address of the anyfunc needs to be stable, thus the boxed value.
|
||||
///
|
||||
/// * `signatures` - this stores the collection of shared signatures registered for every
|
||||
/// usable host functions with this engine.
|
||||
struct EngineHostFuncs {
|
||||
anyfuncs: HashMap<InstanceHandle, Box<VMCallerCheckedAnyfunc>>,
|
||||
trampolines: TrampolineMap,
|
||||
signatures: SignatureCollection,
|
||||
}
|
||||
|
||||
impl EngineHostFuncs {
|
||||
fn new(registry: &SignatureRegistry) -> Self {
|
||||
Self {
|
||||
anyfuncs: HashMap::new(),
|
||||
signatures: SignatureCollection::new(registry),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// This is safe for send and sync as it is read-only once the
|
||||
@@ -58,18 +68,10 @@ struct EngineInner {
|
||||
config: Config,
|
||||
compiler: Compiler,
|
||||
allocator: Box<dyn InstanceAllocator>,
|
||||
signatures: RwLock<SignatureRegistry>,
|
||||
signatures: SignatureRegistry,
|
||||
host_funcs: EngineHostFuncs,
|
||||
}
|
||||
|
||||
impl Drop for EngineInner {
|
||||
fn drop(&mut self) {
|
||||
let mut signatures = self.signatures.write().unwrap();
|
||||
signatures.unregister(self.host_funcs.trampolines.indexes());
|
||||
assert!(signatures.is_empty());
|
||||
}
|
||||
}
|
||||
|
||||
impl Engine {
|
||||
/// Creates a new [`Engine`] with the specified compilation and
|
||||
/// configuration settings.
|
||||
@@ -77,20 +79,20 @@ impl Engine {
|
||||
debug_builtins::ensure_exported();
|
||||
config.validate()?;
|
||||
let allocator = config.build_allocator()?;
|
||||
let mut signatures = SignatureRegistry::default();
|
||||
let mut host_funcs = EngineHostFuncs::default();
|
||||
let registry = SignatureRegistry::new();
|
||||
let mut host_funcs = EngineHostFuncs::new(®istry);
|
||||
|
||||
// Register all the host function signatures
|
||||
// Register all the host function signatures with the collection
|
||||
for func in config.host_funcs() {
|
||||
let sig = signatures.register(func.ty.as_wasm_func_type());
|
||||
let sig = host_funcs
|
||||
.signatures
|
||||
.register(func.ty.as_wasm_func_type(), func.trampoline);
|
||||
|
||||
// Cloning the instance handle is safe as host functions outlive the engine
|
||||
host_funcs.anyfuncs.insert(
|
||||
unsafe { func.instance.clone() },
|
||||
Box::new(func.anyfunc(sig)),
|
||||
);
|
||||
|
||||
host_funcs.trampolines.insert(sig, func.trampoline);
|
||||
}
|
||||
|
||||
Ok(Engine {
|
||||
@@ -98,7 +100,7 @@ impl Engine {
|
||||
config: config.clone(),
|
||||
compiler: config.build_compiler(allocator.as_ref()),
|
||||
allocator,
|
||||
signatures: RwLock::new(signatures),
|
||||
signatures: registry,
|
||||
host_funcs,
|
||||
}),
|
||||
})
|
||||
@@ -128,40 +130,12 @@ impl Engine {
|
||||
Arc::ptr_eq(&a.inner, &b.inner)
|
||||
}
|
||||
|
||||
pub(crate) fn register_module_signatures(
|
||||
&self,
|
||||
signatures: &PrimaryMap<SignatureIndex, WasmFuncType>,
|
||||
trampolines: impl Iterator<Item = (SignatureIndex, VMTrampoline)>,
|
||||
) -> (SharedSignatures, TrampolineMap) {
|
||||
self.inner
|
||||
.signatures
|
||||
.write()
|
||||
.unwrap()
|
||||
.register_module(signatures, trampolines)
|
||||
pub(crate) fn signatures(&self) -> &SignatureRegistry {
|
||||
&self.inner.signatures
|
||||
}
|
||||
|
||||
pub(crate) fn register_signature(&self, ty: &WasmFuncType) -> VMSharedSignatureIndex {
|
||||
self.inner.signatures.write().unwrap().register(ty)
|
||||
}
|
||||
|
||||
pub(crate) fn unregister_signatures(
|
||||
&self,
|
||||
indexes: impl Iterator<Item = VMSharedSignatureIndex>,
|
||||
) {
|
||||
self.inner.signatures.write().unwrap().unregister(indexes);
|
||||
}
|
||||
|
||||
pub(crate) fn lookup_func_type(&self, index: VMSharedSignatureIndex) -> Option<WasmFuncType> {
|
||||
self.inner
|
||||
.signatures
|
||||
.read()
|
||||
.unwrap()
|
||||
.lookup_type(index)
|
||||
.cloned()
|
||||
}
|
||||
|
||||
pub(crate) fn host_func_trampolines(&self) -> &TrampolineMap {
|
||||
&self.inner.host_funcs.trampolines
|
||||
pub(crate) fn host_func_signatures(&self) -> &SignatureCollection {
|
||||
&self.inner.host_funcs.signatures
|
||||
}
|
||||
|
||||
pub(crate) fn host_func_anyfunc(
|
||||
|
||||
@@ -778,31 +778,20 @@ impl Func {
|
||||
self.instance
|
||||
.store
|
||||
.engine()
|
||||
.lookup_func_type(self.sig_index())
|
||||
.signatures()
|
||||
.lookup_type(self.sig_index())
|
||||
.expect("signature should be registered"),
|
||||
)
|
||||
}
|
||||
|
||||
/// Returns the number of parameters that this function takes.
|
||||
pub fn param_arity(&self) -> usize {
|
||||
let sig = self
|
||||
.instance
|
||||
.store
|
||||
.engine()
|
||||
.lookup_func_type(self.sig_index())
|
||||
.expect("signature should be registered");
|
||||
sig.params.len()
|
||||
self.ty().params().len()
|
||||
}
|
||||
|
||||
/// Returns the number of results this function produces.
|
||||
pub fn result_arity(&self) -> usize {
|
||||
let sig = self
|
||||
.instance
|
||||
.store
|
||||
.engine()
|
||||
.lookup_func_type(self.sig_index())
|
||||
.expect("signature should be registered");
|
||||
sig.returns.len()
|
||||
self.ty().results().len()
|
||||
}
|
||||
|
||||
/// Invokes this function with the `params` given, returning the results and
|
||||
@@ -927,7 +916,7 @@ impl Func {
|
||||
Func {
|
||||
instance: store.existing_vmctx(anyfunc.vmctx),
|
||||
export: export.clone(),
|
||||
trampoline: store.lookup_trampoline(anyfunc.type_index),
|
||||
trampoline: store.lookup_trampoline(&*anyfunc),
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1813,7 +1802,7 @@ macro_rules! impl_into_func {
|
||||
// If not given a store, use a default signature index that is guaranteed to trap.
|
||||
// If the function is called indirectly without first being associated with a store (a bug condition).
|
||||
let shared_signature_id = store
|
||||
.map(|s| s.register_signature(ty.as_wasm_func_type(), trampoline))
|
||||
.map(|s| s.signatures().borrow_mut().register(ty.as_wasm_func_type(), trampoline))
|
||||
.unwrap_or(VMSharedSignatureIndex::default());
|
||||
|
||||
let instance = unsafe {
|
||||
|
||||
@@ -13,9 +13,9 @@ use wasmtime_environ::wasm::{
|
||||
};
|
||||
use wasmtime_environ::Initializer;
|
||||
use wasmtime_runtime::{
|
||||
Imports, InstanceAllocationRequest, InstantiationError, RuntimeInstance, StackMapLookup,
|
||||
VMContext, VMExternRefActivationsTable, VMFunctionBody, VMFunctionImport, VMGlobalImport,
|
||||
VMMemoryImport, VMTableImport,
|
||||
Imports, InstanceAllocationRequest, InstantiationError, RuntimeInstance, VMContext,
|
||||
VMExternRefActivationsTable, VMFunctionBody, VMFunctionImport, VMGlobalImport, VMMemoryImport,
|
||||
VMTableImport,
|
||||
};
|
||||
|
||||
/// An instantiated WebAssembly module.
|
||||
@@ -506,10 +506,9 @@ impl<'a> Instantiator<'a> {
|
||||
fn instantiate_raw(&self) -> Result<StoreInstanceHandle> {
|
||||
let compiled_module = self.cur.module.compiled_module();
|
||||
|
||||
// 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`.
|
||||
self.store.register_module(&self.cur.module);
|
||||
// Register the module just before instantiation to ensure we keep the module
|
||||
// properly referenced while in use by the store.
|
||||
self.store.modules().borrow_mut().register(&self.cur.module);
|
||||
|
||||
unsafe {
|
||||
let engine = self.store.engine();
|
||||
@@ -519,13 +518,13 @@ impl<'a> Instantiator<'a> {
|
||||
module: compiled_module.module().clone(),
|
||||
finished_functions: compiled_module.finished_functions(),
|
||||
imports: self.cur.build(),
|
||||
shared_signatures: self.cur.module.signatures().into(),
|
||||
shared_signatures: self.cur.module.signatures().as_module_map().into(),
|
||||
host_state: Box::new(()),
|
||||
interrupts: self.store.interrupts(),
|
||||
externref_activations_table: self.store.externref_activations_table()
|
||||
as *const VMExternRefActivationsTable
|
||||
as *mut _,
|
||||
stack_map_lookup: Some(self.store.stack_map_lookup() as *const dyn StackMapLookup),
|
||||
stack_map_lookup: Some(std::mem::transmute(self.store.stack_map_lookup())),
|
||||
})?;
|
||||
|
||||
// After we've created the `InstanceHandle` we still need to run
|
||||
|
||||
@@ -1,5 +1,5 @@
|
||||
use crate::{
|
||||
signatures::{SharedSignatures, TrampolineMap},
|
||||
signatures::SignatureCollection,
|
||||
types::{ExportType, ExternType, ImportType},
|
||||
};
|
||||
use crate::{Engine, ModuleType};
|
||||
@@ -11,9 +11,8 @@ use wasmparser::Validator;
|
||||
#[cfg(feature = "cache")]
|
||||
use wasmtime_cache::ModuleCacheEntry;
|
||||
use wasmtime_environ::entity::PrimaryMap;
|
||||
use wasmtime_environ::wasm::{ModuleIndex, SignatureIndex};
|
||||
use wasmtime_environ::wasm::ModuleIndex;
|
||||
use wasmtime_jit::{CompilationArtifacts, CompiledModule, TypeTables};
|
||||
use wasmtime_runtime::VMSharedSignatureIndex;
|
||||
|
||||
mod registry;
|
||||
mod serialization;
|
||||
@@ -21,26 +20,6 @@ mod serialization;
|
||||
pub use registry::{FrameInfo, FrameSymbol, GlobalModuleRegistry, ModuleRegistry};
|
||||
pub use serialization::SerializedModule;
|
||||
|
||||
// A wrapper around registered signatures and trampolines that will automatically
|
||||
/// unregister the signatures when dropped.
|
||||
pub(crate) struct ModuleSharedSignatures {
|
||||
engine: Engine,
|
||||
signatures: SharedSignatures,
|
||||
trampolines: TrampolineMap,
|
||||
}
|
||||
|
||||
impl Drop for ModuleSharedSignatures {
|
||||
fn drop(&mut self) {
|
||||
if !self.signatures.is_empty() {
|
||||
// Use the shared signatures map to unregister as not every registered
|
||||
// signature will have a trampoline, but every index in the trampoline map
|
||||
// will be present in the shared signatures map.
|
||||
self.engine
|
||||
.unregister_signatures(self.signatures.values().cloned());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A compiled WebAssembly module, ready to be instantiated.
|
||||
///
|
||||
/// A `Module` is a compiled in-memory representation of an input WebAssembly
|
||||
@@ -129,7 +108,7 @@ struct ModuleInner {
|
||||
/// modules.
|
||||
types: Arc<TypeTables>,
|
||||
/// Registered shared signature for the module.
|
||||
signatures: Arc<ModuleSharedSignatures>,
|
||||
signatures: Arc<SignatureCollection>,
|
||||
}
|
||||
|
||||
impl Module {
|
||||
@@ -350,10 +329,11 @@ impl Module {
|
||||
// Validate the module can be used with the current allocator
|
||||
engine.allocator().validate(modules[main_module].module())?;
|
||||
|
||||
let (signatures, trampolines) = engine.register_module_signatures(
|
||||
let signatures = Arc::new(SignatureCollection::new_for_module(
|
||||
engine.signatures(),
|
||||
&types.wasm_signatures,
|
||||
modules.iter().flat_map(|m| m.trampolines().iter().cloned()),
|
||||
);
|
||||
));
|
||||
|
||||
let module = modules.remove(main_module);
|
||||
|
||||
@@ -364,11 +344,7 @@ impl Module {
|
||||
types: Arc::new(types),
|
||||
artifact_upvars: modules,
|
||||
module_upvars: Vec::new(),
|
||||
signatures: Arc::new(ModuleSharedSignatures {
|
||||
engine: engine.clone(),
|
||||
signatures,
|
||||
trampolines,
|
||||
}),
|
||||
}),
|
||||
})
|
||||
}
|
||||
@@ -488,11 +464,7 @@ impl Module {
|
||||
&self.inner.types
|
||||
}
|
||||
|
||||
pub(crate) fn signatures(&self) -> &PrimaryMap<SignatureIndex, VMSharedSignatureIndex> {
|
||||
&self.inner.signatures.signatures
|
||||
}
|
||||
|
||||
pub(crate) fn shared_signatures(&self) -> &Arc<ModuleSharedSignatures> {
|
||||
pub(crate) fn signatures(&self) -> &Arc<SignatureCollection> {
|
||||
&self.inner.signatures
|
||||
}
|
||||
|
||||
|
||||
@@ -1,6 +1,6 @@
|
||||
//! Implements a registry of modules for a store.
|
||||
|
||||
use crate::{module::ModuleSharedSignatures, Module};
|
||||
use crate::{signatures::SignatureCollection, Module};
|
||||
use std::{
|
||||
collections::BTreeMap,
|
||||
sync::{Arc, Mutex},
|
||||
@@ -9,12 +9,17 @@ use wasmtime_environ::{
|
||||
entity::EntityRef, ir, wasm::DefinedFuncIndex, FunctionAddressMap, TrapInformation,
|
||||
};
|
||||
use wasmtime_jit::CompiledModule;
|
||||
use wasmtime_runtime::{VMSharedSignatureIndex, VMTrampoline};
|
||||
use wasmtime_runtime::{VMCallerCheckedAnyfunc, VMTrampoline};
|
||||
|
||||
lazy_static::lazy_static! {
|
||||
static ref GLOBAL_MODULES: Mutex<GlobalModuleRegistry> = Default::default();
|
||||
}
|
||||
|
||||
fn func_by_pc(module: &CompiledModule, pc: usize) -> Option<(DefinedFuncIndex, u32)> {
|
||||
let (index, start, _) = module.func_by_pc(pc)?;
|
||||
Some((index, (pc - start) as u32))
|
||||
}
|
||||
|
||||
/// Used for registering modules with a store.
|
||||
///
|
||||
/// The map is from the ending (exclusive) address for the module code to
|
||||
@@ -62,7 +67,7 @@ impl ModuleRegistry {
|
||||
let compiled_module = module.compiled_module();
|
||||
let (start, end) = compiled_module.code().range();
|
||||
|
||||
// Ignore modules with no code, finished functions, or if the module is already registered
|
||||
// Ignore modules with no code or finished functions
|
||||
if start == end || compiled_module.finished_functions().is_empty() {
|
||||
return;
|
||||
}
|
||||
@@ -71,7 +76,10 @@ impl ModuleRegistry {
|
||||
// may be a valid PC value
|
||||
let end = end - 1;
|
||||
|
||||
if self.0.get(&end).is_some() {
|
||||
// Ensure the module isn't already present in the registry
|
||||
// This is expected when a module is instantiated multiple times in the same store
|
||||
if let Some(m) = self.0.get(&end) {
|
||||
assert_eq!(m.start, start);
|
||||
return;
|
||||
}
|
||||
|
||||
@@ -89,7 +97,7 @@ impl ModuleRegistry {
|
||||
RegisteredModule {
|
||||
start,
|
||||
module: compiled_module.clone(),
|
||||
signatures: module.shared_signatures().clone(),
|
||||
signatures: module.signatures().clone(),
|
||||
},
|
||||
);
|
||||
assert!(prev.is_none());
|
||||
@@ -97,18 +105,10 @@ impl ModuleRegistry {
|
||||
GLOBAL_MODULES.lock().unwrap().register(start, end, module);
|
||||
}
|
||||
|
||||
/// Looks up a trampoline from a shared signature index.
|
||||
///
|
||||
/// This will search all modules associated with the store for a suitable trampoline
|
||||
/// given the shared signature index.
|
||||
pub fn lookup_trampoline(&self, index: VMSharedSignatureIndex) -> Option<VMTrampoline> {
|
||||
for (_, m) in &self.0 {
|
||||
if let Some(trampoline) = m.signatures.trampolines.get(index) {
|
||||
return Some(trampoline);
|
||||
}
|
||||
}
|
||||
|
||||
None
|
||||
/// Looks up a trampoline from an anyfunc.
|
||||
pub fn lookup_trampoline(&self, anyfunc: &VMCallerCheckedAnyfunc) -> Option<VMTrampoline> {
|
||||
let module = self.module(anyfunc.func_ptr.as_ptr() as usize)?;
|
||||
module.signatures.trampoline(anyfunc.type_index)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -124,7 +124,7 @@ impl Drop for ModuleRegistry {
|
||||
struct RegisteredModule {
|
||||
start: usize,
|
||||
module: Arc<CompiledModule>,
|
||||
signatures: Arc<ModuleSharedSignatures>,
|
||||
signatures: Arc<SignatureCollection>,
|
||||
}
|
||||
|
||||
impl RegisteredModule {
|
||||
@@ -138,9 +138,9 @@ impl RegisteredModule {
|
||||
/// Returns an object if this `pc` is known to this module, or returns `None`
|
||||
/// if no information can be found.
|
||||
pub fn lookup_frame_info(&self, pc: usize) -> Option<FrameInfo> {
|
||||
let (index, offset) = self.func(pc)?;
|
||||
let (addr_map, _, _) = self.module.func_info(index);
|
||||
let pos = Self::instr_pos(offset, addr_map);
|
||||
let (index, offset) = func_by_pc(&self.module, pc)?;
|
||||
let info = self.module.func_info(index);
|
||||
let pos = Self::instr_pos(offset, &info.address_map);
|
||||
|
||||
// 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
|
||||
@@ -149,8 +149,8 @@ impl RegisteredModule {
|
||||
debug_assert!(pos.is_some(), "failed to find instruction for {:x}", pc);
|
||||
|
||||
let instr = match pos {
|
||||
Some(pos) => addr_map.instructions[pos].srcloc,
|
||||
None => addr_map.start_srcloc,
|
||||
Some(pos) => info.address_map.instructions[pos].srcloc,
|
||||
None => info.address_map.start_srcloc,
|
||||
};
|
||||
|
||||
// Use our wasm-relative pc to symbolize this frame. If there's a
|
||||
@@ -193,25 +193,26 @@ impl RegisteredModule {
|
||||
func_index: index.index() as u32,
|
||||
func_name: module.func_names.get(&index).cloned(),
|
||||
instr,
|
||||
func_start: addr_map.start_srcloc,
|
||||
func_start: info.address_map.start_srcloc,
|
||||
symbols,
|
||||
})
|
||||
}
|
||||
|
||||
/// Fetches trap information about a program counter in a backtrace.
|
||||
pub fn lookup_trap_info(&self, pc: usize) -> Option<&TrapInformation> {
|
||||
let (index, offset) = self.func(pc)?;
|
||||
let (_, traps, _) = self.module.func_info(index);
|
||||
let idx = traps
|
||||
let (index, offset) = func_by_pc(&self.module, pc)?;
|
||||
let info = self.module.func_info(index);
|
||||
let idx = info
|
||||
.traps
|
||||
.binary_search_by_key(&offset, |info| info.code_offset)
|
||||
.ok()?;
|
||||
Some(&traps[idx])
|
||||
Some(&info.traps[idx])
|
||||
}
|
||||
|
||||
/// Looks up a stack map from a program counter
|
||||
pub fn lookup_stack_map(&self, pc: usize) -> Option<&ir::StackMap> {
|
||||
let (index, offset) = self.func(pc)?;
|
||||
let (_, _, stack_maps) = self.module.func_info(index);
|
||||
let (index, offset) = func_by_pc(&self.module, pc)?;
|
||||
let info = self.module.func_info(index);
|
||||
|
||||
// Do a binary search to find the stack map for the given offset.
|
||||
//
|
||||
@@ -255,7 +256,10 @@ impl RegisteredModule {
|
||||
// or reload from the stack slots (which would have been updated to
|
||||
// point to the moved objects).
|
||||
|
||||
let index = match stack_maps.binary_search_by_key(&offset, |i| i.code_offset) {
|
||||
let index = match info
|
||||
.stack_maps
|
||||
.binary_search_by_key(&offset, |i| i.code_offset)
|
||||
{
|
||||
// Exact hit.
|
||||
Ok(i) => i,
|
||||
|
||||
@@ -269,12 +273,7 @@ impl RegisteredModule {
|
||||
Err(i) => i - 1,
|
||||
};
|
||||
|
||||
Some(&stack_maps[index].stack_map)
|
||||
}
|
||||
|
||||
fn func(&self, pc: usize) -> Option<(DefinedFuncIndex, u32)> {
|
||||
let (index, start, _) = self.module.func_by_pc(pc)?;
|
||||
Some((index, (pc - start) as u32))
|
||||
Some(&info.stack_maps[index].stack_map)
|
||||
}
|
||||
|
||||
fn instr_pos(offset: u32, addr_map: &FunctionAddressMap) -> Option<usize> {
|
||||
@@ -298,6 +297,17 @@ impl RegisteredModule {
|
||||
}
|
||||
}
|
||||
|
||||
// Counterpart to `RegisteredModule`, but stored in the global registry.
|
||||
struct GlobalRegisteredModule {
|
||||
start: usize,
|
||||
module: Arc<CompiledModule>,
|
||||
/// 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 registry when this
|
||||
/// reference count reaches 0.
|
||||
references: usize,
|
||||
}
|
||||
|
||||
/// This is the global module registry that stores information for all modules
|
||||
/// that are currently in use by any `Store`.
|
||||
///
|
||||
@@ -318,18 +328,18 @@ impl GlobalModuleRegistry {
|
||||
/// 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_MODULES.lock().unwrap();
|
||||
let modules = GLOBAL_MODULES.lock().unwrap();
|
||||
|
||||
match info.0.range(pc..).next() {
|
||||
Some((end, info)) => {
|
||||
if pc < info.module.start || *end < pc {
|
||||
match modules.0.range(pc..).next() {
|
||||
Some((end, entry)) => {
|
||||
if pc < entry.start || *end < pc {
|
||||
return false;
|
||||
}
|
||||
|
||||
match info.module.func(pc) {
|
||||
match func_by_pc(&entry.module, pc) {
|
||||
Some((index, offset)) => {
|
||||
let (addr_map, _, _) = info.module.module.func_info(index);
|
||||
RegisteredModule::instr_pos(offset, addr_map).is_some()
|
||||
let info = entry.module.func_info(index);
|
||||
RegisteredModule::instr_pos(offset, &info.address_map).is_some()
|
||||
}
|
||||
None => false,
|
||||
}
|
||||
@@ -342,18 +352,15 @@ impl GlobalModuleRegistry {
|
||||
/// the given function information, with the global information.
|
||||
fn register(&mut self, start: usize, end: usize, module: &Module) {
|
||||
let info = self.0.entry(end).or_insert_with(|| GlobalRegisteredModule {
|
||||
module: RegisteredModule {
|
||||
start,
|
||||
module: module.compiled_module().clone(),
|
||||
signatures: module.shared_signatures().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.module.start, start);
|
||||
assert_eq!(info.start, start);
|
||||
info.references += 1;
|
||||
}
|
||||
|
||||
@@ -368,17 +375,6 @@ impl GlobalModuleRegistry {
|
||||
}
|
||||
}
|
||||
|
||||
/// This is the equivalent of `RegisteredModule` except it keeps a reference count.
|
||||
struct GlobalRegisteredModule {
|
||||
module: RegisteredModule,
|
||||
|
||||
/// 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 registry when this
|
||||
/// reference count reaches 0.
|
||||
references: usize,
|
||||
}
|
||||
|
||||
/// Description of a frame in a backtrace for a [`Trap`].
|
||||
///
|
||||
/// Whenever a WebAssembly trap occurs an instance of [`Trap`] is created. Each
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
//! Implements module serialization.
|
||||
|
||||
use super::{ModuleInner, ModuleSharedSignatures};
|
||||
use crate::{Engine, Module, OptLevel};
|
||||
use super::ModuleInner;
|
||||
use crate::{signatures::SignatureCollection, Engine, Module, OptLevel};
|
||||
use anyhow::{anyhow, bail, Context, Result};
|
||||
use bincode::Options;
|
||||
use serde::{Deserialize, Serialize};
|
||||
@@ -300,16 +300,11 @@ impl<'a> SerializedModule<'a> {
|
||||
.allocator()
|
||||
.validate(modules.last().unwrap().module())?;
|
||||
|
||||
let (signatures, trampolines) = engine.register_module_signatures(
|
||||
let signatures = Arc::new(SignatureCollection::new_for_module(
|
||||
engine.signatures(),
|
||||
&types.wasm_signatures,
|
||||
modules.iter().flat_map(|m| m.trampolines().iter().cloned()),
|
||||
);
|
||||
|
||||
let signatures = Arc::new(ModuleSharedSignatures {
|
||||
engine: engine.clone(),
|
||||
signatures,
|
||||
trampolines,
|
||||
});
|
||||
));
|
||||
|
||||
let module = modules.pop().unwrap();
|
||||
|
||||
@@ -347,7 +342,7 @@ impl<'a> SerializedModule<'a> {
|
||||
module_index: usize,
|
||||
artifact_upvars: &[usize],
|
||||
module_upvars: &[SerializedModuleUpvar],
|
||||
signatures: &Arc<ModuleSharedSignatures>,
|
||||
signatures: &Arc<SignatureCollection>,
|
||||
) -> Result<Module> {
|
||||
Ok(Module {
|
||||
inner: Arc::new(ModuleInner {
|
||||
|
||||
@@ -1,61 +1,108 @@
|
||||
//! Implement a registry of function signatures, for fast indirect call
|
||||
//! signature checking.
|
||||
|
||||
use std::collections::{hash_map::Entry, HashMap};
|
||||
use std::convert::TryFrom;
|
||||
use std::{
|
||||
collections::{hash_map::Entry, HashMap},
|
||||
sync::RwLock,
|
||||
};
|
||||
use std::{convert::TryFrom, sync::Arc};
|
||||
use wasmtime_environ::entity::PrimaryMap;
|
||||
use wasmtime_environ::wasm::{SignatureIndex, WasmFuncType};
|
||||
use wasmtime_runtime::{VMSharedSignatureIndex, VMTrampoline};
|
||||
|
||||
/// Represents a mapping of shared signature index to trampolines.
|
||||
/// Represents a collection of shared signatures.
|
||||
///
|
||||
/// This is used in various places to store trampolines associated with shared
|
||||
/// signature indexes.
|
||||
/// This is used to register shared signatures with a shared signature registry.
|
||||
///
|
||||
/// As multiple trampolines may exist for a single signature, the map entries
|
||||
/// are internally reference counted.
|
||||
#[derive(Default)]
|
||||
pub struct TrampolineMap(HashMap<VMSharedSignatureIndex, (usize, VMTrampoline)>);
|
||||
/// The collection will unregister any contained signatures with the registry
|
||||
/// when dropped.
|
||||
#[derive(Debug)]
|
||||
pub struct SignatureCollection {
|
||||
registry: Arc<RwLock<SignatureRegistryInner>>,
|
||||
signatures: PrimaryMap<SignatureIndex, VMSharedSignatureIndex>,
|
||||
trampolines: HashMap<VMSharedSignatureIndex, (usize, VMTrampoline)>,
|
||||
}
|
||||
|
||||
impl TrampolineMap {
|
||||
/// Inserts a trampoline into the map.
|
||||
pub fn insert(&mut self, index: VMSharedSignatureIndex, trampoline: VMTrampoline) {
|
||||
let entry = match self.0.entry(index) {
|
||||
impl SignatureCollection {
|
||||
/// Creates a new, empty signature collection given a signature registry.
|
||||
pub fn new(registry: &SignatureRegistry) -> Self {
|
||||
Self {
|
||||
registry: registry.0.clone(),
|
||||
signatures: PrimaryMap::new(),
|
||||
trampolines: HashMap::new(),
|
||||
}
|
||||
}
|
||||
|
||||
/// Creates a signature collection for a module given the module's signatures
|
||||
/// and trampolines.
|
||||
pub fn new_for_module(
|
||||
registry: &SignatureRegistry,
|
||||
signatures: &PrimaryMap<SignatureIndex, WasmFuncType>,
|
||||
trampolines: impl Iterator<Item = (SignatureIndex, VMTrampoline)>,
|
||||
) -> Self {
|
||||
let (signatures, trampolines) = registry
|
||||
.0
|
||||
.write()
|
||||
.unwrap()
|
||||
.register_for_module(signatures, trampolines);
|
||||
|
||||
Self {
|
||||
registry: registry.0.clone(),
|
||||
signatures,
|
||||
trampolines,
|
||||
}
|
||||
}
|
||||
|
||||
/// Treats the signature collection as a map from a module signature index to
|
||||
/// registered shared signature indexes.
|
||||
///
|
||||
/// This is used for looking up module shared signature indexes during module
|
||||
/// instantiation.
|
||||
pub fn as_module_map(&self) -> &PrimaryMap<SignatureIndex, VMSharedSignatureIndex> {
|
||||
&self.signatures
|
||||
}
|
||||
|
||||
/// Gets the shared signature index given a module signature index.
|
||||
pub fn shared_signature(&self, index: SignatureIndex) -> Option<VMSharedSignatureIndex> {
|
||||
self.signatures.get(index).copied()
|
||||
}
|
||||
|
||||
/// Gets a trampoline for a registered signature.
|
||||
pub fn trampoline(&self, index: VMSharedSignatureIndex) -> Option<VMTrampoline> {
|
||||
self.trampolines
|
||||
.get(&index)
|
||||
.map(|(_, trampoline)| *trampoline)
|
||||
}
|
||||
|
||||
/// Registers a single function with the collection.
|
||||
///
|
||||
/// Returns the shared signature index for the function.
|
||||
pub fn register(
|
||||
&mut self,
|
||||
ty: &WasmFuncType,
|
||||
trampoline: VMTrampoline,
|
||||
) -> VMSharedSignatureIndex {
|
||||
let index = self.registry.write().unwrap().register(ty);
|
||||
|
||||
let entry = match self.trampolines.entry(index) {
|
||||
Entry::Occupied(e) => e.into_mut(),
|
||||
Entry::Vacant(e) => e.insert((0, trampoline)),
|
||||
};
|
||||
|
||||
// Increment the ref count
|
||||
entry.0 += 1;
|
||||
}
|
||||
|
||||
/// Gets a trampoline from the map.
|
||||
pub fn get(&self, index: VMSharedSignatureIndex) -> Option<VMTrampoline> {
|
||||
self.0.get(&index).map(|(_, trampoline)| *trampoline)
|
||||
}
|
||||
|
||||
/// Iterates the shared signature indexes stored in the map.
|
||||
///
|
||||
/// A shared signature index will be returned by the iterator for every
|
||||
/// trampoline registered for that index, so duplicates may be present.
|
||||
///
|
||||
/// This iterator can be used for deregistering signatures with the
|
||||
/// signature registry.
|
||||
pub fn indexes<'a>(&'a self) -> impl Iterator<Item = VMSharedSignatureIndex> + 'a {
|
||||
self.0
|
||||
.iter()
|
||||
.flat_map(|(index, (count, _))| std::iter::repeat(*index).take(*count))
|
||||
}
|
||||
|
||||
/// Determines if the trampoline map is empty.
|
||||
pub fn is_empty(&self) -> bool {
|
||||
self.0.is_empty()
|
||||
index
|
||||
}
|
||||
}
|
||||
|
||||
/// Represents a map between module signature indexes and
|
||||
/// shared signature indexes.
|
||||
pub type SharedSignatures = PrimaryMap<SignatureIndex, VMSharedSignatureIndex>;
|
||||
impl Drop for SignatureCollection {
|
||||
fn drop(&mut self) {
|
||||
if !self.signatures.is_empty() || !self.trampolines.is_empty() {
|
||||
self.registry.write().unwrap().unregister_signatures(self);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct RegistryEntry {
|
||||
@@ -63,44 +110,37 @@ struct RegistryEntry {
|
||||
ty: WasmFuncType,
|
||||
}
|
||||
|
||||
/// WebAssembly requires that the caller and callee signatures in an indirect
|
||||
/// call must match. To implement this efficiently, keep a registry of all
|
||||
/// signatures, shared by all instances, so that call sites can just do an
|
||||
/// index comparison.
|
||||
#[derive(Debug, Default)]
|
||||
pub struct SignatureRegistry {
|
||||
struct SignatureRegistryInner {
|
||||
map: HashMap<WasmFuncType, VMSharedSignatureIndex>,
|
||||
entries: Vec<Option<RegistryEntry>>,
|
||||
free: Vec<VMSharedSignatureIndex>,
|
||||
}
|
||||
|
||||
impl SignatureRegistry {
|
||||
/// Registers a module with the signature registry from the collection of
|
||||
/// all signatures and trampolines in the module.
|
||||
pub fn register_module(
|
||||
impl SignatureRegistryInner {
|
||||
fn register_for_module(
|
||||
&mut self,
|
||||
signatures: &PrimaryMap<SignatureIndex, WasmFuncType>,
|
||||
trampolines: impl Iterator<Item = (SignatureIndex, VMTrampoline)>,
|
||||
) -> (SharedSignatures, TrampolineMap) {
|
||||
let mut sigs = SharedSignatures::default();
|
||||
let mut map = TrampolineMap::default();
|
||||
) -> (
|
||||
PrimaryMap<SignatureIndex, VMSharedSignatureIndex>,
|
||||
HashMap<VMSharedSignatureIndex, (usize, VMTrampoline)>,
|
||||
) {
|
||||
let mut sigs = PrimaryMap::default();
|
||||
let mut map = HashMap::default();
|
||||
|
||||
for (_, ty) in signatures.iter() {
|
||||
sigs.push(self.register(ty));
|
||||
}
|
||||
|
||||
for (index, trampoline) in trampolines {
|
||||
let index = self.map[&signatures[index]];
|
||||
map.insert(index, trampoline);
|
||||
map.insert(sigs[index], (1, trampoline));
|
||||
}
|
||||
|
||||
(sigs, map)
|
||||
}
|
||||
|
||||
/// Registers a single signature with the registry.
|
||||
///
|
||||
/// This is used for registering host functions created with the Wasmtime API.
|
||||
pub fn register(&mut self, ty: &WasmFuncType) -> VMSharedSignatureIndex {
|
||||
fn register(&mut self, ty: &WasmFuncType) -> VMSharedSignatureIndex {
|
||||
let len = self.map.len();
|
||||
|
||||
let index = match self.map.entry(ty.clone()) {
|
||||
@@ -124,6 +164,10 @@ impl SignatureRegistry {
|
||||
}
|
||||
};
|
||||
|
||||
// The entry should be missing for one just allocated or
|
||||
// taken from the free list
|
||||
assert!(entry.is_none());
|
||||
|
||||
*entry = Some(RegistryEntry {
|
||||
references: 0,
|
||||
ty: ty.clone(),
|
||||
@@ -141,14 +185,28 @@ impl SignatureRegistry {
|
||||
index
|
||||
}
|
||||
|
||||
/// Unregisters a collection of shared indexes from the registry.
|
||||
pub fn unregister(&mut self, indexes: impl Iterator<Item = VMSharedSignatureIndex>) {
|
||||
for index in indexes {
|
||||
fn unregister_signatures(&mut self, collection: &SignatureCollection) {
|
||||
// If the collection has a populated signatures map, use it to deregister
|
||||
// This is always 1:1 from entry to registration
|
||||
if !collection.signatures.is_empty() {
|
||||
for (_, index) in collection.signatures.iter() {
|
||||
self.unregister_entry(*index, 1);
|
||||
}
|
||||
} else {
|
||||
// Otherwise, use the trampolines map, which has reference counts related
|
||||
// to the stored index
|
||||
for (index, (count, _)) in collection.trampolines.iter() {
|
||||
self.unregister_entry(*index, *count);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fn unregister_entry(&mut self, index: VMSharedSignatureIndex, count: usize) {
|
||||
let removed = {
|
||||
let entry = self.entries[index.bits() as usize].as_mut().unwrap();
|
||||
|
||||
debug_assert!(entry.references > 0);
|
||||
entry.references -= 1;
|
||||
debug_assert!(entry.references >= count);
|
||||
entry.references -= count;
|
||||
|
||||
if entry.references == 0 {
|
||||
self.map.remove(&entry.ty);
|
||||
@@ -163,23 +221,42 @@ impl SignatureRegistry {
|
||||
self.entries[index.bits() as usize] = None;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// `SignatureRegistryInner` implements `Drop` in debug builds to assert that
|
||||
// all signatures have been unregistered for the registry.
|
||||
#[cfg(debug_assertions)]
|
||||
impl Drop for SignatureRegistryInner {
|
||||
fn drop(&mut self) {
|
||||
assert!(
|
||||
self.map.is_empty() && self.free.len() == self.entries.len(),
|
||||
"signature registry not empty"
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
/// Implements a shared signature registry.
|
||||
///
|
||||
/// WebAssembly requires that the caller and callee signatures in an indirect
|
||||
/// call must match. To implement this efficiently, keep a registry of all
|
||||
/// signatures, shared by all instances, so that call sites can just do an
|
||||
/// index comparison.
|
||||
#[derive(Debug)]
|
||||
pub struct SignatureRegistry(Arc<RwLock<SignatureRegistryInner>>);
|
||||
|
||||
impl SignatureRegistry {
|
||||
/// Creates a new shared signature registry.
|
||||
pub fn new() -> Self {
|
||||
Self(Arc::new(RwLock::new(SignatureRegistryInner::default())))
|
||||
}
|
||||
|
||||
/// Looks up a function type from a shared signature index.
|
||||
pub fn lookup_type(&self, index: VMSharedSignatureIndex) -> Option<&WasmFuncType> {
|
||||
self.entries
|
||||
pub fn lookup_type(&self, index: VMSharedSignatureIndex) -> Option<WasmFuncType> {
|
||||
self.0
|
||||
.read()
|
||||
.unwrap()
|
||||
.entries
|
||||
.get(index.bits() as usize)
|
||||
.and_then(|e| e.as_ref().map(|e| &e.ty))
|
||||
}
|
||||
|
||||
/// Determines if the registry is semantically empty.
|
||||
pub fn is_empty(&self) -> bool {
|
||||
// If the map is empty, assert that all remaining entries are "free"
|
||||
if self.map.is_empty() {
|
||||
assert!(self.free.len() == self.entries.len());
|
||||
true
|
||||
} else {
|
||||
false
|
||||
}
|
||||
.and_then(|e| e.as_ref().map(|e| &e.ty).cloned())
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,6 +1,6 @@
|
||||
use crate::{
|
||||
module::ModuleRegistry, signatures::TrampolineMap, trampoline::StoreInstanceHandle, Engine,
|
||||
Func, Module, Trap,
|
||||
module::ModuleRegistry, signatures::SignatureCollection, trampoline::StoreInstanceHandle,
|
||||
Engine, Func, Module, Trap,
|
||||
};
|
||||
use anyhow::{bail, Result};
|
||||
use std::any::{Any, TypeId};
|
||||
@@ -15,11 +15,10 @@ use std::ptr;
|
||||
use std::rc::Rc;
|
||||
use std::sync::Arc;
|
||||
use std::task::{Context, Poll};
|
||||
use wasmtime_environ::wasm::WasmFuncType;
|
||||
use wasmtime_jit::ModuleCode;
|
||||
use wasmtime_runtime::{
|
||||
InstanceAllocator, InstanceHandle, OnDemandInstanceAllocator, SignalHandler, TrapInfo,
|
||||
VMContext, VMExternRef, VMExternRefActivationsTable, VMInterrupts, VMSharedSignatureIndex,
|
||||
VMCallerCheckedAnyfunc, VMContext, VMExternRef, VMExternRefActivationsTable, VMInterrupts,
|
||||
VMTrampoline,
|
||||
};
|
||||
|
||||
@@ -76,7 +75,8 @@ pub(crate) struct StoreInner {
|
||||
signal_handler: RefCell<Option<Box<SignalHandler<'static>>>>,
|
||||
externref_activations_table: VMExternRefActivationsTable,
|
||||
modules: RefCell<ModuleRegistry>,
|
||||
trampolines: RefCell<TrampolineMap>,
|
||||
// The signatures and trampolines for `Func` objects
|
||||
signatures: RefCell<SignatureCollection>,
|
||||
// Numbers of resources instantiated in this store.
|
||||
instance_count: Cell<usize>,
|
||||
memory_count: Cell<usize>,
|
||||
@@ -139,7 +139,7 @@ impl Store {
|
||||
signal_handler: RefCell::new(None),
|
||||
externref_activations_table: VMExternRefActivationsTable::new(),
|
||||
modules: RefCell::new(ModuleRegistry::default()),
|
||||
trampolines: RefCell::new(TrampolineMap::default()),
|
||||
signatures: RefCell::new(SignatureCollection::new(engine.signatures())),
|
||||
instance_count: Default::default(),
|
||||
memory_count: Default::default(),
|
||||
table_count: Default::default(),
|
||||
@@ -203,41 +203,31 @@ impl Store {
|
||||
}
|
||||
}
|
||||
|
||||
pub(crate) fn register_module(&self, module: &Module) {
|
||||
// Register the module with the registry
|
||||
self.inner.modules.borrow_mut().register(module);
|
||||
pub(crate) fn signatures(&self) -> &RefCell<SignatureCollection> {
|
||||
&self.inner.signatures
|
||||
}
|
||||
|
||||
// This is used to register a `Func` with the store
|
||||
pub(crate) fn register_signature(
|
||||
&self,
|
||||
ty: &WasmFuncType,
|
||||
trampoline: VMTrampoline,
|
||||
) -> VMSharedSignatureIndex {
|
||||
let index = self.inner.engine.register_signature(ty);
|
||||
self.inner
|
||||
.trampolines
|
||||
.borrow_mut()
|
||||
.insert(index, trampoline);
|
||||
index
|
||||
}
|
||||
|
||||
pub(crate) fn lookup_trampoline(&self, index: VMSharedSignatureIndex) -> VMTrampoline {
|
||||
pub(crate) fn lookup_trampoline(&self, anyfunc: &VMCallerCheckedAnyfunc) -> VMTrampoline {
|
||||
// Look up the trampoline with the store's trampolines (from `Func`).
|
||||
if let Some(trampoline) = self.inner.trampolines.borrow().get(index) {
|
||||
if let Some(trampoline) = self
|
||||
.inner
|
||||
.signatures
|
||||
.borrow()
|
||||
.trampoline(anyfunc.type_index)
|
||||
{
|
||||
return trampoline;
|
||||
}
|
||||
|
||||
// Look up the trampoline with the registered modules
|
||||
if let Some(trampoline) = self.inner.modules.borrow().lookup_trampoline(index) {
|
||||
if let Some(trampoline) = self.inner.modules.borrow().lookup_trampoline(anyfunc) {
|
||||
return trampoline;
|
||||
}
|
||||
|
||||
// Lastly, check with the engine (for `HostFunc`)
|
||||
self.inner
|
||||
.engine
|
||||
.host_func_trampolines()
|
||||
.get(index)
|
||||
.host_func_signatures()
|
||||
.trampoline(anyfunc.type_index)
|
||||
.expect("trampoline missing")
|
||||
}
|
||||
|
||||
@@ -451,7 +441,7 @@ impl Store {
|
||||
}
|
||||
|
||||
#[inline]
|
||||
pub(crate) fn stack_map_lookup(&self) -> *const dyn wasmtime_runtime::StackMapLookup {
|
||||
pub(crate) fn stack_map_lookup(&self) -> &dyn wasmtime_runtime::StackMapLookup {
|
||||
self.inner.as_ref()
|
||||
}
|
||||
|
||||
@@ -918,16 +908,10 @@ impl Drop for StoreInner {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let trampolines = self.trampolines.borrow();
|
||||
|
||||
if !trampolines.is_empty() {
|
||||
self.engine.unregister_signatures(trampolines.indexes());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl wasmtime_runtime::StackMapLookup for StoreInner {
|
||||
unsafe impl wasmtime_runtime::StackMapLookup for StoreInner {
|
||||
fn lookup(&self, pc: usize) -> Option<*const wasmtime_environ::ir::StackMap> {
|
||||
// The address of the stack map is stable for the lifetime of the store
|
||||
self.modules.borrow().lookup_stack_map(pc).map(|m| m as _)
|
||||
|
||||
@@ -77,7 +77,7 @@ fn create_handle(
|
||||
externref_activations_table: store.externref_activations_table()
|
||||
as *const VMExternRefActivationsTable
|
||||
as *mut _,
|
||||
stack_map_lookup: Some(store.stack_map_lookup()),
|
||||
stack_map_lookup: Some(std::mem::transmute(store.stack_map_lookup())),
|
||||
},
|
||||
)?;
|
||||
|
||||
|
||||
@@ -270,7 +270,11 @@ pub fn create_function(
|
||||
// If there is no store, use the default signature index which is
|
||||
// guaranteed to trap if there is ever an indirect call on the function (should not happen)
|
||||
let shared_signature_id = store
|
||||
.map(|s| s.register_signature(ft.as_wasm_func_type(), trampoline))
|
||||
.map(|s| {
|
||||
s.signatures()
|
||||
.borrow_mut()
|
||||
.register(ft.as_wasm_func_type(), trampoline)
|
||||
})
|
||||
.unwrap_or(VMSharedSignatureIndex::default());
|
||||
|
||||
unsafe {
|
||||
|
||||
@@ -1,4 +1,4 @@
|
||||
use crate::{signatures::SharedSignatures, Extern, Store};
|
||||
use crate::{signatures::SignatureCollection, Extern, Store};
|
||||
use anyhow::{bail, Context, Result};
|
||||
use wasmtime_environ::wasm::{
|
||||
EntityType, Global, InstanceTypeIndex, Memory, ModuleTypeIndex, SignatureIndex, Table,
|
||||
@@ -6,7 +6,7 @@ use wasmtime_environ::wasm::{
|
||||
use wasmtime_jit::TypeTables;
|
||||
|
||||
pub struct MatchCx<'a> {
|
||||
pub signatures: &'a SharedSignatures,
|
||||
pub signatures: &'a SignatureCollection,
|
||||
pub types: &'a TypeTables,
|
||||
pub store: &'a Store,
|
||||
}
|
||||
@@ -71,8 +71,8 @@ impl MatchCx<'_> {
|
||||
}
|
||||
|
||||
pub fn func(&self, expected: SignatureIndex, actual: &crate::Func) -> Result<()> {
|
||||
let matches = match self.signatures.get(expected) {
|
||||
Some(idx) => actual.sig_index() == *idx,
|
||||
let matches = match self.signatures.shared_signature(expected) {
|
||||
Some(idx) => actual.sig_index() == idx,
|
||||
// If our expected signature isn't registered, then there's no way
|
||||
// that `actual` can match it.
|
||||
None => false,
|
||||
@@ -133,7 +133,7 @@ impl MatchCx<'_> {
|
||||
fn imports_match<'a>(
|
||||
&self,
|
||||
expected: ModuleTypeIndex,
|
||||
actual_signatures: &SharedSignatures,
|
||||
actual_signatures: &SignatureCollection,
|
||||
actual_types: &TypeTables,
|
||||
actual_imports: impl Iterator<Item = (&'a str, EntityType)>,
|
||||
) -> Result<()> {
|
||||
@@ -162,7 +162,7 @@ impl MatchCx<'_> {
|
||||
fn exports_match(
|
||||
&self,
|
||||
expected: InstanceTypeIndex,
|
||||
actual_signatures: &SharedSignatures,
|
||||
actual_signatures: &SignatureCollection,
|
||||
actual_types: &TypeTables,
|
||||
lookup: impl Fn(&str) -> Option<EntityType>,
|
||||
) -> Result<()> {
|
||||
@@ -186,7 +186,7 @@ impl MatchCx<'_> {
|
||||
&self,
|
||||
expected: &EntityType,
|
||||
actual_ty: &EntityType,
|
||||
actual_signatures: &SharedSignatures,
|
||||
actual_signatures: &SignatureCollection,
|
||||
actual_types: &TypeTables,
|
||||
) -> Result<()> {
|
||||
let actual_desc = match actual_ty {
|
||||
|
||||
Reference in New Issue
Block a user