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:
Peter Huene
2021-04-15 20:15:24 -07:00
parent ea72c621f0
commit 510fc71728
13 changed files with 336 additions and 344 deletions

View File

@@ -176,11 +176,13 @@ struct FinishedFunctions(PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]>);
unsafe impl Send for FinishedFunctions {} unsafe impl Send for FinishedFunctions {}
unsafe impl Sync for FinishedFunctions {} unsafe impl Sync for FinishedFunctions {}
/// Information about a function, such as trap information, address map,
/// and stack maps.
#[derive(Serialize, Deserialize, Clone)] #[derive(Serialize, Deserialize, Clone)]
struct FunctionInfo { pub struct FunctionInfo {
traps: Vec<TrapInformation>, pub traps: Vec<TrapInformation>,
address_map: FunctionAddressMap, pub address_map: FunctionAddressMap,
stack_maps: Vec<StackMapInformation>, pub stack_maps: Vec<StackMapInformation>,
} }
/// This is intended to mirror the type tables in `wasmtime_environ`, except that /// 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. /// Gets the function information for a given function index.
pub fn func_info( pub fn func_info(&self, index: DefinedFuncIndex) -> &FunctionInfo {
&self,
index: DefinedFuncIndex,
) -> (
&FunctionAddressMap,
&[TrapInformation],
&[StackMapInformation],
) {
self.artifacts self.artifacts
.funcs .funcs
.get(index) .get(index)
.map(|f| (&f.address_map, f.traps.as_ref(), f.stack_maps.as_ref()))
.expect("defined function should be present") .expect("defined function should be present")
} }

View File

@@ -742,14 +742,22 @@ impl VMExternRefActivationsTable {
} }
/// Used by the runtime to lookup a stack map from a PC value. /// 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. /// Lookup the stack map at a program counter (PC) value.
fn lookup(&self, pc: usize) -> Option<*const StackMap>; fn lookup(&self, pc: usize) -> Option<*const StackMap>;
} }
pub(crate) struct EmptyStackMapLookup; pub(crate) struct EmptyStackMapLookup;
impl StackMapLookup for EmptyStackMapLookup { unsafe impl StackMapLookup for EmptyStackMapLookup {
fn lookup(&self, _pc: usize) -> Option<*const StackMap> { fn lookup(&self, _pc: usize) -> Option<*const StackMap> {
None None
} }

View File

@@ -1,24 +1,34 @@
use crate::signatures::{SharedSignatures, SignatureRegistry, TrampolineMap}; use crate::signatures::{SignatureCollection, SignatureRegistry};
use crate::Config; use crate::Config;
use anyhow::Result; use anyhow::Result;
use std::collections::HashMap; use std::collections::HashMap;
use std::sync::{Arc, RwLock}; use std::sync::Arc;
#[cfg(feature = "cache")] #[cfg(feature = "cache")]
use wasmtime_cache::CacheConfig; use wasmtime_cache::CacheConfig;
use wasmtime_environ::{
entity::PrimaryMap,
wasm::{SignatureIndex, WasmFuncType},
};
use wasmtime_jit::Compiler; use wasmtime_jit::Compiler;
use wasmtime_runtime::{ use wasmtime_runtime::{debug_builtins, InstanceAllocator, InstanceHandle, VMCallerCheckedAnyfunc};
debug_builtins, InstanceAllocator, InstanceHandle, VMCallerCheckedAnyfunc,
VMSharedSignatureIndex, VMTrampoline,
};
#[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 { struct EngineHostFuncs {
anyfuncs: HashMap<InstanceHandle, Box<VMCallerCheckedAnyfunc>>, 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 // This is safe for send and sync as it is read-only once the
@@ -58,18 +68,10 @@ struct EngineInner {
config: Config, config: Config,
compiler: Compiler, compiler: Compiler,
allocator: Box<dyn InstanceAllocator>, allocator: Box<dyn InstanceAllocator>,
signatures: RwLock<SignatureRegistry>, signatures: SignatureRegistry,
host_funcs: EngineHostFuncs, 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 { impl Engine {
/// Creates a new [`Engine`] with the specified compilation and /// Creates a new [`Engine`] with the specified compilation and
/// configuration settings. /// configuration settings.
@@ -77,20 +79,20 @@ impl Engine {
debug_builtins::ensure_exported(); debug_builtins::ensure_exported();
config.validate()?; config.validate()?;
let allocator = config.build_allocator()?; let allocator = config.build_allocator()?;
let mut signatures = SignatureRegistry::default(); let registry = SignatureRegistry::new();
let mut host_funcs = EngineHostFuncs::default(); let mut host_funcs = EngineHostFuncs::new(&registry);
// Register all the host function signatures // Register all the host function signatures with the collection
for func in config.host_funcs() { 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 // Cloning the instance handle is safe as host functions outlive the engine
host_funcs.anyfuncs.insert( host_funcs.anyfuncs.insert(
unsafe { func.instance.clone() }, unsafe { func.instance.clone() },
Box::new(func.anyfunc(sig)), Box::new(func.anyfunc(sig)),
); );
host_funcs.trampolines.insert(sig, func.trampoline);
} }
Ok(Engine { Ok(Engine {
@@ -98,7 +100,7 @@ impl Engine {
config: config.clone(), config: config.clone(),
compiler: config.build_compiler(allocator.as_ref()), compiler: config.build_compiler(allocator.as_ref()),
allocator, allocator,
signatures: RwLock::new(signatures), signatures: registry,
host_funcs, host_funcs,
}), }),
}) })
@@ -128,40 +130,12 @@ impl Engine {
Arc::ptr_eq(&a.inner, &b.inner) Arc::ptr_eq(&a.inner, &b.inner)
} }
pub(crate) fn register_module_signatures( pub(crate) fn signatures(&self) -> &SignatureRegistry {
&self, &self.inner.signatures
signatures: &PrimaryMap<SignatureIndex, WasmFuncType>,
trampolines: impl Iterator<Item = (SignatureIndex, VMTrampoline)>,
) -> (SharedSignatures, TrampolineMap) {
self.inner
.signatures
.write()
.unwrap()
.register_module(signatures, trampolines)
} }
pub(crate) fn register_signature(&self, ty: &WasmFuncType) -> VMSharedSignatureIndex { pub(crate) fn host_func_signatures(&self) -> &SignatureCollection {
self.inner.signatures.write().unwrap().register(ty) &self.inner.host_funcs.signatures
}
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_anyfunc( pub(crate) fn host_func_anyfunc(

View File

@@ -778,31 +778,20 @@ impl Func {
self.instance self.instance
.store .store
.engine() .engine()
.lookup_func_type(self.sig_index()) .signatures()
.lookup_type(self.sig_index())
.expect("signature should be registered"), .expect("signature should be registered"),
) )
} }
/// Returns the number of parameters that this function takes. /// Returns the number of parameters that this function takes.
pub fn param_arity(&self) -> usize { pub fn param_arity(&self) -> usize {
let sig = self self.ty().params().len()
.instance
.store
.engine()
.lookup_func_type(self.sig_index())
.expect("signature should be registered");
sig.params.len()
} }
/// Returns the number of results this function produces. /// Returns the number of results this function produces.
pub fn result_arity(&self) -> usize { pub fn result_arity(&self) -> usize {
let sig = self self.ty().results().len()
.instance
.store
.engine()
.lookup_func_type(self.sig_index())
.expect("signature should be registered");
sig.returns.len()
} }
/// Invokes this function with the `params` given, returning the results and /// Invokes this function with the `params` given, returning the results and
@@ -927,7 +916,7 @@ impl Func {
Func { Func {
instance: store.existing_vmctx(anyfunc.vmctx), instance: store.existing_vmctx(anyfunc.vmctx),
export: export.clone(), 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 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). // If the function is called indirectly without first being associated with a store (a bug condition).
let shared_signature_id = store 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()); .unwrap_or(VMSharedSignatureIndex::default());
let instance = unsafe { let instance = unsafe {

View File

@@ -13,9 +13,9 @@ use wasmtime_environ::wasm::{
}; };
use wasmtime_environ::Initializer; use wasmtime_environ::Initializer;
use wasmtime_runtime::{ use wasmtime_runtime::{
Imports, InstanceAllocationRequest, InstantiationError, RuntimeInstance, StackMapLookup, Imports, InstanceAllocationRequest, InstantiationError, RuntimeInstance, VMContext,
VMContext, VMExternRefActivationsTable, VMFunctionBody, VMFunctionImport, VMGlobalImport, VMExternRefActivationsTable, VMFunctionBody, VMFunctionImport, VMGlobalImport, VMMemoryImport,
VMMemoryImport, VMTableImport, VMTableImport,
}; };
/// An instantiated WebAssembly module. /// An instantiated WebAssembly module.
@@ -506,10 +506,9 @@ impl<'a> Instantiator<'a> {
fn instantiate_raw(&self) -> Result<StoreInstanceHandle> { fn instantiate_raw(&self) -> Result<StoreInstanceHandle> {
let compiled_module = self.cur.module.compiled_module(); let compiled_module = self.cur.module.compiled_module();
// Register the module just before instantiation to ensure we have a // Register the module just before instantiation to ensure we keep the module
// trampoline registered for every signature and to preserve the module's // properly referenced while in use by the store.
// compiled JIT code within the `Store`. self.store.modules().borrow_mut().register(&self.cur.module);
self.store.register_module(&self.cur.module);
unsafe { unsafe {
let engine = self.store.engine(); let engine = self.store.engine();
@@ -519,13 +518,13 @@ impl<'a> Instantiator<'a> {
module: compiled_module.module().clone(), module: compiled_module.module().clone(),
finished_functions: compiled_module.finished_functions(), finished_functions: compiled_module.finished_functions(),
imports: self.cur.build(), 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(()), host_state: Box::new(()),
interrupts: self.store.interrupts(), interrupts: self.store.interrupts(),
externref_activations_table: self.store.externref_activations_table() externref_activations_table: self.store.externref_activations_table()
as *const VMExternRefActivationsTable as *const VMExternRefActivationsTable
as *mut _, 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 // After we've created the `InstanceHandle` we still need to run

View File

@@ -1,5 +1,5 @@
use crate::{ use crate::{
signatures::{SharedSignatures, TrampolineMap}, signatures::SignatureCollection,
types::{ExportType, ExternType, ImportType}, types::{ExportType, ExternType, ImportType},
}; };
use crate::{Engine, ModuleType}; use crate::{Engine, ModuleType};
@@ -11,9 +11,8 @@ use wasmparser::Validator;
#[cfg(feature = "cache")] #[cfg(feature = "cache")]
use wasmtime_cache::ModuleCacheEntry; use wasmtime_cache::ModuleCacheEntry;
use wasmtime_environ::entity::PrimaryMap; use wasmtime_environ::entity::PrimaryMap;
use wasmtime_environ::wasm::{ModuleIndex, SignatureIndex}; use wasmtime_environ::wasm::ModuleIndex;
use wasmtime_jit::{CompilationArtifacts, CompiledModule, TypeTables}; use wasmtime_jit::{CompilationArtifacts, CompiledModule, TypeTables};
use wasmtime_runtime::VMSharedSignatureIndex;
mod registry; mod registry;
mod serialization; mod serialization;
@@ -21,26 +20,6 @@ mod serialization;
pub use registry::{FrameInfo, FrameSymbol, GlobalModuleRegistry, ModuleRegistry}; pub use registry::{FrameInfo, FrameSymbol, GlobalModuleRegistry, ModuleRegistry};
pub use serialization::SerializedModule; 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 compiled WebAssembly module, ready to be instantiated.
/// ///
/// A `Module` is a compiled in-memory representation of an input WebAssembly /// A `Module` is a compiled in-memory representation of an input WebAssembly
@@ -129,7 +108,7 @@ struct ModuleInner {
/// modules. /// modules.
types: Arc<TypeTables>, types: Arc<TypeTables>,
/// Registered shared signature for the module. /// Registered shared signature for the module.
signatures: Arc<ModuleSharedSignatures>, signatures: Arc<SignatureCollection>,
} }
impl Module { impl Module {
@@ -350,10 +329,11 @@ impl Module {
// Validate the module can be used with the current allocator // Validate the module can be used with the current allocator
engine.allocator().validate(modules[main_module].module())?; 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, &types.wasm_signatures,
modules.iter().flat_map(|m| m.trampolines().iter().cloned()), modules.iter().flat_map(|m| m.trampolines().iter().cloned()),
); ));
let module = modules.remove(main_module); let module = modules.remove(main_module);
@@ -364,11 +344,7 @@ impl Module {
types: Arc::new(types), types: Arc::new(types),
artifact_upvars: modules, artifact_upvars: modules,
module_upvars: Vec::new(), module_upvars: Vec::new(),
signatures: Arc::new(ModuleSharedSignatures { signatures,
engine: engine.clone(),
signatures,
trampolines,
}),
}), }),
}) })
} }
@@ -488,11 +464,7 @@ impl Module {
&self.inner.types &self.inner.types
} }
pub(crate) fn signatures(&self) -> &PrimaryMap<SignatureIndex, VMSharedSignatureIndex> { pub(crate) fn signatures(&self) -> &Arc<SignatureCollection> {
&self.inner.signatures.signatures
}
pub(crate) fn shared_signatures(&self) -> &Arc<ModuleSharedSignatures> {
&self.inner.signatures &self.inner.signatures
} }

View File

@@ -1,6 +1,6 @@
//! Implements a registry of modules for a store. //! Implements a registry of modules for a store.
use crate::{module::ModuleSharedSignatures, Module}; use crate::{signatures::SignatureCollection, Module};
use std::{ use std::{
collections::BTreeMap, collections::BTreeMap,
sync::{Arc, Mutex}, sync::{Arc, Mutex},
@@ -9,12 +9,17 @@ use wasmtime_environ::{
entity::EntityRef, ir, wasm::DefinedFuncIndex, FunctionAddressMap, TrapInformation, entity::EntityRef, ir, wasm::DefinedFuncIndex, FunctionAddressMap, TrapInformation,
}; };
use wasmtime_jit::CompiledModule; use wasmtime_jit::CompiledModule;
use wasmtime_runtime::{VMSharedSignatureIndex, VMTrampoline}; use wasmtime_runtime::{VMCallerCheckedAnyfunc, VMTrampoline};
lazy_static::lazy_static! { lazy_static::lazy_static! {
static ref GLOBAL_MODULES: Mutex<GlobalModuleRegistry> = Default::default(); 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. /// Used for registering modules with a store.
/// ///
/// The map is from the ending (exclusive) address for the module code to /// 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 compiled_module = module.compiled_module();
let (start, end) = compiled_module.code().range(); 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() { if start == end || compiled_module.finished_functions().is_empty() {
return; return;
} }
@@ -71,7 +76,10 @@ impl ModuleRegistry {
// may be a valid PC value // may be a valid PC value
let end = end - 1; 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; return;
} }
@@ -89,7 +97,7 @@ impl ModuleRegistry {
RegisteredModule { RegisteredModule {
start, start,
module: compiled_module.clone(), module: compiled_module.clone(),
signatures: module.shared_signatures().clone(), signatures: module.signatures().clone(),
}, },
); );
assert!(prev.is_none()); assert!(prev.is_none());
@@ -97,18 +105,10 @@ impl ModuleRegistry {
GLOBAL_MODULES.lock().unwrap().register(start, end, module); GLOBAL_MODULES.lock().unwrap().register(start, end, module);
} }
/// Looks up a trampoline from a shared signature index. /// Looks up a trampoline from an anyfunc.
/// pub fn lookup_trampoline(&self, anyfunc: &VMCallerCheckedAnyfunc) -> Option<VMTrampoline> {
/// This will search all modules associated with the store for a suitable trampoline let module = self.module(anyfunc.func_ptr.as_ptr() as usize)?;
/// given the shared signature index. module.signatures.trampoline(anyfunc.type_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
} }
} }
@@ -124,7 +124,7 @@ impl Drop for ModuleRegistry {
struct RegisteredModule { struct RegisteredModule {
start: usize, start: usize,
module: Arc<CompiledModule>, module: Arc<CompiledModule>,
signatures: Arc<ModuleSharedSignatures>, signatures: Arc<SignatureCollection>,
} }
impl RegisteredModule { impl RegisteredModule {
@@ -138,9 +138,9 @@ impl RegisteredModule {
/// Returns an object if this `pc` is known to this module, or returns `None` /// Returns an object if this `pc` is known to this module, or returns `None`
/// if no information can be found. /// if no information can be found.
pub fn lookup_frame_info(&self, pc: usize) -> Option<FrameInfo> { pub fn lookup_frame_info(&self, pc: usize) -> Option<FrameInfo> {
let (index, offset) = self.func(pc)?; let (index, offset) = func_by_pc(&self.module, pc)?;
let (addr_map, _, _) = self.module.func_info(index); let info = self.module.func_info(index);
let pos = Self::instr_pos(offset, addr_map); let pos = Self::instr_pos(offset, &info.address_map);
// In debug mode for now assert that we found a mapping for `pc` within // 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 // 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); debug_assert!(pos.is_some(), "failed to find instruction for {:x}", pc);
let instr = match pos { let instr = match pos {
Some(pos) => addr_map.instructions[pos].srcloc, Some(pos) => info.address_map.instructions[pos].srcloc,
None => addr_map.start_srcloc, None => info.address_map.start_srcloc,
}; };
// Use our wasm-relative pc to symbolize this frame. If there's a // 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_index: index.index() as u32,
func_name: module.func_names.get(&index).cloned(), func_name: module.func_names.get(&index).cloned(),
instr, instr,
func_start: addr_map.start_srcloc, func_start: info.address_map.start_srcloc,
symbols, symbols,
}) })
} }
/// Fetches trap information about a program counter in a backtrace. /// Fetches trap information about a program counter in a backtrace.
pub fn lookup_trap_info(&self, pc: usize) -> Option<&TrapInformation> { pub fn lookup_trap_info(&self, pc: usize) -> Option<&TrapInformation> {
let (index, offset) = self.func(pc)?; let (index, offset) = func_by_pc(&self.module, pc)?;
let (_, traps, _) = self.module.func_info(index); let info = self.module.func_info(index);
let idx = traps let idx = info
.traps
.binary_search_by_key(&offset, |info| info.code_offset) .binary_search_by_key(&offset, |info| info.code_offset)
.ok()?; .ok()?;
Some(&traps[idx]) Some(&info.traps[idx])
} }
/// Looks up a stack map from a program counter /// Looks up a stack map from a program counter
pub fn lookup_stack_map(&self, pc: usize) -> Option<&ir::StackMap> { pub fn lookup_stack_map(&self, pc: usize) -> Option<&ir::StackMap> {
let (index, offset) = self.func(pc)?; let (index, offset) = func_by_pc(&self.module, pc)?;
let (_, _, stack_maps) = self.module.func_info(index); let info = self.module.func_info(index);
// Do a binary search to find the stack map for the given offset. // 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 // or reload from the stack slots (which would have been updated to
// point to the moved objects). // 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. // Exact hit.
Ok(i) => i, Ok(i) => i,
@@ -269,12 +273,7 @@ impl RegisteredModule {
Err(i) => i - 1, Err(i) => i - 1,
}; };
Some(&stack_maps[index].stack_map) Some(&info.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))
} }
fn instr_pos(offset: u32, addr_map: &FunctionAddressMap) -> Option<usize> { 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 /// This is the global module registry that stores information for all modules
/// that are currently in use by any `Store`. /// that are currently in use by any `Store`.
/// ///
@@ -318,18 +328,18 @@ impl GlobalModuleRegistry {
/// Returns whether the `pc`, according to globally registered information, /// Returns whether the `pc`, according to globally registered information,
/// is a wasm trap or not. /// is a wasm trap or not.
pub(crate) fn is_wasm_pc(pc: usize) -> bool { 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() { match modules.0.range(pc..).next() {
Some((end, info)) => { Some((end, entry)) => {
if pc < info.module.start || *end < pc { if pc < entry.start || *end < pc {
return false; return false;
} }
match info.module.func(pc) { match func_by_pc(&entry.module, pc) {
Some((index, offset)) => { Some((index, offset)) => {
let (addr_map, _, _) = info.module.module.func_info(index); let info = entry.module.func_info(index);
RegisteredModule::instr_pos(offset, addr_map).is_some() RegisteredModule::instr_pos(offset, &info.address_map).is_some()
} }
None => false, None => false,
} }
@@ -342,18 +352,15 @@ impl GlobalModuleRegistry {
/// the given function information, with the global information. /// the given function information, with the global information.
fn register(&mut self, start: usize, end: usize, module: &Module) { fn register(&mut self, start: usize, end: usize, module: &Module) {
let info = self.0.entry(end).or_insert_with(|| GlobalRegisteredModule { let info = self.0.entry(end).or_insert_with(|| GlobalRegisteredModule {
module: RegisteredModule { start,
start, module: module.compiled_module().clone(),
module: module.compiled_module().clone(),
signatures: module.shared_signatures().clone(),
},
references: 0, references: 0,
}); });
// Note that ideally we'd debug_assert that the information previously // Note that ideally we'd debug_assert that the information previously
// stored, if any, matches the `functions` we were given, but for now we // stored, if any, matches the `functions` we were given, but for now we
// just do some simple checks to hope it's the same. // 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; 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`]. /// Description of a frame in a backtrace for a [`Trap`].
/// ///
/// Whenever a WebAssembly trap occurs an instance of [`Trap`] is created. Each /// Whenever a WebAssembly trap occurs an instance of [`Trap`] is created. Each

View File

@@ -1,7 +1,7 @@
//! Implements module serialization. //! Implements module serialization.
use super::{ModuleInner, ModuleSharedSignatures}; use super::ModuleInner;
use crate::{Engine, Module, OptLevel}; use crate::{signatures::SignatureCollection, Engine, Module, OptLevel};
use anyhow::{anyhow, bail, Context, Result}; use anyhow::{anyhow, bail, Context, Result};
use bincode::Options; use bincode::Options;
use serde::{Deserialize, Serialize}; use serde::{Deserialize, Serialize};
@@ -300,16 +300,11 @@ impl<'a> SerializedModule<'a> {
.allocator() .allocator()
.validate(modules.last().unwrap().module())?; .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, &types.wasm_signatures,
modules.iter().flat_map(|m| m.trampolines().iter().cloned()), modules.iter().flat_map(|m| m.trampolines().iter().cloned()),
); ));
let signatures = Arc::new(ModuleSharedSignatures {
engine: engine.clone(),
signatures,
trampolines,
});
let module = modules.pop().unwrap(); let module = modules.pop().unwrap();
@@ -347,7 +342,7 @@ impl<'a> SerializedModule<'a> {
module_index: usize, module_index: usize,
artifact_upvars: &[usize], artifact_upvars: &[usize],
module_upvars: &[SerializedModuleUpvar], module_upvars: &[SerializedModuleUpvar],
signatures: &Arc<ModuleSharedSignatures>, signatures: &Arc<SignatureCollection>,
) -> Result<Module> { ) -> Result<Module> {
Ok(Module { Ok(Module {
inner: Arc::new(ModuleInner { inner: Arc::new(ModuleInner {

View File

@@ -1,61 +1,108 @@
//! Implement a registry of function signatures, for fast indirect call //! Implement a registry of function signatures, for fast indirect call
//! signature checking. //! signature checking.
use std::collections::{hash_map::Entry, HashMap}; use std::{
use std::convert::TryFrom; collections::{hash_map::Entry, HashMap},
sync::RwLock,
};
use std::{convert::TryFrom, sync::Arc};
use wasmtime_environ::entity::PrimaryMap; use wasmtime_environ::entity::PrimaryMap;
use wasmtime_environ::wasm::{SignatureIndex, WasmFuncType}; use wasmtime_environ::wasm::{SignatureIndex, WasmFuncType};
use wasmtime_runtime::{VMSharedSignatureIndex, VMTrampoline}; 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 /// This is used to register shared signatures with a shared signature registry.
/// signature indexes.
/// ///
/// As multiple trampolines may exist for a single signature, the map entries /// The collection will unregister any contained signatures with the registry
/// are internally reference counted. /// when dropped.
#[derive(Default)] #[derive(Debug)]
pub struct TrampolineMap(HashMap<VMSharedSignatureIndex, (usize, VMTrampoline)>); pub struct SignatureCollection {
registry: Arc<RwLock<SignatureRegistryInner>>,
signatures: PrimaryMap<SignatureIndex, VMSharedSignatureIndex>,
trampolines: HashMap<VMSharedSignatureIndex, (usize, VMTrampoline)>,
}
impl TrampolineMap { impl SignatureCollection {
/// Inserts a trampoline into the map. /// Creates a new, empty signature collection given a signature registry.
pub fn insert(&mut self, index: VMSharedSignatureIndex, trampoline: VMTrampoline) { pub fn new(registry: &SignatureRegistry) -> Self {
let entry = match self.0.entry(index) { 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::Occupied(e) => e.into_mut(),
Entry::Vacant(e) => e.insert((0, trampoline)), Entry::Vacant(e) => e.insert((0, trampoline)),
}; };
// Increment the ref count // Increment the ref count
entry.0 += 1; entry.0 += 1;
}
/// Gets a trampoline from the map. index
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()
} }
} }
/// Represents a map between module signature indexes and impl Drop for SignatureCollection {
/// shared signature indexes. fn drop(&mut self) {
pub type SharedSignatures = PrimaryMap<SignatureIndex, VMSharedSignatureIndex>; if !self.signatures.is_empty() || !self.trampolines.is_empty() {
self.registry.write().unwrap().unregister_signatures(self);
}
}
}
#[derive(Debug)] #[derive(Debug)]
struct RegistryEntry { struct RegistryEntry {
@@ -63,44 +110,37 @@ struct RegistryEntry {
ty: WasmFuncType, 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)] #[derive(Debug, Default)]
pub struct SignatureRegistry { struct SignatureRegistryInner {
map: HashMap<WasmFuncType, VMSharedSignatureIndex>, map: HashMap<WasmFuncType, VMSharedSignatureIndex>,
entries: Vec<Option<RegistryEntry>>, entries: Vec<Option<RegistryEntry>>,
free: Vec<VMSharedSignatureIndex>, free: Vec<VMSharedSignatureIndex>,
} }
impl SignatureRegistry { impl SignatureRegistryInner {
/// Registers a module with the signature registry from the collection of fn register_for_module(
/// all signatures and trampolines in the module.
pub fn register_module(
&mut self, &mut self,
signatures: &PrimaryMap<SignatureIndex, WasmFuncType>, signatures: &PrimaryMap<SignatureIndex, WasmFuncType>,
trampolines: impl Iterator<Item = (SignatureIndex, VMTrampoline)>, trampolines: impl Iterator<Item = (SignatureIndex, VMTrampoline)>,
) -> (SharedSignatures, TrampolineMap) { ) -> (
let mut sigs = SharedSignatures::default(); PrimaryMap<SignatureIndex, VMSharedSignatureIndex>,
let mut map = TrampolineMap::default(); HashMap<VMSharedSignatureIndex, (usize, VMTrampoline)>,
) {
let mut sigs = PrimaryMap::default();
let mut map = HashMap::default();
for (_, ty) in signatures.iter() { for (_, ty) in signatures.iter() {
sigs.push(self.register(ty)); sigs.push(self.register(ty));
} }
for (index, trampoline) in trampolines { for (index, trampoline) in trampolines {
let index = self.map[&signatures[index]]; map.insert(sigs[index], (1, trampoline));
map.insert(index, trampoline);
} }
(sigs, map) (sigs, map)
} }
/// Registers a single signature with the registry. fn register(&mut self, ty: &WasmFuncType) -> VMSharedSignatureIndex {
///
/// This is used for registering host functions created with the Wasmtime API.
pub fn register(&mut self, ty: &WasmFuncType) -> VMSharedSignatureIndex {
let len = self.map.len(); let len = self.map.len();
let index = match self.map.entry(ty.clone()) { 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 { *entry = Some(RegistryEntry {
references: 0, references: 0,
ty: ty.clone(), ty: ty.clone(),
@@ -141,45 +185,78 @@ impl SignatureRegistry {
index index
} }
/// Unregisters a collection of shared indexes from the registry. fn unregister_signatures(&mut self, collection: &SignatureCollection) {
pub fn unregister(&mut self, indexes: impl Iterator<Item = VMSharedSignatureIndex>) { // If the collection has a populated signatures map, use it to deregister
for index in indexes { // This is always 1:1 from entry to registration
let removed = { if !collection.signatures.is_empty() {
let entry = self.entries[index.bits() as usize].as_mut().unwrap(); for (_, index) in collection.signatures.iter() {
self.unregister_entry(*index, 1);
debug_assert!(entry.references > 0); }
entry.references -= 1; } else {
// Otherwise, use the trampolines map, which has reference counts related
if entry.references == 0 { // to the stored index
self.map.remove(&entry.ty); for (index, (count, _)) in collection.trampolines.iter() {
self.free.push(index); self.unregister_entry(*index, *count);
true
} else {
false
}
};
if removed {
self.entries[index.bits() as usize] = None;
} }
} }
} }
/// Looks up a function type from a shared signature index. fn unregister_entry(&mut self, index: VMSharedSignatureIndex, count: usize) {
pub fn lookup_type(&self, index: VMSharedSignatureIndex) -> Option<&WasmFuncType> { let removed = {
self.entries let entry = self.entries[index.bits() as usize].as_mut().unwrap();
.get(index.bits() as usize)
.and_then(|e| e.as_ref().map(|e| &e.ty))
}
/// Determines if the registry is semantically empty. debug_assert!(entry.references >= count);
pub fn is_empty(&self) -> bool { entry.references -= count;
// If the map is empty, assert that all remaining entries are "free"
if self.map.is_empty() { if entry.references == 0 {
assert!(self.free.len() == self.entries.len()); self.map.remove(&entry.ty);
true self.free.push(index);
} else { true
false } else {
false
}
};
if removed {
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.0
.read()
.unwrap()
.entries
.get(index.bits() as usize)
.and_then(|e| e.as_ref().map(|e| &e.ty).cloned())
}
}

View File

@@ -1,6 +1,6 @@
use crate::{ use crate::{
module::ModuleRegistry, signatures::TrampolineMap, trampoline::StoreInstanceHandle, Engine, module::ModuleRegistry, signatures::SignatureCollection, trampoline::StoreInstanceHandle,
Func, Module, Trap, Engine, Func, Module, Trap,
}; };
use anyhow::{bail, Result}; use anyhow::{bail, Result};
use std::any::{Any, TypeId}; use std::any::{Any, TypeId};
@@ -15,11 +15,10 @@ use std::ptr;
use std::rc::Rc; use std::rc::Rc;
use std::sync::Arc; use std::sync::Arc;
use std::task::{Context, Poll}; use std::task::{Context, Poll};
use wasmtime_environ::wasm::WasmFuncType;
use wasmtime_jit::ModuleCode; use wasmtime_jit::ModuleCode;
use wasmtime_runtime::{ use wasmtime_runtime::{
InstanceAllocator, InstanceHandle, OnDemandInstanceAllocator, SignalHandler, TrapInfo, InstanceAllocator, InstanceHandle, OnDemandInstanceAllocator, SignalHandler, TrapInfo,
VMContext, VMExternRef, VMExternRefActivationsTable, VMInterrupts, VMSharedSignatureIndex, VMCallerCheckedAnyfunc, VMContext, VMExternRef, VMExternRefActivationsTable, VMInterrupts,
VMTrampoline, VMTrampoline,
}; };
@@ -76,7 +75,8 @@ pub(crate) struct StoreInner {
signal_handler: RefCell<Option<Box<SignalHandler<'static>>>>, signal_handler: RefCell<Option<Box<SignalHandler<'static>>>>,
externref_activations_table: VMExternRefActivationsTable, externref_activations_table: VMExternRefActivationsTable,
modules: RefCell<ModuleRegistry>, modules: RefCell<ModuleRegistry>,
trampolines: RefCell<TrampolineMap>, // The signatures and trampolines for `Func` objects
signatures: RefCell<SignatureCollection>,
// Numbers of resources instantiated in this store. // Numbers of resources instantiated in this store.
instance_count: Cell<usize>, instance_count: Cell<usize>,
memory_count: Cell<usize>, memory_count: Cell<usize>,
@@ -139,7 +139,7 @@ impl Store {
signal_handler: RefCell::new(None), signal_handler: RefCell::new(None),
externref_activations_table: VMExternRefActivationsTable::new(), externref_activations_table: VMExternRefActivationsTable::new(),
modules: RefCell::new(ModuleRegistry::default()), modules: RefCell::new(ModuleRegistry::default()),
trampolines: RefCell::new(TrampolineMap::default()), signatures: RefCell::new(SignatureCollection::new(engine.signatures())),
instance_count: Default::default(), instance_count: Default::default(),
memory_count: Default::default(), memory_count: Default::default(),
table_count: Default::default(), table_count: Default::default(),
@@ -203,41 +203,31 @@ impl Store {
} }
} }
pub(crate) fn register_module(&self, module: &Module) { pub(crate) fn signatures(&self) -> &RefCell<SignatureCollection> {
// Register the module with the registry &self.inner.signatures
self.inner.modules.borrow_mut().register(module);
} }
// This is used to register a `Func` with the store pub(crate) fn lookup_trampoline(&self, anyfunc: &VMCallerCheckedAnyfunc) -> VMTrampoline {
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 {
// Look up the trampoline with the store's trampolines (from `Func`). // 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; return trampoline;
} }
// Look up the trampoline with the registered modules // 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; return trampoline;
} }
// Lastly, check with the engine (for `HostFunc`) // Lastly, check with the engine (for `HostFunc`)
self.inner self.inner
.engine .engine
.host_func_trampolines() .host_func_signatures()
.get(index) .trampoline(anyfunc.type_index)
.expect("trampoline missing") .expect("trampoline missing")
} }
@@ -451,7 +441,7 @@ impl Store {
} }
#[inline] #[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() 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> { 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 // 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 _) self.modules.borrow().lookup_stack_map(pc).map(|m| m as _)

View File

@@ -77,7 +77,7 @@ fn create_handle(
externref_activations_table: store.externref_activations_table() externref_activations_table: store.externref_activations_table()
as *const VMExternRefActivationsTable as *const VMExternRefActivationsTable
as *mut _, as *mut _,
stack_map_lookup: Some(store.stack_map_lookup()), stack_map_lookup: Some(std::mem::transmute(store.stack_map_lookup())),
}, },
)?; )?;

View File

@@ -270,7 +270,11 @@ pub fn create_function(
// If there is no store, use the default signature index which is // 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) // guaranteed to trap if there is ever an indirect call on the function (should not happen)
let shared_signature_id = store 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()); .unwrap_or(VMSharedSignatureIndex::default());
unsafe { unsafe {

View File

@@ -1,4 +1,4 @@
use crate::{signatures::SharedSignatures, Extern, Store}; use crate::{signatures::SignatureCollection, Extern, Store};
use anyhow::{bail, Context, Result}; use anyhow::{bail, Context, Result};
use wasmtime_environ::wasm::{ use wasmtime_environ::wasm::{
EntityType, Global, InstanceTypeIndex, Memory, ModuleTypeIndex, SignatureIndex, Table, EntityType, Global, InstanceTypeIndex, Memory, ModuleTypeIndex, SignatureIndex, Table,
@@ -6,7 +6,7 @@ use wasmtime_environ::wasm::{
use wasmtime_jit::TypeTables; use wasmtime_jit::TypeTables;
pub struct MatchCx<'a> { pub struct MatchCx<'a> {
pub signatures: &'a SharedSignatures, pub signatures: &'a SignatureCollection,
pub types: &'a TypeTables, pub types: &'a TypeTables,
pub store: &'a Store, pub store: &'a Store,
} }
@@ -71,8 +71,8 @@ impl MatchCx<'_> {
} }
pub fn func(&self, expected: SignatureIndex, actual: &crate::Func) -> Result<()> { pub fn func(&self, expected: SignatureIndex, actual: &crate::Func) -> Result<()> {
let matches = match self.signatures.get(expected) { let matches = match self.signatures.shared_signature(expected) {
Some(idx) => actual.sig_index() == *idx, Some(idx) => actual.sig_index() == idx,
// If our expected signature isn't registered, then there's no way // If our expected signature isn't registered, then there's no way
// that `actual` can match it. // that `actual` can match it.
None => false, None => false,
@@ -133,7 +133,7 @@ impl MatchCx<'_> {
fn imports_match<'a>( fn imports_match<'a>(
&self, &self,
expected: ModuleTypeIndex, expected: ModuleTypeIndex,
actual_signatures: &SharedSignatures, actual_signatures: &SignatureCollection,
actual_types: &TypeTables, actual_types: &TypeTables,
actual_imports: impl Iterator<Item = (&'a str, EntityType)>, actual_imports: impl Iterator<Item = (&'a str, EntityType)>,
) -> Result<()> { ) -> Result<()> {
@@ -162,7 +162,7 @@ impl MatchCx<'_> {
fn exports_match( fn exports_match(
&self, &self,
expected: InstanceTypeIndex, expected: InstanceTypeIndex,
actual_signatures: &SharedSignatures, actual_signatures: &SignatureCollection,
actual_types: &TypeTables, actual_types: &TypeTables,
lookup: impl Fn(&str) -> Option<EntityType>, lookup: impl Fn(&str) -> Option<EntityType>,
) -> Result<()> { ) -> Result<()> {
@@ -186,7 +186,7 @@ impl MatchCx<'_> {
&self, &self,
expected: &EntityType, expected: &EntityType,
actual_ty: &EntityType, actual_ty: &EntityType,
actual_signatures: &SharedSignatures, actual_signatures: &SignatureCollection,
actual_types: &TypeTables, actual_types: &TypeTables,
) -> Result<()> { ) -> Result<()> {
let actual_desc = match actual_ty { let actual_desc = match actual_ty {