From 07bd52fa40008a9b8c93687f1ca89ba963373b45 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Tue, 17 Jun 2025 16:37:51 -0600 Subject: [PATCH 01/30] generalize async fiber abstraction As part of the work implementing the new Component Model async ABI in the `wasip3-prototyping` repo, I've generalized the `FiberFuture` abstraction in `wasmtime::runtime::store::async_` to support fibers which can either retain exclusive access to the store across suspend points or release it. The latter allows the store to be used by the `component-model-async` event loop and/or other fibers to run before the original fiber resumes, which is the key to allowing multiple fibers to run concurrently, passing control of the store back and forth. In the case of Pulley, the above generalization means we also need to give each fiber its own `Interpreter` so that multiple concurrent fibers don't clobber each other's state. Concretely, this moves a lot of the code out of `async_.rs` and into a new `fiber.rs` submodule which will be shared with the `component-model-async` implementation. This also pulls in a new `StoreToken` utility which has been useful in `wasip3-prototyping` to safely convert from a `&mut dyn VMStore` to a `StoreContextMut<'a, T>` when we previously witnessed a conversion in the other direction. Note that I've added a `'static` bound to the `VMStore` trait, which simplifies use of `&mut dyn VMStore`, avoiding thorny lifetime issues. Signed-off-by: Joel Dice --- benches/call.rs | 5 +- crates/wasmtime/Cargo.toml | 2 + crates/wasmtime/src/runtime.rs | 2 + crates/wasmtime/src/runtime/component/func.rs | 2 +- .../src/runtime/component/func/typed.rs | 2 +- .../wasmtime/src/runtime/component/linker.rs | 20 +- crates/wasmtime/src/runtime/fiber.rs | 700 ++++++++++++++++++ crates/wasmtime/src/runtime/func.rs | 26 +- crates/wasmtime/src/runtime/func/typed.rs | 5 +- crates/wasmtime/src/runtime/linker.rs | 25 +- crates/wasmtime/src/runtime/store.rs | 101 ++- crates/wasmtime/src/runtime/store/async_.rs | 546 +------------- crates/wasmtime/src/runtime/store/gc.rs | 9 +- crates/wasmtime/src/runtime/store/token.rs | 36 + crates/wasmtime/src/runtime/vm.rs | 2 +- 15 files changed, 864 insertions(+), 619 deletions(-) create mode 100644 crates/wasmtime/src/runtime/fiber.rs create mode 100644 crates/wasmtime/src/runtime/store/token.rs diff --git a/benches/call.rs b/benches/call.rs index c2028ad7c5f1..c7fc9ecf2f63 100644 --- a/benches/call.rs +++ b/benches/call.rs @@ -135,7 +135,7 @@ fn bench_host_to_wasm( typed_results: Results, ) where Params: WasmParams + ToVals + Copy, - Results: WasmResults + ToVals + Copy + PartialEq + Debug, + Results: WasmResults + ToVals + Copy + PartialEq + Debug + 'static, { // Benchmark the "typed" version, which should be faster than the versions // below. @@ -628,7 +628,8 @@ mod component { + PartialEq + Debug + Send - + Sync, + + Sync + + 'static, { // Benchmark the "typed" version. c.bench_function(&format!("component - host-to-wasm - typed - {name}"), |b| { diff --git a/crates/wasmtime/Cargo.toml b/crates/wasmtime/Cargo.toml index 02188fa8fd91..c7bea7991a26 100644 --- a/crates/wasmtime/Cargo.toml +++ b/crates/wasmtime/Cargo.toml @@ -194,8 +194,10 @@ async = [ "dep:wasmtime-fiber", "dep:async-trait", "dep:trait-variant", + "dep:futures", "wasmtime-component-macro?/async", "runtime", + "futures/std", ] # Enables support for the pooling instance allocation strategy diff --git a/crates/wasmtime/src/runtime.rs b/crates/wasmtime/src/runtime.rs index 230178ce11b9..bc54da96cb67 100644 --- a/crates/wasmtime/src/runtime.rs +++ b/crates/wasmtime/src/runtime.rs @@ -34,6 +34,8 @@ pub(crate) mod code_memory; #[cfg(feature = "debug-builtins")] pub(crate) mod debug; pub(crate) mod externals; +#[cfg(feature = "async")] +pub(crate) mod fiber; pub(crate) mod gc; pub(crate) mod instance; pub(crate) mod instantiate; diff --git a/crates/wasmtime/src/runtime/component/func.rs b/crates/wasmtime/src/runtime/component/func.rs index 12ce0b10d57d..2d021369216d 100644 --- a/crates/wasmtime/src/runtime/component/func.rs +++ b/crates/wasmtime/src/runtime/component/func.rs @@ -547,7 +547,7 @@ impl Func { let mut store = store.as_context_mut(); assert!( store.0.async_support(), - "cannot use `call_async` without enabling async support in the config" + "cannot use `post_return_async` without enabling async support in the config" ); // Future optimization opportunity: conditionally use a fiber here since // some func's post_return will not need the async context (i.e. end up diff --git a/crates/wasmtime/src/runtime/component/func/typed.rs b/crates/wasmtime/src/runtime/component/func/typed.rs index c378742da236..f87447af7a0e 100644 --- a/crates/wasmtime/src/runtime/component/func/typed.rs +++ b/crates/wasmtime/src/runtime/component/func/typed.rs @@ -179,7 +179,7 @@ where ) -> Result where Params: Send + Sync, - Return: Send + Sync, + Return: Send + Sync + 'static, { let mut store = store.as_context_mut(); assert!( diff --git a/crates/wasmtime/src/runtime/component/linker.rs b/crates/wasmtime/src/runtime/component/linker.rs index 8d32f35ee8d5..4bcd1d6c566e 100644 --- a/crates/wasmtime/src/runtime/component/linker.rs +++ b/crates/wasmtime/src/runtime/component/linker.rs @@ -7,6 +7,8 @@ use crate::component::types; use crate::component::{ Component, ComponentNamedList, Instance, InstancePre, Lift, Lower, ResourceType, Val, }; +#[cfg(feature = "async")] +use crate::fiber::AsyncCx; use crate::hash_map::HashMap; use crate::prelude::*; use crate::{AsContextMut, Engine, Module, StoreContextMut}; @@ -442,9 +444,9 @@ impl LinkerInstance<'_, T> { "cannot use `func_wrap_async` without enabling async support in the config" ); let ff = move |mut store: StoreContextMut<'_, T>, params: Params| -> Result { - let async_cx = store.as_context_mut().0.async_cx().expect("async cx"); - let future = f(store.as_context_mut(), params); - unsafe { async_cx.block_on(Pin::from(future)) }? + let async_cx = AsyncCx::new(&mut store.0); + let mut future = Pin::from(f(store.as_context_mut(), params)); + async_cx.block_on(future.as_mut())? }; self.func_wrap(name, ff) } @@ -604,9 +606,9 @@ impl LinkerInstance<'_, T> { "cannot use `func_new_async` without enabling async support in the config" ); let ff = move |mut store: StoreContextMut<'_, T>, params: &[Val], results: &mut [Val]| { - let async_cx = store.as_context_mut().0.async_cx().expect("async cx"); - let future = f(store.as_context_mut(), params, results); - unsafe { async_cx.block_on(Pin::from(future)) }? + let async_cx = AsyncCx::new(&mut store.0); + let mut future = Pin::from(f(store.as_context_mut(), params, results)); + async_cx.block_on(future.as_mut())? }; self.func_new(name, ff) } @@ -676,9 +678,9 @@ impl LinkerInstance<'_, T> { let dtor = Arc::new(crate::func::HostFunc::wrap_inner( &self.engine, move |mut cx: crate::Caller<'_, T>, (param,): (u32,)| { - let async_cx = cx.as_context_mut().0.async_cx().expect("async cx"); - let future = dtor(cx.as_context_mut(), param); - match unsafe { async_cx.block_on(Pin::from(future)) } { + let async_cx = AsyncCx::new(&mut cx.as_context_mut().0); + let mut future = Pin::from(dtor(cx.as_context_mut(), param)); + match async_cx.block_on(future.as_mut()) { Ok(Ok(())) => Ok(()), Ok(Err(trap)) | Err(trap) => Err(trap), } diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs new file mode 100644 index 000000000000..ba34e9e6b6f4 --- /dev/null +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -0,0 +1,700 @@ +#![deny(unsafe_op_in_unsafe_fn)] + +use { + crate::{ + Engine, + store::{Executor, StoreOpaque}, + vm::{ + AsyncWasmCallState, Interpreter, SendSyncPtr, VMStore, + mpk::{self, ProtectionMask}, + }, + }, + anyhow::{Result, anyhow}, + futures::channel::oneshot, + std::{ + future, mem, + ops::Range, + pin::Pin, + ptr::{self, NonNull}, + task::{Context, Poll}, + }, + wasmtime_environ::TripleExt, + wasmtime_fiber::{Fiber, Suspend}, +}; + +/// Helper struct for reseting a raw pointer to its original value on drop. +struct Reset(*mut T, T); + +impl Drop for Reset { + fn drop(&mut self) { + unsafe { + *self.0 = self.1; + } + } +} + +/// Represents the context of a `Future::poll` operation which involves resuming +/// a fiber. +/// +/// See `self::poll_fn` for details. +#[derive(Clone, Copy)] +struct PollContext { + future_context: *mut Context<'static>, + guard_range_start: *mut u8, + guard_range_end: *mut u8, +} + +impl Default for PollContext { + fn default() -> PollContext { + PollContext { + future_context: ptr::null_mut(), + guard_range_start: ptr::null_mut(), + guard_range_end: ptr::null_mut(), + } + } +} + +/// Represents the state of a currently executing fiber which has been resumed +/// via `self::poll_fn`. +pub(crate) struct AsyncState { + /// The `Suspend` for the current fiber (or null if no such fiber is running). + /// + /// See `StoreFiber` for an explanation of the signature types we use here. + current_suspend: *mut Suspend, StoreFiberYield, Result<()>>, + + /// The current Wasm executor. + /// + /// Each fiber has its own executor, and we update this pointer to point to + /// the appropriate one whenever we switch fibers. + pub(crate) current_executor: *mut Executor, + + /// See `PollContext` + current_poll_cx: PollContext, + + /// The last fiber stack that was in use by the store. + /// + /// We use this to cache and reuse stacks as a performance optimization. + // TODO: With stack switching and the Component Model Async ABI, there may + // be multiple concurrent fibers in play; consider caching more than one + // stack at a time and making the number tunable via `Config`. + pub(crate) last_fiber_stack: Option, +} + +impl Default for AsyncState { + fn default() -> Self { + Self { + current_suspend: ptr::null_mut(), + current_executor: ptr::null_mut(), + current_poll_cx: PollContext::default(), + last_fiber_stack: None, + } + } +} + +impl AsyncState { + pub(crate) fn async_guard_range(&self) -> Range<*mut u8> { + let context = self.current_poll_cx; + context.guard_range_start..context.guard_range_end + } +} + +// Lots of pesky unsafe cells and pointers in this structure. This means we need +// to declare explicitly that we use this in a threadsafe fashion. +unsafe impl Send for AsyncState {} +unsafe impl Sync for AsyncState {} + +/// Used to "stackfully" poll a future by suspending the current fiber +/// repeatedly in a loop until the future completes. +pub(crate) struct AsyncCx { + current_suspend: *mut *mut wasmtime_fiber::Suspend, StoreFiberYield, Result<()>>, + current_stack_limit: *mut usize, + current_poll_cx: *mut PollContext, +} + +impl AsyncCx { + /// Create a new `AsyncCx`. + /// + /// This will panic if called outside the scope of a `self::poll_fn` call. + /// Consider using `Self::try_new` instead to avoid panicking. + pub(crate) fn new(store: &mut StoreOpaque) -> Self { + Self::try_new(store).unwrap() + } + + /// Create a new `AsyncCx`. + /// + /// This will return `None` if called outside the scope of a `self::poll_fn` + /// call. + pub(crate) fn try_new(store: &mut StoreOpaque) -> Option { + let current_poll_cx = unsafe { &raw mut (*store.async_state()).current_poll_cx }; + if unsafe { (*current_poll_cx).future_context.is_null() } { + None + } else { + Some(Self { + current_suspend: unsafe { &raw mut (*store.async_state()).current_suspend }, + current_stack_limit: store.vm_store_context().stack_limit.get(), + current_poll_cx, + }) + } + } + + /// Poll the specified future using `Self::current_poll_cx`. + /// + /// This will panic if called recursively using the same `AsyncState`. + fn poll(&self, mut future: Pin<&mut (dyn Future + Send)>) -> Poll { + unsafe { + let poll_cx = *self.current_poll_cx; + let _reset = Reset(self.current_poll_cx, poll_cx); + *self.current_poll_cx = PollContext::default(); + assert!(!poll_cx.future_context.is_null()); + future.as_mut().poll(&mut *poll_cx.future_context) + } + } + + /// Blocks on the asynchronous computation represented by `future` and + /// produces the result here, in-line. + /// + /// This function is designed to only work when it's currently executing on + /// a native fiber. This fiber provides the ability for us to handle the + /// future's `Pending` state as "jump back to whomever called the fiber in + /// an asynchronous fashion and propagate `Pending`". This tight coupling + /// with `on_fiber` below is what powers the asynchronicity of calling wasm. + /// + /// This function takes a `future` and will (appear to) synchronously wait + /// on the result. While this function is executing it will fiber switch + /// to-and-from the original frame calling `on_fiber` which should be a + /// guarantee due to how async stores are configured. + /// + /// The return value here is either the output of the future `T`, or a trap + /// which represents that the asynchronous computation was cancelled. It is + /// not recommended to catch the trap and try to keep executing wasm, so + /// we've tried to liberally document this. + /// + /// Note that this function suspends (if needed) with + /// `StoreFiberYield::KeepStore`, indicating that the store must not be used + /// (and that no other fibers may be resumed) until this fiber resumes. + /// Therefore, it is not appropriate for use in e.g. guest calls to + /// async-lowered imports implemented as host functions, since it will + /// prevent any other tasks from being run. Use `Instance::suspend` to + /// suspend and release the store to allow other tasks to run before this + /// fiber is resumed. + pub(crate) fn block_on( + &self, + mut future: Pin<&mut (dyn Future + Send)>, + ) -> Result { + loop { + match self.poll(future.as_mut()) { + Poll::Ready(v) => break Ok(v), + Poll::Pending => { + self.suspend(StoreFiberYield::KeepStore)?; + } + } + } + } + + /// Suspend the current fiber, optionally transfering exclusive access to + /// the store back to the code which resumed it. + pub(crate) fn suspend(&self, yield_: StoreFiberYield) -> Result<()> { + unsafe { suspend_fiber(self.current_suspend, self.current_stack_limit, yield_) } + } +} + +/// Indicates whether or not a fiber needs to retain exclusive access to its +/// store across a suspend/resume interval. +pub(crate) enum StoreFiberYield { + /// Indicates the fiber needs to retain exclusive access, meaning the store + /// should not be used outside of the fiber until after the fiber either + /// suspends with `ReleaseStore` or resolves. + KeepStore, + /// Indicates the fiber does _not_ need exclusive access across the + /// suspend/resume interval, meaning the store may be used as needed until + /// the fiber is resumed. + // TODO: This will be used once full `component-model-async` support is + // merged: + #[allow(dead_code)] + ReleaseStore, +} + +pub(crate) struct StoreFiber<'a> { + /// The raw `wasmtime_fiber::Fiber`. + /// + /// Note that using `StoreFiberYield` as the `Yield` type parameter allows + /// the fiber to indicate whether it needs exclusive access to the store + /// across suspend points (in which case it will pass `KeepStore` when + /// suspending , meaning the store must not be used at all until the fiber + /// is resumed again) or whether it is giving up exclusive access (in which + /// case it will pass `ReleaseStore` when yielding, meaning exclusive access + /// may be given to another fiber that runs concurrently. + /// + /// Note also that every `StoreFiber` is implicitly granted exclusive access + /// to the store when it is resumed. + pub(crate) fiber: Option, StoreFiberYield, Result<()>>>, + /// See `FiberResumeState` + state: Option, + /// The Wasmtime `Engine` to which this fiber belongs. + engine: Engine, + /// The current `Suspend` for this fiber (or null if it's not currently + /// running). + suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, + executor_ptr: *mut *mut Executor, + executor: Executor, +} + +impl StoreFiber<'_> { + pub(crate) fn guard_range(&self) -> (Option>, Option>) { + self.fiber + .as_ref() + .unwrap() + .stack() + .guard_range() + .map(|r| { + ( + NonNull::new(r.start).map(SendSyncPtr::new), + NonNull::new(r.end).map(SendSyncPtr::new), + ) + }) + .unwrap_or((None, None)) + } +} + +// Here we run the risk of dropping an in-progress fiber, and if we were to do +// nothing then the fiber would leak all its owned stack resources. +// +// To handle this we implement `Drop` here and, if the fiber isn't done, resume +// execution of the fiber saying "hey please stop you're interrupted". Our +// `Trap` created here (which has the stack trace of whomever dropped us) should +// then get propagate all the way back up to the original fiber start, finishing +// execution. +// +// We don't actually care about the fiber's return value here (no one's around +// to look at it), we just assert the fiber finished to completion. +impl Drop for StoreFiber<'_> { + fn drop(&mut self) { + if self.fiber.is_none() { + return; + } + + if !self.fiber.as_ref().unwrap().done() { + // SAFETY: We must temporarily grant the fiber exclusive access to + // its store until resolves, meaning this function must only be + // called from a context where that's sound. As of this writing, + // the only place unresolved fibers are dropped is in + // `ComponentStoreData::drop_fibers` which does in fact have `&mut + // StoreOpaque`. + let result = unsafe { resume_fiber_raw(self, Err(anyhow!("future dropped"))) }; + debug_assert!(result.is_ok()); + } + + self.state.take().unwrap().dispose(); + + unsafe { + self.engine + .allocator() + .deallocate_fiber_stack(self.fiber.take().unwrap().into_stack()); + } + } +} + +// This is surely the most dangerous `unsafe impl Send` in the entire +// crate. There are two members in `FiberFuture` which cause it to not be +// `Send`. One is `current_poll_cx` and is entirely uninteresting. This is just +// used to manage `Context` pointers across `await` points in the future, and +// requires raw pointers to get it to happen easily. Nothing too weird about +// the `Send`-ness, values aren't actually crossing threads. +// +// The really interesting piece is `fiber`. Now the "fiber" here is actual +// honest-to-god Rust code which we're moving around. What we're doing is the +// equivalent of moving our thread's stack to another OS thread. Turns out we, +// in general, have no idea what's on the stack and would generally have no way +// to verify that this is actually safe to do! +// +// Thankfully, though, Wasmtime has the power. Without being glib it's actually +// worth examining what's on the stack. It's unfortunately not super-local to +// this function itself. Our closure to `Fiber::new` runs `func`, which is given +// to us from the outside. Thankfully, though, we have tight control over +// this. Usage of `on_fiber` or `Instance::resume_fiber` is typically done +// *just* before entering WebAssembly itself, so we'll have a few stack frames +// of Rust code (all in Wasmtime itself) before we enter wasm. +// +// Once we've entered wasm, well then we have a whole bunch of wasm frames on +// the stack. We've got this nifty thing called Cranelift, though, which allows +// us to also have complete control over everything on the stack! +// +// Finally, when wasm switches back to the fiber's starting pointer (this future +// we're returning) then it means wasm has reentered Rust. Suspension can only +// happen via either `block_on` or `Instance::suspend`. This, conveniently, also +// happens entirely in Wasmtime controlled code! +// +// There's an extremely important point that should be called out here. +// User-provided futures **are not on the stack** during suspension points. This +// is extremely crucial because we in general cannot reason about Send/Sync for +// stack-local variables since rustc doesn't analyze them at all. With our +// construction, though, we are guaranteed that Wasmtime owns all stack frames +// between the stack of a fiber and when the fiber suspends (and it could move +// across threads). At this time the only user-provided piece of data on the +// stack is the future itself given to us. Lo-and-behold as you might notice the +// future is required to be `Send`! +// +// What this all boils down to is that we, as the authors of Wasmtime, need to +// be extremely careful that on the async fiber stack we only store Send +// things. For example we can't start using `Rc` willy nilly by accident and +// leave a copy in TLS somewhere. (similarly we have to be ready for TLS to +// change while we're executing wasm code between suspension points). +// +// While somewhat onerous it shouldn't be too too hard (the TLS bit is the +// hardest bit so far). This does mean, though, that no user should ever have to +// worry about the `Send`-ness of Wasmtime. If rustc says it's ok, then it's ok. +// +// With all that in mind we unsafely assert here that Wasmtime is correct. We +// declare the fiber as only containing Send data on its stack, despite not +// knowing for sure at compile time that this is correct. That's what `unsafe` +// in Rust is all about, though, right? +unsafe impl Send for StoreFiber<'_> {} +// See the docs about the `Send` impl above, which also apply to this `Sync` +// impl. `Sync` is needed since we store `StoreFiber`s and switch between them +// when executing components that export async-lifted functions. +unsafe impl Sync for StoreFiber<'_> {} + +/// State of the world when a fiber last suspended. +/// +/// This structure represents global state that a fiber clobbers during its +/// execution. For example TLS variables are updated, system resources like MPK +/// masks are updated, etc. The purpose of this structure is to track all of +/// this state and appropriately save/restore it around fiber suspension points. +struct FiberResumeState { + /// Saved list of `CallThreadState` activations that are stored on a fiber + /// stack. + /// + /// This is a linked list that references stack-stored nodes on the fiber + /// stack that is currently suspended. The `AsyncWasmCallState` type + /// documents this more thoroughly but the general gist is that when we this + /// fiber is resumed this linked list needs to be pushed on to the current + /// thread's linked list of activations. + tls: crate::runtime::vm::AsyncWasmCallState, + + /// Saved MPK protection mask, if enabled. + /// + /// When MPK is enabled then executing WebAssembly will modify the + /// processor's current mask of addressable protection keys. This means that + /// our current state may get clobbered when a fiber suspends. To ensure + /// that this function preserves context it will, when MPK is enabled, save + /// the current mask when this function is called and then restore the mask + /// when the function returns (aka the fiber suspends). + mpk: Option, +} + +impl FiberResumeState { + unsafe fn replace(self) -> PriorFiberResumeState { + let tls = unsafe { self.tls.push() }; + let mpk = swap_mpk_states(self.mpk); + PriorFiberResumeState { tls, mpk } + } + + fn dispose(self) { + self.tls.assert_null(); + } +} + +struct PriorFiberResumeState { + tls: crate::runtime::vm::PreviousAsyncWasmCallState, + mpk: Option, +} + +impl PriorFiberResumeState { + unsafe fn replace(self) -> FiberResumeState { + let tls = unsafe { self.tls.restore() }; + let mpk = swap_mpk_states(self.mpk); + FiberResumeState { tls, mpk } + } +} + +fn swap_mpk_states(mask: Option) -> Option { + mask.map(|mask| { + let current = mpk::current_mask(); + mpk::allow(mask); + current + }) +} + +/// Resume the specified fiber, granting it exclusive access to the store with +/// which it was created. +/// +/// This will return `Ok(result)` if the fiber resolved, where `result` is the +/// returned value; it will return `Err(yield_)` if the fiber suspended, where +/// `yield_` indicates whether it released access to the store or not. See +/// `StoreFiber::fiber` for details. +/// +/// SAFETY: The caller must confer exclusive access to the store to the fiber +/// until the fiber is either dropped, resolved, or forgotten, or until it +/// releases the store when suspending. +unsafe fn resume_fiber_raw<'a>( + fiber: &mut StoreFiber<'a>, + result: Result<()>, +) -> Result, StoreFiberYield> { + struct Restore<'a, 'b> { + fiber: &'b mut StoreFiber<'a>, + state: Option, + } + + impl Drop for Restore<'_, '_> { + fn drop(&mut self) { + unsafe { + self.fiber.state = Some(self.state.take().unwrap().replace()); + } + } + } + unsafe { + let _reset_executor = Reset(fiber.executor_ptr, *fiber.executor_ptr); + *fiber.executor_ptr = &raw mut fiber.executor; + let _reset_suspend = Reset(fiber.suspend, *fiber.suspend); + let prev = fiber.state.take().unwrap().replace(); + let restore = Restore { + fiber, + state: Some(prev), + }; + restore.fiber.fiber.as_ref().unwrap().resume(result) + } +} + +/// Create a new `StoreFiber` which runs the specified closure. +pub(crate) fn make_fiber<'a>( + store: &mut dyn VMStore, + fun: impl FnOnce(&mut dyn VMStore) -> Result<()> + 'a, +) -> Result> { + let engine = store.engine().clone(); + #[cfg(has_host_compiler_backend)] + let executor = if cfg!(feature = "pulley") && engine.target().is_pulley() { + Executor::Interpreter(Interpreter::new(&engine)) + } else { + Executor::Native + }; + #[cfg(not(has_host_compiler_backend))] + let executor = { + debug_assert!(engine.target().is_pulley()); + Executor::Interpreter(Interpreter::new(&engine)) + }; + let stack = store.store_opaque_mut().allocate_fiber_stack()?; + let suspend = unsafe { &raw mut (*store.store_opaque_mut().async_state()).current_suspend }; + let executor_ptr = + unsafe { &raw mut (*store.store_opaque_mut().async_state()).current_executor }; + let track_pkey_context_switch = store.has_pkey(); + let store = &raw mut *store; + Ok(StoreFiber { + fiber: Some(Fiber::new(stack, move |result: Result<()>, suspend| { + if result.is_err() { + result + } else { + // SAFETY: Per the documented contract for + // `resume_fiber_raw`, we've been given exclusive access to + // the store until we exit or yield it back to the resumer. + let store_ref = unsafe { &mut *store }; + let suspend_ptr = unsafe { + &raw mut (*store_ref.store_opaque_mut().async_state()).current_suspend + }; + // Configure our store's suspension context for the rest of the + // execution of this fiber. Note that a raw pointer is stored here + // which is only valid for the duration of this closure. + // Consequently we at least replace it with the previous value when + // we're done. This reset is also required for correctness because + // otherwise our value will overwrite another active fiber's value. + // There should be a test that segfaults in `async_functions.rs` if + // this `Reset` is removed. + // + // SAFETY: The resumer is responsible for setting + // `current_suspend` to a valid pointer. + let _reset = Reset(suspend_ptr, unsafe { *suspend_ptr }); + unsafe { *suspend_ptr = suspend }; + fun(store_ref) + } + })?), + state: Some(FiberResumeState { + tls: crate::runtime::vm::AsyncWasmCallState::new(), + mpk: if track_pkey_context_switch { + Some(ProtectionMask::all()) + } else { + None + }, + }), + engine, + suspend, + executor_ptr, + executor, + }) +} + +/// See `resume_fiber_raw` +pub(crate) unsafe fn resume_fiber( + fiber: &mut StoreFiber, + result: Result<()>, +) -> Result, StoreFiberYield> { + match unsafe { resume_fiber_raw(fiber, result) } { + Ok(result) => Ok(result), + Err(yield_) => { + // If `Err` is returned that means the fiber suspended, so we + // propagate that here. + // + // An additional safety check is performed when leaving this + // function to help bolster the guarantees of `unsafe impl Send` + // above. Notably this future may get re-polled on a different + // thread. Wasmtime's thread-local state points to the stack, + // however, meaning that it would be incorrect to leave a pointer in + // TLS when this function returns. This function performs a runtime + // assert to verify that this is the case, notably that the one TLS + // pointer Wasmtime uses is not pointing anywhere within the + // stack. If it is then that's a bug indicating that TLS management + // in Wasmtime is incorrect. + if let Some(range) = fiber.fiber.as_ref().unwrap().stack().range() { + AsyncWasmCallState::assert_current_state_not_in_range(range); + } + + Err(yield_) + } + } +} + +/// Suspend the current fiber, optionally returning exclusive access to the +/// specified store to the code which resumed the fiber. +/// +/// SAFETY: `suspend` must be a valid pointer. Additionally, if a store pointer +/// is provided, the fiber must give up access to the store until it is given +/// back access when next resumed. +unsafe fn suspend_fiber( + suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, + stack_limit: *mut usize, + yield_: StoreFiberYield, +) -> Result<()> { + // Take our current `Suspend` context which was configured as soon as our + // fiber started. Note that we must load it at the front here and save it on + // our stack frame. While we're polling the future other fibers may be + // started for recursive computations, and the current suspend context is + // only preserved at the edges of the fiber, not during the fiber itself. + // + // For a little bit of extra safety we also replace the current value with + // null to try to catch any accidental bugs on our part early. This is all + // pretty unsafe so we're trying to be careful... + // + // Note that there should be a segfaulting test in `async_functions.rs` if + // this `Reset` is removed. + unsafe { + let reset_suspend = Reset(suspend, *suspend); + *suspend = ptr::null_mut(); + let _reset_stack_limit = Reset(stack_limit, *stack_limit); + assert!(!(reset_suspend.1).is_null()); + (*reset_suspend.1).suspend(yield_) + } +} + +/// Run the specified function on a newly-created fiber and `.await` its +/// completion. +pub(crate) async fn on_fiber( + store: &mut StoreOpaque, + func: impl FnOnce(&mut StoreOpaque) -> R + Send, +) -> Result { + on_fiber_raw(store.traitobj_mut(), move |store| { + func((*store).store_opaque_mut()) + }) + .await +} + +/// Wrap the specified function in a fiber and return it. +fn prepare_fiber<'a, R: Send + 'static>( + store: &mut dyn VMStore, + func: impl FnOnce(&mut dyn VMStore) -> R + Send + 'a, +) -> Result<(StoreFiber<'a>, oneshot::Receiver)> { + let (tx, rx) = oneshot::channel(); + let fiber = make_fiber(store, { + move |store| { + _ = tx.send(func(store)); + Ok(()) + } + })?; + Ok((fiber, rx)) +} + +/// Run the specified function on a newly-created fiber and `.await` its +/// completion. +async fn on_fiber_raw( + store: &mut StoreOpaque, + func: impl FnOnce(&mut dyn VMStore) -> R + Send, +) -> Result { + let config = store.engine().config(); + debug_assert!(store.async_support()); + debug_assert!(config.async_stack_size > 0); + + let (fiber, mut rx) = prepare_fiber(store.traitobj_mut(), func)?; + + let guard_range = fiber.guard_range(); + let mut fiber = Some(fiber); + let mut fiber = poll_fn(store, guard_range, move || { + // SAFETY: We confer exclusive access to the store to the fiber here, + // only taking it back when the fiber resolves. + match unsafe { resume_fiber(fiber.as_mut().unwrap(), Ok(())) } { + Ok(result) => Poll::Ready(result.map(|()| fiber.take().unwrap())), + Err(_) => Poll::Pending, + } + }) + .await?; + + let stack = fiber.fiber.take().map(|f| f.into_stack()); + drop(fiber); + if let Some(stack) = stack { + store.deallocate_fiber_stack(stack); + } + + Ok(rx.try_recv().unwrap().unwrap()) +} + +/// Wrap the specified function in a future which, when polled, will store a +/// pointer to the `Context` in the `AsyncState::current_poll_cx` field for the +/// specified store and then call the function. +/// +/// This is intended for use with functions that resume fibers which may need to +/// poll futures using the stored `Context` pointer. +pub(crate) async fn poll_fn( + store: &mut StoreOpaque, + guard_range: (Option>, Option>), + mut fun: impl FnMut() -> Poll, +) -> R { + #[derive(Clone, Copy)] + struct PollCx(*mut PollContext); + + unsafe impl Send for PollCx {} + + let poll_cx = PollCx(unsafe { &raw mut (*store.async_state()).current_poll_cx }); + future::poll_fn({ + move |cx| { + let _reset = Reset(poll_cx.0, unsafe { *poll_cx.0 }); + let guard_range_start = guard_range.0.map(|v| v.as_ptr()).unwrap_or(ptr::null_mut()); + let guard_range_end = guard_range.1.map(|v| v.as_ptr()).unwrap_or(ptr::null_mut()); + // We need to carry over this `cx` into our fiber's runtime for when + // it tries to poll sub-futures that are created. Doing this must be + // done unsafely, however, since `cx` is only alive for this one + // singular function call. Here we do a `transmute` to extend the + // lifetime of `Context` so it can be stored in our `Store`, and + // then we replace the current polling context with this one. + // + // Note that the replace is done for weird situations where futures + // might be switching contexts and there's multiple wasmtime futures + // in a chain of futures. + // + // On exit from this function, though, we reset the polling context + // back to what it was to signify that `Store` no longer has access + // to this pointer. + // + // SAFETY: We store the pointer to the `Context` only for the + // duration of this call and then reset it to its previous value + // afterward, thereby ensuring `fun` never sees a stale pointer. + unsafe { + *poll_cx.0 = PollContext { + future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), + guard_range_start, + guard_range_end, + }; + } + #[allow(dropping_copy_types)] + drop(poll_cx); + + fun() + } + }) + .await +} diff --git a/crates/wasmtime/src/runtime/func.rs b/crates/wasmtime/src/runtime/func.rs index ad71706ff10f..49859fec3e34 100644 --- a/crates/wasmtime/src/runtime/func.rs +++ b/crates/wasmtime/src/runtime/func.rs @@ -1,3 +1,5 @@ +#[cfg(feature = "async")] +use crate::fiber::AsyncCx; use crate::prelude::*; use crate::runtime::Uninhabited; use crate::runtime::vm::{ @@ -514,14 +516,9 @@ impl Func { ); assert!(ty.comes_from_same_engine(store.as_context().engine())); Func::new(store, ty, move |mut caller, params, results| { - let async_cx = caller - .store - .as_context_mut() - .0 - .async_cx() - .expect("Attempt to spawn new action on dying fiber"); - let future = func(caller, params, results); - match unsafe { async_cx.block_on(Pin::from(future)) } { + let async_cx = AsyncCx::new(&mut caller.store.0); + let mut future = Pin::from(func(caller, params, results)); + match async_cx.block_on(future.as_mut()) { Ok(Ok(())) => Ok(()), Ok(Err(trap)) | Err(trap) => Err(trap), } @@ -840,15 +837,10 @@ impl Func { concat!("cannot use `wrap_async` without enabling async support on the config") ); Func::wrap_inner(store, move |mut caller: Caller<'_, T>, args| { - let async_cx = caller - .store - .as_context_mut() - .0 - .async_cx() - .expect("Attempt to start async function on dying fiber"); - let future = func(caller, args); - - match unsafe { async_cx.block_on(Pin::from(future)) } { + let async_cx = AsyncCx::new(&mut caller.store.0); + let mut future = Pin::from(func(caller, args)); + + match async_cx.block_on(future.as_mut()) { Ok(ret) => ret.into_fallible(), Err(e) => R::fallible_from_error(e), } diff --git a/crates/wasmtime/src/runtime/func/typed.rs b/crates/wasmtime/src/runtime/func/typed.rs index 0df0dcc400c2..e9304b75515b 100644 --- a/crates/wasmtime/src/runtime/func/typed.rs +++ b/crates/wasmtime/src/runtime/func/typed.rs @@ -132,7 +132,10 @@ where &self, mut store: impl AsContextMut, params: Params, - ) -> Result { + ) -> Result + where + Results: 'static, + { let mut store = store.as_context_mut(); assert!( store.0.async_support(), diff --git a/crates/wasmtime/src/runtime/linker.rs b/crates/wasmtime/src/runtime/linker.rs index cb49fa2d727c..389011b446c2 100644 --- a/crates/wasmtime/src/runtime/linker.rs +++ b/crates/wasmtime/src/runtime/linker.rs @@ -1,3 +1,5 @@ +#[cfg(feature = "async")] +use crate::fiber::AsyncCx; use crate::func::HostFunc; use crate::hash_map::{Entry, HashMap}; use crate::instance::InstancePre; @@ -471,14 +473,9 @@ impl Linker { ); assert!(ty.comes_from_same_engine(self.engine())); self.func_new(module, name, ty, move |mut caller, params, results| { - let async_cx = caller - .store - .as_context_mut() - .0 - .async_cx() - .expect("Attempt to spawn new function on dying fiber"); - let future = func(caller, params, results); - match unsafe { async_cx.block_on(Pin::from(future)) } { + let async_cx = AsyncCx::new(&mut caller.store.0); + let mut future = Pin::from(func(caller, params, results)); + match async_cx.block_on(future.as_mut()) { Ok(Ok(())) => Ok(()), Ok(Err(trap)) | Err(trap) => Err(trap), } @@ -578,14 +575,10 @@ impl Linker { let func = HostFunc::wrap_inner( &self.engine, move |mut caller: Caller<'_, T>, args: Params| { - let async_cx = caller - .store - .as_context_mut() - .0 - .async_cx() - .expect("Attempt to start async function on dying fiber"); - let future = func(caller, args); - match unsafe { async_cx.block_on(Pin::from(future)) } { + let async_cx = AsyncCx::new(&mut caller.store.0); + let mut future = Pin::from(func(caller, args)); + + match async_cx.block_on(future.as_mut()) { Ok(ret) => ret.into_fallible(), Err(e) => Args::fallible_from_error(e), } diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index 018042159af6..bcbc630d61ea 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -77,6 +77,8 @@ //! `wasmtime`, must uphold for the public interface to be safe. use crate::RootSet; +#[cfg(feature = "async")] +use crate::fiber::{self, AsyncCx}; use crate::module::RegisteredModuleId; use crate::prelude::*; #[cfg(feature = "gc")] @@ -109,11 +111,13 @@ pub use self::data::*; mod func_refs; use func_refs::FuncRefs; #[cfg(feature = "async")] +mod token; +#[cfg(feature = "async")] +pub(crate) use token::StoreToken; +#[cfg(feature = "async")] mod async_; #[cfg(all(feature = "async", feature = "call-hook"))] pub use self::async_::CallHookHandler; -#[cfg(feature = "async")] -use self::async_::*; #[cfg(feature = "gc")] mod gc; @@ -353,7 +357,7 @@ pub struct StoreOpaque { table_count: usize, table_limit: usize, #[cfg(feature = "async")] - async_state: AsyncState, + async_state: fiber::AsyncState, // If fuel_yield_interval is enabled, then we store the remaining fuel (that isn't in // runtime_limits) here. The total amount of fuel is the runtime limits and reserve added @@ -401,7 +405,7 @@ pub struct StoreOpaque { /// /// Effectively stores Pulley interpreter state and handles conditional support /// for Cranelift at compile time. -enum Executor { +pub(crate) enum Executor { Interpreter(Interpreter), #[cfg(has_host_compiler_backend)] Native, @@ -553,7 +557,7 @@ impl Store { table_count: 0, table_limit: crate::DEFAULT_TABLE_LIMIT, #[cfg(feature = "async")] - async_state: AsyncState::default(), + async_state: Default::default(), fuel_reserve: 0, fuel_yield_interval: None, store_data, @@ -589,6 +593,11 @@ impl Store { data: ManuallyDrop::new(data), }); + #[cfg(feature = "async")] + { + inner.async_state.current_executor = &raw mut inner.executor; + } + inner.traitobj = StorePtr::new(NonNull::from(&mut *inner)); // Wasmtime uses the callee argument to host functions to learn about @@ -1094,16 +1103,13 @@ impl StoreInner { CallHookInner::Sync(hook) => hook((&mut *self).as_context_mut(), s), #[cfg(all(feature = "async", feature = "call-hook"))] - CallHookInner::Async(handler) => unsafe { - self.inner - .async_cx() - .ok_or_else(|| anyhow!("couldn't grab async_cx for call hook"))? - .block_on( - handler - .handle_call_event((&mut *self).as_context_mut(), s) - .as_mut(), - )? - }, + CallHookInner::Async(handler) => AsyncCx::try_new(&mut self.inner) + .ok_or_else(|| anyhow!("couldn't grab async_cx for call hook"))? + .block_on( + handler + .handle_call_event((&mut *self).as_context_mut(), s) + .as_mut(), + )?, CallHookInner::ForceTypeParameterToBeUsed { uninhabited, .. } => { let _ = s; @@ -1926,13 +1932,28 @@ at https://bytecodealliance.org/security. ) } + #[cfg(feature = "async")] + pub(crate) fn async_state(&mut self) -> *mut fiber::AsyncState { + &raw mut self.async_state + } + + #[cfg(feature = "async")] + pub(crate) fn has_pkey(&self) -> bool { + self.pkey.is_some() + } + #[cfg(not(feature = "async"))] pub(crate) fn async_guard_range(&self) -> core::ops::Range<*mut u8> { core::ptr::null_mut()..core::ptr::null_mut() } pub(crate) fn executor(&mut self) -> ExecutorRef<'_> { - match &mut self.executor { + #[cfg(feature = "async")] + let executor = unsafe { &mut *self.async_state.current_executor }; + #[cfg(not(feature = "async"))] + let executor = &mut self.executor; + + match executor { Executor::Interpreter(i) => ExecutorRef::Interpreter(i.as_interpreter_ref()), #[cfg(has_host_compiler_backend)] Executor::Native => ExecutorRef::Native, @@ -1940,7 +1961,12 @@ at https://bytecodealliance.org/security. } pub(crate) fn unwinder(&self) -> &'static dyn Unwind { - match &self.executor { + #[cfg(feature = "async")] + let executor = unsafe { &*self.async_state.current_executor }; + #[cfg(not(feature = "async"))] + let executor = &self.executor; + + match executor { Executor::Interpreter(i) => i.unwinder(), #[cfg(has_host_compiler_backend)] Executor::Native => &vm::UnwindHost, @@ -2094,16 +2120,13 @@ unsafe impl vm::VMStore for StoreInner { limiter(&mut self.data).memory_growing(current, desired, maximum) } #[cfg(feature = "async")] - Some(ResourceLimiterInner::Async(ref mut limiter)) => unsafe { - self.inner - .async_cx() - .expect("ResourceLimiterAsync requires async Store") - .block_on( - limiter(&mut self.data) - .memory_growing(current, desired, maximum) - .as_mut(), - )? - }, + Some(ResourceLimiterInner::Async(ref mut limiter)) => AsyncCx::try_new(&mut self.inner) + .expect("ResourceLimiterAsync requires async Store") + .block_on( + limiter(&mut self.data) + .memory_growing(current, desired, maximum) + .as_mut(), + )?, None => Ok(true), } } @@ -2137,7 +2160,7 @@ unsafe impl vm::VMStore for StoreInner { let async_cx = if self.async_support() && matches!(self.limiter, Some(ResourceLimiterInner::Async(_))) { - Some(self.async_cx().unwrap()) + AsyncCx::try_new(&mut self.inner) } else { None }; @@ -2147,11 +2170,13 @@ unsafe impl vm::VMStore for StoreInner { limiter(&mut self.data).table_growing(current, desired, maximum) } #[cfg(feature = "async")] - Some(ResourceLimiterInner::Async(ref mut limiter)) => unsafe { - async_cx - .expect("ResourceLimiterAsync requires async Store") - .block_on(limiter(&mut self.data).table_growing(current, desired, maximum))? - }, + Some(ResourceLimiterInner::Async(ref mut limiter)) => async_cx + .expect("ResourceLimiterAsync requires async Store") + .block_on( + limiter(&mut self.data) + .table_growing(current, desired, maximum) + .as_mut(), + )?, None => Ok(true), } } @@ -2205,7 +2230,7 @@ unsafe impl vm::VMStore for StoreInner { delta } #[cfg(feature = "async")] - UpdateDeadline::YieldCustom(delta, future) => { + UpdateDeadline::YieldCustom(delta, mut future) => { assert!( self.async_support(), "cannot use `UpdateDeadline::YieldCustom` without enabling async support in the config" @@ -2218,11 +2243,9 @@ unsafe impl vm::VMStore for StoreInner { // to clean up this fiber. Do so by raising a trap which will // abort all wasm and get caught on the other side to clean // things up. - unsafe { - self.async_cx() - .expect("attempted to pull async context during shutdown") - .block_on(future)? - } + AsyncCx::try_new(self) + .expect("attempted to pull async context during shutdown") + .block_on(future.as_mut())?; delta } }; diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index 08551b7ea47c..c34d1dac482c 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -1,15 +1,11 @@ #[cfg(feature = "call-hook")] use crate::CallHook; +use crate::fiber::{self, AsyncCx}; use crate::prelude::*; -use crate::runtime::vm::mpk::{self, ProtectionMask}; -use crate::store::{ResourceLimiterInner, StoreInner, StoreOpaque}; -use crate::{Engine, Store, StoreContextMut, UpdateDeadline}; -use core::cell::UnsafeCell; -use core::future::Future; +use crate::store::{ResourceLimiterInner, StoreInner, StoreOpaque, StoreToken}; +use crate::{AsContextMut, Store, StoreContextMut, UpdateDeadline}; use core::ops::Range; -use core::pin::{Pin, pin}; -use core::ptr; -use core::task::{Context, Poll}; +use core::pin::Pin; /// An object that can take callbacks when the runtime enters or exits hostcalls. #[cfg(feature = "call-hook")] @@ -20,45 +16,6 @@ pub trait CallHookHandler: Send { async fn handle_call_event(&self, t: StoreContextMut<'_, T>, ch: CallHook) -> Result<()>; } -pub struct AsyncState { - current_suspend: UnsafeCell<*mut wasmtime_fiber::Suspend, (), Result<()>>>, - current_poll_cx: UnsafeCell, - /// The last fiber stack that was in use by this store. - last_fiber_stack: Option, -} - -impl Default for AsyncState { - fn default() -> AsyncState { - AsyncState { - current_suspend: UnsafeCell::new(ptr::null_mut()), - current_poll_cx: UnsafeCell::new(PollContext::default()), - last_fiber_stack: None, - } - } -} - -// Lots of pesky unsafe cells and pointers in this structure. This means we need -// to declare explicitly that we use this in a threadsafe fashion. -unsafe impl Send for AsyncState {} -unsafe impl Sync for AsyncState {} - -#[derive(Clone, Copy)] -struct PollContext { - future_context: *mut Context<'static>, - guard_range_start: *mut u8, - guard_range_end: *mut u8, -} - -impl Default for PollContext { - fn default() -> PollContext { - PollContext { - future_context: core::ptr::null_mut(), - guard_range_start: core::ptr::null_mut(), - guard_range_end: core::ptr::null_mut(), - } - } -} - impl Store { /// Configures the [`ResourceLimiterAsync`](crate::ResourceLimiterAsync) /// used to limit resource creation within this [`Store`]. @@ -214,368 +171,11 @@ impl StoreOpaque { /// This function will convert the synchronous `func` into an asynchronous /// future. This is done by running `func` in a fiber on a separate native /// stack which can be suspended and resumed from. - /// - /// Most of the nitty-gritty here is how we juggle the various contexts - /// necessary to suspend the fiber later on and poll sub-futures. It's hoped - /// that the various comments are illuminating as to what's going on here. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut Self) -> R + Send, ) -> Result { - let config = self.engine().config(); - debug_assert!(self.async_support()); - debug_assert!(config.async_stack_size > 0); - - let mut slot = None; - let mut future = { - let current_poll_cx = self.async_state.current_poll_cx.get(); - let current_suspend = self.async_state.current_suspend.get(); - let stack = self.allocate_fiber_stack()?; - let track_pkey_context_switch = self.pkey.is_some(); - - let engine = self.engine().clone(); - let slot = &mut slot; - let this = &mut *self; - let fiber = wasmtime_fiber::Fiber::new(stack, move |keep_going, suspend| { - // First check and see if we were interrupted/dropped, and only - // continue if we haven't been. - keep_going?; - - // Configure our store's suspension context for the rest of the - // execution of this fiber. Note that a raw pointer is stored here - // which is only valid for the duration of this closure. - // Consequently we at least replace it with the previous value when - // we're done. This reset is also required for correctness because - // otherwise our value will overwrite another active fiber's value. - // There should be a test that segfaults in `async_functions.rs` if - // this `Replace` is removed. - unsafe { - let _reset = Reset(current_suspend, *current_suspend); - *current_suspend = suspend; - - *slot = Some(func(this)); - Ok(()) - } - })?; - - // Once we have the fiber representing our synchronous computation, we - // wrap that in a custom future implementation which does the - // translation from the future protocol to our fiber API. - FiberFuture { - fiber: Some(fiber), - current_poll_cx, - engine, - fiber_resume_state: Some(FiberResumeState { - tls: crate::runtime::vm::AsyncWasmCallState::new(), - mpk: if track_pkey_context_switch { - Some(ProtectionMask::all()) - } else { - None - }, - }), - } - }; - (&mut future).await?; - let stack = future.fiber.take().map(|f| f.into_stack()); - drop(future); - if let Some(stack) = stack { - self.deallocate_fiber_stack(stack); - } - - return Ok(slot.unwrap()); - - struct FiberFuture<'a> { - fiber: Option, (), Result<()>>>, - current_poll_cx: *mut PollContext, - engine: Engine, - // See comments in `FiberResumeState` for this - fiber_resume_state: Option, - } - - // This is surely the most dangerous `unsafe impl Send` in the entire - // crate. There are two members in `FiberFuture` which cause it to not - // be `Send`. One is `current_poll_cx` and is entirely uninteresting. - // This is just used to manage `Context` pointers across `await` points - // in the future, and requires raw pointers to get it to happen easily. - // Nothing too weird about the `Send`-ness, values aren't actually - // crossing threads. - // - // The really interesting piece is `fiber`. Now the "fiber" here is - // actual honest-to-god Rust code which we're moving around. What we're - // doing is the equivalent of moving our thread's stack to another OS - // thread. Turns out we, in general, have no idea what's on the stack - // and would generally have no way to verify that this is actually safe - // to do! - // - // Thankfully, though, Wasmtime has the power. Without being glib it's - // actually worth examining what's on the stack. It's unfortunately not - // super-local to this function itself. Our closure to `Fiber::new` runs - // `func`, which is given to us from the outside. Thankfully, though, we - // have tight control over this. Usage of `on_fiber` is typically done - // *just* before entering WebAssembly itself, so we'll have a few stack - // frames of Rust code (all in Wasmtime itself) before we enter wasm. - // - // Once we've entered wasm, well then we have a whole bunch of wasm - // frames on the stack. We've got this nifty thing called Cranelift, - // though, which allows us to also have complete control over everything - // on the stack! - // - // Finally, when wasm switches back to the fiber's starting pointer - // (this future we're returning) then it means wasm has reentered Rust. - // Suspension can only happen via the `block_on` function of an - // `AsyncCx`. This, conveniently, also happens entirely in Wasmtime - // controlled code! - // - // There's an extremely important point that should be called out here. - // User-provided futures **are not on the stack** during suspension - // points. This is extremely crucial because we in general cannot reason - // about Send/Sync for stack-local variables since rustc doesn't analyze - // them at all. With our construction, though, we are guaranteed that - // Wasmtime owns all stack frames between the stack of a fiber and when - // the fiber suspends (and it could move across threads). At this time - // the only user-provided piece of data on the stack is the future - // itself given to us. Lo-and-behold as you might notice the future is - // required to be `Send`! - // - // What this all boils down to is that we, as the authors of Wasmtime, - // need to be extremely careful that on the async fiber stack we only - // store Send things. For example we can't start using `Rc` willy nilly - // by accident and leave a copy in TLS somewhere. (similarly we have to - // be ready for TLS to change while we're executing wasm code between - // suspension points). - // - // While somewhat onerous it shouldn't be too too hard (the TLS bit is - // the hardest bit so far). This does mean, though, that no user should - // ever have to worry about the `Send`-ness of Wasmtime. If rustc says - // it's ok, then it's ok. - // - // With all that in mind we unsafely assert here that wasmtime is - // correct. We declare the fiber as only containing Send data on its - // stack, despite not knowing for sure at compile time that this is - // correct. That's what `unsafe` in Rust is all about, though, right? - unsafe impl Send for FiberFuture<'_> {} - - impl FiberFuture<'_> { - fn fiber(&self) -> &wasmtime_fiber::Fiber<'_, Result<()>, (), Result<()>> { - self.fiber.as_ref().unwrap() - } - - /// This is a helper function to call `resume` on the underlying - /// fiber while correctly managing Wasmtime's state that the fiber - /// may clobber. - /// - /// ## Return Value - /// - /// * `Ok(Ok(()))` - the fiber successfully completed and yielded a - /// successful result. - /// * `Ok(Err(e))` - the fiber successfully completed and yielded - /// an error as a result of computation. - /// * `Err(())` - the fiber has not finished and it is suspended. - fn resume(&mut self, val: Result<()>) -> Result, ()> { - unsafe { - let prev = self.fiber_resume_state.take().unwrap().replace(); - let restore = Restore { - fiber: self, - prior_fiber_state: Some(prev), - }; - return restore.fiber.fiber().resume(val); - } - - struct Restore<'a, 'b> { - fiber: &'a mut FiberFuture<'b>, - prior_fiber_state: Option, - } - - impl Drop for Restore<'_, '_> { - fn drop(&mut self) { - unsafe { - self.fiber.fiber_resume_state = - Some(self.prior_fiber_state.take().unwrap().replace()); - } - } - } - } - } - - impl Future for FiberFuture<'_> { - type Output = Result<()>; - - fn poll(mut self: Pin<&mut Self>, cx: &mut Context) -> Poll { - // We need to carry over this `cx` into our fiber's runtime - // for when it tries to poll sub-futures that are created. Doing - // this must be done unsafely, however, since `cx` is only alive - // for this one singular function call. Here we do a `transmute` - // to extend the lifetime of `Context` so it can be stored in - // our `Store`, and then we replace the current polling context - // with this one. - // - // Note that the replace is done for weird situations where - // futures might be switching contexts and there's multiple - // wasmtime futures in a chain of futures. - // - // On exit from this function, though, we reset the polling - // context back to what it was to signify that `Store` no longer - // has access to this pointer. - let guard = self - .fiber() - .stack() - .guard_range() - .unwrap_or(core::ptr::null_mut()..core::ptr::null_mut()); - unsafe { - let _reset = Reset(self.current_poll_cx, *self.current_poll_cx); - *self.current_poll_cx = PollContext { - future_context: core::mem::transmute::< - &mut Context<'_>, - *mut Context<'static>, - >(cx), - guard_range_start: guard.start, - guard_range_end: guard.end, - }; - - // After that's set up we resume execution of the fiber, which - // may also start the fiber for the first time. This either - // returns `Ok` saying the fiber finished (yay!) or it - // returns `Err` with the payload passed to `suspend`, which - // in our case is `()`. - match self.resume(Ok(())) { - Ok(result) => Poll::Ready(result), - - // If `Err` is returned that means the fiber polled a - // future but it said "Pending", so we propagate that - // here. - // - // An additional safety check is performed when leaving - // this function to help bolster the guarantees of - // `unsafe impl Send` above. Notably this future may get - // re-polled on a different thread. Wasmtime's - // thread-local state points to the stack, however, - // meaning that it would be incorrect to leave a pointer - // in TLS when this function returns. This function - // performs a runtime assert to verify that this is the - // case, notably that the one TLS pointer Wasmtime uses - // is not pointing anywhere within the stack. If it is - // then that's a bug indicating that TLS management in - // Wasmtime is incorrect. - Err(()) => { - if let Some(range) = self.fiber().stack().range() { - crate::runtime::vm::AsyncWasmCallState::assert_current_state_not_in_range(range); - } - Poll::Pending - } - } - } - } - } - - // Dropping futures is pretty special in that it means the future has - // been requested to be cancelled. Here we run the risk of dropping an - // in-progress fiber, and if we were to do nothing then the fiber would - // leak all its owned stack resources. - // - // To handle this we implement `Drop` here and, if the fiber isn't done, - // resume execution of the fiber saying "hey please stop you're - // interrupted". Our `Trap` created here (which has the stack trace - // of whomever dropped us) will then get propagated in whatever called - // `block_on`, and the idea is that the trap propagates all the way back - // up to the original fiber start, finishing execution. - // - // We don't actually care about the fiber's return value here (no one's - // around to look at it), we just assert the fiber finished to - // completion. - impl Drop for FiberFuture<'_> { - fn drop(&mut self) { - if self.fiber.is_none() { - return; - } - - if !self.fiber().done() { - let result = self.resume(Err(anyhow!("future dropped"))); - // This resumption with an error should always complete the - // fiber. While it's technically possible for host code to - // catch the trap and re-resume, we'd ideally like to - // signal that to callers that they shouldn't be doing - // that. - debug_assert!(result.is_ok()); - - // Note that `result` is `Ok(r)` where `r` is either - // `Ok(())` or `Err(e)`. If it's an error that's disposed of - // here. It's expected to be a propagation of the `future - // dropped` error created above. - } - - self.fiber_resume_state.take().unwrap().dispose(); - - unsafe { - self.engine - .allocator() - .deallocate_fiber_stack(self.fiber.take().unwrap().into_stack()); - } - } - } - - /// State of the world when a fiber last suspended. - /// - /// This structure represents global state that a fiber clobbers during - /// its execution. For example TLS variables are updated, system - /// resources like MPK masks are updated, etc. The purpose of this - /// structure is to track all of this state and appropriately - /// save/restore it around fiber suspension points. - struct FiberResumeState { - /// Saved list of `CallThreadState` activations that are stored on a - /// fiber stack. - /// - /// This is a linked list that references stack-stored nodes on the - /// fiber stack that is currently suspended. The - /// `AsyncWasmCallState` type documents this more thoroughly but the - /// general gist is that when we this fiber is resumed this linked - /// list needs to be pushed on to the current thread's linked list - /// of activations. - tls: crate::runtime::vm::AsyncWasmCallState, - - /// Saved MPK protection mask, if enabled. - /// - /// When MPK is enabled then executing WebAssembly will modify the - /// processor's current mask of addressable protection keys. This - /// means that our current state may get clobbered when a fiber - /// suspends. To ensure that this function preserves context it - /// will, when MPK is enabled, save the current mask when this - /// function is called and then restore the mask when the function - /// returns (aka the fiber suspends). - mpk: Option, - } - - impl FiberResumeState { - unsafe fn replace(self) -> PriorFiberResumeState { - let tls = self.tls.push(); - let mpk = swap_mpk_states(self.mpk); - PriorFiberResumeState { tls, mpk } - } - - fn dispose(self) { - self.tls.assert_null(); - } - } - - struct PriorFiberResumeState { - tls: crate::runtime::vm::PreviousAsyncWasmCallState, - mpk: Option, - } - - impl PriorFiberResumeState { - unsafe fn replace(self) -> FiberResumeState { - let tls = self.tls.restore(); - let mpk = swap_mpk_states(self.mpk); - FiberResumeState { tls, mpk } - } - } - - fn swap_mpk_states(mask: Option) -> Option { - mask.map(|mask| { - let current = mpk::current_mask(); - mpk::allow(mask); - current - }) - } + fiber::on_fiber(self, func).await } #[cfg(feature = "gc")] @@ -636,30 +236,6 @@ impl StoreOpaque { log::trace!("End trace GC roots") } - /// Yields the async context, assuming that we are executing on a fiber and - /// that fiber is not in the process of dying. This function will return - /// None in the latter case (the fiber is dying), and panic if - /// `async_support()` is false. - #[inline] - pub fn async_cx(&self) -> Option { - assert!(self.async_support()); - - let poll_cx_box_ptr = self.async_state.current_poll_cx.get(); - if poll_cx_box_ptr.is_null() { - return None; - } - - let poll_cx_inner_ptr = unsafe { *poll_cx_box_ptr }; - if poll_cx_inner_ptr.future_context.is_null() { - return None; - } - - Some(AsyncCx { - current_suspend: self.async_state.current_suspend.get(), - current_poll_cx: unsafe { &raw mut (*poll_cx_box_ptr).future_context }, - }) - } - /// Yields execution to the caller on out-of-gas or epoch interruption. /// /// This only works on async futures and stores, and assumes that we're @@ -676,21 +252,23 @@ impl StoreOpaque { // to clean up this fiber. Do so by raising a trap which will // abort all wasm and get caught on the other side to clean // things up. - unsafe { - self.async_cx() - .expect("attempted to pull async context during shutdown") - .block_on(Pin::new_unchecked(&mut future)) - } + AsyncCx::try_new(self) + .expect("attempted to pull async context during shutdown") + .block_on(unsafe { Pin::new_unchecked(&mut future) }) + } + + pub(crate) fn async_guard_range(&mut self) -> Range<*mut u8> { + unsafe { (*self.async_state()).async_guard_range() } } - fn allocate_fiber_stack(&mut self) -> Result { + pub(crate) fn allocate_fiber_stack(&mut self) -> Result { if let Some(stack) = self.async_state.last_fiber_stack.take() { return Ok(stack); } self.engine().allocator().allocate_fiber_stack() } - fn deallocate_fiber_stack(&mut self, stack: wasmtime_fiber::FiberStack) { + pub(crate) fn deallocate_fiber_stack(&mut self, stack: wasmtime_fiber::FiberStack) { self.flush_fiber_stack(); self.async_state.last_fiber_stack = Some(stack); } @@ -704,108 +282,20 @@ impl StoreOpaque { } } } - - pub(crate) fn async_guard_range(&self) -> Range<*mut u8> { - unsafe { - let ptr = self.async_state.current_poll_cx.get(); - (*ptr).guard_range_start..(*ptr).guard_range_end - } - } } impl StoreContextMut<'_, T> { /// Executes a synchronous computation `func` asynchronously on a new fiber. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut StoreContextMut<'_, T>) -> R + Send, ) -> Result where T: Send + 'static, { + let token = StoreToken::new(self.as_context_mut()); self.0 - .on_fiber(|opaque| { - let store = unsafe { opaque.traitobj().cast::>().as_mut() }; - func(&mut StoreContextMut(store)) - }) + .on_fiber(|opaque| func(&mut token.as_context_mut(opaque.traitobj_mut()))) .await } } - -pub struct AsyncCx { - current_suspend: *mut *mut wasmtime_fiber::Suspend, (), Result<()>>, - current_poll_cx: *mut *mut Context<'static>, -} - -impl AsyncCx { - /// Blocks on the asynchronous computation represented by `future` and - /// produces the result here, in-line. - /// - /// This function is designed to only work when it's currently executing on - /// a native fiber. This fiber provides the ability for us to handle the - /// future's `Pending` state as "jump back to whomever called the fiber in - /// an asynchronous fashion and propagate `Pending`". This tight coupling - /// with `on_fiber` below is what powers the asynchronicity of calling wasm. - /// Note that the asynchronous part only applies to host functions, wasm - /// itself never really does anything asynchronous at this time. - /// - /// This function takes a `future` and will (appear to) synchronously wait - /// on the result. While this function is executing it will fiber switch - /// to-and-from the original frame calling `on_fiber` which should be a - /// guarantee due to how async stores are configured. - /// - /// The return value here is either the output of the future `T`, or a trap - /// which represents that the asynchronous computation was cancelled. It is - /// not recommended to catch the trap and try to keep executing wasm, so - /// we've tried to liberally document this. - pub unsafe fn block_on(&self, future: F) -> Result - where - F: Future + Send, - { - let mut future = pin!(future); - - // Take our current `Suspend` context which was configured as soon as - // our fiber started. Note that we must load it at the front here and - // save it on our stack frame. While we're polling the future other - // fibers may be started for recursive computations, and the current - // suspend context is only preserved at the edges of the fiber, not - // during the fiber itself. - // - // For a little bit of extra safety we also replace the current value - // with null to try to catch any accidental bugs on our part early. - // This is all pretty unsafe so we're trying to be careful... - // - // Note that there should be a segfaulting test in `async_functions.rs` - // if this `Reset` is removed. - let suspend = *self.current_suspend; - let _reset = Reset(self.current_suspend, suspend); - *self.current_suspend = ptr::null_mut(); - assert!(!suspend.is_null()); - - loop { - let future_result = { - let poll_cx = *self.current_poll_cx; - let _reset = Reset(self.current_poll_cx, poll_cx); - *self.current_poll_cx = ptr::null_mut(); - assert!(!poll_cx.is_null()); - future.as_mut().poll(&mut *poll_cx) - }; - - match future_result { - Poll::Ready(t) => break Ok(t), - Poll::Pending => {} - } - - (*suspend).suspend(())?; - } - } -} - -struct Reset(*mut T, T); - -impl Drop for Reset { - fn drop(&mut self) { - unsafe { - *self.0 = self.1; - } - } -} diff --git a/crates/wasmtime/src/runtime/store/gc.rs b/crates/wasmtime/src/runtime/store/gc.rs index 3171aa420532..f81ca34f0b53 100644 --- a/crates/wasmtime/src/runtime/store/gc.rs +++ b/crates/wasmtime/src/runtime/store/gc.rs @@ -2,6 +2,8 @@ use super::*; use crate::GcHeapOutOfMemory; +#[cfg(feature = "async")] +use crate::fiber::AsyncCx; impl StoreOpaque { /// Collect garbage, potentially growing the GC heap. @@ -40,10 +42,9 @@ impl StoreOpaque { if scope.async_support() { #[cfg(feature = "async")] { - scope - .async_cx() - .expect("attempted to access async context during shutdown") - .block_on(scope.grow_or_collect_gc_heap_async(bytes_needed))?; + let async_cx = AsyncCx::new(&mut scope); + let future = scope.grow_or_collect_gc_heap_async(bytes_needed); + async_cx.block_on(Box::pin(future).as_mut())?; } } else { scope.grow_or_collect_gc_heap(bytes_needed); diff --git a/crates/wasmtime/src/runtime/store/token.rs b/crates/wasmtime/src/runtime/store/token.rs new file mode 100644 index 000000000000..0bfb269632ed --- /dev/null +++ b/crates/wasmtime/src/runtime/store/token.rs @@ -0,0 +1,36 @@ +use crate::runtime::vm::VMStore; +use crate::{StoreContextMut, store::StoreId}; +use core::marker::PhantomData; + +pub struct StoreToken { + id: StoreId, + _phantom: PhantomData T>, +} + +impl Clone for StoreToken { + fn clone(&self) -> Self { + Self { + id: self.id, + _phantom: PhantomData, + } + } +} + +impl Copy for StoreToken {} + +impl StoreToken { + pub fn new(store: StoreContextMut) -> Self { + Self { + id: store.0.id(), + _phantom: PhantomData, + } + } + + pub fn as_context_mut<'a>(&self, store: &'a mut dyn VMStore) -> StoreContextMut<'a, T> { + assert_eq!(store.store_opaque().id(), self.id); + // We know the store with this ID has data type parameter `T` because + // we witnessed that in `Self::new`, which is the only way `self` could + // have been safely created: + unsafe { store.unchecked_context_mut::() } + } +} diff --git a/crates/wasmtime/src/runtime/vm.rs b/crates/wasmtime/src/runtime/vm.rs index 2761114298fa..e32167a76b0f 100644 --- a/crates/wasmtime/src/runtime/vm.rs +++ b/crates/wasmtime/src/runtime/vm.rs @@ -182,7 +182,7 @@ cfg_if::cfg_if! { /// APIs using `Store` are correctly inferring send/sync on the returned /// values (e.g. futures) and that internally in the runtime we aren't doing /// anything "weird" with threads for example. -pub unsafe trait VMStore { +pub unsafe trait VMStore: 'static { /// Get a shared borrow of this store's `StoreOpaque`. fn store_opaque(&self) -> &StoreOpaque; From 66ce1cd266e271aa075eb6d591cce0c05e6e9149 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Tue, 24 Jun 2025 11:38:03 -0600 Subject: [PATCH 02/30] address review feedback Signed-off-by: Joel Dice --- crates/wasmtime/Cargo.toml | 1 - .../src/runtime/component/func/typed.rs | 2 +- crates/wasmtime/src/runtime/fiber.rs | 41 ++++++++----------- crates/wasmtime/src/runtime/func/typed.rs | 5 +-- crates/wasmtime/src/runtime/linker.rs | 6 +-- crates/wasmtime/src/runtime/store/async_.rs | 4 +- crates/wasmtime/src/runtime/store/token.rs | 20 +++++++-- 7 files changed, 39 insertions(+), 40 deletions(-) diff --git a/crates/wasmtime/Cargo.toml b/crates/wasmtime/Cargo.toml index c7bea7991a26..f7b420e2db2d 100644 --- a/crates/wasmtime/Cargo.toml +++ b/crates/wasmtime/Cargo.toml @@ -197,7 +197,6 @@ async = [ "dep:futures", "wasmtime-component-macro?/async", "runtime", - "futures/std", ] # Enables support for the pooling instance allocation strategy diff --git a/crates/wasmtime/src/runtime/component/func/typed.rs b/crates/wasmtime/src/runtime/component/func/typed.rs index f87447af7a0e..c378742da236 100644 --- a/crates/wasmtime/src/runtime/component/func/typed.rs +++ b/crates/wasmtime/src/runtime/component/func/typed.rs @@ -179,7 +179,7 @@ where ) -> Result where Params: Send + Sync, - Return: Send + Sync + 'static, + Return: Send + Sync, { let mut store = store.as_context_mut(); assert!( diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index ba34e9e6b6f4..e5b6999bea42 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -1,26 +1,19 @@ #![deny(unsafe_op_in_unsafe_fn)] -use { - crate::{ - Engine, - store::{Executor, StoreOpaque}, - vm::{ - AsyncWasmCallState, Interpreter, SendSyncPtr, VMStore, - mpk::{self, ProtectionMask}, - }, - }, - anyhow::{Result, anyhow}, - futures::channel::oneshot, - std::{ - future, mem, - ops::Range, - pin::Pin, - ptr::{self, NonNull}, - task::{Context, Poll}, - }, - wasmtime_environ::TripleExt, - wasmtime_fiber::{Fiber, Suspend}, -}; +use crate::Engine; +use crate::store::{Executor, StoreOpaque}; +use crate::vm::mpk::{self, ProtectionMask}; +use crate::vm::{AsyncWasmCallState, Interpreter, SendSyncPtr, VMStore}; +use anyhow::{Result, anyhow}; +use core::future; +use core::mem; +use core::ops::Range; +use core::pin::Pin; +use core::ptr::{self, NonNull}; +use core::task::{Context, Poll}; +use futures::channel::oneshot; +use wasmtime_environ::TripleExt; +use wasmtime_fiber::{Fiber, Suspend}; /// Helper struct for reseting a raw pointer to its original value on drop. struct Reset(*mut T, T); @@ -585,7 +578,7 @@ unsafe fn suspend_fiber( /// Run the specified function on a newly-created fiber and `.await` its /// completion. -pub(crate) async fn on_fiber( +pub(crate) async fn on_fiber( store: &mut StoreOpaque, func: impl FnOnce(&mut StoreOpaque) -> R + Send, ) -> Result { @@ -596,7 +589,7 @@ pub(crate) async fn on_fiber( } /// Wrap the specified function in a fiber and return it. -fn prepare_fiber<'a, R: Send + 'static>( +fn prepare_fiber<'a, R: Send + 'a>( store: &mut dyn VMStore, func: impl FnOnce(&mut dyn VMStore) -> R + Send + 'a, ) -> Result<(StoreFiber<'a>, oneshot::Receiver)> { @@ -612,7 +605,7 @@ fn prepare_fiber<'a, R: Send + 'static>( /// Run the specified function on a newly-created fiber and `.await` its /// completion. -async fn on_fiber_raw( +async fn on_fiber_raw( store: &mut StoreOpaque, func: impl FnOnce(&mut dyn VMStore) -> R + Send, ) -> Result { diff --git a/crates/wasmtime/src/runtime/func/typed.rs b/crates/wasmtime/src/runtime/func/typed.rs index e9304b75515b..0df0dcc400c2 100644 --- a/crates/wasmtime/src/runtime/func/typed.rs +++ b/crates/wasmtime/src/runtime/func/typed.rs @@ -132,10 +132,7 @@ where &self, mut store: impl AsContextMut, params: Params, - ) -> Result - where - Results: 'static, - { + ) -> Result { let mut store = store.as_context_mut(); assert!( store.0.async_support(), diff --git a/crates/wasmtime/src/runtime/linker.rs b/crates/wasmtime/src/runtime/linker.rs index 389011b446c2..36fadb89dc4e 100644 --- a/crates/wasmtime/src/runtime/linker.rs +++ b/crates/wasmtime/src/runtime/linker.rs @@ -1,5 +1,3 @@ -#[cfg(feature = "async")] -use crate::fiber::AsyncCx; use crate::func::HostFunc; use crate::hash_map::{Entry, HashMap}; use crate::instance::InstancePre; @@ -473,7 +471,7 @@ impl Linker { ); assert!(ty.comes_from_same_engine(self.engine())); self.func_new(module, name, ty, move |mut caller, params, results| { - let async_cx = AsyncCx::new(&mut caller.store.0); + let async_cx = crate::fiber::AsyncCx::new(&mut caller.store.0); let mut future = Pin::from(func(caller, params, results)); match async_cx.block_on(future.as_mut()) { Ok(Ok(())) => Ok(()), @@ -575,7 +573,7 @@ impl Linker { let func = HostFunc::wrap_inner( &self.engine, move |mut caller: Caller<'_, T>, args: Params| { - let async_cx = AsyncCx::new(&mut caller.store.0); + let async_cx = crate::fiber::AsyncCx::new(&mut caller.store.0); let mut future = Pin::from(func(caller, args)); match async_cx.block_on(future.as_mut()) { diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index c34d1dac482c..f1c8a7454800 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -171,7 +171,7 @@ impl StoreOpaque { /// This function will convert the synchronous `func` into an asynchronous /// future. This is done by running `func` in a fiber on a separate native /// stack which can be suspended and resumed from. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut Self) -> R + Send, ) -> Result { @@ -286,7 +286,7 @@ impl StoreOpaque { impl StoreContextMut<'_, T> { /// Executes a synchronous computation `func` asynchronously on a new fiber. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut StoreContextMut<'_, T>) -> R + Send, ) -> Result diff --git a/crates/wasmtime/src/runtime/store/token.rs b/crates/wasmtime/src/runtime/store/token.rs index 0bfb269632ed..b5891da02ac2 100644 --- a/crates/wasmtime/src/runtime/store/token.rs +++ b/crates/wasmtime/src/runtime/store/token.rs @@ -2,6 +2,12 @@ use crate::runtime::vm::VMStore; use crate::{StoreContextMut, store::StoreId}; use core::marker::PhantomData; +/// Represents a proof that a store with a given `StoreId` has a data type +/// parameter `T`. +/// +/// This may be used to safely convert a `&mut dyn VMStore` into a +/// `StoreContextMut` using `StoreToken::as_context_mut` having witnessed +/// that the type parameter matches what was seen earlier in `StoreToken::new`. pub struct StoreToken { id: StoreId, _phantom: PhantomData T>, @@ -18,7 +24,9 @@ impl Clone for StoreToken { impl Copy for StoreToken {} -impl StoreToken { +impl StoreToken { + /// Create a new `StoreToken`, witnessing that this store has data type + /// parameter `T`. pub fn new(store: StoreContextMut) -> Self { Self { id: store.0.id(), @@ -26,11 +34,15 @@ impl StoreToken { } } + /// Convert the specified `&mut dyn VMStore` into a `StoreContextMut`. + /// + /// This will panic if passed a store with a different `StoreId` than the + /// one passed to `StoreToken::new` when creating this object. pub fn as_context_mut<'a>(&self, store: &'a mut dyn VMStore) -> StoreContextMut<'a, T> { assert_eq!(store.store_opaque().id(), self.id); - // We know the store with this ID has data type parameter `T` because - // we witnessed that in `Self::new`, which is the only way `self` could - // have been safely created: + // SAFETY: We know the store with this ID has data type parameter `T` + // because we witnessed that in `Self::new`, which is the only way + // `self` could have been safely created: unsafe { store.unchecked_context_mut::() } } } From ae227b658643761ebb3603a428879a826e4f0762 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Tue, 24 Jun 2025 12:20:12 -0600 Subject: [PATCH 03/30] fix miri-flagged stacked borrow violation As part of my earlier effort to unify the fiber abstractions in the `wasmtime` crate, I changed a `*mut StoreFiber` field to a `&mut StoreFiber`, not realizing that it resulted in a mutable alias at runtime and thus undefined behavior. Miri caught it, fortunately. Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/fiber.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index e5b6999bea42..cb6b242b38b6 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -423,15 +423,15 @@ unsafe fn resume_fiber_raw<'a>( fiber: &mut StoreFiber<'a>, result: Result<()>, ) -> Result, StoreFiberYield> { - struct Restore<'a, 'b> { - fiber: &'b mut StoreFiber<'a>, + struct Restore<'a> { + fiber: *mut StoreFiber<'a>, state: Option, } - impl Drop for Restore<'_, '_> { + impl Drop for Restore<'_> { fn drop(&mut self) { unsafe { - self.fiber.state = Some(self.state.take().unwrap().replace()); + (*self.fiber).state = Some(self.state.take().unwrap().replace()); } } } @@ -441,10 +441,10 @@ unsafe fn resume_fiber_raw<'a>( let _reset_suspend = Reset(fiber.suspend, *fiber.suspend); let prev = fiber.state.take().unwrap().replace(); let restore = Restore { - fiber, + fiber: &raw mut *fiber, state: Some(prev), }; - restore.fiber.fiber.as_ref().unwrap().resume(result) + (*restore.fiber).fiber.as_ref().unwrap().resume(result) } } From 8b8ecb14d2bf54c23181d4eff3aadfeb8f64912c Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Tue, 24 Jun 2025 12:49:42 -0600 Subject: [PATCH 04/30] remove unneeded `Send` bounds Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/component/func/typed.rs | 2 +- crates/wasmtime/src/runtime/fiber.rs | 6 +++--- crates/wasmtime/src/runtime/store/async_.rs | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/crates/wasmtime/src/runtime/component/func/typed.rs b/crates/wasmtime/src/runtime/component/func/typed.rs index c378742da236..68c6819fbcab 100644 --- a/crates/wasmtime/src/runtime/component/func/typed.rs +++ b/crates/wasmtime/src/runtime/component/func/typed.rs @@ -179,7 +179,7 @@ where ) -> Result where Params: Send + Sync, - Return: Send + Sync, + Return: Sync, { let mut store = store.as_context_mut(); assert!( diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index cb6b242b38b6..63a003ef637b 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -578,7 +578,7 @@ unsafe fn suspend_fiber( /// Run the specified function on a newly-created fiber and `.await` its /// completion. -pub(crate) async fn on_fiber( +pub(crate) async fn on_fiber( store: &mut StoreOpaque, func: impl FnOnce(&mut StoreOpaque) -> R + Send, ) -> Result { @@ -589,7 +589,7 @@ pub(crate) async fn on_fiber( } /// Wrap the specified function in a fiber and return it. -fn prepare_fiber<'a, R: Send + 'a>( +fn prepare_fiber<'a, R: 'a>( store: &mut dyn VMStore, func: impl FnOnce(&mut dyn VMStore) -> R + Send + 'a, ) -> Result<(StoreFiber<'a>, oneshot::Receiver)> { @@ -605,7 +605,7 @@ fn prepare_fiber<'a, R: Send + 'a>( /// Run the specified function on a newly-created fiber and `.await` its /// completion. -async fn on_fiber_raw( +async fn on_fiber_raw( store: &mut StoreOpaque, func: impl FnOnce(&mut dyn VMStore) -> R + Send, ) -> Result { diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index f1c8a7454800..e8f75d17ff34 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -171,7 +171,7 @@ impl StoreOpaque { /// This function will convert the synchronous `func` into an asynchronous /// future. This is done by running `func` in a fiber on a separate native /// stack which can be suspended and resumed from. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut Self) -> R + Send, ) -> Result { @@ -286,7 +286,7 @@ impl StoreOpaque { impl StoreContextMut<'_, T> { /// Executes a synchronous computation `func` asynchronously on a new fiber. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut StoreContextMut<'_, T>) -> R + Send, ) -> Result From bd20d142a1d13f92f2d0be7f63c2aa6f2a498510 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Tue, 24 Jun 2025 16:24:27 -0600 Subject: [PATCH 05/30] address more review feedback Main changes: - Make `resume_fiber[_raw]` take a `&mut StoreOpaque` parameter to make its unsafe internals easier to reason about, safety-wise. - Panic if `StoreFiber::drop` is called on an in-progress fiber without having called `StoreFiber::dispose` to gracefully end it first. - (Re)introduce `FiberFuture`, which closes over a `&mut StoreOpaque` and uses it to call `StoreFiber::dispose` on drop. This will require a few more changes to make it usable by `concurrent.rs`, but I'll save those changes for a later PR. Signed-off-by: Joel Dice --- .../wasmtime/src/runtime/component/linker.rs | 12 +- crates/wasmtime/src/runtime/fiber.rs | 257 +++++++++--------- crates/wasmtime/src/runtime/func.rs | 9 +- crates/wasmtime/src/runtime/linker.rs | 9 +- crates/wasmtime/src/runtime/store.rs | 78 +++--- crates/wasmtime/src/runtime/store/async_.rs | 14 +- 6 files changed, 190 insertions(+), 189 deletions(-) diff --git a/crates/wasmtime/src/runtime/component/linker.rs b/crates/wasmtime/src/runtime/component/linker.rs index 4bcd1d6c566e..09241f8bd32b 100644 --- a/crates/wasmtime/src/runtime/component/linker.rs +++ b/crates/wasmtime/src/runtime/component/linker.rs @@ -446,7 +446,9 @@ impl LinkerInstance<'_, T> { let ff = move |mut store: StoreContextMut<'_, T>, params: Params| -> Result { let async_cx = AsyncCx::new(&mut store.0); let mut future = Pin::from(f(store.as_context_mut(), params)); - async_cx.block_on(future.as_mut())? + // SAFETY: The `Store` we used to create the `AsyncCx` above remains + // valid. + unsafe { async_cx.block_on(future.as_mut())? } }; self.func_wrap(name, ff) } @@ -608,7 +610,9 @@ impl LinkerInstance<'_, T> { let ff = move |mut store: StoreContextMut<'_, T>, params: &[Val], results: &mut [Val]| { let async_cx = AsyncCx::new(&mut store.0); let mut future = Pin::from(f(store.as_context_mut(), params, results)); - async_cx.block_on(future.as_mut())? + // SAFETY: The `Store` we used to create the `AsyncCx` above remains + // valid. + unsafe { async_cx.block_on(future.as_mut())? } }; self.func_new(name, ff) } @@ -680,7 +684,9 @@ impl LinkerInstance<'_, T> { move |mut cx: crate::Caller<'_, T>, (param,): (u32,)| { let async_cx = AsyncCx::new(&mut cx.as_context_mut().0); let mut future = Pin::from(dtor(cx.as_context_mut(), param)); - match async_cx.block_on(future.as_mut()) { + // SAFETY: The `Store` we used to create the `AsyncCx` above + // remains valid. + match unsafe { async_cx.block_on(future.as_mut()) } { Ok(Ok(())) => Ok(()), Ok(Err(trap)) | Err(trap) => Err(trap), } diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 63a003ef637b..9ab7724f4152 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -1,15 +1,14 @@ #![deny(unsafe_op_in_unsafe_fn)] use crate::Engine; -use crate::store::{Executor, StoreOpaque}; +use crate::store::{Executor, StoreId, StoreOpaque}; use crate::vm::mpk::{self, ProtectionMask}; -use crate::vm::{AsyncWasmCallState, Interpreter, SendSyncPtr, VMStore}; +use crate::vm::{AsyncWasmCallState, Interpreter, VMStore}; use anyhow::{Result, anyhow}; -use core::future; use core::mem; use core::ops::Range; use core::pin::Pin; -use core::ptr::{self, NonNull}; +use core::ptr; use core::task::{Context, Poll}; use futures::channel::oneshot; use wasmtime_environ::TripleExt; @@ -55,12 +54,6 @@ pub(crate) struct AsyncState { /// See `StoreFiber` for an explanation of the signature types we use here. current_suspend: *mut Suspend, StoreFiberYield, Result<()>>, - /// The current Wasm executor. - /// - /// Each fiber has its own executor, and we update this pointer to point to - /// the appropriate one whenever we switch fibers. - pub(crate) current_executor: *mut Executor, - /// See `PollContext` current_poll_cx: PollContext, @@ -70,14 +63,13 @@ pub(crate) struct AsyncState { // TODO: With stack switching and the Component Model Async ABI, there may // be multiple concurrent fibers in play; consider caching more than one // stack at a time and making the number tunable via `Config`. - pub(crate) last_fiber_stack: Option, + last_fiber_stack: Option, } impl Default for AsyncState { fn default() -> Self { Self { current_suspend: ptr::null_mut(), - current_executor: ptr::null_mut(), current_poll_cx: PollContext::default(), last_fiber_stack: None, } @@ -89,6 +81,10 @@ impl AsyncState { let context = self.current_poll_cx; context.guard_range_start..context.guard_range_end } + + pub(crate) fn last_fiber_stack(&mut self) -> &mut Option { + &mut self.last_fiber_stack + } } // Lots of pesky unsafe cells and pointers in this structure. This means we need @@ -133,7 +129,10 @@ impl AsyncCx { /// Poll the specified future using `Self::current_poll_cx`. /// /// This will panic if called recursively using the same `AsyncState`. - fn poll(&self, mut future: Pin<&mut (dyn Future + Send)>) -> Poll { + /// + /// SAFETY: `self` contains pointers into the `Store` with which it was + /// created and must not be used after that `Store` has been disposed. + unsafe fn poll(&self, mut future: Pin<&mut (dyn Future + Send)>) -> Poll { unsafe { let poll_cx = *self.current_poll_cx; let _reset = Reset(self.current_poll_cx, poll_cx); @@ -170,23 +169,27 @@ impl AsyncCx { /// prevent any other tasks from being run. Use `Instance::suspend` to /// suspend and release the store to allow other tasks to run before this /// fiber is resumed. - pub(crate) fn block_on( + /// + /// SAFETY: `self` contains pointers into the `Store` with which it was + /// created and must not be used after that `Store` has been disposed. + pub(crate) unsafe fn block_on( &self, mut future: Pin<&mut (dyn Future + Send)>, ) -> Result { loop { - match self.poll(future.as_mut()) { + match unsafe { self.poll(future.as_mut()) } { Poll::Ready(v) => break Ok(v), - Poll::Pending => { - self.suspend(StoreFiberYield::KeepStore)?; - } + Poll::Pending => unsafe { self.suspend(StoreFiberYield::KeepStore)? }, } } } /// Suspend the current fiber, optionally transfering exclusive access to /// the store back to the code which resumed it. - pub(crate) fn suspend(&self, yield_: StoreFiberYield) -> Result<()> { + /// + /// SAFETY: `self` contains pointers into the `Store` with which it was + /// created and must not be used after that `Store` has been disposed. + pub(crate) unsafe fn suspend(&self, yield_: StoreFiberYield) -> Result<()> { unsafe { suspend_fiber(self.current_suspend, self.current_stack_limit, yield_) } } } @@ -220,7 +223,7 @@ pub(crate) struct StoreFiber<'a> { /// /// Note also that every `StoreFiber` is implicitly granted exclusive access /// to the store when it is resumed. - pub(crate) fiber: Option, StoreFiberYield, Result<()>>>, + fiber: Option, StoreFiberYield, Result<()>>>, /// See `FiberResumeState` state: Option, /// The Wasmtime `Engine` to which this fiber belongs. @@ -228,54 +231,34 @@ pub(crate) struct StoreFiber<'a> { /// The current `Suspend` for this fiber (or null if it's not currently /// running). suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, - executor_ptr: *mut *mut Executor, executor: Executor, + id: StoreId, } impl StoreFiber<'_> { - pub(crate) fn guard_range(&self) -> (Option>, Option>) { - self.fiber - .as_ref() - .unwrap() - .stack() - .guard_range() - .map(|r| { - ( - NonNull::new(r.start).map(SendSyncPtr::new), - NonNull::new(r.end).map(SendSyncPtr::new), - ) - }) - .unwrap_or((None, None)) + fn dispose(&mut self, store: &mut StoreOpaque) { + if !self.fiber.as_ref().unwrap().done() { + let result = resume_fiber_raw(store, self, Err(anyhow!("future dropped"))); + debug_assert!(result.is_ok()); + } } } -// Here we run the risk of dropping an in-progress fiber, and if we were to do -// nothing then the fiber would leak all its owned stack resources. -// -// To handle this we implement `Drop` here and, if the fiber isn't done, resume -// execution of the fiber saying "hey please stop you're interrupted". Our -// `Trap` created here (which has the stack trace of whomever dropped us) should -// then get propagate all the way back up to the original fiber start, finishing -// execution. -// -// We don't actually care about the fiber's return value here (no one's around -// to look at it), we just assert the fiber finished to completion. +// Note that this implementation will panic if the fiber is in-progress, which +// will abort the process if there is already a panic being unwound. That +// should only happen if we failed to call `StoreFiber::dispose` on the +// in-progress fiber prior to dropping it, which indicates a bug in this crate +// which must be fixed. impl Drop for StoreFiber<'_> { fn drop(&mut self) { if self.fiber.is_none() { return; } - if !self.fiber.as_ref().unwrap().done() { - // SAFETY: We must temporarily grant the fiber exclusive access to - // its store until resolves, meaning this function must only be - // called from a context where that's sound. As of this writing, - // the only place unresolved fibers are dropped is in - // `ComponentStoreData::drop_fibers` which does in fact have `&mut - // StoreOpaque`. - let result = unsafe { resume_fiber_raw(self, Err(anyhow!("future dropped"))) }; - debug_assert!(result.is_ok()); - } + assert!( + self.fiber.as_ref().unwrap().done(), + "attempted to drop in-progress fiber without first calling `StoreFiber::dispose`" + ); self.state.take().unwrap().dispose(); @@ -415,15 +398,15 @@ fn swap_mpk_states(mask: Option) -> Option { /// returned value; it will return `Err(yield_)` if the fiber suspended, where /// `yield_` indicates whether it released access to the store or not. See /// `StoreFiber::fiber` for details. -/// -/// SAFETY: The caller must confer exclusive access to the store to the fiber -/// until the fiber is either dropped, resolved, or forgotten, or until it -/// releases the store when suspending. -unsafe fn resume_fiber_raw<'a>( +fn resume_fiber_raw<'a>( + store: &mut StoreOpaque, fiber: &mut StoreFiber<'a>, result: Result<()>, ) -> Result, StoreFiberYield> { + assert_eq!(store.id(), fiber.id); + struct Restore<'a> { + store: *mut StoreOpaque, fiber: *mut StoreFiber<'a>, state: Option, } @@ -432,15 +415,16 @@ unsafe fn resume_fiber_raw<'a>( fn drop(&mut self) { unsafe { (*self.fiber).state = Some(self.state.take().unwrap().replace()); + (*self.store).swap_executor(&mut (*self.fiber).executor); } } } unsafe { - let _reset_executor = Reset(fiber.executor_ptr, *fiber.executor_ptr); - *fiber.executor_ptr = &raw mut fiber.executor; let _reset_suspend = Reset(fiber.suspend, *fiber.suspend); let prev = fiber.state.take().unwrap().replace(); + store.swap_executor(&mut fiber.executor); let restore = Restore { + store: &raw mut *store, fiber: &raw mut *fiber, state: Some(prev), }; @@ -465,10 +449,9 @@ pub(crate) fn make_fiber<'a>( debug_assert!(engine.target().is_pulley()); Executor::Interpreter(Interpreter::new(&engine)) }; + let id = store.store_opaque().id(); let stack = store.store_opaque_mut().allocate_fiber_stack()?; let suspend = unsafe { &raw mut (*store.store_opaque_mut().async_state()).current_suspend }; - let executor_ptr = - unsafe { &raw mut (*store.store_opaque_mut().async_state()).current_executor }; let track_pkey_context_switch = store.has_pkey(); let store = &raw mut *store; Ok(StoreFiber { @@ -509,17 +492,18 @@ pub(crate) fn make_fiber<'a>( }), engine, suspend, - executor_ptr, executor, + id, }) } /// See `resume_fiber_raw` -pub(crate) unsafe fn resume_fiber( +pub(crate) fn resume_fiber( + store: &mut StoreOpaque, fiber: &mut StoreFiber, result: Result<()>, ) -> Result, StoreFiberYield> { - match unsafe { resume_fiber_raw(fiber, result) } { + match resume_fiber_raw(store, fiber, result) { Ok(result) => Ok(result), Err(yield_) => { // If `Err` is returned that means the fiber suspended, so we @@ -603,6 +587,75 @@ fn prepare_fiber<'a, R: 'a>( Ok((fiber, rx)) } +struct FiberFuture<'a, 'b> { + store: &'a mut StoreOpaque, + fiber: Option>, +} + +impl<'b> Future for FiberFuture<'_, 'b> { + type Output = Result>; + + fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { + let me = self.get_mut(); + + let fiber = me.fiber.take().unwrap(); + + let poll_cx = unsafe { &raw mut (*me.store.async_state()).current_poll_cx }; + let _reset = Reset(poll_cx, unsafe { *poll_cx }); + let (guard_range_start, guard_range_end) = fiber + .fiber + .as_ref() + .unwrap() + .stack() + .guard_range() + .map(|r| (r.start, r.end)) + .unwrap_or((ptr::null_mut(), ptr::null_mut())); + + // We need to carry over this `cx` into our fiber's runtime for when + // it tries to poll sub-futures that are created. Doing this must be + // done unsafely, however, since `cx` is only alive for this one + // singular function call. Here we do a `transmute` to extend the + // lifetime of `Context` so it can be stored in our `Store`, and + // then we replace the current polling context with this one. + // + // Note that the replace is done for weird situations where futures + // might be switching contexts and there's multiple wasmtime futures + // in a chain of futures. + // + // On exit from this function, though, we reset the polling context + // back to what it was to signify that `Store` no longer has access + // to this pointer. + // + // SAFETY: We store the pointer to the `Context` only for the + // duration of this call and then reset it to its previous value + // afterward, thereby ensuring `fun` never sees a stale pointer. + unsafe { + *poll_cx = PollContext { + future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), + guard_range_start, + guard_range_end, + }; + } + + let mut fiber = Some(fiber); + match resume_fiber(me.store, fiber.as_mut().unwrap(), Ok(())) { + Ok(result) => Poll::Ready(result.map(|()| fiber.take().unwrap())), + Err(_) => { + me.fiber = fiber; + Poll::Pending + } + } + } +} + +impl Drop for FiberFuture<'_, '_> { + fn drop(&mut self) { + if let Some(mut fiber) = self.fiber.take() { + fiber.dispose(self.store); + } + } +} + /// Run the specified function on a newly-created fiber and `.await` its /// completion. async fn on_fiber_raw( @@ -615,16 +668,10 @@ async fn on_fiber_raw( let (fiber, mut rx) = prepare_fiber(store.traitobj_mut(), func)?; - let guard_range = fiber.guard_range(); - let mut fiber = Some(fiber); - let mut fiber = poll_fn(store, guard_range, move || { - // SAFETY: We confer exclusive access to the store to the fiber here, - // only taking it back when the fiber resolves. - match unsafe { resume_fiber(fiber.as_mut().unwrap(), Ok(())) } { - Ok(result) => Poll::Ready(result.map(|()| fiber.take().unwrap())), - Err(_) => Poll::Pending, - } - }) + let mut fiber = FiberFuture { + store, + fiber: Some(fiber), + } .await?; let stack = fiber.fiber.take().map(|f| f.into_stack()); @@ -635,59 +682,3 @@ async fn on_fiber_raw( Ok(rx.try_recv().unwrap().unwrap()) } - -/// Wrap the specified function in a future which, when polled, will store a -/// pointer to the `Context` in the `AsyncState::current_poll_cx` field for the -/// specified store and then call the function. -/// -/// This is intended for use with functions that resume fibers which may need to -/// poll futures using the stored `Context` pointer. -pub(crate) async fn poll_fn( - store: &mut StoreOpaque, - guard_range: (Option>, Option>), - mut fun: impl FnMut() -> Poll, -) -> R { - #[derive(Clone, Copy)] - struct PollCx(*mut PollContext); - - unsafe impl Send for PollCx {} - - let poll_cx = PollCx(unsafe { &raw mut (*store.async_state()).current_poll_cx }); - future::poll_fn({ - move |cx| { - let _reset = Reset(poll_cx.0, unsafe { *poll_cx.0 }); - let guard_range_start = guard_range.0.map(|v| v.as_ptr()).unwrap_or(ptr::null_mut()); - let guard_range_end = guard_range.1.map(|v| v.as_ptr()).unwrap_or(ptr::null_mut()); - // We need to carry over this `cx` into our fiber's runtime for when - // it tries to poll sub-futures that are created. Doing this must be - // done unsafely, however, since `cx` is only alive for this one - // singular function call. Here we do a `transmute` to extend the - // lifetime of `Context` so it can be stored in our `Store`, and - // then we replace the current polling context with this one. - // - // Note that the replace is done for weird situations where futures - // might be switching contexts and there's multiple wasmtime futures - // in a chain of futures. - // - // On exit from this function, though, we reset the polling context - // back to what it was to signify that `Store` no longer has access - // to this pointer. - // - // SAFETY: We store the pointer to the `Context` only for the - // duration of this call and then reset it to its previous value - // afterward, thereby ensuring `fun` never sees a stale pointer. - unsafe { - *poll_cx.0 = PollContext { - future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), - guard_range_start, - guard_range_end, - }; - } - #[allow(dropping_copy_types)] - drop(poll_cx); - - fun() - } - }) - .await -} diff --git a/crates/wasmtime/src/runtime/func.rs b/crates/wasmtime/src/runtime/func.rs index 49859fec3e34..a010bfa7e120 100644 --- a/crates/wasmtime/src/runtime/func.rs +++ b/crates/wasmtime/src/runtime/func.rs @@ -518,7 +518,9 @@ impl Func { Func::new(store, ty, move |mut caller, params, results| { let async_cx = AsyncCx::new(&mut caller.store.0); let mut future = Pin::from(func(caller, params, results)); - match async_cx.block_on(future.as_mut()) { + // SAFETY: The `Store` we used to create the `AsyncCx` above remains + // valid. + match unsafe { async_cx.block_on(future.as_mut()) } { Ok(Ok(())) => Ok(()), Ok(Err(trap)) | Err(trap) => Err(trap), } @@ -839,8 +841,9 @@ impl Func { Func::wrap_inner(store, move |mut caller: Caller<'_, T>, args| { let async_cx = AsyncCx::new(&mut caller.store.0); let mut future = Pin::from(func(caller, args)); - - match async_cx.block_on(future.as_mut()) { + // SAFETY: The `Store` we used to create the `AsyncCx` above remains + // valid. + match unsafe { async_cx.block_on(future.as_mut()) } { Ok(ret) => ret.into_fallible(), Err(e) => R::fallible_from_error(e), } diff --git a/crates/wasmtime/src/runtime/linker.rs b/crates/wasmtime/src/runtime/linker.rs index 36fadb89dc4e..e664dd80d867 100644 --- a/crates/wasmtime/src/runtime/linker.rs +++ b/crates/wasmtime/src/runtime/linker.rs @@ -473,7 +473,9 @@ impl Linker { self.func_new(module, name, ty, move |mut caller, params, results| { let async_cx = crate::fiber::AsyncCx::new(&mut caller.store.0); let mut future = Pin::from(func(caller, params, results)); - match async_cx.block_on(future.as_mut()) { + // SAFETY: The `Store` we used to create the `AsyncCx` above remains + // valid. + match unsafe { async_cx.block_on(future.as_mut()) } { Ok(Ok(())) => Ok(()), Ok(Err(trap)) | Err(trap) => Err(trap), } @@ -575,8 +577,9 @@ impl Linker { move |mut caller: Caller<'_, T>, args: Params| { let async_cx = crate::fiber::AsyncCx::new(&mut caller.store.0); let mut future = Pin::from(func(caller, args)); - - match async_cx.block_on(future.as_mut()) { + // SAFETY: The `Store` we used to create the `AsyncCx` above + // remains valid. + match unsafe { async_cx.block_on(future.as_mut()) } { Ok(ret) => ret.into_fallible(), Err(e) => Args::fallible_from_error(e), } diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index bcbc630d61ea..d88371547464 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -593,11 +593,6 @@ impl Store { data: ManuallyDrop::new(data), }); - #[cfg(feature = "async")] - { - inner.async_state.current_executor = &raw mut inner.executor; - } - inner.traitobj = StorePtr::new(NonNull::from(&mut *inner)); // Wasmtime uses the callee argument to host functions to learn about @@ -1103,13 +1098,15 @@ impl StoreInner { CallHookInner::Sync(hook) => hook((&mut *self).as_context_mut(), s), #[cfg(all(feature = "async", feature = "call-hook"))] - CallHookInner::Async(handler) => AsyncCx::try_new(&mut self.inner) - .ok_or_else(|| anyhow!("couldn't grab async_cx for call hook"))? - .block_on( - handler - .handle_call_event((&mut *self).as_context_mut(), s) - .as_mut(), - )?, + CallHookInner::Async(handler) => unsafe { + AsyncCx::try_new(&mut self.inner) + .ok_or_else(|| anyhow!("couldn't grab async_cx for call hook"))? + .block_on( + handler + .handle_call_event((&mut *self).as_context_mut(), s) + .as_mut(), + )? + }, CallHookInner::ForceTypeParameterToBeUsed { uninhabited, .. } => { let _ = s; @@ -1948,25 +1945,19 @@ at https://bytecodealliance.org/security. } pub(crate) fn executor(&mut self) -> ExecutorRef<'_> { - #[cfg(feature = "async")] - let executor = unsafe { &mut *self.async_state.current_executor }; - #[cfg(not(feature = "async"))] - let executor = &mut self.executor; - - match executor { + match &mut self.executor { Executor::Interpreter(i) => ExecutorRef::Interpreter(i.as_interpreter_ref()), #[cfg(has_host_compiler_backend)] Executor::Native => ExecutorRef::Native, } } - pub(crate) fn unwinder(&self) -> &'static dyn Unwind { - #[cfg(feature = "async")] - let executor = unsafe { &*self.async_state.current_executor }; - #[cfg(not(feature = "async"))] - let executor = &self.executor; + pub(crate) fn swap_executor(&mut self, executor: &mut Executor) { + mem::swap(&mut self.executor, executor); + } - match executor { + pub(crate) fn unwinder(&self) -> &'static dyn Unwind { + match &self.executor { Executor::Interpreter(i) => i.unwinder(), #[cfg(has_host_compiler_backend)] Executor::Native => &vm::UnwindHost, @@ -2120,13 +2111,15 @@ unsafe impl vm::VMStore for StoreInner { limiter(&mut self.data).memory_growing(current, desired, maximum) } #[cfg(feature = "async")] - Some(ResourceLimiterInner::Async(ref mut limiter)) => AsyncCx::try_new(&mut self.inner) - .expect("ResourceLimiterAsync requires async Store") - .block_on( - limiter(&mut self.data) - .memory_growing(current, desired, maximum) - .as_mut(), - )?, + Some(ResourceLimiterInner::Async(ref mut limiter)) => unsafe { + AsyncCx::try_new(&mut self.inner) + .expect("ResourceLimiterAsync requires async Store") + .block_on( + limiter(&mut self.data) + .memory_growing(current, desired, maximum) + .as_mut(), + )? + }, None => Ok(true), } } @@ -2170,13 +2163,15 @@ unsafe impl vm::VMStore for StoreInner { limiter(&mut self.data).table_growing(current, desired, maximum) } #[cfg(feature = "async")] - Some(ResourceLimiterInner::Async(ref mut limiter)) => async_cx - .expect("ResourceLimiterAsync requires async Store") - .block_on( - limiter(&mut self.data) - .table_growing(current, desired, maximum) - .as_mut(), - )?, + Some(ResourceLimiterInner::Async(ref mut limiter)) => unsafe { + async_cx + .expect("ResourceLimiterAsync requires async Store") + .block_on( + limiter(&mut self.data) + .table_growing(current, desired, maximum) + .as_mut(), + )? + }, None => Ok(true), } } @@ -2243,9 +2238,10 @@ unsafe impl vm::VMStore for StoreInner { // to clean up this fiber. Do so by raising a trap which will // abort all wasm and get caught on the other side to clean // things up. - AsyncCx::try_new(self) + unsafe {AsyncCx::try_new(self) .expect("attempted to pull async context during shutdown") - .block_on(future.as_mut())?; + .block_on(future.as_mut())?; + } delta } }; @@ -2337,7 +2333,7 @@ impl Drop for Store { fn drop(&mut self) { self.inner.flush_fiber_stack(); - // for documentation on this `unsafe`, see `into_data`. + // for documentation on this `unsafe`, see `inrto_data`. unsafe { ManuallyDrop::drop(&mut self.inner.data); ManuallyDrop::drop(&mut self.inner); diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index e8f75d17ff34..231a49f8713c 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -252,9 +252,11 @@ impl StoreOpaque { // to clean up this fiber. Do so by raising a trap which will // abort all wasm and get caught on the other side to clean // things up. - AsyncCx::try_new(self) - .expect("attempted to pull async context during shutdown") - .block_on(unsafe { Pin::new_unchecked(&mut future) }) + unsafe { + AsyncCx::try_new(self) + .expect("attempted to pull async context during shutdown") + .block_on(Pin::new_unchecked(&mut future)) + } } pub(crate) fn async_guard_range(&mut self) -> Range<*mut u8> { @@ -262,7 +264,7 @@ impl StoreOpaque { } pub(crate) fn allocate_fiber_stack(&mut self) -> Result { - if let Some(stack) = self.async_state.last_fiber_stack.take() { + if let Some(stack) = self.async_state.last_fiber_stack().take() { return Ok(stack); } self.engine().allocator().allocate_fiber_stack() @@ -270,13 +272,13 @@ impl StoreOpaque { pub(crate) fn deallocate_fiber_stack(&mut self, stack: wasmtime_fiber::FiberStack) { self.flush_fiber_stack(); - self.async_state.last_fiber_stack = Some(stack); + *self.async_state.last_fiber_stack() = Some(stack); } /// Releases the last fiber stack to the underlying instance allocator, if /// present. pub fn flush_fiber_stack(&mut self) { - if let Some(stack) = self.async_state.last_fiber_stack.take() { + if let Some(stack) = self.async_state.last_fiber_stack().take() { unsafe { self.engine.allocator().deallocate_fiber_stack(stack); } From 45859eb85576eb9914953e2188af571e7c01f12e Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Tue, 24 Jun 2025 16:49:36 -0600 Subject: [PATCH 06/30] address more review feedback Signed-off-by: Joel Dice --- benches/call.rs | 2 +- .../src/runtime/component/func/typed.rs | 2 +- crates/wasmtime/src/runtime/fiber.rs | 29 ++++++------------- crates/wasmtime/src/runtime/func/typed.rs | 5 +++- crates/wasmtime/src/runtime/store.rs | 29 ++++++++++++------- crates/wasmtime/src/runtime/store/async_.rs | 8 ++--- 6 files changed, 37 insertions(+), 38 deletions(-) diff --git a/benches/call.rs b/benches/call.rs index c7fc9ecf2f63..6d4d68b95d09 100644 --- a/benches/call.rs +++ b/benches/call.rs @@ -134,7 +134,7 @@ fn bench_host_to_wasm( typed_params: Params, typed_results: Results, ) where - Params: WasmParams + ToVals + Copy, + Params: WasmParams + ToVals + Copy + Sync, Results: WasmResults + ToVals + Copy + PartialEq + Debug + 'static, { // Benchmark the "typed" version, which should be faster than the versions diff --git a/crates/wasmtime/src/runtime/component/func/typed.rs b/crates/wasmtime/src/runtime/component/func/typed.rs index 68c6819fbcab..c378742da236 100644 --- a/crates/wasmtime/src/runtime/component/func/typed.rs +++ b/crates/wasmtime/src/runtime/component/func/typed.rs @@ -179,7 +179,7 @@ where ) -> Result where Params: Send + Sync, - Return: Sync, + Return: Send + Sync, { let mut store = store.as_context_mut(); assert!( diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 9ab7724f4152..d64c76d65dfa 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -3,7 +3,7 @@ use crate::Engine; use crate::store::{Executor, StoreId, StoreOpaque}; use crate::vm::mpk::{self, ProtectionMask}; -use crate::vm::{AsyncWasmCallState, Interpreter, VMStore}; +use crate::vm::{AsyncWasmCallState, VMStore}; use anyhow::{Result, anyhow}; use core::mem; use core::ops::Range; @@ -11,7 +11,6 @@ use core::pin::Pin; use core::ptr; use core::task::{Context, Poll}; use futures::channel::oneshot; -use wasmtime_environ::TripleExt; use wasmtime_fiber::{Fiber, Suspend}; /// Helper struct for reseting a raw pointer to its original value on drop. @@ -435,20 +434,10 @@ fn resume_fiber_raw<'a>( /// Create a new `StoreFiber` which runs the specified closure. pub(crate) fn make_fiber<'a>( store: &mut dyn VMStore, - fun: impl FnOnce(&mut dyn VMStore) -> Result<()> + 'a, + fun: impl FnOnce(&mut dyn VMStore) -> Result<()> + Send + Sync + 'a, ) -> Result> { let engine = store.engine().clone(); - #[cfg(has_host_compiler_backend)] - let executor = if cfg!(feature = "pulley") && engine.target().is_pulley() { - Executor::Interpreter(Interpreter::new(&engine)) - } else { - Executor::Native - }; - #[cfg(not(has_host_compiler_backend))] - let executor = { - debug_assert!(engine.target().is_pulley()); - Executor::Interpreter(Interpreter::new(&engine)) - }; + let executor = Executor::new(&engine); let id = store.store_opaque().id(); let stack = store.store_opaque_mut().allocate_fiber_stack()?; let suspend = unsafe { &raw mut (*store.store_opaque_mut().async_state()).current_suspend }; @@ -562,9 +551,9 @@ unsafe fn suspend_fiber( /// Run the specified function on a newly-created fiber and `.await` its /// completion. -pub(crate) async fn on_fiber( +pub(crate) async fn on_fiber( store: &mut StoreOpaque, - func: impl FnOnce(&mut StoreOpaque) -> R + Send, + func: impl FnOnce(&mut StoreOpaque) -> R + Send + Sync, ) -> Result { on_fiber_raw(store.traitobj_mut(), move |store| { func((*store).store_opaque_mut()) @@ -573,9 +562,9 @@ pub(crate) async fn on_fiber( } /// Wrap the specified function in a fiber and return it. -fn prepare_fiber<'a, R: 'a>( +fn prepare_fiber<'a, R: Send + 'a>( store: &mut dyn VMStore, - func: impl FnOnce(&mut dyn VMStore) -> R + Send + 'a, + func: impl FnOnce(&mut dyn VMStore) -> R + Send + Sync + 'a, ) -> Result<(StoreFiber<'a>, oneshot::Receiver)> { let (tx, rx) = oneshot::channel(); let fiber = make_fiber(store, { @@ -658,9 +647,9 @@ impl Drop for FiberFuture<'_, '_> { /// Run the specified function on a newly-created fiber and `.await` its /// completion. -async fn on_fiber_raw( +async fn on_fiber_raw( store: &mut StoreOpaque, - func: impl FnOnce(&mut dyn VMStore) -> R + Send, + func: impl FnOnce(&mut dyn VMStore) -> R + Send + Sync, ) -> Result { let config = store.engine().config(); debug_assert!(store.async_support()); diff --git a/crates/wasmtime/src/runtime/func/typed.rs b/crates/wasmtime/src/runtime/func/typed.rs index 0df0dcc400c2..ee325f4c86c9 100644 --- a/crates/wasmtime/src/runtime/func/typed.rs +++ b/crates/wasmtime/src/runtime/func/typed.rs @@ -132,7 +132,10 @@ where &self, mut store: impl AsContextMut, params: Params, - ) -> Result { + ) -> Result + where + Params: Sync, + { let mut store = store.as_context_mut(); assert!( store.0.async_support(), diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index d88371547464..7047543600bc 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -411,6 +411,22 @@ pub(crate) enum Executor { Native, } +impl Executor { + pub(crate) fn new(engine: &Engine) -> Self { + #[cfg(has_host_compiler_backend)] + if cfg!(feature = "pulley") && engine.target().is_pulley() { + Executor::Interpreter(Interpreter::new(engine)) + } else { + Executor::Native + } + #[cfg(not(has_host_compiler_backend))] + { + debug_assert!(engine.target().is_pulley()); + Executor::Interpreter(Interpreter::new(engine)) + } + } +} + /// A borrowed reference to `Executor` above. pub(crate) enum ExecutorRef<'a> { Interpreter(InterpreterRef<'a>), @@ -572,17 +588,7 @@ impl Store { component_calls: Default::default(), #[cfg(feature = "component-model")] host_resource_data: Default::default(), - #[cfg(has_host_compiler_backend)] - executor: if cfg!(feature = "pulley") && engine.target().is_pulley() { - Executor::Interpreter(Interpreter::new(engine)) - } else { - Executor::Native - }, - #[cfg(not(has_host_compiler_backend))] - executor: { - debug_assert!(engine.target().is_pulley()); - Executor::Interpreter(Interpreter::new(engine)) - }, + executor: Executor::new(engine), }; let mut inner = Box::new(StoreInner { inner, @@ -1952,6 +1958,7 @@ at https://bytecodealliance.org/security. } } + #[cfg(feature = "async")] pub(crate) fn swap_executor(&mut self, executor: &mut Executor) { mem::swap(&mut self.executor, executor); } diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index 231a49f8713c..d1233f0c32bb 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -171,9 +171,9 @@ impl StoreOpaque { /// This function will convert the synchronous `func` into an asynchronous /// future. This is done by running `func` in a fiber on a separate native /// stack which can be suspended and resumed from. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, - func: impl FnOnce(&mut Self) -> R + Send, + func: impl FnOnce(&mut Self) -> R + Send + Sync, ) -> Result { fiber::on_fiber(self, func).await } @@ -288,9 +288,9 @@ impl StoreOpaque { impl StoreContextMut<'_, T> { /// Executes a synchronous computation `func` asynchronously on a new fiber. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, - func: impl FnOnce(&mut StoreContextMut<'_, T>) -> R + Send, + func: impl FnOnce(&mut StoreContextMut<'_, T>) -> R + Send + Sync, ) -> Result where T: Send + 'static, From 77d0a804e75caba01e6458310171de2167a6e3ba Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Tue, 24 Jun 2025 17:32:58 -0600 Subject: [PATCH 07/30] update `impl Send For StoreFiber` comment Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/fiber.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index d64c76d65dfa..4b614d903ba3 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -270,11 +270,11 @@ impl Drop for StoreFiber<'_> { } // This is surely the most dangerous `unsafe impl Send` in the entire -// crate. There are two members in `FiberFuture` which cause it to not be -// `Send`. One is `current_poll_cx` and is entirely uninteresting. This is just -// used to manage `Context` pointers across `await` points in the future, and -// requires raw pointers to get it to happen easily. Nothing too weird about -// the `Send`-ness, values aren't actually crossing threads. +// crate. There are two members in `StoreFiber` which cause it to not be +// `Send`. One is `suspend` and is entirely uninteresting. This is just used to +// manage `Suspend` when resuming, and requires raw pointers to get it to happen +// easily. Nothing too weird about the `Send`-ness, values aren't actually +// crossing threads. // // The really interesting piece is `fiber`. Now the "fiber" here is actual // honest-to-god Rust code which we're moving around. What we're doing is the From 8a8768ea95c49e41c3e59d7c6152b9791eace67c Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:01:07 -0700 Subject: [PATCH 08/30] Remove currently-extraneous `Result<()>` from fibers May be needed for concurrent bits, but for now not necessary. --- crates/wasmtime/src/runtime/fiber.rs | 69 ++++++++++++++-------------- 1 file changed, 34 insertions(+), 35 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 4b614d903ba3..30d85c2205d2 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -51,7 +51,7 @@ pub(crate) struct AsyncState { /// The `Suspend` for the current fiber (or null if no such fiber is running). /// /// See `StoreFiber` for an explanation of the signature types we use here. - current_suspend: *mut Suspend, StoreFiberYield, Result<()>>, + current_suspend: *mut Suspend, StoreFiberYield, ()>, /// See `PollContext` current_poll_cx: PollContext, @@ -94,7 +94,7 @@ unsafe impl Sync for AsyncState {} /// Used to "stackfully" poll a future by suspending the current fiber /// repeatedly in a loop until the future completes. pub(crate) struct AsyncCx { - current_suspend: *mut *mut wasmtime_fiber::Suspend, StoreFiberYield, Result<()>>, + current_suspend: *mut *mut Suspend, StoreFiberYield, ()>, current_stack_limit: *mut usize, current_poll_cx: *mut PollContext, } @@ -222,14 +222,14 @@ pub(crate) struct StoreFiber<'a> { /// /// Note also that every `StoreFiber` is implicitly granted exclusive access /// to the store when it is resumed. - fiber: Option, StoreFiberYield, Result<()>>>, + fiber: Option, StoreFiberYield, ()>>, /// See `FiberResumeState` state: Option, /// The Wasmtime `Engine` to which this fiber belongs. engine: Engine, /// The current `Suspend` for this fiber (or null if it's not currently /// running). - suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, + suspend: *mut *mut Suspend, StoreFiberYield, ()>, executor: Executor, id: StoreId, } @@ -401,7 +401,7 @@ fn resume_fiber_raw<'a>( store: &mut StoreOpaque, fiber: &mut StoreFiber<'a>, result: Result<()>, -) -> Result, StoreFiberYield> { +) -> Result<(), StoreFiberYield> { assert_eq!(store.id(), fiber.id); struct Restore<'a> { @@ -434,7 +434,7 @@ fn resume_fiber_raw<'a>( /// Create a new `StoreFiber` which runs the specified closure. pub(crate) fn make_fiber<'a>( store: &mut dyn VMStore, - fun: impl FnOnce(&mut dyn VMStore) -> Result<()> + Send + Sync + 'a, + fun: impl FnOnce(&mut dyn VMStore) + Send + Sync + 'a, ) -> Result> { let engine = store.engine().clone(); let executor = Executor::new(&engine); @@ -445,31 +445,31 @@ pub(crate) fn make_fiber<'a>( let store = &raw mut *store; Ok(StoreFiber { fiber: Some(Fiber::new(stack, move |result: Result<()>, suspend| { + // Cancelled before we started? Just return. if result.is_err() { - result - } else { - // SAFETY: Per the documented contract for - // `resume_fiber_raw`, we've been given exclusive access to - // the store until we exit or yield it back to the resumer. - let store_ref = unsafe { &mut *store }; - let suspend_ptr = unsafe { - &raw mut (*store_ref.store_opaque_mut().async_state()).current_suspend - }; - // Configure our store's suspension context for the rest of the - // execution of this fiber. Note that a raw pointer is stored here - // which is only valid for the duration of this closure. - // Consequently we at least replace it with the previous value when - // we're done. This reset is also required for correctness because - // otherwise our value will overwrite another active fiber's value. - // There should be a test that segfaults in `async_functions.rs` if - // this `Reset` is removed. - // - // SAFETY: The resumer is responsible for setting - // `current_suspend` to a valid pointer. - let _reset = Reset(suspend_ptr, unsafe { *suspend_ptr }); - unsafe { *suspend_ptr = suspend }; - fun(store_ref) + return; } + + // SAFETY: Per the documented contract for + // `resume_fiber_raw`, we've been given exclusive access to + // the store until we exit or yield it back to the resumer. + let store_ref = unsafe { &mut *store }; + let suspend_ptr = + unsafe { &raw mut (*store_ref.store_opaque_mut().async_state()).current_suspend }; + // Configure our store's suspension context for the rest of the + // execution of this fiber. Note that a raw pointer is stored here + // which is only valid for the duration of this closure. + // Consequently we at least replace it with the previous value when + // we're done. This reset is also required for correctness because + // otherwise our value will overwrite another active fiber's value. + // There should be a test that segfaults in `async_functions.rs` if + // this `Reset` is removed. + // + // SAFETY: The resumer is responsible for setting + // `current_suspend` to a valid pointer. + let _reset = Reset(suspend_ptr, unsafe { *suspend_ptr }); + unsafe { *suspend_ptr = suspend }; + fun(store_ref) })?), state: Some(FiberResumeState { tls: crate::runtime::vm::AsyncWasmCallState::new(), @@ -491,7 +491,7 @@ pub(crate) fn resume_fiber( store: &mut StoreOpaque, fiber: &mut StoreFiber, result: Result<()>, -) -> Result, StoreFiberYield> { +) -> Result<(), StoreFiberYield> { match resume_fiber_raw(store, fiber, result) { Ok(result) => Ok(result), Err(yield_) => { @@ -524,7 +524,7 @@ pub(crate) fn resume_fiber( /// is provided, the fiber must give up access to the store until it is given /// back access when next resumed. unsafe fn suspend_fiber( - suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, + suspend: *mut *mut Suspend, StoreFiberYield, ()>, stack_limit: *mut usize, yield_: StoreFiberYield, ) -> Result<()> { @@ -570,7 +570,6 @@ fn prepare_fiber<'a, R: Send + 'a>( let fiber = make_fiber(store, { move |store| { _ = tx.send(func(store)); - Ok(()) } })?; Ok((fiber, rx)) @@ -582,7 +581,7 @@ struct FiberFuture<'a, 'b> { } impl<'b> Future for FiberFuture<'_, 'b> { - type Output = Result>; + type Output = StoreFiber<'b>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let me = self.get_mut(); @@ -628,7 +627,7 @@ impl<'b> Future for FiberFuture<'_, 'b> { let mut fiber = Some(fiber); match resume_fiber(me.store, fiber.as_mut().unwrap(), Ok(())) { - Ok(result) => Poll::Ready(result.map(|()| fiber.take().unwrap())), + Ok(()) => Poll::Ready(fiber.take().unwrap()), Err(_) => { me.fiber = fiber; Poll::Pending @@ -661,7 +660,7 @@ async fn on_fiber_raw( store, fiber: Some(fiber), } - .await?; + .await; let stack = fiber.fiber.take().map(|f| f.into_stack()); drop(fiber); From b67b8b8b2848d21bb5b3ec7f030dee4770ebf82d Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:02:17 -0700 Subject: [PATCH 09/30] Use safe pointers instead of raw pointers It's predicted Miri won't like this, but for now in-repo it's ok with it. --- crates/wasmtime/src/runtime/fiber.rs | 20 +++++++++----------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 30d85c2205d2..cef1c84780b5 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -404,18 +404,16 @@ fn resume_fiber_raw<'a>( ) -> Result<(), StoreFiberYield> { assert_eq!(store.id(), fiber.id); - struct Restore<'a> { - store: *mut StoreOpaque, - fiber: *mut StoreFiber<'a>, + struct Restore<'a, 'b> { + store: &'b mut StoreOpaque, + fiber: &'b mut StoreFiber<'a>, state: Option, } - impl Drop for Restore<'_> { + impl Drop for Restore<'_, '_> { fn drop(&mut self) { - unsafe { - (*self.fiber).state = Some(self.state.take().unwrap().replace()); - (*self.store).swap_executor(&mut (*self.fiber).executor); - } + self.fiber.state = Some(unsafe { self.state.take().unwrap().replace() }); + self.store.swap_executor(&mut self.fiber.executor); } } unsafe { @@ -423,11 +421,11 @@ fn resume_fiber_raw<'a>( let prev = fiber.state.take().unwrap().replace(); store.swap_executor(&mut fiber.executor); let restore = Restore { - store: &raw mut *store, - fiber: &raw mut *fiber, + store, + fiber, state: Some(prev), }; - (*restore.fiber).fiber.as_ref().unwrap().resume(result) + restore.fiber.fiber.as_ref().unwrap().resume(result) } } From fee87eea42720ea3781f746162815342b1bc3609 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:05:40 -0700 Subject: [PATCH 10/30] Fold more responsibility into `resume_fiber_raw` Remove the need for the function entirely and replace it with `resume_fiber`. --- crates/wasmtime/src/runtime/fiber.rs | 86 ++++++++++++++-------------- 1 file changed, 42 insertions(+), 44 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index cef1c84780b5..10b56b3d92b7 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -236,9 +236,11 @@ pub(crate) struct StoreFiber<'a> { impl StoreFiber<'_> { fn dispose(&mut self, store: &mut StoreOpaque) { - if !self.fiber.as_ref().unwrap().done() { - let result = resume_fiber_raw(store, self, Err(anyhow!("future dropped"))); - debug_assert!(result.is_ok()); + if let Some(fiber) = &mut self.fiber { + if !fiber.done() { + let result = resume_fiber(store, self, Err(anyhow!("future dropped"))); + debug_assert!(result.is_ok()); + } } } } @@ -397,7 +399,7 @@ fn swap_mpk_states(mask: Option) -> Option { /// returned value; it will return `Err(yield_)` if the fiber suspended, where /// `yield_` indicates whether it released access to the store or not. See /// `StoreFiber::fiber` for details. -fn resume_fiber_raw<'a>( +pub(crate) fn resume_fiber<'a>( store: &mut StoreOpaque, fiber: &mut StoreFiber<'a>, result: Result<()>, @@ -416,7 +418,7 @@ fn resume_fiber_raw<'a>( self.store.swap_executor(&mut self.fiber.executor); } } - unsafe { + let result = unsafe { let _reset_suspend = Reset(fiber.suspend, *fiber.suspend); let prev = fiber.state.take().unwrap().replace(); store.swap_executor(&mut fiber.executor); @@ -426,7 +428,40 @@ fn resume_fiber_raw<'a>( state: Some(prev), }; restore.fiber.fiber.as_ref().unwrap().resume(result) + }; + + match &result { + // The fiber has finished, so recycle its stack by disposing of the + // underlying fiber itself. + Ok(_) => { + let stack = fiber.fiber.take().map(|f| f.into_stack()); + if let Some(stack) = stack { + store.deallocate_fiber_stack(stack); + } + } + + // The fiber has not yet finished, so it stays as-is. + Err(_) => { + // If `Err` is returned that means the fiber suspended, so we + // propagate that here. + // + // An additional safety check is performed when leaving this + // function to help bolster the guarantees of `unsafe impl Send` + // above. Notably this future may get re-polled on a different + // thread. Wasmtime's thread-local state points to the stack, + // however, meaning that it would be incorrect to leave a pointer in + // TLS when this function returns. This function performs a runtime + // assert to verify that this is the case, notably that the one TLS + // pointer Wasmtime uses is not pointing anywhere within the + // stack. If it is then that's a bug indicating that TLS management + // in Wasmtime is incorrect. + if let Some(range) = fiber.fiber.as_ref().unwrap().stack().range() { + AsyncWasmCallState::assert_current_state_not_in_range(range); + } + } } + + result } /// Create a new `StoreFiber` which runs the specified closure. @@ -449,7 +484,7 @@ pub(crate) fn make_fiber<'a>( } // SAFETY: Per the documented contract for - // `resume_fiber_raw`, we've been given exclusive access to + // `resume_fiber`, we've been given exclusive access to // the store until we exit or yield it back to the resumer. let store_ref = unsafe { &mut *store }; let suspend_ptr = @@ -484,37 +519,6 @@ pub(crate) fn make_fiber<'a>( }) } -/// See `resume_fiber_raw` -pub(crate) fn resume_fiber( - store: &mut StoreOpaque, - fiber: &mut StoreFiber, - result: Result<()>, -) -> Result<(), StoreFiberYield> { - match resume_fiber_raw(store, fiber, result) { - Ok(result) => Ok(result), - Err(yield_) => { - // If `Err` is returned that means the fiber suspended, so we - // propagate that here. - // - // An additional safety check is performed when leaving this - // function to help bolster the guarantees of `unsafe impl Send` - // above. Notably this future may get re-polled on a different - // thread. Wasmtime's thread-local state points to the stack, - // however, meaning that it would be incorrect to leave a pointer in - // TLS when this function returns. This function performs a runtime - // assert to verify that this is the case, notably that the one TLS - // pointer Wasmtime uses is not pointing anywhere within the - // stack. If it is then that's a bug indicating that TLS management - // in Wasmtime is incorrect. - if let Some(range) = fiber.fiber.as_ref().unwrap().stack().range() { - AsyncWasmCallState::assert_current_state_not_in_range(range); - } - - Err(yield_) - } - } -} - /// Suspend the current fiber, optionally returning exclusive access to the /// specified store to the code which resumed the fiber. /// @@ -654,17 +658,11 @@ async fn on_fiber_raw( let (fiber, mut rx) = prepare_fiber(store.traitobj_mut(), func)?; - let mut fiber = FiberFuture { + FiberFuture { store, fiber: Some(fiber), } .await; - let stack = fiber.fiber.take().map(|f| f.into_stack()); - drop(fiber); - if let Some(stack) = stack { - store.deallocate_fiber_stack(stack); - } - Ok(rx.try_recv().unwrap().unwrap()) } From 3de82822675247cb20774b5c8e15297522b6a9c1 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:10:27 -0700 Subject: [PATCH 11/30] Remove channels from async fibers Can use stack-based closures/results to transmit the result instead of needing a channel. --- benches/call.rs | 2 +- crates/wasmtime/Cargo.toml | 1 - crates/wasmtime/src/runtime/fiber.rs | 24 +++++---------------- crates/wasmtime/src/runtime/func/typed.rs | 1 + crates/wasmtime/src/runtime/store/async_.rs | 4 ++-- 5 files changed, 9 insertions(+), 23 deletions(-) diff --git a/benches/call.rs b/benches/call.rs index 6d4d68b95d09..95d9a3379746 100644 --- a/benches/call.rs +++ b/benches/call.rs @@ -135,7 +135,7 @@ fn bench_host_to_wasm( typed_results: Results, ) where Params: WasmParams + ToVals + Copy + Sync, - Results: WasmResults + ToVals + Copy + PartialEq + Debug + 'static, + Results: WasmResults + ToVals + Copy + Sync + PartialEq + Debug + 'static, { // Benchmark the "typed" version, which should be faster than the versions // below. diff --git a/crates/wasmtime/Cargo.toml b/crates/wasmtime/Cargo.toml index f7b420e2db2d..02188fa8fd91 100644 --- a/crates/wasmtime/Cargo.toml +++ b/crates/wasmtime/Cargo.toml @@ -194,7 +194,6 @@ async = [ "dep:wasmtime-fiber", "dep:async-trait", "dep:trait-variant", - "dep:futures", "wasmtime-component-macro?/async", "runtime", ] diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 10b56b3d92b7..a58c15aed7a1 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -10,7 +10,6 @@ use core::ops::Range; use core::pin::Pin; use core::ptr; use core::task::{Context, Poll}; -use futures::channel::oneshot; use wasmtime_fiber::{Fiber, Suspend}; /// Helper struct for reseting a raw pointer to its original value on drop. @@ -553,7 +552,7 @@ unsafe fn suspend_fiber( /// Run the specified function on a newly-created fiber and `.await` its /// completion. -pub(crate) async fn on_fiber( +pub(crate) async fn on_fiber( store: &mut StoreOpaque, func: impl FnOnce(&mut StoreOpaque) -> R + Send + Sync, ) -> Result { @@ -563,20 +562,6 @@ pub(crate) async fn on_fiber( .await } -/// Wrap the specified function in a fiber and return it. -fn prepare_fiber<'a, R: Send + 'a>( - store: &mut dyn VMStore, - func: impl FnOnce(&mut dyn VMStore) -> R + Send + Sync + 'a, -) -> Result<(StoreFiber<'a>, oneshot::Receiver)> { - let (tx, rx) = oneshot::channel(); - let fiber = make_fiber(store, { - move |store| { - _ = tx.send(func(store)); - } - })?; - Ok((fiber, rx)) -} - struct FiberFuture<'a, 'b> { store: &'a mut StoreOpaque, fiber: Option>, @@ -648,7 +633,7 @@ impl Drop for FiberFuture<'_, '_> { /// Run the specified function on a newly-created fiber and `.await` its /// completion. -async fn on_fiber_raw( +async fn on_fiber_raw( store: &mut StoreOpaque, func: impl FnOnce(&mut dyn VMStore) -> R + Send + Sync, ) -> Result { @@ -656,7 +641,8 @@ async fn on_fiber_raw( debug_assert!(store.async_support()); debug_assert!(config.async_stack_size > 0); - let (fiber, mut rx) = prepare_fiber(store.traitobj_mut(), func)?; + let mut result = None; + let fiber = make_fiber(store.traitobj_mut(), |store| result = Some(func(store)))?; FiberFuture { store, @@ -664,5 +650,5 @@ async fn on_fiber_raw( } .await; - Ok(rx.try_recv().unwrap().unwrap()) + Ok(result.unwrap()) } diff --git a/crates/wasmtime/src/runtime/func/typed.rs b/crates/wasmtime/src/runtime/func/typed.rs index ee325f4c86c9..be808a636703 100644 --- a/crates/wasmtime/src/runtime/func/typed.rs +++ b/crates/wasmtime/src/runtime/func/typed.rs @@ -135,6 +135,7 @@ where ) -> Result where Params: Sync, + Results: Sync, { let mut store = store.as_context_mut(); assert!( diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index d1233f0c32bb..13cc11d311f3 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -171,7 +171,7 @@ impl StoreOpaque { /// This function will convert the synchronous `func` into an asynchronous /// future. This is done by running `func` in a fiber on a separate native /// stack which can be suspended and resumed from. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut Self) -> R + Send + Sync, ) -> Result { @@ -288,7 +288,7 @@ impl StoreOpaque { impl StoreContextMut<'_, T> { /// Executes a synchronous computation `func` asynchronously on a new fiber. - pub(crate) async fn on_fiber( + pub(crate) async fn on_fiber( &mut self, func: impl FnOnce(&mut StoreContextMut<'_, T>) -> R + Send + Sync, ) -> Result From dbc0b7d213711953131e4e9ff5c3b9fe7a59049a Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:11:40 -0700 Subject: [PATCH 12/30] Fold `on_fiber_raw` directly into `on_fiber` The `on_fiber` function is small enough it should be possible to do so. --- crates/wasmtime/src/runtime/fiber.rs | 40 ++++++++++------------------ 1 file changed, 14 insertions(+), 26 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index a58c15aed7a1..ec39a336d287 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -556,10 +556,20 @@ pub(crate) async fn on_fiber( store: &mut StoreOpaque, func: impl FnOnce(&mut StoreOpaque) -> R + Send + Sync, ) -> Result { - on_fiber_raw(store.traitobj_mut(), move |store| { - func((*store).store_opaque_mut()) - }) - .await + let config = store.engine().config(); + debug_assert!(store.async_support()); + debug_assert!(config.async_stack_size > 0); + + let mut result = None; + let fiber = make_fiber(store.traitobj_mut(), |store| result = Some(func(store)))?; + + FiberFuture { + store, + fiber: Some(fiber), + } + .await; + + Ok(result.unwrap()) } struct FiberFuture<'a, 'b> { @@ -630,25 +640,3 @@ impl Drop for FiberFuture<'_, '_> { } } } - -/// Run the specified function on a newly-created fiber and `.await` its -/// completion. -async fn on_fiber_raw( - store: &mut StoreOpaque, - func: impl FnOnce(&mut dyn VMStore) -> R + Send + Sync, -) -> Result { - let config = store.engine().config(); - debug_assert!(store.async_support()); - debug_assert!(config.async_stack_size > 0); - - let mut result = None; - let fiber = make_fiber(store.traitobj_mut(), |store| result = Some(func(store)))?; - - FiberFuture { - store, - fiber: Some(fiber), - } - .await; - - Ok(result.unwrap()) -} From 75629053479e9f7bb2549a71d1b024febddce759 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:12:49 -0700 Subject: [PATCH 13/30] Don't use `Option` in `FiberFuture` Leave the fiber non-optional at-rest so it's always available for the destructor. --- crates/wasmtime/src/runtime/fiber.rs | 29 +++++++++------------------- 1 file changed, 9 insertions(+), 20 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index ec39a336d287..e54a8e8018c2 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -563,31 +563,26 @@ pub(crate) async fn on_fiber( let mut result = None; let fiber = make_fiber(store.traitobj_mut(), |store| result = Some(func(store)))?; - FiberFuture { - store, - fiber: Some(fiber), - } - .await; + FiberFuture { store, fiber }.await; Ok(result.unwrap()) } struct FiberFuture<'a, 'b> { store: &'a mut StoreOpaque, - fiber: Option>, + fiber: StoreFiber<'b>, } impl<'b> Future for FiberFuture<'_, 'b> { - type Output = StoreFiber<'b>; + type Output = (); fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let me = self.get_mut(); - let fiber = me.fiber.take().unwrap(); - let poll_cx = unsafe { &raw mut (*me.store.async_state()).current_poll_cx }; let _reset = Reset(poll_cx, unsafe { *poll_cx }); - let (guard_range_start, guard_range_end) = fiber + let (guard_range_start, guard_range_end) = me + .fiber .fiber .as_ref() .unwrap() @@ -622,21 +617,15 @@ impl<'b> Future for FiberFuture<'_, 'b> { }; } - let mut fiber = Some(fiber); - match resume_fiber(me.store, fiber.as_mut().unwrap(), Ok(())) { - Ok(()) => Poll::Ready(fiber.take().unwrap()), - Err(_) => { - me.fiber = fiber; - Poll::Pending - } + match resume_fiber(me.store, &mut me.fiber, Ok(())) { + Ok(()) => Poll::Ready(()), + Err(_) => Poll::Pending, } } } impl Drop for FiberFuture<'_, '_> { fn drop(&mut self) { - if let Some(mut fiber) = self.fiber.take() { - fiber.dispose(self.store); - } + self.fiber.dispose(self.store); } } From b9926c689900471e4f111c14211635379e5f14fa Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:36:16 -0700 Subject: [PATCH 14/30] Fold `suspend` functions together Small shims, not otherwise public at this time, so remove a layer of indirection. --- crates/wasmtime/src/runtime/fiber.rs | 52 ++++++++++------------------ 1 file changed, 19 insertions(+), 33 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index e54a8e8018c2..5e5555274ac0 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -188,7 +188,25 @@ impl AsyncCx { /// SAFETY: `self` contains pointers into the `Store` with which it was /// created and must not be used after that `Store` has been disposed. pub(crate) unsafe fn suspend(&self, yield_: StoreFiberYield) -> Result<()> { - unsafe { suspend_fiber(self.current_suspend, self.current_stack_limit, yield_) } + // Take our current `Suspend` context which was configured as soon as our + // fiber started. Note that we must load it at the front here and save it on + // our stack frame. While we're polling the future other fibers may be + // started for recursive computations, and the current suspend context is + // only preserved at the edges of the fiber, not during the fiber itself. + // + // For a little bit of extra safety we also replace the current value with + // null to try to catch any accidental bugs on our part early. This is all + // pretty unsafe so we're trying to be careful... + // + // Note that there should be a segfaulting test in `async_functions.rs` if + // this `Reset` is removed. + unsafe { + let reset_suspend = Reset(self.current_suspend, *self.current_suspend); + *self.current_suspend = ptr::null_mut(); + let _reset_stack_limit = Reset(self.current_stack_limit, *self.current_stack_limit); + assert!(!(reset_suspend.1).is_null()); + (*reset_suspend.1).suspend(yield_) + } } } @@ -518,38 +536,6 @@ pub(crate) fn make_fiber<'a>( }) } -/// Suspend the current fiber, optionally returning exclusive access to the -/// specified store to the code which resumed the fiber. -/// -/// SAFETY: `suspend` must be a valid pointer. Additionally, if a store pointer -/// is provided, the fiber must give up access to the store until it is given -/// back access when next resumed. -unsafe fn suspend_fiber( - suspend: *mut *mut Suspend, StoreFiberYield, ()>, - stack_limit: *mut usize, - yield_: StoreFiberYield, -) -> Result<()> { - // Take our current `Suspend` context which was configured as soon as our - // fiber started. Note that we must load it at the front here and save it on - // our stack frame. While we're polling the future other fibers may be - // started for recursive computations, and the current suspend context is - // only preserved at the edges of the fiber, not during the fiber itself. - // - // For a little bit of extra safety we also replace the current value with - // null to try to catch any accidental bugs on our part early. This is all - // pretty unsafe so we're trying to be careful... - // - // Note that there should be a segfaulting test in `async_functions.rs` if - // this `Reset` is removed. - unsafe { - let reset_suspend = Reset(suspend, *suspend); - *suspend = ptr::null_mut(); - let _reset_stack_limit = Reset(stack_limit, *stack_limit); - assert!(!(reset_suspend.1).is_null()); - (*reset_suspend.1).suspend(yield_) - } -} - /// Run the specified function on a newly-created fiber and `.await` its /// completion. pub(crate) async fn on_fiber( From 44c96f81ee6d5a4b7dc81e93683f89e56ccdcbcf Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 09:44:23 -0700 Subject: [PATCH 15/30] Move stack limit management to `FiberResumeState` Helps remove some raw pointers that are held for a long time within `AsyncCx` --- crates/wasmtime/src/runtime/fiber.rs | 46 ++++++++++++++++++++++------ 1 file changed, 37 insertions(+), 9 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 5e5555274ac0..22a1e433fe2d 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -94,7 +94,6 @@ unsafe impl Sync for AsyncState {} /// repeatedly in a loop until the future completes. pub(crate) struct AsyncCx { current_suspend: *mut *mut Suspend, StoreFiberYield, ()>, - current_stack_limit: *mut usize, current_poll_cx: *mut PollContext, } @@ -118,7 +117,6 @@ impl AsyncCx { } else { Some(Self { current_suspend: unsafe { &raw mut (*store.async_state()).current_suspend }, - current_stack_limit: store.vm_store_context().stack_limit.get(), current_poll_cx, }) } @@ -203,7 +201,6 @@ impl AsyncCx { unsafe { let reset_suspend = Reset(self.current_suspend, *self.current_suspend); *self.current_suspend = ptr::null_mut(); - let _reset_stack_limit = Reset(self.current_stack_limit, *self.current_stack_limit); assert!(!(reset_suspend.1).is_null()); (*reset_suspend.1).suspend(yield_) } @@ -374,13 +371,26 @@ struct FiberResumeState { /// the current mask when this function is called and then restore the mask /// when the function returns (aka the fiber suspends). mpk: Option, + + /// The current wasm stack limit, if in use. + /// + /// This field stores the old of `VMStoreContext::stack_limit` that this + /// fiber should be using during its execution. This is saved/restored when + /// a fiber is suspended/resumed to ensure that when there are multiple + /// fibers within the store they all maintain an appropriate fiber-relative + /// stack limit. + stack_limit: usize, } impl FiberResumeState { - unsafe fn replace(self) -> PriorFiberResumeState { + unsafe fn replace(self, store: &mut StoreOpaque) -> PriorFiberResumeState { let tls = unsafe { self.tls.push() }; let mpk = swap_mpk_states(self.mpk); - PriorFiberResumeState { tls, mpk } + PriorFiberResumeState { + tls, + mpk, + stack_limit: store.replace_stack_limit(self.stack_limit), + } } fn dispose(self) { @@ -388,16 +398,33 @@ impl FiberResumeState { } } +impl StoreOpaque { + /// Helper function to swap the `stack_limit` field in the `VMStoreContext` + /// within this store. + fn replace_stack_limit(&mut self, stack_limit: usize) -> usize { + // SAFETY: the `VMStoreContext` points to within this store itself but + // is accessed through raw pointers to assist with Miri. The `&mut + // StoreOpaque` passed to this function shows that this has permission + // to mutate state in the store, however. + unsafe { mem::replace(&mut *self.vm_store_context().stack_limit.get(), stack_limit) } + } +} + struct PriorFiberResumeState { tls: crate::runtime::vm::PreviousAsyncWasmCallState, mpk: Option, + stack_limit: usize, } impl PriorFiberResumeState { - unsafe fn replace(self) -> FiberResumeState { + unsafe fn replace(self, store: &mut StoreOpaque) -> FiberResumeState { let tls = unsafe { self.tls.restore() }; let mpk = swap_mpk_states(self.mpk); - FiberResumeState { tls, mpk } + FiberResumeState { + tls, + mpk, + stack_limit: store.replace_stack_limit(self.stack_limit), + } } } @@ -431,13 +458,13 @@ pub(crate) fn resume_fiber<'a>( impl Drop for Restore<'_, '_> { fn drop(&mut self) { - self.fiber.state = Some(unsafe { self.state.take().unwrap().replace() }); + self.fiber.state = Some(unsafe { self.state.take().unwrap().replace(self.store) }); self.store.swap_executor(&mut self.fiber.executor); } } let result = unsafe { let _reset_suspend = Reset(fiber.suspend, *fiber.suspend); - let prev = fiber.state.take().unwrap().replace(); + let prev = fiber.state.take().unwrap().replace(store); store.swap_executor(&mut fiber.executor); let restore = Restore { store, @@ -528,6 +555,7 @@ pub(crate) fn make_fiber<'a>( } else { None }, + stack_limit: usize::MAX, }), engine, suspend, From 97e6aefb2a272b9964eb6f80fb85fa798e04ddb6 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Wed, 25 Jun 2025 10:31:28 -0600 Subject: [PATCH 16/30] add some doc comments to `fiber.rs` Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/fiber.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 22a1e433fe2d..815b4264e6be 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -244,7 +244,16 @@ pub(crate) struct StoreFiber<'a> { /// The current `Suspend` for this fiber (or null if it's not currently /// running). suspend: *mut *mut Suspend, StoreFiberYield, ()>, + /// The executor (e.g. the Pulley interpreter state) belonging to this + /// fiber. + /// + /// This is swapped with `StoreOpaque::executor` whenever this fiber is + /// resumed, suspended, or resolved. executor: Executor, + /// The id of the store with which this fiber was created. + /// + /// Any attempt to resume a fiber with a different store than the one with + /// which it was created will panic. id: StoreId, } @@ -582,6 +591,12 @@ pub(crate) async fn on_fiber( Ok(result.unwrap()) } +/// A `Future` implementation for running a `StoreFiber` to completion, giving +/// it exclusive access to its store until it resolves. +/// +/// This is used to implement `on_fiber`, where the returned `Future` closes +/// over the `&mut StoreOpaque`. It is not appropriate for use with fibers +/// which might need to release access to the store when suspending. struct FiberFuture<'a, 'b> { store: &'a mut StoreOpaque, fiber: StoreFiber<'b>, From 9d95bbbbbd010b56ffdb84abc423dbfd6f1d5ad6 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Wed, 25 Jun 2025 11:55:18 -0600 Subject: [PATCH 17/30] update `fiber.rs` and friends to match CM async requirements This adds a `resolve_or_release` function, which `Instance::resume_fiber` will use when current `concurrent.rs` stub is replaced by a real implementation. Signed-off-by: Joel Dice --- .../src/runtime/component/func/typed.rs | 2 +- .../wasmtime/src/runtime/component/store.rs | 7 + crates/wasmtime/src/runtime/fiber.rs | 169 ++++++++++++------ crates/wasmtime/src/runtime/store.rs | 19 +- 4 files changed, 135 insertions(+), 62 deletions(-) diff --git a/crates/wasmtime/src/runtime/component/func/typed.rs b/crates/wasmtime/src/runtime/component/func/typed.rs index c378742da236..f87447af7a0e 100644 --- a/crates/wasmtime/src/runtime/component/func/typed.rs +++ b/crates/wasmtime/src/runtime/component/func/typed.rs @@ -179,7 +179,7 @@ where ) -> Result where Params: Send + Sync, - Return: Send + Sync, + Return: Send + Sync + 'static, { let mut store = store.as_context_mut(); assert!( diff --git a/crates/wasmtime/src/runtime/component/store.rs b/crates/wasmtime/src/runtime/component/store.rs index f33530051924..7889334f11bd 100644 --- a/crates/wasmtime/src/runtime/component/store.rs +++ b/crates/wasmtime/src/runtime/component/store.rs @@ -28,6 +28,13 @@ impl ComponentStoreData { pub fn next_component_instance_id(&self) -> ComponentInstanceId { self.instances.next_key() } + + #[cfg(feature = "component-model-async")] + pub(crate) fn drop_fibers(store: &mut StoreOpaque) { + _ = store; + // This function will actually do something when runtime support for + // `component-model-async` is merged. + } } impl StoreData { diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 815b4264e6be..7e467d440d22 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -50,7 +50,7 @@ pub(crate) struct AsyncState { /// The `Suspend` for the current fiber (or null if no such fiber is running). /// /// See `StoreFiber` for an explanation of the signature types we use here. - current_suspend: *mut Suspend, StoreFiberYield, ()>, + current_suspend: *mut Suspend, StoreFiberYield, Result<()>>, /// See `PollContext` current_poll_cx: PollContext, @@ -93,7 +93,7 @@ unsafe impl Sync for AsyncState {} /// Used to "stackfully" poll a future by suspending the current fiber /// repeatedly in a loop until the future completes. pub(crate) struct AsyncCx { - current_suspend: *mut *mut Suspend, StoreFiberYield, ()>, + current_suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, current_poll_cx: *mut PollContext, } @@ -236,14 +236,14 @@ pub(crate) struct StoreFiber<'a> { /// /// Note also that every `StoreFiber` is implicitly granted exclusive access /// to the store when it is resumed. - fiber: Option, StoreFiberYield, ()>>, + fiber: Option, StoreFiberYield, Result<()>>>, /// See `FiberResumeState` state: Option, /// The Wasmtime `Engine` to which this fiber belongs. engine: Engine, /// The current `Suspend` for this fiber (or null if it's not currently /// running). - suspend: *mut *mut Suspend, StoreFiberYield, ()>, + suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, /// The executor (e.g. the Pulley interpreter state) belonging to this /// fiber. /// @@ -258,7 +258,7 @@ pub(crate) struct StoreFiber<'a> { } impl StoreFiber<'_> { - fn dispose(&mut self, store: &mut StoreOpaque) { + pub(crate) fn dispose(&mut self, store: &mut StoreOpaque) { if let Some(fiber) = &mut self.fiber { if !fiber.done() { let result = resume_fiber(store, self, Err(anyhow!("future dropped"))); @@ -456,7 +456,7 @@ pub(crate) fn resume_fiber<'a>( store: &mut StoreOpaque, fiber: &mut StoreFiber<'a>, result: Result<()>, -) -> Result<(), StoreFiberYield> { +) -> Result, StoreFiberYield> { assert_eq!(store.id(), fiber.id); struct Restore<'a, 'b> { @@ -520,7 +520,7 @@ pub(crate) fn resume_fiber<'a>( /// Create a new `StoreFiber` which runs the specified closure. pub(crate) fn make_fiber<'a>( store: &mut dyn VMStore, - fun: impl FnOnce(&mut dyn VMStore) + Send + Sync + 'a, + fun: impl FnOnce(&mut dyn VMStore) -> Result<()> + Send + Sync + 'a, ) -> Result> { let engine = store.engine().clone(); let executor = Executor::new(&engine); @@ -533,7 +533,7 @@ pub(crate) fn make_fiber<'a>( fiber: Some(Fiber::new(stack, move |result: Result<()>, suspend| { // Cancelled before we started? Just return. if result.is_err() { - return; + return Ok(()); } // SAFETY: Per the documented contract for @@ -584,77 +584,130 @@ pub(crate) async fn on_fiber( debug_assert!(config.async_stack_size > 0); let mut result = None; - let fiber = make_fiber(store.traitobj_mut(), |store| result = Some(func(store)))?; + let fiber = make_fiber(store.traitobj_mut(), |store| { + result = Some(func(store)); + Ok(()) + })?; - FiberFuture { store, fiber }.await; + { + let fiber = FiberFuture { + store, + fiber: Some(fiber), + on_release: OnRelease::ReturnPending, + } + .await + .unwrap(); + + debug_assert!(fiber.is_none()); + } Ok(result.unwrap()) } +/// Run the specified fiber until it either suspends with +/// `StoreFiberYield::ReleaseStore` or resolves. +/// +/// This will return `Some` if the fiber suspends with +/// `StoreFiberYield::ReleaseStore` or else `None` if it resolves. +#[cfg(feature = "component-model-async")] +// This will be used when `component-model-async` support is merged. +#[allow(dead_code)] +pub(crate) async fn resolve_or_release<'a>( + store: &mut StoreOpaque, + fiber: StoreFiber<'a>, +) -> Result>> { + FiberFuture { + store, + fiber: Some(fiber), + on_release: OnRelease::ReturnReady, + } + .await +} + +fn poll_fiber( + cx: &mut Context<'_>, + store: &mut StoreOpaque, + fiber: &mut StoreFiber<'_>, +) -> Result, StoreFiberYield> { + let poll_cx = unsafe { &raw mut (*store.async_state()).current_poll_cx }; + let _reset = Reset(poll_cx, unsafe { *poll_cx }); + let (guard_range_start, guard_range_end) = fiber + .fiber + .as_ref() + .unwrap() + .stack() + .guard_range() + .map(|r| (r.start, r.end)) + .unwrap_or((ptr::null_mut(), ptr::null_mut())); + + // We need to carry over this `cx` into our fiber's runtime for when + // it tries to poll sub-futures that are created. Doing this must be + // done unsafely, however, since `cx` is only alive for this one + // singular function call. Here we do a `transmute` to extend the + // lifetime of `Context` so it can be stored in our `Store`, and + // then we replace the current polling context with this one. + // + // Note that the replace is done for weird situations where futures + // might be switching contexts and there's multiple wasmtime futures + // in a chain of futures. + // + // On exit from this function, though, we reset the polling context + // back to what it was to signify that `Store` no longer has access + // to this pointer. + // + // SAFETY: We store the pointer to the `Context` only for the + // duration of this call and then reset it to its previous value + // afterward, thereby ensuring `fun` never sees a stale pointer. + unsafe { + *poll_cx = PollContext { + future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), + guard_range_start, + guard_range_end, + }; + } + + resume_fiber(store, fiber, Ok(())) +} + +/// Tells a `FiberFuture` what to do if `poll_fiber` returns +/// `Err(StoreFiberYield::ReleaseStore)`. +enum OnRelease { + /// Return `Poll::Pending` from `FiberFuture::poll` + ReturnPending, + /// Return `Poll::Ready` from `FiberFuture::poll`, handing ownership of the + /// `StoreFiber` to the caller. + ReturnReady, +} + /// A `Future` implementation for running a `StoreFiber` to completion, giving /// it exclusive access to its store until it resolves. -/// -/// This is used to implement `on_fiber`, where the returned `Future` closes -/// over the `&mut StoreOpaque`. It is not appropriate for use with fibers -/// which might need to release access to the store when suspending. struct FiberFuture<'a, 'b> { store: &'a mut StoreOpaque, - fiber: StoreFiber<'b>, + fiber: Option>, + on_release: OnRelease, } impl<'b> Future for FiberFuture<'_, 'b> { - type Output = (); + type Output = Result>>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let me = self.get_mut(); - let poll_cx = unsafe { &raw mut (*me.store.async_state()).current_poll_cx }; - let _reset = Reset(poll_cx, unsafe { *poll_cx }); - let (guard_range_start, guard_range_end) = me - .fiber - .fiber - .as_ref() - .unwrap() - .stack() - .guard_range() - .map(|r| (r.start, r.end)) - .unwrap_or((ptr::null_mut(), ptr::null_mut())); - - // We need to carry over this `cx` into our fiber's runtime for when - // it tries to poll sub-futures that are created. Doing this must be - // done unsafely, however, since `cx` is only alive for this one - // singular function call. Here we do a `transmute` to extend the - // lifetime of `Context` so it can be stored in our `Store`, and - // then we replace the current polling context with this one. - // - // Note that the replace is done for weird situations where futures - // might be switching contexts and there's multiple wasmtime futures - // in a chain of futures. - // - // On exit from this function, though, we reset the polling context - // back to what it was to signify that `Store` no longer has access - // to this pointer. - // - // SAFETY: We store the pointer to the `Context` only for the - // duration of this call and then reset it to its previous value - // afterward, thereby ensuring `fun` never sees a stale pointer. - unsafe { - *poll_cx = PollContext { - future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), - guard_range_start, - guard_range_end, - }; - } - - match resume_fiber(me.store, &mut me.fiber, Ok(())) { - Ok(()) => Poll::Ready(()), - Err(_) => Poll::Pending, + match poll_fiber(cx, me.store, me.fiber.as_mut().unwrap()) { + Ok(result) => Poll::Ready(result.map(|()| None)), + Err(StoreFiberYield::KeepStore) => Poll::Pending, + Err(StoreFiberYield::ReleaseStore) => match &me.on_release { + OnRelease::ReturnPending => Poll::Pending, + OnRelease::ReturnReady => Poll::Ready(Ok(me.fiber.take())), + }, } } } impl Drop for FiberFuture<'_, '_> { fn drop(&mut self) { - self.fiber.dispose(self.store); + if let Some(fiber) = &mut self.fiber { + fiber.dispose(self.store); + } } } diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index 7047543600bc..87349ea6a846 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -77,6 +77,8 @@ //! `wasmtime`, must uphold for the public interface to be safe. use crate::RootSet; +#[cfg(feature = "component-model-async")] +use crate::component::ComponentStoreData; #[cfg(feature = "async")] use crate::fiber::{self, AsyncCx}; use crate::module::RegisteredModuleId; @@ -647,9 +649,20 @@ impl Store { self.inner.data_mut() } + fn drop_everything_but_data(&mut self) { + // We need to drop the fibers of each component instance before + // attempting to drop the instances themselves since the fibers may need + // to be resumed and allowed to exit cleanly before we yank the state + // out from under them. + #[cfg(feature = "component-model-async")] + ComponentStoreData::drop_fibers(&mut self.inner); + + self.inner.flush_fiber_stack(); + } + /// Consumes this [`Store`], destroying it, and returns the underlying data. pub fn into_data(mut self) -> T { - self.inner.flush_fiber_stack(); + self.drop_everything_but_data(); // This is an unsafe operation because we want to avoid having a runtime // check or boolean for whether the data is actually contained within a @@ -2338,9 +2351,9 @@ impl fmt::Debug for Store { impl Drop for Store { fn drop(&mut self) { - self.inner.flush_fiber_stack(); + self.drop_everything_but_data(); - // for documentation on this `unsafe`, see `inrto_data`. + // for documentation on this `unsafe`, see `into_data`. unsafe { ManuallyDrop::drop(&mut self.inner.data); ManuallyDrop::drop(&mut self.inner); From 8edeb245aed3098c71bdb93945ce191659737570 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Wed, 25 Jun 2025 16:15:40 -0600 Subject: [PATCH 18/30] fix non-component-model-async build warnings Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/fiber.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 7e467d440d22..c357626c4a4b 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -676,6 +676,7 @@ enum OnRelease { ReturnPending, /// Return `Poll::Ready` from `FiberFuture::poll`, handing ownership of the /// `StoreFiber` to the caller. + #[cfg_attr(not(feature = "component-model-async"), allow(dead_code))] ReturnReady, } From 3806f680882b0055a45378e6c1c7b09bc4f95cdb Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Wed, 25 Jun 2025 16:49:47 -0600 Subject: [PATCH 19/30] make `resume_fiber` private in `fiber.rs` Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/fiber.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index c357626c4a4b..a76ea3d3dbd6 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -452,7 +452,7 @@ fn swap_mpk_states(mask: Option) -> Option { /// returned value; it will return `Err(yield_)` if the fiber suspended, where /// `yield_` indicates whether it released access to the store or not. See /// `StoreFiber::fiber` for details. -pub(crate) fn resume_fiber<'a>( +fn resume_fiber<'a>( store: &mut StoreOpaque, fiber: &mut StoreFiber<'a>, result: Result<()>, From 7afd823ad6dd37f591e57a318d613c421d7fd4ef Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 11:05:22 -0700 Subject: [PATCH 20/30] Shrink `PollContext` state Move management of the async guard range elsewhere to the normal save/restore area. --- crates/wasmtime/src/runtime/fiber.rs | 176 +++++++++--------- crates/wasmtime/src/runtime/store.rs | 10 +- crates/wasmtime/src/runtime/store/async_.rs | 5 - .../src/runtime/vm/sys/unix/signals.rs | 5 +- .../wasmtime/src/runtime/vm/traphandlers.rs | 9 - crates/wasmtime/src/runtime/vm/vmcontext.rs | 14 ++ 6 files changed, 107 insertions(+), 112 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index a76ea3d3dbd6..43589e00caca 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -30,16 +30,12 @@ impl Drop for Reset { #[derive(Clone, Copy)] struct PollContext { future_context: *mut Context<'static>, - guard_range_start: *mut u8, - guard_range_end: *mut u8, } impl Default for PollContext { fn default() -> PollContext { PollContext { future_context: ptr::null_mut(), - guard_range_start: ptr::null_mut(), - guard_range_end: ptr::null_mut(), } } } @@ -75,11 +71,6 @@ impl Default for AsyncState { } impl AsyncState { - pub(crate) fn async_guard_range(&self) -> Range<*mut u8> { - let context = self.current_poll_cx; - context.guard_range_start..context.guard_range_end - } - pub(crate) fn last_fiber_stack(&mut self) -> &mut Option { &mut self.last_fiber_stack } @@ -392,13 +383,25 @@ struct FiberResumeState { } impl FiberResumeState { - unsafe fn replace(self, store: &mut StoreOpaque) -> PriorFiberResumeState { + unsafe fn replace( + self, + store: &mut StoreOpaque, + fiber: &mut StoreFiber<'_>, + ) -> PriorFiberResumeState { let tls = unsafe { self.tls.push() }; let mpk = swap_mpk_states(self.mpk); + let async_guard_range = fiber + .fiber + .as_ref() + .unwrap() + .stack() + .guard_range() + .unwrap_or(ptr::null_mut()..ptr::null_mut()); PriorFiberResumeState { tls, mpk, stack_limit: store.replace_stack_limit(self.stack_limit), + async_guard_range: store.replace_async_guard_range(async_guard_range), } } @@ -411,11 +414,16 @@ impl StoreOpaque { /// Helper function to swap the `stack_limit` field in the `VMStoreContext` /// within this store. fn replace_stack_limit(&mut self, stack_limit: usize) -> usize { - // SAFETY: the `VMStoreContext` points to within this store itself but - // is accessed through raw pointers to assist with Miri. The `&mut - // StoreOpaque` passed to this function shows that this has permission - // to mutate state in the store, however. - unsafe { mem::replace(&mut *self.vm_store_context().stack_limit.get(), stack_limit) } + mem::replace( + &mut self.vm_store_context_mut().stack_limit.get_mut(), + stack_limit, + ) + } + + /// Helper function to swap the `async_guard_range` field in the `VMStoreContext` + /// within this store. + fn replace_async_guard_range(&mut self, range: Range<*mut u8>) -> Range<*mut u8> { + mem::replace(&mut self.vm_store_context_mut().async_guard_range, range) } } @@ -423,12 +431,16 @@ struct PriorFiberResumeState { tls: crate::runtime::vm::PreviousAsyncWasmCallState, mpk: Option, stack_limit: usize, + async_guard_range: Range<*mut u8>, } impl PriorFiberResumeState { unsafe fn replace(self, store: &mut StoreOpaque) -> FiberResumeState { let tls = unsafe { self.tls.restore() }; let mpk = swap_mpk_states(self.mpk); + // No need to save `_my_guard` since we can re-infer it from the fiber + // that this state is attached to. + let _my_guard = store.replace_async_guard_range(self.async_guard_range); FiberResumeState { tls, mpk, @@ -473,7 +485,7 @@ fn resume_fiber<'a>( } let result = unsafe { let _reset_suspend = Reset(fiber.suspend, *fiber.suspend); - let prev = fiber.state.take().unwrap().replace(store); + let prev = fiber.state.take().unwrap().replace(store, fiber); store.swap_executor(&mut fiber.executor); let restore = Restore { store, @@ -529,34 +541,34 @@ pub(crate) fn make_fiber<'a>( let suspend = unsafe { &raw mut (*store.store_opaque_mut().async_state()).current_suspend }; let track_pkey_context_switch = store.has_pkey(); let store = &raw mut *store; - Ok(StoreFiber { - fiber: Some(Fiber::new(stack, move |result: Result<()>, suspend| { - // Cancelled before we started? Just return. - if result.is_err() { - return Ok(()); - } + let fiber = Fiber::new(stack, move |result: Result<()>, suspend| { + // Cancelled before we started? Just return. + if result.is_err() { + return Ok(()); + } - // SAFETY: Per the documented contract for - // `resume_fiber`, we've been given exclusive access to - // the store until we exit or yield it back to the resumer. - let store_ref = unsafe { &mut *store }; - let suspend_ptr = - unsafe { &raw mut (*store_ref.store_opaque_mut().async_state()).current_suspend }; - // Configure our store's suspension context for the rest of the - // execution of this fiber. Note that a raw pointer is stored here - // which is only valid for the duration of this closure. - // Consequently we at least replace it with the previous value when - // we're done. This reset is also required for correctness because - // otherwise our value will overwrite another active fiber's value. - // There should be a test that segfaults in `async_functions.rs` if - // this `Reset` is removed. - // - // SAFETY: The resumer is responsible for setting - // `current_suspend` to a valid pointer. - let _reset = Reset(suspend_ptr, unsafe { *suspend_ptr }); - unsafe { *suspend_ptr = suspend }; - fun(store_ref) - })?), + // SAFETY: Per the documented contract for + // `resume_fiber`, we've been given exclusive access to + // the store until we exit or yield it back to the resumer. + let store_ref = unsafe { &mut *store }; + let suspend_ptr = + unsafe { &raw mut (*store_ref.store_opaque_mut().async_state()).current_suspend }; + // Configure our store's suspension context for the rest of the + // execution of this fiber. Note that a raw pointer is stored here + // which is only valid for the duration of this closure. + // Consequently we at least replace it with the previous value when + // we're done. This reset is also required for correctness because + // otherwise our value will overwrite another active fiber's value. + // There should be a test that segfaults in `async_functions.rs` if + // this `Reset` is removed. + // + // SAFETY: The resumer is responsible for setting + // `current_suspend` to a valid pointer. + let _reset = Reset(suspend_ptr, unsafe { *suspend_ptr }); + unsafe { *suspend_ptr = suspend }; + fun(store_ref) + })?; + Ok(StoreFiber { state: Some(FiberResumeState { tls: crate::runtime::vm::AsyncWasmCallState::new(), mpk: if track_pkey_context_switch { @@ -570,6 +582,7 @@ pub(crate) fn make_fiber<'a>( suspend, executor, id, + fiber: Some(fiber), }) } @@ -624,51 +637,6 @@ pub(crate) async fn resolve_or_release<'a>( .await } -fn poll_fiber( - cx: &mut Context<'_>, - store: &mut StoreOpaque, - fiber: &mut StoreFiber<'_>, -) -> Result, StoreFiberYield> { - let poll_cx = unsafe { &raw mut (*store.async_state()).current_poll_cx }; - let _reset = Reset(poll_cx, unsafe { *poll_cx }); - let (guard_range_start, guard_range_end) = fiber - .fiber - .as_ref() - .unwrap() - .stack() - .guard_range() - .map(|r| (r.start, r.end)) - .unwrap_or((ptr::null_mut(), ptr::null_mut())); - - // We need to carry over this `cx` into our fiber's runtime for when - // it tries to poll sub-futures that are created. Doing this must be - // done unsafely, however, since `cx` is only alive for this one - // singular function call. Here we do a `transmute` to extend the - // lifetime of `Context` so it can be stored in our `Store`, and - // then we replace the current polling context with this one. - // - // Note that the replace is done for weird situations where futures - // might be switching contexts and there's multiple wasmtime futures - // in a chain of futures. - // - // On exit from this function, though, we reset the polling context - // back to what it was to signify that `Store` no longer has access - // to this pointer. - // - // SAFETY: We store the pointer to the `Context` only for the - // duration of this call and then reset it to its previous value - // afterward, thereby ensuring `fun` never sees a stale pointer. - unsafe { - *poll_cx = PollContext { - future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), - guard_range_start, - guard_range_end, - }; - } - - resume_fiber(store, fiber, Ok(())) -} - /// Tells a `FiberFuture` what to do if `poll_fiber` returns /// `Err(StoreFiberYield::ReleaseStore)`. enum OnRelease { @@ -694,8 +662,36 @@ impl<'b> Future for FiberFuture<'_, 'b> { fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let me = self.get_mut(); - match poll_fiber(cx, me.store, me.fiber.as_mut().unwrap()) { - Ok(result) => Poll::Ready(result.map(|()| None)), + let poll_cx = unsafe { &raw mut (*me.store.async_state()).current_poll_cx }; + let _reset = Reset(poll_cx, unsafe { *poll_cx }); + + // We need to carry over this `cx` into our fiber's runtime for when + // it tries to poll sub-futures that are created. Doing this must be + // done unsafely, however, since `cx` is only alive for this one + // singular function call. Here we do a `transmute` to extend the + // lifetime of `Context` so it can be stored in our `Store`, and + // then we replace the current polling context with this one. + // + // Note that the replace is done for weird situations where futures + // might be switching contexts and there's multiple wasmtime futures + // in a chain of futures. + // + // On exit from this function, though, we reset the polling context + // back to what it was to signify that `Store` no longer has access + // to this pointer. + // + // SAFETY: We store the pointer to the `Context` only for the + // duration of this call and then reset it to its previous value + // afterward, thereby ensuring `fun` never sees a stale pointer. + unsafe { + *poll_cx = PollContext { + future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), + }; + } + + match resume_fiber(me.store, me.fiber.as_mut().unwrap(), Ok(())) { + Ok(Ok(())) => Poll::Ready(Ok(None)), + Ok(Err(e)) => Poll::Ready(Err(e)), Err(StoreFiberYield::KeepStore) => Poll::Pending, Err(StoreFiberYield::ReleaseStore) => match &me.on_release { OnRelease::ReturnPending => Poll::Pending, diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index 87349ea6a846..fff7e91e2266 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -1367,6 +1367,11 @@ impl StoreOpaque { &self.vm_store_context } + #[inline] + pub fn vm_store_context_mut(&mut self) -> &mut VMStoreContext { + &mut self.vm_store_context + } + #[inline(never)] pub(crate) fn allocate_gc_heap(&mut self) -> Result<()> { log::trace!("allocating GC heap for store {:?}", self.id()); @@ -1958,11 +1963,6 @@ at https://bytecodealliance.org/security. self.pkey.is_some() } - #[cfg(not(feature = "async"))] - pub(crate) fn async_guard_range(&self) -> core::ops::Range<*mut u8> { - core::ptr::null_mut()..core::ptr::null_mut() - } - pub(crate) fn executor(&mut self) -> ExecutorRef<'_> { match &mut self.executor { Executor::Interpreter(i) => ExecutorRef::Interpreter(i.as_interpreter_ref()), diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index 13cc11d311f3..c7de58da6cb4 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -4,7 +4,6 @@ use crate::fiber::{self, AsyncCx}; use crate::prelude::*; use crate::store::{ResourceLimiterInner, StoreInner, StoreOpaque, StoreToken}; use crate::{AsContextMut, Store, StoreContextMut, UpdateDeadline}; -use core::ops::Range; use core::pin::Pin; /// An object that can take callbacks when the runtime enters or exits hostcalls. @@ -259,10 +258,6 @@ impl StoreOpaque { } } - pub(crate) fn async_guard_range(&mut self) -> Range<*mut u8> { - unsafe { (*self.async_state()).async_guard_range() } - } - pub(crate) fn allocate_fiber_stack(&mut self) -> Result { if let Some(stack) = self.async_state.last_fiber_stack().take() { return Ok(stack); diff --git a/crates/wasmtime/src/runtime/vm/sys/unix/signals.rs b/crates/wasmtime/src/runtime/vm/sys/unix/signals.rs index 4bd800ba6ef8..a9f1033337a0 100644 --- a/crates/wasmtime/src/runtime/vm/sys/unix/signals.rs +++ b/crates/wasmtime/src/runtime/vm/sys/unix/signals.rs @@ -169,9 +169,8 @@ unsafe extern "C" fn trap_handler( let jmp_buf = match test { TrapTest::NotWasm => { if let Some(faulting_addr) = faulting_addr { - let start = info.async_guard_range.start; - let end = info.async_guard_range.end; - if start as usize <= faulting_addr && faulting_addr < end as usize { + let range = &info.vm_store_context.as_ref().async_guard_range; + if range.start.addr() <= faulting_addr && faulting_addr < range.end.addr() { abort_stack_overflow(); } } diff --git a/crates/wasmtime/src/runtime/vm/traphandlers.rs b/crates/wasmtime/src/runtime/vm/traphandlers.rs index 4c57fa7a05d1..a96a747efe67 100644 --- a/crates/wasmtime/src/runtime/vm/traphandlers.rs +++ b/crates/wasmtime/src/runtime/vm/traphandlers.rs @@ -23,7 +23,6 @@ use crate::{EntryStoreContext, prelude::*}; use crate::{StoreContextMut, WasmBacktrace}; use core::cell::Cell; use core::num::NonZeroU32; -use core::ops::Range; use core::ptr::{self, NonNull}; pub use self::backtrace::Backtrace; @@ -477,8 +476,6 @@ mod call_thread_state { pub(crate) unwinder: &'static dyn Unwind, pub(super) prev: Cell, - #[cfg(all(has_native_signals, unix))] - pub(crate) async_guard_range: Range<*mut u8>, // The state of the runtime for the *previous* `CallThreadState` for // this same store. Our *current* state is saved in `self.vm_store_context`, @@ -507,10 +504,6 @@ mod call_thread_state { store: &mut StoreOpaque, old_state: *const EntryStoreContext, ) -> CallThreadState { - // Don't try to plumb #[cfg] everywhere for this field, just pretend - // we're using it on miri/windows to silence compiler warnings. - let _: Range<_> = store.async_guard_range(); - CallThreadState { unwind: Cell::new(None), unwinder: store.unwinder(), @@ -521,8 +514,6 @@ mod call_thread_state { #[cfg(feature = "coredump")] capture_coredump: store.engine().config().coredump_on_trap, vm_store_context: store.vm_store_context_ptr(), - #[cfg(all(has_native_signals, unix))] - async_guard_range: store.async_guard_range(), prev: Cell::new(ptr::null()), old_state, } diff --git a/crates/wasmtime/src/runtime/vm/vmcontext.rs b/crates/wasmtime/src/runtime/vm/vmcontext.rs index 9af83b1db8cd..79e0dbf837ac 100644 --- a/crates/wasmtime/src/runtime/vm/vmcontext.rs +++ b/crates/wasmtime/src/runtime/vm/vmcontext.rs @@ -13,6 +13,7 @@ use core::ffi::c_void; use core::fmt; use core::marker; use core::mem::{self, MaybeUninit}; +use core::ops::Range; use core::ptr::{self, NonNull}; use core::sync::atomic::{AtomicUsize, Ordering}; use wasmtime_environ::{ @@ -1114,6 +1115,18 @@ pub struct VMStoreContext { /// Stack information used by stack switching instructions. See documentation /// on `VMStackChain` for details. pub stack_chain: UnsafeCell, + + /// The range, in addresses, of the guard page that is currently in use. + /// + /// This field is used when signal handlers are run to determine whether a + /// faulting address lies within the guard page of an async stack for + /// example. If this happens then the signal handler aborts with a stack + /// overflow message similar to what would happen had the stack overflow + /// happened on the main thread. This field is, by default a null..null + /// range indicating that no async guard is in use (aka no fiber). In such a + /// situation while this field is read it'll never classify a fault as an + /// guard page fault. + pub async_guard_range: Range<*mut u8>, } // The `VMStoreContext` type is a pod-type with no destructor, and we don't @@ -1140,6 +1153,7 @@ impl Default for VMStoreContext { last_wasm_exit_pc: UnsafeCell::new(0), last_wasm_entry_fp: UnsafeCell::new(0), stack_chain: UnsafeCell::new(VMStackChain::Absent), + async_guard_range: ptr::null_mut()..ptr::null_mut(), } } } From 057a9238f88e8b5c608d66132ea812d67a52648e Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Wed, 25 Jun 2025 15:57:15 -0700 Subject: [PATCH 21/30] Refactor `AsyncCx`, reduce `unsafe` * Remove the `AsyncCx` type from Wasmtime as it's inherently `unsafe` to use, instead bundle operations directly on a `Store*` reference. * Don't retain pointers-to-pointers within the roughly-equivalent `BlockingContext` created in this PR. Instead when a blocking context is created "take" the metadata from the store to assert exclusive ownership of the pointers. * Refactor how `&mut Context<'_>` is passed around, namely thread it through fiber parameters to model resumption as registering a new context to poll with. * Remove `PollContext` in favor of directly storing a pointer as it's now mostly an empty structure. * Minor refactorings to make things more future-refactorable and/or clear in a few places. * Refactor management of the "current suspend" and "current future context" pointers. These are now null'd out on resumption and asserted null on suspension. * Remove the need for a generic `Reset` structure in the fiber bits as it's a pretty dangerous structure to have in general. The end result of this refactoring is that all usage of `block_on` is now safe and additionally many of the internals of the implementation are safer than they were before --- .../wasmtime/src/runtime/component/linker.rs | 30 +- crates/wasmtime/src/runtime/fiber.rs | 586 +++++++++++++----- crates/wasmtime/src/runtime/func.rs | 45 +- crates/wasmtime/src/runtime/linker.rs | 38 +- crates/wasmtime/src/runtime/store.rs | 70 +-- crates/wasmtime/src/runtime/store/async_.rs | 13 +- crates/wasmtime/src/runtime/store/gc.rs | 8 +- 7 files changed, 500 insertions(+), 290 deletions(-) diff --git a/crates/wasmtime/src/runtime/component/linker.rs b/crates/wasmtime/src/runtime/component/linker.rs index 09241f8bd32b..30c5fe6b9356 100644 --- a/crates/wasmtime/src/runtime/component/linker.rs +++ b/crates/wasmtime/src/runtime/component/linker.rs @@ -7,8 +7,6 @@ use crate::component::types; use crate::component::{ Component, ComponentNamedList, Instance, InstancePre, Lift, Lower, ResourceType, Val, }; -#[cfg(feature = "async")] -use crate::fiber::AsyncCx; use crate::hash_map::HashMap; use crate::prelude::*; use crate::{AsContextMut, Engine, Module, StoreContextMut}; @@ -443,12 +441,8 @@ impl LinkerInstance<'_, T> { self.engine.config().async_support, "cannot use `func_wrap_async` without enabling async support in the config" ); - let ff = move |mut store: StoreContextMut<'_, T>, params: Params| -> Result { - let async_cx = AsyncCx::new(&mut store.0); - let mut future = Pin::from(f(store.as_context_mut(), params)); - // SAFETY: The `Store` we used to create the `AsyncCx` above remains - // valid. - unsafe { async_cx.block_on(future.as_mut())? } + let ff = move |store: StoreContextMut<'_, T>, params: Params| -> Result { + store.block_on(|store| f(store, params).into())? }; self.func_wrap(name, ff) } @@ -607,14 +601,10 @@ impl LinkerInstance<'_, T> { self.engine.config().async_support, "cannot use `func_new_async` without enabling async support in the config" ); - let ff = move |mut store: StoreContextMut<'_, T>, params: &[Val], results: &mut [Val]| { - let async_cx = AsyncCx::new(&mut store.0); - let mut future = Pin::from(f(store.as_context_mut(), params, results)); - // SAFETY: The `Store` we used to create the `AsyncCx` above remains - // valid. - unsafe { async_cx.block_on(future.as_mut())? } + let ff = move |store: StoreContextMut<'_, T>, params: &[Val], results: &mut [Val]| { + store.with_blocking(|store, cx| cx.block_on(Pin::from(f(store, params, results)))?) }; - self.func_new(name, ff) + return self.func_new(name, ff); } /// Defines a [`Module`] within this instance. @@ -682,14 +672,8 @@ impl LinkerInstance<'_, T> { let dtor = Arc::new(crate::func::HostFunc::wrap_inner( &self.engine, move |mut cx: crate::Caller<'_, T>, (param,): (u32,)| { - let async_cx = AsyncCx::new(&mut cx.as_context_mut().0); - let mut future = Pin::from(dtor(cx.as_context_mut(), param)); - // SAFETY: The `Store` we used to create the `AsyncCx` above - // remains valid. - match unsafe { async_cx.block_on(future.as_mut()) } { - Ok(Ok(())) => Ok(()), - Ok(Err(trap)) | Err(trap) => Err(trap), - } + cx.as_context_mut() + .block_on(|store| dtor(store, param).into())? }, )); self.insert(name, Definition::Resource(ty, dtor))?; diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 43589e00caca..b2a8043354de 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -1,9 +1,10 @@ #![deny(unsafe_op_in_unsafe_fn)] -use crate::Engine; -use crate::store::{Executor, StoreId, StoreOpaque}; +use crate::prelude::*; +use crate::store::{Executor, StoreId, StoreInner, StoreOpaque}; use crate::vm::mpk::{self, ProtectionMask}; use crate::vm::{AsyncWasmCallState, VMStore}; +use crate::{Engine, StoreContextMut}; use anyhow::{Result, anyhow}; use core::mem; use core::ops::Range; @@ -12,44 +13,71 @@ use core::ptr; use core::task::{Context, Poll}; use wasmtime_fiber::{Fiber, Suspend}; -/// Helper struct for reseting a raw pointer to its original value on drop. -struct Reset(*mut T, T); +type WasmtimeResume = Result<*mut Context<'static>>; +type WasmtimeYield = StoreFiberYield; +type WasmtimeComplete = Result<()>; +type WasmtimeSuspend = Suspend; -impl Drop for Reset { - fn drop(&mut self) { - unsafe { - *self.0 = self.1; - } - } -} - -/// Represents the context of a `Future::poll` operation which involves resuming -/// a fiber. +/// State related to asynchronous computations stored within a `Store`. /// -/// See `self::poll_fn` for details. -#[derive(Clone, Copy)] -struct PollContext { - future_context: *mut Context<'static>, -} - -impl Default for PollContext { - fn default() -> PollContext { - PollContext { - future_context: ptr::null_mut(), - } - } -} - -/// Represents the state of a currently executing fiber which has been resumed -/// via `self::poll_fn`. +/// This structure resides inside of a `Store` and is used to manage the +/// various pieces of state associated with asynchronous computations. Chiefly +/// this manages the `WasmtimeSuspend` pointer as well as `&mut Context<'_>` +/// when polling futures. This serves as storage to use these pointers across a +/// WebAssembly function boundary, for example, where the values cannot +/// otherwise be explicitly threaded through. pub(crate) struct AsyncState { - /// The `Suspend` for the current fiber (or null if no such fiber is running). + /// The `Suspend` for the current fiber (or null if no such fiber is + /// running). /// - /// See `StoreFiber` for an explanation of the signature types we use here. - current_suspend: *mut Suspend, StoreFiberYield, Result<()>>, - - /// See `PollContext` - current_poll_cx: PollContext, + /// This pointer is provided by the `wasmtime_fiber` crate when a fiber + /// first starts, but this pointer is unable to be carried through + /// WebAssembly frames for example. This serves as an alternative storage + /// location for the pointer provided by `wasmtime_fiber` within a fiber's + /// execution. + /// + /// This pointer is null when a fiber is not executing, but it is also null + /// when a `BlockingContext` is created. Note that when a fiber is suspended + /// it's always through a `BlockingContext` so this field is null whenever a + /// fiber is suspended as well. Fiber resumption will save the prior value + /// in a store and then set it to null, where suspension will then restore + /// what was previously in the store. + current_suspend: *mut WasmtimeSuspend, + + /// The `Context` pointer last provided in `Future for FiberFuture`. + /// + /// Like `current_suspend` above this is an example of a piece of context + /// which needs to be carried over a WebAssembly function frame which + /// otherwise doesn't take this as a parameter. This differs from + /// `current_suspend` though in that it is provided as part of a `Future` + /// poll operation but is "gone" after that poll operation completes. That + /// means that while `current_suspend` is the same for the lifetime of a + /// future this field is always changing. + /// + /// Like `current_suspend` though this is null either when a fiber isn't + /// running or when a `BlockingContext` is created (in which case this is + /// "take"en). That means that this is null on suspension/resumption of a + /// fiber. + /// + /// The value for this pointer is threaded directly through the + /// `WasmtimeResume` type which is how a pointer flows into this field from + /// a future-related poll call. This means that the `BlockingContext` + /// creation may take one value of a pointer here but restore another. That + /// would represent suspending in one call to `Future::poll` and then + /// resuming some time later in a different call to `Future::poll`. + /// + /// # Safety + /// + /// Note that this is a pretty unsafe field for two reasons. One is that + /// it's a raw pointer to a `Context` provided ephemerally to some call to + /// `Future::poll` on the stack. Another reason is that the lifetime + /// parameter of `Context` is unsafely changed to `'static` here which is + /// not correct. The ephemeral nature of this pointer is managed through the + /// take-style operations in `BlockingContext` and the `'static` lifetime is + /// handled by ensuring the signatures that work with `BlockingContext` all + /// use constrained anonymous lifetimes that are guaranteed to be shorter + /// than the original `Context` lifetime. + current_future_cx: *mut Context<'static>, /// The last fiber stack that was in use by the store. /// @@ -60,11 +88,18 @@ pub(crate) struct AsyncState { last_fiber_stack: Option, } +// Lots of pesky unsafe cells and pointers in this structure. This means we need +// to declare explicitly that we use this in a threadsafe fashion. +// +// TODO: replace pointers above with `SendSyncPtr<..>` and remove this +unsafe impl Send for AsyncState {} +unsafe impl Sync for AsyncState {} + impl Default for AsyncState { fn default() -> Self { Self { current_suspend: ptr::null_mut(), - current_poll_cx: PollContext::default(), + current_future_cx: ptr::null_mut(), last_fiber_stack: None, } } @@ -76,56 +111,143 @@ impl AsyncState { } } -// Lots of pesky unsafe cells and pointers in this structure. This means we need -// to declare explicitly that we use this in a threadsafe fashion. -unsafe impl Send for AsyncState {} -unsafe impl Sync for AsyncState {} - -/// Used to "stackfully" poll a future by suspending the current fiber -/// repeatedly in a loop until the future completes. -pub(crate) struct AsyncCx { - current_suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, - current_poll_cx: *mut PollContext, +trait AsStoreOpaque { + fn as_store_opaque(&mut self) -> &mut StoreOpaque; } -impl AsyncCx { - /// Create a new `AsyncCx`. - /// - /// This will panic if called outside the scope of a `self::poll_fn` call. - /// Consider using `Self::try_new` instead to avoid panicking. - pub(crate) fn new(store: &mut StoreOpaque) -> Self { - Self::try_new(store).unwrap() +impl AsStoreOpaque for StoreOpaque { + fn as_store_opaque(&mut self) -> &mut StoreOpaque { + self } +} - /// Create a new `AsyncCx`. - /// - /// This will return `None` if called outside the scope of a `self::poll_fn` - /// call. - pub(crate) fn try_new(store: &mut StoreOpaque) -> Option { - let current_poll_cx = unsafe { &raw mut (*store.async_state()).current_poll_cx }; - if unsafe { (*current_poll_cx).future_context.is_null() } { - None - } else { - Some(Self { - current_suspend: unsafe { &raw mut (*store.async_state()).current_suspend }, - current_poll_cx, - }) - } +impl AsStoreOpaque for StoreInner { + fn as_store_opaque(&mut self) -> &mut StoreOpaque { + self } +} - /// Poll the specified future using `Self::current_poll_cx`. +/// A helper structure used to block a fiber. +/// +/// This is acquired via either `StoreContextMut::with_blocking` or +/// `StoreOpaque::with_blocking`. This structure represents the "taken" state of +/// pointers from a store's `AsyncState`, then modeling them as safe pointers. +/// +/// Note that the lifetimes here are carefully controlled in instances of this +/// structure through the construction of the `with` function. +pub(crate) struct BlockingContext<'a, 'b> { + /// Pointer to `wasmtime_fiber::Suspend` which was supplied when a fiber + /// first started. + /// + /// When a `BlockingContext` is first created this pointer is "taken" from + /// the store (the store is null'd out) and then the raw pointer previously + /// in the store is unsafely transformed to this safe pointer. This + /// represents how a `BlockingContext` temporarily has access to this + /// suspend but when the `BlockingContext` goes away this'll make its way + /// back into the store. + suspend: &'a mut WasmtimeSuspend, + + /// Pointer to the future `Context` that this fiber is being polled with. + /// + /// Similar to `suspend` above this is taken from a store when a + /// `BlockingContext` is created and it's restored when the + /// `BlockingContext` goes away. Note though that unlike `suspend`, as + /// alluded to in the documentation on `AsyncState`, this value changes over + /// time as calls to poll are made. This field becomes `None` during a + /// suspension because that means that the context is released and no longer + /// available. Upon resumption the context here is *optionally* provided. + /// Cancellation is a case where it isn't passed back and a re-poll is a + /// case where it's passed back. + future_cx: Option<&'a mut Context<'b>>, +} + +impl<'a, 'b> BlockingContext<'a, 'b> { + /// Method to go from a `store` provided (which internally contains a + /// `StoreOpaque`) to a `BlockingContext`. + /// + /// This function will "take" context from `store`'s `AsyncState` field. It + /// will then construct a `BlockingContext` and yield it to the closure `f` + /// provided. The closure can then block on futures, suspend, etc. + /// + /// Upon return of the closure `f` the state from `BlockingContext` is + /// restored within the store. The return value of `f` is the return value + /// of this function. + /// + /// Note that the `store` must be provided to this function as an argument + /// to originally acquire state from `AsyncState`. This store is then + /// supplied back to the closure `f` provided here so the store can be used + /// to construct an asynchronous or blocking computation which the + /// `BlockingContext` tries to block on. /// - /// This will panic if called recursively using the same `AsyncState`. + /// # Safety /// - /// SAFETY: `self` contains pointers into the `Store` with which it was - /// created and must not be used after that `Store` has been disposed. - unsafe fn poll(&self, mut future: Pin<&mut (dyn Future + Send)>) -> Poll { + /// This method is safe to call at any time, but it's worth noting that the + /// safety of this function relies on the signature of this function. + /// Notably the lifetime parameters of `BlockingContext` in the `f` closure + /// here must be anonymous. That ensures that the `BlockingContext` that + /// callers get access to cannot be persisted outside of that closure call + /// and everything is scoped to just the closure `f` provided with nothing + /// escaping. + fn with(store: &mut S, f: impl FnOnce(&mut S, &mut BlockingContext<'_, '_>) -> R) -> R + where + S: AsStoreOpaque, + { + let opaque = store.as_store_opaque(); + let future_cx; + let suspend; + + // SAFETY: this is taking pointers from `AsyncState` and then unsafely + // turning them into safe references. Lifetime-wise this should be safe + // because the inferred lifetimes for all these pointers is constrained + // by the signature of `f` provided here. That ensures that everything + // is scoped purely to the closure `f` and nothing should be persisted + // outside of this function call. This, for example, ensures that the + // `Context<'static>` doesn't leak out, it's only with an anonymous + // lifetime that's forcibly shorter. + // + // Provenance-wise this should be safe as if these fields in the store + // are non-null then the pointers are provided up-the-stack on this + // fiber and for this fiber. The "take" pattern here ensures that if + // this `BlockingContext` context acquires the pointers then there are + // no other instances of these pointers in use anywhere else. unsafe { - let poll_cx = *self.current_poll_cx; - let _reset = Reset(self.current_poll_cx, poll_cx); - *self.current_poll_cx = PollContext::default(); - assert!(!poll_cx.future_context.is_null()); - future.as_mut().poll(&mut *poll_cx.future_context) + let state = opaque.async_state(); + assert!(!(*state).current_future_cx.is_null()); + assert!(!(*state).current_suspend.is_null()); + future_cx = Some(&mut *(*state).current_future_cx); + (*state).current_future_cx = ptr::null_mut(); + suspend = &mut *(*state).current_suspend; + (*state).current_suspend = ptr::null_mut(); + } + + let mut reset = ResetBlockingContext { + store, + cx: BlockingContext { future_cx, suspend }, + }; + return f(&mut reset.store, &mut reset.cx); + + struct ResetBlockingContext<'a, 'b, S: AsStoreOpaque> { + store: &'a mut S, + cx: BlockingContext<'a, 'b>, + } + + impl Drop for ResetBlockingContext<'_, '_, S> { + fn drop(&mut self) { + let store = self.store.as_store_opaque(); + let state = store.async_state(); + + // SAFETY: TODO need to remove the unsafety from + // reading/writing `async_state`. Otherwise this is all state + // owned by the store. + unsafe { + debug_assert!((*state).current_future_cx.is_null()); + debug_assert!((*state).current_suspend.is_null()); + (*state).current_suspend = self.cx.suspend; + if let Some(cx) = &mut self.cx.future_cx { + (*state).current_future_cx = change_context_lifetime(cx); + } + } + } } } @@ -157,44 +279,140 @@ impl AsyncCx { /// suspend and release the store to allow other tasks to run before this /// fiber is resumed. /// - /// SAFETY: `self` contains pointers into the `Store` with which it was - /// created and must not be used after that `Store` has been disposed. - pub(crate) unsafe fn block_on( - &self, - mut future: Pin<&mut (dyn Future + Send)>, - ) -> Result { + /// # Return Value + /// + /// A return value of `Ok(value)` means that the future completed with + /// `value`. A return value of `Err(e)` means that the fiber and its future + /// have been cancelled and the fiber needs to exit and complete ASAP. + /// + /// # Safety + /// + /// This function is safe to call at any time but relies on a trait bound + /// that is manually placed here the compiler does not otherwise require. + /// Notably the `Send` bound on the future provided here is not required + /// insofar as things compile without that. The purpose of this, however, is + /// to make the `unsafe impl Send for StoreFiber` more safe. The `future` + /// here is state that is stored on the stack during the suspension of this + /// fiber and is otherwise not visible to the compiler. By having a `Send` + /// bound here it ensures that the future doesn't have things like `Rc` or + /// similar pointing into thread locals which would not be sound if this + /// fiber crosses threads. + pub(crate) fn block_on(&mut self, future: F) -> Result + where + F: Future + Send, + { + let mut future = core::pin::pin!(future); loop { - match unsafe { self.poll(future.as_mut()) } { + match future.as_mut().poll(self.future_cx.as_mut().unwrap()) { Poll::Ready(v) => break Ok(v), - Poll::Pending => unsafe { self.suspend(StoreFiberYield::KeepStore)? }, + Poll::Pending => self.suspend(StoreFiberYield::KeepStore)?, } } } - /// Suspend the current fiber, optionally transfering exclusive access to - /// the store back to the code which resumed it. - /// - /// SAFETY: `self` contains pointers into the `Store` with which it was - /// created and must not be used after that `Store` has been disposed. - pub(crate) unsafe fn suspend(&self, yield_: StoreFiberYield) -> Result<()> { - // Take our current `Suspend` context which was configured as soon as our - // fiber started. Note that we must load it at the front here and save it on - // our stack frame. While we're polling the future other fibers may be - // started for recursive computations, and the current suspend context is - // only preserved at the edges of the fiber, not during the fiber itself. - // - // For a little bit of extra safety we also replace the current value with - // null to try to catch any accidental bugs on our part early. This is all - // pretty unsafe so we're trying to be careful... - // - // Note that there should be a segfaulting test in `async_functions.rs` if - // this `Reset` is removed. + /// Suspend this fiber with `yield_` as the reason. + /// + /// This function will suspend the current fiber and only return after the + /// fiber has resumed. This function return `Ok(())` if the fiber was + /// resumed to be completed, and `Err(e)` indicates that the fiber has been + /// cancelled and needs to exit/complete ASAP. + pub(crate) fn suspend(&mut self, yield_: StoreFiberYield) -> Result<()> { + // Over a suspension point we're guaranteed that the `Context` provided + // here is no longer valid, so discard it. If we're supposed to be able + // to poll afterwards this will be given back as part of the resume + // value given back. + self.future_cx.take(); + + let new_future_cx: *mut Context<'static> = self.suspend.suspend(yield_)?; + + // SAFETY: this function is unsafe as we're doing "funky" things to the + // `new_future_cx` we have been given. The safety here relies on the + // fact that the lifetimes of `BlockingContext` are all "smaller" than + // the original `Context` itself, and that should be guaranteed through + // the exclusive constructor of this type `BlockingContext::with`. unsafe { - let reset_suspend = Reset(self.current_suspend, *self.current_suspend); - *self.current_suspend = ptr::null_mut(); - assert!(!(reset_suspend.1).is_null()); - (*reset_suspend.1).suspend(yield_) + self.future_cx = Some(change_context_lifetime(&mut *new_future_cx)); } + Ok(()) + } +} + +impl StoreContextMut<'_, T> { + /// Blocks on the future computed by `f`. + /// + /// # Panics + /// + /// Panics if this is invoked outside the context of a fiber. + pub(crate) fn block_on( + self, + f: impl FnOnce(StoreContextMut<'_, T>) -> Pin + Send + '_>>, + ) -> Result { + BlockingContext::with(self.0, |store, cx| { + cx.block_on(f(StoreContextMut(store)).as_mut()) + }) + } + + /// Creates a `BlockingContext` suitable for blocking on futures or + /// suspending the current fiber. + /// + /// # Panics + /// + /// Panics if this is invoked outside the context of a fiber. + pub(crate) fn with_blocking( + self, + f: impl FnOnce(StoreContextMut<'_, T>, &mut BlockingContext<'_, '_>) -> R, + ) -> R { + BlockingContext::with(self.0, |store, cx| f(StoreContextMut(store), cx)) + } +} + +impl crate::store::StoreInner { + /// Blocks on the future computed by `f`. + /// + /// # Panics + /// + /// Panics if this is invoked outside the context of a fiber. + pub(crate) fn block_on( + &mut self, + f: impl FnOnce(StoreContextMut<'_, T>) -> Pin + Send + '_>>, + ) -> Result { + BlockingContext::with(self, |store, cx| { + cx.block_on(f(StoreContextMut(store)).as_mut()) + }) + } +} + +impl StoreOpaque { + /// Blocks on the future computed by `f`. + /// + /// # Panics + /// + /// Panics if this is invoked outside the context of a fiber. + pub(crate) fn block_on( + &mut self, + f: impl FnOnce(&mut Self) -> Pin + Send + '_>>, + ) -> Result { + BlockingContext::with(self, |store, cx| cx.block_on(f(store).as_mut())) + } + + /// Creates a `BlockingContext` suitable for blocking on futures or + /// suspending the current fiber. + /// + /// # Panics + /// + /// Panics if this is invoked outside the context of a fiber. + #[expect(dead_code, reason = "will be used by async things soon")] + pub(crate) fn with_blocking( + &mut self, + f: impl FnOnce(&mut Self, &mut BlockingContext<'_, '_>) -> R, + ) -> R { + BlockingContext::with(self, |store, cx| f(store, cx)) + } + + /// Returns whether `block_on` will succeed or panic. + pub(crate) fn can_block(&mut self) -> bool { + let current_future_cx = unsafe { (*self.async_state()).current_future_cx }; + !current_future_cx.is_null() } } @@ -227,14 +445,11 @@ pub(crate) struct StoreFiber<'a> { /// /// Note also that every `StoreFiber` is implicitly granted exclusive access /// to the store when it is resumed. - fiber: Option, StoreFiberYield, Result<()>>>, + fiber: Option>, /// See `FiberResumeState` state: Option, /// The Wasmtime `Engine` to which this fiber belongs. engine: Engine, - /// The current `Suspend` for this fiber (or null if it's not currently - /// running). - suspend: *mut *mut Suspend, StoreFiberYield, Result<()>>, /// The executor (e.g. the Pulley interpreter state) belonging to this /// fiber. /// @@ -402,6 +617,12 @@ impl FiberResumeState { mpk, stack_limit: store.replace_stack_limit(self.stack_limit), async_guard_range: store.replace_async_guard_range(async_guard_range), + + // The current suspend/future_cx are always null upon resumption, so + // insert null. Save the old values through to get preserved across + // this resume/suspend. + current_suspend: store.replace_current_suspend(ptr::null_mut()), + current_future_cx: store.replace_current_future_cx(ptr::null_mut()), } } @@ -425,6 +646,14 @@ impl StoreOpaque { fn replace_async_guard_range(&mut self, range: Range<*mut u8>) -> Range<*mut u8> { mem::replace(&mut self.vm_store_context_mut().async_guard_range, range) } + + fn replace_current_suspend(&mut self, ptr: *mut WasmtimeSuspend) -> *mut WasmtimeSuspend { + unsafe { mem::replace(&mut (*self.async_state()).current_suspend, ptr) } + } + + fn replace_current_future_cx(&mut self, ptr: *mut Context<'static>) -> *mut Context<'static> { + unsafe { mem::replace(&mut (*self.async_state()).current_future_cx, ptr) } + } } struct PriorFiberResumeState { @@ -432,6 +661,8 @@ struct PriorFiberResumeState { mpk: Option, stack_limit: usize, async_guard_range: Range<*mut u8>, + current_suspend: *mut WasmtimeSuspend, + current_future_cx: *mut Context<'static>, } impl PriorFiberResumeState { @@ -441,6 +672,15 @@ impl PriorFiberResumeState { // No need to save `_my_guard` since we can re-infer it from the fiber // that this state is attached to. let _my_guard = store.replace_async_guard_range(self.async_guard_range); + + // Restore the previous values of current_{suspend,future_cx} but we + // should be guaranteed that the prior values are null, so double-check + // that here. + let prev = store.replace_current_suspend(self.current_suspend); + assert!(prev.is_null()); + let prev = store.replace_current_future_cx(self.current_future_cx); + assert!(prev.is_null()); + FiberResumeState { tls, mpk, @@ -467,8 +707,8 @@ fn swap_mpk_states(mask: Option) -> Option { fn resume_fiber<'a>( store: &mut StoreOpaque, fiber: &mut StoreFiber<'a>, - result: Result<()>, -) -> Result, StoreFiberYield> { + result: WasmtimeResume, +) -> Result { assert_eq!(store.id(), fiber.id); struct Restore<'a, 'b> { @@ -484,7 +724,6 @@ fn resume_fiber<'a>( } } let result = unsafe { - let _reset_suspend = Reset(fiber.suspend, *fiber.suspend); let prev = fiber.state.take().unwrap().replace(store, fiber); store.swap_executor(&mut fiber.executor); let restore = Restore { @@ -538,35 +777,57 @@ pub(crate) fn make_fiber<'a>( let executor = Executor::new(&engine); let id = store.store_opaque().id(); let stack = store.store_opaque_mut().allocate_fiber_stack()?; - let suspend = unsafe { &raw mut (*store.store_opaque_mut().async_state()).current_suspend }; let track_pkey_context_switch = store.has_pkey(); let store = &raw mut *store; - let fiber = Fiber::new(stack, move |result: Result<()>, suspend| { - // Cancelled before we started? Just return. - if result.is_err() { - return Ok(()); - } + let fiber = Fiber::new(stack, move |result: WasmtimeResume, suspend| { + let future_cx = match result { + Ok(cx) => cx, + // Cancelled before we started? Just return. + Err(_) => return Ok(()), + }; // SAFETY: Per the documented contract for // `resume_fiber`, we've been given exclusive access to // the store until we exit or yield it back to the resumer. let store_ref = unsafe { &mut *store }; - let suspend_ptr = - unsafe { &raw mut (*store_ref.store_opaque_mut().async_state()).current_suspend }; - // Configure our store's suspension context for the rest of the - // execution of this fiber. Note that a raw pointer is stored here - // which is only valid for the duration of this closure. - // Consequently we at least replace it with the previous value when - // we're done. This reset is also required for correctness because - // otherwise our value will overwrite another active fiber's value. - // There should be a test that segfaults in `async_functions.rs` if - // this `Reset` is removed. + let async_state = store_ref.store_opaque_mut().async_state(); + + // It should be a guarantee that the store has null pointers here upon + // starting a fiber, so now's the time to fill in the pointers now that + // the fiber is running and `future_cx` and `suspend` are both in scope. + // Note that these pointers are removed when this function returns as + // that's when they fall out of scope. // - // SAFETY: The resumer is responsible for setting - // `current_suspend` to a valid pointer. - let _reset = Reset(suspend_ptr, unsafe { *suspend_ptr }); - unsafe { *suspend_ptr = suspend }; - fun(store_ref) + // SAFETY: TODO make manipulation of `AsyncState` safer. + unsafe { + assert!((*async_state).current_suspend.is_null()); + assert!((*async_state).current_future_cx.is_null()); + + (*async_state).current_suspend = suspend; + (*async_state).current_future_cx = future_cx; + } + + struct ResetCurrentPointersToNull<'a>(&'a mut dyn VMStore); + + impl Drop for ResetCurrentPointersToNull<'_> { + fn drop(&mut self) { + let state = self.0.async_state(); + unsafe { + // Double-check that the current suspension isn't null (it + // should be what's in this closure). Note though that we + // can't check `current_future_cx` because it may either be + // here or not be here depending on whether this was + // cancelled or not. + debug_assert!(!(*state).current_suspend.is_null()); + + (*state).current_suspend = ptr::null_mut(); + (*state).current_future_cx = ptr::null_mut(); + } + } + } + let reset = ResetCurrentPointersToNull(store_ref); + + fun(reset.0) })?; Ok(StoreFiber { state: Some(FiberResumeState { @@ -579,7 +840,6 @@ pub(crate) fn make_fiber<'a>( stack_limit: usize::MAX, }), engine, - suspend, executor, id, fiber: Some(fiber), @@ -662,34 +922,21 @@ impl<'b> Future for FiberFuture<'_, 'b> { fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let me = self.get_mut(); - let poll_cx = unsafe { &raw mut (*me.store.async_state()).current_poll_cx }; - let _reset = Reset(poll_cx, unsafe { *poll_cx }); - - // We need to carry over this `cx` into our fiber's runtime for when - // it tries to poll sub-futures that are created. Doing this must be - // done unsafely, however, since `cx` is only alive for this one + // SAFETY: We need to carry over this `cx` into our fiber's runtime for + // when it tries to poll sub-futures that are created. Doing this must + // be done unsafely, however, since `cx` is only alive for this one // singular function call. Here we do a `transmute` to extend the - // lifetime of `Context` so it can be stored in our `Store`, and - // then we replace the current polling context with this one. - // - // Note that the replace is done for weird situations where futures - // might be switching contexts and there's multiple wasmtime futures - // in a chain of futures. - // - // On exit from this function, though, we reset the polling context - // back to what it was to signify that `Store` no longer has access - // to this pointer. + // lifetime of `Context` so it can be stored in our `Store`, and then we + // replace the current polling context with this one. // - // SAFETY: We store the pointer to the `Context` only for the - // duration of this call and then reset it to its previous value - // afterward, thereby ensuring `fun` never sees a stale pointer. - unsafe { - *poll_cx = PollContext { - future_context: mem::transmute::<&mut Context<'_>, *mut Context<'static>>(cx), - }; - } - - match resume_fiber(me.store, me.fiber.as_mut().unwrap(), Ok(())) { + // The safety of this extension relies on never actually using + // `Context<'static>` with `'static` actually there, which should be + // satisfied by the users of this in the `BlockingContext` structure + // where the lifetime parameters there are always more constrained than + // they are here. + let cx: *mut Context<'static> = unsafe { change_context_lifetime(cx) }; + + match resume_fiber(me.store, me.fiber.as_mut().unwrap(), Ok(cx)) { Ok(Ok(())) => Poll::Ready(Ok(None)), Ok(Err(e)) => Poll::Ready(Err(e)), Err(StoreFiberYield::KeepStore) => Poll::Pending, @@ -708,3 +955,16 @@ impl Drop for FiberFuture<'_, '_> { } } } + +/// Changes the lifetime `'l` in `Context<'l>` to something else. +/// +/// # Safety +/// +/// Not a safe operation. Requires external knowledge about how the pointer is +/// being used to determine whether it's actually safe or not. See docs on +/// callers of this function. The purpose of this is to scope the `transmute` to +/// as small an operation as possible. +unsafe fn change_context_lifetime<'a, 'b>(cx: &'a mut Context<'_>) -> &'a mut Context<'b> { + // SAFETY: See the function documentation, this is not safe in general. + unsafe { mem::transmute::<&mut Context<'_>, &mut Context<'b>>(cx) } +} diff --git a/crates/wasmtime/src/runtime/func.rs b/crates/wasmtime/src/runtime/func.rs index a010bfa7e120..27f69a047930 100644 --- a/crates/wasmtime/src/runtime/func.rs +++ b/crates/wasmtime/src/runtime/func.rs @@ -1,5 +1,3 @@ -#[cfg(feature = "async")] -use crate::fiber::AsyncCx; use crate::prelude::*; use crate::runtime::Uninhabited; use crate::runtime::vm::{ @@ -15,10 +13,10 @@ use crate::{ }; use alloc::sync::Arc; use core::ffi::c_void; +#[cfg(feature = "async")] +use core::future::Future; use core::mem::{self, MaybeUninit}; use core::ptr::NonNull; -#[cfg(feature = "async")] -use core::{future::Future, pin::Pin}; use wasmtime_environ::VMSharedTypeIndex; /// A reference to the abstract `nofunc` heap value. @@ -515,16 +513,19 @@ impl Func { "cannot use `new_async` without enabling async support in the config" ); assert!(ty.comes_from_same_engine(store.as_context().engine())); - Func::new(store, ty, move |mut caller, params, results| { - let async_cx = AsyncCx::new(&mut caller.store.0); - let mut future = Pin::from(func(caller, params, results)); - // SAFETY: The `Store` we used to create the `AsyncCx` above remains - // valid. - match unsafe { async_cx.block_on(future.as_mut()) } { - Ok(Ok(())) => Ok(()), - Ok(Err(trap)) | Err(trap) => Err(trap), - } - }) + return Func::new( + store, + ty, + move |Caller { store, caller }, params, results| { + store.with_blocking(|store, cx| { + cx.block_on(core::pin::Pin::from(func( + Caller { store, caller }, + params, + results, + ))) + })? + }, + ); } pub(crate) unsafe fn from_vm_func_ref( @@ -838,12 +839,8 @@ impl Func { store.as_context().async_support(), concat!("cannot use `wrap_async` without enabling async support on the config") ); - Func::wrap_inner(store, move |mut caller: Caller<'_, T>, args| { - let async_cx = AsyncCx::new(&mut caller.store.0); - let mut future = Pin::from(func(caller, args)); - // SAFETY: The `Store` we used to create the `AsyncCx` above remains - // valid. - match unsafe { async_cx.block_on(future.as_mut()) } { + Func::wrap_inner(store, move |Caller { store, caller }, args| { + match store.block_on(|store| func(Caller { store, caller }, args).into()) { Ok(ret) => ret.into_fallible(), Err(e) => R::fallible_from_error(e), } @@ -2030,6 +2027,14 @@ pub struct Caller<'a, T: 'static> { } impl Caller<'_, T> { + pub(crate) fn new(store: StoreContextMut<'_, T>, caller: Instance) -> Caller<'_, T> { + Caller { store, caller } + } + + pub(crate) fn caller(&self) -> Instance { + self.caller + } + unsafe fn with(caller: NonNull, f: F) -> R where // The closure must be valid for any `Caller` it is given; it doesn't diff --git a/crates/wasmtime/src/runtime/linker.rs b/crates/wasmtime/src/runtime/linker.rs index e664dd80d867..6ca250c038a8 100644 --- a/crates/wasmtime/src/runtime/linker.rs +++ b/crates/wasmtime/src/runtime/linker.rs @@ -9,9 +9,9 @@ use crate::{ use crate::{IntoFunc, prelude::*}; use alloc::sync::Arc; use core::fmt::{self, Debug}; -use core::marker; #[cfg(feature = "async")] -use core::{future::Future, pin::Pin}; +use core::future::Future; +use core::marker; use log::warn; /// Structure used to link wasm modules/instances together. @@ -470,15 +470,12 @@ impl Linker { "cannot use `func_new_async` without enabling async support in the config" ); assert!(ty.comes_from_same_engine(self.engine())); - self.func_new(module, name, ty, move |mut caller, params, results| { - let async_cx = crate::fiber::AsyncCx::new(&mut caller.store.0); - let mut future = Pin::from(func(caller, params, results)); - // SAFETY: The `Store` we used to create the `AsyncCx` above remains - // valid. - match unsafe { async_cx.block_on(future.as_mut()) } { - Ok(Ok(())) => Ok(()), - Ok(Err(trap)) | Err(trap) => Err(trap), - } + self.func_new(module, name, ty, move |caller, params, results| { + let instance = caller.caller(); + caller.store.with_blocking(|store, cx| { + let caller = Caller::new(store, instance); + cx.block_on(core::pin::Pin::from(func(caller, params, results))) + })? }) } @@ -572,19 +569,18 @@ impl Linker { self.engine.config().async_support, "cannot use `func_wrap_async` without enabling async support on the config", ); - let func = HostFunc::wrap_inner( - &self.engine, - move |mut caller: Caller<'_, T>, args: Params| { - let async_cx = crate::fiber::AsyncCx::new(&mut caller.store.0); - let mut future = Pin::from(func(caller, args)); - // SAFETY: The `Store` we used to create the `AsyncCx` above - // remains valid. - match unsafe { async_cx.block_on(future.as_mut()) } { + let func = + HostFunc::wrap_inner(&self.engine, move |caller: Caller<'_, T>, args: Params| { + let instance = caller.caller(); + let result = caller.store.block_on(|store| { + let caller = Caller::new(store, instance); + func(caller, args).into() + }); + match result { Ok(ret) => ret.into_fallible(), Err(e) => Args::fallible_from_error(e), } - }, - ); + }); let key = self.import_key(module, Some(name)); self.insert(key, Definition::HostFunc(Arc::new(func)))?; Ok(self) diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index fff7e91e2266..8f8cbf8b8d45 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -80,7 +80,7 @@ use crate::RootSet; #[cfg(feature = "component-model-async")] use crate::component::ComponentStoreData; #[cfg(feature = "async")] -use crate::fiber::{self, AsyncCx}; +use crate::fiber; use crate::module::RegisteredModuleId; use crate::prelude::*; #[cfg(feature = "gc")] @@ -1117,15 +1117,14 @@ impl StoreInner { CallHookInner::Sync(hook) => hook((&mut *self).as_context_mut(), s), #[cfg(all(feature = "async", feature = "call-hook"))] - CallHookInner::Async(handler) => unsafe { - AsyncCx::try_new(&mut self.inner) - .ok_or_else(|| anyhow!("couldn't grab async_cx for call hook"))? - .block_on( - handler - .handle_call_event((&mut *self).as_context_mut(), s) - .as_mut(), - )? - }, + CallHookInner::Async(handler) => { + if !self.can_block() { + bail!("couldn't grab async_cx for call hook") + } + return (&mut *self) + .as_context_mut() + .with_blocking(|store, cx| cx.block_on(handler.handle_call_event(store, s)))?; + } CallHookInner::ForceTypeParameterToBeUsed { uninhabited, .. } => { let _ = s; @@ -2131,15 +2130,13 @@ unsafe impl vm::VMStore for StoreInner { limiter(&mut self.data).memory_growing(current, desired, maximum) } #[cfg(feature = "async")] - Some(ResourceLimiterInner::Async(ref mut limiter)) => unsafe { - AsyncCx::try_new(&mut self.inner) - .expect("ResourceLimiterAsync requires async Store") - .block_on( - limiter(&mut self.data) - .memory_growing(current, desired, maximum) - .as_mut(), - )? - }, + Some(ResourceLimiterInner::Async(_)) => self.block_on(|store| { + let limiter = match &mut store.0.limiter { + Some(ResourceLimiterInner::Async(limiter)) => limiter, + _ => unreachable!(), + }; + limiter(&mut store.0.data).memory_growing(current, desired, maximum) + })?, None => Ok(true), } } @@ -2166,32 +2163,18 @@ unsafe impl vm::VMStore for StoreInner { desired: usize, maximum: Option, ) -> Result { - // Need to borrow async_cx before the mut borrow of the limiter. - // self.async_cx() panicks when used with a non-async store, so - // wrap this in an option. - #[cfg(feature = "async")] - let async_cx = if self.async_support() - && matches!(self.limiter, Some(ResourceLimiterInner::Async(_))) - { - AsyncCx::try_new(&mut self.inner) - } else { - None - }; - match self.limiter { Some(ResourceLimiterInner::Sync(ref mut limiter)) => { limiter(&mut self.data).table_growing(current, desired, maximum) } #[cfg(feature = "async")] - Some(ResourceLimiterInner::Async(ref mut limiter)) => unsafe { - async_cx - .expect("ResourceLimiterAsync requires async Store") - .block_on( - limiter(&mut self.data) - .table_growing(current, desired, maximum) - .as_mut(), - )? - }, + Some(ResourceLimiterInner::Async(_)) => self.block_on(|store| { + let limiter = match &mut store.0.limiter { + Some(ResourceLimiterInner::Async(limiter)) => limiter, + _ => unreachable!(), + }; + limiter(&mut store.0.data).table_growing(current, desired, maximum) + })?, None => Ok(true), } } @@ -2245,7 +2228,7 @@ unsafe impl vm::VMStore for StoreInner { delta } #[cfg(feature = "async")] - UpdateDeadline::YieldCustom(delta, mut future) => { + UpdateDeadline::YieldCustom(delta, future) => { assert!( self.async_support(), "cannot use `UpdateDeadline::YieldCustom` without enabling async support in the config" @@ -2258,10 +2241,7 @@ unsafe impl vm::VMStore for StoreInner { // to clean up this fiber. Do so by raising a trap which will // abort all wasm and get caught on the other side to clean // things up. - unsafe {AsyncCx::try_new(self) - .expect("attempted to pull async context during shutdown") - .block_on(future.as_mut())?; - } + self.block_on(|_| future)?; delta } }; diff --git a/crates/wasmtime/src/runtime/store/async_.rs b/crates/wasmtime/src/runtime/store/async_.rs index c7de58da6cb4..4119d046239c 100644 --- a/crates/wasmtime/src/runtime/store/async_.rs +++ b/crates/wasmtime/src/runtime/store/async_.rs @@ -1,10 +1,9 @@ #[cfg(feature = "call-hook")] use crate::CallHook; -use crate::fiber::{self, AsyncCx}; +use crate::fiber::{self}; use crate::prelude::*; use crate::store::{ResourceLimiterInner, StoreInner, StoreOpaque, StoreToken}; use crate::{AsContextMut, Store, StoreContextMut, UpdateDeadline}; -use core::pin::Pin; /// An object that can take callbacks when the runtime enters or exits hostcalls. #[cfg(feature = "call-hook")] @@ -240,10 +239,6 @@ impl StoreOpaque { /// This only works on async futures and stores, and assumes that we're /// executing on a fiber. This will yield execution back to the caller once. pub fn async_yield_impl(&mut self) -> Result<()> { - use crate::runtime::vm::Yield; - - let mut future = Yield::new(); - // When control returns, we have a `Result<()>` passed // in from the host fiber. If this finished successfully then // we were resumed normally via a `poll`, so keep going. If @@ -251,11 +246,7 @@ impl StoreOpaque { // to clean up this fiber. Do so by raising a trap which will // abort all wasm and get caught on the other side to clean // things up. - unsafe { - AsyncCx::try_new(self) - .expect("attempted to pull async context during shutdown") - .block_on(Pin::new_unchecked(&mut future)) - } + self.block_on(|_| Box::pin(crate::runtime::vm::Yield::new())) } pub(crate) fn allocate_fiber_stack(&mut self) -> Result { diff --git a/crates/wasmtime/src/runtime/store/gc.rs b/crates/wasmtime/src/runtime/store/gc.rs index f81ca34f0b53..9a68729739a7 100644 --- a/crates/wasmtime/src/runtime/store/gc.rs +++ b/crates/wasmtime/src/runtime/store/gc.rs @@ -2,8 +2,6 @@ use super::*; use crate::GcHeapOutOfMemory; -#[cfg(feature = "async")] -use crate::fiber::AsyncCx; impl StoreOpaque { /// Collect garbage, potentially growing the GC heap. @@ -41,11 +39,7 @@ impl StoreOpaque { if scope.async_support() { #[cfg(feature = "async")] - { - let async_cx = AsyncCx::new(&mut scope); - let future = scope.grow_or_collect_gc_heap_async(bytes_needed); - async_cx.block_on(Box::pin(future).as_mut())?; - } + scope.block_on(|scope| Box::pin(scope.grow_or_collect_gc_heap_async(bytes_needed)))?; } else { scope.grow_or_collect_gc_heap(bytes_needed); } From 2942248a58fd4a9d3b6a1b88eb64340045676913 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Thu, 26 Jun 2025 09:45:13 -0700 Subject: [PATCH 22/30] Adjust some lint attributes --- crates/wasmtime/src/runtime/fiber.rs | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index b2a8043354de..ccc331edeb0b 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -426,9 +426,7 @@ pub(crate) enum StoreFiberYield { /// Indicates the fiber does _not_ need exclusive access across the /// suspend/resume interval, meaning the store may be used as needed until /// the fiber is resumed. - // TODO: This will be used once full `component-model-async` support is - // merged: - #[allow(dead_code)] + #[expect(dead_code, reason = "will be used by async thing soon")] ReleaseStore, } @@ -883,8 +881,7 @@ pub(crate) async fn on_fiber( /// This will return `Some` if the fiber suspends with /// `StoreFiberYield::ReleaseStore` or else `None` if it resolves. #[cfg(feature = "component-model-async")] -// This will be used when `component-model-async` support is merged. -#[allow(dead_code)] +#[expect(dead_code, reason = "will be used by async thing soon")] pub(crate) async fn resolve_or_release<'a>( store: &mut StoreOpaque, fiber: StoreFiber<'a>, @@ -904,7 +901,7 @@ enum OnRelease { ReturnPending, /// Return `Poll::Ready` from `FiberFuture::poll`, handing ownership of the /// `StoreFiber` to the caller. - #[cfg_attr(not(feature = "component-model-async"), allow(dead_code))] + #[cfg(feature = "component-model-async")] ReturnReady, } @@ -942,6 +939,7 @@ impl<'b> Future for FiberFuture<'_, 'b> { Err(StoreFiberYield::KeepStore) => Poll::Pending, Err(StoreFiberYield::ReleaseStore) => match &me.on_release { OnRelease::ReturnPending => Poll::Pending, + #[cfg(feature = "component-model-async")] OnRelease::ReturnReady => Poll::Ready(Ok(me.fiber.take())), }, } From 3e9249213e3be4025d1de963bf717e896ff50b07 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Thu, 26 Jun 2025 09:57:21 -0700 Subject: [PATCH 23/30] Make manipulation of `AsyncState` safe No need for raw pointers with recent refactorings. --- crates/wasmtime/src/runtime/fiber.rs | 88 ++++++++++++---------------- crates/wasmtime/src/runtime/store.rs | 4 +- 2 files changed, 41 insertions(+), 51 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index ccc331edeb0b..89b1a6f6e983 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -193,8 +193,10 @@ impl<'a, 'b> BlockingContext<'a, 'b> { S: AsStoreOpaque, { let opaque = store.as_store_opaque(); - let future_cx; - let suspend; + + let state = opaque.fiber_async_state_mut(); + assert!(!state.current_future_cx.is_null()); + assert!(!state.current_suspend.is_null()); // SAFETY: this is taking pointers from `AsyncState` and then unsafely // turning them into safe references. Lifetime-wise this should be safe @@ -210,15 +212,11 @@ impl<'a, 'b> BlockingContext<'a, 'b> { // fiber and for this fiber. The "take" pattern here ensures that if // this `BlockingContext` context acquires the pointers then there are // no other instances of these pointers in use anywhere else. - unsafe { - let state = opaque.async_state(); - assert!(!(*state).current_future_cx.is_null()); - assert!(!(*state).current_suspend.is_null()); - future_cx = Some(&mut *(*state).current_future_cx); - (*state).current_future_cx = ptr::null_mut(); - suspend = &mut *(*state).current_suspend; - (*state).current_suspend = ptr::null_mut(); - } + let future_cx = unsafe { Some(&mut *state.current_future_cx) }; + let suspend = unsafe { &mut *state.current_suspend }; + + state.current_future_cx = ptr::null_mut(); + state.current_suspend = ptr::null_mut(); let mut reset = ResetBlockingContext { store, @@ -234,18 +232,17 @@ impl<'a, 'b> BlockingContext<'a, 'b> { impl Drop for ResetBlockingContext<'_, '_, S> { fn drop(&mut self) { let store = self.store.as_store_opaque(); - let state = store.async_state(); - - // SAFETY: TODO need to remove the unsafety from - // reading/writing `async_state`. Otherwise this is all state - // owned by the store. - unsafe { - debug_assert!((*state).current_future_cx.is_null()); - debug_assert!((*state).current_suspend.is_null()); - (*state).current_suspend = self.cx.suspend; - if let Some(cx) = &mut self.cx.future_cx { - (*state).current_future_cx = change_context_lifetime(cx); - } + let state = store.fiber_async_state_mut(); + + debug_assert!(state.current_future_cx.is_null()); + debug_assert!(state.current_suspend.is_null()); + state.current_suspend = self.cx.suspend; + + if let Some(cx) = &mut self.cx.future_cx { + // SAFETY: while this is changing the lifetime to `'static` + // it should never be used while it's `'static` given this + // `BlockingContext` abstraction. + state.current_future_cx = unsafe { change_context_lifetime(cx) }; } } } @@ -411,8 +408,7 @@ impl StoreOpaque { /// Returns whether `block_on` will succeed or panic. pub(crate) fn can_block(&mut self) -> bool { - let current_future_cx = unsafe { (*self.async_state()).current_future_cx }; - !current_future_cx.is_null() + !self.fiber_async_state_mut().current_future_cx.is_null() } } @@ -646,11 +642,11 @@ impl StoreOpaque { } fn replace_current_suspend(&mut self, ptr: *mut WasmtimeSuspend) -> *mut WasmtimeSuspend { - unsafe { mem::replace(&mut (*self.async_state()).current_suspend, ptr) } + mem::replace(&mut self.fiber_async_state_mut().current_suspend, ptr) } fn replace_current_future_cx(&mut self, ptr: *mut Context<'static>) -> *mut Context<'static> { - unsafe { mem::replace(&mut (*self.async_state()).current_future_cx, ptr) } + mem::replace(&mut self.fiber_async_state_mut().current_future_cx, ptr) } } @@ -788,39 +784,33 @@ pub(crate) fn make_fiber<'a>( // `resume_fiber`, we've been given exclusive access to // the store until we exit or yield it back to the resumer. let store_ref = unsafe { &mut *store }; - let async_state = store_ref.store_opaque_mut().async_state(); // It should be a guarantee that the store has null pointers here upon // starting a fiber, so now's the time to fill in the pointers now that // the fiber is running and `future_cx` and `suspend` are both in scope. // Note that these pointers are removed when this function returns as // that's when they fall out of scope. - // - // SAFETY: TODO make manipulation of `AsyncState` safer. - unsafe { - assert!((*async_state).current_suspend.is_null()); - assert!((*async_state).current_future_cx.is_null()); - - (*async_state).current_suspend = suspend; - (*async_state).current_future_cx = future_cx; - } + let async_state = store_ref.store_opaque_mut().fiber_async_state_mut(); + assert!(async_state.current_suspend.is_null()); + assert!(async_state.current_future_cx.is_null()); + async_state.current_suspend = suspend; + async_state.current_future_cx = future_cx; struct ResetCurrentPointersToNull<'a>(&'a mut dyn VMStore); impl Drop for ResetCurrentPointersToNull<'_> { fn drop(&mut self) { - let state = self.0.async_state(); - unsafe { - // Double-check that the current suspension isn't null (it - // should be what's in this closure). Note though that we - // can't check `current_future_cx` because it may either be - // here or not be here depending on whether this was - // cancelled or not. - debug_assert!(!(*state).current_suspend.is_null()); - - (*state).current_suspend = ptr::null_mut(); - (*state).current_future_cx = ptr::null_mut(); - } + let state = self.0.fiber_async_state_mut(); + + // Double-check that the current suspension isn't null (it + // should be what's in this closure). Note though that we + // can't check `current_future_cx` because it may either be + // here or not be here depending on whether this was + // cancelled or not. + debug_assert!(!state.current_suspend.is_null()); + + state.current_suspend = ptr::null_mut(); + state.current_future_cx = ptr::null_mut(); } } let reset = ResetCurrentPointersToNull(store_ref); diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index 8f8cbf8b8d45..93863057b703 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -1953,8 +1953,8 @@ at https://bytecodealliance.org/security. } #[cfg(feature = "async")] - pub(crate) fn async_state(&mut self) -> *mut fiber::AsyncState { - &raw mut self.async_state + pub(crate) fn fiber_async_state_mut(&mut self) -> &mut fiber::AsyncState { + &mut self.async_state } #[cfg(feature = "async")] From f45ea3aa10de9fedd04b0e5e54c058c7f42d6526 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Thu, 26 Jun 2025 10:57:59 -0700 Subject: [PATCH 24/30] Fix dead code warning --- crates/wasmtime/src/runtime/fiber.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 89b1a6f6e983..a44c9fca59b1 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -407,6 +407,7 @@ impl StoreOpaque { } /// Returns whether `block_on` will succeed or panic. + #[cfg(feature = "call-hook")] pub(crate) fn can_block(&mut self) -> bool { !self.fiber_async_state_mut().current_future_cx.is_null() } From 37630c0975b7cef429a9199114d061a42db4ff09 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Thu, 26 Jun 2025 12:02:27 -0700 Subject: [PATCH 25/30] More dead code warnings --- crates/wasmtime/src/runtime/func.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/crates/wasmtime/src/runtime/func.rs b/crates/wasmtime/src/runtime/func.rs index 27f69a047930..0769ad75100b 100644 --- a/crates/wasmtime/src/runtime/func.rs +++ b/crates/wasmtime/src/runtime/func.rs @@ -2027,10 +2027,12 @@ pub struct Caller<'a, T: 'static> { } impl Caller<'_, T> { + #[cfg(feature = "async")] pub(crate) fn new(store: StoreContextMut<'_, T>, caller: Instance) -> Caller<'_, T> { Caller { store, caller } } + #[cfg(feature = "async")] pub(crate) fn caller(&self) -> Instance { self.caller } From c28dfaa78c91717b73889c7d8b4f9e31d2696733 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Thu, 26 Jun 2025 12:13:09 -0700 Subject: [PATCH 26/30] Cut down on raw pointers in fiber.rs --- crates/wasmtime/src/runtime/fiber.rs | 86 +++++++++++++++------------- 1 file changed, 46 insertions(+), 40 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index a44c9fca59b1..5bf024eee9de 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -9,11 +9,11 @@ use anyhow::{Result, anyhow}; use core::mem; use core::ops::Range; use core::pin::Pin; -use core::ptr; +use core::ptr::{self, NonNull}; use core::task::{Context, Poll}; use wasmtime_fiber::{Fiber, Suspend}; -type WasmtimeResume = Result<*mut Context<'static>>; +type WasmtimeResume = Result>>; type WasmtimeYield = StoreFiberYield; type WasmtimeComplete = Result<()>; type WasmtimeSuspend = Suspend; @@ -42,7 +42,7 @@ pub(crate) struct AsyncState { /// fiber is suspended as well. Fiber resumption will save the prior value /// in a store and then set it to null, where suspension will then restore /// what was previously in the store. - current_suspend: *mut WasmtimeSuspend, + current_suspend: Option>, /// The `Context` pointer last provided in `Future for FiberFuture`. /// @@ -77,7 +77,7 @@ pub(crate) struct AsyncState { /// handled by ensuring the signatures that work with `BlockingContext` all /// use constrained anonymous lifetimes that are guaranteed to be shorter /// than the original `Context` lifetime. - current_future_cx: *mut Context<'static>, + current_future_cx: Option>>, /// The last fiber stack that was in use by the store. /// @@ -88,18 +88,21 @@ pub(crate) struct AsyncState { last_fiber_stack: Option, } -// Lots of pesky unsafe cells and pointers in this structure. This means we need -// to declare explicitly that we use this in a threadsafe fashion. -// -// TODO: replace pointers above with `SendSyncPtr<..>` and remove this +// SAFETY: it's known that `std::task::Context` is neither `Send` nor `Sync`, +// but despite this the storage here is purely temporary in getting these +// pointers across function frames. The actual types are not sent across threads +// as when a store isn't polling anything the pointer values are all set to +// `None`. Thus if a store is being sent across threads that's done because no +// fibers are active, and once fibers are active everything will stick within +// the same thread. unsafe impl Send for AsyncState {} unsafe impl Sync for AsyncState {} impl Default for AsyncState { fn default() -> Self { Self { - current_suspend: ptr::null_mut(), - current_future_cx: ptr::null_mut(), + current_suspend: None, + current_future_cx: None, last_fiber_stack: None, } } @@ -195,8 +198,6 @@ impl<'a, 'b> BlockingContext<'a, 'b> { let opaque = store.as_store_opaque(); let state = opaque.fiber_async_state_mut(); - assert!(!state.current_future_cx.is_null()); - assert!(!state.current_suspend.is_null()); // SAFETY: this is taking pointers from `AsyncState` and then unsafely // turning them into safe references. Lifetime-wise this should be safe @@ -212,11 +213,8 @@ impl<'a, 'b> BlockingContext<'a, 'b> { // fiber and for this fiber. The "take" pattern here ensures that if // this `BlockingContext` context acquires the pointers then there are // no other instances of these pointers in use anywhere else. - let future_cx = unsafe { Some(&mut *state.current_future_cx) }; - let suspend = unsafe { &mut *state.current_suspend }; - - state.current_future_cx = ptr::null_mut(); - state.current_suspend = ptr::null_mut(); + let future_cx = unsafe { Some(state.current_future_cx.take().unwrap().as_mut()) }; + let suspend = unsafe { state.current_suspend.take().unwrap().as_mut() }; let mut reset = ResetBlockingContext { store, @@ -234,15 +232,16 @@ impl<'a, 'b> BlockingContext<'a, 'b> { let store = self.store.as_store_opaque(); let state = store.fiber_async_state_mut(); - debug_assert!(state.current_future_cx.is_null()); - debug_assert!(state.current_suspend.is_null()); - state.current_suspend = self.cx.suspend; + debug_assert!(state.current_future_cx.is_none()); + debug_assert!(state.current_suspend.is_none()); + state.current_suspend = Some(NonNull::from(&mut *self.cx.suspend)); if let Some(cx) = &mut self.cx.future_cx { // SAFETY: while this is changing the lifetime to `'static` // it should never be used while it's `'static` given this // `BlockingContext` abstraction. - state.current_future_cx = unsafe { change_context_lifetime(cx) }; + state.current_future_cx = + Some(NonNull::from(unsafe { change_context_lifetime(cx) })); } } } @@ -320,7 +319,7 @@ impl<'a, 'b> BlockingContext<'a, 'b> { // value given back. self.future_cx.take(); - let new_future_cx: *mut Context<'static> = self.suspend.suspend(yield_)?; + let mut new_future_cx: NonNull> = self.suspend.suspend(yield_)?; // SAFETY: this function is unsafe as we're doing "funky" things to the // `new_future_cx` we have been given. The safety here relies on the @@ -328,7 +327,7 @@ impl<'a, 'b> BlockingContext<'a, 'b> { // the original `Context` itself, and that should be guaranteed through // the exclusive constructor of this type `BlockingContext::with`. unsafe { - self.future_cx = Some(change_context_lifetime(&mut *new_future_cx)); + self.future_cx = Some(change_context_lifetime(new_future_cx.as_mut())); } Ok(()) } @@ -409,7 +408,7 @@ impl StoreOpaque { /// Returns whether `block_on` will succeed or panic. #[cfg(feature = "call-hook")] pub(crate) fn can_block(&mut self) -> bool { - !self.fiber_async_state_mut().current_future_cx.is_null() + self.fiber_async_state_mut().current_future_cx.is_some() } } @@ -616,8 +615,8 @@ impl FiberResumeState { // The current suspend/future_cx are always null upon resumption, so // insert null. Save the old values through to get preserved across // this resume/suspend. - current_suspend: store.replace_current_suspend(ptr::null_mut()), - current_future_cx: store.replace_current_future_cx(ptr::null_mut()), + current_suspend: store.replace_current_suspend(None), + current_future_cx: store.replace_current_future_cx(None), } } @@ -642,11 +641,17 @@ impl StoreOpaque { mem::replace(&mut self.vm_store_context_mut().async_guard_range, range) } - fn replace_current_suspend(&mut self, ptr: *mut WasmtimeSuspend) -> *mut WasmtimeSuspend { + fn replace_current_suspend( + &mut self, + ptr: Option>, + ) -> Option> { mem::replace(&mut self.fiber_async_state_mut().current_suspend, ptr) } - fn replace_current_future_cx(&mut self, ptr: *mut Context<'static>) -> *mut Context<'static> { + fn replace_current_future_cx( + &mut self, + ptr: Option>>, + ) -> Option>> { mem::replace(&mut self.fiber_async_state_mut().current_future_cx, ptr) } } @@ -656,8 +661,8 @@ struct PriorFiberResumeState { mpk: Option, stack_limit: usize, async_guard_range: Range<*mut u8>, - current_suspend: *mut WasmtimeSuspend, - current_future_cx: *mut Context<'static>, + current_suspend: Option>, + current_future_cx: Option>>, } impl PriorFiberResumeState { @@ -672,9 +677,9 @@ impl PriorFiberResumeState { // should be guaranteed that the prior values are null, so double-check // that here. let prev = store.replace_current_suspend(self.current_suspend); - assert!(prev.is_null()); + assert!(prev.is_none()); let prev = store.replace_current_future_cx(self.current_future_cx); - assert!(prev.is_null()); + assert!(prev.is_none()); FiberResumeState { tls, @@ -792,10 +797,10 @@ pub(crate) fn make_fiber<'a>( // Note that these pointers are removed when this function returns as // that's when they fall out of scope. let async_state = store_ref.store_opaque_mut().fiber_async_state_mut(); - assert!(async_state.current_suspend.is_null()); - assert!(async_state.current_future_cx.is_null()); - async_state.current_suspend = suspend; - async_state.current_future_cx = future_cx; + assert!(async_state.current_suspend.is_none()); + assert!(async_state.current_future_cx.is_none()); + async_state.current_suspend = Some(NonNull::from(suspend)); + async_state.current_future_cx = Some(future_cx); struct ResetCurrentPointersToNull<'a>(&'a mut dyn VMStore); @@ -808,10 +813,10 @@ pub(crate) fn make_fiber<'a>( // can't check `current_future_cx` because it may either be // here or not be here depending on whether this was // cancelled or not. - debug_assert!(!state.current_suspend.is_null()); + debug_assert!(state.current_suspend.is_some()); - state.current_suspend = ptr::null_mut(); - state.current_future_cx = ptr::null_mut(); + state.current_suspend = None; + state.current_future_cx = None; } } let reset = ResetCurrentPointersToNull(store_ref); @@ -922,7 +927,8 @@ impl<'b> Future for FiberFuture<'_, 'b> { // satisfied by the users of this in the `BlockingContext` structure // where the lifetime parameters there are always more constrained than // they are here. - let cx: *mut Context<'static> = unsafe { change_context_lifetime(cx) }; + let cx: &mut Context<'static> = unsafe { change_context_lifetime(cx) }; + let cx = NonNull::from(cx); match resume_fiber(me.store, me.fiber.as_mut().unwrap(), Ok(cx)) { Ok(Ok(())) => Poll::Ready(Ok(None)), From 8adfffbed3d77847004fcd1bebdc2c25ac1e2bc8 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Thu, 26 Jun 2025 12:16:04 -0700 Subject: [PATCH 27/30] Move executor save/restore to normal fiber state save/restore --- crates/wasmtime/src/runtime/fiber.rs | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 5bf024eee9de..07088e3487be 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -444,12 +444,6 @@ pub(crate) struct StoreFiber<'a> { state: Option, /// The Wasmtime `Engine` to which this fiber belongs. engine: Engine, - /// The executor (e.g. the Pulley interpreter state) belonging to this - /// fiber. - /// - /// This is swapped with `StoreOpaque::executor` whenever this fiber is - /// resumed, suspended, or resolved. - executor: Executor, /// The id of the store with which this fiber was created. /// /// Any attempt to resume a fiber with a different store than the one with @@ -589,6 +583,13 @@ struct FiberResumeState { /// fibers within the store they all maintain an appropriate fiber-relative /// stack limit. stack_limit: usize, + + /// The executor (e.g. the Pulley interpreter state) belonging to this + /// fiber. + /// + /// This is swapped with `StoreOpaque::executor` whenever this fiber is + /// resumed, suspended, or resolved. + executor: Executor, } impl FiberResumeState { @@ -606,9 +607,12 @@ impl FiberResumeState { .stack() .guard_range() .unwrap_or(ptr::null_mut()..ptr::null_mut()); + let mut executor = self.executor; + store.swap_executor(&mut executor); PriorFiberResumeState { tls, mpk, + executor, stack_limit: store.replace_stack_limit(self.stack_limit), async_guard_range: store.replace_async_guard_range(async_guard_range), @@ -663,6 +667,7 @@ struct PriorFiberResumeState { async_guard_range: Range<*mut u8>, current_suspend: Option>, current_future_cx: Option>>, + executor: Executor, } impl PriorFiberResumeState { @@ -681,9 +686,13 @@ impl PriorFiberResumeState { let prev = store.replace_current_future_cx(self.current_future_cx); assert!(prev.is_none()); + let mut executor = self.executor; + store.swap_executor(&mut executor); + FiberResumeState { tls, mpk, + executor, stack_limit: store.replace_stack_limit(self.stack_limit), } } @@ -720,12 +729,10 @@ fn resume_fiber<'a>( impl Drop for Restore<'_, '_> { fn drop(&mut self) { self.fiber.state = Some(unsafe { self.state.take().unwrap().replace(self.store) }); - self.store.swap_executor(&mut self.fiber.executor); } } let result = unsafe { let prev = fiber.state.take().unwrap().replace(store, fiber); - store.swap_executor(&mut fiber.executor); let restore = Restore { store, fiber, @@ -832,9 +839,9 @@ pub(crate) fn make_fiber<'a>( None }, stack_limit: usize::MAX, + executor, }), engine, - executor, id, fiber: Some(fiber), }) From 141d70e6080d28de3c767a3cd5e422c9bc7c5dcf Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Thu, 26 Jun 2025 12:31:25 -0700 Subject: [PATCH 28/30] Bikeshed a method name --- crates/wasmtime/src/runtime/store.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index 93863057b703..8c883a7be63a 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -649,7 +649,7 @@ impl Store { self.inner.data_mut() } - fn drop_everything_but_data(&mut self) { + fn run_manual_drop_routines(&mut self) { // We need to drop the fibers of each component instance before // attempting to drop the instances themselves since the fibers may need // to be resumed and allowed to exit cleanly before we yank the state @@ -657,12 +657,14 @@ impl Store { #[cfg(feature = "component-model-async")] ComponentStoreData::drop_fibers(&mut self.inner); + // Ensure all fiber stacks, even cached ones, are all flushed out to the + // instance allocator. self.inner.flush_fiber_stack(); } /// Consumes this [`Store`], destroying it, and returns the underlying data. pub fn into_data(mut self) -> T { - self.drop_everything_but_data(); + self.run_manual_drop_routines(); // This is an unsafe operation because we want to avoid having a runtime // check or boolean for whether the data is actually contained within a @@ -2331,7 +2333,7 @@ impl fmt::Debug for Store { impl Drop for Store { fn drop(&mut self) { - self.drop_everything_but_data(); + self.run_manual_drop_routines(); // for documentation on this `unsafe`, see `into_data`. unsafe { From 691e58fb78422edd2fc14dd64abc165b02a2bb0d Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Thu, 26 Jun 2025 14:59:38 -0600 Subject: [PATCH 29/30] update comment in make_fiber Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/fiber.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 07088e3487be..11d0c41e4173 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -793,9 +793,9 @@ pub(crate) fn make_fiber<'a>( Err(_) => return Ok(()), }; - // SAFETY: Per the documented contract for - // `resume_fiber`, we've been given exclusive access to - // the store until we exit or yield it back to the resumer. + // SAFETY: This fiber will only be resumed using `resume_fiber`, which + // takes a `&mut StoreOpaque` parameter and has given us exclusive + // access to the store until we exit or yield it back to the resumer. let store_ref = unsafe { &mut *store }; // It should be a guarantee that the store has null pointers here upon From c3cb1846c0a942c1bde88f67f72afbc0ed48ee68 Mon Sep 17 00:00:00 2001 From: Joel Dice Date: Thu, 26 Jun 2025 15:38:28 -0600 Subject: [PATCH 30/30] fix machports build Signed-off-by: Joel Dice --- crates/wasmtime/src/runtime/vm/sys/unix/machports.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/crates/wasmtime/src/runtime/vm/sys/unix/machports.rs b/crates/wasmtime/src/runtime/vm/sys/unix/machports.rs index 25bd49ede33c..8bf45c8a5455 100644 --- a/crates/wasmtime/src/runtime/vm/sys/unix/machports.rs +++ b/crates/wasmtime/src/runtime/vm/sys/unix/machports.rs @@ -140,9 +140,8 @@ unsafe extern "C" fn sigbus_handler( None => return, }; let faulting_addr = (*siginfo).si_addr() as usize; - let start = info.async_guard_range.start; - let end = info.async_guard_range.end; - if start as usize <= faulting_addr && faulting_addr < end as usize { + let range = &info.vm_store_context.as_ref().async_guard_range; + if range.start.addr() <= faulting_addr && faulting_addr < range.end.addr() { super::signals::abort_stack_overflow(); } });