Improve signature lookup happening during instantiation (#2818)

This commit is intended to be a perf improvement for instantiation of
modules with lots of functions. Previously the `lookup_shared_signature`
callback was showing up quite high in profiles as part of instantiation.

As some background, this callback is used to translate from a module's
`SignatureIndex` to a `VMSharedSignatureIndex` which the instance
stores. This callback is called for two reasons, one is to translate all
of the module's own types into `VMSharedSignatureIndex` for the purposes
of `call_indirect` (the translation of that loads from this table to
compare indices). The second reason is that a `VMCallerCheckedAnyfunc`
is prepared for all functions and this embeds a `VMSharedSignatureIndex`
inside of it.

The slow part today is that the lookup callback was called
once-per-function and each lookup involved hashing a full
`WasmFuncType`. Albeit our hash algorithm is still Rust's default
SipHash algorithm which is quite slow, but we also shouldn't need to
re-hash each signature if we see it multiple times anyway.

The fix applied in this commit is to change this lookup callback to an
`enum` where one variant is that there's a table to lookup from. This
table is a `PrimaryMap` which means that lookup is quite fast. The only
thing we need to do is to prepare the table ahead of time. Currently
this happens on the instantiation path because in my measurments the
creation of the table is quite fast compared to the rest of
instantiation. If this becomes an issue, though, we can look into
creating the table as part of `SigRegistry::register_module` and caching
it somewhere (I'm not entirely sure where but I'm sure we can figure it
out).

There's in generally not a ton of efficiency around the `SigRegistry`
type. I'm hoping though that this fixes the next-lowest-hanging-fruit in
terms of performance without complicating the implementation too much. I
tried a few variants and this change seemed like the best balance
between simplicity and still a nice performance gain.

Locally I measured an improvement in instantiation time for a large-ish
module by reducing the time from ~3ms to ~2.6ms per instance.
This commit is contained in:
Alex Crichton
2021-04-08 15:04:18 -05:00
committed by GitHub
parent 8e495ac79d
commit 18dd82ba7d
13 changed files with 144 additions and 101 deletions

View File

@@ -298,6 +298,7 @@ impl CompiledModule {
} }
/// Returns the map of all finished JIT functions compiled for this module /// Returns the map of all finished JIT functions compiled for this module
#[inline]
pub fn finished_functions(&self) -> &PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]> { pub fn finished_functions(&self) -> &PrimaryMap<DefinedFuncIndex, *mut [VMFunctionBody]> {
&self.finished_functions.0 &self.finished_functions.0
} }

View File

@@ -828,16 +828,6 @@ impl StackMapRegistry {
let mut inner = self.inner.borrow_mut(); let mut inner = self.inner.borrow_mut();
// Check if we've already registered this module.
if let Some(existing_module) = inner.ranges.get(&max) {
assert_eq!(existing_module.range, module_stack_maps.range);
debug_assert_eq!(
existing_module.pc_to_stack_map,
module_stack_maps.pc_to_stack_map,
);
return;
}
// Assert that this chunk of ranges doesn't collide with any other known // Assert that this chunk of ranges doesn't collide with any other known
// chunks. // chunks.
if let Some((_, prev)) = inner.ranges.range(max..).next() { if let Some((_, prev)) = inner.ranges.range(max..).next() {

View File

@@ -45,8 +45,8 @@ pub struct InstanceAllocationRequest<'a> {
/// The imports to use for the instantiation. /// The imports to use for the instantiation.
pub imports: Imports<'a>, pub imports: Imports<'a>,
/// A callback for looking up shared signature indexes. /// Translation from `SignatureIndex` to `VMSharedSignatureIndex`
pub lookup_shared_signature: &'a dyn Fn(SignatureIndex) -> VMSharedSignatureIndex, pub shared_signatures: SharedSignatures<'a>,
/// The host state to associate with the instance. /// The host state to associate with the instance.
pub host_state: Box<dyn Any>, pub host_state: Box<dyn Any>,
@@ -165,6 +165,46 @@ pub unsafe trait InstanceAllocator: Send + Sync {
unsafe fn deallocate_fiber_stack(&self, stack: &wasmtime_fiber::FiberStack); unsafe fn deallocate_fiber_stack(&self, stack: &wasmtime_fiber::FiberStack);
} }
pub enum SharedSignatures<'a> {
/// Used for instantiating user-defined modules
Table(&'a PrimaryMap<SignatureIndex, VMSharedSignatureIndex>),
/// Used for instance creation that has only a single function
Always(VMSharedSignatureIndex),
/// Used for instance creation that has no functions
None,
}
impl SharedSignatures<'_> {
fn lookup(&self, index: SignatureIndex) -> VMSharedSignatureIndex {
match self {
SharedSignatures::Table(table) => table[index],
SharedSignatures::Always(index) => *index,
SharedSignatures::None => unreachable!(),
}
}
}
impl<'a> From<VMSharedSignatureIndex> for SharedSignatures<'a> {
fn from(val: VMSharedSignatureIndex) -> SharedSignatures<'a> {
SharedSignatures::Always(val)
}
}
impl<'a> From<Option<VMSharedSignatureIndex>> for SharedSignatures<'a> {
fn from(val: Option<VMSharedSignatureIndex>) -> SharedSignatures<'a> {
match val {
Some(idx) => SharedSignatures::Always(idx),
None => SharedSignatures::None,
}
}
}
impl<'a> From<&'a PrimaryMap<SignatureIndex, VMSharedSignatureIndex>> for SharedSignatures<'a> {
fn from(val: &'a PrimaryMap<SignatureIndex, VMSharedSignatureIndex>) -> SharedSignatures<'a> {
SharedSignatures::Table(val)
}
}
fn get_table_init_start( fn get_table_init_start(
init: &TableInitializer, init: &TableInitializer,
instance: &Instance, instance: &Instance,
@@ -413,7 +453,7 @@ unsafe fn initialize_vmcontext(instance: &Instance, req: InstanceAllocationReque
let mut ptr = instance.signature_ids_ptr(); let mut ptr = instance.signature_ids_ptr();
for sig in module.types.values() { for sig in module.types.values() {
*ptr = match sig { *ptr = match sig {
ModuleType::Function(sig) => (req.lookup_shared_signature)(*sig), ModuleType::Function(sig) => req.shared_signatures.lookup(*sig),
_ => VMSharedSignatureIndex::new(u32::max_value()), _ => VMSharedSignatureIndex::new(u32::max_value()),
}; };
ptr = ptr.add(1); ptr = ptr.add(1);
@@ -453,7 +493,7 @@ unsafe fn initialize_vmcontext(instance: &Instance, req: InstanceAllocationReque
// Initialize the functions // Initialize the functions
for (index, sig) in instance.module.functions.iter() { for (index, sig) in instance.module.functions.iter() {
let type_index = (req.lookup_shared_signature)(*sig); let type_index = req.shared_signatures.lookup(*sig);
let (func_ptr, vmctx) = if let Some(def_index) = instance.module.defined_func_index(index) { let (func_ptr, vmctx) = if let Some(def_index) = instance.module.defined_func_index(index) {
( (

View File

@@ -1366,7 +1366,7 @@ mod test {
memories: &[], memories: &[],
globals: &[], globals: &[],
}, },
lookup_shared_signature: &|_| VMSharedSignatureIndex::default(), shared_signatures: VMSharedSignatureIndex::default().into(),
host_state: Box::new(()), host_state: Box::new(()),
interrupts: std::ptr::null(), interrupts: std::ptr::null(),
externref_activations_table: std::ptr::null_mut(), externref_activations_table: std::ptr::null_mut(),
@@ -1390,7 +1390,7 @@ mod test {
memories: &[], memories: &[],
globals: &[], globals: &[],
}, },
lookup_shared_signature: &|_| VMSharedSignatureIndex::default(), shared_signatures: VMSharedSignatureIndex::default().into(),
host_state: Box::new(()), host_state: Box::new(()),
interrupts: std::ptr::null(), interrupts: std::ptr::null(),
externref_activations_table: std::ptr::null_mut(), externref_activations_table: std::ptr::null_mut(),

View File

@@ -519,7 +519,7 @@ mod test {
memories: &[], memories: &[],
globals: &[], globals: &[],
}, },
lookup_shared_signature: &|_| VMSharedSignatureIndex::default(), shared_signatures: VMSharedSignatureIndex::default().into(),
host_state: Box::new(()), host_state: Box::new(()),
interrupts: ptr::null(), interrupts: ptr::null(),
externref_activations_table: ptr::null_mut(), externref_activations_table: ptr::null_mut(),

View File

@@ -43,15 +43,6 @@ impl StoreFrameInfo {
.map(|info| (info, module.has_unparsed_debuginfo())) .map(|info| (info, module.has_unparsed_debuginfo()))
} }
/// Returns whether the `pc` specified is contained within some module's
/// function.
pub fn contains_pc(&self, pc: usize) -> bool {
match self.module(pc) {
Some(module) => module.contains_pc(pc),
None => false,
}
}
/// 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> {
self.module(pc)?.lookup_trap_info(pc) self.module(pc)?.lookup_trap_info(pc)
@@ -79,10 +70,6 @@ impl StoreFrameInfo {
// may be a valid PC value // may be a valid PC value
let end = end - 1; let end = end - 1;
if self.contains_pc(start) {
return;
}
// Assert that this module's code doesn't collide with any other registered modules // Assert that this module's code doesn't collide with any other registered modules
if let Some((_, prev)) = self.ranges.range(end..).next() { if let Some((_, prev)) = self.ranges.range(end..).next() {
assert!(prev.start > end); assert!(prev.start > end);
@@ -191,12 +178,6 @@ impl ModuleFrameInfo {
}) })
} }
/// Returns whether the `pc` specified is contained within some module's
/// function.
pub fn contains_pc(&self, pc: usize) -> bool {
self.func(pc).is_some()
}
/// Fetches trap information about a program counter in a backtrace. /// 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) = self.func(pc)?;

View File

@@ -1810,7 +1810,7 @@ macro_rules! impl_into_func {
// If not given a registry, use a default signature index that is guaranteed to trap // If not given a registry, 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 = registry let shared_signature_id = registry
.map(|r| r.register(ty.as_wasm_func_type(), Some(trampoline))) .map(|r| r.register(ty.as_wasm_func_type(), trampoline))
.unwrap_or(VMSharedSignatureIndex::default()); .unwrap_or(VMSharedSignatureIndex::default());
let instance = unsafe { let instance = unsafe {

View File

@@ -513,14 +513,14 @@ impl<'a> Instantiator<'a> {
unsafe { unsafe {
let engine = self.store.engine(); let engine = self.store.engine();
let allocator = engine.allocator(); let allocator = engine.allocator();
let signatures = self.store.signatures().borrow();
let signatures = signatures.lookup_table(&self.cur.module);
let instance = allocator.allocate(InstanceAllocationRequest { let instance = allocator.allocate(InstanceAllocationRequest {
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(),
lookup_shared_signature: &self shared_signatures: (&signatures).into(),
.store
.lookup_shared_signature(self.cur.module.types()),
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()

View File

@@ -1,9 +1,11 @@
//! 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 crate::Module;
use std::collections::{hash_map, HashMap}; use std::collections::{hash_map, HashMap};
use std::convert::TryFrom; use std::convert::TryFrom;
use wasmtime_environ::wasm::WasmFuncType; use wasmtime_environ::entity::PrimaryMap;
use wasmtime_environ::wasm::{SignatureIndex, WasmFuncType};
use wasmtime_runtime::{VMSharedSignatureIndex, VMTrampoline}; use wasmtime_runtime::{VMSharedSignatureIndex, VMTrampoline};
/// WebAssembly requires that the caller and callee signatures in an indirect /// WebAssembly requires that the caller and callee signatures in an indirect
@@ -33,12 +35,40 @@ struct Entry {
} }
impl SignatureRegistry { impl SignatureRegistry {
/// Register a signature and return its unique index. /// Registers all signatures within a module into this registry all at once.
/// ///
/// Note that `trampoline` can be `None` which indicates that an index is /// This will also internally register trampolines compiled in the module.
/// desired for this signature but the trampoline for it is not compiled or pub fn register_module(&mut self, module: &Module) {
/// available. // Register a unique index for all types in this module, even if they
// don't have a trampoline.
let signatures = &module.types().wasm_signatures;
for ty in module.compiled_module().module().types.values() {
if let wasmtime_environ::ModuleType::Function(index) = ty {
self.register_one(&signatures[*index], None);
}
}
// Once we've got a shared index for all types used then also fill in
// any trampolines that the module has compiled as well.
for (index, trampoline) in module.compiled_module().trampolines() {
let shared = self.wasm2index[&signatures[*index]];
let entry = &mut self.index_map[shared.bits() as usize];
if entry.trampoline.is_none() {
entry.trampoline = Some(*trampoline);
}
}
}
/// Register a signature and return its unique index.
pub fn register( pub fn register(
&mut self,
wasm: &WasmFuncType,
trampoline: VMTrampoline,
) -> VMSharedSignatureIndex {
self.register_one(wasm, Some(trampoline))
}
fn register_one(
&mut self, &mut self,
wasm: &WasmFuncType, wasm: &WasmFuncType,
trampoline: Option<VMTrampoline>, trampoline: Option<VMTrampoline>,
@@ -80,6 +110,34 @@ impl SignatureRegistry {
self.wasm2index.get(wasm).cloned() self.wasm2index.get(wasm).cloned()
} }
/// Builds a lookup table for a module from the possible module's signature
/// indices to the shared signature index within this registry.
pub fn lookup_table(
&self,
module: &Module,
) -> PrimaryMap<SignatureIndex, VMSharedSignatureIndex> {
// For module-linking using modules this builds up a map that is
// too large. This builds up a map for everything in `TypeTables` but
// that's all the types for all modules in a whole module linking graph,
// which our `module` may not be using.
//
// For all non-module-linking-using modules, though, this is not an
// issue. This is optimizing for the non-module-linking case right now
// and it seems like module linking will likely change to the point that
// this will no longer be an issue in the future.
let signatures = &module.types().wasm_signatures;
let mut map = PrimaryMap::with_capacity(signatures.len());
for wasm in signatures.values() {
map.push(
self.wasm2index
.get(wasm)
.cloned()
.unwrap_or(VMSharedSignatureIndex::new(u32::MAX)),
);
}
map
}
/// Looks up information known about a shared signature index. /// Looks up information known about a shared signature index.
/// ///
/// Note that for this operation to be semantically correct the `idx` must /// Note that for this operation to be semantically correct the `idx` must

View File

@@ -17,11 +17,11 @@ 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; use wasmtime_environ::wasm;
use wasmtime_jit::{CompiledModule, ModuleCode, TypeTables}; use wasmtime_jit::{CompiledModule, ModuleCode};
use wasmtime_runtime::{ use wasmtime_runtime::{
Export, InstanceAllocator, InstanceHandle, OnDemandInstanceAllocator, SignalHandler, Export, InstanceAllocator, InstanceHandle, OnDemandInstanceAllocator, SignalHandler,
StackMapRegistry, TrapInfo, VMCallerCheckedAnyfunc, VMContext, VMExternRef, StackMapRegistry, TrapInfo, VMCallerCheckedAnyfunc, VMContext, VMExternRef,
VMExternRefActivationsTable, VMInterrupts, VMSharedSignatureIndex, VMTrampoline, VMExternRefActivationsTable, VMInterrupts, VMTrampoline,
}; };
/// Used to associate instances with the store. /// Used to associate instances with the store.
@@ -202,7 +202,7 @@ impl Store {
.inner .inner
.signatures .signatures
.borrow_mut() .borrow_mut()
.register(ty.as_wasm_func_type(), Some(trampoline)); .register(ty.as_wasm_func_type(), trampoline);
Box::new(anyfunc) Box::new(anyfunc)
}); });
@@ -248,19 +248,23 @@ impl Store {
&self.inner.signatures &self.inner.signatures
} }
pub(crate) fn lookup_shared_signature<'a>( pub(crate) fn register_module(&self, module: &Module) {
&'a self, // With a module being instantiated into this `Store` we need to
types: &'a TypeTables, // preserve its jit-code. References to this module's code and
) -> impl Fn(wasm::SignatureIndex) -> VMSharedSignatureIndex + 'a { // trampolines are not owning-references so it's our responsibility to
move |index| { // keep it all alive within the `Store`.
self.signatures() //
.borrow() // If this module is already present in the store then we skip all
.lookup(&types.wasm_signatures[index]) // further registration steps.
.expect("signature not previously registered") let first = self
} .inner
.modules
.borrow_mut()
.insert(ArcModuleCode(module.compiled_module().code().clone()));
if !first {
return;
} }
pub(crate) fn register_module(&self, module: &Module) {
// All modules register their JIT code in a store for two reasons // All modules register their JIT code in a store for two reasons
// currently: // currently:
// //
@@ -271,7 +275,10 @@ impl Store {
// * Second when generating a backtrace we'll use this mapping to // * Second when generating a backtrace we'll use this mapping to
// only generate wasm frames for instruction pointers that fall // only generate wasm frames for instruction pointers that fall
// within jit code. // within jit code.
self.register_jit_code(module.compiled_module()); self.inner
.frame_info
.borrow_mut()
.register(module.compiled_module());
// We need to know about all the stack maps of all instantiated modules // We need to know about all the stack maps of all instantiated modules
// so when performing a GC we know about all wasm frames that we find // so when performing a GC we know about all wasm frames that we find
@@ -282,20 +289,7 @@ impl Store {
// once-per-module (and once-per-signature). This allows us to create // once-per-module (and once-per-signature). This allows us to create
// a `Func` wrapper for any function in the module, which requires that // a `Func` wrapper for any function in the module, which requires that
// we know about the signature and trampoline for all instances. // we know about the signature and trampoline for all instances.
self.register_signatures(module); self.signatures().borrow_mut().register_module(module);
// And finally with a module being instantiated into this `Store` we
// need to preserve its jit-code. References to this module's code and
// trampolines are not owning-references so it's our responsibility to
// keep it all alive within the `Store`.
self.inner
.modules
.borrow_mut()
.insert(ArcModuleCode(module.compiled_module().code().clone()));
}
fn register_jit_code(&self, module: &Arc<CompiledModule>) {
self.inner.frame_info.borrow_mut().register(module)
} }
fn register_stack_maps(&self, module: &CompiledModule) { fn register_stack_maps(&self, module: &CompiledModule) {
@@ -310,27 +304,6 @@ impl Store {
})); }));
} }
fn register_signatures(&self, module: &Module) {
let mut signatures = self.signatures().borrow_mut();
let types = module.types();
// Register a unique index for all types in this module, even if they
// don't have a trampoline.
for (_, ty) in module.compiled_module().module().types.iter() {
if let wasmtime_environ::ModuleType::Function(index) = ty {
let wasm = &types.wasm_signatures[*index];
signatures.register(wasm, None);
}
}
// Afterwards register all compiled trampolines for this module with the
// signature registry as well.
for (index, trampoline) in module.compiled_module().trampolines() {
let wasm = &types.wasm_signatures[*index];
signatures.register(wasm, Some(*trampoline));
}
}
pub(crate) fn bump_resource_counts(&self, module: &Module) -> Result<()> { pub(crate) fn bump_resource_counts(&self, module: &Module) -> Result<()> {
let config = self.engine().config(); let config = self.engine().config();

View File

@@ -71,7 +71,7 @@ fn create_handle(
module: Arc::new(module), module: Arc::new(module),
finished_functions: &finished_functions, finished_functions: &finished_functions,
imports, imports,
lookup_shared_signature: &|_| shared_signature_id.unwrap(), shared_signatures: shared_signature_id.into(),
host_state, host_state,
interrupts: store.interrupts(), interrupts: store.interrupts(),
externref_activations_table: store.externref_activations_table() externref_activations_table: store.externref_activations_table()

View File

@@ -37,7 +37,7 @@ pub(crate) fn create_handle(
module: module.clone(), module: module.clone(),
finished_functions: &finished_functions, finished_functions: &finished_functions,
imports, imports,
lookup_shared_signature: &|_| shared_signature_id.unwrap(), shared_signatures: shared_signature_id.into(),
host_state, host_state,
interrupts: store.interrupts(), interrupts: store.interrupts(),
externref_activations_table: store.externref_activations_table() externref_activations_table: store.externref_activations_table()

View File

@@ -270,7 +270,7 @@ pub fn create_function(
// If there is no signature registry, use the default signature index which is // If there is no signature registry, 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 = registry let shared_signature_id = registry
.map(|r| r.register(ft.as_wasm_func_type(), Some(trampoline))) .map(|r| r.register(ft.as_wasm_func_type(), trampoline))
.unwrap_or(VMSharedSignatureIndex::default()); .unwrap_or(VMSharedSignatureIndex::default());
unsafe { unsafe {
@@ -279,7 +279,7 @@ pub fn create_function(
module: Arc::new(module), module: Arc::new(module),
finished_functions: &finished_functions, finished_functions: &finished_functions,
imports: Imports::default(), imports: Imports::default(),
lookup_shared_signature: &|_| shared_signature_id, shared_signatures: shared_signature_id.into(),
host_state: Box::new(trampoline_state), host_state: Box::new(trampoline_state),
interrupts: std::ptr::null(), interrupts: std::ptr::null(),
externref_activations_table: std::ptr::null_mut(), externref_activations_table: std::ptr::null_mut(),
@@ -311,7 +311,7 @@ pub unsafe fn create_raw_function(
module: Arc::new(module), module: Arc::new(module),
finished_functions: &finished_functions, finished_functions: &finished_functions,
imports: Imports::default(), imports: Imports::default(),
lookup_shared_signature: &|_| shared_signature_id, shared_signatures: shared_signature_id.into(),
host_state, host_state,
interrupts: std::ptr::null(), interrupts: std::ptr::null(),
externref_activations_table: std::ptr::null_mut(), externref_activations_table: std::ptr::null_mut(),