From 2c0783372b5ae30be6838d4df4a32b7f19790095 Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Thu, 18 Dec 2025 18:54:51 -0800 Subject: [PATCH 1/8] Debugger: add top-level crate and async wrapper allowing for a "debug environment". The debug support in Wasmtime so far is structured around async callbacks that occur at certain kinds of events, like breakpoints. This is a suitable foundation but makes for an awkward implementation of a top-level debugger implementation, which likely has an event loop dealing with user commands (via a UI or a protocol connection) and expects to perform actions such as "run until next breakpoint". This PR introduces a new crate that wraps a `Store` in a `Debugger`. This wrapper embodies an inner async body that can perform whatever actions it likes on the `Store` that is passed back in. This inner body is spawned as an async task. The debugger wrapper registers its own `DebugHandler` callback that communicates with the outside world via bidirectional command/response queues. On the "outside", the `Debugger` presents an interface suitable for inserting into a debug protocol server or UI: an async method that runs until next event and returns that event, and a method that permits querying or modifying the store whenever the `run` method is not executing. The latter operates by sending a closure over the queue, because the `Store` must continue to be owned by the async task that is (still) running and suspended in async callbacks. Right now, this is exercised only via a few unit tests, but the intent is to next build up the "top half" of the debugger using this abstraction, e.g. by running a gdbstub protocol server (likely as a Wasm component in a "debug-main WIT world" -- RFC needed for this). Also, when we eventually move debugging over to native use of `run_concurrent`, this paradigm should remain mostly unchanged at this level of API: there can still be an object that has an async method that runs and yields the next event, and there can still be a method that takes a closure that can operate (within its scope only) on the `Store`. A few warts that I could use feedback on: - Cancelation safety is weird. Fibers panic when dropped before execution of their body completes, and this seems to mean that we can't allow a `Debugger` to drop early (or at least, the `tokio::test` unit test that owns the runtime that runs the async task to finish before the debugged body completes!). If there is a better way to handle cancelation safety here, I'm all ears. - It's not clear to me if the boxed-closure-and-`Any` approach to providing access to the `Store` is the best we can do, but I suspect it is. --- Cargo.lock | 11 + Cargo.toml | 2 + crates/debugger/Cargo.toml | 31 +++ crates/debugger/src/lib.rs | 536 +++++++++++++++++++++++++++++++++++++ 4 files changed, 580 insertions(+) create mode 100644 crates/debugger/Cargo.toml create mode 100644 crates/debugger/src/lib.rs diff --git a/Cargo.lock b/Cargo.lock index c4f75c076d07..133f226a52da 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4833,6 +4833,17 @@ dependencies = [ "wasmtime-internal-versioned-export-macros", ] +[[package]] +name = "wasmtime-internal-debugger" +version = "41.0.0" +dependencies = [ + "anyhow", + "env_logger 0.11.5", + "log", + "tokio", + "wasmtime", +] + [[package]] name = "wasmtime-internal-error" version = "41.0.0" diff --git a/Cargo.toml b/Cargo.toml index 8c3e9146afbc..362709308127 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -164,6 +164,7 @@ members = [ "crates/wasi-preview1-component-adapter", "crates/wasi-preview1-component-adapter/verify", "crates/wasi-tls-nativetls", + "crates/debugger", "crates/wizer/fuzz", "crates/wizer/tests/regex-test", "crates/wizer/benches/regex-bench", @@ -276,6 +277,7 @@ wasmtime-jit-icache-coherence = { path = "crates/jit-icache-coherence", version wasmtime-wit-bindgen = { path = "crates/wit-bindgen", version = "=41.0.0", package = 'wasmtime-internal-wit-bindgen' } wasmtime-math = { path = "crates/math", version = "=41.0.0", package = 'wasmtime-internal-math' } wasmtime-unwinder = { path = "crates/unwinder", version = "=41.0.0", package = 'wasmtime-internal-unwinder' } +wasmtime-debugger = { path = "crates/debugger", version = "=41.0.0", package = "wasmtime-internal-debugger" } wasmtime-wizer = { path = "crates/wizer", version = "41.0.0" } # Miscellaneous crates without a `wasmtime-*` prefix in their name but still diff --git a/crates/debugger/Cargo.toml b/crates/debugger/Cargo.toml new file mode 100644 index 000000000000..0a22f28dbaa2 --- /dev/null +++ b/crates/debugger/Cargo.toml @@ -0,0 +1,31 @@ +[package] +name = "wasmtime-internal-debugger" +authors.workspace = true +version.workspace = true +description = "INTERNAL: Wasmtime's guest-debugger functionality" +license = "Apache-2.0 WITH LLVM-exception" +documentation = "https://docs.rs/wasmtime-debugger" +repository = "https://github.com/bytecodealliance/wasmtime" +categories = ["no-std"] +readme = "README.md" +keywords = ["debugger"] +edition.workspace = true +rust-version.workspace = true + +[lints] +workspace = true + +[dependencies] +wasmtime = { workspace = true, features = ["debug", "std", "async"] } +tokio = { workspace = true, features = ["rt", "sync", "macros"] } +anyhow = { workspace = true } +log = { workspace = true } + +[dev-dependencies] +# Depend on `wasmtime` again to get `cranelift` so we can write unit +# tests that build modules. +wasmtime = { workspace = true, features = ["debug", "std", "async", "cranelift", "wat"] } +env_logger = { workspace = true } + +[features] +default = [] diff --git a/crates/debugger/src/lib.rs b/crates/debugger/src/lib.rs new file mode 100644 index 000000000000..74a109af6208 --- /dev/null +++ b/crates/debugger/src/lib.rs @@ -0,0 +1,536 @@ +//! Wasmtime debugger functionality. +//! +//! This crate builds on top of the core Wasmtime crate's +//! guest-debugger APIs to present an environment where a debugger +//! runs as a "co-running process" and sees the debugee as a a +//! provider of a stream of events, on which actions can be taken +//! between each event. +//! +//! In the future, this crate will also provide a WIT-level API and +//! world in which to run debugger components. + +use std::{any::Any, sync::Arc}; +use tokio::{ + sync::{Mutex, mpsc}, + task::JoinHandle, +}; +use wasmtime::{ + AsContextMut, DebugEvent, DebugHandler, ExnRef, OwnedRooted, Result, Store, StoreContextMut, + Trap, +}; + +/// A `Debugger` wraps up state associated with debugging the code +/// running in a single `Store`. +/// +/// It acts as a Future combinator, wrapping an inner async body that +/// performs some actions on a store. Those actions are subject to the +/// debugger, and debugger events will be raised as appropriate. From +/// the "outside" of this combinator, it is always in one of two +/// states: running or stopped. When stopped, it acts as a +/// `StoreContextMut` and can allow examining the stopped execution's +/// state. One runs until the next event suspends execution by +/// invoking `Debugger::run`. +/// +/// Note that because of limitations in Wasmtime's future cancelation +/// handling, all events must be consumed until the inner body +/// completes and `Debugger::is_complete` returns +/// true. `Debugger::finish` continues execution ignoring all further +/// events to allow clean completion if needed. +pub struct Debugger { + /// State: either a task handle or the store when passed out of + /// the complete task. + state: DebuggerState, + in_tx: mpsc::Sender>, + out_rx: mpsc::Receiver, +} + +enum DebuggerState { + /// Inner body is running in an async task. + Running(JoinHandle>>), + /// Temporary state while we are joining. + Joining, + /// Inner body is complete and has passed the store back. + Complete(Store), + /// Debugger has been disassembled via `into_store()`. Allows the + /// `Drop` impl to verify that the debugger is complete. + Destructed, +} + +/// Message from "outside" to the debug hook. +/// +/// The `Query` catch-all with a boxed closure is a little janky, but +/// is the way that we provide access +/// from outside to the Store (which is owned by `inner` above) +/// only during pauses. Note that the future cannot take full +/// ownership or a mutable borrow of the Store, because it cannot +/// hold this across async yield points. +/// +/// Instead, the debugger body sends boxed closures which take the +/// Store as a parameter (lifetime-limited not to escape that +/// closure) out to this crate's implementation that runs inside of +/// debugger-instrumentation callbacks (which have access to the +/// Store during their duration). We send return values +/// back. Return values are boxed Any values. +/// +/// If we wanted to make this a little more principled, we could +/// come up with a Command/Response pair of enums for all possible +/// closures and make everything more statically typed and less +/// Box'd, but that would severely restrict the flexibility of the +/// abstraction here and essentially require writing a full proxy +/// of the debugger API. +/// +/// Furthermore, we expect to rip this out eventually when we move +/// the debugger over to an async implementation based on +/// `run_concurrent` and `Accessor`s (see #11896). Building things +/// this way now will actually allow a less painful transition at +/// that time, because we will have a bunch of closures accessing +/// the store already and we can run those "with an accessor" +/// instead. +enum Command { + Continue, + Query(Box) -> Box + Send>), +} + +enum Response { + Stopped(DebugRunResult), + QueryResponse(Box), + Finished, +} + +struct HandlerInner { + in_rx: Mutex>>, + out_tx: mpsc::Sender, +} + +struct Handler(Arc>); + +impl std::clone::Clone for Handler { + fn clone(&self) -> Self { + Handler(self.0.clone()) + } +} + +impl DebugHandler for Handler { + type Data = T; + fn handle( + &self, + mut store: StoreContextMut<'_, T>, + event: DebugEvent<'_>, + ) -> impl Future + Send { + async move { + let mut in_rx = self.0.in_rx.lock().await; + + let result = match event { + DebugEvent::HostcallError(_) => DebugRunResult::HostcallError, + DebugEvent::CaughtExceptionThrown(exn) => { + DebugRunResult::CaughtExceptionThrown(exn) + } + DebugEvent::UncaughtExceptionThrown(exn) => { + DebugRunResult::UncaughtExceptionThrown(exn) + } + DebugEvent::Trap(trap) => DebugRunResult::Trap(trap), + DebugEvent::Breakpoint => DebugRunResult::Breakpoint, + }; + self.0 + .out_tx + .send(Response::Stopped(result)) + .await + .expect("outbound channel closed prematurely"); + + while let Some(cmd) = in_rx.recv().await { + match cmd { + Command::Query(closure) => { + let result = closure(store.as_context_mut()); + self.0 + .out_tx + .send(Response::QueryResponse(result)) + .await + .expect("outbound channel closed prematurely"); + } + Command::Continue => { + break; + } + } + } + } + } +} + +impl Debugger { + /// Create a new Debugger that attaches to the given Store and + /// runs the given inner body. + /// + /// The debugger is always in one of two states: running or + /// stopped. + /// + /// When stopped, the holder of this object can invoke + /// `Debugger::run` to enter the running state. The inner body + /// will run until stopped by a debug event. While running, the + /// future returned by either of these methods owns the `Debugger` + /// and hence no other methods can be invoked. + /// + /// When stopped, the holder of this object can access the `Store` + /// indirectly by providing a closure + pub fn new(mut store: Store, inner: F) -> Debugger + where + I: Future>> + Send + 'static, + F: for<'a> FnOnce(Store) -> I + Send + 'static, + { + let (in_tx, mut in_rx) = mpsc::channel(1); + let (out_tx, out_rx) = mpsc::channel(1); + + let inner = tokio::spawn(async move { + // Receive one "continue" command on the inbound channel + // before continuing. + match in_rx.recv().await { + Some(cmd) => { + assert!(matches!(cmd, Command::Continue)); + } + None => { + // Premature exit due to closed channel. Just drop `inner`. + anyhow::bail!("Debugger channel dropped"); + } + } + + let out_tx_clone = out_tx.clone(); + store.set_debug_handler(Handler(Arc::new(HandlerInner { + in_rx: Mutex::new(in_rx), + out_tx, + }))); + let result = inner(store).await; + let _ = out_tx_clone.send(Response::Finished).await; + result + }); + + Debugger { + state: DebuggerState::Running(inner), + in_tx, + out_rx, + } + } + + /// Is the inner body done running? + pub fn is_complete(&self) -> bool { + match &self.state { + DebuggerState::Running(_) | DebuggerState::Joining => false, + DebuggerState::Complete(_) => true, + DebuggerState::Destructed => { + panic!("Should not see this state outside of `into_store()`") + } + } + } + + /// Run the inner body until the next debug event. + pub async fn run(&mut self) -> Result { + anyhow::ensure!(!self.is_complete(), "Debugger body is already complete"); + + self.in_tx + .send(Command::Continue) + .await + .map_err(|_| anyhow::anyhow!("Failed to send over debug channel"))?; + + let response = self + .out_rx + .recv() + .await + .ok_or_else(|| anyhow::anyhow!("Premature close of debugger channel"))?; + + match response { + Response::Finished => { + let DebuggerState::Running(joinhandle) = + std::mem::replace(&mut self.state, DebuggerState::Joining) + else { + panic!("State was verified to be `Running` above"); + }; + let store = joinhandle.await??; + self.state = DebuggerState::Complete(store); + Ok(DebugRunResult::Finished) + } + Response::Stopped(result) => Ok(result), + Response::QueryResponse(_) => { + anyhow::bail!("Invalid debug response"); + } + } + } + + /// Run the debugger body until completion, with no further events. + pub async fn finish(&mut self) -> Result<()> { + if self.is_complete() { + return Ok(()); + } + loop { + match self.run().await? { + DebugRunResult::Finished => break, + e => { + log::trace!("finish: event {e:?}"); + } + } + } + assert!(self.is_complete()); + Ok(()) + } + + /// Perform some action on the contained `Store` while not running. + pub async fn with_store< + F: FnOnce(StoreContextMut<'_, T>) -> R + Send + 'static, + R: Send + 'static, + >( + &mut self, + f: F, + ) -> Result { + match &mut self.state { + DebuggerState::Running(_) => { + self.in_tx + .send(Command::Query(Box::new(|store| Box::new(f(store))))) + .await + .map_err(|_| anyhow::anyhow!("Premature close of debugger channel"))?; + let response = self + .out_rx + .recv() + .await + .ok_or_else(|| anyhow::anyhow!("Premature close of debugger channel"))?; + let Response::QueryResponse(resp) = response else { + anyhow::bail!("Incorrect response from debugger task"); + }; + Ok(*resp.downcast::().expect("type mismatch")) + } + DebuggerState::Joining => anyhow::bail!("Join failed with error and Store is lost"), + DebuggerState::Complete(store) => Ok(f(store.as_context_mut())), + DebuggerState::Destructed => { + panic!("Should not see `Destructed` state outside of `into_store`") + } + } + } + + /// Drop the Debugger once complete, returning the inner `Store` + /// around which it was wrapped. + pub fn into_store(mut self) -> Store { + let state = std::mem::replace(&mut self.state, DebuggerState::Destructed); + let mut store = match state { + DebuggerState::Complete(store) => store, + _ => panic!("Cannot invoke into_store() on a non-complete Debugger"), + }; + store.clear_debug_handler(); + store + } +} + +impl Drop for Debugger { + fn drop(&mut self) { + // We cannot allow this because the fiber implementation will + // panic if a `Func::call_async` future is dropped prematurely + // -- in general, Wasmtime's futures that embody Wasm + // execution are not cancel-safe, so we have to wait for the + // inner body to finish before the Debugger is dropped. + match &self.state { + DebuggerState::Complete(_) | DebuggerState::Destructed => {} + _ => panic!("Dropping Debugger before inner body is complete"), + } + } +} + +/// The result of one call to `Debugger::run()`. +/// +/// This is similar to `DebugEvent` but without the lifetime, so it +/// can be sent across async tasks, and incorporates the possibility +/// of completion (`Finished`) as well. +#[derive(Debug)] +pub enum DebugRunResult { + /// Execution of the inner body finished. + Finished, + /// An error was raised by a hostcall. + HostcallError, + /// An exception is thrown and caught by Wasm. The current state + /// is at the throw-point. + CaughtExceptionThrown(OwnedRooted), + /// An exception was not caught and is escaping to the host. + UncaughtExceptionThrown(OwnedRooted), + /// A Wasm trap occurred. + Trap(Trap), + /// A breakpoint was reached. + Breakpoint, +} + +#[cfg(test)] +mod test { + use super::*; + use wasmtime::*; + + #[tokio::test] + #[cfg_attr(miri, ignore)] + async fn basic_debugger() -> anyhow::Result<()> { + let _ = env_logger::try_init(); + + let mut config = Config::new(); + config.guest_debug(true); + config.async_support(true); + let engine = Engine::new(&config)?; + let module = Module::new( + &engine, + r#" + (module + (func (export "main") (param i32 i32) (result i32) + local.get 0 + local.get 1 + i32.add)) + "#, + )?; + + let mut store = Store::new(&engine, ()); + let instance = Instance::new_async(&mut store, &module, &[]).await?; + let main = instance.get_func(&mut store, "main").unwrap(); + + let mut debugger = Debugger::new(store, move |mut store| async move { + let mut results = [Val::I32(0)]; + store.edit_breakpoints().unwrap().single_step(true).unwrap(); + main.call_async(&mut store, &[Val::I32(1), Val::I32(2)], &mut results[..]) + .await?; + assert_eq!(results[0].unwrap_i32(), 3); + main.call_async(&mut store, &[Val::I32(3), Val::I32(4)], &mut results[..]) + .await?; + assert_eq!(results[0].unwrap_i32(), 7); + Ok(store) + }); + + let event = debugger.run().await?; + assert!(matches!(event, DebugRunResult::Breakpoint)); + // At (before executing) first `local.get`. + debugger + .with_store(|store| { + let mut frame = store.debug_frames().unwrap(); + assert!(!frame.done()); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().0.as_u32(), 0); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().1, 36); + assert_eq!(frame.num_locals(), 2); + assert_eq!(frame.num_stacks(), 0); + assert_eq!(frame.local(0).unwrap_i32(), 1); + assert_eq!(frame.local(1).unwrap_i32(), 2); + assert_eq!(frame.move_to_parent(), FrameParentResult::SameActivation); + assert!(frame.done()); + }) + .await?; + + let event = debugger.run().await?; + // At second `local.get`. + assert!(matches!(event, DebugRunResult::Breakpoint)); + debugger + .with_store(|store| { + let mut frame = store.debug_frames().unwrap(); + assert!(!frame.done()); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().0.as_u32(), 0); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().1, 38); + assert_eq!(frame.num_locals(), 2); + assert_eq!(frame.num_stacks(), 1); + assert_eq!(frame.local(0).unwrap_i32(), 1); + assert_eq!(frame.local(1).unwrap_i32(), 2); + assert_eq!(frame.stack(0).unwrap_i32(), 1); + assert_eq!(frame.move_to_parent(), FrameParentResult::SameActivation); + assert!(frame.done()); + }) + .await?; + + let event = debugger.run().await?; + // At `i32.add`. + assert!(matches!(event, DebugRunResult::Breakpoint)); + debugger + .with_store(|store| { + let mut frame = store.debug_frames().unwrap(); + assert!(!frame.done()); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().0.as_u32(), 0); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().1, 40); + assert_eq!(frame.num_locals(), 2); + assert_eq!(frame.num_stacks(), 2); + assert_eq!(frame.local(0).unwrap_i32(), 1); + assert_eq!(frame.local(1).unwrap_i32(), 2); + assert_eq!(frame.stack(0).unwrap_i32(), 1); + assert_eq!(frame.stack(1).unwrap_i32(), 2); + assert_eq!(frame.move_to_parent(), FrameParentResult::SameActivation); + assert!(frame.done()); + }) + .await?; + + let event = debugger.run().await?; + // At return point. + assert!(matches!(event, DebugRunResult::Breakpoint)); + debugger + .with_store(|store| { + let mut frame = store.debug_frames().unwrap(); + assert!(!frame.done()); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().0.as_u32(), 0); + assert_eq!(frame.wasm_function_index_and_pc().unwrap().1, 41); + assert_eq!(frame.num_locals(), 2); + assert_eq!(frame.num_stacks(), 1); + assert_eq!(frame.local(0).unwrap_i32(), 1); + assert_eq!(frame.local(1).unwrap_i32(), 2); + assert_eq!(frame.stack(0).unwrap_i32(), 3); + assert_eq!(frame.move_to_parent(), FrameParentResult::SameActivation); + assert!(frame.done()); + }) + .await?; + + // Now disable breakpoints before continuing. Second call should proceed with no more events. + debugger + .with_store(|store| { + store + .edit_breakpoints() + .unwrap() + .single_step(false) + .unwrap(); + }) + .await?; + + let event = debugger.run().await?; + assert!(matches!(event, DebugRunResult::Finished)); + + assert!(debugger.is_complete()); + + // Ensure the store still works and the debug handler is + // removed. + let mut store = debugger.into_store(); + let mut results = [Val::I32(0)]; + main.call_async(&mut store, &[Val::I32(10), Val::I32(20)], &mut results[..]) + .await?; + assert_eq!(results[0].unwrap_i32(), 30); + + Ok(()) + } + + #[tokio::test] + #[cfg_attr(miri, ignore)] + async fn early_finish() -> Result<()> { + let _ = env_logger::try_init(); + + let mut config = Config::new(); + config.guest_debug(true); + config.async_support(true); + let engine = Engine::new(&config)?; + let module = Module::new( + &engine, + r#" + (module + (func (export "main") (param i32 i32) (result i32) + local.get 0 + local.get 1 + i32.add)) + "#, + )?; + + let mut store = Store::new(&engine, ()); + let instance = Instance::new_async(&mut store, &module, &[]).await?; + let main = instance.get_func(&mut store, "main").unwrap(); + + let mut debugger = Debugger::new(store, move |mut store| async move { + let mut results = [Val::I32(0)]; + store.edit_breakpoints().unwrap().single_step(true).unwrap(); + main.call_async(&mut store, &[Val::I32(1), Val::I32(2)], &mut results[..]) + .await?; + assert_eq!(results[0].unwrap_i32(), 3); + Ok(store) + }); + + debugger.finish().await?; + assert!(debugger.is_complete()); + + Ok(()) + } +} From 978f7b729c58d778f16b8cab80c05f9176de103e Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Thu, 18 Dec 2025 22:34:39 -0800 Subject: [PATCH 2/8] Cancel safety! --- crates/debugger/src/lib.rs | 216 +++++++++++++++++++++++++++---------- 1 file changed, 158 insertions(+), 58 deletions(-) diff --git a/crates/debugger/src/lib.rs b/crates/debugger/src/lib.rs index 74a109af6208..87beceaae055 100644 --- a/crates/debugger/src/lib.rs +++ b/crates/debugger/src/lib.rs @@ -37,23 +37,38 @@ use wasmtime::{ /// true. `Debugger::finish` continues execution ignoring all further /// events to allow clean completion if needed. pub struct Debugger { + /// The inner task that this debugger wraps. + inner: Option>>>, /// State: either a task handle or the store when passed out of /// the complete task. - state: DebuggerState, + state: DebuggerState, in_tx: mpsc::Sender>, out_rx: mpsc::Receiver, } -enum DebuggerState { - /// Inner body is running in an async task. - Running(JoinHandle>>), - /// Temporary state while we are joining. - Joining, - /// Inner body is complete and has passed the store back. - Complete(Store), - /// Debugger has been disassembled via `into_store()`. Allows the - /// `Drop` impl to verify that the debugger is complete. - Destructed, +/// State machine from the perspective of the outer logic. +/// +/// The intermediate states here, and the separation of these states +/// from the `JoinHandle` above, are what allow us to implement a +/// cancel-safe version of `Debugger::run` below. +#[derive(Clone, Copy, Debug, PartialEq, Eq)] +enum DebuggerState { + /// Inner body is running in an async task and not in a debugger + /// callback. Outer logic is waiting for a `Response::Stopped` or + /// `Response::Complete`. + Running, + /// Inner body is running in an async task and at a debugger + /// callback (or in the initial trampoline waiting for the first + /// `Continue`). `Response::Stopped` has been received. Outer + /// logic has not sent any commands. + Stopped, + /// We have sent a command to the inner body and are waiting for a + /// response. + Queried, + /// Inner body is complete (has sent `Response::Finished` and we + /// have received it). We may or may not have joined yet; if so, + /// the `Option>` will be `None`. + Complete, } /// Message from "outside" to the debug hook. @@ -203,7 +218,8 @@ impl Debugger { }); Debugger { - state: DebuggerState::Running(inner), + inner: Some(inner), + state: DebuggerState::Stopped, in_tx, out_rx, } @@ -211,24 +227,69 @@ impl Debugger { /// Is the inner body done running? pub fn is_complete(&self) -> bool { - match &self.state { - DebuggerState::Running(_) | DebuggerState::Joining => false, - DebuggerState::Complete(_) => true, - DebuggerState::Destructed => { - panic!("Should not see this state outside of `into_store()`") - } + match self.state { + DebuggerState::Complete => true, + _ => false, } } /// Run the inner body until the next debug event. + /// + /// This method is cancel-safe, and no events will be lost. pub async fn run(&mut self) -> Result { - anyhow::ensure!(!self.is_complete(), "Debugger body is already complete"); + log::trace!("running: state is {:?}", self.state); + match self.state { + DebuggerState::Stopped => { + log::trace!("sending Continue"); + self.in_tx + .send(Command::Continue) + .await + .map_err(|_| anyhow::anyhow!("Failed to send over debug channel"))?; + log::trace!("sent Continue"); + + // If that `send` was canceled, the command was not + // sent, so it's fine to remain in `Stopped`. If it + // succeeded and we reached here, transition to + // `Running` so we don't re-send. + self.state = DebuggerState::Running; + } + DebuggerState::Running => { + // Previous `run()` must have been canceled; no action + // to take here. + } + DebuggerState::Queried => { + // We expect to receive a `QueryResponse`; drop it if + // the query was canceled, then transition back to + // `Stopped`. + log::trace!("in Queried; receiving"); + let response = self + .out_rx + .recv() + .await + .ok_or_else(|| anyhow::anyhow!("Premature close of debugger channel"))?; + log::trace!("in Queried; received, dropping"); + assert!(matches!(response, Response::QueryResponse(_))); + self.state = DebuggerState::Stopped; - self.in_tx - .send(Command::Continue) - .await - .map_err(|_| anyhow::anyhow!("Failed to send over debug channel"))?; + // Now send a `Continue`, as above. + log::trace!("in Stopped; sending Continue"); + self.in_tx + .send(Command::Continue) + .await + .map_err(|_| anyhow::anyhow!("Failed to send over debug channel"))?; + self.state = DebuggerState::Running; + } + DebuggerState::Complete => { + panic!("Cannot `run()` an already-complete Debugger"); + } + } + // At this point, the inner task is in Running state. We + // expect to receive a message when it next stops or + // completes. If this `recv()` is canceled, no message is + // lost, and the state above accurately reflects what must be + // done on the next `run()`. + log::trace!("waiting for response"); let response = self .out_rx .recv() @@ -237,18 +298,17 @@ impl Debugger { match response { Response::Finished => { - let DebuggerState::Running(joinhandle) = - std::mem::replace(&mut self.state, DebuggerState::Joining) - else { - panic!("State was verified to be `Running` above"); - }; - let store = joinhandle.await??; - self.state = DebuggerState::Complete(store); + log::trace!("got Finished"); + self.state = DebuggerState::Complete; Ok(DebugRunResult::Finished) } - Response::Stopped(result) => Ok(result), + Response::Stopped(result) => { + log::trace!("got Stopped"); + self.state = DebuggerState::Stopped; + Ok(result) + } Response::QueryResponse(_) => { - anyhow::bail!("Invalid debug response"); + panic!("Invalid debug response"); } } } @@ -271,6 +331,16 @@ impl Debugger { } /// Perform some action on the contained `Store` while not running. + /// + /// This may only be invoked before the inner body finishes and + /// when it is stopped; that is, when the `Debugger` is initially + /// created and after any call to `run()` returns a result other + /// than `DebugRunResult::Finished`. If an earlier `run()` + /// invocation was canceled, it must be re-invoked and return + /// successfully before a query is made. + /// + /// This is cancel-safe; if canceled, the result of the query will + /// be dropped. pub async fn with_store< F: FnOnce(StoreContextMut<'_, T>) -> R + Send + 'static, R: Send + 'static, @@ -278,40 +348,71 @@ impl Debugger { &mut self, f: F, ) -> Result { - match &mut self.state { - DebuggerState::Running(_) => { - self.in_tx - .send(Command::Query(Box::new(|store| Box::new(f(store))))) - .await - .map_err(|_| anyhow::anyhow!("Premature close of debugger channel"))?; + assert!(!self.is_complete()); + + match self.state { + DebuggerState::Queried => { + // Earlier query canceled; drop its response first. let response = self .out_rx .recv() .await .ok_or_else(|| anyhow::anyhow!("Premature close of debugger channel"))?; - let Response::QueryResponse(resp) = response else { - anyhow::bail!("Incorrect response from debugger task"); - }; - Ok(*resp.downcast::().expect("type mismatch")) + assert!(matches!(response, Response::QueryResponse(_))); + self.state = DebuggerState::Stopped; + } + DebuggerState::Running => { + // Results from a canceled `run()`; `run()` must + // complete before this can be invoked. + panic!("Cannot query in Running state"); + } + DebuggerState::Complete => { + panic!("Cannot query when complete"); } - DebuggerState::Joining => anyhow::bail!("Join failed with error and Store is lost"), - DebuggerState::Complete(store) => Ok(f(store.as_context_mut())), - DebuggerState::Destructed => { - panic!("Should not see `Destructed` state outside of `into_store`") + DebuggerState::Stopped => { + // OK -- this is the state we want. } } + + self.in_tx + .send(Command::Query(Box::new(|store| Box::new(f(store))))) + .await + .map_err(|_| anyhow::anyhow!("Premature close of debugger channel"))?; + self.state = DebuggerState::Queried; + + let response = self + .out_rx + .recv() + .await + .ok_or_else(|| anyhow::anyhow!("Premature close of debugger channel"))?; + let Response::QueryResponse(resp) = response else { + anyhow::bail!("Incorrect response from debugger task"); + }; + self.state = DebuggerState::Stopped; + + Ok(*resp.downcast::().expect("type mismatch")) } /// Drop the Debugger once complete, returning the inner `Store` /// around which it was wrapped. - pub fn into_store(mut self) -> Store { - let state = std::mem::replace(&mut self.state, DebuggerState::Destructed); - let mut store = match state { - DebuggerState::Complete(store) => store, - _ => panic!("Cannot invoke into_store() on a non-complete Debugger"), - }; - store.clear_debug_handler(); - store + /// + /// Only valid to invoke once `run()` returns + /// `DebugRunResult::Finished`. + /// + /// This is cancel-safe, but if canceled, the Store is lost. + pub async fn take_store(&mut self) -> Result>> { + match self.state { + DebuggerState::Complete => { + let inner = match self.inner.take() { + Some(inner) => inner, + None => return Ok(None), + }; + let mut store = inner.await??; + store.clear_debug_handler(); + Ok(Some(store)) + } + _ => panic!("Invalid state: debugger not yet complete"), + } } } @@ -322,9 +423,8 @@ impl Drop for Debugger { // -- in general, Wasmtime's futures that embody Wasm // execution are not cancel-safe, so we have to wait for the // inner body to finish before the Debugger is dropped. - match &self.state { - DebuggerState::Complete(_) | DebuggerState::Destructed => {} - _ => panic!("Dropping Debugger before inner body is complete"), + if self.state != DebuggerState::Complete { + panic!("Dropping Debugger before inner body is complete"); } } } @@ -486,7 +586,7 @@ mod test { // Ensure the store still works and the debug handler is // removed. - let mut store = debugger.into_store(); + let mut store = debugger.take_store().await?.unwrap(); let mut results = [Val::I32(0)]; main.call_async(&mut store, &[Val::I32(10), Val::I32(20)], &mut results[..]) .await?; From 1fcef74adf9bfdf42160881447749f5e20c68438 Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Fri, 19 Dec 2025 09:43:01 -0800 Subject: [PATCH 3/8] Add new crate to publish.rs script. --- scripts/publish.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/scripts/publish.rs b/scripts/publish.rs index 7106e05cc6bc..3db4341fe6c4 100644 --- a/scripts/publish.rs +++ b/scripts/publish.rs @@ -87,6 +87,7 @@ const CRATES_TO_PUBLISH: &[&str] = &[ "wasmtime-wizer", "wasmtime-cli-flags", "wasmtime-internal-explorer", + "wasmtime-internal-debugger", "wasmtime-cli", ]; From fac011ac15ccdef503841b073b1886215bd43bb2 Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Mon, 22 Dec 2025 09:19:37 -0800 Subject: [PATCH 4/8] Review feedback. --- crates/debugger/Cargo.toml | 7 +-- crates/debugger/src/lib.rs | 114 +++++++++++++++++-------------------- 2 files changed, 56 insertions(+), 65 deletions(-) diff --git a/crates/debugger/Cargo.toml b/crates/debugger/Cargo.toml index 0a22f28dbaa2..619925688468 100644 --- a/crates/debugger/Cargo.toml +++ b/crates/debugger/Cargo.toml @@ -6,7 +6,6 @@ description = "INTERNAL: Wasmtime's guest-debugger functionality" license = "Apache-2.0 WITH LLVM-exception" documentation = "https://docs.rs/wasmtime-debugger" repository = "https://github.com/bytecodealliance/wasmtime" -categories = ["no-std"] readme = "README.md" keywords = ["debugger"] edition.workspace = true @@ -22,9 +21,9 @@ anyhow = { workspace = true } log = { workspace = true } [dev-dependencies] -# Depend on `wasmtime` again to get `cranelift` so we can write unit -# tests that build modules. -wasmtime = { workspace = true, features = ["debug", "std", "async", "cranelift", "wat"] } +# Depend on `wasmtime` again to get `cranelift` and `wat` so we can +# write unit tests that build modules from textual WAT. +wasmtime = { workspace = true, features = ["cranelift", "wat"] } env_logger = { workspace = true } [features] diff --git a/crates/debugger/src/lib.rs b/crates/debugger/src/lib.rs index 87beceaae055..4ab15ac38c3c 100644 --- a/crates/debugger/src/lib.rs +++ b/crates/debugger/src/lib.rs @@ -26,8 +26,8 @@ use wasmtime::{ /// performs some actions on a store. Those actions are subject to the /// debugger, and debugger events will be raised as appropriate. From /// the "outside" of this combinator, it is always in one of two -/// states: running or stopped. When stopped, it acts as a -/// `StoreContextMut` and can allow examining the stopped execution's +/// states: running or paused. When paused, it acts as a +/// `StoreContextMut` and can allow examining the paused execution's /// state. One runs until the next event suspends execution by /// invoking `Debugger::run`. /// @@ -54,14 +54,14 @@ pub struct Debugger { #[derive(Clone, Copy, Debug, PartialEq, Eq)] enum DebuggerState { /// Inner body is running in an async task and not in a debugger - /// callback. Outer logic is waiting for a `Response::Stopped` or + /// callback. Outer logic is waiting for a `Response::Paused` or /// `Response::Complete`. Running, /// Inner body is running in an async task and at a debugger /// callback (or in the initial trampoline waiting for the first - /// `Continue`). `Response::Stopped` has been received. Outer + /// `Continue`). `Response::Paused` has been received. Outer /// logic has not sent any commands. - Stopped, + Paused, /// We have sent a command to the inner body and are waiting for a /// response. Queried, @@ -107,7 +107,7 @@ enum Command { } enum Response { - Stopped(DebugRunResult), + Paused(DebugRunResult), QueryResponse(Box), Finished, } @@ -127,44 +127,36 @@ impl std::clone::Clone for Handler { impl DebugHandler for Handler { type Data = T; - fn handle( - &self, - mut store: StoreContextMut<'_, T>, - event: DebugEvent<'_>, - ) -> impl Future + Send { - async move { - let mut in_rx = self.0.in_rx.lock().await; - - let result = match event { - DebugEvent::HostcallError(_) => DebugRunResult::HostcallError, - DebugEvent::CaughtExceptionThrown(exn) => { - DebugRunResult::CaughtExceptionThrown(exn) - } - DebugEvent::UncaughtExceptionThrown(exn) => { - DebugRunResult::UncaughtExceptionThrown(exn) + async fn handle(&self, mut store: StoreContextMut<'_, T>, event: DebugEvent<'_>) { + let mut in_rx = self.0.in_rx.lock().await; + + let result = match event { + DebugEvent::HostcallError(_) => DebugRunResult::HostcallError, + DebugEvent::CaughtExceptionThrown(exn) => DebugRunResult::CaughtExceptionThrown(exn), + DebugEvent::UncaughtExceptionThrown(exn) => { + DebugRunResult::UncaughtExceptionThrown(exn) + } + DebugEvent::Trap(trap) => DebugRunResult::Trap(trap), + DebugEvent::Breakpoint => DebugRunResult::Breakpoint, + }; + self.0 + .out_tx + .send(Response::Paused(result)) + .await + .expect("outbound channel closed prematurely"); + + while let Some(cmd) = in_rx.recv().await { + match cmd { + Command::Query(closure) => { + let result = closure(store.as_context_mut()); + self.0 + .out_tx + .send(Response::QueryResponse(result)) + .await + .expect("outbound channel closed prematurely"); } - DebugEvent::Trap(trap) => DebugRunResult::Trap(trap), - DebugEvent::Breakpoint => DebugRunResult::Breakpoint, - }; - self.0 - .out_tx - .send(Response::Stopped(result)) - .await - .expect("outbound channel closed prematurely"); - - while let Some(cmd) = in_rx.recv().await { - match cmd { - Command::Query(closure) => { - let result = closure(store.as_context_mut()); - self.0 - .out_tx - .send(Response::QueryResponse(result)) - .await - .expect("outbound channel closed prematurely"); - } - Command::Continue => { - break; - } + Command::Continue => { + break; } } } @@ -176,15 +168,15 @@ impl Debugger { /// runs the given inner body. /// /// The debugger is always in one of two states: running or - /// stopped. + /// paused. /// - /// When stopped, the holder of this object can invoke + /// When paused, the holder of this object can invoke /// `Debugger::run` to enter the running state. The inner body - /// will run until stopped by a debug event. While running, the + /// will run until paused by a debug event. While running, the /// future returned by either of these methods owns the `Debugger` /// and hence no other methods can be invoked. /// - /// When stopped, the holder of this object can access the `Store` + /// When paused, the holder of this object can access the `Store` /// indirectly by providing a closure pub fn new(mut store: Store, inner: F) -> Debugger where @@ -219,7 +211,7 @@ impl Debugger { Debugger { inner: Some(inner), - state: DebuggerState::Stopped, + state: DebuggerState::Paused, in_tx, out_rx, } @@ -239,7 +231,7 @@ impl Debugger { pub async fn run(&mut self) -> Result { log::trace!("running: state is {:?}", self.state); match self.state { - DebuggerState::Stopped => { + DebuggerState::Paused => { log::trace!("sending Continue"); self.in_tx .send(Command::Continue) @@ -248,7 +240,7 @@ impl Debugger { log::trace!("sent Continue"); // If that `send` was canceled, the command was not - // sent, so it's fine to remain in `Stopped`. If it + // sent, so it's fine to remain in `Paused`. If it // succeeded and we reached here, transition to // `Running` so we don't re-send. self.state = DebuggerState::Running; @@ -260,7 +252,7 @@ impl Debugger { DebuggerState::Queried => { // We expect to receive a `QueryResponse`; drop it if // the query was canceled, then transition back to - // `Stopped`. + // `Paused`. log::trace!("in Queried; receiving"); let response = self .out_rx @@ -269,10 +261,10 @@ impl Debugger { .ok_or_else(|| anyhow::anyhow!("Premature close of debugger channel"))?; log::trace!("in Queried; received, dropping"); assert!(matches!(response, Response::QueryResponse(_))); - self.state = DebuggerState::Stopped; + self.state = DebuggerState::Paused; // Now send a `Continue`, as above. - log::trace!("in Stopped; sending Continue"); + log::trace!("in Paused; sending Continue"); self.in_tx .send(Command::Continue) .await @@ -285,7 +277,7 @@ impl Debugger { } // At this point, the inner task is in Running state. We - // expect to receive a message when it next stops or + // expect to receive a message when it next pauses or // completes. If this `recv()` is canceled, no message is // lost, and the state above accurately reflects what must be // done on the next `run()`. @@ -302,9 +294,9 @@ impl Debugger { self.state = DebuggerState::Complete; Ok(DebugRunResult::Finished) } - Response::Stopped(result) => { - log::trace!("got Stopped"); - self.state = DebuggerState::Stopped; + Response::Paused(result) => { + log::trace!("got Paused"); + self.state = DebuggerState::Paused; Ok(result) } Response::QueryResponse(_) => { @@ -333,7 +325,7 @@ impl Debugger { /// Perform some action on the contained `Store` while not running. /// /// This may only be invoked before the inner body finishes and - /// when it is stopped; that is, when the `Debugger` is initially + /// when it is paused; that is, when the `Debugger` is initially /// created and after any call to `run()` returns a result other /// than `DebugRunResult::Finished`. If an earlier `run()` /// invocation was canceled, it must be re-invoked and return @@ -359,7 +351,7 @@ impl Debugger { .await .ok_or_else(|| anyhow::anyhow!("Premature close of debugger channel"))?; assert!(matches!(response, Response::QueryResponse(_))); - self.state = DebuggerState::Stopped; + self.state = DebuggerState::Paused; } DebuggerState::Running => { // Results from a canceled `run()`; `run()` must @@ -369,7 +361,7 @@ impl Debugger { DebuggerState::Complete => { panic!("Cannot query when complete"); } - DebuggerState::Stopped => { + DebuggerState::Paused => { // OK -- this is the state we want. } } @@ -388,7 +380,7 @@ impl Debugger { let Response::QueryResponse(resp) = response else { anyhow::bail!("Incorrect response from debugger task"); }; - self.state = DebuggerState::Stopped; + self.state = DebuggerState::Paused; Ok(*resp.downcast::().expect("type mismatch")) } From cce7e85d8ca1c449f2237cb42e5867de0ec2ad8d Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Mon, 22 Dec 2025 10:20:05 -0800 Subject: [PATCH 5/8] Review feedback: state diagram. --- crates/debugger/src/lib.rs | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/crates/debugger/src/lib.rs b/crates/debugger/src/lib.rs index 4ab15ac38c3c..67333ec7ba2c 100644 --- a/crates/debugger/src/lib.rs +++ b/crates/debugger/src/lib.rs @@ -51,6 +51,34 @@ pub struct Debugger { /// The intermediate states here, and the separation of these states /// from the `JoinHandle` above, are what allow us to implement a /// cancel-safe version of `Debugger::run` below. +/// +/// The state diagram for the outer logic is: +/// +/// ```plain +/// (start) +/// v +/// | +/// .--->---------. v +/// | .----< Paused <-----------------------------------------------. +/// | | v | +/// | | | (async fn run() starts, sends Command::Continue) | +/// | | | | +/// | | v ^ +/// | | Running | +/// | | v v (async fn run() receives Response::Paused, returns) | +/// | | | |_____________________________________________________| +/// | | | +/// | | | (async fn run() receives Response::Finished, returns) +/// | | v +/// | | Complete +/// | | +/// ^ | (async fn with_store() starts, sends Command::Query) +/// | v +/// | Queried +/// | | +/// | | (async fn with_store() receives Response::QueryResponse, returns) +/// `---<-' +/// ``` #[derive(Clone, Copy, Debug, PartialEq, Eq)] enum DebuggerState { /// Inner body is running in an async task and not in a debugger From 014c18f7073cf4a0402135d646f24c13aacec200 Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Mon, 22 Dec 2025 12:44:25 -0800 Subject: [PATCH 6/8] Update after merge from main: new DebugEvent for epoch yields. --- crates/debugger/src/lib.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/crates/debugger/src/lib.rs b/crates/debugger/src/lib.rs index 67333ec7ba2c..6ff5133bdd27 100644 --- a/crates/debugger/src/lib.rs +++ b/crates/debugger/src/lib.rs @@ -166,6 +166,7 @@ impl DebugHandler for Handler { } DebugEvent::Trap(trap) => DebugRunResult::Trap(trap), DebugEvent::Breakpoint => DebugRunResult::Breakpoint, + DebugEvent::EpochYield => DebugRunResult::EpochYield, }; self.0 .out_tx @@ -460,6 +461,8 @@ pub enum DebugRunResult { Finished, /// An error was raised by a hostcall. HostcallError, + /// Wasm execution was interrupted by an epoch change. + EpochYield, /// An exception is thrown and caught by Wasm. The current state /// is at the throw-point. CaughtExceptionThrown(OwnedRooted), From 80a5467cc096eb2ae7708a5df68175eeef86bf79 Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Mon, 22 Dec 2025 12:49:23 -0800 Subject: [PATCH 7/8] Make Debugger drop-safe by making debug event callbacks compatible with fiber teardown. --- crates/debugger/src/lib.rs | 64 +++++++++++++++++++--------- crates/wasmtime/src/runtime/fiber.rs | 2 +- crates/wasmtime/src/runtime/store.rs | 3 ++ 3 files changed, 48 insertions(+), 21 deletions(-) diff --git a/crates/debugger/src/lib.rs b/crates/debugger/src/lib.rs index 6ff5133bdd27..e575dfc6c9a7 100644 --- a/crates/debugger/src/lib.rs +++ b/crates/debugger/src/lib.rs @@ -30,12 +30,6 @@ use wasmtime::{ /// `StoreContextMut` and can allow examining the paused execution's /// state. One runs until the next event suspends execution by /// invoking `Debugger::run`. -/// -/// Note that because of limitations in Wasmtime's future cancelation -/// handling, all events must be consumed until the inner body -/// completes and `Debugger::is_complete` returns -/// true. `Debugger::finish` continues execution ignoring all further -/// events to allow clean completion if needed. pub struct Debugger { /// The inner task that this debugger wraps. inner: Option>>>, @@ -418,7 +412,8 @@ impl Debugger { /// around which it was wrapped. /// /// Only valid to invoke once `run()` returns - /// `DebugRunResult::Finished`. + /// `DebugRunResult::Finished` or after calling `finish()` (which + /// finishes execution while dropping all further debug events). /// /// This is cancel-safe, but if canceled, the Store is lost. pub async fn take_store(&mut self) -> Result>> { @@ -437,19 +432,6 @@ impl Debugger { } } -impl Drop for Debugger { - fn drop(&mut self) { - // We cannot allow this because the fiber implementation will - // panic if a `Func::call_async` future is dropped prematurely - // -- in general, Wasmtime's futures that embody Wasm - // execution are not cancel-safe, so we have to wait for the - // inner body to finish before the Debugger is dropped. - if self.state != DebuggerState::Complete { - panic!("Dropping Debugger before inner body is complete"); - } - } -} - /// The result of one call to `Debugger::run()`. /// /// This is similar to `DebugEvent` but without the lifetime, so it @@ -656,4 +638,46 @@ mod test { Ok(()) } + + #[tokio::test] + #[cfg_attr(miri, ignore)] + async fn drop_debugger_and_store() -> Result<()> { + let _ = env_logger::try_init(); + + let mut config = Config::new(); + config.guest_debug(true); + config.async_support(true); + let engine = Engine::new(&config)?; + let module = Module::new( + &engine, + r#" + (module + (func (export "main") (param i32 i32) (result i32) + local.get 0 + local.get 1 + i32.add)) + "#, + )?; + + let mut store = Store::new(&engine, ()); + let instance = Instance::new_async(&mut store, &module, &[]).await?; + let main = instance.get_func(&mut store, "main").unwrap(); + + let mut debugger = Debugger::new(store, move |mut store| async move { + let mut results = [Val::I32(0)]; + store.edit_breakpoints().unwrap().single_step(true).unwrap(); + main.call_async(&mut store, &[Val::I32(1), Val::I32(2)], &mut results[..]) + .await?; + assert_eq!(results[0].unwrap_i32(), 3); + Ok(store) + }); + + // Step once, then drop everything at the end of this + // function. Wasmtime's fiber cleanup should safely happen + // without attempting to raise debug async handler calls with + // missing async context. + let _ = debugger.run().await?; + + Ok(()) + } } diff --git a/crates/wasmtime/src/runtime/fiber.rs b/crates/wasmtime/src/runtime/fiber.rs index 68dce6884f36..1ab5120ef30e 100644 --- a/crates/wasmtime/src/runtime/fiber.rs +++ b/crates/wasmtime/src/runtime/fiber.rs @@ -358,7 +358,7 @@ impl StoreOpaque { } /// Returns whether `block_on` will succeed or panic. - #[cfg(feature = "call-hook")] + #[cfg(any(feature = "call-hook", feature = "debug"))] pub(crate) fn can_block(&mut self) -> bool { self.fiber_async_state_mut().current_future_cx.is_some() } diff --git a/crates/wasmtime/src/runtime/store.rs b/crates/wasmtime/src/runtime/store.rs index 0ce69fc6c341..aae8397212ac 100644 --- a/crates/wasmtime/src/runtime/store.rs +++ b/crates/wasmtime/src/runtime/store.rs @@ -2839,6 +2839,9 @@ unsafe impl VMStore for StoreInner { #[cfg(feature = "debug")] fn block_on_debug_handler(&mut self, event: crate::DebugEvent<'_>) -> anyhow::Result<()> { if let Some(handler) = self.debug_handler.take() { + if !self.can_block() { + bail!("could not invoke debug handler without async context"); + } log::trace!("about to raise debug event {event:?}"); StoreContextMut(self).with_blocking(|store, cx| { cx.block_on(Pin::from(handler.handle(store, event)).as_mut()) From 2b720faa1e01bdcd891dc1dd8262dcdc52dcf779 Mon Sep 17 00:00:00 2001 From: Chris Fallin Date: Wed, 24 Dec 2025 10:47:02 -0800 Subject: [PATCH 8/8] CI fix on `debugger` crate manifest. - Do not link to non-existent README in new crate. - Remove a few other attributes that our internal crates don't have (now the set of attributes at the top level is the same as for the new error crate). --- crates/debugger/Cargo.toml | 3 --- 1 file changed, 3 deletions(-) diff --git a/crates/debugger/Cargo.toml b/crates/debugger/Cargo.toml index 619925688468..a0a5a383f2bc 100644 --- a/crates/debugger/Cargo.toml +++ b/crates/debugger/Cargo.toml @@ -4,9 +4,6 @@ authors.workspace = true version.workspace = true description = "INTERNAL: Wasmtime's guest-debugger functionality" license = "Apache-2.0 WITH LLVM-exception" -documentation = "https://docs.rs/wasmtime-debugger" -repository = "https://github.com/bytecodealliance/wasmtime" -readme = "README.md" keywords = ["debugger"] edition.workspace = true rust-version.workspace = true