From c0ad52b07344a73bfb6c7dda453a613de973890a Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Fri, 15 Aug 2025 14:29:03 -0700 Subject: [PATCH 1/2] Make table growth a true `async fn` Upon further refactoring and thinking about #11430 I've realized that we might be able to sidestep `T: Send` on the store entirely which would be quite the boon if it can be pulled off. The realization I had is that the main reason for this was `&mut dyn VMStore` on the stack, but that itself is actually a bug in Wasmtime (#11178) and shouldn't be done. The functions which have this on the stack should actually ONLY have the resource limiter, if configured. This means that while the `ResourceLimiter{,Async}` traits need a `Send` supertrait that's relatively easy to add without much impact. My hunch is that plumbing this through to the end will enable all the benefits of #11430 without requiring adding `T: Send` to the store. This commit starts out on this journey by making table growth a true `async fn`. A new internal type is added to represent a store's limiter which is plumbed to growth functions. This represents a hierarchy of borrows that look like: * `StoreInner` * `StoreResourceLimiter<'_>` * `StoreOpaque` * `Pin<&mut Instance>` * `&mut vm::Table` This notably, safely, allows operating on `vm::Table` with a `StoreResourceLimiter` at the same time. This is exactly what's needed and prevents needing to have `&mut dyn VMStore`, the previous argument, on the stack. This refactoring cleans up `unsafe` blocks in table growth right now which manually uses raw pointers to work around the borrow checker. No more now! I'll note as well that this is just an incremental step. What I plan on doing next is handling other locations like memory growth, memory allocation, and table allocation. Each of those will require further refactorings to ensure that things like GC are correctly accounted for so they're going to be split into separate PRs. Functionally though this PR should have no impact other than a fiber is no longer required for `Table::grow_async`. --- .../wasmtime/src/runtime/externals/table.rs | 125 ++++++++++-------- crates/wasmtime/src/runtime/limits.rs | 4 +- crates/wasmtime/src/runtime/store.rs | 50 +++++++ crates/wasmtime/src/runtime/store/data.rs | 12 +- crates/wasmtime/src/runtime/vm.rs | 49 ++++++- crates/wasmtime/src/runtime/vm/instance.rs | 6 +- crates/wasmtime/src/runtime/vm/libcalls.rs | 111 +++++++++++++--- crates/wasmtime/src/runtime/vm/table.rs | 54 +++++--- 8 files changed, 306 insertions(+), 105 deletions(-) diff --git a/crates/wasmtime/src/runtime/externals/table.rs b/crates/wasmtime/src/runtime/externals/table.rs index 49647cfe8dbf..a8cfcb5ec7a5 100644 --- a/crates/wasmtime/src/runtime/externals/table.rs +++ b/crates/wasmtime/src/runtime/externals/table.rs @@ -1,11 +1,13 @@ use crate::prelude::*; use crate::runtime::RootedGcRefImpl; -use crate::runtime::vm::{self as runtime, GcStore, TableElementType, VMFuncRef, VMGcRef}; +use crate::runtime::vm::{ + self, GcStore, SendSyncPtr, TableElementType, VMFuncRef, VMGcRef, VMStore, +}; use crate::store::{AutoAssertNoGc, StoreInstanceId, StoreOpaque}; use crate::trampoline::generate_table_export; use crate::{ - AnyRef, AsContext, AsContextMut, ExnRef, ExternRef, Func, HeapType, Ref, RefType, TableType, - Trap, + AnyRef, AsContext, AsContextMut, ExnRef, ExternRef, Func, HeapType, Ref, RefType, + StoreContextMut, TableType, Trap, }; use core::iter; use core::ptr::NonNull; @@ -139,7 +141,7 @@ impl Table { TableType::from_wasmtime_table(store.engine(), self.wasmtime_ty(store)) } - /// Returns the `runtime::Table` within `store` as well as the optional + /// Returns the `vm::Table` within `store` as well as the optional /// `GcStore` in use within `store`. /// /// # Panics @@ -149,7 +151,7 @@ impl Table { &self, store: &'a mut StoreOpaque, lazy_init_range: impl IntoIterator, - ) -> (&'a mut runtime::Table, Option<&'a mut GcStore>) { + ) -> (&'a mut vm::Table, Option<&'a mut GcStore>) { self.instance.assert_belongs_to(store.id()); let (store, instance) = store.optional_gc_store_and_instance_mut(self.instance.instance()); @@ -281,42 +283,60 @@ impl Table { /// When using an async resource limiter, use [`Table::grow_async`] /// instead. pub fn grow(&self, mut store: impl AsContextMut, delta: u64, init: Ref) -> Result { - let store = store.as_context_mut().0; + vm::one_poll(self._grow(store.as_context_mut(), delta, init)) + .expect("must use `grow_async` when async resource limiters are in use") + } + + async fn _grow(&self, store: StoreContextMut<'_, T>, delta: u64, init: Ref) -> Result { + let store = store.0; let ty = self.ty(&store); - let (table, _gc_store) = self.wasmtime_table(store, iter::empty()); - // FIXME(#11179) shouldn't need to subvert the borrow checker - let table: *mut _ = table; - unsafe { - let result = match element_type(&ty) { - TableElementType::Func => { - let element = init.into_table_func(store, ty.element())?; - (*table).grow_func(store, delta, element)? - } - TableElementType::GcRef => { - // FIXME: `grow_gc_ref` shouldn't require the whole store - // and should require `AutoAssertNoGc`. For now though we - // know that table growth doesn't trigger GC so it should be - // ok to create a copy of the GC reference even though it's - // not tracked anywhere. - let element = init - .into_table_gc_ref(&mut AutoAssertNoGc::new(store), ty.element())? - .map(|r| r.unchecked_copy()); - (*table).grow_gc_ref(store, delta, element.as_ref())? - } - // TODO(#10248) Required to support stack switching in the - // embedder API. - TableElementType::Cont => bail!("unimplemented table for cont"), - }; - match result { - Some(size) => { - let vm = (*table).vmtable(); - store[self.instance].table_ptr(self.index).write(vm); - // unwrap here should be ok because the runtime should always guarantee - // that we can fit the table size in a 64-bit integer. - Ok(u64::try_from(size).unwrap()) - } - None => bail!("failed to grow table by `{}`", delta), + let (mut limiter, store) = store.resource_limiter_and_store_opaque(); + let limiter = limiter.as_mut(); + let result = match element_type(&ty) { + TableElementType::Func => { + let element = init + .into_table_func(store, ty.element())? + .map(SendSyncPtr::new); + self.instance + .get_mut(store) + .defined_table_grow(self.index, async |table| { + // SAFETY: in the context of `defined_table_grow` this + // is safe to call as it'll update the internal table + // pointer in the instance. + unsafe { table.grow_func(limiter, delta, element).await } + }) + .await? } + TableElementType::GcRef => { + let mut store = AutoAssertNoGc::new(store); + let element = init + .into_table_gc_ref(&mut store, ty.element())? + .map(|r| r.unchecked_copy()); + let (gc_store, instance) = self.instance.get_with_gc_store_mut(&mut store); + instance + .defined_table_grow(self.index, async |table| { + // SAFETY: in the context of `defined_table_grow` this + // is safe to call as it'll update the internal table + // pointer in the instance. + unsafe { + table + .grow_gc_ref(limiter, gc_store, delta, element.as_ref()) + .await + } + }) + .await? + } + // TODO(#10248) Required to support stack switching in the + // embedder API. + TableElementType::Cont => bail!("unimplemented table for cont"), + }; + match result { + Some(size) => { + // unwrap here should be ok because the runtime should always + // guarantee that we can fit the table size in a 64-bit integer. + Ok(u64::try_from(size).unwrap()) + } + None => bail!("failed to grow table by `{}`", delta), } } @@ -334,14 +354,7 @@ impl Table { delta: u64, init: Ref, ) -> Result { - let mut store = store.as_context_mut(); - assert!( - store.0.async_support(), - "cannot use `grow_async` without enabling async support on the config" - ); - store - .on_fiber(|store| self.grow(store, delta, init)) - .await? + self._grow(store.as_context_mut(), delta, init).await } /// Copy `len` elements from `src_table[src_index..]` into @@ -516,11 +529,7 @@ impl Table { } #[cfg(feature = "gc")] - pub(crate) fn trace_roots( - &self, - store: &mut StoreOpaque, - gc_roots_list: &mut crate::runtime::vm::GcRootsList, - ) { + pub(crate) fn trace_roots(&self, store: &mut StoreOpaque, gc_roots_list: &mut vm::GcRootsList) { if !self ._ty(store) .element() @@ -549,9 +558,9 @@ impl Table { &module.tables[index] } - pub(crate) fn vmimport(&self, store: &StoreOpaque) -> crate::runtime::vm::VMTableImport { + pub(crate) fn vmimport(&self, store: &StoreOpaque) -> vm::VMTableImport { let instance = &store[self.instance]; - crate::runtime::vm::VMTableImport { + vm::VMTableImport { from: instance.table_ptr(self.index).into(), vmctx: instance.vmctx().into(), index: self.index, @@ -691,4 +700,12 @@ mod tests { Ok(()) } + + #[test] + fn grow_is_send() { + fn _assert_send(_: T) {} + fn _grow(table: &Table, store: &mut Store<()>, init: Ref) { + _assert_send(table.grow(store, 0, init)) + } + } } diff --git a/crates/wasmtime/src/runtime/limits.rs b/crates/wasmtime/src/runtime/limits.rs index a3d3bad646ca..26f7fcb28103 100644 --- a/crates/wasmtime/src/runtime/limits.rs +++ b/crates/wasmtime/src/runtime/limits.rs @@ -29,7 +29,7 @@ pub const DEFAULT_MEMORY_LIMIT: usize = 10000; /// or not and you're otherwise working in an asynchronous context the /// [`ResourceLimiterAsync`] trait is also provided to avoid blocking an OS /// thread while a limit is determined. -pub trait ResourceLimiter { +pub trait ResourceLimiter: Send { /// Notifies the resource limiter that an instance's linear memory has been /// requested to grow. /// @@ -172,7 +172,7 @@ pub trait ResourceLimiter { /// answer the question whether growing a memory or table is allowed. #[cfg(feature = "async")] #[async_trait::async_trait] -pub trait ResourceLimiterAsync { +pub trait ResourceLimiterAsync: Send { /// Async version of [`ResourceLimiter::memory_growing`] async fn memory_growing( &mut self, diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index cea1f82d1cef..36251b29f80f 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -247,6 +247,43 @@ enum ResourceLimiterInner { Async(Box &mut dyn crate::ResourceLimiterAsync) + Send + Sync>), } +/// Representation of a configured resource limiter for a store. +/// +/// This is acquired with `resource_limiter_and_store_opaque` for example and is +/// threaded through to growth operations on tables/memories. Note that this is +/// passed around as `Option<&mut StoreResourceLimiter<'_>>` to make it +/// efficient to pass around (nullable pointer) and it's also notably passed +/// around as an `Option` to represent how this is optionally specified within a +/// store. +pub enum StoreResourceLimiter<'a> { + Sync(&'a mut dyn crate::ResourceLimiter), + #[cfg(feature = "async")] + Async(&'a mut dyn crate::ResourceLimiterAsync), +} + +impl StoreResourceLimiter<'_> { + pub(crate) async fn table_growing( + &mut self, + current: usize, + desired: usize, + maximum: Option, + ) -> Result { + match self { + Self::Sync(s) => s.table_growing(current, desired, maximum), + #[cfg(feature = "async")] + Self::Async(s) => s.table_growing(current, desired, maximum).await, + } + } + + pub(crate) fn table_grow_failed(&mut self, error: anyhow::Error) -> Result<()> { + match self { + Self::Sync(s) => s.table_grow_failed(error), + #[cfg(feature = "async")] + Self::Async(s) => s.table_grow_failed(error), + } + } +} + enum CallHookInner { #[cfg(feature = "call-hook")] Sync(Box, CallHook) -> Result<()> + Send + Sync>), @@ -2234,6 +2271,19 @@ unsafe impl vm::VMStore for StoreInner { &mut self.inner } + fn resource_limiter_and_store_opaque( + &mut self, + ) -> (Option>, &mut StoreOpaque) { + ( + self.limiter.as_mut().map(|l| match l { + ResourceLimiterInner::Sync(s) => StoreResourceLimiter::Sync(s(&mut self.data)), + #[cfg(feature = "async")] + ResourceLimiterInner::Async(s) => StoreResourceLimiter::Async(s(&mut self.data)), + }), + &mut self.inner, + ) + } + fn memory_growing( &mut self, current: usize, diff --git a/crates/wasmtime/src/runtime/store/data.rs b/crates/wasmtime/src/runtime/store/data.rs index 19b0110abe73..71599f879492 100644 --- a/crates/wasmtime/src/runtime/store/data.rs +++ b/crates/wasmtime/src/runtime/store/data.rs @@ -1,4 +1,4 @@ -use crate::runtime::vm::{self, VMStore}; +use crate::runtime::vm::{self, GcStore, VMStore}; use crate::store::StoreOpaque; use crate::{StoreContext, StoreContextMut}; use core::num::NonZeroU64; @@ -252,6 +252,16 @@ impl StoreInstanceId { self.assert_belongs_to(store.id()); store.instance_mut(self.instance) } + + /// Same as [`Self::get_mut`], but also returns the `GcStore`. + #[inline] + pub(crate) fn get_with_gc_store_mut<'a>( + &self, + store: &'a mut StoreOpaque, + ) -> (Option<&'a mut GcStore>, Pin<&'a mut vm::Instance>) { + self.assert_belongs_to(store.id()); + store.optional_gc_store_and_instance_mut(self.instance) + } } impl Index for StoreOpaque { diff --git a/crates/wasmtime/src/runtime/vm.rs b/crates/wasmtime/src/runtime/vm.rs index 70ebc795f17f..0ff21c4b0f9c 100644 --- a/crates/wasmtime/src/runtime/vm.rs +++ b/crates/wasmtime/src/runtime/vm.rs @@ -35,15 +35,15 @@ pub(crate) struct f64x2(crate::uninhabited::Uninhabited); use crate::StoreContextMut; use crate::prelude::*; -use crate::store::StoreInner; -use crate::store::StoreOpaque; +use crate::store::{StoreInner, StoreOpaque, StoreResourceLimiter}; use crate::type_registry::RegisteredType; use alloc::sync::Arc; use core::fmt; -use core::ops::Deref; -use core::ops::DerefMut; +use core::ops::{Deref, DerefMut}; +use core::pin::pin; use core::ptr::NonNull; use core::sync::atomic::{AtomicUsize, Ordering}; +use core::task::{Context, Poll, Waker}; use wasmtime_environ::{ DefinedFuncIndex, DefinedMemoryIndex, HostPtr, VMOffsets, VMSharedTypeIndex, }; @@ -201,6 +201,12 @@ pub unsafe trait VMStore: 'static { /// Get an exclusive borrow of this store's `StoreOpaque`. fn store_opaque_mut(&mut self) -> &mut StoreOpaque; + /// Returns a split borrow to the limiter plus `StoreOpaque` at the same + /// time. + fn resource_limiter_and_store_opaque( + &mut self, + ) -> (Option>, &mut StoreOpaque); + /// Callback invoked to allow the store's resource limiter to reject a /// memory grow operation. fn memory_growing( @@ -509,3 +515,38 @@ impl fmt::Display for WasmFault { ) } } + +/// Asserts that the future `f` is ready and returns its output. +/// +/// This function is intended to be used when `async_support` is verified as +/// disabled. Internals of Wasmtime are generally `async` when they optionally +/// can be, meaning that synchronous entrypoints will invoke this function +/// after invoking the asynchronous internals. Due to `async_support` being +/// disabled there should be no way to introduce a yield point meaning that all +/// futures built from internal functions should always be ready. +/// +/// # Panics +/// +/// Panics if `f` is not yet ready. +pub fn assert_ready(f: F) -> F::Output { + one_poll(f).unwrap() +} + +/// Attempts one poll of `f` to see if its output is available. +/// +/// This function is intended for a few minor entrypoints into the Wasmtime API +/// where a synchronous function is documented to work even when `async_support` +/// is enabled. For example growing a `Memory` can be done with a synchronous +/// function, but it's documented to panic with an async resource limiter. +/// +/// This function provides the opportunity to poll `f` once to see if its output +/// is available. If it isn't then `None` is returned and an appropriate panic +/// message should be generated recommending to use an async function (e.g. +/// `grow_async` instead of `grow`). +pub fn one_poll(f: F) -> Option { + let mut context = Context::from_waker(&Waker::noop()); + match pin!(f).poll(&mut context) { + Poll::Ready(output) => Some(output), + Poll::Pending => None, + } +} diff --git a/crates/wasmtime/src/runtime/vm/instance.rs b/crates/wasmtime/src/runtime/vm/instance.rs index ce807fa8ab67..6e2676d881e1 100644 --- a/crates/wasmtime/src/runtime/vm/instance.rs +++ b/crates/wasmtime/src/runtime/vm/instance.rs @@ -766,13 +766,13 @@ impl Instance { /// Performs a grow operation on the `table_index` specified using `grow`. /// /// This will handle updating the VMTableDefinition internally as necessary. - pub(crate) fn defined_table_grow( + pub(crate) async fn defined_table_grow( mut self: Pin<&mut Self>, table_index: DefinedTableIndex, - grow: impl FnOnce(&mut Table) -> Result>, + grow: impl AsyncFnOnce(&mut Table) -> Result>, ) -> Result> { let table = self.as_mut().get_defined_table(table_index); - let result = grow(table); + let result = grow(table).await; let element = table.vmtable(); self.set_table(table_index, element); result diff --git a/crates/wasmtime/src/runtime/vm/libcalls.rs b/crates/wasmtime/src/runtime/vm/libcalls.rs index f6c6cfb846d7..c3c5581116a8 100644 --- a/crates/wasmtime/src/runtime/vm/libcalls.rs +++ b/crates/wasmtime/src/runtime/vm/libcalls.rs @@ -57,13 +57,14 @@ #[cfg(feature = "stack-switching")] use super::stack_switching::VMContObj; use crate::prelude::*; -use crate::runtime::store::StoreInstanceId; +use crate::runtime::store::{StoreInstanceId, StoreOpaque}; #[cfg(feature = "gc")] use crate::runtime::vm::VMGcRef; use crate::runtime::vm::table::TableElementType; use crate::runtime::vm::vmcontext::VMFuncRef; use crate::runtime::vm::{ - HostResultHasUnwindSentinel, Instance, TrapReason, VMStore, f32x4, f64x2, i8x16, + self, HostResultHasUnwindSentinel, Instance, SendSyncPtr, TrapReason, VMStore, f32x4, f64x2, + i8x16, }; use core::convert::Infallible; use core::pin::Pin; @@ -166,6 +167,52 @@ pub mod raw { wasmtime_environ::foreach_builtin_function!(libcall); } +/// Uses the `$store` provided to invoke the async closure `$f` and block on the +/// result. +/// +/// This will internally multiplex on `$store.with_blocking(...)` vs simply +/// asserting the closure is ready depending on whether a store's +/// `async_support` flag is set or not. +/// +/// FIXME: ideally this would be a function, not a macro. If this is a function +/// though it would require placing a bound on the async closure $f where the +/// returned future is itself `Send`. That's not possible in Rust right now, +/// unfortunately. +/// +/// As a workaround this takes advantage of the fact that we can assume that the +/// compiler can infer that the future returned by `$f` is indeed `Send` so long +/// as we don't try to name the type or place it behind a generic. In the future +/// when we can bound the return future of async functions with `Send` this +/// macro should be replaced with an equivalent function. +macro_rules! block_on { + ($store:expr, $f:expr) => {{ + let store: &mut StoreOpaque = $store; + let closure = assert_async_fn_closure($f); + if store.async_support() { + #[cfg(feature = "async")] + { + store.with_blocking(|store, cx| cx.block_on(closure(store))) + } + #[cfg(not(feature = "async"))] + { + unreachable!() + } + } else { + // Note that if `async_support` is disabled then it should not be + // possible to introduce await points so the provided future should + // always be ready. + anyhow::Ok(vm::assert_ready(closure(store))) + } + }}; +} + +fn assert_async_fn_closure(f: F) -> F +where + F: AsyncFnOnce(&mut StoreOpaque) -> R, +{ + f +} + fn memory_grow( store: &mut dyn VMStore, mut instance: Pin<&mut Instance>, @@ -235,13 +282,18 @@ unsafe fn table_grow_func_ref( instance.as_mut().table_element_type(table_index), TableElementType::Func, )); - let element = NonNull::new(init_value.cast::()); - let result = instance - .defined_table_grow(defined_table_index, |table| unsafe { - table.grow_func(store, delta, element) - })? - .map(AllocationSize); - Ok(result) + let element = NonNull::new(init_value.cast::()).map(SendSyncPtr::new); + let (mut limiter, store) = store.resource_limiter_and_store_opaque(); + let limiter = limiter.as_mut(); + block_on!(store, async |_store| { + let result = instance + .defined_table_grow(defined_table_index, async |table| unsafe { + table.grow_func(limiter, delta, element).await + }) + .await? + .map(AllocationSize); + Ok(result) + })? } /// Implementation of `table.grow` for GC-reference tables. @@ -261,12 +313,24 @@ unsafe fn table_grow_gc_ref( )); let element = VMGcRef::from_raw_u32(init_value); - let result = instance - .defined_table_grow(defined_table_index, |table| unsafe { - table.grow_gc_ref(store, delta, element.as_ref()) - })? - .map(AllocationSize); - Ok(result) + let (mut limiter, store) = store.resource_limiter_and_store_opaque(); + let limiter = limiter.as_mut(); + block_on!(store, async |store| { + let result = instance + .defined_table_grow(defined_table_index, async |table| unsafe { + table + .grow_gc_ref( + limiter, + store.optional_gc_store_mut(), + delta, + element.as_ref(), + ) + .await + }) + .await? + .map(AllocationSize); + Ok(result) + })? } #[cfg(feature = "stack-switching")] @@ -287,12 +351,17 @@ unsafe fn table_grow_cont_obj( TableElementType::Cont, )); let element = unsafe { VMContObj::from_raw_parts(init_value_contref, init_value_revision) }; - let result = instance - .defined_table_grow(defined_table_index, |table| unsafe { - table.grow_cont(store, delta, element) - })? - .map(AllocationSize); - Ok(result) + let (mut limiter, store) = store.resource_limiter_and_store_opaque(); + let limiter = limiter.as_mut(); + block_on!(store, async |_store| { + let result = instance + .defined_table_grow(defined_table_index, async |table| unsafe { + table.grow_cont(limiter, delta, element).await + }) + .await? + .map(AllocationSize); + Ok(result) + })? } /// Implementation of `table.fill` for `funcref`s. diff --git a/crates/wasmtime/src/runtime/vm/table.rs b/crates/wasmtime/src/runtime/vm/table.rs index f8ad2c4a5898..1405d4d1b7db 100644 --- a/crates/wasmtime/src/runtime/vm/table.rs +++ b/crates/wasmtime/src/runtime/vm/table.rs @@ -3,6 +3,7 @@ //! `Table` is to WebAssembly tables what `LinearMemory` is to WebAssembly linear memories. use crate::prelude::*; +use crate::runtime::store::StoreResourceLimiter; use crate::runtime::vm::stack_switching::VMContObj; use crate::runtime::vm::vmcontext::{VMFuncRef, VMTableDefinition}; use crate::runtime::vm::{GcStore, SendSyncPtr, VMGcRef, VMStore, VmPtr}; @@ -607,46 +608,50 @@ impl Table { /// /// Generally, prefer using `InstanceHandle::table_grow`, which encapsulates /// this unsafety. - pub unsafe fn grow_func( + pub async unsafe fn grow_func( &mut self, - store: &mut dyn VMStore, + limiter: Option<&mut StoreResourceLimiter<'_>>, delta: u64, - init_value: Option>, + init_value: Option>, ) -> Result, Error> { - self._grow(delta, store, |me, _store, base, len| { - me.fill_func(base, init_value, len) + self._grow(delta, limiter, |me, base, len| { + me.fill_func(base, init_value.map(|p| p.as_non_null()), len) }) + .await } /// Same as [`Self::grow_func`], but for GC references. - pub unsafe fn grow_gc_ref( + pub async unsafe fn grow_gc_ref( &mut self, - store: &mut dyn VMStore, + limiter: Option<&mut StoreResourceLimiter<'_>>, + gc_store: Option<&mut GcStore>, delta: u64, init_value: Option<&VMGcRef>, ) -> Result, Error> { - self._grow(delta, store, |me, store, base, len| { - me.fill_gc_ref(store, base, init_value, len) + self._grow(delta, limiter, |me, base, len| { + me.fill_gc_ref(gc_store, base, init_value, len) }) + .await } /// Same as [`Self::grow_func`], but for continuations. - pub unsafe fn grow_cont( + pub async unsafe fn grow_cont( &mut self, - store: &mut dyn VMStore, + limiter: Option<&mut StoreResourceLimiter<'_>>, delta: u64, init_value: Option, ) -> Result, Error> { - self._grow(delta, store, |me, _store, base, len| { + self._grow(delta, limiter, |me, base, len| { me.fill_cont(base, init_value, len) }) + .await } - fn _grow( + async fn _grow( &mut self, delta: u64, - store: &mut dyn VMStore, - fill: impl FnOnce(&mut Self, Option<&mut GcStore>, u64, u64) -> Result<(), Trap>, + mut limiter: Option<&mut StoreResourceLimiter<'_>>, + fill: impl FnOnce(&mut Self, u64, u64) -> Result<(), Trap>, ) -> Result, Error> { let old_size = self.size(); @@ -660,13 +665,21 @@ impl Table { let new_size = match old_size.checked_add(delta) { Some(s) => s, None => { - store.table_grow_failed(format_err!("overflow calculating new table size"))?; + if let Some(limiter) = limiter { + limiter + .table_grow_failed(format_err!("overflow calculating new table size"))?; + } return Ok(None); } }; - if !store.table_growing(old_size, new_size, self.maximum())? { - return Ok(None); + if let Some(limiter) = &mut limiter { + if !limiter + .table_growing(old_size, new_size, self.maximum()) + .await? + { + return Ok(None); + } } // The WebAssembly spec requires failing a `table.grow` request if @@ -674,7 +687,9 @@ impl Table { // limits in the instance allocator as well. if let Some(max) = self.maximum() { if new_size > max { - store.table_grow_failed(format_err!("Table maximum size exceeded"))?; + if let Some(limiter) = limiter { + limiter.table_grow_failed(format_err!("Table maximum size exceeded"))?; + } return Ok(None); } } @@ -720,7 +735,6 @@ impl Table { fill( self, - store.store_opaque_mut().optional_gc_store_mut(), u64::try_from(old_size).unwrap(), u64::try_from(delta).unwrap(), ) From 17098c67d66e784787db3a7d3466dbc78184e345 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Fri, 15 Aug 2025 15:02:54 -0700 Subject: [PATCH 2/2] Remove #[cfg] gate --- crates/wasmtime/src/runtime/fiber.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index a7f805d384dc..7d08a8033d8d 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -393,7 +393,6 @@ impl StoreOpaque { /// # Panics /// /// Panics if this is invoked outside the context of a fiber. - #[cfg(feature = "component-model-async")] pub(crate) fn with_blocking( &mut self, f: impl FnOnce(&mut Self, &mut BlockingContext<'_, '_>) -> R,