From 6ba02589fc521b115196506c6eb0488dd1790e18 Mon Sep 17 00:00:00 2001 From: Roman Volosatovs Date: Mon, 15 Sep 2025 16:25:30 +0200 Subject: [PATCH 1/3] async: add stream/future producers and blanket impl for `Future` (#11684) * async: add `EmptyProducer` and blanket impl for `Future` Signed-off-by: Roman Volosatovs * move more future/stream producers to `wasmtime` crate Signed-off-by: Roman Volosatovs * async: allow blanket `Future` impl to trap Signed-off-by: Roman Volosatovs * async: remove the need for `{Ready,Empty}Producer` Signed-off-by: Roman Volosatovs --------- Signed-off-by: Roman Volosatovs --- crates/wasi-http/src/p3/body.rs | 10 +- crates/wasi/src/p3/filesystem/host.rs | 36 +-- crates/wasi/src/p3/mod.rs | 81 +----- crates/wasi/src/p3/sockets/host/types/tcp.rs | 17 +- .../concurrent/futures_and_streams.rs | 261 +++++++++++++++++- 5 files changed, 274 insertions(+), 131 deletions(-) diff --git a/crates/wasi-http/src/p3/body.rs b/crates/wasi-http/src/p3/body.rs index dc1f10d31182..d9b40e421208 100644 --- a/crates/wasi-http/src/p3/body.rs +++ b/crates/wasi-http/src/p3/body.rs @@ -2,6 +2,7 @@ use crate::p3::bindings::http::types::{ErrorCode, Fields, Trailers}; use crate::p3::{WasiHttp, WasiHttpCtxView}; use anyhow::Context as _; use bytes::Bytes; +use core::iter; use core::num::NonZeroUsize; use core::pin::Pin; use core::task::{Context, Poll, ready}; @@ -17,7 +18,6 @@ use wasmtime::component::{ StreamProducer, StreamReader, StreamResult, }; use wasmtime::{AsContextMut, StoreContextMut}; -use wasmtime_wasi::p3::{FutureOneshotProducer, StreamEmptyProducer}; /// The concrete type behind a `wasi:http/types/body` resource. pub(crate) enum Body { @@ -75,7 +75,7 @@ impl Body { // https://github.com/WebAssembly/wasi-http/issues/176 _ = result_tx.send(Box::new(async { Ok(()) })); Ok(( - StreamReader::new(instance, &mut store, StreamEmptyProducer::default()), + StreamReader::new(instance, &mut store, iter::empty()), trailers_rx, )) } @@ -95,11 +95,7 @@ impl Body { getter, }, ), - FutureReader::new( - instance, - &mut store, - FutureOneshotProducer::from(trailers_rx), - ), + FutureReader::new(instance, &mut store, trailers_rx), )) } Body::Consumed => Err(()), diff --git a/crates/wasi/src/p3/filesystem/host.rs b/crates/wasi/src/p3/filesystem/host.rs index 6bba67ab568d..df94cdefb789 100644 --- a/crates/wasi/src/p3/filesystem/host.rs +++ b/crates/wasi/src/p3/filesystem/host.rs @@ -5,16 +5,13 @@ use crate::p3::bindings::filesystem::types::{ Filesize, MetadataHashValue, NewTimestamp, OpenFlags, PathFlags, }; use crate::p3::filesystem::{FilesystemError, FilesystemResult, preopens}; -use crate::p3::{ - DEFAULT_BUFFER_CAPACITY, FallibleIteratorProducer, FutureOneshotProducer, FutureReadyProducer, - StreamEmptyProducer, -}; +use crate::p3::{DEFAULT_BUFFER_CAPACITY, FallibleIteratorProducer}; use crate::{DirPerms, FilePerms}; use anyhow::Context as _; use bytes::BytesMut; -use core::mem; use core::pin::Pin; use core::task::{Context, Poll, ready}; +use core::{iter, mem}; use std::io::{self, Cursor}; use std::sync::Arc; use system_interface::fs::FileIoExt as _; @@ -498,12 +495,10 @@ impl types::HostDescriptorWithStore for WasiFilesystem { let file = get_file(store.get().table, &fd)?; if !file.perms.contains(FilePerms::READ) { return Ok(( - StreamReader::new(instance, &mut store, StreamEmptyProducer::default()), - FutureReader::new( - instance, - &mut store, - FutureReadyProducer(Some(Err(ErrorCode::NotPermitted))), - ), + StreamReader::new(instance, &mut store, iter::empty()), + FutureReader::new(instance, &mut store, async { + anyhow::Ok(Err(ErrorCode::NotPermitted)) + }), )); } @@ -520,7 +515,7 @@ impl types::HostDescriptorWithStore for WasiFilesystem { task: None, }, ), - FutureReader::new(instance, &mut store, FutureOneshotProducer(result_rx)), + FutureReader::new(instance, &mut store, result_rx), )) }) } @@ -644,12 +639,10 @@ impl types::HostDescriptorWithStore for WasiFilesystem { let dir = get_dir(store.get().table, &fd)?; if !dir.perms.contains(DirPerms::READ) { return Ok(( - StreamReader::new(instance, &mut store, StreamEmptyProducer::default()), - FutureReader::new( - instance, - &mut store, - FutureReadyProducer(Some(Err(ErrorCode::NotPermitted))), - ), + StreamReader::new(instance, &mut store, iter::empty()), + FutureReader::new(instance, &mut store, async { + anyhow::Ok(Err(ErrorCode::NotPermitted)) + }), )); } let allow_blocking_current_thread = dir.allow_blocking_current_thread; @@ -667,16 +660,13 @@ impl types::HostDescriptorWithStore for WasiFilesystem { ), Err(e) => { result_tx.send(Err(e.into())).unwrap(); - StreamReader::new(instance, &mut store, StreamEmptyProducer::default()) + StreamReader::new(instance, &mut store, iter::empty()) } } } else { StreamReader::new(instance, &mut store, ReadDirStream::new(dir, result_tx)) }; - Ok(( - stream, - FutureReader::new(instance, &mut store, FutureOneshotProducer(result_rx)), - )) + Ok((stream, FutureReader::new(instance, &mut store, result_rx))) }) } diff --git a/crates/wasi/src/p3/mod.rs b/crates/wasi/src/p3/mod.rs index 5ca938895def..5e5fac2d65a3 100644 --- a/crates/wasi/src/p3/mod.rs +++ b/crates/wasi/src/p3/mod.rs @@ -17,94 +17,15 @@ pub mod sockets; use crate::WasiView; use crate::p3::bindings::LinkOptions; -use anyhow::Context as _; -use core::marker::PhantomData; use core::pin::Pin; use core::task::{Context, Poll}; use tokio::sync::oneshot; use wasmtime::StoreContextMut; -use wasmtime::component::{ - Destination, FutureProducer, Linker, StreamProducer, StreamResult, VecBuffer, -}; +use wasmtime::component::{Destination, Linker, StreamProducer, StreamResult, VecBuffer}; // Default buffer capacity to use for reads of byte-sized values. const DEFAULT_BUFFER_CAPACITY: usize = 8192; -pub struct StreamEmptyProducer(PhantomData T>); - -impl Default for StreamEmptyProducer { - fn default() -> Self { - Self(PhantomData) - } -} - -impl StreamProducer for StreamEmptyProducer { - type Item = T; - type Buffer = Option; - - fn poll_produce<'a>( - self: Pin<&mut Self>, - _: &mut Context<'_>, - _: StoreContextMut<'a, D>, - _: Destination<'a, Self::Item, Self::Buffer>, - _: bool, - ) -> Poll> { - Poll::Ready(Ok(StreamResult::Dropped)) - } -} - -struct FutureReadyProducer(Option); - -impl FutureProducer for FutureReadyProducer -where - T: Send + Unpin + 'static, -{ - type Item = T; - - fn poll_produce( - self: Pin<&mut Self>, - _: &mut Context<'_>, - _: StoreContextMut, - _: bool, - ) -> Poll>> { - let v = self - .get_mut() - .0 - .take() - .context("polled after returning `Ready`")?; - Poll::Ready(Ok(Some(v))) - } -} - -pub struct FutureOneshotProducer(oneshot::Receiver); - -impl From> for FutureOneshotProducer { - fn from(rx: oneshot::Receiver) -> Self { - Self(rx) - } -} - -impl FutureProducer for FutureOneshotProducer -where - T: Send + 'static, -{ - type Item = T; - - fn poll_produce( - self: Pin<&mut Self>, - cx: &mut Context<'_>, - _: StoreContextMut, - finish: bool, - ) -> Poll>> { - match Pin::new(&mut self.get_mut().0).poll(cx) { - Poll::Ready(Ok(v)) => Poll::Ready(Ok(Some(v))), - Poll::Ready(Err(err)) => Poll::Ready(Err(err).context("oneshot sender dropped")), - Poll::Pending if finish => Poll::Ready(Ok(None)), - Poll::Pending => Poll::Pending, - } - } -} - /// Helper structure to convert an iterator of `Result` into a `stream` /// plus a `future>` in WIT. /// diff --git a/crates/wasi/src/p3/sockets/host/types/tcp.rs b/crates/wasi/src/p3/sockets/host/types/tcp.rs index abd6fa3ae4e9..441ce14db711 100644 --- a/crates/wasi/src/p3/sockets/host/types/tcp.rs +++ b/crates/wasi/src/p3/sockets/host/types/tcp.rs @@ -1,15 +1,14 @@ use super::is_addr_allowed; +use crate::p3::DEFAULT_BUFFER_CAPACITY; use crate::p3::bindings::sockets::types::{ Duration, ErrorCode, HostTcpSocket, HostTcpSocketWithStore, IpAddressFamily, IpSocketAddress, TcpSocket, }; use crate::p3::sockets::{SocketError, SocketResult, WasiSockets}; -use crate::p3::{ - DEFAULT_BUFFER_CAPACITY, FutureOneshotProducer, FutureReadyProducer, StreamEmptyProducer, -}; use crate::sockets::{NonInheritedOptions, SocketAddrUse, SocketAddressFamily, WasiSocketsCtxView}; use anyhow::Context as _; use bytes::BytesMut; +use core::iter; use core::pin::Pin; use core::task::{Context, Poll}; use io_lifetimes::AsSocketlike as _; @@ -351,16 +350,14 @@ impl HostTcpSocketWithStore for WasiSockets { result: Some(result_tx), }, ), - FutureReader::new(instance, &mut store, FutureOneshotProducer(result_rx)), + FutureReader::new(instance, &mut store, result_rx), )) } None => Ok(( - StreamReader::new(instance, &mut store, StreamEmptyProducer::default()), - FutureReader::new( - instance, - &mut store, - FutureReadyProducer(Some(Err(ErrorCode::InvalidState))), - ), + StreamReader::new(instance, &mut store, iter::empty()), + FutureReader::new(instance, &mut store, async { + anyhow::Ok(Err(ErrorCode::InvalidState)) + }), )), } }) diff --git a/crates/wasmtime/src/runtime/component/concurrent/futures_and_streams.rs b/crates/wasmtime/src/runtime/component/concurrent/futures_and_streams.rs index 0cad9aee3258..83f71b6501b4 100644 --- a/crates/wasmtime/src/runtime/component/concurrent/futures_and_streams.rs +++ b/crates/wasmtime/src/runtime/component/concurrent/futures_and_streams.rs @@ -9,21 +9,21 @@ use crate::store::{StoreOpaque, StoreToken}; use crate::vm::component::{ComponentInstance, HandleTable, TransmitLocalState}; use crate::vm::{AlwaysMut, VMStore}; use crate::{AsContextMut, StoreContextMut, ValRaw}; -use anyhow::{Context as _, Result, anyhow, bail}; +use anyhow::{Context as _, Error, Result, anyhow, bail}; use buffers::{Extender, SliceBuffer, UntypedWriteBuffer}; -use futures::FutureExt; +use core::fmt; +use core::future; +use core::iter; +use core::marker::PhantomData; +use core::mem::{self, MaybeUninit}; +use core::pin::Pin; +use core::task::{Context, Poll, Waker, ready}; use futures::channel::oneshot; +use futures::{FutureExt as _, stream}; use std::boxed::Box; -use std::fmt; -use std::future; use std::io::Cursor; -use std::iter; -use std::marker::PhantomData; -use std::mem::{self, MaybeUninit}; -use std::pin::Pin; use std::string::{String, ToString}; use std::sync::{Arc, Mutex}; -use std::task::{self, Context, Poll, Waker}; use std::vec::Vec; use wasmtime_environ::component::{ CanonicalAbiInfo, ComponentTypes, InterfaceType, OptionsIndex, @@ -509,6 +509,136 @@ pub trait StreamProducer: Send + 'static { ) -> Poll>; } +impl StreamProducer for iter::Empty +where + T: Send + Sync + 'static, +{ + type Item = T; + type Buffer = Option; + + fn poll_produce<'a>( + self: Pin<&mut Self>, + _: &mut Context<'_>, + _: StoreContextMut<'a, D>, + _: Destination<'a, Self::Item, Self::Buffer>, + _: bool, + ) -> Poll> { + Poll::Ready(Ok(StreamResult::Dropped)) + } +} + +impl StreamProducer for stream::Empty +where + T: Send + Sync + 'static, +{ + type Item = T; + type Buffer = Option; + + fn poll_produce<'a>( + self: Pin<&mut Self>, + _: &mut Context<'_>, + _: StoreContextMut<'a, D>, + _: Destination<'a, Self::Item, Self::Buffer>, + _: bool, + ) -> Poll> { + Poll::Ready(Ok(StreamResult::Dropped)) + } +} + +impl StreamProducer for Vec +where + T: Unpin + Send + Sync + 'static, +{ + type Item = T; + type Buffer = VecBuffer; + + fn poll_produce<'a>( + self: Pin<&mut Self>, + _: &mut Context<'_>, + _: StoreContextMut<'a, D>, + mut dst: Destination<'a, Self::Item, Self::Buffer>, + _: bool, + ) -> Poll> { + dst.set_buffer(mem::take(self.get_mut()).into()); + Poll::Ready(Ok(StreamResult::Dropped)) + } +} + +impl StreamProducer for Box<[T]> +where + T: Unpin + Send + Sync + 'static, +{ + type Item = T; + type Buffer = VecBuffer; + + fn poll_produce<'a>( + self: Pin<&mut Self>, + _: &mut Context<'_>, + _: StoreContextMut<'a, D>, + mut dst: Destination<'a, Self::Item, Self::Buffer>, + _: bool, + ) -> Poll> { + dst.set_buffer(mem::take(self.get_mut()).into_vec().into()); + Poll::Ready(Ok(StreamResult::Dropped)) + } +} + +#[cfg(feature = "component-model-async-bytes")] +impl StreamProducer for bytes::Bytes { + type Item = u8; + type Buffer = Cursor; + + fn poll_produce<'a>( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + mut store: StoreContextMut<'a, D>, + mut dst: Destination<'a, Self::Item, Self::Buffer>, + _: bool, + ) -> Poll> { + let cap = dst.remaining(&mut store); + let Some(cap) = cap.and_then(core::num::NonZeroUsize::new) else { + // on 0-length or host reads, buffer the bytes + dst.set_buffer(Cursor::new(mem::take(self.get_mut()))); + return Poll::Ready(Ok(StreamResult::Dropped)); + }; + let cap = cap.into(); + // data does not fit in destination, fill it and buffer the rest + dst.set_buffer(Cursor::new(self.split_off(cap))); + let mut dst = dst.as_direct(store, cap); + dst.remaining().copy_from_slice(&self); + dst.mark_written(cap); + Poll::Ready(Ok(StreamResult::Dropped)) + } +} + +#[cfg(feature = "component-model-async-bytes")] +impl StreamProducer for bytes::BytesMut { + type Item = u8; + type Buffer = Cursor; + + fn poll_produce<'a>( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + mut store: StoreContextMut<'a, D>, + mut dst: Destination<'a, Self::Item, Self::Buffer>, + _: bool, + ) -> Poll> { + let cap = dst.remaining(&mut store); + let Some(cap) = cap.and_then(core::num::NonZeroUsize::new) else { + // on 0-length or host reads, buffer the bytes + dst.set_buffer(Cursor::new(mem::take(self.get_mut()))); + return Poll::Ready(Ok(StreamResult::Dropped)); + }; + let cap = cap.into(); + // data does not fit in destination, fill it and buffer the rest + dst.set_buffer(Cursor::new(self.split_off(cap))); + let mut dst = dst.as_direct(store, cap); + dst.remaining().copy_from_slice(&self); + dst.mark_written(cap); + Poll::Ready(Ok(StreamResult::Dropped)) + } +} + /// Represents the buffer for a host- or guest-initiated stream write. pub struct Source<'a, T> { instance: Instance, @@ -818,6 +948,28 @@ pub trait FutureProducer: Send + 'static { ) -> Poll>>; } +impl FutureProducer for Fut +where + E: Into, + Fut: Future> + ?Sized + Send + 'static, +{ + type Item = T; + + fn poll_produce<'a>( + self: Pin<&mut Self>, + cx: &mut Context<'_>, + _: StoreContextMut<'a, D>, + finish: bool, + ) -> Poll>> { + match self.poll(cx) { + Poll::Ready(Ok(v)) => Poll::Ready(Ok(Some(v))), + Poll::Ready(Err(err)) => Poll::Ready(Err(err.into())), + Poll::Pending if finish => Poll::Ready(Ok(None)), + Poll::Pending => Poll::Pending, + } + } +} + /// Represents a host-owned read end of a future. pub trait FutureConsumer: Send + 'static { /// The payload type of this future. @@ -885,7 +1037,7 @@ impl FutureReader { let producer = unsafe { self.map_unchecked_mut(|v| &mut v.0) }; Poll::Ready(Ok( - if let Some(value) = task::ready!(producer.poll_produce(cx, store, finish))? { + if let Some(value) = ready!(producer.poll_produce(cx, store, finish))? { destination.set_buffer(Some(value)); // Here we return `StreamResult::Completed` even though @@ -946,7 +1098,7 @@ impl FutureReader { // out of `self`. let consumer = unsafe { self.map_unchecked_mut(|v| &mut v.0) }; - task::ready!(consumer.poll_consume( + ready!(consumer.poll_consume( cx, store.as_context_mut(), source.reborrow(), @@ -3998,3 +4150,90 @@ pub(crate) struct ResourcePair { pub(crate) write: u32, pub(crate) read: u32, } + +#[cfg(test)] +mod tests { + use super::*; + use crate::{Engine, Store}; + use core::future::pending; + use core::pin::pin; + use std::sync::LazyLock; + + static ENGINE: LazyLock = LazyLock::new(Engine::default); + + fn poll_future_producer(rx: Pin<&mut T>, finish: bool) -> Poll>> + where + T: FutureProducer<()>, + { + rx.poll_produce( + &mut Context::from_waker(Waker::noop()), + Store::new(&ENGINE, ()).as_context_mut(), + finish, + ) + } + + #[test] + fn future_producer() { + let mut fut = pin!(async { anyhow::Ok(()) }); + assert!(matches!( + poll_future_producer(fut.as_mut(), false), + Poll::Ready(Ok(Some(()))), + )); + + let mut fut = pin!(async { anyhow::Ok(()) }); + assert!(matches!( + poll_future_producer(fut.as_mut(), true), + Poll::Ready(Ok(Some(()))), + )); + + let mut fut = pin!(pending::>()); + assert!(matches!( + poll_future_producer(fut.as_mut(), false), + Poll::Pending, + )); + assert!(matches!( + poll_future_producer(fut.as_mut(), true), + Poll::Ready(Ok(None)), + )); + + let (tx, rx) = oneshot::channel(); + let mut rx = pin!(rx); + assert!(matches!( + poll_future_producer(rx.as_mut(), false), + Poll::Pending, + )); + assert!(matches!( + poll_future_producer(rx.as_mut(), true), + Poll::Ready(Ok(None)), + )); + tx.send(()).unwrap(); + assert!(matches!( + poll_future_producer(rx.as_mut(), true), + Poll::Ready(Ok(Some(()))), + )); + + let (tx, rx) = oneshot::channel(); + let mut rx = pin!(rx); + tx.send(()).unwrap(); + assert!(matches!( + poll_future_producer(rx.as_mut(), false), + Poll::Ready(Ok(Some(()))), + )); + + let (tx, rx) = oneshot::channel::<()>(); + let mut rx = pin!(rx); + drop(tx); + assert!(matches!( + poll_future_producer(rx.as_mut(), false), + Poll::Ready(Err(..)), + )); + + let (tx, rx) = oneshot::channel::<()>(); + let mut rx = pin!(rx); + drop(tx); + assert!(matches!( + poll_future_producer(rx.as_mut(), true), + Poll::Ready(Err(..)), + )); + } +} From 8bdd2e2df1441dd89a33c092380d110bf1338827 Mon Sep 17 00:00:00 2001 From: Alex Crichton Date: Tue, 16 Sep 2025 14:25:30 -0500 Subject: [PATCH 2/3] Fix a missing outparam write in the p1 adapter (#11702) This fixes an issue found through #11701, notably the "release mode infinite loops" behavior that was seen. The cause was when `fd_readdir` returned after skipping all directory entries it forgot to write the final `*bufused = 0` out-param in the early-exit, unlike the end of the function which already sets this. --- .../src/bin/preview1_fd_readdir.rs | 29 +++++++++++++++++++ .../src/lib.rs | 5 +++- 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/crates/test-programs/src/bin/preview1_fd_readdir.rs b/crates/test-programs/src/bin/preview1_fd_readdir.rs index b5a04e3b3d9e..1fdb63cc088f 100644 --- a/crates/test-programs/src/bin/preview1_fd_readdir.rs +++ b/crates/test-programs/src/bin/preview1_fd_readdir.rs @@ -241,6 +241,34 @@ unsafe fn test_fd_readdir_unicode_boundary(dir_fd: wasip1::Fd) { wasip1::path_unlink_file(dir_fd, filename).expect("removing a file"); } +unsafe fn test_fd_readdir_past_end(dir_fd: wasip1::Fd) { + let file_fd = wasip1::path_open( + dir_fd, + 0, + "a", + wasip1::OFLAGS_CREAT, + wasip1::RIGHTS_FD_READ | wasip1::RIGHTS_FD_WRITE, + 0, + 0, + ) + .expect("failed to create file"); + wasip1::fd_close(file_fd).expect("closing a file"); + + let mut buf = vec![0; 128]; + let len = wasip1::fd_readdir(dir_fd, buf.as_mut_ptr(), buf.capacity(), 0).unwrap(); + + let next = ReadDir::from_slice(&buf[..len]) + .last() + .unwrap() + .dirent + .d_next; + + let len = wasip1::fd_readdir(dir_fd, buf.as_mut_ptr(), buf.capacity(), next + 1).unwrap(); + assert_eq!(len, 0); + + wasip1::path_unlink_file(dir_fd, "a").expect("removing a file"); +} + fn main() { let mut args = env::args(); let prog = args.next().unwrap(); @@ -264,4 +292,5 @@ fn main() { unsafe { test_fd_readdir(dir_fd) } unsafe { test_fd_readdir_lots(dir_fd) } unsafe { test_fd_readdir_unicode_boundary(dir_fd) } + unsafe { test_fd_readdir_past_end(dir_fd) } } diff --git a/crates/wasi-preview1-component-adapter/src/lib.rs b/crates/wasi-preview1-component-adapter/src/lib.rs index 79bd515d7cb7..d744c0d6599f 100644 --- a/crates/wasi-preview1-component-adapter/src/lib.rs +++ b/crates/wasi-preview1-component-adapter/src/lib.rs @@ -1406,7 +1406,10 @@ pub unsafe extern "C" fn fd_readdir( match iter.next() { Some(Ok(_)) => {} Some(Err(e)) => return Err(e), - None => return Ok(()), + None => { + *bufused = 0; + return Ok(()); + } } } } From 928e7e000a5807a3bf8840cb4a14e06778b87ca2 Mon Sep 17 00:00:00 2001 From: Roman Volosatovs Date: Wed, 17 Sep 2025 17:17:13 +0200 Subject: [PATCH 3/3] feat(p3-http): implement `consume-body` changes (#11653) * feat(p3-http): implement `consume-body` changes Signed-off-by: Roman Volosatovs * Update WITs to the latest snapshot * Use the 2025-09-16 tag instead of 2025-08-15. * Pulls in `wasi:http` updates "officially". * Pulls in minor `wasi:cli` updates, and that's implemented here as well. --------- Signed-off-by: Roman Volosatovs Co-authored-by: Alex Crichton --- ci/vendor-wit.sh | 22 ++-- .../src/bin/cli_p3_hello_stdout.rs | 11 +- .../src/bin/cli_p3_much_stdout.rs | 15 ++- crates/test-programs/src/bin/p3_cli.rs | 35 ++++-- crates/test-programs/src/bin/p3_http_echo.rs | 6 +- .../src/bin/p3_http_middleware.rs | 9 +- .../src/bin/p3_http_middleware_with_chain.rs | 4 +- crates/test-programs/src/p3/http.rs | 5 +- crates/test-programs/src/p3/mod.rs | 29 +++-- crates/wasi-http/src/p3/bindings.rs | 4 +- crates/wasi-http/src/p3/body.rs | 95 +++++++--------- crates/wasi-http/src/p3/host/handler.rs | 3 +- crates/wasi-http/src/p3/host/types.rs | 47 ++++---- crates/wasi-http/src/p3/response.rs | 3 +- .../wasi-http/src/p3/wit/deps/cli/command.wit | 8 +- .../src/p3/wit/deps/cli/environment.wit | 8 +- crates/wasi-http/src/p3/wit/deps/cli/exit.wit | 4 +- .../wasi-http/src/p3/wit/deps/cli/imports.wit | 40 +++---- crates/wasi-http/src/p3/wit/deps/cli/run.wit | 6 +- .../wasi-http/src/p3/wit/deps/cli/stdio.wit | 66 +++++++++-- .../src/p3/wit/deps/cli/terminal.wit | 26 ++--- .../p3/wit/deps/clocks/monotonic-clock.wit | 27 +++-- .../src/p3/wit/deps/clocks/timezone.wit | 2 +- .../src/p3/wit/deps/clocks/types.wit | 8 ++ .../src/p3/wit/deps/clocks/wall-clock.wit | 10 +- .../src/p3/wit/deps/clocks/world.wit | 8 +- .../src/p3/wit/deps/filesystem/preopens.wit | 8 +- .../src/p3/wit/deps/filesystem/types.wit | 80 ++++++------- .../src/p3/wit/deps/filesystem/world.wit | 8 +- .../wasi-http/src/p3/wit/deps/http/proxy.wit | 12 +- .../wasi-http/src/p3/wit/deps/http/types.wit | 38 ++----- .../src/p3/wit/deps/random/insecure-seed.wit | 6 +- .../src/p3/wit/deps/random/insecure.wit | 8 +- .../src/p3/wit/deps/random/random.wit | 8 +- .../src/p3/wit/deps/random/world.wit | 10 +- .../p3/wit/deps/sockets/ip-name-lookup.wit | 8 +- .../src/p3/wit/deps/sockets/types.wit | 106 +++++++++--------- .../src/p3/wit/deps/sockets/world.wit | 8 +- crates/wasi-http/src/p3/wit/world.wit | 2 +- crates/wasi-http/tests/all/p3/mod.rs | 2 +- crates/wasi/src/p3/bindings.rs | 2 +- crates/wasi/src/p3/cli/host.rs | 75 ++++++++++--- crates/wasi/src/p3/wit/deps/cli/command.wit | 8 +- .../wasi/src/p3/wit/deps/cli/environment.wit | 8 +- crates/wasi/src/p3/wit/deps/cli/exit.wit | 4 +- crates/wasi/src/p3/wit/deps/cli/imports.wit | 40 +++---- crates/wasi/src/p3/wit/deps/cli/run.wit | 6 +- crates/wasi/src/p3/wit/deps/cli/stdio.wit | 66 +++++++++-- crates/wasi/src/p3/wit/deps/cli/terminal.wit | 26 ++--- .../p3/wit/deps/clocks/monotonic-clock.wit | 27 +++-- .../wasi/src/p3/wit/deps/clocks/timezone.wit | 2 +- crates/wasi/src/p3/wit/deps/clocks/types.wit | 8 ++ .../src/p3/wit/deps/clocks/wall-clock.wit | 10 +- crates/wasi/src/p3/wit/deps/clocks/world.wit | 8 +- .../src/p3/wit/deps/filesystem/preopens.wit | 8 +- .../wasi/src/p3/wit/deps/filesystem/types.wit | 80 ++++++------- .../wasi/src/p3/wit/deps/filesystem/world.wit | 8 +- .../src/p3/wit/deps/random/insecure-seed.wit | 6 +- .../wasi/src/p3/wit/deps/random/insecure.wit | 8 +- crates/wasi/src/p3/wit/deps/random/random.wit | 8 +- crates/wasi/src/p3/wit/deps/random/world.wit | 10 +- .../p3/wit/deps/sockets/ip-name-lookup.wit | 8 +- crates/wasi/src/p3/wit/deps/sockets/types.wit | 106 +++++++++--------- crates/wasi/src/p3/wit/deps/sockets/world.wit | 8 +- 64 files changed, 749 insertions(+), 596 deletions(-) create mode 100644 crates/wasi-http/src/p3/wit/deps/clocks/types.wit create mode 100644 crates/wasi/src/p3/wit/deps/clocks/types.wit diff --git a/ci/vendor-wit.sh b/ci/vendor-wit.sh index cd41fd2a64d0..d6d7e3ce04ad 100755 --- a/ci/vendor-wit.sh +++ b/ci/vendor-wit.sh @@ -70,20 +70,20 @@ make_vendor "wasi-config" "config@f4d699b" make_vendor "wasi-keyvalue" "keyvalue@219ea36" make_vendor "wasi/src/p3" " - cli@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - clocks@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - filesystem@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - random@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - sockets@v0.3.0-rc-2025-08-15@wit-0.3.0-draft + cli@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + clocks@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + filesystem@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + random@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + sockets@v0.3.0-rc-2025-09-16@wit-0.3.0-draft " make_vendor "wasi-http/src/p3" " - cli@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - clocks@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - filesystem@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - http@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - random@v0.3.0-rc-2025-08-15@wit-0.3.0-draft - sockets@v0.3.0-rc-2025-08-15@wit-0.3.0-draft + cli@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + clocks@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + filesystem@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + http@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + random@v0.3.0-rc-2025-09-16@wit-0.3.0-draft + sockets@v0.3.0-rc-2025-09-16@wit-0.3.0-draft " rm -rf $cache_dir diff --git a/crates/test-programs/src/bin/cli_p3_hello_stdout.rs b/crates/test-programs/src/bin/cli_p3_hello_stdout.rs index 9db7c7efbbba..d8e79cfbc3ed 100644 --- a/crates/test-programs/src/bin/cli_p3_hello_stdout.rs +++ b/crates/test-programs/src/bin/cli_p3_hello_stdout.rs @@ -7,8 +7,15 @@ export!(Component); impl exports::wasi::cli::run::Guest for Component { async fn run() -> Result<(), ()> { let (mut tx, rx) = wit_stream::new(); - wasi::cli::stdout::set_stdout(rx); - tx.write(b"hello, world\n".to_vec()).await; + futures::join!( + async { + wasi::cli::stdout::write_via_stream(rx).await.unwrap(); + }, + async { + tx.write(b"hello, world\n".to_vec()).await; + drop(tx); + }, + ); Ok(()) } } diff --git a/crates/test-programs/src/bin/cli_p3_much_stdout.rs b/crates/test-programs/src/bin/cli_p3_much_stdout.rs index d13a98cf09d9..c490cc3afa04 100644 --- a/crates/test-programs/src/bin/cli_p3_much_stdout.rs +++ b/crates/test-programs/src/bin/cli_p3_much_stdout.rs @@ -12,11 +12,16 @@ impl test_programs::p3::exports::wasi::cli::run::Guest for Component { let bytes = string_to_write.as_bytes(); let (mut tx, rx) = wit_stream::new(); - wasi::cli::stdout::set_stdout(rx); - for _ in 0..times_to_write { - let result = tx.write_all(bytes.to_vec()).await; - assert!(result.is_empty()); - } + futures::join!( + async { wasi::cli::stdout::write_via_stream(rx).await.unwrap() }, + async { + for _ in 0..times_to_write { + let result = tx.write_all(bytes.to_vec()).await; + assert!(result.is_empty()); + } + drop(tx); + } + ); Ok(()) } } diff --git a/crates/test-programs/src/bin/p3_cli.rs b/crates/test-programs/src/bin/p3_cli.rs index 2373d512182d..276a7146ef0e 100644 --- a/crates/test-programs/src/bin/p3_cli.rs +++ b/crates/test-programs/src/bin/p3_cli.rs @@ -18,20 +18,37 @@ impl test_programs::p3::exports::wasi::cli::run::Guest for Component { assert!(terminal_stdout::get_terminal_stdout().is_none()); assert!(terminal_stderr::get_terminal_stderr().is_none()); - let mut stdin = stdin::get_stdin(); + let (mut stdin, result) = stdin::read_via_stream(); assert!(stdin.next().await.is_none()); let (mut stdout_tx, stdout_rx) = wit_stream::new(); - stdout::set_stdout(stdout_rx); - let (res, buf) = stdout_tx.write(b"hello stdout\n".into()).await; - assert_eq!(res, StreamResult::Complete(13)); - assert_eq!(buf.into_vec(), []); + futures::join!( + async { + stdout::write_via_stream(stdout_rx).await.unwrap(); + }, + async { + let (res, buf) = stdout_tx.write(b"hello stdout\n".into()).await; + assert_eq!(res, StreamResult::Complete(13)); + assert_eq!(buf.into_vec(), []); + drop(stdout_tx); + } + ); let (mut stderr_tx, stderr_rx) = wit_stream::new(); - stderr::set_stderr(stderr_rx); - let (res, buf) = stderr_tx.write(b"hello stderr\n".into()).await; - assert_eq!(res, StreamResult::Complete(13)); - assert_eq!(buf.into_vec(), []); + futures::join!( + async { + stderr::write_via_stream(stderr_rx).await.unwrap(); + }, + async { + let (res, buf) = stderr_tx.write(b"hello stderr\n".into()).await; + assert_eq!(res, StreamResult::Complete(13)); + assert_eq!(buf.into_vec(), []); + drop(stderr_tx); + } + ); + + drop(stdin); + result.await.unwrap(); Ok(()) } diff --git a/crates/test-programs/src/bin/p3_http_echo.rs b/crates/test-programs/src/bin/p3_http_echo.rs index 77ec189f5d78..a502d546fe40 100644 --- a/crates/test-programs/src/bin/p3_http_echo.rs +++ b/crates/test-programs/src/bin/p3_http_echo.rs @@ -15,9 +15,8 @@ impl Handler for Component { /// Return a response which echoes the request headers, body, and trailers. async fn handle(request: Request) -> Result { let headers = request.get_headers(); - let (body, trailers) = request.consume_body().unwrap(); - - // let (headers, body) = Request::into_parts(request); + let (_, result_rx) = wit_future::new(|| Ok(())); + let (body, trailers) = Request::consume_body(request, result_rx); let (response, _result) = if false { // This is the easy and efficient way to do it... @@ -47,7 +46,6 @@ impl Handler for Component { drop(pipe_tx); trailers_tx.write(trailers.await).await.unwrap(); - drop(request); }); Response::new(headers, Some(pipe_rx), trailers_rx) diff --git a/crates/test-programs/src/bin/p3_http_middleware.rs b/crates/test-programs/src/bin/p3_http_middleware.rs index 2384e05ef9da..566b8bc4ba6f 100644 --- a/crates/test-programs/src/bin/p3_http_middleware.rs +++ b/crates/test-programs/src/bin/p3_http_middleware.rs @@ -48,7 +48,8 @@ impl Handler for Component { } _ => true, }); - let (mut body, trailers) = request.consume_body().unwrap(); + let (_, result_rx) = wit_future::new(|| Ok(())); + let (mut body, trailers) = Request::consume_body(request, result_rx); let (body, trailers) = if content_deflated { // Next, spawn a task to pipe and decode the original request body and trailers into a new request @@ -77,8 +78,6 @@ impl Handler for Component { } trailers_tx.write(trailers.await).await.unwrap(); - - drop(request); }); (pipe_rx, trailers_rx) @@ -110,7 +109,8 @@ impl Handler for Component { headers.push(("content-encoding".into(), b"deflate".into())); } - let (mut body, trailers) = response.consume_body().unwrap(); + let (_, result_rx) = wit_future::new(|| Ok(())); + let (mut body, trailers) = Response::consume_body(response, result_rx); let (body, trailers) = if accept_deflated { headers.retain(|(name, _value)| name != "content-length"); @@ -141,7 +141,6 @@ impl Handler for Component { } trailers_tx.write(trailers.await).await.unwrap(); - drop(response); }); (pipe_rx, trailers_rx) diff --git a/crates/test-programs/src/bin/p3_http_middleware_with_chain.rs b/crates/test-programs/src/bin/p3_http_middleware_with_chain.rs index 7b4acf3b8990..aed9f70b5407 100644 --- a/crates/test-programs/src/bin/p3_http_middleware_with_chain.rs +++ b/crates/test-programs/src/bin/p3_http_middleware_with_chain.rs @@ -6,13 +6,13 @@ mod bindings { package local:local; world middleware-with-chain { - include wasi:http/proxy@0.3.0-rc-2025-08-15; + include wasi:http/proxy@0.3.0-rc-2025-09-16; import chain-http; } interface chain-http { - use wasi:http/types@0.3.0-rc-2025-08-15.{request, response, error-code}; + use wasi:http/types@0.3.0-rc-2025-09-16.{request, response, error-code}; handle: async func(request: request) -> result; } diff --git a/crates/test-programs/src/p3/http.rs b/crates/test-programs/src/p3/http.rs index c65fa9a99877..c51f1221f286 100644 --- a/crates/test-programs/src/p3/http.rs +++ b/crates/test-programs/src/p3/http.rs @@ -94,9 +94,8 @@ pub async fn request( let response = handler::handle(request).await?; let status = response.get_status_code(); let headers = response.get_headers().copy_all(); - let (body_rx, trailers_rx) = response - .consume_body() - .expect("failed to get response body"); + let (_, result_rx) = wit_future::new(|| Ok(())); + let (body_rx, trailers_rx) = types::Response::consume_body(response, result_rx); let ((), rx) = join!( async { if let Some(buf) = body { diff --git a/crates/test-programs/src/p3/mod.rs b/crates/test-programs/src/p3/mod.rs index adeb45d22019..87ddddb2d7cf 100644 --- a/crates/test-programs/src/p3/mod.rs +++ b/crates/test-programs/src/p3/mod.rs @@ -6,19 +6,16 @@ wit_bindgen::generate!({ package wasmtime:test; world testp3 { - include wasi:cli/imports@0.3.0-rc-2025-08-15; - include wasi:http/imports@0.3.0-rc-2025-08-15; + include wasi:cli/imports@0.3.0-rc-2025-09-16; + include wasi:http/imports@0.3.0-rc-2025-09-16; - export wasi:cli/run@0.3.0-rc-2025-08-15; + export wasi:cli/run@0.3.0-rc-2025-09-16; } ", path: "../wasi-http/src/p3/wit", world: "wasmtime:test/testp3", default_bindings_module: "test_programs::p3", pub_export_macro: true, - async: [ - "wasi:cli/run@0.3.0-rc-2025-08-15#run", - ], generate_all, }); @@ -28,7 +25,7 @@ pub mod proxy { package wasmtime:test; world proxyp3 { - include wasi:http/proxy@0.3.0-rc-2025-08-15; + include wasi:http/proxy@0.3.0-rc-2025-09-16; } ", path: "../wasi-http/src/p3/wit", @@ -36,14 +33,16 @@ pub mod proxy { default_bindings_module: "test_programs::p3::proxy", pub_export_macro: true, with: { - "wasi:http/handler@0.3.0-rc-2025-08-15": generate, - "wasi:http/types@0.3.0-rc-2025-08-15": crate::p3::wasi::http::types, - "wasi:random/random@0.3.0-rc-2025-08-15": crate::p3::wasi::random::random, - "wasi:cli/stdout@0.3.0-rc-2025-08-15": crate::p3::wasi::cli::stdout, - "wasi:cli/stderr@0.3.0-rc-2025-08-15": crate::p3::wasi::cli::stderr, - "wasi:cli/stdin@0.3.0-rc-2025-08-15": crate::p3::wasi::cli::stdin, - "wasi:clocks/monotonic-clock@0.3.0-rc-2025-08-15": crate::p3::wasi::clocks::monotonic_clock, - "wasi:clocks/wall-clock@0.3.0-rc-2025-08-15": crate::p3::wasi::clocks::wall_clock, + "wasi:http/handler@0.3.0-rc-2025-09-16": generate, + "wasi:http/types@0.3.0-rc-2025-09-16": crate::p3::wasi::http::types, + "wasi:random/random@0.3.0-rc-2025-09-16": crate::p3::wasi::random::random, + "wasi:cli/stdout@0.3.0-rc-2025-09-16": crate::p3::wasi::cli::stdout, + "wasi:cli/stderr@0.3.0-rc-2025-09-16": crate::p3::wasi::cli::stderr, + "wasi:cli/stdin@0.3.0-rc-2025-09-16": crate::p3::wasi::cli::stdin, + "wasi:cli/types@0.3.0-rc-2025-09-16": crate::p3::wasi::cli::types, + "wasi:clocks/monotonic-clock@0.3.0-rc-2025-09-16": crate::p3::wasi::clocks::monotonic_clock, + "wasi:clocks/wall-clock@0.3.0-rc-2025-09-16": crate::p3::wasi::clocks::wall_clock, + "wasi:clocks/types@0.3.0-rc-2025-09-16": crate::p3::wasi::clocks::types, }, }); } diff --git a/crates/wasi-http/src/p3/bindings.rs b/crates/wasi-http/src/p3/bindings.rs index f467df417b97..ba9c497a8ee7 100644 --- a/crates/wasi-http/src/p3/bindings.rs +++ b/crates/wasi-http/src/p3/bindings.rs @@ -9,9 +9,9 @@ mod generated { "wasi:http/handler/[async]handle": async | store | trappable | tracing, "wasi:http/types/[drop]request": store | trappable | tracing, "wasi:http/types/[drop]response": store | trappable | tracing, - "wasi:http/types/[method]request.consume-body": async | store | trappable | tracing, - "wasi:http/types/[method]response.consume-body": async | store | trappable | tracing, + "wasi:http/types/[static]request.consume-body": async | store | trappable | tracing, "wasi:http/types/[static]request.new": async | store | trappable | tracing, + "wasi:http/types/[static]response.consume-body": async | store | trappable | tracing, "wasi:http/types/[static]response.new": async | store | trappable | tracing, default: trappable | tracing, }, diff --git a/crates/wasi-http/src/p3/body.rs b/crates/wasi-http/src/p3/body.rs index d9b40e421208..ff4e60b6514c 100644 --- a/crates/wasi-http/src/p3/body.rs +++ b/crates/wasi-http/src/p3/body.rs @@ -37,8 +37,33 @@ pub(crate) enum Body { /// Channel, on which transmission result will be written result_tx: oneshot::Sender> + Send>>, }, - /// Body is consumed. - Consumed, +} + +/// [FutureConsumer] implementation for future passed to `consume-body`. +struct BodyResultConsumer( + Option> + Send>>>, +); + +impl FutureConsumer for BodyResultConsumer +where + D: 'static, +{ + type Item = Result<(), ErrorCode>; + + fn poll_consume( + mut self: Pin<&mut Self>, + _: &mut Context<'_>, + store: StoreContextMut, + mut src: Source<'_, Self::Item>, + _: bool, + ) -> Poll> { + let mut res = None; + src.read(store, &mut res).context("failed to read result")?; + let res = res.context("result value missing")?; + let tx = self.0.take().context("polled after returning `Ready`")?; + _ = tx.send(Box::new(async { res })); + Poll::Ready(Ok(())) + } } impl Body { @@ -46,46 +71,38 @@ impl Body { pub(crate) fn consume( self, mut store: Access<'_, T, WasiHttp>, + fut: FutureReader>, getter: fn(&mut T) -> WasiHttpCtxView<'_>, - ) -> Result< - ( - StreamReader, - FutureReader>, ErrorCode>>, - ), - (), - > { + ) -> ( + StreamReader, + FutureReader>, ErrorCode>>, + ) { match self { Body::Guest { contents_rx: Some(contents_rx), trailers_rx, result_tx, } => { - // TODO: Use a result specified by the caller - // https://github.com/WebAssembly/wasi-http/issues/176 - _ = result_tx.send(Box::new(async { Ok(()) })); - Ok((contents_rx, trailers_rx)) + fut.pipe(&mut store, BodyResultConsumer(Some(result_tx))); + (contents_rx, trailers_rx) } Body::Guest { contents_rx: None, trailers_rx, result_tx, } => { + fut.pipe(&mut store, BodyResultConsumer(Some(result_tx))); let instance = store.instance(); - // TODO: Use a result specified by the caller - // https://github.com/WebAssembly/wasi-http/issues/176 - _ = result_tx.send(Box::new(async { Ok(()) })); - Ok(( + ( StreamReader::new(instance, &mut store, iter::empty()), trailers_rx, - )) + ) } Body::Host { body, result_tx } => { + fut.pipe(&mut store, BodyResultConsumer(Some(result_tx))); let instance = store.instance(); - // TODO: Use a result specified by the caller - // https://github.com/WebAssembly/wasi-http/issues/176 - _ = result_tx.send(Box::new(async { Ok(()) })); let (trailers_tx, trailers_rx) = oneshot::channel(); - Ok(( + ( StreamReader::new( instance, &mut store, @@ -96,9 +113,8 @@ impl Body { }, ), FutureReader::new(instance, &mut store, trailers_rx), - )) + ) } - Body::Consumed => Err(()), } } @@ -390,31 +406,6 @@ impl http_body::Body for GuestBody { } } -/// [http_body::Body] that has been consumed. -pub(crate) struct ConsumedBody; - -impl http_body::Body for ConsumedBody { - type Data = Bytes; - type Error = ErrorCode; - - fn poll_frame( - self: Pin<&mut Self>, - _cx: &mut Context<'_>, - ) -> Poll, Self::Error>>> { - Poll::Ready(Some(Err(ErrorCode::InternalError(Some( - "body consumed".into(), - ))))) - } - - fn is_end_stream(&self) -> bool { - true - } - - fn size_hint(&self) -> http_body::SizeHint { - http_body::SizeHint::with_exact(0) - } -} - /// [FutureConsumer] implementation for trailers originating in the guest. struct GuestTrailerConsumer { tx: Option>, ErrorCode>>>, @@ -434,10 +425,10 @@ where mut src: Source<'_, Self::Item>, _: bool, ) -> Poll> { - let mut result = None; - src.read(store.as_context_mut(), &mut result) + let mut res = None; + src.read(&mut store, &mut res) .context("failed to read result")?; - let res = match result.context("result value missing")? { + let res = match res.context("result value missing")? { Ok(Some(trailers)) => { let WasiHttpCtxView { table, .. } = (self.getter)(store.data_mut()); let trailers = table diff --git a/crates/wasi-http/src/p3/host/handler.rs b/crates/wasi-http/src/p3/host/handler.rs index 60affada2100..4f3da56deb1c 100644 --- a/crates/wasi-http/src/p3/host/handler.rs +++ b/crates/wasi-http/src/p3/host/handler.rs @@ -1,7 +1,7 @@ use crate::get_content_length; use crate::p3::bindings::http::handler::{Host, HostWithStore}; use crate::p3::bindings::http::types::{ErrorCode, Request, Response}; -use crate::p3::body::{Body, ConsumedBody, GuestBody}; +use crate::p3::body::{Body, GuestBody}; use crate::p3::{HttpError, HttpResult, WasiHttp, WasiHttpCtxView}; use anyhow::Context as _; use bytes::Bytes; @@ -275,7 +275,6 @@ impl HostWithStore for WasiHttp { body.with_state(io_task_rx).boxed() } } - Body::Consumed => ConsumedBody.boxed(), }; let WasiHttpCtxView { ctx, .. } = store.get(); diff --git a/crates/wasi-http/src/p3/host/types.rs b/crates/wasi-http/src/p3/host/types.rs index 840702546b82..8f78117dcbc1 100644 --- a/crates/wasi-http/src/p3/host/types.rs +++ b/crates/wasi-http/src/p3/host/types.rs @@ -7,7 +7,6 @@ use crate::p3::bindings::http::types::{ use crate::p3::body::Body; use crate::p3::{HeaderResult, HttpError, RequestOptionsResult, WasiHttp, WasiHttpCtxView}; use anyhow::Context as _; -use core::mem; use core::pin::Pin; use core::task::{Context, Poll, ready}; use http::header::CONTENT_LENGTH; @@ -356,20 +355,19 @@ impl HostRequestWithStore for WasiHttp { async fn consume_body( store: &Accessor, req: Resource, - ) -> wasmtime::Result< - Result< - ( - StreamReader, - FutureReader>, ErrorCode>>, - ), - (), - >, - > { + fut: FutureReader>, + ) -> wasmtime::Result<( + StreamReader, + FutureReader>, ErrorCode>>, + )> { let getter = store.getter(); store.with(|mut store| { - let Request { body, .. } = get_request_mut(store.get().table, &req)?; - let body = mem::replace(body, Body::Consumed); - Ok(body.consume(store, getter)) + let Request { body, .. } = store + .get() + .table + .delete(req) + .context("failed to delete request from table")?; + Ok(body.consume(store, fut, getter)) }) } @@ -633,20 +631,19 @@ impl HostResponseWithStore for WasiHttp { async fn consume_body( store: &Accessor, res: Resource, - ) -> wasmtime::Result< - Result< - ( - StreamReader, - FutureReader>, ErrorCode>>, - ), - (), - >, - > { + fut: FutureReader>, + ) -> wasmtime::Result<( + StreamReader, + FutureReader>, ErrorCode>>, + )> { let getter = store.getter(); store.with(|mut store| { - let Response { body, .. } = get_response_mut(store.get().table, &res)?; - let body = mem::replace(body, Body::Consumed); - Ok(body.consume(store, getter)) + let Response { body, .. } = store + .get() + .table + .delete(res) + .context("failed to delete response from table")?; + Ok(body.consume(store, fut, getter)) }) } diff --git a/crates/wasi-http/src/p3/response.rs b/crates/wasi-http/src/p3/response.rs index c7248c984e2c..4f3ec7953d25 100644 --- a/crates/wasi-http/src/p3/response.rs +++ b/crates/wasi-http/src/p3/response.rs @@ -1,6 +1,6 @@ use crate::get_content_length; use crate::p3::bindings::http::types::ErrorCode; -use crate::p3::body::{Body, ConsumedBody, GuestBody}; +use crate::p3::body::{Body, GuestBody}; use crate::p3::{WasiHttpCtxView, WasiHttpView}; use anyhow::Context as _; use bytes::Bytes; @@ -86,7 +86,6 @@ impl Response { _ = result_tx.send(Box::new(fut)); body } - Body::Consumed => ConsumedBody.boxed(), }; Ok(http::Response::from_parts(res, body)) } diff --git a/crates/wasi-http/src/p3/wit/deps/cli/command.wit b/crates/wasi-http/src/p3/wit/deps/cli/command.wit index 13e92bdec21d..f2f613e5524a 100644 --- a/crates/wasi-http/src/p3/wit/deps/cli/command.wit +++ b/crates/wasi-http/src/p3/wit/deps/cli/command.wit @@ -1,10 +1,10 @@ -package wasi:cli@0.3.0-rc-2025-08-15; +package wasi:cli@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world command { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) include imports; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) export run; } diff --git a/crates/wasi-http/src/p3/wit/deps/cli/environment.wit b/crates/wasi-http/src/p3/wit/deps/cli/environment.wit index 769858af1af1..3763f2f6ced5 100644 --- a/crates/wasi-http/src/p3/wit/deps/cli/environment.wit +++ b/crates/wasi-http/src/p3/wit/deps/cli/environment.wit @@ -1,4 +1,4 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface environment { /// Get the POSIX-style environment variables. /// @@ -8,15 +8,15 @@ interface environment { /// Morally, these are a value import, but until value imports are available /// in the component model, this import function should return the same /// values each time it is called. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-environment: func() -> list>; /// Get the POSIX-style arguments to the program. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-arguments: func() -> list; /// Return a path that programs should use as their initial current working /// directory, interpreting `.` as shorthand for this. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-initial-cwd: func() -> option; } diff --git a/crates/wasi-http/src/p3/wit/deps/cli/exit.wit b/crates/wasi-http/src/p3/wit/deps/cli/exit.wit index dcf791a2019c..1efba7d68320 100644 --- a/crates/wasi-http/src/p3/wit/deps/cli/exit.wit +++ b/crates/wasi-http/src/p3/wit/deps/cli/exit.wit @@ -1,7 +1,7 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface exit { /// Exit the current instance and any linked instances. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) exit: func(status: result); /// Exit the current instance and any linked instances, reporting the diff --git a/crates/wasi-http/src/p3/wit/deps/cli/imports.wit b/crates/wasi-http/src/p3/wit/deps/cli/imports.wit index 2d513ead9eaf..660a2dd95a99 100644 --- a/crates/wasi-http/src/p3/wit/deps/cli/imports.wit +++ b/crates/wasi-http/src/p3/wit/deps/cli/imports.wit @@ -1,34 +1,34 @@ -package wasi:cli@0.3.0-rc-2025-08-15; +package wasi:cli@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) - include wasi:clocks/imports@0.3.0-rc-2025-08-15; - @since(version = 0.3.0-rc-2025-08-15) - include wasi:filesystem/imports@0.3.0-rc-2025-08-15; - @since(version = 0.3.0-rc-2025-08-15) - include wasi:sockets/imports@0.3.0-rc-2025-08-15; - @since(version = 0.3.0-rc-2025-08-15) - include wasi:random/imports@0.3.0-rc-2025-08-15; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:clocks/imports@0.3.0-rc-2025-09-16; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:filesystem/imports@0.3.0-rc-2025-09-16; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:sockets/imports@0.3.0-rc-2025-09-16; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:random/imports@0.3.0-rc-2025-09-16; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import environment; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import exit; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import stdin; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import stdout; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import stderr; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-input; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-output; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-stdin; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-stdout; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-stderr; } diff --git a/crates/wasi-http/src/p3/wit/deps/cli/run.wit b/crates/wasi-http/src/p3/wit/deps/cli/run.wit index 6149db760ee3..631441a3f294 100644 --- a/crates/wasi-http/src/p3/wit/deps/cli/run.wit +++ b/crates/wasi-http/src/p3/wit/deps/cli/run.wit @@ -1,6 +1,6 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface run { /// Run the program. - @since(version = 0.3.0-rc-2025-08-15) - run: func() -> result; + @since(version = 0.3.0-rc-2025-09-16) + run: async func() -> result; } diff --git a/crates/wasi-http/src/p3/wit/deps/cli/stdio.wit b/crates/wasi-http/src/p3/wit/deps/cli/stdio.wit index 6c99a56f1648..51e5ae4b4094 100644 --- a/crates/wasi-http/src/p3/wit/deps/cli/stdio.wit +++ b/crates/wasi-http/src/p3/wit/deps/cli/stdio.wit @@ -1,17 +1,65 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) +interface types { + @since(version = 0.3.0-rc-2025-09-16) + enum error-code { + /// Input/output error + io, + /// Invalid or incomplete multibyte or wide character + illegal-byte-sequence, + /// Broken pipe + pipe, + } +} + +@since(version = 0.3.0-rc-2025-09-16) interface stdin { - @since(version = 0.3.0-rc-2025-08-15) - get-stdin: func() -> stream; + use types.{error-code}; + + /// Return a stream for reading from stdin. + /// + /// This function returns a stream which provides data read from stdin, + /// and a future to signal read results. + /// + /// If the stream's readable end is dropped the future will resolve to success. + /// + /// If the stream's writable end is dropped the future will either resolve to + /// success if stdin was closed by the writer or to an error-code if reading + /// failed for some other reason. + /// + /// Multiple streams may be active at the same time. The behavior of concurrent + /// reads is implementation-specific. + @since(version = 0.3.0-rc-2025-09-16) + read-via-stream: func() -> tuple, future>>; } -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface stdout { - @since(version = 0.3.0-rc-2025-08-15) - set-stdout: func(data: stream); + use types.{error-code}; + + /// Write the given stream to stdout. + /// + /// If the stream's writable end is dropped this function will either return + /// success once the entire contents of the stream have been written or an + /// error-code representing a failure. + /// + /// Otherwise if there is an error the readable end of the stream will be + /// dropped and this function will return an error-code. + @since(version = 0.3.0-rc-2025-09-16) + write-via-stream: async func(data: stream) -> result<_, error-code>; } -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface stderr { - @since(version = 0.3.0-rc-2025-08-15) - set-stderr: func(data: stream); + use types.{error-code}; + + /// Write the given stream to stderr. + /// + /// If the stream's writable end is dropped this function will either return + /// success once the entire contents of the stream have been written or an + /// error-code representing a failure. + /// + /// Otherwise if there is an error the readable end of the stream will be + /// dropped and this function will return an error-code. + @since(version = 0.3.0-rc-2025-09-16) + write-via-stream: async func(data: stream) -> result<_, error-code>; } diff --git a/crates/wasi-http/src/p3/wit/deps/cli/terminal.wit b/crates/wasi-http/src/p3/wit/deps/cli/terminal.wit index 3b2b72bbaa03..74c17694a480 100644 --- a/crates/wasi-http/src/p3/wit/deps/cli/terminal.wit +++ b/crates/wasi-http/src/p3/wit/deps/cli/terminal.wit @@ -3,10 +3,10 @@ /// In the future, this may include functions for disabling echoing, /// disabling input buffering so that keyboard events are sent through /// immediately, querying supported features, and so on. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-input { /// The input side of a terminal. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource terminal-input; } @@ -15,48 +15,48 @@ interface terminal-input { /// In the future, this may include functions for querying the terminal /// size, being notified of terminal size changes, querying supported /// features, and so on. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-output { /// The output side of a terminal. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource terminal-output; } /// An interface providing an optional `terminal-input` for stdin as a /// link-time authority. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-stdin { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use terminal-input.{terminal-input}; /// If stdin is connected to a terminal, return a `terminal-input` handle /// allowing further interaction with it. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-terminal-stdin: func() -> option; } /// An interface providing an optional `terminal-output` for stdout as a /// link-time authority. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-stdout { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use terminal-output.{terminal-output}; /// If stdout is connected to a terminal, return a `terminal-output` handle /// allowing further interaction with it. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-terminal-stdout: func() -> option; } /// An interface providing an optional `terminal-output` for stderr as a /// link-time authority. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-stderr { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use terminal-output.{terminal-output}; /// If stderr is connected to a terminal, return a `terminal-output` handle /// allowing further interaction with it. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-terminal-stderr: func() -> option; } diff --git a/crates/wasi-http/src/p3/wit/deps/clocks/monotonic-clock.wit b/crates/wasi-http/src/p3/wit/deps/clocks/monotonic-clock.wit index 0c58241fffeb..a91d495c6c15 100644 --- a/crates/wasi-http/src/p3/wit/deps/clocks/monotonic-clock.wit +++ b/crates/wasi-http/src/p3/wit/deps/clocks/monotonic-clock.wit @@ -1,4 +1,4 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; /// WASI Monotonic Clock is a clock API intended to let users measure elapsed /// time. /// @@ -7,38 +7,41 @@ package wasi:clocks@0.3.0-rc-2025-08-15; /// /// A monotonic clock is a clock which has an unspecified initial value, and /// successive reads of the clock will produce non-decreasing values. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface monotonic-clock { + use types.{duration}; + /// An instant in time, in nanoseconds. An instant is relative to an /// unspecified initial value, and can only be compared to instances from /// the same monotonic-clock. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type instant = u64; - /// A duration of time, in nanoseconds. - @since(version = 0.3.0-rc-2025-08-15) - type duration = u64; - /// Read the current value of the clock. /// /// The clock is monotonic, therefore calling this function repeatedly will /// produce a sequence of non-decreasing values. - @since(version = 0.3.0-rc-2025-08-15) + /// + /// For completeness, this function traps if it's not possible to represent + /// the value of the clock in an `instant`. Consequently, implementations + /// should ensure that the starting time is low enough to avoid the + /// possibility of overflow in practice. + @since(version = 0.3.0-rc-2025-09-16) now: func() -> instant; /// Query the resolution of the clock. Returns the duration of time /// corresponding to a clock tick. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-resolution: func() -> duration; /// Wait until the specified instant has occurred. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) wait-until: async func( when: instant, ); - /// Wait for the specified duration has elapsed. - @since(version = 0.3.0-rc-2025-08-15) + /// Wait for the specified duration to elapse. + @since(version = 0.3.0-rc-2025-09-16) wait-for: async func( how-long: duration, ); diff --git a/crates/wasi-http/src/p3/wit/deps/clocks/timezone.wit b/crates/wasi-http/src/p3/wit/deps/clocks/timezone.wit index 2ee16ab2007d..ab8f5c0801f3 100644 --- a/crates/wasi-http/src/p3/wit/deps/clocks/timezone.wit +++ b/crates/wasi-http/src/p3/wit/deps/clocks/timezone.wit @@ -1,4 +1,4 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; @unstable(feature = clocks-timezone) interface timezone { diff --git a/crates/wasi-http/src/p3/wit/deps/clocks/types.wit b/crates/wasi-http/src/p3/wit/deps/clocks/types.wit new file mode 100644 index 000000000000..aff7c2a22a79 --- /dev/null +++ b/crates/wasi-http/src/p3/wit/deps/clocks/types.wit @@ -0,0 +1,8 @@ +package wasi:clocks@0.3.0-rc-2025-09-16; +/// This interface common types used throughout wasi:clocks. +@since(version = 0.3.0-rc-2025-09-16) +interface types { + /// A duration of time, in nanoseconds. + @since(version = 0.3.0-rc-2025-09-16) + type duration = u64; +} diff --git a/crates/wasi-http/src/p3/wit/deps/clocks/wall-clock.wit b/crates/wasi-http/src/p3/wit/deps/clocks/wall-clock.wit index 2e3b2d441af0..ea940500faab 100644 --- a/crates/wasi-http/src/p3/wit/deps/clocks/wall-clock.wit +++ b/crates/wasi-http/src/p3/wit/deps/clocks/wall-clock.wit @@ -1,4 +1,4 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; /// WASI Wall Clock is a clock API intended to let users query the current /// time. The name "wall" makes an analogy to a "clock on the wall", which /// is not necessarily monotonic as it may be reset. @@ -13,10 +13,10 @@ package wasi:clocks@0.3.0-rc-2025-08-15; /// monotonic, making it unsuitable for measuring elapsed time. /// /// It is intended for reporting the current date and time for humans. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface wall-clock { /// A time and date in seconds plus nanoseconds. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record datetime { seconds: u64, nanoseconds: u32, @@ -35,12 +35,12 @@ interface wall-clock { /// /// [POSIX's Seconds Since the Epoch]: https://pubs.opengroup.org/onlinepubs/9699919799/xrat/V4_xbd_chap04.html#tag_21_04_16 /// [Unix Time]: https://en.wikipedia.org/wiki/Unix_time - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) now: func() -> datetime; /// Query the resolution of the clock. /// /// The nanoseconds field of the output is always less than 1000000000. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-resolution: func() -> datetime; } diff --git a/crates/wasi-http/src/p3/wit/deps/clocks/world.wit b/crates/wasi-http/src/p3/wit/deps/clocks/world.wit index 94068c75ceed..a6b885f07011 100644 --- a/crates/wasi-http/src/p3/wit/deps/clocks/world.wit +++ b/crates/wasi-http/src/p3/wit/deps/clocks/world.wit @@ -1,10 +1,10 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import monotonic-clock; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import wall-clock; @unstable(feature = clocks-timezone) import timezone; diff --git a/crates/wasi-http/src/p3/wit/deps/filesystem/preopens.wit b/crates/wasi-http/src/p3/wit/deps/filesystem/preopens.wit index b0f609aa9893..9036e90e8882 100644 --- a/crates/wasi-http/src/p3/wit/deps/filesystem/preopens.wit +++ b/crates/wasi-http/src/p3/wit/deps/filesystem/preopens.wit @@ -1,11 +1,11 @@ -package wasi:filesystem@0.3.0-rc-2025-08-15; +package wasi:filesystem@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface preopens { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use types.{descriptor}; /// Return the set of preopened directories, and their paths. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-directories: func() -> list>; } diff --git a/crates/wasi-http/src/p3/wit/deps/filesystem/types.wit b/crates/wasi-http/src/p3/wit/deps/filesystem/types.wit index 07d770a83bb3..41d91beee571 100644 --- a/crates/wasi-http/src/p3/wit/deps/filesystem/types.wit +++ b/crates/wasi-http/src/p3/wit/deps/filesystem/types.wit @@ -1,4 +1,4 @@ -package wasi:filesystem@0.3.0-rc-2025-08-15; +package wasi:filesystem@0.3.0-rc-2025-09-16; /// WASI filesystem is a filesystem API primarily intended to let users run WASI /// programs that access their files on their existing filesystems, without /// significant overhead. @@ -23,19 +23,19 @@ package wasi:filesystem@0.3.0-rc-2025-08-15; /// [WASI filesystem path resolution]. /// /// [WASI filesystem path resolution]: https://github.com/WebAssembly/wasi-filesystem/blob/main/path-resolution.md -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface types { - @since(version = 0.3.0-rc-2025-08-15) - use wasi:clocks/wall-clock@0.3.0-rc-2025-08-15.{datetime}; + @since(version = 0.3.0-rc-2025-09-16) + use wasi:clocks/wall-clock@0.3.0-rc-2025-09-16.{datetime}; /// File size or length of a region within a file. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type filesize = u64; /// The type of a filesystem object referenced by a descriptor. /// /// Note: This was called `filetype` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum descriptor-type { /// The type of the descriptor or file is unknown or is different from /// any of the other types specified. @@ -59,7 +59,7 @@ interface types { /// Descriptor flags. /// /// Note: This was called `fdflags` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) flags descriptor-flags { /// Read mode: Data can be read. read, @@ -103,7 +103,7 @@ interface types { /// File attributes. /// /// Note: This was called `filestat` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record descriptor-stat { /// File type. %type: descriptor-type, @@ -130,7 +130,7 @@ interface types { } /// Flags determining the method of how paths are resolved. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) flags path-flags { /// As long as the resolved path corresponds to a symbolic link, it is /// expanded. @@ -138,7 +138,7 @@ interface types { } /// Open flags used by `open-at`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) flags open-flags { /// Create file if it does not exist, similar to `O_CREAT` in POSIX. create, @@ -151,11 +151,11 @@ interface types { } /// Number of hard links to an inode. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type link-count = u64; /// When setting a timestamp, this gives the value to set it to. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) variant new-timestamp { /// Leave the timestamp set to its previous value. no-change, @@ -255,7 +255,7 @@ interface types { } /// File or memory access pattern advisory information. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum advice { /// The application has no advice to give on its behavior with respect /// to the specified data. @@ -279,7 +279,7 @@ interface types { /// A 128-bit hash value, split into parts because wasm doesn't have a /// 128-bit integer type. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record metadata-hash-value { /// 64 bits of a 128-bit hash value. lower: u64, @@ -290,7 +290,7 @@ interface types { /// A descriptor is a reference to a filesystem object, which may be a file, /// directory, named pipe, special file, or other object on which filesystem /// calls may be made. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource descriptor { /// Return a stream for reading from a file. /// @@ -308,7 +308,7 @@ interface types { /// resolves to `err` with an `error-code`. /// /// Note: This is similar to `pread` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) read-via-stream: func( /// The offset within the file at which to start reading. offset: filesize, @@ -326,7 +326,7 @@ interface types { /// written or an error is encountered. /// /// Note: This is similar to `pwrite` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) write-via-stream: async func( /// Data to write data: stream, @@ -342,13 +342,13 @@ interface types { /// written or an error is encountered. /// /// Note: This is similar to `write` with `O_APPEND` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) append-via-stream: async func(data: stream) -> result<_, error-code>; /// Provide file advisory information on a descriptor. /// /// This is similar to `posix_fadvise` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) advise: async func( /// The offset within the file to which the advisory applies. offset: filesize, @@ -364,7 +364,7 @@ interface types { /// opened for writing. /// /// Note: This is similar to `fdatasync` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) sync-data: async func() -> result<_, error-code>; /// Get flags associated with a descriptor. @@ -373,7 +373,7 @@ interface types { /// /// Note: This returns the value that was the `fs_flags` value returned /// from `fdstat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-flags: async func() -> result; /// Get the dynamic type of a descriptor. @@ -386,14 +386,14 @@ interface types { /// /// Note: This returns the value that was the `fs_filetype` value returned /// from `fdstat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-type: async func() -> result; /// Adjust the size of an open file. If this increases the file's size, the /// extra bytes are filled with zeros. /// /// Note: This was called `fd_filestat_set_size` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-size: async func(size: filesize) -> result<_, error-code>; /// Adjust the timestamps of an open file or directory. @@ -401,7 +401,7 @@ interface types { /// Note: This is similar to `futimens` in POSIX. /// /// Note: This was called `fd_filestat_set_times` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-times: async func( /// The desired values of the data access timestamp. data-access-timestamp: new-timestamp, @@ -421,7 +421,7 @@ interface types { /// /// This function returns a future, which will resolve to an error code if /// reading full contents of the directory fails. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) read-directory: async func() -> tuple, future>>; /// Synchronize the data and metadata of a file to disk. @@ -430,13 +430,13 @@ interface types { /// opened for writing. /// /// Note: This is similar to `fsync` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) sync: async func() -> result<_, error-code>; /// Create a directory. /// /// Note: This is similar to `mkdirat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) create-directory-at: async func( /// The relative path at which to create the directory. path: string, @@ -451,7 +451,7 @@ interface types { /// modified, use `metadata-hash`. /// /// Note: This was called `fd_filestat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) stat: async func() -> result; /// Return the attributes of a file or directory. @@ -461,7 +461,7 @@ interface types { /// discussion of alternatives. /// /// Note: This was called `path_filestat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) stat-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, @@ -475,7 +475,7 @@ interface types { /// /// Note: This was called `path_filestat_set_times` in earlier versions of /// WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-times-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, @@ -494,7 +494,7 @@ interface types { /// `error-code::not-permitted` if the old path is not a file. /// /// Note: This is similar to `linkat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) link-at: async func( /// Flags determining the method of how the path is resolved. old-path-flags: path-flags, @@ -518,7 +518,7 @@ interface types { /// `error-code::read-only`. /// /// Note: This is similar to `openat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) open-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, @@ -536,7 +536,7 @@ interface types { /// filesystem, this function fails with `error-code::not-permitted`. /// /// Note: This is similar to `readlinkat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) readlink-at: async func( /// The relative path of the symbolic link from which to read. path: string, @@ -547,7 +547,7 @@ interface types { /// Return `error-code::not-empty` if the directory is not empty. /// /// Note: This is similar to `unlinkat(fd, path, AT_REMOVEDIR)` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) remove-directory-at: async func( /// The relative path to a directory to remove. path: string, @@ -556,7 +556,7 @@ interface types { /// Rename a filesystem object. /// /// Note: This is similar to `renameat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) rename-at: async func( /// The relative source path of the file or directory to rename. old-path: string, @@ -572,7 +572,7 @@ interface types { /// `error-code::not-permitted`. /// /// Note: This is similar to `symlinkat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) symlink-at: async func( /// The contents of the symbolic link. old-path: string, @@ -584,7 +584,7 @@ interface types { /// /// Return `error-code::is-directory` if the path refers to a directory. /// Note: This is similar to `unlinkat(fd, path, 0)` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) unlink-file-at: async func( /// The relative path to a file to unlink. path: string, @@ -596,7 +596,7 @@ interface types { /// same device (`st_dev`) and inode (`st_ino` or `d_ino`) numbers. /// wasi-filesystem does not expose device and inode numbers, so this function /// may be used instead. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) is-same-object: async func(other: borrow) -> bool; /// Return a hash of the metadata associated with a filesystem object referred @@ -618,14 +618,14 @@ interface types { /// computed hash. /// /// However, none of these is required. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) metadata-hash: async func() -> result; /// Return a hash of the metadata associated with a filesystem object referred /// to by a directory descriptor and a relative path. /// /// This performs the same hash computation as `metadata-hash`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) metadata-hash-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, diff --git a/crates/wasi-http/src/p3/wit/deps/filesystem/world.wit b/crates/wasi-http/src/p3/wit/deps/filesystem/world.wit index b9dcfd562a2d..87fc72716591 100644 --- a/crates/wasi-http/src/p3/wit/deps/filesystem/world.wit +++ b/crates/wasi-http/src/p3/wit/deps/filesystem/world.wit @@ -1,9 +1,9 @@ -package wasi:filesystem@0.3.0-rc-2025-08-15; +package wasi:filesystem@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import types; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import preopens; } diff --git a/crates/wasi-http/src/p3/wit/deps/http/proxy.wit b/crates/wasi-http/src/p3/wit/deps/http/proxy.wit index 31e5facd03bd..223083ea2870 100644 --- a/crates/wasi-http/src/p3/wit/deps/http/proxy.wit +++ b/crates/wasi-http/src/p3/wit/deps/http/proxy.wit @@ -1,22 +1,22 @@ -package wasi:http@0.3.0-rc-2025-08-15; +package wasi:http@0.3.0-rc-2025-09-16; /// The `wasi:http/imports` world imports all the APIs for HTTP proxies. /// It is intended to be `include`d in other worlds. world imports { /// HTTP proxies have access to time and randomness. - include wasi:clocks/imports@0.3.0-rc-2025-08-15; - import wasi:random/random@0.3.0-rc-2025-08-15; + include wasi:clocks/imports@0.3.0-rc-2025-09-16; + import wasi:random/random@0.3.0-rc-2025-09-16; /// Proxies have standard output and error streams which are expected to /// terminate in a developer-facing console provided by the host. - import wasi:cli/stdout@0.3.0-rc-2025-08-15; - import wasi:cli/stderr@0.3.0-rc-2025-08-15; + import wasi:cli/stdout@0.3.0-rc-2025-09-16; + import wasi:cli/stderr@0.3.0-rc-2025-09-16; /// TODO: this is a temporary workaround until component tooling is able to /// gracefully handle the absence of stdin. Hosts must return an eof stream /// for this import, which is what wasi-libc + tooling will do automatically /// when this import is properly removed. - import wasi:cli/stdin@0.3.0-rc-2025-08-15; + import wasi:cli/stdin@0.3.0-rc-2025-09-16; /// This is the default handler to use when user code simply wants to make an /// HTTP request (e.g., via `fetch()`). diff --git a/crates/wasi-http/src/p3/wit/deps/http/types.wit b/crates/wasi-http/src/p3/wit/deps/http/types.wit index 3d1d0ac78884..8269eea2028d 100644 --- a/crates/wasi-http/src/p3/wit/deps/http/types.wit +++ b/crates/wasi-http/src/p3/wit/deps/http/types.wit @@ -1,7 +1,7 @@ /// This interface defines all of the types and methods for implementing HTTP /// Requests and Responses, as well as their headers, trailers, and bodies. interface types { - use wasi:clocks/monotonic-clock@0.3.0-rc-2025-08-15.{duration}; + use wasi:clocks/monotonic-clock@0.3.0-rc-2025-09-16.{duration}; /// This type corresponds to HTTP standard Methods. variant method { @@ -316,20 +316,12 @@ interface types { /// future to determine whether the body was received successfully. /// The future will only resolve after the stream is reported as closed. /// - /// The stream and future returned by this method are children: - /// they should be closed or consumed before the parent `response` - /// is dropped, or its ownership is transferred to another component - /// by e.g. `handler.handle`. + /// This function takes a `res` future as a parameter, which can be used to + /// communicate an error in handling of the request. /// - /// This method may be called multiple times. - /// - /// This method will return an error if it is called while either: - /// - a stream or future returned by a previous call to this method is still open - /// - a stream returned by a previous call to this method has reported itself as closed - /// Thus there will always be at most one readable stream open for a given body. - /// Each subsequent stream picks up where the previous one left off, - /// continuing until the entire body has been consumed. - consume-body: func() -> result, future, error-code>>>>; + /// Note that function will move the `request`, but references to headers or + /// request options acquired from it previously will remain valid. + consume-body: static func(this: request, res: future>) -> tuple, future, error-code>>>; } /// Parameters for making an HTTP Request. Each of these parameters is @@ -417,19 +409,11 @@ interface types { /// future to determine whether the body was received successfully. /// The future will only resolve after the stream is reported as closed. /// - /// The stream and future returned by this method are children: - /// they should be closed or consumed before the parent `response` - /// is dropped, or its ownership is transferred to another component - /// by e.g. `handler.handle`. - /// - /// This method may be called multiple times. + /// This function takes a `res` future as a parameter, which can be used to + /// communicate an error in handling of the response. /// - /// This method will return an error if it is called while either: - /// - a stream or future returned by a previous call to this method is still open - /// - a stream returned by a previous call to this method has reported itself as closed - /// Thus there will always be at most one readable stream open for a given body. - /// Each subsequent stream picks up where the previous one left off, - /// continuing until the entire body has been consumed. - consume-body: func() -> result, future, error-code>>>>; + /// Note that function will move the `response`, but references to headers + /// acquired from it previously will remain valid. + consume-body: static func(this: response, res: future>) -> tuple, future, error-code>>>; } } diff --git a/crates/wasi-http/src/p3/wit/deps/random/insecure-seed.wit b/crates/wasi-http/src/p3/wit/deps/random/insecure-seed.wit index ecd5043182e4..302151ba65fa 100644 --- a/crates/wasi-http/src/p3/wit/deps/random/insecure-seed.wit +++ b/crates/wasi-http/src/p3/wit/deps/random/insecure-seed.wit @@ -1,9 +1,9 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; /// The insecure-seed interface for seeding hash-map DoS resistance. /// /// It is intended to be portable at least between Unix-family platforms and /// Windows. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface insecure-seed { /// Return a 128-bit value that may contain a pseudo-random value. /// @@ -22,6 +22,6 @@ interface insecure-seed { /// This will likely be changed to a value import, to prevent it from being /// called multiple times and potentially used for purposes other than DoS /// protection. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-insecure-seed: func() -> tuple; } diff --git a/crates/wasi-http/src/p3/wit/deps/random/insecure.wit b/crates/wasi-http/src/p3/wit/deps/random/insecure.wit index d08d85e0b1d7..39146e39101e 100644 --- a/crates/wasi-http/src/p3/wit/deps/random/insecure.wit +++ b/crates/wasi-http/src/p3/wit/deps/random/insecure.wit @@ -1,9 +1,9 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; /// The insecure interface for insecure pseudo-random numbers. /// /// It is intended to be portable at least between Unix-family platforms and /// Windows. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface insecure { /// Return `len` insecure pseudo-random bytes. /// @@ -13,13 +13,13 @@ interface insecure { /// There are no requirements on the values of the returned bytes, however /// implementations are encouraged to return evenly distributed values with /// a long period. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-insecure-random-bytes: func(len: u64) -> list; /// Return an insecure pseudo-random `u64` value. /// /// This function returns the same type of pseudo-random data as /// `get-insecure-random-bytes`, represented as a `u64`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-insecure-random-u64: func() -> u64; } diff --git a/crates/wasi-http/src/p3/wit/deps/random/random.wit b/crates/wasi-http/src/p3/wit/deps/random/random.wit index 3808c3095bfe..fa1f111dc7cb 100644 --- a/crates/wasi-http/src/p3/wit/deps/random/random.wit +++ b/crates/wasi-http/src/p3/wit/deps/random/random.wit @@ -1,9 +1,9 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; /// WASI Random is a random data API. /// /// It is intended to be portable at least between Unix-family platforms and /// Windows. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface random { /// Return `len` cryptographically-secure random or pseudo-random bytes. /// @@ -17,13 +17,13 @@ interface random { /// This function must always return fresh data. Deterministic environments /// must omit this function, rather than implementing it with deterministic /// data. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-random-bytes: func(len: u64) -> list; /// Return a cryptographically-secure random or pseudo-random `u64` value. /// /// This function returns the same type of data as `get-random-bytes`, /// represented as a `u64`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-random-u64: func() -> u64; } diff --git a/crates/wasi-http/src/p3/wit/deps/random/world.wit b/crates/wasi-http/src/p3/wit/deps/random/world.wit index e8f05cc438f6..08c5ed88b7d1 100644 --- a/crates/wasi-http/src/p3/wit/deps/random/world.wit +++ b/crates/wasi-http/src/p3/wit/deps/random/world.wit @@ -1,13 +1,13 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import random; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import insecure; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import insecure-seed; } diff --git a/crates/wasi-http/src/p3/wit/deps/sockets/ip-name-lookup.wit b/crates/wasi-http/src/p3/wit/deps/sockets/ip-name-lookup.wit index ab4156e25d82..6a652ff23263 100644 --- a/crates/wasi-http/src/p3/wit/deps/sockets/ip-name-lookup.wit +++ b/crates/wasi-http/src/p3/wit/deps/sockets/ip-name-lookup.wit @@ -1,10 +1,10 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface ip-name-lookup { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use types.{ip-address}; /// Lookup error codes. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum error-code { /// Unknown error unknown, @@ -57,6 +57,6 @@ interface ip-name-lookup { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resolve-addresses: async func(name: string) -> result, error-code>; } diff --git a/crates/wasi-http/src/p3/wit/deps/sockets/types.wit b/crates/wasi-http/src/p3/wit/deps/sockets/types.wit index 3a7915ce9862..2ed1912e48d3 100644 --- a/crates/wasi-http/src/p3/wit/deps/sockets/types.wit +++ b/crates/wasi-http/src/p3/wit/deps/sockets/types.wit @@ -1,7 +1,7 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface types { - @since(version = 0.3.0-rc-2025-08-15) - use wasi:clocks/monotonic-clock@0.3.0-rc-2025-08-15.{duration}; + @since(version = 0.3.0-rc-2025-09-16) + use wasi:clocks/monotonic-clock@0.3.0-rc-2025-09-16.{duration}; /// Error codes. /// @@ -14,7 +14,7 @@ interface types { /// - `out-of-memory` /// /// See each individual API for what the POSIX equivalents are. They sometimes differ per API. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum error-code { /// Unknown error unknown, @@ -70,7 +70,7 @@ interface types { datagram-too-large, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum ip-address-family { /// Similar to `AF_INET` in POSIX. ipv4, @@ -79,18 +79,18 @@ interface types { ipv6, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type ipv4-address = tuple; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type ipv6-address = tuple; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) variant ip-address { ipv4(ipv4-address), ipv6(ipv6-address), } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record ipv4-socket-address { /// sin_port port: u16, @@ -98,7 +98,7 @@ interface types { address: ipv4-address, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record ipv6-socket-address { /// sin6_port port: u16, @@ -110,7 +110,7 @@ interface types { scope-id: u32, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) variant ip-socket-address { ipv4(ipv4-socket-address), ipv6(ipv6-socket-address), @@ -135,7 +135,7 @@ interface types { /// In addition to the general error codes documented on the /// `types::error-code` type, TCP socket methods may always return /// `error(invalid-state)` when in the `closed` state. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource tcp-socket { /// Create a new TCP socket. @@ -152,7 +152,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) create: static func(address-family: ip-address-family) -> result; /// Bind the socket to the provided IP address and port. @@ -186,7 +186,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) bind: func(local-address: ip-socket-address) -> result<_, error-code>; /// Connect to a remote endpoint. @@ -218,7 +218,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) connect: async func(remote-address: ip-socket-address) -> result<_, error-code>; /// Start listening and return a stream of new inbound connections. @@ -285,7 +285,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) listen: func() -> result, error-code>; /// Transmit data to peer. @@ -308,7 +308,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) send: async func(data: stream) -> result<_, error-code>; /// Read data from peer. @@ -342,7 +342,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) receive: func() -> tuple, future>>; /// Get the bound local address. @@ -361,7 +361,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-local-address: func() -> result; /// Get the remote address. @@ -374,13 +374,13 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-remote-address: func() -> result; /// Whether the socket is in the `listening` state. /// /// Equivalent to the SO_ACCEPTCONN socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-is-listening: func() -> bool; /// Whether this is a IPv4 or IPv6 socket. @@ -388,7 +388,7 @@ interface types { /// This is the value passed to the constructor. /// /// Equivalent to the SO_DOMAIN socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-address-family: func() -> ip-address-family; /// Hints the desired listen queue size. Implementations are free to ignore this. @@ -400,7 +400,7 @@ interface types { /// - `not-supported`: (set) The platform does not support changing the backlog size after the initial listen. /// - `invalid-argument`: (set) The provided value was 0. /// - `invalid-state`: (set) The socket is in the `connecting` or `connected` state. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-listen-backlog-size: func(value: u64) -> result<_, error-code>; /// Enables or disables keepalive. @@ -412,9 +412,9 @@ interface types { /// These properties can be configured while `keep-alive-enabled` is false, but only come into effect when `keep-alive-enabled` is true. /// /// Equivalent to the SO_KEEPALIVE socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-enabled: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-enabled: func(value: bool) -> result<_, error-code>; /// Amount of time the connection has to be idle before TCP starts sending keepalive packets. @@ -427,9 +427,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-idle-time: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-idle-time: func(value: duration) -> result<_, error-code>; /// The time between keepalive packets. @@ -442,9 +442,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-interval: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-interval: func(value: duration) -> result<_, error-code>; /// The maximum amount of keepalive packets TCP should send before aborting the connection. @@ -457,9 +457,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-count: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-count: func(value: u32) -> result<_, error-code>; /// Equivalent to the IP_TTL & IPV6_UNICAST_HOPS socket options. @@ -468,9 +468,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The TTL value must be 1 or higher. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-hop-limit: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-hop-limit: func(value: u8) -> result<_, error-code>; /// The kernel buffer space reserved for sends/receives on this socket. @@ -483,18 +483,18 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-receive-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-receive-buffer-size: func(value: u64) -> result<_, error-code>; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-send-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-send-buffer-size: func(value: u64) -> result<_, error-code>; } /// A UDP socket handle. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource udp-socket { /// Create a new UDP socket. @@ -511,7 +511,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) create: static func(address-family: ip-address-family) -> result; /// Bind the socket to the provided IP address and port. @@ -532,7 +532,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) bind: func(local-address: ip-socket-address) -> result<_, error-code>; /// Associate this socket with a specific peer address. @@ -571,7 +571,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) connect: func(remote-address: ip-socket-address) -> result<_, error-code>; /// Dissociate this socket from its peer address. @@ -589,7 +589,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) disconnect: func() -> result<_, error-code>; /// Send a message on the socket to a particular peer. @@ -623,7 +623,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) send: async func(data: list, remote-address: option) -> result<_, error-code>; /// Receive a message on the socket. @@ -649,7 +649,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) receive: async func() -> result, ip-socket-address>, error-code>; /// Get the current bound address. @@ -668,7 +668,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-local-address: func() -> result; /// Get the address the socket is currently "connected" to. @@ -681,7 +681,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-remote-address: func() -> result; /// Whether this is a IPv4 or IPv6 socket. @@ -689,7 +689,7 @@ interface types { /// This is the value passed to the constructor. /// /// Equivalent to the SO_DOMAIN socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-address-family: func() -> ip-address-family; /// Equivalent to the IP_TTL & IPV6_UNICAST_HOPS socket options. @@ -698,9 +698,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The TTL value must be 1 or higher. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-unicast-hop-limit: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-unicast-hop-limit: func(value: u8) -> result<_, error-code>; /// The kernel buffer space reserved for sends/receives on this socket. @@ -713,13 +713,13 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-receive-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-receive-buffer-size: func(value: u64) -> result<_, error-code>; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-send-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-send-buffer-size: func(value: u64) -> result<_, error-code>; } } diff --git a/crates/wasi-http/src/p3/wit/deps/sockets/world.wit b/crates/wasi-http/src/p3/wit/deps/sockets/world.wit index dfafac2aeacd..44cc427ed194 100644 --- a/crates/wasi-http/src/p3/wit/deps/sockets/world.wit +++ b/crates/wasi-http/src/p3/wit/deps/sockets/world.wit @@ -1,9 +1,9 @@ -package wasi:sockets@0.3.0-rc-2025-08-15; +package wasi:sockets@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import types; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import ip-name-lookup; } diff --git a/crates/wasi-http/src/p3/wit/world.wit b/crates/wasi-http/src/p3/wit/world.wit index 0f8a5b04642a..17d3ee0e5765 100644 --- a/crates/wasi-http/src/p3/wit/world.wit +++ b/crates/wasi-http/src/p3/wit/world.wit @@ -2,5 +2,5 @@ package wasmtime:wasi-http; world bindings { - include wasi:http/proxy@0.3.0-rc-2025-08-15; + include wasi:http/proxy@0.3.0-rc-2025-09-16; } diff --git a/crates/wasi-http/tests/all/p3/mod.rs b/crates/wasi-http/tests/all/p3/mod.rs index 10171b17b324..8701cb3f6a24 100644 --- a/crates/wasi-http/tests/all/p3/mod.rs +++ b/crates/wasi-http/tests/all/p3/mod.rs @@ -320,7 +320,7 @@ async fn p3_http_middleware_with_chain() -> Result<()> { "local:local/chain-http".to_owned(), InstantiationArg { instance: "local:local/chain-http".into(), - export: Some("wasi:http/handler@0.3.0-rc-2025-08-15".into()), + export: Some("wasi:http/handler@0.3.0-rc-2025-09-16".into()), }, )] .into_iter() diff --git a/crates/wasi/src/p3/bindings.rs b/crates/wasi/src/p3/bindings.rs index f2894d519845..f0237ad08fd5 100644 --- a/crates/wasi/src/p3/bindings.rs +++ b/crates/wasi/src/p3/bindings.rs @@ -23,7 +23,7 @@ //! // An example of extending the `wasi:cli/command` world with a //! // custom host interface. //! world my-world { -//! include wasi:cli/command@0.3.0-rc-2025-08-15; +//! include wasi:cli/command@0.3.0-rc-2025-09-16; //! //! import custom-host; //! } diff --git a/crates/wasi/src/p3/cli/host.rs b/crates/wasi/src/p3/cli/host.rs index 0b3d5b177cdc..f18e3c144462 100644 --- a/crates/wasi/src/p3/cli/host.rs +++ b/crates/wasi/src/p3/cli/host.rs @@ -1,6 +1,7 @@ use crate::I32Exit; use crate::cli::{IsTerminal, WasiCli, WasiCliCtxView}; use crate::p3::DEFAULT_BUFFER_CAPACITY; +use crate::p3::bindings::cli::types::ErrorCode; use crate::p3::bindings::cli::{ environment, exit, stderr, stdin, stdout, terminal_input, terminal_output, terminal_stderr, terminal_stdin, terminal_stdout, @@ -10,16 +11,28 @@ use anyhow::{Context as _, anyhow}; use bytes::BytesMut; use core::pin::Pin; use core::task::{Context, Poll}; -use std::io::Cursor; +use std::io::{self, Cursor}; use tokio::io::{AsyncRead, AsyncWrite, ReadBuf}; +use tokio::sync::oneshot; use wasmtime::component::{ - Accessor, Destination, Resource, Source, StreamConsumer, StreamProducer, StreamReader, - StreamResult, + Accessor, Destination, FutureReader, Resource, Source, StreamConsumer, StreamProducer, + StreamReader, StreamResult, }; use wasmtime::{AsContextMut as _, StoreContextMut}; struct InputStreamProducer { rx: Pin>, + result_tx: Option>, +} + +fn io_error_to_error_code(err: io::Error) -> ErrorCode { + match err.kind() { + io::ErrorKind::BrokenPipe => ErrorCode::Pipe, + other => { + tracing::warn!("stdio error: {other}"); + ErrorCode::Io + } + } } impl StreamProducer for InputStreamProducer { @@ -55,8 +68,12 @@ impl StreamProducer for InputStreamProducer { dst.mark_written(n); Poll::Ready(Ok(StreamResult::Completed)) } - Poll::Ready(Err(..)) => { - // TODO: Report the error to the guest + Poll::Ready(Err(e)) => { + let _ = self + .result_tx + .take() + .unwrap() + .send(io_error_to_error_code(e)); Poll::Ready(Ok(StreamResult::Dropped)) } Poll::Pending if finish => Poll::Ready(Ok(StreamResult::Cancelled)), @@ -67,6 +84,7 @@ impl StreamProducer for InputStreamProducer { struct OutputStreamConsumer { tx: Pin>, + result_tx: Option>, } impl StreamConsumer for OutputStreamConsumer { @@ -98,9 +116,11 @@ impl StreamConsumer for OutputStreamConsumer { Poll::Ready(Ok(StreamResult::Completed)) } Poll::Ready(Err(e)) => { - // FIXME(WebAssembly/wasi-cli#81) should communicate this - // error to the guest somehow. - tracing::warn!("dropping stdin error: {e}"); + let _ = self + .result_tx + .take() + .unwrap() + .send(io_error_to_error_code(e)); Poll::Ready(Ok(StreamResult::Dropped)) } Poll::Pending if finish => Poll::Ready(Ok(StreamResult::Cancelled)), @@ -173,17 +193,28 @@ impl terminal_stderr::Host for WasiCliCtxView<'_> { } impl stdin::HostWithStore for WasiCli { - async fn get_stdin(store: &Accessor) -> wasmtime::Result> { + async fn read_via_stream( + store: &Accessor, + ) -> wasmtime::Result<(StreamReader, FutureReader>)> { let instance = store.instance(); store.with(|mut store| { let rx = store.get().ctx.stdin.async_stream(); - Ok(StreamReader::new( + let (result_tx, result_rx) = oneshot::channel(); + let stream = StreamReader::new( instance, &mut store, InputStreamProducer { rx: Box::into_pin(rx), + result_tx: Some(result_tx), }, - )) + ); + let future = FutureReader::new(instance, &mut store, async { + anyhow::Ok(match result_rx.await { + Ok(err) => Err(err), + Err(_) => Ok(()), + }) + }); + Ok((stream, future)) }) } } @@ -191,19 +222,24 @@ impl stdin::HostWithStore for WasiCli { impl stdin::Host for WasiCliCtxView<'_> {} impl stdout::HostWithStore for WasiCli { - async fn set_stdout( + async fn write_via_stream( store: &Accessor, data: StreamReader, - ) -> wasmtime::Result<()> { + ) -> wasmtime::Result> { + let (result_tx, result_rx) = oneshot::channel(); store.with(|mut store| { let tx = store.get().ctx.stdout.async_stream(); data.pipe( store, OutputStreamConsumer { tx: Box::into_pin(tx), + result_tx: Some(result_tx), }, ); - Ok(()) + }); + Ok(match result_rx.await { + Ok(err) => Err(err), + Err(_) => Ok(()), }) } } @@ -211,19 +247,24 @@ impl stdout::HostWithStore for WasiCli { impl stdout::Host for WasiCliCtxView<'_> {} impl stderr::HostWithStore for WasiCli { - async fn set_stderr( + async fn write_via_stream( store: &Accessor, data: StreamReader, - ) -> wasmtime::Result<()> { + ) -> wasmtime::Result> { + let (result_tx, result_rx) = oneshot::channel(); store.with(|mut store| { let tx = store.get().ctx.stderr.async_stream(); data.pipe( store, OutputStreamConsumer { tx: Box::into_pin(tx), + result_tx: Some(result_tx), }, ); - Ok(()) + }); + Ok(match result_rx.await { + Ok(err) => Err(err), + Err(_) => Ok(()), }) } } diff --git a/crates/wasi/src/p3/wit/deps/cli/command.wit b/crates/wasi/src/p3/wit/deps/cli/command.wit index 13e92bdec21d..f2f613e5524a 100644 --- a/crates/wasi/src/p3/wit/deps/cli/command.wit +++ b/crates/wasi/src/p3/wit/deps/cli/command.wit @@ -1,10 +1,10 @@ -package wasi:cli@0.3.0-rc-2025-08-15; +package wasi:cli@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world command { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) include imports; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) export run; } diff --git a/crates/wasi/src/p3/wit/deps/cli/environment.wit b/crates/wasi/src/p3/wit/deps/cli/environment.wit index 769858af1af1..3763f2f6ced5 100644 --- a/crates/wasi/src/p3/wit/deps/cli/environment.wit +++ b/crates/wasi/src/p3/wit/deps/cli/environment.wit @@ -1,4 +1,4 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface environment { /// Get the POSIX-style environment variables. /// @@ -8,15 +8,15 @@ interface environment { /// Morally, these are a value import, but until value imports are available /// in the component model, this import function should return the same /// values each time it is called. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-environment: func() -> list>; /// Get the POSIX-style arguments to the program. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-arguments: func() -> list; /// Return a path that programs should use as their initial current working /// directory, interpreting `.` as shorthand for this. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-initial-cwd: func() -> option; } diff --git a/crates/wasi/src/p3/wit/deps/cli/exit.wit b/crates/wasi/src/p3/wit/deps/cli/exit.wit index dcf791a2019c..1efba7d68320 100644 --- a/crates/wasi/src/p3/wit/deps/cli/exit.wit +++ b/crates/wasi/src/p3/wit/deps/cli/exit.wit @@ -1,7 +1,7 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface exit { /// Exit the current instance and any linked instances. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) exit: func(status: result); /// Exit the current instance and any linked instances, reporting the diff --git a/crates/wasi/src/p3/wit/deps/cli/imports.wit b/crates/wasi/src/p3/wit/deps/cli/imports.wit index 2d513ead9eaf..660a2dd95a99 100644 --- a/crates/wasi/src/p3/wit/deps/cli/imports.wit +++ b/crates/wasi/src/p3/wit/deps/cli/imports.wit @@ -1,34 +1,34 @@ -package wasi:cli@0.3.0-rc-2025-08-15; +package wasi:cli@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) - include wasi:clocks/imports@0.3.0-rc-2025-08-15; - @since(version = 0.3.0-rc-2025-08-15) - include wasi:filesystem/imports@0.3.0-rc-2025-08-15; - @since(version = 0.3.0-rc-2025-08-15) - include wasi:sockets/imports@0.3.0-rc-2025-08-15; - @since(version = 0.3.0-rc-2025-08-15) - include wasi:random/imports@0.3.0-rc-2025-08-15; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:clocks/imports@0.3.0-rc-2025-09-16; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:filesystem/imports@0.3.0-rc-2025-09-16; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:sockets/imports@0.3.0-rc-2025-09-16; + @since(version = 0.3.0-rc-2025-09-16) + include wasi:random/imports@0.3.0-rc-2025-09-16; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import environment; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import exit; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import stdin; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import stdout; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import stderr; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-input; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-output; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-stdin; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-stdout; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import terminal-stderr; } diff --git a/crates/wasi/src/p3/wit/deps/cli/run.wit b/crates/wasi/src/p3/wit/deps/cli/run.wit index 6149db760ee3..631441a3f294 100644 --- a/crates/wasi/src/p3/wit/deps/cli/run.wit +++ b/crates/wasi/src/p3/wit/deps/cli/run.wit @@ -1,6 +1,6 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface run { /// Run the program. - @since(version = 0.3.0-rc-2025-08-15) - run: func() -> result; + @since(version = 0.3.0-rc-2025-09-16) + run: async func() -> result; } diff --git a/crates/wasi/src/p3/wit/deps/cli/stdio.wit b/crates/wasi/src/p3/wit/deps/cli/stdio.wit index 6c99a56f1648..51e5ae4b4094 100644 --- a/crates/wasi/src/p3/wit/deps/cli/stdio.wit +++ b/crates/wasi/src/p3/wit/deps/cli/stdio.wit @@ -1,17 +1,65 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) +interface types { + @since(version = 0.3.0-rc-2025-09-16) + enum error-code { + /// Input/output error + io, + /// Invalid or incomplete multibyte or wide character + illegal-byte-sequence, + /// Broken pipe + pipe, + } +} + +@since(version = 0.3.0-rc-2025-09-16) interface stdin { - @since(version = 0.3.0-rc-2025-08-15) - get-stdin: func() -> stream; + use types.{error-code}; + + /// Return a stream for reading from stdin. + /// + /// This function returns a stream which provides data read from stdin, + /// and a future to signal read results. + /// + /// If the stream's readable end is dropped the future will resolve to success. + /// + /// If the stream's writable end is dropped the future will either resolve to + /// success if stdin was closed by the writer or to an error-code if reading + /// failed for some other reason. + /// + /// Multiple streams may be active at the same time. The behavior of concurrent + /// reads is implementation-specific. + @since(version = 0.3.0-rc-2025-09-16) + read-via-stream: func() -> tuple, future>>; } -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface stdout { - @since(version = 0.3.0-rc-2025-08-15) - set-stdout: func(data: stream); + use types.{error-code}; + + /// Write the given stream to stdout. + /// + /// If the stream's writable end is dropped this function will either return + /// success once the entire contents of the stream have been written or an + /// error-code representing a failure. + /// + /// Otherwise if there is an error the readable end of the stream will be + /// dropped and this function will return an error-code. + @since(version = 0.3.0-rc-2025-09-16) + write-via-stream: async func(data: stream) -> result<_, error-code>; } -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface stderr { - @since(version = 0.3.0-rc-2025-08-15) - set-stderr: func(data: stream); + use types.{error-code}; + + /// Write the given stream to stderr. + /// + /// If the stream's writable end is dropped this function will either return + /// success once the entire contents of the stream have been written or an + /// error-code representing a failure. + /// + /// Otherwise if there is an error the readable end of the stream will be + /// dropped and this function will return an error-code. + @since(version = 0.3.0-rc-2025-09-16) + write-via-stream: async func(data: stream) -> result<_, error-code>; } diff --git a/crates/wasi/src/p3/wit/deps/cli/terminal.wit b/crates/wasi/src/p3/wit/deps/cli/terminal.wit index 3b2b72bbaa03..74c17694a480 100644 --- a/crates/wasi/src/p3/wit/deps/cli/terminal.wit +++ b/crates/wasi/src/p3/wit/deps/cli/terminal.wit @@ -3,10 +3,10 @@ /// In the future, this may include functions for disabling echoing, /// disabling input buffering so that keyboard events are sent through /// immediately, querying supported features, and so on. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-input { /// The input side of a terminal. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource terminal-input; } @@ -15,48 +15,48 @@ interface terminal-input { /// In the future, this may include functions for querying the terminal /// size, being notified of terminal size changes, querying supported /// features, and so on. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-output { /// The output side of a terminal. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource terminal-output; } /// An interface providing an optional `terminal-input` for stdin as a /// link-time authority. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-stdin { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use terminal-input.{terminal-input}; /// If stdin is connected to a terminal, return a `terminal-input` handle /// allowing further interaction with it. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-terminal-stdin: func() -> option; } /// An interface providing an optional `terminal-output` for stdout as a /// link-time authority. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-stdout { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use terminal-output.{terminal-output}; /// If stdout is connected to a terminal, return a `terminal-output` handle /// allowing further interaction with it. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-terminal-stdout: func() -> option; } /// An interface providing an optional `terminal-output` for stderr as a /// link-time authority. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface terminal-stderr { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use terminal-output.{terminal-output}; /// If stderr is connected to a terminal, return a `terminal-output` handle /// allowing further interaction with it. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-terminal-stderr: func() -> option; } diff --git a/crates/wasi/src/p3/wit/deps/clocks/monotonic-clock.wit b/crates/wasi/src/p3/wit/deps/clocks/monotonic-clock.wit index 0c58241fffeb..a91d495c6c15 100644 --- a/crates/wasi/src/p3/wit/deps/clocks/monotonic-clock.wit +++ b/crates/wasi/src/p3/wit/deps/clocks/monotonic-clock.wit @@ -1,4 +1,4 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; /// WASI Monotonic Clock is a clock API intended to let users measure elapsed /// time. /// @@ -7,38 +7,41 @@ package wasi:clocks@0.3.0-rc-2025-08-15; /// /// A monotonic clock is a clock which has an unspecified initial value, and /// successive reads of the clock will produce non-decreasing values. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface monotonic-clock { + use types.{duration}; + /// An instant in time, in nanoseconds. An instant is relative to an /// unspecified initial value, and can only be compared to instances from /// the same monotonic-clock. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type instant = u64; - /// A duration of time, in nanoseconds. - @since(version = 0.3.0-rc-2025-08-15) - type duration = u64; - /// Read the current value of the clock. /// /// The clock is monotonic, therefore calling this function repeatedly will /// produce a sequence of non-decreasing values. - @since(version = 0.3.0-rc-2025-08-15) + /// + /// For completeness, this function traps if it's not possible to represent + /// the value of the clock in an `instant`. Consequently, implementations + /// should ensure that the starting time is low enough to avoid the + /// possibility of overflow in practice. + @since(version = 0.3.0-rc-2025-09-16) now: func() -> instant; /// Query the resolution of the clock. Returns the duration of time /// corresponding to a clock tick. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-resolution: func() -> duration; /// Wait until the specified instant has occurred. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) wait-until: async func( when: instant, ); - /// Wait for the specified duration has elapsed. - @since(version = 0.3.0-rc-2025-08-15) + /// Wait for the specified duration to elapse. + @since(version = 0.3.0-rc-2025-09-16) wait-for: async func( how-long: duration, ); diff --git a/crates/wasi/src/p3/wit/deps/clocks/timezone.wit b/crates/wasi/src/p3/wit/deps/clocks/timezone.wit index 2ee16ab2007d..ab8f5c0801f3 100644 --- a/crates/wasi/src/p3/wit/deps/clocks/timezone.wit +++ b/crates/wasi/src/p3/wit/deps/clocks/timezone.wit @@ -1,4 +1,4 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; @unstable(feature = clocks-timezone) interface timezone { diff --git a/crates/wasi/src/p3/wit/deps/clocks/types.wit b/crates/wasi/src/p3/wit/deps/clocks/types.wit new file mode 100644 index 000000000000..aff7c2a22a79 --- /dev/null +++ b/crates/wasi/src/p3/wit/deps/clocks/types.wit @@ -0,0 +1,8 @@ +package wasi:clocks@0.3.0-rc-2025-09-16; +/// This interface common types used throughout wasi:clocks. +@since(version = 0.3.0-rc-2025-09-16) +interface types { + /// A duration of time, in nanoseconds. + @since(version = 0.3.0-rc-2025-09-16) + type duration = u64; +} diff --git a/crates/wasi/src/p3/wit/deps/clocks/wall-clock.wit b/crates/wasi/src/p3/wit/deps/clocks/wall-clock.wit index 2e3b2d441af0..ea940500faab 100644 --- a/crates/wasi/src/p3/wit/deps/clocks/wall-clock.wit +++ b/crates/wasi/src/p3/wit/deps/clocks/wall-clock.wit @@ -1,4 +1,4 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; /// WASI Wall Clock is a clock API intended to let users query the current /// time. The name "wall" makes an analogy to a "clock on the wall", which /// is not necessarily monotonic as it may be reset. @@ -13,10 +13,10 @@ package wasi:clocks@0.3.0-rc-2025-08-15; /// monotonic, making it unsuitable for measuring elapsed time. /// /// It is intended for reporting the current date and time for humans. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface wall-clock { /// A time and date in seconds plus nanoseconds. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record datetime { seconds: u64, nanoseconds: u32, @@ -35,12 +35,12 @@ interface wall-clock { /// /// [POSIX's Seconds Since the Epoch]: https://pubs.opengroup.org/onlinepubs/9699919799/xrat/V4_xbd_chap04.html#tag_21_04_16 /// [Unix Time]: https://en.wikipedia.org/wiki/Unix_time - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) now: func() -> datetime; /// Query the resolution of the clock. /// /// The nanoseconds field of the output is always less than 1000000000. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-resolution: func() -> datetime; } diff --git a/crates/wasi/src/p3/wit/deps/clocks/world.wit b/crates/wasi/src/p3/wit/deps/clocks/world.wit index 94068c75ceed..a6b885f07011 100644 --- a/crates/wasi/src/p3/wit/deps/clocks/world.wit +++ b/crates/wasi/src/p3/wit/deps/clocks/world.wit @@ -1,10 +1,10 @@ -package wasi:clocks@0.3.0-rc-2025-08-15; +package wasi:clocks@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import monotonic-clock; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import wall-clock; @unstable(feature = clocks-timezone) import timezone; diff --git a/crates/wasi/src/p3/wit/deps/filesystem/preopens.wit b/crates/wasi/src/p3/wit/deps/filesystem/preopens.wit index b0f609aa9893..9036e90e8882 100644 --- a/crates/wasi/src/p3/wit/deps/filesystem/preopens.wit +++ b/crates/wasi/src/p3/wit/deps/filesystem/preopens.wit @@ -1,11 +1,11 @@ -package wasi:filesystem@0.3.0-rc-2025-08-15; +package wasi:filesystem@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface preopens { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use types.{descriptor}; /// Return the set of preopened directories, and their paths. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-directories: func() -> list>; } diff --git a/crates/wasi/src/p3/wit/deps/filesystem/types.wit b/crates/wasi/src/p3/wit/deps/filesystem/types.wit index 07d770a83bb3..41d91beee571 100644 --- a/crates/wasi/src/p3/wit/deps/filesystem/types.wit +++ b/crates/wasi/src/p3/wit/deps/filesystem/types.wit @@ -1,4 +1,4 @@ -package wasi:filesystem@0.3.0-rc-2025-08-15; +package wasi:filesystem@0.3.0-rc-2025-09-16; /// WASI filesystem is a filesystem API primarily intended to let users run WASI /// programs that access their files on their existing filesystems, without /// significant overhead. @@ -23,19 +23,19 @@ package wasi:filesystem@0.3.0-rc-2025-08-15; /// [WASI filesystem path resolution]. /// /// [WASI filesystem path resolution]: https://github.com/WebAssembly/wasi-filesystem/blob/main/path-resolution.md -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface types { - @since(version = 0.3.0-rc-2025-08-15) - use wasi:clocks/wall-clock@0.3.0-rc-2025-08-15.{datetime}; + @since(version = 0.3.0-rc-2025-09-16) + use wasi:clocks/wall-clock@0.3.0-rc-2025-09-16.{datetime}; /// File size or length of a region within a file. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type filesize = u64; /// The type of a filesystem object referenced by a descriptor. /// /// Note: This was called `filetype` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum descriptor-type { /// The type of the descriptor or file is unknown or is different from /// any of the other types specified. @@ -59,7 +59,7 @@ interface types { /// Descriptor flags. /// /// Note: This was called `fdflags` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) flags descriptor-flags { /// Read mode: Data can be read. read, @@ -103,7 +103,7 @@ interface types { /// File attributes. /// /// Note: This was called `filestat` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record descriptor-stat { /// File type. %type: descriptor-type, @@ -130,7 +130,7 @@ interface types { } /// Flags determining the method of how paths are resolved. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) flags path-flags { /// As long as the resolved path corresponds to a symbolic link, it is /// expanded. @@ -138,7 +138,7 @@ interface types { } /// Open flags used by `open-at`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) flags open-flags { /// Create file if it does not exist, similar to `O_CREAT` in POSIX. create, @@ -151,11 +151,11 @@ interface types { } /// Number of hard links to an inode. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type link-count = u64; /// When setting a timestamp, this gives the value to set it to. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) variant new-timestamp { /// Leave the timestamp set to its previous value. no-change, @@ -255,7 +255,7 @@ interface types { } /// File or memory access pattern advisory information. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum advice { /// The application has no advice to give on its behavior with respect /// to the specified data. @@ -279,7 +279,7 @@ interface types { /// A 128-bit hash value, split into parts because wasm doesn't have a /// 128-bit integer type. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record metadata-hash-value { /// 64 bits of a 128-bit hash value. lower: u64, @@ -290,7 +290,7 @@ interface types { /// A descriptor is a reference to a filesystem object, which may be a file, /// directory, named pipe, special file, or other object on which filesystem /// calls may be made. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource descriptor { /// Return a stream for reading from a file. /// @@ -308,7 +308,7 @@ interface types { /// resolves to `err` with an `error-code`. /// /// Note: This is similar to `pread` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) read-via-stream: func( /// The offset within the file at which to start reading. offset: filesize, @@ -326,7 +326,7 @@ interface types { /// written or an error is encountered. /// /// Note: This is similar to `pwrite` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) write-via-stream: async func( /// Data to write data: stream, @@ -342,13 +342,13 @@ interface types { /// written or an error is encountered. /// /// Note: This is similar to `write` with `O_APPEND` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) append-via-stream: async func(data: stream) -> result<_, error-code>; /// Provide file advisory information on a descriptor. /// /// This is similar to `posix_fadvise` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) advise: async func( /// The offset within the file to which the advisory applies. offset: filesize, @@ -364,7 +364,7 @@ interface types { /// opened for writing. /// /// Note: This is similar to `fdatasync` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) sync-data: async func() -> result<_, error-code>; /// Get flags associated with a descriptor. @@ -373,7 +373,7 @@ interface types { /// /// Note: This returns the value that was the `fs_flags` value returned /// from `fdstat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-flags: async func() -> result; /// Get the dynamic type of a descriptor. @@ -386,14 +386,14 @@ interface types { /// /// Note: This returns the value that was the `fs_filetype` value returned /// from `fdstat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-type: async func() -> result; /// Adjust the size of an open file. If this increases the file's size, the /// extra bytes are filled with zeros. /// /// Note: This was called `fd_filestat_set_size` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-size: async func(size: filesize) -> result<_, error-code>; /// Adjust the timestamps of an open file or directory. @@ -401,7 +401,7 @@ interface types { /// Note: This is similar to `futimens` in POSIX. /// /// Note: This was called `fd_filestat_set_times` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-times: async func( /// The desired values of the data access timestamp. data-access-timestamp: new-timestamp, @@ -421,7 +421,7 @@ interface types { /// /// This function returns a future, which will resolve to an error code if /// reading full contents of the directory fails. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) read-directory: async func() -> tuple, future>>; /// Synchronize the data and metadata of a file to disk. @@ -430,13 +430,13 @@ interface types { /// opened for writing. /// /// Note: This is similar to `fsync` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) sync: async func() -> result<_, error-code>; /// Create a directory. /// /// Note: This is similar to `mkdirat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) create-directory-at: async func( /// The relative path at which to create the directory. path: string, @@ -451,7 +451,7 @@ interface types { /// modified, use `metadata-hash`. /// /// Note: This was called `fd_filestat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) stat: async func() -> result; /// Return the attributes of a file or directory. @@ -461,7 +461,7 @@ interface types { /// discussion of alternatives. /// /// Note: This was called `path_filestat_get` in earlier versions of WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) stat-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, @@ -475,7 +475,7 @@ interface types { /// /// Note: This was called `path_filestat_set_times` in earlier versions of /// WASI. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-times-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, @@ -494,7 +494,7 @@ interface types { /// `error-code::not-permitted` if the old path is not a file. /// /// Note: This is similar to `linkat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) link-at: async func( /// Flags determining the method of how the path is resolved. old-path-flags: path-flags, @@ -518,7 +518,7 @@ interface types { /// `error-code::read-only`. /// /// Note: This is similar to `openat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) open-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, @@ -536,7 +536,7 @@ interface types { /// filesystem, this function fails with `error-code::not-permitted`. /// /// Note: This is similar to `readlinkat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) readlink-at: async func( /// The relative path of the symbolic link from which to read. path: string, @@ -547,7 +547,7 @@ interface types { /// Return `error-code::not-empty` if the directory is not empty. /// /// Note: This is similar to `unlinkat(fd, path, AT_REMOVEDIR)` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) remove-directory-at: async func( /// The relative path to a directory to remove. path: string, @@ -556,7 +556,7 @@ interface types { /// Rename a filesystem object. /// /// Note: This is similar to `renameat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) rename-at: async func( /// The relative source path of the file or directory to rename. old-path: string, @@ -572,7 +572,7 @@ interface types { /// `error-code::not-permitted`. /// /// Note: This is similar to `symlinkat` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) symlink-at: async func( /// The contents of the symbolic link. old-path: string, @@ -584,7 +584,7 @@ interface types { /// /// Return `error-code::is-directory` if the path refers to a directory. /// Note: This is similar to `unlinkat(fd, path, 0)` in POSIX. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) unlink-file-at: async func( /// The relative path to a file to unlink. path: string, @@ -596,7 +596,7 @@ interface types { /// same device (`st_dev`) and inode (`st_ino` or `d_ino`) numbers. /// wasi-filesystem does not expose device and inode numbers, so this function /// may be used instead. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) is-same-object: async func(other: borrow) -> bool; /// Return a hash of the metadata associated with a filesystem object referred @@ -618,14 +618,14 @@ interface types { /// computed hash. /// /// However, none of these is required. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) metadata-hash: async func() -> result; /// Return a hash of the metadata associated with a filesystem object referred /// to by a directory descriptor and a relative path. /// /// This performs the same hash computation as `metadata-hash`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) metadata-hash-at: async func( /// Flags determining the method of how the path is resolved. path-flags: path-flags, diff --git a/crates/wasi/src/p3/wit/deps/filesystem/world.wit b/crates/wasi/src/p3/wit/deps/filesystem/world.wit index b9dcfd562a2d..87fc72716591 100644 --- a/crates/wasi/src/p3/wit/deps/filesystem/world.wit +++ b/crates/wasi/src/p3/wit/deps/filesystem/world.wit @@ -1,9 +1,9 @@ -package wasi:filesystem@0.3.0-rc-2025-08-15; +package wasi:filesystem@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import types; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import preopens; } diff --git a/crates/wasi/src/p3/wit/deps/random/insecure-seed.wit b/crates/wasi/src/p3/wit/deps/random/insecure-seed.wit index ecd5043182e4..302151ba65fa 100644 --- a/crates/wasi/src/p3/wit/deps/random/insecure-seed.wit +++ b/crates/wasi/src/p3/wit/deps/random/insecure-seed.wit @@ -1,9 +1,9 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; /// The insecure-seed interface for seeding hash-map DoS resistance. /// /// It is intended to be portable at least between Unix-family platforms and /// Windows. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface insecure-seed { /// Return a 128-bit value that may contain a pseudo-random value. /// @@ -22,6 +22,6 @@ interface insecure-seed { /// This will likely be changed to a value import, to prevent it from being /// called multiple times and potentially used for purposes other than DoS /// protection. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-insecure-seed: func() -> tuple; } diff --git a/crates/wasi/src/p3/wit/deps/random/insecure.wit b/crates/wasi/src/p3/wit/deps/random/insecure.wit index d08d85e0b1d7..39146e39101e 100644 --- a/crates/wasi/src/p3/wit/deps/random/insecure.wit +++ b/crates/wasi/src/p3/wit/deps/random/insecure.wit @@ -1,9 +1,9 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; /// The insecure interface for insecure pseudo-random numbers. /// /// It is intended to be portable at least between Unix-family platforms and /// Windows. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface insecure { /// Return `len` insecure pseudo-random bytes. /// @@ -13,13 +13,13 @@ interface insecure { /// There are no requirements on the values of the returned bytes, however /// implementations are encouraged to return evenly distributed values with /// a long period. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-insecure-random-bytes: func(len: u64) -> list; /// Return an insecure pseudo-random `u64` value. /// /// This function returns the same type of pseudo-random data as /// `get-insecure-random-bytes`, represented as a `u64`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-insecure-random-u64: func() -> u64; } diff --git a/crates/wasi/src/p3/wit/deps/random/random.wit b/crates/wasi/src/p3/wit/deps/random/random.wit index 3808c3095bfe..fa1f111dc7cb 100644 --- a/crates/wasi/src/p3/wit/deps/random/random.wit +++ b/crates/wasi/src/p3/wit/deps/random/random.wit @@ -1,9 +1,9 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; /// WASI Random is a random data API. /// /// It is intended to be portable at least between Unix-family platforms and /// Windows. -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface random { /// Return `len` cryptographically-secure random or pseudo-random bytes. /// @@ -17,13 +17,13 @@ interface random { /// This function must always return fresh data. Deterministic environments /// must omit this function, rather than implementing it with deterministic /// data. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-random-bytes: func(len: u64) -> list; /// Return a cryptographically-secure random or pseudo-random `u64` value. /// /// This function returns the same type of data as `get-random-bytes`, /// represented as a `u64`. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-random-u64: func() -> u64; } diff --git a/crates/wasi/src/p3/wit/deps/random/world.wit b/crates/wasi/src/p3/wit/deps/random/world.wit index e8f05cc438f6..08c5ed88b7d1 100644 --- a/crates/wasi/src/p3/wit/deps/random/world.wit +++ b/crates/wasi/src/p3/wit/deps/random/world.wit @@ -1,13 +1,13 @@ -package wasi:random@0.3.0-rc-2025-08-15; +package wasi:random@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import random; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import insecure; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import insecure-seed; } diff --git a/crates/wasi/src/p3/wit/deps/sockets/ip-name-lookup.wit b/crates/wasi/src/p3/wit/deps/sockets/ip-name-lookup.wit index ab4156e25d82..6a652ff23263 100644 --- a/crates/wasi/src/p3/wit/deps/sockets/ip-name-lookup.wit +++ b/crates/wasi/src/p3/wit/deps/sockets/ip-name-lookup.wit @@ -1,10 +1,10 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface ip-name-lookup { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) use types.{ip-address}; /// Lookup error codes. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum error-code { /// Unknown error unknown, @@ -57,6 +57,6 @@ interface ip-name-lookup { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resolve-addresses: async func(name: string) -> result, error-code>; } diff --git a/crates/wasi/src/p3/wit/deps/sockets/types.wit b/crates/wasi/src/p3/wit/deps/sockets/types.wit index 3a7915ce9862..2ed1912e48d3 100644 --- a/crates/wasi/src/p3/wit/deps/sockets/types.wit +++ b/crates/wasi/src/p3/wit/deps/sockets/types.wit @@ -1,7 +1,7 @@ -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) interface types { - @since(version = 0.3.0-rc-2025-08-15) - use wasi:clocks/monotonic-clock@0.3.0-rc-2025-08-15.{duration}; + @since(version = 0.3.0-rc-2025-09-16) + use wasi:clocks/monotonic-clock@0.3.0-rc-2025-09-16.{duration}; /// Error codes. /// @@ -14,7 +14,7 @@ interface types { /// - `out-of-memory` /// /// See each individual API for what the POSIX equivalents are. They sometimes differ per API. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum error-code { /// Unknown error unknown, @@ -70,7 +70,7 @@ interface types { datagram-too-large, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) enum ip-address-family { /// Similar to `AF_INET` in POSIX. ipv4, @@ -79,18 +79,18 @@ interface types { ipv6, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type ipv4-address = tuple; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) type ipv6-address = tuple; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) variant ip-address { ipv4(ipv4-address), ipv6(ipv6-address), } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record ipv4-socket-address { /// sin_port port: u16, @@ -98,7 +98,7 @@ interface types { address: ipv4-address, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) record ipv6-socket-address { /// sin6_port port: u16, @@ -110,7 +110,7 @@ interface types { scope-id: u32, } - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) variant ip-socket-address { ipv4(ipv4-socket-address), ipv6(ipv6-socket-address), @@ -135,7 +135,7 @@ interface types { /// In addition to the general error codes documented on the /// `types::error-code` type, TCP socket methods may always return /// `error(invalid-state)` when in the `closed` state. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource tcp-socket { /// Create a new TCP socket. @@ -152,7 +152,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) create: static func(address-family: ip-address-family) -> result; /// Bind the socket to the provided IP address and port. @@ -186,7 +186,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) bind: func(local-address: ip-socket-address) -> result<_, error-code>; /// Connect to a remote endpoint. @@ -218,7 +218,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) connect: async func(remote-address: ip-socket-address) -> result<_, error-code>; /// Start listening and return a stream of new inbound connections. @@ -285,7 +285,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) listen: func() -> result, error-code>; /// Transmit data to peer. @@ -308,7 +308,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) send: async func(data: stream) -> result<_, error-code>; /// Read data from peer. @@ -342,7 +342,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) receive: func() -> tuple, future>>; /// Get the bound local address. @@ -361,7 +361,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-local-address: func() -> result; /// Get the remote address. @@ -374,13 +374,13 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-remote-address: func() -> result; /// Whether the socket is in the `listening` state. /// /// Equivalent to the SO_ACCEPTCONN socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-is-listening: func() -> bool; /// Whether this is a IPv4 or IPv6 socket. @@ -388,7 +388,7 @@ interface types { /// This is the value passed to the constructor. /// /// Equivalent to the SO_DOMAIN socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-address-family: func() -> ip-address-family; /// Hints the desired listen queue size. Implementations are free to ignore this. @@ -400,7 +400,7 @@ interface types { /// - `not-supported`: (set) The platform does not support changing the backlog size after the initial listen. /// - `invalid-argument`: (set) The provided value was 0. /// - `invalid-state`: (set) The socket is in the `connecting` or `connected` state. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-listen-backlog-size: func(value: u64) -> result<_, error-code>; /// Enables or disables keepalive. @@ -412,9 +412,9 @@ interface types { /// These properties can be configured while `keep-alive-enabled` is false, but only come into effect when `keep-alive-enabled` is true. /// /// Equivalent to the SO_KEEPALIVE socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-enabled: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-enabled: func(value: bool) -> result<_, error-code>; /// Amount of time the connection has to be idle before TCP starts sending keepalive packets. @@ -427,9 +427,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-idle-time: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-idle-time: func(value: duration) -> result<_, error-code>; /// The time between keepalive packets. @@ -442,9 +442,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-interval: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-interval: func(value: duration) -> result<_, error-code>; /// The maximum amount of keepalive packets TCP should send before aborting the connection. @@ -457,9 +457,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-keep-alive-count: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-keep-alive-count: func(value: u32) -> result<_, error-code>; /// Equivalent to the IP_TTL & IPV6_UNICAST_HOPS socket options. @@ -468,9 +468,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The TTL value must be 1 or higher. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-hop-limit: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-hop-limit: func(value: u8) -> result<_, error-code>; /// The kernel buffer space reserved for sends/receives on this socket. @@ -483,18 +483,18 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-receive-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-receive-buffer-size: func(value: u64) -> result<_, error-code>; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-send-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-send-buffer-size: func(value: u64) -> result<_, error-code>; } /// A UDP socket handle. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) resource udp-socket { /// Create a new UDP socket. @@ -511,7 +511,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) create: static func(address-family: ip-address-family) -> result; /// Bind the socket to the provided IP address and port. @@ -532,7 +532,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) bind: func(local-address: ip-socket-address) -> result<_, error-code>; /// Associate this socket with a specific peer address. @@ -571,7 +571,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) connect: func(remote-address: ip-socket-address) -> result<_, error-code>; /// Dissociate this socket from its peer address. @@ -589,7 +589,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) disconnect: func() -> result<_, error-code>; /// Send a message on the socket to a particular peer. @@ -623,7 +623,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) send: async func(data: list, remote-address: option) -> result<_, error-code>; /// Receive a message on the socket. @@ -649,7 +649,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) receive: async func() -> result, ip-socket-address>, error-code>; /// Get the current bound address. @@ -668,7 +668,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-local-address: func() -> result; /// Get the address the socket is currently "connected" to. @@ -681,7 +681,7 @@ interface types { /// - /// - /// - - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-remote-address: func() -> result; /// Whether this is a IPv4 or IPv6 socket. @@ -689,7 +689,7 @@ interface types { /// This is the value passed to the constructor. /// /// Equivalent to the SO_DOMAIN socket option. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-address-family: func() -> ip-address-family; /// Equivalent to the IP_TTL & IPV6_UNICAST_HOPS socket options. @@ -698,9 +698,9 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The TTL value must be 1 or higher. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-unicast-hop-limit: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-unicast-hop-limit: func(value: u8) -> result<_, error-code>; /// The kernel buffer space reserved for sends/receives on this socket. @@ -713,13 +713,13 @@ interface types { /// /// # Typical errors /// - `invalid-argument`: (set) The provided value was 0. - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-receive-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-receive-buffer-size: func(value: u64) -> result<_, error-code>; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) get-send-buffer-size: func() -> result; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) set-send-buffer-size: func(value: u64) -> result<_, error-code>; } } diff --git a/crates/wasi/src/p3/wit/deps/sockets/world.wit b/crates/wasi/src/p3/wit/deps/sockets/world.wit index dfafac2aeacd..44cc427ed194 100644 --- a/crates/wasi/src/p3/wit/deps/sockets/world.wit +++ b/crates/wasi/src/p3/wit/deps/sockets/world.wit @@ -1,9 +1,9 @@ -package wasi:sockets@0.3.0-rc-2025-08-15; +package wasi:sockets@0.3.0-rc-2025-09-16; -@since(version = 0.3.0-rc-2025-08-15) +@since(version = 0.3.0-rc-2025-09-16) world imports { - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import types; - @since(version = 0.3.0-rc-2025-08-15) + @since(version = 0.3.0-rc-2025-09-16) import ip-name-lookup; }