diff --git a/crates/wasmtime/src/runtime/externals/table.rs b/crates/wasmtime/src/runtime/externals/table.rs
index 789560c8f214..1ec560768a6e 100644
--- a/crates/wasmtime/src/runtime/externals/table.rs
+++ b/crates/wasmtime/src/runtime/externals/table.rs
@@ -3,7 +3,7 @@ use crate::runtime::RootedGcRefImpl;
use crate::runtime::vm::{
self, GcStore, SendSyncPtr, TableElementType, VMFuncRef, VMGcRef, VMStore,
};
-use crate::store::{AutoAssertNoGc, StoreInstanceId, StoreOpaque};
+use crate::store::{AutoAssertNoGc, StoreInstanceId, StoreOpaque, StoreResourceLimiter};
use crate::trampoline::generate_table_export;
use crate::{
AnyRef, AsContext, AsContextMut, ExnRef, ExternRef, Func, HeapType, Ref, RefType,
@@ -94,7 +94,8 @@ impl Table {
/// # }
/// ```
pub fn new(mut store: impl AsContextMut, ty: TableType, init: Ref) -> Result
{
- vm::one_poll(Table::_new(store.as_context_mut().0, ty, init))
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ vm::one_poll(Table::_new(store, limiter.as_mut(), ty, init))
.expect("must use `new_async` when async resource limiters are in use")
}
@@ -112,11 +113,17 @@ impl Table {
ty: TableType,
init: Ref,
) -> Result {
- Table::_new(store.as_context_mut().0, ty, init).await
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ Table::_new(store, limiter.as_mut(), ty, init).await
}
- async fn _new(store: &mut StoreOpaque, ty: TableType, init: Ref) -> Result {
- let table = generate_table_export(store, &ty).await?;
+ async fn _new(
+ store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
+ ty: TableType,
+ init: Ref,
+ ) -> Result {
+ let table = generate_table_export(store, limiter, &ty).await?;
table._fill(store, 0, init, ty.minimum())?;
Ok(table)
}
diff --git a/crates/wasmtime/src/runtime/gc/enabled/arrayref.rs b/crates/wasmtime/src/runtime/gc/enabled/arrayref.rs
index c229e2583c72..f8d3a0ee7a94 100644
--- a/crates/wasmtime/src/runtime/gc/enabled/arrayref.rs
+++ b/crates/wasmtime/src/runtime/gc/enabled/arrayref.rs
@@ -1,7 +1,7 @@
//! Working with GC `array` objects.
-use crate::runtime::vm::VMGcRef;
-use crate::store::StoreId;
+use crate::runtime::vm::{VMGcRef, VMStore};
+use crate::store::{StoreId, StoreResourceLimiter};
use crate::vm::{self, VMArrayRef, VMGcHeader};
use crate::{AnyRef, FieldType};
use crate::{
@@ -297,9 +297,15 @@ impl ArrayRef {
elem: &Val,
len: u32,
) -> Result> {
- let store = store.as_context_mut().0;
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
assert!(!store.async_support());
- vm::assert_ready(Self::_new_async(store, allocator, elem, len))
+ vm::assert_ready(Self::_new_async(
+ store,
+ limiter.as_mut(),
+ allocator,
+ elem,
+ len,
+ ))
}
/// Asynchronously allocate a new `array` of the given length, with every
@@ -341,17 +347,19 @@ impl ArrayRef {
elem: &Val,
len: u32,
) -> Result> {
- Self::_new_async(store.as_context_mut().0, allocator, elem, len).await
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ Self::_new_async(store, limiter.as_mut(), allocator, elem, len).await
}
pub(crate) async fn _new_async(
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
allocator: &ArrayRefPre,
elem: &Val,
len: u32,
) -> Result> {
store
- .retry_after_gc_async((), |store, ()| {
+ .retry_after_gc_async(limiter, (), |store, ()| {
Self::new_from_iter(store, allocator, RepeatN(elem, len))
})
.await
@@ -445,9 +453,14 @@ impl ArrayRef {
allocator: &ArrayRefPre,
elems: &[Val],
) -> Result> {
- let store = store.as_context_mut().0;
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
assert!(!store.async_support());
- vm::assert_ready(Self::_new_fixed_async(store, allocator, elems))
+ vm::assert_ready(Self::_new_fixed_async(
+ store,
+ limiter.as_mut(),
+ allocator,
+ elems,
+ ))
}
/// Asynchronously allocate a new `array` containing the given elements.
@@ -491,16 +504,18 @@ impl ArrayRef {
allocator: &ArrayRefPre,
elems: &[Val],
) -> Result> {
- Self::_new_fixed_async(store.as_context_mut().0, allocator, elems).await
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ Self::_new_fixed_async(store, limiter.as_mut(), allocator, elems).await
}
pub(crate) async fn _new_fixed_async(
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
allocator: &ArrayRefPre,
elems: &[Val],
) -> Result> {
store
- .retry_after_gc_async((), |store, ()| {
+ .retry_after_gc_async(limiter, (), |store, ()| {
Self::new_from_iter(store, allocator, elems.iter())
})
.await
diff --git a/crates/wasmtime/src/runtime/gc/enabled/exnref.rs b/crates/wasmtime/src/runtime/gc/enabled/exnref.rs
index fe2afc998c94..03ba35ad3bc0 100644
--- a/crates/wasmtime/src/runtime/gc/enabled/exnref.rs
+++ b/crates/wasmtime/src/runtime/gc/enabled/exnref.rs
@@ -1,7 +1,7 @@
//! Implementation of `exnref` in Wasmtime.
-use crate::runtime::vm::VMGcRef;
-use crate::store::StoreId;
+use crate::runtime::vm::{VMGcRef, VMStore};
+use crate::store::{StoreId, StoreResourceLimiter};
use crate::vm::{self, VMExnRef, VMGcHeader};
use crate::{
AsContext, AsContextMut, GcRefImpl, GcRootIndex, HeapType, ManuallyRooted, RefType, Result,
@@ -205,9 +205,15 @@ impl ExnRef {
tag: &Tag,
fields: &[Val],
) -> Result> {
- let store = store.as_context_mut().0;
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
assert!(!store.async_support());
- vm::assert_ready(Self::_new_async(store, allocator, tag, fields))
+ vm::assert_ready(Self::_new_async(
+ store,
+ limiter.as_mut(),
+ allocator,
+ tag,
+ fields,
+ ))
}
/// Asynchronously allocate a new exception object and get a
@@ -245,18 +251,20 @@ impl ExnRef {
tag: &Tag,
fields: &[Val],
) -> Result> {
- Self::_new_async(store.as_context_mut().0, allocator, tag, fields).await
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ Self::_new_async(store, limiter.as_mut(), allocator, tag, fields).await
}
pub(crate) async fn _new_async(
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
allocator: &ExnRefPre,
tag: &Tag,
fields: &[Val],
) -> Result> {
Self::type_check_tag_and_fields(store, allocator, tag, fields)?;
store
- .retry_after_gc_async((), |store, ()| {
+ .retry_after_gc_async(limiter, (), |store, ()| {
Self::new_unchecked(store, allocator, tag, fields)
})
.await
diff --git a/crates/wasmtime/src/runtime/gc/enabled/externref.rs b/crates/wasmtime/src/runtime/gc/enabled/externref.rs
index 2c98a773b01c..8564843b84da 100644
--- a/crates/wasmtime/src/runtime/gc/enabled/externref.rs
+++ b/crates/wasmtime/src/runtime/gc/enabled/externref.rs
@@ -2,11 +2,11 @@
use super::{AnyRef, RootedGcRefImpl};
use crate::prelude::*;
-use crate::runtime::vm::{self, VMGcRef};
+use crate::runtime::vm::{self, VMGcRef, VMStore};
use crate::{
AsContextMut, GcHeapOutOfMemory, GcRefImpl, GcRootIndex, HeapType, ManuallyRooted, RefType,
Result, Rooted, StoreContext, StoreContextMut, ValRaw, ValType, WasmTy,
- store::{AutoAssertNoGc, StoreOpaque},
+ store::{AutoAssertNoGc, StoreOpaque, StoreResourceLimiter},
};
use core::any::Any;
use core::mem;
@@ -210,13 +210,13 @@ impl ExternRef {
/// Panics if the `context` is configured for async; use
/// [`ExternRef::new_async`][crate::ExternRef::new_async] to perform
/// asynchronous allocation instead.
- pub fn new(mut context: impl AsContextMut, value: T) -> Result>
+ pub fn new(mut store: impl AsContextMut, value: T) -> Result>
where
T: 'static + Any + Send + Sync,
{
- let ctx = context.as_context_mut().0;
- assert!(!ctx.async_support());
- vm::assert_ready(Self::_new_async(ctx, value))
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ assert!(!store.async_support());
+ vm::assert_ready(Self::_new_async(store, limiter.as_mut(), value))
}
/// Asynchronously allocates a new `ExternRef` wrapping the given value.
@@ -295,16 +295,17 @@ impl ExternRef {
/// [`ExternRef::new`][crate::ExternRef::new] to perform synchronous
/// allocation instead.
#[cfg(feature = "async")]
- pub async fn new_async(mut context: impl AsContextMut, value: T) -> Result>
+ pub async fn new_async(mut store: impl AsContextMut, value: T) -> Result>
where
T: 'static + Any + Send + Sync,
{
- let ctx = context.as_context_mut().0;
- Self::_new_async(ctx, value).await
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ Self::_new_async(store, limiter.as_mut(), value).await
}
pub(crate) async fn _new_async(
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
value: T,
) -> Result>
where
@@ -315,7 +316,7 @@ impl ExternRef {
let value: Box = Box::new(value);
let gc_ref = store
- .retry_after_gc_async(value, |store, value| {
+ .retry_after_gc_async(limiter, value, |store, value| {
store
.require_gc_store_mut()?
.alloc_externref(value)
diff --git a/crates/wasmtime/src/runtime/gc/enabled/structref.rs b/crates/wasmtime/src/runtime/gc/enabled/structref.rs
index da8cee81f886..234cf0baf4ad 100644
--- a/crates/wasmtime/src/runtime/gc/enabled/structref.rs
+++ b/crates/wasmtime/src/runtime/gc/enabled/structref.rs
@@ -2,13 +2,13 @@
use crate::runtime::vm::VMGcRef;
use crate::store::StoreId;
-use crate::vm::{self, VMGcHeader, VMStructRef};
+use crate::vm::{self, VMGcHeader, VMStore, VMStructRef};
use crate::{AnyRef, FieldType};
use crate::{
AsContext, AsContextMut, EqRef, GcHeapOutOfMemory, GcRefImpl, GcRootIndex, HeapType,
ManuallyRooted, RefType, Rooted, StructType, Val, ValRaw, ValType, WasmTy,
prelude::*,
- store::{AutoAssertNoGc, StoreContextMut, StoreOpaque},
+ store::{AutoAssertNoGc, StoreContextMut, StoreOpaque, StoreResourceLimiter},
};
use core::mem::{self, MaybeUninit};
use wasmtime_environ::{GcLayout, GcStructLayout, VMGcKind, VMSharedTypeIndex};
@@ -231,9 +231,9 @@ impl StructRef {
allocator: &StructRefPre,
fields: &[Val],
) -> Result> {
- let store = store.as_context_mut().0;
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
assert!(!store.async_support());
- vm::assert_ready(Self::_new_async(store, allocator, fields))
+ vm::assert_ready(Self::_new_async(store, limiter.as_mut(), allocator, fields))
}
/// Asynchronously allocate a new `struct` and get a reference to it.
@@ -264,17 +264,19 @@ impl StructRef {
allocator: &StructRefPre,
fields: &[Val],
) -> Result> {
- Self::_new_async(store.as_context_mut().0, allocator, fields).await
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ Self::_new_async(store, limiter.as_mut(), allocator, fields).await
}
pub(crate) async fn _new_async(
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
allocator: &StructRefPre,
fields: &[Val],
) -> Result> {
Self::type_check_fields(store, allocator, fields)?;
store
- .retry_after_gc_async((), |store, ()| {
+ .retry_after_gc_async(limiter, (), |store, ()| {
Self::new_unchecked(store, allocator, fields)
})
.await
diff --git a/crates/wasmtime/src/runtime/instance.rs b/crates/wasmtime/src/runtime/instance.rs
index d92e4e2145c1..e13b54660ec7 100644
--- a/crates/wasmtime/src/runtime/instance.rs
+++ b/crates/wasmtime/src/runtime/instance.rs
@@ -2,9 +2,11 @@ use crate::linker::{Definition, DefinitionType};
use crate::prelude::*;
use crate::runtime::vm::{
self, Imports, ModuleRuntimeInfo, VMFuncRef, VMFunctionImport, VMGlobalImport, VMMemoryImport,
- VMTableImport, VMTagImport,
+ VMStore, VMTableImport, VMTagImport,
+};
+use crate::store::{
+ AllocateInstanceKind, InstanceId, StoreInstanceId, StoreOpaque, StoreResourceLimiter,
};
-use crate::store::{AllocateInstanceKind, InstanceId, StoreInstanceId, StoreOpaque};
use crate::types::matching;
use crate::{
AsContextMut, Engine, Export, Extern, Func, Global, Memory, Module, ModuleExport, SharedMemory,
@@ -248,9 +250,12 @@ impl Instance {
module: &Module,
imports: Imports<'_>,
) -> Result {
- // SAFETY: the safety contract of `new_raw` is the same as this
- // function.
- let (instance, start) = unsafe { Instance::new_raw(store.0, module, imports).await? };
+ let (instance, start) = {
+ let (mut limiter, store) = store.0.resource_limiter_and_store_opaque();
+ // SAFETY: the safety contract of `new_raw` is the same as this
+ // function.
+ unsafe { Instance::new_raw(store, limiter.as_mut(), module, imports).await? }
+ };
if let Some(start) = start {
if store.0.async_support() {
#[cfg(feature = "async")]
@@ -285,6 +290,7 @@ impl Instance {
/// provided as well.
async unsafe fn new_raw(
store: &mut StoreOpaque,
+ mut limiter: Option<&mut StoreResourceLimiter<'_>>,
module: &Module,
imports: Imports<'_>,
) -> Result<(Instance, Option)> {
@@ -295,7 +301,7 @@ impl Instance {
// Allocate the GC heap, if necessary.
if module.env_module().needs_gc_heap {
- store.ensure_gc_store().await?;
+ store.ensure_gc_store(limiter.as_deref_mut()).await?;
}
let compiled_module = module.compiled_module();
@@ -313,6 +319,7 @@ impl Instance {
let id = unsafe {
store
.allocate_instance(
+ limiter.as_deref_mut(),
AllocateInstanceKind::Module(module_id),
&ModuleRuntimeInfo::Module(module.clone()),
imports,
@@ -349,7 +356,7 @@ impl Instance {
.features()
.contains(WasmFeatures::BULK_MEMORY);
- vm::initialize_instance(store, id, compiled_module.module(), bulk_memory).await?;
+ vm::initialize_instance(store, limiter, id, compiled_module.module(), bulk_memory).await?;
Ok((instance, compiled_module.module().start_func))
}
diff --git a/crates/wasmtime/src/runtime/memory.rs b/crates/wasmtime/src/runtime/memory.rs
index c2ab4d4164c9..196a5055934e 100644
--- a/crates/wasmtime/src/runtime/memory.rs
+++ b/crates/wasmtime/src/runtime/memory.rs
@@ -260,7 +260,8 @@ impl Memory {
/// # }
/// ```
pub fn new(mut store: impl AsContextMut, ty: MemoryType) -> Result {
- vm::one_poll(Self::_new(store.as_context_mut().0, ty))
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ vm::one_poll(Self::_new(store, limiter.as_mut(), ty))
.expect("must use `new_async` when async resource limiters are in use")
}
@@ -274,12 +275,17 @@ impl Memory {
/// [`Store`](`crate::Store`).
#[cfg(feature = "async")]
pub async fn new_async(mut store: impl AsContextMut, ty: MemoryType) -> Result {
- Self::_new(store.as_context_mut().0, ty).await
+ let (mut limiter, store) = store.as_context_mut().0.resource_limiter_and_store_opaque();
+ Self::_new(store, limiter.as_mut(), ty).await
}
/// Helper function for attaching the memory to a "frankenstein" instance
- async fn _new(store: &mut StoreOpaque, ty: MemoryType) -> Result {
- generate_memory_export(store, &ty, None).await
+ async fn _new(
+ store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
+ ty: MemoryType,
+ ) -> Result {
+ generate_memory_export(store, limiter, &ty, None).await
}
/// Returns the underlying type of this memory.
@@ -1002,11 +1008,16 @@ impl SharedMemory {
/// [`SharedMemory`] into other modules.
pub(crate) fn vmimport(&self, store: &mut StoreOpaque) -> crate::runtime::vm::VMMemoryImport {
// Note `vm::assert_ready` shouldn't panic here because this isn't
- // actually allocating any new memory so resource limiting shouldn't
- // kick in.
- vm::assert_ready(generate_memory_export(store, &self.ty(), Some(&self.vm)))
- .unwrap()
- .vmimport(store)
+ // actually allocating any new memory (also no limiter), so resource
+ // limiting shouldn't kick in.
+ vm::assert_ready(generate_memory_export(
+ store,
+ None,
+ &self.ty(),
+ Some(&self.vm),
+ ))
+ .unwrap()
+ .vmimport(store)
}
/// Create a [`SharedMemory`] from an [`ExportMemory`] definition. This
diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs
index 60719641f3ce..960c4a3af785 100644
--- a/crates/wasmtime/src/runtime/store.rs
+++ b/crates/wasmtime/src/runtime/store.rs
@@ -702,9 +702,11 @@ impl Store {
unsafe {
// Note that this dummy instance doesn't allocate tables or memories
- // so it won't have an async await point meaning that it should be
- // ok to assert the future is always ready.
+ // (also no limiter is passed in) so it won't have an async await
+ // point meaning that it should be ok to assert the future is
+ // always ready.
let id = vm::assert_ready(inner.allocate_instance(
+ None,
AllocateInstanceKind::Dummy {
allocator: &allocator,
},
@@ -1502,15 +1504,21 @@ impl StoreOpaque {
/// `ResourceLimiterAsync` which means that this should only be executed
/// in a fiber context at this time.
#[inline]
- pub(crate) async fn ensure_gc_store(&mut self) -> Result<&mut GcStore> {
+ pub(crate) async fn ensure_gc_store(
+ &mut self,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
+ ) -> Result<&mut GcStore> {
if self.gc_store.is_some() {
return Ok(self.gc_store.as_mut().unwrap());
}
- self.allocate_gc_store().await
+ self.allocate_gc_store(limiter).await
}
#[inline(never)]
- async fn allocate_gc_store(&mut self) -> Result<&mut GcStore> {
+ async fn allocate_gc_store(
+ &mut self,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
+ ) -> Result<&mut GcStore> {
log::trace!("allocating GC heap for store {:?}", self.id());
assert!(self.gc_store.is_none());
@@ -1520,22 +1528,17 @@ impl StoreOpaque {
);
assert_eq!(self.vm_store_context.gc_heap.current_length(), 0);
- let gc_store = allocate_gc_store(self).await?;
+ let gc_store = allocate_gc_store(self, limiter).await?;
self.vm_store_context.gc_heap = gc_store.vmmemory_definition();
return Ok(self.gc_store.insert(gc_store));
#[cfg(feature = "gc")]
- async fn allocate_gc_store(store: &mut StoreOpaque) -> Result {
+ async fn allocate_gc_store(
+ store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
+ ) -> Result {
use wasmtime_environ::packed_option::ReservedValue;
- // FIXME(#11409) this is not a sound widening borrow
- let (mut limiter, store) = unsafe {
- store
- .traitobj()
- .as_mut()
- .resource_limiter_and_store_opaque()
- };
-
let engine = store.engine();
let mem_ty = engine.tunables().gc_heap_memory_type();
ensure!(
@@ -1551,7 +1554,7 @@ impl StoreOpaque {
)),
imports: vm::Imports::default(),
store,
- limiter: limiter.as_mut(),
+ limiter,
};
let (mem_alloc_index, mem) = engine
@@ -1573,7 +1576,10 @@ impl StoreOpaque {
}
#[cfg(not(feature = "gc"))]
- async fn allocate_gc_store(_: &mut StoreOpaque) -> Result {
+ async fn allocate_gc_store(
+ _: &mut StoreOpaque,
+ _: Option<&mut StoreResourceLimiter<'_>>,
+ ) -> Result {
bail!("cannot allocate a GC store: the `gc` feature was disabled at compile time")
}
}
@@ -2222,6 +2228,7 @@ at https://bytecodealliance.org/security.
/// being allocated.
pub(crate) async unsafe fn allocate_instance(
&mut self,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
kind: AllocateInstanceKind<'_>,
runtime_info: &ModuleRuntimeInfo,
imports: Imports<'_>,
@@ -2235,15 +2242,13 @@ at https://bytecodealliance.org/security.
// SAFETY: this function's own contract is the same as
// `allocate_module`, namely the imports provided are valid.
let handle = unsafe {
- // FIXME(#11409) this is not a sound widening borrow
- let (mut limiter, store) = self.traitobj().as_mut().resource_limiter_and_store_opaque();
allocator
.allocate_module(InstanceAllocationRequest {
id,
runtime_info,
imports,
- store,
- limiter: limiter.as_mut(),
+ store: self,
+ limiter,
})
.await?
};
diff --git a/crates/wasmtime/src/runtime/store/gc.rs b/crates/wasmtime/src/runtime/store/gc.rs
index 7eb4b6287b98..5be8a7d7178c 100644
--- a/crates/wasmtime/src/runtime/store/gc.rs
+++ b/crates/wasmtime/src/runtime/store/gc.rs
@@ -36,21 +36,6 @@ impl StoreOpaque {
})
}
- /// Same as [`Self::gc`], but less safe.
- ///
- /// FIXME(#11409) this method should not need to exist, but performing such
- /// a refactoring will require making memory creation async.
- async unsafe fn gc_unsafe_get_limiter(
- &mut self,
- root: Option,
- bytes_needed: Option,
- ) -> Option {
- // SAFETY: this isn't sound, see #11409
- let (mut limiter, store) =
- unsafe { self.traitobj().as_mut().resource_limiter_and_store_opaque() };
- store.gc(limiter.as_mut(), root, bytes_needed).await
- }
-
async fn grow_or_collect_gc_heap(
&mut self,
limiter: Option<&mut StoreResourceLimiter<'_>>,
@@ -169,23 +154,20 @@ impl StoreOpaque {
/// retry.
pub(crate) async fn retry_after_gc_async(
&mut self,
+ mut limiter: Option<&mut StoreResourceLimiter<'_>>,
value: T,
alloc_func: impl Fn(&mut Self, T) -> Result,
) -> Result
where
T: Send + Sync + 'static,
{
- self.ensure_gc_store().await?;
+ self.ensure_gc_store(limiter.as_deref_mut()).await?;
match alloc_func(self, value) {
Ok(x) => Ok(x),
Err(e) => match e.downcast::>() {
Ok(oom) => {
let (value, oom) = oom.take_inner();
- // SAFETY: FIXME(#11409)
- unsafe {
- self.gc_unsafe_get_limiter(None, Some(oom.bytes_needed()))
- .await;
- }
+ self.gc(limiter, None, Some(oom.bytes_needed())).await;
alloc_func(self, value)
}
Err(e) => Err(e),
diff --git a/crates/wasmtime/src/runtime/trampoline.rs b/crates/wasmtime/src/runtime/trampoline.rs
index 58b821b6443d..faa4321ff380 100644
--- a/crates/wasmtime/src/runtime/trampoline.rs
+++ b/crates/wasmtime/src/runtime/trampoline.rs
@@ -15,25 +15,30 @@ use self::table::create_table;
use self::tag::create_tag;
use crate::prelude::*;
use crate::runtime::vm::SharedMemory;
-use crate::store::StoreOpaque;
+use crate::store::{StoreOpaque, StoreResourceLimiter};
use crate::{MemoryType, TableType, TagType};
use wasmtime_environ::{MemoryIndex, TableIndex, TagIndex};
pub async fn generate_memory_export(
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
m: &MemoryType,
preallocation: Option<&SharedMemory>,
) -> Result {
let id = store.id();
- let instance = create_memory(store, m, preallocation).await?;
+ let instance = create_memory(store, limiter, m, preallocation).await?;
Ok(store
.instance_mut(instance)
.get_exported_memory(id, MemoryIndex::from_u32(0)))
}
-pub async fn generate_table_export(store: &mut StoreOpaque, t: &TableType) -> Result {
+pub async fn generate_table_export(
+ store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
+ t: &TableType,
+) -> Result {
let id = store.id();
- let instance = create_table(store, t).await?;
+ let instance = create_table(store, limiter, t).await?;
Ok(store
.instance_mut(instance)
.get_exported_table(id, TableIndex::from_u32(0)))
diff --git a/crates/wasmtime/src/runtime/trampoline/memory.rs b/crates/wasmtime/src/runtime/trampoline/memory.rs
index caae5a753654..27566a7570da 100644
--- a/crates/wasmtime/src/runtime/trampoline/memory.rs
+++ b/crates/wasmtime/src/runtime/trampoline/memory.rs
@@ -7,7 +7,7 @@ use crate::runtime::vm::{
MemoryBase, ModuleRuntimeInfo, OnDemandInstanceAllocator, RuntimeLinearMemory,
RuntimeMemoryCreator, SharedMemory, Table, TableAllocationIndex,
};
-use crate::store::{AllocateInstanceKind, InstanceId, StoreOpaque};
+use crate::store::{AllocateInstanceKind, InstanceId, StoreOpaque, StoreResourceLimiter};
use alloc::sync::Arc;
use wasmtime_environ::{
DefinedMemoryIndex, DefinedTableIndex, EntityIndex, HostPtr, Module, Tunables, VMOffsets,
@@ -26,6 +26,7 @@ use wasmtime_environ::{
/// outside: a host-provided memory import, shared memory.
pub async fn create_memory(
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
memory_ty: &MemoryType,
preallocation: Option<&SharedMemory>,
) -> Result {
@@ -54,6 +55,7 @@ pub async fn create_memory(
unsafe {
store
.allocate_instance(
+ limiter,
AllocateInstanceKind::Dummy {
allocator: &allocator,
},
@@ -170,7 +172,7 @@ unsafe impl InstanceAllocator for SingleMemoryInstance<'_> {
async fn allocate_memory(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Memory,
memory_index: Option,
) -> Result<(MemoryAllocationIndex, Memory)> {
@@ -208,7 +210,7 @@ unsafe impl InstanceAllocator for SingleMemoryInstance<'_> {
async fn allocate_table(
&self,
- req: &mut InstanceAllocationRequest<'_>,
+ req: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Table,
table_index: DefinedTableIndex,
) -> Result<(TableAllocationIndex, Table)> {
diff --git a/crates/wasmtime/src/runtime/trampoline/table.rs b/crates/wasmtime/src/runtime/trampoline/table.rs
index 2efd313d9e28..a0516f5bd7ad 100644
--- a/crates/wasmtime/src/runtime/trampoline/table.rs
+++ b/crates/wasmtime/src/runtime/trampoline/table.rs
@@ -1,11 +1,15 @@
use crate::TableType;
use crate::prelude::*;
use crate::runtime::vm::{Imports, ModuleRuntimeInfo, OnDemandInstanceAllocator};
-use crate::store::{AllocateInstanceKind, InstanceId, StoreOpaque};
+use crate::store::{AllocateInstanceKind, InstanceId, StoreOpaque, StoreResourceLimiter};
use alloc::sync::Arc;
use wasmtime_environ::{EntityIndex, Module, TypeTrace};
-pub async fn create_table(store: &mut StoreOpaque, table: &TableType) -> Result {
+pub async fn create_table(
+ store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
+ table: &TableType,
+) -> Result {
let mut module = Module::new();
let wasmtime_table = *table.wasmtime_table();
@@ -31,6 +35,7 @@ pub async fn create_table(store: &mut StoreOpaque, table: &TableType) -> Result<
let module = Arc::new(module);
store
.allocate_instance(
+ limiter,
AllocateInstanceKind::Dummy {
allocator: &allocator,
},
diff --git a/crates/wasmtime/src/runtime/trampoline/tag.rs b/crates/wasmtime/src/runtime/trampoline/tag.rs
index bd024802b4cd..4c9e25fc058b 100644
--- a/crates/wasmtime/src/runtime/trampoline/tag.rs
+++ b/crates/wasmtime/src/runtime/trampoline/tag.rs
@@ -39,9 +39,11 @@ pub fn create_tag(store: &mut StoreOpaque, ty: &TagType) -> Result {
let module = Arc::new(module);
// Note that `assert_ready` should be valid here because this module
- // doesn't allocate tables or memories meaning it shouldn't need an
- // await point.
+ // doesn't allocate tables or memories meaning it shouldn't need a
+ // resource limiter so `None` is passed. As a result no `await` points
+ // should ever be hit.
vm::assert_ready(store.allocate_instance(
+ None,
AllocateInstanceKind::Dummy {
allocator: &allocator,
},
diff --git a/crates/wasmtime/src/runtime/vm/const_expr.rs b/crates/wasmtime/src/runtime/vm/const_expr.rs
index 19350744f358..ae4cd0d63882 100644
--- a/crates/wasmtime/src/runtime/vm/const_expr.rs
+++ b/crates/wasmtime/src/runtime/vm/const_expr.rs
@@ -2,7 +2,7 @@
use crate::prelude::*;
use crate::runtime::vm;
-use crate::store::{AutoAssertNoGc, InstanceId, StoreOpaque};
+use crate::store::{AutoAssertNoGc, InstanceId, StoreOpaque, StoreResourceLimiter};
#[cfg(feature = "gc")]
use crate::{
AnyRef, ArrayRef, ArrayRefPre, ArrayType, ExternRef, I31, StructRef, StructRefPre, StructType,
@@ -72,12 +72,13 @@ impl ConstEvalContext {
async fn struct_new(
&mut self,
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
shared_ty: VMSharedTypeIndex,
fields: &[Val],
) -> Result {
let struct_ty = StructType::from_shared_type_index(store.engine(), shared_ty);
let allocator = StructRefPre::_new(store, struct_ty);
- let struct_ref = StructRef::_new_async(store, &allocator, &fields).await?;
+ let struct_ref = StructRef::_new_async(store, limiter, &allocator, &fields).await?;
Ok(Val::AnyRef(Some(struct_ref.into())))
}
@@ -85,6 +86,7 @@ impl ConstEvalContext {
async fn struct_new_default(
&mut self,
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
shared_ty: VMSharedTypeIndex,
) -> Result {
let module = store
@@ -130,7 +132,7 @@ impl ConstEvalContext {
})
.collect::>();
- self.struct_new(store, shared_ty, &fields).await
+ self.struct_new(store, limiter, shared_ty, &fields).await
}
}
@@ -157,7 +159,7 @@ impl ConstExprEvaluator {
// Note that `assert_ready` here should be valid as production of an
// integer cannot involve GC meaning that async operations aren't used.
let mut scope = OpaqueRootScope::new(store);
- vm::assert_ready(self.eval_loop(&mut scope, context, expr))
+ vm::assert_ready(self.eval_loop(&mut scope, None, context, expr))
}
/// Attempts to peek into `expr` to see if it's trivial to evaluate, e.g.
@@ -187,6 +189,7 @@ impl ConstExprEvaluator {
pub async fn eval(
&mut self,
store: &mut OpaqueRootScope<&mut StoreOpaque>,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
context: &mut ConstEvalContext,
expr: &ConstExpr,
) -> Result<&Val> {
@@ -195,7 +198,7 @@ impl ConstExprEvaluator {
if self.try_simple(expr).is_some() {
return Ok(&self.simple);
}
- self.eval_loop(store, context, expr).await
+ self.eval_loop(store, limiter, context, expr).await
}
#[inline]
@@ -211,6 +214,7 @@ impl ConstExprEvaluator {
async fn eval_loop(
&mut self,
store: &mut OpaqueRootScope<&mut StoreOpaque>,
+ mut limiter: Option<&mut StoreResourceLimiter<'_>>,
context: &mut ConstEvalContext,
expr: &ConstExpr,
) -> Result<&Val> {
@@ -218,6 +222,10 @@ impl ConstExprEvaluator {
self.stack.clear();
+ // On GC-less builds ensure that this is always considered used an
+ // needed-mutable.
+ let _ = &mut limiter;
+
for op in expr.ops() {
log::trace!("const-evaluating op: {op:?}");
match op {
@@ -299,7 +307,12 @@ impl ConstExprEvaluator {
let start = self.stack.len() - len;
let s = context
- .struct_new(store, interned_type_index, &self.stack[start..])
+ .struct_new(
+ store,
+ limiter.as_deref_mut(),
+ interned_type_index,
+ &self.stack[start..],
+ )
.await?;
self.stack.truncate(start);
self.stack.push(s);
@@ -310,8 +323,11 @@ impl ConstExprEvaluator {
let ty = store.instance(context.instance).env_module().types
[*struct_type_index]
.unwrap_engine_type_index();
- self.stack
- .push(context.struct_new_default(store, ty).await?);
+ self.stack.push(
+ context
+ .struct_new_default(store, limiter.as_deref_mut(), ty)
+ .await?,
+ );
}
#[cfg(feature = "gc")]
@@ -325,7 +341,9 @@ impl ConstExprEvaluator {
let elem = self.pop()?;
let pre = ArrayRefPre::_new(store, ty);
- let array = ArrayRef::_new_async(store, &pre, &elem, len).await?;
+ let array =
+ ArrayRef::_new_async(store, limiter.as_deref_mut(), &pre, &elem, len)
+ .await?;
self.stack.push(Val::AnyRef(Some(array.into())));
}
@@ -342,7 +360,9 @@ impl ConstExprEvaluator {
.expect("type should have a default value");
let pre = ArrayRefPre::_new(store, ty);
- let array = ArrayRef::_new_async(store, &pre, &elem, len).await?;
+ let array =
+ ArrayRef::_new_async(store, limiter.as_deref_mut(), &pre, &elem, len)
+ .await?;
self.stack.push(Val::AnyRef(Some(array.into())));
}
@@ -372,7 +392,9 @@ impl ConstExprEvaluator {
.collect::>();
let pre = ArrayRefPre::_new(store, ty);
- let array = ArrayRef::_new_fixed_async(store, &pre, &elems).await?;
+ let array =
+ ArrayRef::_new_fixed_async(store, limiter.as_deref_mut(), &pre, &elems)
+ .await?;
self.stack.push(Val::AnyRef(Some(array.into())));
}
diff --git a/crates/wasmtime/src/runtime/vm/instance.rs b/crates/wasmtime/src/runtime/vm/instance.rs
index a7ba3625e38a..899204e811a4 100644
--- a/crates/wasmtime/src/runtime/vm/instance.rs
+++ b/crates/wasmtime/src/runtime/vm/instance.rs
@@ -868,6 +868,7 @@ impl Instance {
pub(crate) async fn table_init(
self: Pin<&mut Self>,
store: &mut StoreOpaque,
+ limiter: Option<&mut StoreResourceLimiter<'_>>,
table_index: TableIndex,
elem_index: ElemIndex,
dst: u64,
@@ -879,6 +880,7 @@ impl Instance {
let mut const_evaluator = ConstExprEvaluator::default();
Self::table_init_segment(
store,
+ limiter,
self.id,
&mut const_evaluator,
table_index,
@@ -892,6 +894,7 @@ impl Instance {
pub(crate) async fn table_init_segment(
store: &mut StoreOpaque,
+ mut limiter: Option<&mut StoreResourceLimiter<'_>>,
elements_instance_id: InstanceId,
const_evaluator: &mut ConstExprEvaluator,
table_index: TableIndex,
@@ -947,7 +950,7 @@ impl Instance {
let mut context = ConstEvalContext::new(elements_instance_id);
for (i, expr) in positions.zip(exprs) {
let element = const_evaluator
- .eval(&mut store, &mut context, expr)
+ .eval(&mut store, limiter.as_deref_mut(), &mut context, expr)
.await
.expect("const expr should be valid");
table.set_(&mut store, i, element.ref_().unwrap()).unwrap();
diff --git a/crates/wasmtime/src/runtime/vm/instance/allocator.rs b/crates/wasmtime/src/runtime/vm/instance/allocator.rs
index 988dae1fe0b7..136aa4b10321 100644
--- a/crates/wasmtime/src/runtime/vm/instance/allocator.rs
+++ b/crates/wasmtime/src/runtime/vm/instance/allocator.rs
@@ -35,7 +35,7 @@ pub use self::pooling::{
};
/// Represents a request for a new runtime instance.
-pub struct InstanceAllocationRequest<'a> {
+pub struct InstanceAllocationRequest<'a, 'b> {
/// The instance id that this will be assigned within the store once the
/// allocation has finished.
pub id: InstanceId,
@@ -54,7 +54,7 @@ pub struct InstanceAllocationRequest<'a> {
pub store: &'a StoreOpaque,
/// The store's resource limiter, if configured by the embedder.
- pub limiter: Option<&'a mut StoreResourceLimiter<'a>>,
+ pub limiter: Option<&'a mut StoreResourceLimiter<'b>>,
}
/// The index of a memory allocation within an `InstanceAllocator`.
@@ -186,7 +186,7 @@ pub unsafe trait InstanceAllocator: Send + Sync {
/// Allocate a memory for an instance.
async fn allocate_memory(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Memory,
memory_index: Option,
) -> Result<(MemoryAllocationIndex, Memory)>;
@@ -208,7 +208,7 @@ pub unsafe trait InstanceAllocator: Send + Sync {
/// Allocate a table for an instance.
async fn allocate_table(
&self,
- req: &mut InstanceAllocationRequest<'_>,
+ req: &mut InstanceAllocationRequest<'_, '_>,
table: &wasmtime_environ::Table,
table_index: DefinedTableIndex,
) -> Result<(TableAllocationIndex, Table)>;
@@ -304,7 +304,7 @@ impl dyn InstanceAllocator + '_ {
/// correctly sized/typed for the instance being created.
pub(crate) async unsafe fn allocate_module(
&self,
- mut request: InstanceAllocationRequest<'_>,
+ mut request: InstanceAllocationRequest<'_, '_>,
) -> Result {
let module = request.runtime_info.env_module();
@@ -388,7 +388,7 @@ impl dyn InstanceAllocator + '_ {
/// them into `memories`.
async fn allocate_memories(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
memories: &mut PrimaryMap,
) -> Result<()> {
let module = request.runtime_info.env_module();
@@ -441,7 +441,7 @@ impl dyn InstanceAllocator + '_ {
/// into `tables`.
async fn allocate_tables(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
tables: &mut PrimaryMap,
) -> Result<()> {
let module = request.runtime_info.env_module();
@@ -515,6 +515,7 @@ fn check_table_init_bounds(
async fn initialize_tables(
store: &mut StoreOpaque,
+ mut limiter: Option<&mut StoreResourceLimiter<'_>>,
context: &mut ConstEvalContext,
const_evaluator: &mut ConstExprEvaluator,
module: &Module,
@@ -527,7 +528,7 @@ async fn initialize_tables(
TableInitialValue::Expr(expr) => {
let init = const_evaluator
- .eval(&mut store, context, expr)
+ .eval(&mut store, limiter.as_deref_mut(), context, expr)
.await
.expect("const expression should be valid");
let idx = module.table_index(table);
@@ -558,6 +559,7 @@ async fn initialize_tables(
);
Instance::table_init_segment(
&mut store,
+ limiter.as_deref_mut(),
context.instance,
const_evaluator,
segment.table_index,
@@ -738,6 +740,7 @@ fn check_init_bounds(store: &mut StoreOpaque, instance: InstanceId, module: &Mod
async fn initialize_globals(
store: &mut StoreOpaque,
+ mut limiter: Option<&mut StoreResourceLimiter<'_>>,
context: &mut ConstEvalContext,
const_evaluator: &mut ConstExprEvaluator,
module: &Module,
@@ -757,7 +760,7 @@ async fn initialize_globals(
val
} else {
const_evaluator
- .eval(&mut store, context, init)
+ .eval(&mut store, limiter.as_deref_mut(), context, init)
.await
.expect("should be a valid const expr")
};
@@ -794,6 +797,7 @@ async fn initialize_globals(
pub async fn initialize_instance(
store: &mut StoreOpaque,
+ mut limiter: Option<&mut StoreResourceLimiter<'_>>,
instance: InstanceId,
module: &Module,
is_bulk_memory: bool,
@@ -809,8 +813,22 @@ pub async fn initialize_instance(
let mut context = ConstEvalContext::new(instance);
let mut const_evaluator = ConstExprEvaluator::default();
- initialize_globals(store, &mut context, &mut const_evaluator, module).await?;
- initialize_tables(store, &mut context, &mut const_evaluator, module).await?;
+ initialize_globals(
+ store,
+ limiter.as_deref_mut(),
+ &mut context,
+ &mut const_evaluator,
+ module,
+ )
+ .await?;
+ initialize_tables(
+ store,
+ limiter.as_deref_mut(),
+ &mut context,
+ &mut const_evaluator,
+ module,
+ )
+ .await?;
initialize_memories(store, &mut context, &mut const_evaluator, &module)?;
Ok(())
diff --git a/crates/wasmtime/src/runtime/vm/instance/allocator/on_demand.rs b/crates/wasmtime/src/runtime/vm/instance/allocator/on_demand.rs
index 9dab461c8c70..0373ac30c46c 100644
--- a/crates/wasmtime/src/runtime/vm/instance/allocator/on_demand.rs
+++ b/crates/wasmtime/src/runtime/vm/instance/allocator/on_demand.rs
@@ -111,7 +111,7 @@ unsafe impl InstanceAllocator for OnDemandInstanceAllocator {
async fn allocate_memory(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Memory,
memory_index: Option,
) -> Result<(MemoryAllocationIndex, Memory)> {
@@ -150,7 +150,7 @@ unsafe impl InstanceAllocator for OnDemandInstanceAllocator {
async fn allocate_table(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Table,
_table_index: DefinedTableIndex,
) -> Result<(TableAllocationIndex, Table)> {
diff --git a/crates/wasmtime/src/runtime/vm/instance/allocator/pooling.rs b/crates/wasmtime/src/runtime/vm/instance/allocator/pooling.rs
index 52a199a2efe9..bd16bebb96f6 100644
--- a/crates/wasmtime/src/runtime/vm/instance/allocator/pooling.rs
+++ b/crates/wasmtime/src/runtime/vm/instance/allocator/pooling.rs
@@ -648,7 +648,7 @@ unsafe impl InstanceAllocator for PoolingInstanceAllocator {
async fn allocate_memory(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Memory,
memory_index: Option,
) -> Result<(MemoryAllocationIndex, Memory)> {
@@ -708,7 +708,7 @@ unsafe impl InstanceAllocator for PoolingInstanceAllocator {
async fn allocate_table(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Table,
_table_index: DefinedTableIndex,
) -> Result<(super::TableAllocationIndex, Table)> {
diff --git a/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/memory_pool.rs b/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/memory_pool.rs
index 5a298bbad695..472e1f48595a 100644
--- a/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/memory_pool.rs
+++ b/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/memory_pool.rs
@@ -317,7 +317,7 @@ impl MemoryPool {
/// Allocate a single memory for the given instance allocation request.
pub async fn allocate(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Memory,
memory_index: Option,
) -> Result<(MemoryAllocationIndex, Memory)> {
diff --git a/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/table_pool.rs b/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/table_pool.rs
index aa02e6355d36..0047c427ea6a 100644
--- a/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/table_pool.rs
+++ b/crates/wasmtime/src/runtime/vm/instance/allocator/pooling/table_pool.rs
@@ -132,7 +132,7 @@ impl TablePool {
/// Allocate a single table for the given instance allocation request.
pub async fn allocate(
&self,
- request: &mut InstanceAllocationRequest<'_>,
+ request: &mut InstanceAllocationRequest<'_, '_>,
ty: &wasmtime_environ::Table,
) -> Result<(TableAllocationIndex, Table)> {
let tunables = request.store.engine().tunables();
diff --git a/crates/wasmtime/src/runtime/vm/libcalls.rs b/crates/wasmtime/src/runtime/vm/libcalls.rs
index 3e5318d5899e..52b50ac0dc04 100644
--- a/crates/wasmtime/src/runtime/vm/libcalls.rs
+++ b/crates/wasmtime/src/runtime/vm/libcalls.rs
@@ -486,9 +486,18 @@ fn table_init(
let table_index = TableIndex::from_u32(table_index);
let elem_index = ElemIndex::from_u32(elem_index);
+ let (mut limiter, store) = store.resource_limiter_and_store_opaque();
block_on!(store, async |store| {
instance
- .table_init(store, table_index, elem_index, dst, src, len)
+ .table_init(
+ store,
+ limiter.as_mut(),
+ table_index,
+ elem_index,
+ dst,
+ src,
+ len,
+ )
.await
})??;
Ok(())
@@ -673,9 +682,10 @@ unsafe fn gc_alloc_raw(
err.context(e)
})?;
+ let (mut limiter, store) = store.resource_limiter_and_store_opaque();
block_on!(store, async |store| {
let gc_ref = store
- .retry_after_gc_async((), |store, ()| {
+ .retry_after_gc_async(limiter.as_mut(), (), |store, ()| {
store
.unwrap_gc_store_mut()
.alloc_raw(header, layout)?
@@ -764,6 +774,7 @@ unsafe fn array_new_data(
use crate::ArrayType;
use wasmtime_environ::ModuleInternedTypeIndex;
+ let (mut limiter, store) = store.resource_limiter_and_store_opaque();
block_on!(store, async |store| {
let array_type_index = ModuleInternedTypeIndex::from_u32(array_type_index);
let data_index = DataIndex::from_u32(data_index);
@@ -798,7 +809,7 @@ unsafe fn array_new_data(
.expect("array types have GC layouts");
let array_layout = gc_layout.unwrap_array();
let array_ref = store
- .retry_after_gc_async((), |store, ()| {
+ .retry_after_gc_async(limiter.as_mut(), (), |store, ()| {
store
.unwrap_gc_store_mut()
.alloc_uninit_array(shared_ty, len, &array_layout)?
@@ -932,6 +943,7 @@ unsafe fn array_new_elem(
let array_ty = ArrayType::from_shared_type_index(store.engine(), shared_ty);
let pre = ArrayRefPre::_new(store, array_ty);
+ let (mut limiter, store) = store.resource_limiter_and_store_opaque();
block_on!(store, async |store| {
let mut store = OpaqueRootScope::new(store);
// Turn the elements into `Val`s.
@@ -963,7 +975,7 @@ unsafe fn array_new_elem(
for x in xs.iter() {
let val = *const_evaluator
- .eval(&mut store, &mut const_context, x)
+ .eval(&mut store, limiter.as_mut(), &mut const_context, x)
.await
.expect("const expr should be valid");
vals.push(val);
@@ -971,7 +983,7 @@ unsafe fn array_new_elem(
}
}
- let array = ArrayRef::_new_fixed_async(&mut store, &pre, &vals).await?;
+ let array = ArrayRef::_new_fixed_async(&mut store, limiter.as_mut(), &pre, &vals).await?;
let mut store = AutoAssertNoGc::new(&mut store);
let gc_ref = array.try_clone_gc_ref(&mut store)?;
@@ -998,6 +1010,7 @@ unsafe fn array_init_elem(
};
use wasmtime_environ::{ModuleInternedTypeIndex, TableSegmentElements};
+ let (mut limiter, store) = store.resource_limiter_and_store_opaque();
block_on!(store, async |store| {
let mut store = OpaqueRootScope::new(store);
@@ -1057,7 +1070,7 @@ unsafe fn array_init_elem(
.ok_or_else(|| Trap::TableOutOfBounds)?
{
let val = *const_evaluator
- .eval(&mut store, &mut const_context, x)
+ .eval(&mut store, limiter.as_mut(), &mut const_context, x)
.await
.expect("const expr should be valid");
vals.push(val);