Refactor the internals of Store<T> (#3291)

* Refactor the internals of `Store<T>`

This commit is an overdue refactoring and renaming of some internals of
the `Store` type in Wasmtime. The actual implementation of `Store<T>`
has evolved from the original implementation to the point where some of
the aspects of how things are structured no longer makes sense. There's
also always been a lot of unnecessary gymnastics when trying to get
access to various store pieces depending on where you are in `wasmtime`.

This refactoring aims to simplify all this and make the internals much
easier to read/write. The following changes were made:

* The `StoreOpaque<'_>` type is deleted, along with the `opaque()`
  method.

* The `StoreInnermost` type was renamed to `StoreOpaque`.
  `StoreOpaque<'_>` is dead. Long live `StoreOpaque`. This renaming
  and a few small tweaks means that this type now suffices for all
  consumers.

* The `AsContextMut` and `AsContext` traits are now implemented for
  `StoreInner<T>`.

These changes, while subtly small, help clean up a lot of the internals
of `wasmtime`. There's a lot less verbose `&mut
store.as_context_mut().opaque()` now. Additionally many methods can
simply start with `let store = store.as_context_mut().0;` and use things
internally. One of the nicer aspects of using references directly is
that the compiler automatically reborrows references as necessary
meaning there's lots of less manual reborrowing.

The main motivation for this change was actually somewhat roundabout
where I found that when `StoreOpaque<'_>` was being captured in closures
and iterators it's 3 pointers wide which is a lot of data to move
around. Now things capture over `&mut StoreOpaque` which is just one
nice and small pointer to move around. In any case though I've long
wanted to revisit the design of these internals to improve the
ergonomics. It's not expected that this change alone will really have
all that much impact on the performance of `wasmtime`.

Finally a doc comment was added to `store.rs` to try to explain all the
`Store`-related types since there are a nontrivial amount.

* Rustfmt
This commit is contained in:
Alex Crichton
2021-09-03 13:55:18 -05:00
committed by GitHub
parent 50ce19a4a4
commit ca3947911e
17 changed files with 283 additions and 238 deletions

View File

@@ -119,7 +119,7 @@ impl Extern {
pub(crate) unsafe fn from_wasmtime_export(
wasmtime_export: wasmtime_runtime::Export,
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
) -> Extern {
match wasmtime_export {
wasmtime_runtime::Export::Function(f) => {
@@ -137,7 +137,7 @@ impl Extern {
}
}
pub(crate) fn comes_from_same_store(&self, store: &StoreOpaque<'_>) -> bool {
pub(crate) fn comes_from_same_store(&self, store: &StoreOpaque) -> bool {
match self {
Extern::Func(f) => f.comes_from_same_store(store),
Extern::Global(g) => store.store_data().contains(g.0),
@@ -259,10 +259,10 @@ impl Global {
/// # }
/// ```
pub fn new(mut store: impl AsContextMut, ty: GlobalType, val: Val) -> Result<Global> {
Global::_new(&mut store.as_context_mut().opaque(), ty, val)
Global::_new(store.as_context_mut().0, ty, val)
}
fn _new(store: &mut StoreOpaque<'_>, ty: GlobalType, val: Val) -> Result<Global> {
fn _new(store: &mut StoreOpaque, ty: GlobalType, val: Val) -> Result<Global> {
if !val.comes_from_same_store(store) {
bail!("cross-`Store` globals are not supported");
}
@@ -307,7 +307,7 @@ impl Global {
.map(|inner| ExternRef { inner }),
),
ValType::FuncRef => {
from_checked_anyfunc(definition.as_anyfunc() as *mut _, &mut store.opaque())
from_checked_anyfunc(definition.as_anyfunc() as *mut _, store.0)
}
ValType::V128 => Val::V128(*definition.as_u128()),
}
@@ -326,7 +326,7 @@ impl Global {
///
/// Panics if `store` does not own this global.
pub fn set(&self, mut store: impl AsContextMut, val: Val) -> Result<()> {
let store = store.as_context_mut();
let store = store.as_context_mut().0;
let ty = self.ty(&store);
if ty.mutability() != Mutability::Var {
bail!("immutable global cannot be set");
@@ -335,8 +335,7 @@ impl Global {
if val.ty() != *ty {
bail!("global of type {:?} cannot be set to {:?}", ty, val.ty());
}
let mut store = store.opaque();
if !val.comes_from_same_store(&store) {
if !val.comes_from_same_store(store) {
bail!("cross-`Store` values are not supported");
}
unsafe {
@@ -348,7 +347,7 @@ impl Global {
Val::F64(f) => *definition.as_u64_mut() = f,
Val::FuncRef(f) => {
*definition.as_anyfunc_mut() = f.map_or(ptr::null(), |f| {
f.caller_checked_anyfunc(&mut store).as_ptr() as *const _
f.caller_checked_anyfunc(store).as_ptr() as *const _
});
}
Val::ExternRef(x) => {
@@ -363,7 +362,7 @@ impl Global {
pub(crate) unsafe fn from_wasmtime_global(
wasmtime_export: wasmtime_runtime::ExportGlobal,
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
) -> Global {
Global(store.store_data_mut().insert(wasmtime_export))
}
@@ -372,7 +371,7 @@ impl Global {
&data[self.0].global
}
pub(crate) fn vmimport(&self, store: &StoreOpaque<'_>) -> wasmtime_runtime::VMGlobalImport {
pub(crate) fn vmimport(&self, store: &StoreOpaque) -> wasmtime_runtime::VMGlobalImport {
wasmtime_runtime::VMGlobalImport {
from: store[self.0].definition,
}
@@ -435,7 +434,7 @@ impl Table {
/// # }
/// ```
pub fn new(mut store: impl AsContextMut, ty: TableType, init: Val) -> Result<Table> {
Table::_new(&mut store.as_context_mut().opaque(), ty, init)
Table::_new(store.as_context_mut().0, ty, init)
}
fn _new(store: &mut StoreOpaque, ty: TableType, init: Val) -> Result<Table> {
@@ -483,7 +482,7 @@ impl Table {
TableType::from_wasmtime_table(ty)
}
fn wasmtime_table(&self, store: &mut StoreOpaque<'_>) -> *mut runtime::Table {
fn wasmtime_table(&self, store: &mut StoreOpaque) -> *mut runtime::Table {
unsafe {
let export = &store[self.0];
let mut handle = InstanceHandle::from_vmctx(export.vmctx);
@@ -500,11 +499,11 @@ impl Table {
///
/// Panics if `store` does not own this table.
pub fn get(&self, mut store: impl AsContextMut, index: u32) -> Option<Val> {
let mut store = store.as_context_mut().opaque();
let table = self.wasmtime_table(&mut store);
let store = store.as_context_mut().0;
let table = self.wasmtime_table(store);
unsafe {
match (*table).get(index)? {
runtime::TableElement::FuncRef(f) => Some(from_checked_anyfunc(f, &mut store)),
runtime::TableElement::FuncRef(f) => Some(from_checked_anyfunc(f, store)),
runtime::TableElement::ExternRef(None) => Some(Val::ExternRef(None)),
runtime::TableElement::ExternRef(Some(x)) => {
Some(Val::ExternRef(Some(ExternRef { inner: x })))
@@ -525,10 +524,10 @@ impl Table {
///
/// Panics if `store` does not own this table.
pub fn set(&self, mut store: impl AsContextMut, index: u32, val: Val) -> Result<()> {
let store = store.as_context_mut().0;
let ty = self.ty(&store).element().clone();
let mut store = store.as_context_mut().opaque();
let val = val.into_table_element(&mut store, ty)?;
let table = self.wasmtime_table(&mut store);
let val = val.into_table_element(store, ty)?;
let table = self.wasmtime_table(store);
unsafe {
(*table)
.set(index, val)
@@ -562,12 +561,12 @@ impl Table {
///
/// Panics if `store` does not own this table.
pub fn grow(&self, mut store: impl AsContextMut, delta: u32, init: Val) -> Result<u32> {
let store = store.as_context_mut().0;
let ty = self.ty(&store).element().clone();
let init = init.into_table_element(&mut store.as_context_mut().opaque(), ty)?;
let table = self.wasmtime_table(&mut store.as_context_mut().opaque());
let store = store.as_context_mut();
let init = init.into_table_element(store, ty)?;
let table = self.wasmtime_table(store);
unsafe {
match (*table).grow(delta, init, store.0.limiter()) {
match (*table).grow(delta, init, store.limiter()) {
Some(size) => {
let vm = (*table).vmtable();
*store[self.0].definition = vm;
@@ -597,14 +596,13 @@ impl Table {
src_index: u32,
len: u32,
) -> Result<()> {
let store = store.as_context_mut().0;
if dst_table.ty(&store).element() != src_table.ty(&store).element() {
bail!("tables do not have the same element type");
}
let mut store = store.as_context_mut().opaque();
let dst = dst_table.wasmtime_table(&mut store);
let src = src_table.wasmtime_table(&mut store);
let dst = dst_table.wasmtime_table(store);
let src = src_table.wasmtime_table(store);
unsafe {
runtime::Table::copy(dst, src, dst_index, src_index, len)
.map_err(Trap::from_runtime)?;
@@ -629,11 +627,11 @@ impl Table {
///
/// Panics if `store` does not own either `dst_table` or `src_table`.
pub fn fill(&self, mut store: impl AsContextMut, dst: u32, val: Val, len: u32) -> Result<()> {
let store = store.as_context_mut().0;
let ty = self.ty(&store).element().clone();
let mut store = store.as_context_mut().opaque();
let val = val.into_table_element(&mut store, ty)?;
let val = val.into_table_element(store, ty)?;
let table = self.wasmtime_table(&mut store);
let table = self.wasmtime_table(store);
unsafe {
(*table).fill(dst, val, len).map_err(Trap::from_runtime)?;
}
@@ -643,7 +641,7 @@ impl Table {
pub(crate) unsafe fn from_wasmtime_table(
wasmtime_export: wasmtime_runtime::ExportTable,
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
) -> Table {
Table(store.store_data_mut().insert(wasmtime_export))
}
@@ -652,7 +650,7 @@ impl Table {
&data[self.0].table.table
}
pub(crate) fn vmimport(&self, store: &StoreOpaque<'_>) -> wasmtime_runtime::VMTableImport {
pub(crate) fn vmimport(&self, store: &StoreOpaque) -> wasmtime_runtime::VMTableImport {
let export = &store[self.0];
wasmtime_runtime::VMTableImport {
from: export.definition,

View File

@@ -1,4 +1,4 @@
use crate::store::{StoreData, StoreInnermost, StoreOpaque, Stored};
use crate::store::{StoreData, StoreOpaque, Stored};
use crate::{
AsContext, AsContextMut, Engine, Extern, FuncType, Instance, InterruptHandle, StoreContext,
StoreContextMut, Trap, Val, ValType,
@@ -259,7 +259,7 @@ macro_rules! generate_wrap_async_func {
{
assert!(store.as_context().async_support(), concat!("cannot use `wrap", $num, "_async` without enabling async support on the config"));
Func::wrap(store, move |mut caller: Caller<'_, T>, $($args: $args),*| {
let async_cx = caller.store.as_context_mut().opaque().async_cx();
let async_cx = caller.store.as_context_mut().0.async_cx();
let mut future = Pin::from(func(caller, $($args),*));
match unsafe { async_cx.block_on(future.as_mut()) } {
Ok(ret) => ret.into_fallible(),
@@ -307,13 +307,13 @@ impl Func {
ty: FuncType,
func: impl Fn(Caller<'_, T>, &[Val], &mut [Val]) -> Result<(), Trap> + Send + Sync + 'static,
) -> Self {
let mut store = store.as_context_mut().opaque();
let store = store.as_context_mut().0;
// part of this unsafety is about matching the `T` to a `Store<T>`,
// which is done through the `AsContextMut` bound above.
unsafe {
let host = HostFunc::new(store.engine(), ty, func);
host.into_func(&mut store)
host.into_func(store)
}
}
@@ -398,7 +398,7 @@ impl Func {
"cannot use `new_async` without enabling async support in the config"
);
Func::new(store, ty, move |mut caller, params, results| {
let async_cx = caller.store.as_context_mut().opaque().async_cx();
let async_cx = caller.store.as_context_mut().0.async_cx();
let mut future = Pin::from(func(caller, params, results));
match unsafe { async_cx.block_on(future.as_mut()) } {
Ok(Ok(())) => Ok(()),
@@ -631,12 +631,12 @@ impl Func {
mut store: impl AsContextMut<Data = T>,
func: impl IntoFunc<T, Params, Results>,
) -> Func {
let mut store = store.as_context_mut().opaque();
let store = store.as_context_mut().0;
// part of this unsafety is about matching the `T` to a `Store<T>`,
// which is done through the `AsContextMut` bound above.
unsafe {
let host = HostFunc::wrap(store.engine(), func);
host.into_func(&mut store)
host.into_func(store)
}
}
@@ -736,7 +736,7 @@ impl Func {
my_ty: FuncType,
params: &[Val],
) -> Result<Box<[Val]>> {
let mut values_vec = write_params(&mut store.as_context_mut().opaque(), &my_ty, params)?;
let mut values_vec = write_params(store.0, &my_ty, params)?;
// Call the trampoline.
unsafe {
@@ -753,14 +753,10 @@ impl Func {
})?;
}
return Ok(read_results(
&mut store.as_context_mut().opaque(),
&my_ty,
&values_vec,
));
return Ok(read_results(store.0, &my_ty, &values_vec));
fn write_params(
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
ty: &FuncType,
params: &[Val],
) -> Result<Vec<u128>> {
@@ -800,11 +796,7 @@ impl Func {
Ok(values_vec)
}
fn read_results(
store: &mut StoreOpaque<'_>,
ty: &FuncType,
values_vec: &[u128],
) -> Box<[Val]> {
fn read_results(store: &mut StoreOpaque, ty: &FuncType, values_vec: &[u128]) -> Box<[Val]> {
let mut results = Vec::with_capacity(ty.results().len());
for (index, ty) in ty.results().enumerate() {
unsafe {
@@ -819,7 +811,7 @@ impl Func {
#[inline]
pub(crate) fn caller_checked_anyfunc(
&self,
store: &StoreInnermost,
store: &StoreOpaque,
) -> NonNull<VMCallerCheckedAnyfunc> {
store.store_data()[self.0].export().anyfunc
}
@@ -834,7 +826,7 @@ impl Func {
Func(store.store_data_mut().insert(data))
}
pub(crate) fn vmimport(&self, store: &mut StoreOpaque<'_>) -> VMFunctionImport {
pub(crate) fn vmimport(&self, store: &mut StoreOpaque) -> VMFunctionImport {
unsafe {
let f = self.caller_checked_anyfunc(store);
VMFunctionImport {
@@ -861,10 +853,9 @@ impl Func {
const STACK_ARGS: usize = 4;
const STACK_RETURNS: usize = 2;
let mut args: SmallVec<[Val; STACK_ARGS]> = SmallVec::with_capacity(ty.params().len());
let mut store = caller.store.as_context_mut().opaque();
for (i, ty) in ty.params().enumerate() {
unsafe {
let val = Val::read_value_from(&mut store, values_vec.add(i), ty);
let val = Val::read_value_from(caller.store.0, values_vec.add(i), ty);
args.push(val);
}
}
@@ -878,20 +869,19 @@ impl Func {
// values produced are correct. There could be a bug in `func` that
// produces the wrong number, wrong types, or wrong stores of
// values, and we need to catch that here.
let mut store = caller.store.as_context_mut().opaque();
for (i, (ret, ty)) in returns.into_iter().zip(ty.results()).enumerate() {
if ret.ty() != ty {
return Err(Trap::new(
"function attempted to return an incompatible value",
));
}
if !ret.comes_from_same_store(&store) {
if !ret.comes_from_same_store(&caller.store.0) {
return Err(Trap::new(
"cross-`Store` values are not currently supported",
));
}
unsafe {
ret.write_value_to(&mut store, values_vec.add(i));
ret.write_value_to(caller.store.0, values_vec.add(i));
}
}
@@ -1744,8 +1734,7 @@ macro_rules! impl_into_func {
if let Err(trap) = caller.store.0.entering_native_hook() {
return R::fallible_from_trap(trap);
}
let mut _store = caller.sub_caller().store.opaque();
$(let $args = $args::from_abi($args, &mut _store);)*
$(let $args = $args::from_abi($args, caller.store.0);)*
let r = func(
caller.sub_caller(),
$( $args, )*
@@ -1770,11 +1759,10 @@ macro_rules! impl_into_func {
// Because the wrapped function is not `unsafe`, we
// can't assume it returned a value that is
// compatible with this store.
let mut store = caller.store.opaque();
if !ret.compatible_with_store(&store) {
if !ret.compatible_with_store(caller.store.0) {
CallResult::Trap(cross_store_trap())
} else {
match ret.into_abi_for_ret(&mut store, retptr) {
match ret.into_abi_for_ret(caller.store.0, retptr) {
Ok(val) => CallResult::Ok(val),
Err(trap) => CallResult::Trap(trap.into()),
}
@@ -1938,19 +1926,19 @@ impl HostFunc {
///
/// Can only be inserted into stores with a matching `T` relative to when
/// this `HostFunc` was first created.
pub unsafe fn to_func(self: &Arc<Self>, store: &mut StoreOpaque<'_>) -> Func {
pub unsafe fn to_func(self: &Arc<Self>, store: &mut StoreOpaque) -> Func {
self.register_trampoline(store);
let me = self.clone();
Func(store.store_data_mut().insert(FuncData::SharedHost(me)))
}
/// Same as [`HostFunc::to_func`], different ownership.
unsafe fn into_func(self, store: &mut StoreOpaque<'_>) -> Func {
unsafe fn into_func(self, store: &mut StoreOpaque) -> Func {
self.register_trampoline(store);
Func(store.store_data_mut().insert(FuncData::Host(self)))
}
unsafe fn register_trampoline(&self, store: &mut StoreOpaque<'_>) {
unsafe fn register_trampoline(&self, store: &mut StoreOpaque) {
let idx = self.export.anyfunc.as_ref().type_index;
store.register_host_trampoline(idx, self.trampoline);
}

View File

@@ -118,7 +118,7 @@ where
// Validate that all runtime values flowing into this store indeed
// belong within this store, otherwise it would be unsafe for store
// values to cross each other.
let params = match params.into_abi(&mut store.as_context_mut().opaque()) {
let params = match params.into_abi(store.0) {
Some(abi) => abi,
None => {
return Err(Trap::new(
@@ -154,10 +154,7 @@ where
let (_, ret, _, returned) = captures;
debug_assert_eq!(result.is_ok(), returned);
result?;
Ok(Results::from_abi(
&mut store.as_context_mut().opaque(),
ret.assume_init(),
))
Ok(Results::from_abi(store.0, ret.assume_init()))
}
}

View File

@@ -121,12 +121,12 @@ impl Instance {
) -> Result<Instance, Error> {
// This unsafety comes from `Instantiator::new` where we must typecheck
// first, which we are sure to do here.
let mut store = store.as_context_mut();
let mut i = unsafe {
let mut cx = store.as_context_mut().opaque();
typecheck_externs(&mut cx, module, imports)?;
Instantiator::new(&mut cx, module, ImportSource::Externs(imports))?
typecheck_externs(store.0, module, imports)?;
Instantiator::new(store.0, module, ImportSource::Externs(imports))?
};
i.run(&mut store.as_context_mut())
i.run(&mut store)
}
/// Same as [`Instance::new`], except for usage in [asynchronous stores].
@@ -157,12 +157,12 @@ impl Instance {
T: Send,
{
// See `new` for unsafety comments
let mut store = store.as_context_mut();
let mut i = unsafe {
let mut cx = store.as_context_mut().opaque();
typecheck_externs(&mut cx, module, imports)?;
Instantiator::new(&mut cx, module, ImportSource::Externs(imports))?
typecheck_externs(store.0, module, imports)?;
Instantiator::new(store.0, module, ImportSource::Externs(imports))?
};
i.run_async(&mut store.as_context_mut()).await
i.run_async(&mut store).await
}
pub(crate) fn from_wasmtime(handle: InstanceData, store: &mut StoreOpaque) -> Instance {
@@ -211,12 +211,12 @@ impl Instance {
&'a self,
store: impl Into<StoreContextMut<'a, T>>,
) -> impl ExactSizeIterator<Item = Export<'a>> + 'a {
self._exports(store.into().opaque())
self._exports(store.into().0)
}
fn _exports<'a>(
&'a self,
mut store: StoreOpaque<'a>,
store: &'a mut StoreOpaque,
) -> impl ExactSizeIterator<Item = Export<'a>> + 'a {
// If this is an `Instantiated` instance then all the `exports` may not
// be filled in. Fill them all in now if that's the case.
@@ -224,18 +224,17 @@ impl Instance {
if exports.iter().any(|e| e.is_none()) {
let module = Arc::clone(store.instance(*id).module());
for name in module.exports.keys() {
self._get_export(&mut store, name);
self._get_export(store, name);
}
}
}
let inner = store.into_inner();
return match &inner.store_data()[self.0] {
return match &store.store_data()[self.0] {
InstanceData::Synthetic(names) => {
Either::A(names.iter().map(|(k, v)| Export::new(k, v.clone())))
}
InstanceData::Instantiated { exports, id, .. } => {
let module = inner.instance(*id).module();
let module = store.instance(*id).module();
Either::B(
module
.exports
@@ -305,10 +304,10 @@ impl Instance {
/// instantiating a module faster, but also means this method requires a
/// mutable context.
pub fn get_export(&self, mut store: impl AsContextMut, name: &str) -> Option<Extern> {
self._get_export(&mut store.as_context_mut().opaque(), name)
self._get_export(store.as_context_mut().0, name)
}
fn _get_export(&self, store: &mut StoreOpaque<'_>, name: &str) -> Option<Extern> {
fn _get_export(&self, store: &mut StoreOpaque, name: &str) -> Option<Extern> {
match &store[self.0] {
// Synthetic instances always have their entire list of exports
// already specified.
@@ -458,7 +457,7 @@ impl<'a> Instantiator<'a> {
///
/// * The `imports` must all come from the `store` specified.
unsafe fn new(
store: &mut StoreOpaque<'_>,
store: &StoreOpaque,
module: &Module,
imports: ImportSource<'a>,
) -> Result<Instantiator<'a>> {
@@ -481,9 +480,7 @@ impl<'a> Instantiator<'a> {
// NB: this is the same code as `run_async`. It's intentionally
// small but should be kept in sync (modulo the async bits).
loop {
if let Some((instance, start, toplevel)) =
self.step(&mut store.as_context_mut().opaque())?
{
if let Some((instance, start, toplevel)) = self.step(store.0)? {
if let Some(start) = start {
Instantiator::start_raw(store, instance, start)?;
}
@@ -507,7 +504,7 @@ impl<'a> Instantiator<'a> {
// NB: this is the same code as `run`. It's intentionally
// small but should be kept in sync (modulo the async bits).
loop {
let step = self.step(&mut store.as_context_mut().opaque())?;
let step = self.step(store.0)?;
if let Some((instance, start, toplevel)) = step {
if let Some(start) = start {
store
@@ -543,7 +540,7 @@ impl<'a> Instantiator<'a> {
/// defined here.
fn step(
&mut self,
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
) -> Result<Option<(Instance, Option<FuncIndex>, bool)>> {
if self.cur.initializer == 0 {
store.bump_resource_counts(&self.cur.module)?;
@@ -707,7 +704,7 @@ impl<'a> Instantiator<'a> {
fn instantiate_raw(
&mut self,
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
) -> Result<(Instance, Option<FuncIndex>)> {
let compiled_module = self.cur.module.compiled_module();
@@ -737,7 +734,7 @@ impl<'a> Instantiator<'a> {
imports: self.cur.build(),
shared_signatures: self.cur.module.signatures().as_module_map().into(),
host_state: Box::new(Instance(instance_to_be)),
store: Some(store.traitobj),
store: Some(store.traitobj()),
wasm_data: compiled_module.wasm_data(),
})?;
@@ -877,7 +874,7 @@ impl<'a> ImportsBuilder<'a> {
}
}
fn push(&mut self, item: Extern, store: &mut StoreOpaque<'_>) {
fn push(&mut self, item: Extern, store: &mut StoreOpaque) {
match item {
Extern::Func(i) => {
self.functions.push(i.vmimport(store));
@@ -962,16 +959,16 @@ impl<T> InstancePre<T> {
// structure and then othrewise the `T` of `InstancePre<T>` connects any
// host functions we have in our definition list to the `store` that was
// passed in.
let mut store = store.as_context_mut();
let mut instantiator = unsafe {
let mut store = store.as_context_mut().opaque();
self.ensure_comes_from_same_store(&store)?;
self.ensure_comes_from_same_store(&store.0)?;
Instantiator::new(
&mut store,
store.0,
&self.module,
ImportSource::Definitions(&self.items),
)?
};
instantiator.run(&mut store.as_context_mut())
instantiator.run(&mut store)
}
/// Creates a new instance, running the start function asynchronously
@@ -994,11 +991,11 @@ impl<T> InstancePre<T> {
T: Send,
{
// For the unsafety here see above
let mut store = store.as_context_mut();
let mut i = unsafe {
let mut store = store.as_context_mut().opaque();
self.ensure_comes_from_same_store(&store)?;
self.ensure_comes_from_same_store(&store.0)?;
Instantiator::new(
&mut store,
store.0,
&self.module,
ImportSource::Definitions(&self.items),
)?
@@ -1006,7 +1003,7 @@ impl<T> InstancePre<T> {
i.run_async(&mut store.as_context_mut()).await
}
fn ensure_comes_from_same_store(&self, store: &StoreOpaque<'_>) -> Result<()> {
fn ensure_comes_from_same_store(&self, store: &StoreOpaque) -> Result<()> {
for import in self.items.iter() {
if !import.comes_from_same_store(store) {
bail!("cross-`Store` instantiation is not currently supported");

View File

@@ -137,7 +137,7 @@ macro_rules! generate_wrap_async_func {
),
);
self.func_wrap(module, name, move |mut caller: Caller<'_, T>, $($args: $args),*| {
let async_cx = caller.store.as_context_mut().opaque().async_cx();
let async_cx = caller.store.as_context_mut().0.async_cx();
let mut future = Pin::from(func(caller, $($args),*));
match unsafe { async_cx.block_on(future.as_mut()) } {
Ok(ret) => ret.into_fallible(),
@@ -331,7 +331,7 @@ impl<T> Linker<T> {
"cannot use `func_new_async` without enabling async support in the config"
);
self.func_new(module, name, ty, move |mut caller, params, results| {
let async_cx = caller.store.as_context_mut().opaque().async_cx();
let async_cx = caller.store.as_context_mut().0.async_cx();
let mut future = Pin::from(func(caller, params, results));
match unsafe { async_cx.block_on(future.as_mut()) } {
Ok(Ok(())) => Ok(()),
@@ -999,6 +999,7 @@ impl<T> Linker<T> {
mut store: impl AsContextMut<Data = T>,
module: &Module,
) -> Result<InstancePre<T>> {
let store = store.as_context_mut().0;
let imports = module
.imports()
.map(|import| {
@@ -1006,7 +1007,7 @@ impl<T> Linker<T> {
.ok_or_else(|| self.link_error(&import))
})
.collect::<Result<_>>()?;
unsafe { InstancePre::new(&mut store.as_context_mut().opaque(), module, imports) }
unsafe { InstancePre::new(store, module, imports) }
}
fn link_error(&self, import: &ImportType) -> Error {
@@ -1031,12 +1032,12 @@ impl<T> Linker<T> {
mut store: impl AsContextMut<Data = T> + 'p,
) -> impl Iterator<Item = (&str, &str, Extern)> + 'p {
self.map.iter().map(move |(key, item)| {
let mut store = store.as_context_mut().opaque();
let store = store.as_context_mut();
(
&*self.strings[key.module],
&*self.strings[key.name],
// Should be safe since `T` is connecting the linker and store
unsafe { item.to_extern(&mut store) },
unsafe { item.to_extern(store.0) },
)
})
}
@@ -1052,9 +1053,9 @@ impl<T> Linker<T> {
module: &str,
name: Option<&str>,
) -> Option<Extern> {
let mut store = store.as_context_mut().opaque();
let store = store.as_context_mut().0;
// Should be safe since `T` is connecting the linker and store
Some(unsafe { self._get(module, name)?.to_extern(&mut store) })
Some(unsafe { self._get(module, name)?.to_extern(store) })
}
fn _get(&self, module: &str, name: Option<&str>) -> Option<&Definition> {
@@ -1077,9 +1078,9 @@ impl<T> Linker<T> {
mut store: impl AsContextMut<Data = T>,
import: &ImportType,
) -> Option<Extern> {
let mut store = store.as_context_mut().opaque();
let store = store.as_context_mut().0;
// Should be safe since `T` is connecting the linker and store
Some(unsafe { self._get_by_import(import)?.to_extern(&mut store) })
Some(unsafe { self._get_by_import(import)?.to_extern(store) })
}
fn _get_by_import(&self, import: &ImportType) -> Option<Definition> {

View File

@@ -220,10 +220,10 @@ impl Memory {
/// # }
/// ```
pub fn new(mut store: impl AsContextMut, ty: MemoryType) -> Result<Memory> {
Memory::_new(&mut store.as_context_mut().opaque(), ty)
Memory::_new(store.as_context_mut().0, ty)
}
fn _new(store: &mut StoreOpaque<'_>, ty: MemoryType) -> Result<Memory> {
fn _new(store: &mut StoreOpaque, ty: MemoryType) -> Result<Memory> {
unsafe {
let export = generate_memory_export(store, &ty)?;
Ok(Memory::from_wasmtime_memory(export, store))
@@ -450,10 +450,10 @@ impl Memory {
/// # }
/// ```
pub fn grow(&self, mut store: impl AsContextMut, delta: u64) -> Result<u64> {
let mem = self.wasmtime_memory(&mut store.as_context_mut().opaque());
let store = store.as_context_mut();
let store = store.as_context_mut().0;
let mem = self.wasmtime_memory(store);
unsafe {
match (*mem).grow(delta, store.0.limiter()) {
match (*mem).grow(delta, store.limiter()) {
Some(size) => {
let vm = (*mem).vmmemory();
*store[self.0].definition = vm;
@@ -464,7 +464,7 @@ impl Memory {
}
}
fn wasmtime_memory(&self, store: &mut StoreOpaque<'_>) -> *mut wasmtime_runtime::Memory {
fn wasmtime_memory(&self, store: &mut StoreOpaque) -> *mut wasmtime_runtime::Memory {
unsafe {
let export = &store[self.0];
let mut handle = wasmtime_runtime::InstanceHandle::from_vmctx(export.vmctx);
@@ -484,7 +484,7 @@ impl Memory {
&store[self.0].memory.memory
}
pub(crate) fn vmimport(&self, store: &StoreOpaque<'_>) -> wasmtime_runtime::VMMemoryImport {
pub(crate) fn vmimport(&self, store: &StoreOpaque) -> wasmtime_runtime::VMMemoryImport {
let export = &store[self.0];
wasmtime_runtime::VMMemoryImport {
from: export.definition,

View File

@@ -1,3 +1,81 @@
//! Wasmtime's "store" type
//!
//! This module, and its submodules, contain the `Store` type and various types
//! used to interact with it. At first glance this is a pretty confusing module
//! where you need to know the difference between:
//!
//! * `Store<T>`
//! * `StoreContext<T>`
//! * `StoreContextMut<T>`
//! * `AsContext`
//! * `AsContextMut`
//! * `StoreInner<T>`
//! * `StoreOpaque`
//! * `StoreData`
//!
//! There's... quite a lot going on here, and it's easy to be confused. This
//! comment is ideally going to serve the purpose of clarifying what all these
//! types are for and why they're motivated.
//!
//! First it's important to know what's "internal" and what's "external". Almost
//! everything above is defined as `pub`, but only some of the items are
//! reexported to the outside world to be usable from this crate. Otherwise all
//! items are `pub` within this `store` module, and the `store` module is
//! private to the `wasmtime` crate. Notably `Store<T>`, `StoreContext<T>`,
//! `StoreContextMut<T>`, `AsContext`, and `AsContextMut` are all public
//! interfaces to the `wasmtime` crate. You can think of these as:
//!
//! * `Store<T>` - an owned reference to a store, the "root of everything"
//! * `StoreContext<T>` - basically `&StoreInner<T>`
//! * `StoreContextMut<T>` - more-or-less `&mut StoreInner<T>` with caveats.
//! Explained later.
//! * `AsContext` - similar to `AsRef`, but produces `StoreContext<T>`
//! * `AsContextMut` - similar to `AsMut`, but produces `StoreContextMut<T>`
//!
//! Next comes the internal structure of the `Store<T>` itself. This looks like:
//!
//! * `Store<T>` - this type is just a pointer large. It's primarily just
//! intended to be consumed by the outside world. Note that the "just a
//! pointer large" is a load-bearing implementation detail in Wasmtime. This
//! enables it to store a pointer to its own trait object which doesn't need
//! to change over time.
//!
//! * `StoreInner<T>` - the first layer of the contents of a `Store<T>`, what's
//! stored inside the `Box`. This is the general Rust pattern when one struct
//! is a layer over another. The surprising part, though, is that this is
//! further subdivided. This structure only contains things which actually
//! need `T` itself. The downside of this structure is that it's always
//! generic and means that code is monomorphized into consumer crates. We
//! strive to have things be as monomorphic as possible in `wasmtime` so this
//! type is not heavily used.
//!
//! * `StoreOpaque` - this is the primary contents of the `StoreInner<T>` type.
//! Stored inline in the outer type the "opaque" here means that it's a
//! "store" but it doesn't have access to the `T`. This is the primary
//! "internal" reference that Wasmtime uses since `T` is rarely needed by the
//! internals of Wasmtime.
//!
//! * `StoreData` - this is a final helper struct stored within `StoreOpaque`.
//! All references of Wasm items into a `Store` are actually indices into a
//! table in this structure, and the `StoreData` being separate makes it a bit
//! easier to manage/define/work with. There's no real fundamental reason this
//! is split out, although sometimes it's useful to have separate borrows into
//! these tables than the `StoreOpaque`.
//!
//! A major caveat with these representations is that the internal `&mut
//! StoreInner<T>` is never handed out publicly to consumers of this crate, only
//! through a wrapper of `StoreContextMut<'_, T>`. The reason for this is that
//! we want to provide mutable, but not destructive, access to the contents of a
//! `Store`. For example if a `StoreInner<T>` were replaced with some other
//! `StoreInner<T>` then that would drop live instances, possibly those
//! currently executing beneath the current stack frame. This would not be a
//! safe operation.
//!
//! This means, though, that the `wasmtime` crate, which liberally uses `&mut
//! StoreOpaque` internally, has to be careful to never actually destroy the
//! contents of `StoreOpaque`. This is an invariant that we, as the authors of
//! `wasmtime`, must uphold for the public interface to be safe.
use crate::{module::ModuleRegistry, Engine, Module, Trap};
use anyhow::{bail, Result};
use std::cell::UnsafeCell;
@@ -79,14 +157,50 @@ pub struct Store<T> {
inner: ManuallyDrop<Box<StoreInner<T>>>,
}
/// Internal contents of a `Store<T>` that live on the heap.
///
/// The members of this struct are those that need to be generic over `T`, the
/// store's internal type storage. Otherwise all things that don't rely on `T`
/// should go into `StoreOpaque`.
pub struct StoreInner<T> {
// This `StoreInner<T>` structure has references to itself. These aren't
/// Generic metadata about the store that doesn't need access to `T`.
inner: StoreOpaque,
limiter: Option<Box<dyn FnMut(&mut T) -> &mut (dyn crate::ResourceLimiter) + Send + Sync>>,
entering_native_hook: Option<Box<dyn FnMut(&mut T) -> Result<(), crate::Trap> + Send + Sync>>,
exiting_native_hook: Option<Box<dyn FnMut(&mut T) -> Result<(), crate::Trap> + Send + Sync>>,
// for comments about `ManuallyDrop`, see `Store::into_data`
data: ManuallyDrop<T>,
}
// Forward methods on `StoreOpaque` to also being on `StoreInner<T>`
impl<T> Deref for StoreInner<T> {
type Target = StoreOpaque;
fn deref(&self) -> &Self::Target {
&self.inner
}
}
impl<T> DerefMut for StoreInner<T> {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.inner
}
}
/// Monomorphic storage for a `Store<T>`.
///
/// This structure contains the bulk of the metadata about a `Store`. This is
/// used internally in Wasmtime when dependence on the `T` of `Store<T>` isn't
/// necessary, allowing code to be monomorphic and compiled into the `wasmtime`
/// crate itself.
pub struct StoreOpaque {
// This `StoreOpaque` structure has references to itself. These aren't
// immediately evident, however, so we need to tell the compiler that it
// contains self-references. This notably suppresses `noalias` annotations
// when this shows up in compiled code because types of this structure do
// indeed alias itself. The best example of this is `StoreOpaque` which
// contains a `&mut StoreInner` and a `*mut dyn Store` which are actually
// the same pointer, indeed aliasing!
// indeed alias itself. An example of this is `default_callee` holds a
// `*mut dyn Store` to the address of this `StoreOpaque` itself, indeed
// aliasing!
//
// It's somewhat unclear to me at this time if this is 100% sufficient to
// get all the right codegen in all the right places. For example does
@@ -102,30 +216,7 @@ pub struct StoreInner<T> {
// least telling the compiler something about all the aliasing happening
// within a `Store`.
_marker: marker::PhantomPinned,
inner: StoreInnermost,
limiter: Option<Box<dyn FnMut(&mut T) -> &mut (dyn crate::ResourceLimiter) + Send + Sync>>,
entering_native_hook: Option<Box<dyn FnMut(&mut T) -> Result<(), crate::Trap> + Send + Sync>>,
exiting_native_hook: Option<Box<dyn FnMut(&mut T) -> Result<(), crate::Trap> + Send + Sync>>,
// for comments about `ManuallyDrop`, see `Store::into_data`
data: ManuallyDrop<T>,
}
impl<T> Deref for StoreInner<T> {
type Target = StoreInnermost;
fn deref(&self) -> &Self::Target {
&self.inner
}
}
impl<T> DerefMut for StoreInner<T> {
fn deref_mut(&mut self) -> &mut Self::Target {
&mut self.inner
}
}
// I apologize for the convoluted structure and the terrible naming of this struct.
// This exists so that most of wasmtime can be monomorphic on StoreInnermost, without
// having to care about the generic in StoreInner<T>.
pub struct StoreInnermost {
engine: Engine,
interrupts: Arc<VMInterrupts>,
instances: Vec<StoreInstance>,
@@ -217,8 +308,8 @@ impl<T> Store<T> {
.expect("failed to allocate default callee")
};
let mut inner = Box::new(StoreInner {
_marker: marker::PhantomPinned,
inner: StoreInnermost {
inner: StoreOpaque {
_marker: marker::PhantomPinned,
engine: engine.clone(),
interrupts: Default::default(),
instances: Vec::new(),
@@ -249,11 +340,18 @@ impl<T> Store<T> {
});
// Once we've actually allocated the store itself we can configure the
// trait object pointer of the default callee.
let store = StoreContextMut(&mut *inner).opaque().traitobj;
// trait object pointer of the default callee. Note the erasure of the
// lifetime here into `'static`, so in general usage of this trait
// object must be strictly bounded to the `Store` itself, and is a
// variant that we have to maintain throughout Wasmtime.
unsafe {
inner.default_callee.set_store(store);
let traitobj = std::mem::transmute::<
*mut (dyn wasmtime_runtime::Store + '_),
*mut (dyn wasmtime_runtime::Store + 'static),
>(&mut *inner);
inner.default_callee.set_store(traitobj);
}
Self {
inner: ManuallyDrop::new(inner),
}
@@ -696,7 +794,7 @@ impl<T> StoreInner<T> {
}
}
impl StoreInnermost {
impl StoreOpaque {
pub fn bump_resource_counts(&mut self, module: &Module) -> Result<()> {
fn bump(slot: &mut usize, max: usize, amt: usize, desc: &str) -> Result<()> {
let new = slot.saturating_add(amt);
@@ -954,6 +1052,10 @@ impl StoreInnermost {
pub fn default_callee(&self) -> *mut VMContext {
self.default_callee.vmctx_ptr()
}
pub fn traitobj(&self) -> *mut dyn wasmtime_runtime::Store {
self.default_callee.store()
}
}
impl<T> StoreContextMut<'_, T> {
@@ -1243,7 +1345,7 @@ impl AsyncCx {
unsafe impl<T> wasmtime_runtime::Store for StoreInner<T> {
fn vminterrupts(&self) -> *mut VMInterrupts {
<StoreInnermost>::vminterrupts(self)
<StoreOpaque>::vminterrupts(self)
}
fn externref_activations_table(
@@ -1273,7 +1375,7 @@ unsafe impl<T> wasmtime_runtime::Store for StoreInner<T> {
}
*injection_count -= 1;
let fuel = *fuel_to_inject;
StoreContextMut(self).opaque().out_of_gas_yield(fuel)?;
self.out_of_gas_yield(fuel)?;
Ok(())
}
#[cfg(not(feature = "async"))]
@@ -1319,7 +1421,7 @@ impl<T> Drop for Store<T> {
}
}
impl Drop for StoreInnermost {
impl Drop for StoreOpaque {
fn drop(&mut self) {
// NB it's important that this destructor does not access `self.data`.
// That is deallocated by `Drop for Store<T>` above.

View File

@@ -1,5 +1,4 @@
use crate::store::{Store, StoreInner, StoreInnermost};
use std::ops::{Deref, DerefMut};
use crate::store::{Store, StoreInner};
/// A temporary handle to a [`&Store<T>`][`Store`].
///
@@ -37,27 +36,6 @@ impl<'a, T> StoreContextMut<'a, T> {
) -> StoreContextMut<'a, T> {
StoreContextMut(&mut *(store as *mut StoreInner<T>))
}
/// A helper method to erase the `T` on `Self` so the returned type has no
/// generics. For some more information see [`StoreOpaque`] itself.
///
/// The primary purpose of this is to help improve compile times where
/// non-generic code can be compiled into libwasmtime.rlib.
pub(crate) fn opaque(mut self) -> StoreOpaque<'a> {
StoreOpaque {
traitobj: self.traitobj(),
inner: self.0,
}
}
fn traitobj(&mut self) -> *mut dyn wasmtime_runtime::Store {
unsafe {
std::mem::transmute::<
*mut (dyn wasmtime_runtime::Store + '_),
*mut (dyn wasmtime_runtime::Store + 'static),
>(self.0)
}
}
}
/// A trait used to get shared access to a [`Store`] in Wasmtime.
@@ -183,6 +161,33 @@ impl<T> AsContextMut for StoreContextMut<'_, T> {
}
}
// Implementations for internal consumers, but these aren't public types so
// they're not publicly accessible for crate consumers.
impl<T> AsContext for &'_ StoreInner<T> {
type Data = T;
#[inline]
fn as_context(&self) -> StoreContext<'_, T> {
StoreContext(self)
}
}
impl<T> AsContext for &'_ mut StoreInner<T> {
type Data = T;
#[inline]
fn as_context(&self) -> StoreContext<'_, T> {
StoreContext(self)
}
}
impl<T> AsContextMut for &'_ mut StoreInner<T> {
#[inline]
fn as_context_mut(&mut self) -> StoreContextMut<'_, T> {
StoreContextMut(&mut **self)
}
}
// forward AsContext for &T
impl<T: AsContext> AsContext for &'_ T {
type Data = T::Data;
@@ -229,46 +234,3 @@ impl<'a, T: AsContextMut> From<&'a mut T> for StoreContextMut<'a, T::Data> {
t.as_context_mut()
}
}
/// This structure is akin to a `StoreContextMut` except that the `T` is
/// "erased" to an opaque type.
///
/// This structure is used pervasively through wasmtime whenever the `T` isn't
/// needed (quite common!). This allows the compiler to erase generics and
/// compile more code in the wasmtime crate itself instead of monomorphizing
/// everything into consumer crates. The primary purpose of this is to help
/// compile times.
#[doc(hidden)] // this is part of `WasmTy`, but a hidden part, so hide this
pub struct StoreOpaque<'a> {
/// The actual pointer to the `StoreInner` internals.
inner: &'a mut StoreInnermost,
/// A raw trait object that can be used to invoke functions with. Note that
/// this is a pointer which aliases with `inner` above, so extreme care
/// needs to be used when using this (the above `inner` cannot be actively
/// borrowed).
pub traitobj: *mut dyn wasmtime_runtime::Store,
}
impl<'a> StoreOpaque<'a> {
pub fn into_inner(self) -> &'a StoreInnermost {
self.inner
}
}
// Deref impls to forward all methods on `StoreOpaque` to `StoreInner`.
impl<'a> Deref for StoreOpaque<'a> {
type Target = StoreInnermost;
#[inline]
fn deref(&self) -> &Self::Target {
&*self.inner
}
}
impl<'a> DerefMut for StoreOpaque<'a> {
#[inline]
fn deref_mut(&mut self) -> &mut Self::Target {
&mut *self.inner
}
}

View File

@@ -158,7 +158,7 @@ where
}
// forward StoreOpaque => StoreData
impl<I> Index<I> for StoreOpaque<'_>
impl<I> Index<I> for StoreOpaque
where
StoreData: Index<I>,
{
@@ -169,7 +169,7 @@ where
self.store_data().index(index)
}
}
impl<I> IndexMut<I> for StoreOpaque<'_>
impl<I> IndexMut<I> for StoreOpaque
where
StoreData: IndexMut<I>,
{

View File

@@ -24,7 +24,7 @@ use wasmtime_runtime::{
fn create_handle(
module: Module,
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
host_state: Box<dyn Any + Send + Sync>,
func_imports: &[VMFunctionImport],
shared_signature_id: Option<VMSharedSignatureIndex>,
@@ -46,7 +46,7 @@ fn create_handle(
imports,
shared_signatures: shared_signature_id.into(),
host_state,
store: Some(store.traitobj),
store: Some(store.traitobj()),
wasm_data: &[],
},
)?;
@@ -56,7 +56,7 @@ fn create_handle(
}
pub fn generate_global_export(
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
gt: &GlobalType,
val: Val,
) -> Result<wasmtime_runtime::ExportGlobal> {
@@ -69,7 +69,7 @@ pub fn generate_global_export(
}
pub fn generate_memory_export(
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
m: &MemoryType,
) -> Result<wasmtime_runtime::ExportMemory> {
let instance = create_memory(store, m)?;
@@ -81,7 +81,7 @@ pub fn generate_memory_export(
}
pub fn generate_table_export(
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
t: &TableType,
) -> Result<wasmtime_runtime::ExportTable> {
let instance = create_table(store, t)?;

View File

@@ -5,7 +5,7 @@ use anyhow::Result;
use wasmtime_environ::{EntityIndex, Global, GlobalInit, Module, ModuleType, SignatureIndex};
use wasmtime_runtime::VMFunctionImport;
pub fn create_global(store: &mut StoreOpaque<'_>, gt: &GlobalType, val: Val) -> Result<InstanceId> {
pub fn create_global(store: &mut StoreOpaque, gt: &GlobalType, val: Val) -> Result<InstanceId> {
let mut module = Module::new();
let mut func_imports = Vec::new();
let mut externref_init = None;

View File

@@ -8,7 +8,7 @@ use std::sync::Arc;
use wasmtime_environ::{EntityIndex, MemoryPlan, MemoryStyle, Module, WASM_PAGE_SIZE};
use wasmtime_runtime::{RuntimeLinearMemory, RuntimeMemoryCreator, VMMemoryDefinition};
pub fn create_memory(store: &mut StoreOpaque<'_>, memory: &MemoryType) -> Result<InstanceId> {
pub fn create_memory(store: &mut StoreOpaque, memory: &MemoryType) -> Result<InstanceId> {
let mut module = Module::new();
let memory_plan = wasmtime_environ::MemoryPlan::for_memory(

View File

@@ -4,7 +4,7 @@ use crate::TableType;
use anyhow::Result;
use wasmtime_environ::{EntityIndex, Module};
pub fn create_table(store: &mut StoreOpaque<'_>, table: &TableType) -> Result<InstanceId> {
pub fn create_table(store: &mut StoreOpaque, table: &TableType) -> Result<InstanceId> {
let mut module = Module::new();
let table_plan = wasmtime_environ::TablePlan::for_table(
table.wasmtime_table().clone(),

View File

@@ -1,6 +1,6 @@
use crate::instance::InstanceData;
use crate::linker::Definition;
use crate::store::StoreInnermost;
use crate::store::StoreOpaque;
use crate::{signatures::SignatureCollection, Engine, Extern};
use anyhow::{bail, Context, Result};
use wasmtime_environ::{
@@ -12,7 +12,7 @@ use wasmtime_runtime::VMSharedSignatureIndex;
pub struct MatchCx<'a> {
pub signatures: &'a SignatureCollection,
pub types: &'a TypeTables,
pub store: &'a StoreInnermost,
pub store: &'a StoreOpaque,
pub engine: &'a Engine,
}

View File

@@ -33,7 +33,7 @@ impl<T> StoreExt for Store<T> {
+ Sync,
{
self.as_context_mut()
.opaque()
.0
.set_signal_handler(Some(Box::new(handler)));
}
}

View File

@@ -187,7 +187,7 @@ impl Val {
pub(crate) fn into_table_element(
self,
store: &mut StoreOpaque<'_>,
store: &mut StoreOpaque,
ty: ValType,
) -> Result<runtime::TableElement> {
match (self, ty) {
@@ -213,7 +213,7 @@ impl Val {
}
#[inline]
pub(crate) fn comes_from_same_store(&self, store: &StoreOpaque<'_>) -> bool {
pub(crate) fn comes_from_same_store(&self, store: &StoreOpaque) -> bool {
match self {
Val::FuncRef(Some(f)) => f.comes_from_same_store(store),
Val::FuncRef(None) => true,

View File

@@ -27,7 +27,7 @@ impl<T> StoreExt for Store<T> {
H: 'static + Fn(winapi::um::winnt::PEXCEPTION_POINTERS) -> bool + Send + Sync,
{
self.as_context_mut()
.opaque()
.0
.set_signal_handler(Some(Box::new(handler)));
}
}