From 2bf8eb70ebd5d9bbec4349f312118801ae4b82ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 28 Jan 2024 23:14:51 +0800 Subject: [PATCH 01/29] refactor(driver,iocp): make IOCP global to all threads --- Cargo.toml | 1 + compio-driver/Cargo.toml | 3 +- compio-driver/src/iocp/mod.rs | 404 +++++++++++++++++++----------- compio-driver/src/lib.rs | 18 +- compio-runtime/src/runtime/mod.rs | 16 +- 5 files changed, 257 insertions(+), 185 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index dcb21bba..e847af71 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -39,6 +39,7 @@ cfg-if = "1.0.0" criterion = "0.5.1" crossbeam-channel = "0.5.8" crossbeam-queue = "0.3.8" +crossbeam-skiplist = "0.1.3" futures-channel = "0.3.29" futures-util = "0.3.29" libc = "0.2.149" diff --git a/compio-driver/Cargo.toml b/compio-driver/Cargo.toml index f44ce070..da33c1ff 100644 --- a/compio-driver/Cargo.toml +++ b/compio-driver/Cargo.toml @@ -35,6 +35,7 @@ compio-log = { workspace = true } # Utils cfg-if = { workspace = true } crossbeam-channel = { workspace = true } +crossbeam-queue = { workspace = true } slab = { workspace = true } socket2 = { workspace = true } @@ -42,6 +43,7 @@ socket2 = { workspace = true } [target.'cfg(windows)'.dependencies] compio-buf = { workspace = true, features = ["arrayvec"] } aligned-array = "1.0.1" +crossbeam-skiplist = { workspace = true } once_cell = { workspace = true } widestring = { workspace = true } windows-sys = { workspace = true, features = [ @@ -70,7 +72,6 @@ polling = "3.3.0" os_pipe = { workspace = true } [target.'cfg(unix)'.dependencies] -crossbeam-queue = { workspace = true } libc = { workspace = true } [dev-dependencies] diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index f818b976..21619c33 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -1,3 +1,5 @@ +#[cfg(feature = "once_cell_try")] +use std::sync::OnceLock; use std::{ collections::HashSet, io, @@ -7,26 +9,34 @@ use std::{ OwnedHandle, RawHandle, }, pin::Pin, - ptr::{null_mut, NonNull}, - sync::Arc, + ptr::{null, NonNull}, + sync::{ + atomic::{AtomicUsize, Ordering}, + Arc, + }, task::Poll, time::Duration, }; use compio_buf::{arrayvec::ArrayVec, BufResult}; use compio_log::{instrument, trace}; +use crossbeam_queue::SegQueue; +use crossbeam_skiplist::SkipMap; +#[cfg(not(feature = "once_cell_try"))] +use once_cell::sync::OnceCell as OnceLock; use slab::Slab; use windows_sys::Win32::{ Foundation::{ RtlNtStatusToDosError, ERROR_BAD_COMMAND, ERROR_BUSY, ERROR_HANDLE_EOF, - ERROR_IO_INCOMPLETE, ERROR_NO_DATA, ERROR_OPERATION_ABORTED, FACILITY_NTWIN32, - INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, STATUS_SUCCESS, + ERROR_IO_INCOMPLETE, ERROR_NO_DATA, ERROR_OPERATION_ABORTED, ERROR_TIMEOUT, + FACILITY_NTWIN32, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, STATUS_SUCCESS, + WAIT_OBJECT_0, WAIT_TIMEOUT, }, Networking::WinSock::{WSACleanup, WSAStartup, WSADATA}, Storage::FileSystem::SetFileCompletionNotificationModes, System::{ SystemServices::ERROR_SEVERITY_ERROR, - Threading::INFINITE, + Threading::{CreateEventW, SetEvent, WaitForSingleObject, INFINITE}, WindowsProgramming::{FILE_SKIP_COMPLETION_PORT_ON_SUCCESS, FILE_SKIP_SET_EVENT_ON_HANDLE}, IO::{ CreateIoCompletionPort, GetQueuedCompletionStatusEx, PostQueuedCompletionStatus, @@ -109,6 +119,185 @@ impl IntoRawFd for socket2::Socket { } } +struct DriverEntry { + queue: SegQueue, + event: OwnedHandle, +} + +impl DriverEntry { + pub fn new() -> io::Result { + let event = syscall!(BOOL, CreateEventW(null(), 0, 0, null()))?; + Ok(Self { + queue: SegQueue::new(), + event: unsafe { OwnedHandle::from_raw_handle(event as _) }, + }) + } + + pub fn push(&self, entry: Entry) -> io::Result<()> { + self.queue.push(entry); + syscall!(BOOL, SetEvent(self.event.as_raw_handle() as _))?; + Ok(()) + } + + pub fn pop(&self) -> Option { + self.queue.pop() + } + + pub fn wait(&self, timeout: Option) -> io::Result<()> { + let timeout = match timeout { + Some(timeout) => timeout.as_millis() as u32, + None => INFINITE, + }; + let res = unsafe { WaitForSingleObject(self.event.as_raw_handle() as _, timeout) }; + match res { + WAIT_OBJECT_0 => Ok(()), + WAIT_TIMEOUT => Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)), + _ => Err(io::Error::last_os_error()), + } + } +} + +struct CompletionPort { + port: OwnedHandle, + drivers: SkipMap, +} + +impl CompletionPort { + pub fn new() -> io::Result { + let port = syscall!(BOOL, CreateIoCompletionPort(INVALID_HANDLE_VALUE, 0, 0, 0))?; + trace!("new iocp handle: {port}"); + let port = unsafe { OwnedHandle::from_raw_handle(port as _) }; + Ok(Self { + port, + drivers: SkipMap::new(), + }) + } + + pub fn register(&self, driver: usize) -> io::Result<()> { + self.drivers.insert(driver, DriverEntry::new()?); + Ok(()) + } + + pub fn attach(&self, fd: RawFd) -> io::Result<()> { + syscall!( + BOOL, + CreateIoCompletionPort(fd as _, self.port.as_raw_handle() as _, 0, 0) + )?; + syscall!( + BOOL, + SetFileCompletionNotificationModes( + fd as _, + (FILE_SKIP_COMPLETION_PORT_ON_SUCCESS | FILE_SKIP_SET_EVENT_ON_HANDLE) as _ + ) + )?; + Ok(()) + } + + pub fn post( + &self, + res: io::Result, + optr: *mut Overlapped, + ) -> io::Result<()> { + if let Err(e) = &res { + let code = e.raw_os_error().unwrap_or(ERROR_BAD_COMMAND as _); + unsafe { &mut *optr }.base.Internal = ntstatus_from_win32(code) as _; + } + // We have to use CompletionKey to transfer the result because it is large + // enough. It is OK because we set it to zero when attaching handles to IOCP. + syscall!( + BOOL, + PostQueuedCompletionStatus( + self.port.as_raw_handle() as _, + 0, + res.unwrap_or_default(), + optr.cast() + ) + )?; + Ok(()) + } + + pub fn entry(&self, driver: usize) -> crossbeam_skiplist::map::Entry { + self.drivers + .get(&driver) + .expect("driver should register first") + } + + pub fn push(&self, driver: usize, entry: Entry) -> io::Result<()> { + self.entry(driver).value().push(entry) + } + + pub fn wait(&self, driver: usize, timeout: Option) -> io::Result<()> { + self.entry(driver).value().wait(timeout) + } +} + +impl AsRawHandle for CompletionPort { + fn as_raw_handle(&self) -> RawHandle { + self.port.as_raw_handle() + } +} + +static IOCP_PORT: OnceLock = OnceLock::new(); + +#[inline] +fn iocp_port() -> io::Result<&'static CompletionPort> { + IOCP_PORT.get_or_try_init(CompletionPort::new) +} + +fn iocp_start() -> io::Result<()> { + const DEFAULT_CAPACITY: usize = 1024; + + let port = iocp_port()?; + std::thread::spawn(move || { + let mut entries = ArrayVec::::new(); + loop { + let mut recv_count = 0; + syscall!( + BOOL, + GetQueuedCompletionStatusEx( + port.as_raw_handle() as _, + entries.as_mut_ptr(), + DEFAULT_CAPACITY as _, + &mut recv_count, + INFINITE, + 0 + ) + )?; + trace!("recv_count: {recv_count}"); + unsafe { entries.set_len(recv_count as _) }; + + for entry in entries.drain(..) { + let transferred = entry.dwNumberOfBytesTransferred; + trace!("entry transferred: {transferred}"); + // Any thin pointer is OK because we don't use the type of opcode. + let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); + let overlapped = unsafe { &*overlapped_ptr }; + let res = if matches!( + overlapped.base.Internal as NTSTATUS, + STATUS_SUCCESS | STATUS_PENDING + ) { + if entry.lpCompletionKey != 0 { + Ok(entry.lpCompletionKey) + } else { + Ok(transferred as _) + } + } else { + let error = unsafe { RtlNtStatusToDosError(overlapped.base.Internal as _) }; + match error { + ERROR_IO_INCOMPLETE | ERROR_HANDLE_EOF | ERROR_NO_DATA => Ok(0), + _ => Err(io::Error::from_raw_os_error(error as _)), + } + }; + + port.push(overlapped.driver, Entry::new(overlapped.user_data, res))?; + } + } + #[allow(unreachable_code)] + io::Result::Ok(()) + }); + Ok(()) +} + /// Abstraction of IOCP operations. pub trait OpCode { /// Determines that the operation is really overlapped defined by Windows @@ -149,16 +338,18 @@ fn ntstatus_from_win32(x: i32) -> NTSTATUS { } } +static DRIVER_COUNTER: AtomicUsize = AtomicUsize::new(0); +static IOCP_INIT_ONCE: OnceLock<()> = OnceLock::new(); + /// Low-level driver of IOCP. pub(crate) struct Driver { - // IOCP handle could not be duplicated. - port: Arc, + id: usize, cancelled: HashSet, pool: AsyncifyPool, + notify_overlapped: Arc>, } impl Driver { - const DEFAULT_CAPACITY: usize = 1024; const NOTIFY: usize = usize::MAX; pub fn new(builder: &ProactorBuilder) -> io::Result { @@ -166,96 +357,38 @@ impl Driver { let mut data: WSADATA = unsafe { std::mem::zeroed() }; syscall!(SOCKET, WSAStartup(0x202, &mut data))?; - let port = syscall!(BOOL, CreateIoCompletionPort(INVALID_HANDLE_VALUE, 0, 0, 0))?; - trace!("new iocp driver at port: {port}"); - let port = unsafe { OwnedHandle::from_raw_handle(port as _) }; + IOCP_INIT_ONCE.get_or_try_init(iocp_start)?; + + let id = DRIVER_COUNTER.fetch_add(1, Ordering::AcqRel); + iocp_port()?.register(id)?; Ok(Self { - port: Arc::new(port), + id, cancelled: HashSet::default(), pool: builder.create_or_get_thread_pool(), + notify_overlapped: Arc::new(Overlapped::new(id, Self::NOTIFY, ())), }) } - #[inline] - fn poll_impl( - &mut self, - timeout: Option, - iocp_entries: &mut ArrayVec, - ) -> io::Result<()> { - instrument!(compio_log::Level::TRACE, "poll_impl", ?timeout); - let mut recv_count = 0; - let timeout = match timeout { - Some(timeout) => timeout.as_millis() as u32, - None => INFINITE, - }; - syscall!( - BOOL, - GetQueuedCompletionStatusEx( - self.port.as_raw_handle() as _, - iocp_entries.as_mut_ptr(), - N as _, - &mut recv_count, - timeout, - 0, - ) - )?; - trace!("recv_count: {recv_count}"); - unsafe { - iocp_entries.set_len(recv_count as _); - } - Ok(()) + pub fn create_op(&self, user_data: usize, op: T) -> RawOp { + RawOp::new(self.id, user_data, op) } - fn create_entry(&mut self, iocp_entry: OVERLAPPED_ENTRY) -> Option { - if iocp_entry.lpOverlapped.is_null() { - // This entry is posted by `post_driver_nop`. - let user_data = iocp_entry.lpCompletionKey; - trace!("entry {user_data} is posted by post_driver_nop"); - if user_data != Self::NOTIFY { - let result = if self.cancelled.remove(&user_data) { - Err(io::Error::from_raw_os_error(ERROR_OPERATION_ABORTED as _)) - } else { - Ok(0) - }; - Some(Entry::new(user_data, result)) - } else { - None - } - } else { - let transferred = iocp_entry.dwNumberOfBytesTransferred; - // Any thin pointer is OK because we don't use the type of opcode. - trace!("entry transferred: {transferred}"); - let overlapped_ptr: *mut Overlapped<()> = iocp_entry.lpOverlapped.cast(); - let overlapped = unsafe { &*overlapped_ptr }; - let res = if matches!( - overlapped.base.Internal as NTSTATUS, - STATUS_SUCCESS | STATUS_PENDING - ) { - Ok(transferred as _) + fn create_entry(&mut self, entry: Entry) -> Option { + let user_data = entry.user_data(); + if user_data != Self::NOTIFY { + let result = if self.cancelled.remove(&user_data) { + Err(io::Error::from_raw_os_error(ERROR_OPERATION_ABORTED as _)) } else { - let error = unsafe { RtlNtStatusToDosError(overlapped.base.Internal as _) }; - match error { - ERROR_IO_INCOMPLETE | ERROR_HANDLE_EOF | ERROR_NO_DATA => Ok(0), - _ => Err(io::Error::from_raw_os_error(error as _)), - } + entry.into_result() }; - Some(Entry::new(overlapped.user_data, res)) + Some(Entry::new(user_data, result)) + } else { + None } } pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { - syscall!( - BOOL, - CreateIoCompletionPort(fd as _, self.port.as_raw_handle() as _, 0, 0) - )?; - syscall!( - BOOL, - SetFileCompletionNotificationModes( - fd as _, - (FILE_SKIP_COMPLETION_PORT_ON_SUCCESS | FILE_SKIP_SET_EVENT_ON_HANDLE) as _ - ) - )?; - Ok(()) + iocp_port()?.attach(fd) } pub fn cancel(&mut self, user_data: usize, registry: &mut Slab) { @@ -284,7 +417,7 @@ impl Driver { let op_pin = op.as_op_pin(); if op_pin.is_overlapped() { unsafe { op_pin.operate(optr.cast()) } - } else if self.push_blocking(op) { + } else if self.push_blocking(op)? { Poll::Pending } else { Poll::Ready(Err(io::Error::from_raw_os_error(ERROR_BUSY as _))) @@ -292,14 +425,15 @@ impl Driver { } } - fn push_blocking(&mut self, op: &mut RawOp) -> bool { + fn push_blocking(&mut self, op: &mut RawOp) -> io::Result { // Safety: the RawOp is not released before the operation returns. struct SendWrapper(T); unsafe impl Send for SendWrapper {} let optr = SendWrapper(NonNull::from(op)); - let handle = self.as_raw_fd() as _; - self.pool + let port = iocp_port()?; + Ok(self + .pool .dispatch(move || { #[allow(clippy::redundant_locals)] let mut optr = optr; @@ -312,22 +446,9 @@ impl Driver { Poll::Pending => unreachable!("this operation is not overlapped"), Poll::Ready(res) => res, }; - if let Err(e) = &res { - let code = e.raw_os_error().unwrap_or(ERROR_BAD_COMMAND as _); - unsafe { &mut *optr }.base.Internal = ntstatus_from_win32(code) as _; - } - syscall!( - BOOL, - PostQueuedCompletionStatus( - handle, - res.unwrap_or_default() as _, - 0, - optr.cast() - ) - ) - .ok(); + port.post(res, optr).ok(); }) - .is_ok() + .is_ok()) } pub unsafe fn poll( @@ -336,42 +457,24 @@ impl Driver { mut entries: OutEntries>, ) -> io::Result<()> { instrument!(compio_log::Level::TRACE, "poll", ?timeout); - // Prevent stack growth. - let mut iocp_entries = ArrayVec::::new(); - self.poll_impl(timeout, &mut iocp_entries)?; - entries.extend(iocp_entries.drain(..).filter_map(|e| self.create_entry(e))); - // See if there are remaining entries. - loop { - match self.poll_impl(Some(Duration::ZERO), &mut iocp_entries) { - Ok(()) => { - entries.extend(iocp_entries.drain(..).filter_map(|e| self.create_entry(e))); - } - Err(e) => match e.kind() { - io::ErrorKind::TimedOut => { - trace!("poll timeout"); - break; - } - _ => return Err(e), - }, - } + let port = iocp_port()?; + + port.wait(self.id, timeout)?; + + { + let driver_entry = port.entry(self.id); + let completed_entries = driver_entry.value(); + entries.extend( + std::iter::from_fn(|| completed_entries.pop()).filter_map(|e| self.create_entry(e)), + ); } Ok(()) } pub fn handle(&self) -> io::Result { - self.handle_for(Self::NOTIFY) - } - - pub fn handle_for(&self, user_data: usize) -> io::Result { - Ok(NotifyHandle::new(user_data, self.port.clone())) - } -} - -impl AsRawFd for Driver { - fn as_raw_fd(&self) -> RawFd { - self.port.as_raw_handle() + Ok(NotifyHandle::new(self.notify_overlapped.clone())) } } @@ -383,30 +486,20 @@ impl Drop for Driver { /// A notify handle to the inner driver. pub struct NotifyHandle { - user_data: usize, - handle: Arc, + overlapped: Arc>, } -unsafe impl Send for NotifyHandle {} -unsafe impl Sync for NotifyHandle {} - impl NotifyHandle { - fn new(user_data: usize, handle: Arc) -> Self { - Self { user_data, handle } + fn new(overlapped: Arc>) -> Self { + Self { overlapped } } /// Notify the inner driver. pub fn notify(&self) -> io::Result<()> { - syscall!( - BOOL, - PostQueuedCompletionStatus( - self.handle.as_raw_handle() as _, - 0, - self.user_data, - null_mut() - ) - )?; - Ok(()) + iocp_port()?.post( + Ok(0), + self.overlapped.as_ref() as *const _ as *mut Overlapped<()> as _, + ) } } @@ -415,6 +508,8 @@ impl NotifyHandle { pub struct Overlapped { /// The base [`OVERLAPPED`]. pub base: OVERLAPPED, + /// The unique ID of created driver. + pub driver: usize, /// The registered user defined data. pub user_data: usize, /// The opcode. @@ -423,15 +518,20 @@ pub struct Overlapped { } impl Overlapped { - pub(crate) fn new(user_data: usize, op: T) -> Self { + pub(crate) fn new(driver: usize, user_data: usize, op: T) -> Self { Self { base: unsafe { std::mem::zeroed() }, + driver, user_data, op, } } } +// SAFETY: neither field of `OVERLAPPED` is used +unsafe impl Send for Overlapped<()> {} +unsafe impl Sync for Overlapped<()> {} + pub(crate) struct RawOp { op: NonNull>, // The two flags here are manual reference counting. The driver holds the strong ref until it @@ -441,8 +541,8 @@ pub(crate) struct RawOp { } impl RawOp { - pub(crate) fn new(user_data: usize, op: impl OpCode + 'static) -> Self { - let op = Overlapped::new(user_data, op); + pub(crate) fn new(driver: usize, user_data: usize, op: impl OpCode + 'static) -> Self { + let op = Overlapped::new(driver, user_data, op); let op = Box::new(op) as Box>; Self { op: unsafe { NonNull::new_unchecked(Box::into_raw(op)) }, diff --git a/compio-driver/src/lib.rs b/compio-driver/src/lib.rs index 328644b7..5a6080ef 100644 --- a/compio-driver/src/lib.rs +++ b/compio-driver/src/lib.rs @@ -223,7 +223,7 @@ impl Proactor { pub fn push(&mut self, op: T) -> PushEntry, BufResult> { let entry = self.ops.vacant_entry(); let user_data = entry.key(); - let op = RawOp::new(user_data, op); + let op = self.driver.create_op(user_data, op); let op = entry.insert(op); match self.driver.push(user_data, op) { Poll::Pending => PushEntry::Pending(unsafe { Key::new(user_data) }), @@ -277,22 +277,6 @@ impl Proactor { pub fn handle(&self) -> io::Result { self.driver.handle() } - - /// Create a notify handle for specified user_data. - /// - /// # Safety - /// - /// The caller should ensure `user_data` being valid. - #[cfg(windows)] - pub unsafe fn handle_for(&self, user_data: usize) -> io::Result { - self.driver.handle_for(user_data) - } -} - -impl AsRawFd for Proactor { - fn as_raw_fd(&self) -> RawFd { - self.driver.as_raw_fd() - } } /// An completed entry returned from kernel. diff --git a/compio-runtime/src/runtime/mod.rs b/compio-runtime/src/runtime/mod.rs index 2923f49a..9e41d980 100644 --- a/compio-runtime/src/runtime/mod.rs +++ b/compio-runtime/src/runtime/mod.rs @@ -13,9 +13,7 @@ use std::{ use async_task::{Runnable, Task}; use compio_buf::IntoInner; -use compio_driver::{ - op::Asyncify, AsRawFd, Key, OpCode, Proactor, ProactorBuilder, PushEntry, RawFd, -}; +use compio_driver::{op::Asyncify, Key, OpCode, Proactor, ProactorBuilder, PushEntry, RawFd}; use compio_log::{debug, instrument}; use crossbeam_queue::SegQueue; use futures_util::{future::Either, FutureExt}; @@ -238,12 +236,6 @@ impl RuntimeInner { } } -impl AsRawFd for RuntimeInner { - fn as_raw_fd(&self) -> RawFd { - self.driver.borrow().as_raw_fd() - } -} - struct RuntimeContext { depth: usize, ptr: Weak, @@ -423,12 +415,6 @@ impl Runtime { } } -impl AsRawFd for Runtime { - fn as_raw_fd(&self) -> RawFd { - self.inner.as_raw_fd() - } -} - #[cfg(feature = "criterion")] impl criterion::async_executor::AsyncExecutor for Runtime { fn block_on(&self, future: impl Future) -> T { From 64858708659310caf3d2b38b068532e08fd4dff5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Mon, 29 Jan 2024 10:29:05 +0800 Subject: [PATCH 02/29] refactor(runtime): simplify attacher --- compio-dispatcher/tests/listener.rs | 16 +- compio-fs/src/file.rs | 35 +++-- compio-fs/src/named_pipe.rs | 27 ++-- compio-fs/src/open_options/unix.rs | 2 +- compio-fs/src/open_options/windows.rs | 2 +- compio-net/src/socket.rs | 72 ++++----- compio-net/src/tcp.rs | 7 +- compio-net/src/udp.rs | 5 +- compio-net/src/unix.rs | 7 +- compio-runtime/src/attacher.rs | 205 +++++--------------------- compio-runtime/src/runtime/mod.rs | 9 +- compio/examples/dispatcher.rs | 17 +-- compio/tests/runtime.rs | 9 +- 13 files changed, 134 insertions(+), 279 deletions(-) diff --git a/compio-dispatcher/tests/listener.rs b/compio-dispatcher/tests/listener.rs index 9ffda59c..4be7cdb0 100644 --- a/compio-dispatcher/tests/listener.rs +++ b/compio-dispatcher/tests/listener.rs @@ -1,10 +1,10 @@ use std::{num::NonZeroUsize, panic::resume_unwind}; -use compio_buf::{arrayvec::ArrayVec, IntoInner}; +use compio_buf::arrayvec::ArrayVec; use compio_dispatcher::Dispatcher; use compio_io::{AsyncReadExt, AsyncWriteExt}; use compio_net::{TcpListener, TcpStream}; -use compio_runtime::{spawn, Unattached}; +use compio_runtime::spawn; use futures_util::{stream::FuturesUnordered, StreamExt}; #[compio_macros::test] @@ -27,15 +27,11 @@ async fn listener_dispatch() { }); let mut handles = FuturesUnordered::new(); for _i in 0..CLIENT_NUM { - let (srv, _) = listener.accept().await.unwrap(); - let srv = Unattached::new(srv).unwrap(); + let (mut srv, _) = listener.accept().await.unwrap(); let handle = dispatcher - .dispatch(move || { - let mut srv = srv.into_inner(); - async move { - let (_, buf) = srv.read_exact(ArrayVec::::new()).await.unwrap(); - assert_eq!(buf.as_slice(), b"Hello world!"); - } + .dispatch(move || async move { + let (_, buf) = srv.read_exact(ArrayVec::::new()).await.unwrap(); + assert_eq!(buf.as_slice(), b"Hello world!"); }) .unwrap(); handles.push(handle.join()); diff --git a/compio-fs/src/file.rs b/compio-fs/src/file.rs index a114740c..7b02561b 100644 --- a/compio-fs/src/file.rs +++ b/compio-fs/src/file.rs @@ -1,14 +1,13 @@ use std::{future::Future, io, mem::ManuallyDrop, path::Path}; -use compio_buf::{buf_try, BufResult, IntoInner, IoBuf, IoBufMut}; +use compio_buf::{BufResult, IntoInner, IoBuf, IoBufMut}; use compio_driver::{ + impl_raw_fd, op::{BufResultExt, CloseFile, FileStat, ReadAt, Sync, WriteAt}, - syscall, + syscall, AsRawFd, }; use compio_io::{AsyncReadAt, AsyncWriteAt}; -use compio_runtime::{ - impl_attachable, impl_try_as_raw_fd, Attacher, Runtime, TryAsRawFd, TryClone, -}; +use compio_runtime::{impl_attachable, Attacher, Runtime, TryClone}; #[cfg(unix)] use { compio_buf::{IoVectoredBuf, IoVectoredBufMut}, @@ -29,6 +28,12 @@ pub struct File { } impl File { + pub(crate) fn new(file: std::fs::File) -> io::Result { + Ok(Self { + inner: Attacher::new(file)?, + }) + } + /// Attempts to open a file in read-only mode. /// /// See the [`OpenOptions::open`] method for more details. @@ -59,7 +64,7 @@ impl File { // `close` should be cancelled. let this = ManuallyDrop::new(self); async move { - let op = CloseFile::new(this.inner.try_as_raw_fd()?); + let op = CloseFile::new(this.inner.as_raw_fd()); Runtime::current().submit(op).await.0?; Ok(()) } @@ -76,7 +81,7 @@ impl File { /// Queries metadata about the underlying file. pub async fn metadata(&self) -> io::Result { - let op = FileStat::new(self.try_as_raw_fd()?); + let op = FileStat::new(self.as_raw_fd()); let BufResult(res, op) = Runtime::current().submit(op).await; res.map(|_| Metadata::from_stat(op.into_inner())) } @@ -88,7 +93,7 @@ impl File { FileBasicInfo, SetFileInformationByHandle, FILE_BASIC_INFO, }; - let fd = self.try_as_raw_fd()? as _; + let fd = self.as_raw_fd() as _; Runtime::current() .spawn_blocking(move || { let info = FILE_BASIC_INFO { @@ -117,7 +122,7 @@ impl File { pub async fn set_permissions(&self, perm: Permissions) -> io::Result<()> { use std::os::unix::fs::PermissionsExt; - let fd = self.try_as_raw_fd()? as _; + let fd = self.as_raw_fd() as _; Runtime::current() .spawn_blocking(move || { syscall!(libc::fchmod(fd, perm.mode() as libc::mode_t))?; @@ -127,7 +132,7 @@ impl File { } async fn sync_impl(&self, datasync: bool) -> io::Result<()> { - let op = Sync::new(self.try_as_raw_fd()?, datasync); + let op = Sync::new(self.as_raw_fd(), datasync); Runtime::current().submit(op).await.0?; Ok(()) } @@ -158,7 +163,7 @@ impl File { impl AsyncReadAt for File { async fn read_at(&self, buffer: T, pos: u64) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = ReadAt::new(fd, pos, buffer); Runtime::current() .submit(op) @@ -173,7 +178,7 @@ impl AsyncReadAt for File { buffer: T, pos: u64, ) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = ReadVectoredAt::new(fd, pos, buffer); Runtime::current() .submit(op) @@ -202,7 +207,7 @@ impl AsyncWriteAt for File { impl AsyncWriteAt for &File { async fn write_at(&mut self, buffer: T, pos: u64) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = WriteAt::new(fd, pos, buffer); Runtime::current().submit(op).await.into_inner() } @@ -213,12 +218,12 @@ impl AsyncWriteAt for &File { buffer: T, pos: u64, ) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = WriteVectoredAt::new(fd, pos, buffer); Runtime::current().submit(op).await.into_inner() } } -impl_try_as_raw_fd!(File, inner); +impl_raw_fd!(File, inner); impl_attachable!(File, inner); diff --git a/compio-fs/src/named_pipe.rs b/compio-fs/src/named_pipe.rs index 32fedaef..ce8b679a 100644 --- a/compio-fs/src/named_pipe.rs +++ b/compio-fs/src/named_pipe.rs @@ -7,9 +7,9 @@ use std::ptr::null_mut; use std::{ffi::OsStr, io, ptr::null}; use compio_buf::{BufResult, IoBuf, IoBufMut}; -use compio_driver::{op::ConnectNamedPipe, syscall, FromRawFd, RawFd}; +use compio_driver::{impl_raw_fd, op::ConnectNamedPipe, syscall, AsRawFd, FromRawFd, RawFd}; use compio_io::{AsyncRead, AsyncReadAt, AsyncWrite, AsyncWriteAt}; -use compio_runtime::{impl_attachable, impl_try_as_raw_fd, Runtime, TryAsRawFd}; +use compio_runtime::{impl_attachable, Runtime}; use widestring::U16CString; use windows_sys::Win32::{ Security::SECURITY_ATTRIBUTES, @@ -125,8 +125,7 @@ impl NamedPipeServer { /// ``` pub fn info(&self) -> io::Result { // Safety: we're ensuring the lifetime of the named pipe. - // Safety: getting info doesn't need to be attached. - unsafe { named_pipe_info(self.as_raw_fd_unchecked()) } + unsafe { named_pipe_info(self.as_raw_fd()) } } /// Enables a named pipe server process to wait for a client process to @@ -152,7 +151,7 @@ impl NamedPipeServer { /// # std::io::Result::Ok(()) }); /// ``` pub async fn connect(&self) -> io::Result<()> { - let op = ConnectNamedPipe::new(self.handle.try_as_raw_fd()?); + let op = ConnectNamedPipe::new(self.handle.as_raw_fd()); Runtime::current().submit(op).await.0?; Ok(()) } @@ -185,7 +184,7 @@ impl NamedPipeServer { /// # }) /// ``` pub fn disconnect(&self) -> io::Result<()> { - syscall!(BOOL, DisconnectNamedPipe(self.try_as_raw_fd()? as _))?; + syscall!(BOOL, DisconnectNamedPipe(self.as_raw_fd() as _))?; Ok(()) } } @@ -240,7 +239,7 @@ impl AsyncWrite for &NamedPipeServer { } } -impl_try_as_raw_fd!(NamedPipeServer, handle); +impl_raw_fd!(NamedPipeServer, handle); impl_attachable!(NamedPipeServer, handle); @@ -319,7 +318,7 @@ impl NamedPipeClient { pub fn info(&self) -> io::Result { // Safety: we're ensuring the lifetime of the named pipe. // Safety: getting info doesn't need to be attached. - unsafe { named_pipe_info(self.as_raw_fd_unchecked()) } + unsafe { named_pipe_info(self.as_raw_fd()) } } } @@ -373,7 +372,7 @@ impl AsyncWrite for &NamedPipeClient { } } -impl_try_as_raw_fd!(NamedPipeClient, handle); +impl_raw_fd!(NamedPipeClient, handle); impl_attachable!(NamedPipeClient, handle); @@ -738,8 +737,8 @@ impl ServerOptions { /// ``` /// use std::{io, ptr}; /// + /// use compio_driver::AsRawFd; /// use compio_fs::named_pipe::ServerOptions; - /// use compio_runtime::TryAsRawFd; /// use windows_sys::Win32::{ /// Foundation::ERROR_SUCCESS, /// Security::{ @@ -759,7 +758,7 @@ impl ServerOptions { /// assert_eq!( /// ERROR_SUCCESS, /// SetSecurityInfo( - /// pipe.as_raw_fd_unchecked() as _, + /// pipe.as_raw_fd() as _, /// SE_KERNEL_OBJECT, /// DACL_SECURITY_INFORMATION, /// ptr::null_mut(), @@ -775,8 +774,8 @@ impl ServerOptions { /// ``` /// use std::{io, ptr}; /// + /// use compio_driver::AsRawFd; /// use compio_fs::named_pipe::ServerOptions; - /// use compio_runtime::TryAsRawFd; /// use windows_sys::Win32::{ /// Foundation::ERROR_ACCESS_DENIED, /// Security::{ @@ -796,7 +795,7 @@ impl ServerOptions { /// assert_eq!( /// ERROR_ACCESS_DENIED, /// SetSecurityInfo( - /// pipe.as_raw_fd_unchecked() as _, + /// pipe.as_raw_fd() as _, /// SE_KERNEL_OBJECT, /// DACL_SECURITY_INFORMATION, /// ptr::null_mut(), @@ -1165,7 +1164,7 @@ impl ClientOptions { let mode = PIPE_READMODE_MESSAGE; syscall!( BOOL, - SetNamedPipeHandleState(file.as_raw_fd_unchecked() as _, &mode, null(), null()) + SetNamedPipeHandleState(file.as_raw_fd() as _, &mode, null(), null()) )?; } diff --git a/compio-fs/src/open_options/unix.rs b/compio-fs/src/open_options/unix.rs index f35dce4c..a544653c 100644 --- a/compio-fs/src/open_options/unix.rs +++ b/compio-fs/src/open_options/unix.rs @@ -92,6 +92,6 @@ impl OpenOptions { let p = path_string(p)?; let op = OpenFile::new(p, flags, self.mode); let fd = Runtime::current().submit(op).await.0? as RawFd; - Ok(unsafe { File::from_raw_fd(fd) }) + File::new(unsafe { std::fs::File::from_raw_fd(fd) }) } } diff --git a/compio-fs/src/open_options/windows.rs b/compio-fs/src/open_options/windows.rs index 004314a6..9bf5e14f 100644 --- a/compio-fs/src/open_options/windows.rs +++ b/compio-fs/src/open_options/windows.rs @@ -160,6 +160,6 @@ impl OpenOptions { ) )?; } - Ok(unsafe { File::from_raw_fd(fd) }) + File::new(unsafe { std::fs::File::from_raw_fd(fd) }) } } diff --git a/compio-net/src/socket.rs b/compio-net/src/socket.rs index c3d5e195..28217cdb 100644 --- a/compio-net/src/socket.rs +++ b/compio-net/src/socket.rs @@ -1,13 +1,15 @@ use std::{future::Future, io, mem::ManuallyDrop}; -use compio_buf::{buf_try, BufResult, IntoInner, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; -use compio_driver::op::{ - Accept, BufResultExt, CloseSocket, Connect, Recv, RecvFrom, RecvFromVectored, RecvResultExt, - RecvVectored, Send, SendTo, SendToVectored, SendVectored, ShutdownSocket, -}; -use compio_runtime::{ - impl_attachable, impl_try_as_raw_fd, Attacher, Runtime, TryAsRawFd, TryClone, +use compio_buf::{BufResult, IntoInner, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; +use compio_driver::{ + impl_raw_fd, + op::{ + Accept, BufResultExt, CloseSocket, Connect, Recv, RecvFrom, RecvFromVectored, + RecvResultExt, RecvVectored, Send, SendTo, SendToVectored, SendVectored, ShutdownSocket, + }, + AsRawFd, }; +use compio_runtime::{impl_attachable, Attacher, Runtime, TryClone}; use socket2::{Domain, Protocol, SockAddr, Socket as Socket2, Type}; #[derive(Debug)] @@ -16,10 +18,10 @@ pub struct Socket { } impl Socket { - pub fn from_socket2(socket: Socket2) -> Self { - Self { - socket: Attacher::new(socket), - } + pub fn from_socket2(socket: Socket2) -> io::Result { + Ok(Self { + socket: Attacher::new(socket)?, + }) } pub fn try_clone(&self) -> io::Result { @@ -28,11 +30,11 @@ impl Socket { } pub fn peer_addr(&self) -> io::Result { - unsafe { self.socket.get_unchecked() }.peer_addr() + self.socket.peer_addr() } pub fn local_addr(&self) -> io::Result { - unsafe { self.socket.get_unchecked() }.local_addr() + self.socket.local_addr() } pub fn new(domain: Domain, ty: Type, protocol: Option) -> io::Result { @@ -48,25 +50,25 @@ impl Socket { )) { socket.set_nonblocking(true)?; } - Ok(Self::from_socket2(socket)) + Self::from_socket2(socket) } pub fn bind(addr: &SockAddr, ty: Type, protocol: Option) -> io::Result { let socket = Self::new(addr.domain(), ty, protocol)?; - unsafe { socket.socket.get_unchecked() }.bind(addr)?; + socket.socket.bind(addr)?; Ok(socket) } pub fn listen(&self, backlog: i32) -> io::Result<()> { - unsafe { self.socket.get_unchecked() }.listen(backlog) + self.socket.listen(backlog) } pub fn connect(&self, addr: &SockAddr) -> io::Result<()> { - self.socket.try_get()?.connect(addr) + self.socket.connect(addr) } pub async fn connect_async(&self, addr: &SockAddr) -> io::Result<()> { - let op = Connect::new(self.try_as_raw_fd()?, addr.clone()); + let op = Connect::new(self.as_raw_fd(), addr.clone()); let BufResult(res, _op) = Runtime::current().submit(op).await; #[cfg(windows)] { @@ -84,7 +86,7 @@ impl Socket { pub async fn accept(&self) -> io::Result<(Self, SockAddr)> { use compio_driver::FromRawFd; - let op = Accept::new(self.try_as_raw_fd()?); + let op = Accept::new(self.as_raw_fd()); let BufResult(res, op) = Runtime::current().submit(op).await; let accept_sock = unsafe { Socket2::from_raw_fd(res? as _) }; if cfg!(all( @@ -100,21 +102,19 @@ impl Socket { #[cfg(windows)] pub async fn accept(&self) -> io::Result<(Self, SockAddr)> { - use compio_driver::AsRawFd; - let local_addr = self.local_addr()?; // We should allow users sending this accepted socket to a new thread. let accept_sock = Socket2::new( local_addr.domain(), - unsafe { self.socket.get_unchecked() }.r#type()?, - unsafe { self.socket.get_unchecked() }.protocol()?, + self.socket.r#type()?, + self.socket.protocol()?, )?; - let op = Accept::new(self.try_as_raw_fd()?, accept_sock.as_raw_fd() as _); + let op = Accept::new(self.as_raw_fd(), accept_sock.as_raw_fd() as _); let BufResult(res, op) = Runtime::current().submit(op).await; res?; op.update_context()?; let addr = op.into_addr()?; - Ok((Self::from_socket2(accept_sock), addr)) + Ok((Self::from_socket2(accept_sock)?, addr)) } pub fn close(self) -> impl Future> { @@ -123,20 +123,20 @@ impl Socket { // `close` should be cancelled. let this = ManuallyDrop::new(self); async move { - let op = CloseSocket::new(this.try_as_raw_fd()?); + let op = CloseSocket::new(this.as_raw_fd()); Runtime::current().submit(op).await.0?; Ok(()) } } pub async fn shutdown(&self) -> io::Result<()> { - let op = ShutdownSocket::new(self.try_as_raw_fd()?, std::net::Shutdown::Write); + let op = ShutdownSocket::new(self.as_raw_fd(), std::net::Shutdown::Write); Runtime::current().submit(op).await.0?; Ok(()) } pub async fn recv(&self, buffer: B) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = Recv::new(fd, buffer); Runtime::current() .submit(op) @@ -146,7 +146,7 @@ impl Socket { } pub async fn recv_vectored(&self, buffer: V) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = RecvVectored::new(fd, buffer); Runtime::current() .submit(op) @@ -156,19 +156,19 @@ impl Socket { } pub async fn send(&self, buffer: T) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = Send::new(fd, buffer); Runtime::current().submit(op).await.into_inner() } pub async fn send_vectored(&self, buffer: T) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = SendVectored::new(fd, buffer); Runtime::current().submit(op).await.into_inner() } pub async fn recv_from(&self, buffer: T) -> BufResult<(usize, SockAddr), T> { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = RecvFrom::new(fd, buffer); Runtime::current() .submit(op) @@ -182,7 +182,7 @@ impl Socket { &self, buffer: T, ) -> BufResult<(usize, SockAddr), T> { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = RecvFromVectored::new(fd, buffer); Runtime::current() .submit(op) @@ -193,7 +193,7 @@ impl Socket { } pub async fn send_to(&self, buffer: T, addr: &SockAddr) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = SendTo::new(fd, buffer, addr.clone()); Runtime::current().submit(op).await.into_inner() } @@ -203,12 +203,12 @@ impl Socket { buffer: T, addr: &SockAddr, ) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = SendToVectored::new(fd, buffer, addr.clone()); Runtime::current().submit(op).await.into_inner() } } -impl_try_as_raw_fd!(Socket, socket); +impl_raw_fd!(Socket, socket); impl_attachable!(Socket, socket); diff --git a/compio-net/src/tcp.rs b/compio-net/src/tcp.rs index cd025a4f..8c436676 100644 --- a/compio-net/src/tcp.rs +++ b/compio-net/src/tcp.rs @@ -1,8 +1,9 @@ use std::{future::Future, io, net::SocketAddr}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; +use compio_driver::impl_raw_fd; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::{impl_attachable, impl_try_as_raw_fd}; +use compio_runtime::impl_attachable; use socket2::{Protocol, SockAddr, Type}; use crate::{OwnedReadHalf, OwnedWriteHalf, ReadHalf, Socket, ToSocketAddrsAsync, WriteHalf}; @@ -117,7 +118,7 @@ impl TcpListener { } } -impl_try_as_raw_fd!(TcpListener, inner); +impl_raw_fd!(TcpListener, inner); impl_attachable!(TcpListener, inner); @@ -292,6 +293,6 @@ impl AsyncWrite for &TcpStream { } } -impl_try_as_raw_fd!(TcpStream, inner); +impl_raw_fd!(TcpStream, inner); impl_attachable!(TcpStream, inner); diff --git a/compio-net/src/udp.rs b/compio-net/src/udp.rs index 92e2abeb..5e591c8b 100644 --- a/compio-net/src/udp.rs +++ b/compio-net/src/udp.rs @@ -1,7 +1,8 @@ use std::{future::Future, io, net::SocketAddr}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; -use compio_runtime::{impl_attachable, impl_try_as_raw_fd}; +use compio_driver::impl_raw_fd; +use compio_runtime::impl_attachable; use socket2::{Protocol, SockAddr, Type}; use crate::{Socket, ToSocketAddrsAsync}; @@ -259,6 +260,6 @@ impl UdpSocket { } } -impl_try_as_raw_fd!(UdpSocket, inner); +impl_raw_fd!(UdpSocket, inner); impl_attachable!(UdpSocket, inner); diff --git a/compio-net/src/unix.rs b/compio-net/src/unix.rs index 0eff2b02..059b1404 100644 --- a/compio-net/src/unix.rs +++ b/compio-net/src/unix.rs @@ -1,8 +1,9 @@ use std::{future::Future, io, path::Path}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; +use compio_driver::impl_raw_fd; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::{impl_attachable, impl_try_as_raw_fd}; +use compio_runtime::impl_attachable; use socket2::{SockAddr, Type}; use crate::{OwnedReadHalf, OwnedWriteHalf, ReadHalf, Socket, WriteHalf}; @@ -96,7 +97,7 @@ impl UnixListener { } } -impl_try_as_raw_fd!(UnixListener, inner); +impl_raw_fd!(UnixListener, inner); impl_attachable!(UnixListener, inner); @@ -277,7 +278,7 @@ impl AsyncWrite for &UnixStream { } } -impl_try_as_raw_fd!(UnixStream, inner); +impl_raw_fd!(UnixStream, inner); impl_attachable!(UnixStream, inner); diff --git a/compio-runtime/src/attacher.rs b/compio-runtime/src/attacher.rs index 29430865..3958fca0 100644 --- a/compio-runtime/src/attacher.rs +++ b/compio-runtime/src/attacher.rs @@ -4,7 +4,10 @@ use std::os::fd::OwnedFd; use std::os::windows::prelude::{OwnedHandle, OwnedSocket}; #[cfg(feature = "once_cell_try")] use std::sync::OnceLock; -use std::{io, marker::PhantomData}; +use std::{ + io, + ops::{Deref, DerefMut}, +}; use compio_buf::IntoInner; use compio_driver::AsRawFd; @@ -23,71 +26,28 @@ use crate::Runtime; pub struct Attacher { source: S, // Make it thread safe. - once: OnceLock, - _p: PhantomData<*mut ()>, + once: OnceLock<()>, } -impl Attacher { +impl Attacher { /// Create [`Attacher`]. - pub const fn new(source: S) -> Self { - Self { + pub fn new(source: S) -> io::Result { + let this = Self { source, once: OnceLock::new(), - _p: PhantomData, - } + }; + this.attach()?; + Ok(this) } -} -impl Attacher { /// Attach the source. This method could be called many times, but if the /// action fails, the error will only return once. fn attach(&self) -> io::Result<()> { let r = Runtime::current(); let inner = r.inner(); - let id = self.once.get_or_try_init(|| { - inner.attach(self.source.as_raw_fd())?; - io::Result::Ok(inner.id()) - })?; - if id != &inner.id() { - Err(io::Error::new( - io::ErrorKind::InvalidInput, - "the current runtime is not the attached runtime", - )) - } else { - Ok(()) - } - } - - /// Attach the inner source and get the reference. - pub fn try_get(&self) -> io::Result<&S> { - self.attach()?; - Ok(&self.source) - } - - /// Get the reference of the inner source without attaching it. - /// - /// # Safety - /// - /// The caller should ensure it is attached before submit an operation with - /// it. - pub unsafe fn get_unchecked(&self) -> &S { - &self.source - } - - /// Attach the inner source and get the mutable reference. - pub fn try_get_mut(&mut self) -> io::Result<&mut S> { - self.attach()?; - Ok(&mut self.source) - } - - /// Get the mutable reference of the inner source without attaching it. - /// - /// # Safety - /// - /// The caller should ensure it is attached before submit an operation with - /// it. - pub unsafe fn get_unchecked_mut(&mut self) -> &mut S { - &mut self.source + self.once + .get_or_try_init(|| inner.attach(self.source.as_raw_fd()))?; + Ok(()) } } @@ -105,33 +65,22 @@ impl IntoRawFd for Attacher { impl FromRawFd for Attacher { unsafe fn from_raw_fd(fd: RawFd) -> Self { - Self::new(S::from_raw_fd(fd)) + Self { + source: S::from_raw_fd(fd), + once: OnceLock::from(()), + } } } -impl TryClone for Attacher { +impl TryClone for Attacher { /// Try clone self with the cloned source. The attach state will be /// reserved. - /// - /// ## Platform specific - /// * io-uring/polling: it will try to attach in the current thread if - /// needed. fn try_clone(&self) -> io::Result { let source = self.source.try_clone()?; - let new_self = if cfg!(windows) { - Self { - source, - once: self.once.clone(), - _p: PhantomData, - } - } else { - let new_self = Self::new(source); - if self.is_attached() { - new_self.attach()?; - } - new_self - }; - Ok(new_self) + Ok(Self { + source, + once: self.once.clone(), + }) } } @@ -143,6 +92,20 @@ impl IntoInner for Attacher { } } +impl Deref for Attacher { + type Target = S; + + fn deref(&self) -> &Self::Target { + &self.source + } +} + +impl DerefMut for Attacher { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.source + } +} + /// Represents an attachable resource to driver. pub trait Attachable { /// Check if [`Attachable::attach`] has been called. @@ -188,72 +151,6 @@ impl TryClone for OwnedFd { } } -/// Extracts raw fds. -pub trait TryAsRawFd { - /// Get the inner raw fd, while ensuring the source being attached. - fn try_as_raw_fd(&self) -> io::Result; - - /// Get the inner raw fd and don't check if it has been attached. - /// - /// # Safety - /// - /// The caller should ensure it is attached before submit an operation with - /// it. - unsafe fn as_raw_fd_unchecked(&self) -> RawFd; -} - -impl TryAsRawFd for T { - fn try_as_raw_fd(&self) -> io::Result { - Ok(self.as_raw_fd()) - } - - unsafe fn as_raw_fd_unchecked(&self) -> RawFd { - self.as_raw_fd() - } -} - -impl TryAsRawFd for Attacher { - fn try_as_raw_fd(&self) -> io::Result { - Ok(self.try_get()?.as_raw_fd()) - } - - unsafe fn as_raw_fd_unchecked(&self) -> RawFd { - self.source.as_raw_fd() - } -} - -/// A [`Send`] wrapper for attachable resource that has not been attached. The -/// resource should be able to send to another thread before attaching. -pub struct Unattached(T); - -impl Unattached { - /// Create the [`Unattached`] wrapper, or fail if the resource has already - /// been attached. - pub fn new(a: T) -> Result { - if a.is_attached() { Err(a) } else { Ok(Self(a)) } - } - - /// Create [`Unattached`] without checking. - /// - /// # Safety - /// - /// The caller should ensure that the resource has not been attached. - pub unsafe fn new_unchecked(a: T) -> Self { - Self(a) - } -} - -impl IntoInner for Unattached { - type Inner = T; - - fn into_inner(self) -> Self::Inner { - self.0 - } -} - -unsafe impl Send for Unattached {} -unsafe impl Sync for Unattached {} - #[macro_export] #[doc(hidden)] macro_rules! impl_attachable { @@ -265,31 +162,3 @@ macro_rules! impl_attachable { } }; } - -#[macro_export] -#[doc(hidden)] -macro_rules! impl_try_as_raw_fd { - ($t:ty, $inner:ident) => { - impl $crate::TryAsRawFd for $t { - fn try_as_raw_fd(&self) -> ::std::io::Result<$crate::RawFd> { - self.$inner.try_as_raw_fd() - } - - unsafe fn as_raw_fd_unchecked(&self) -> $crate::RawFd { - self.$inner.as_raw_fd_unchecked() - } - } - impl $crate::FromRawFd for $t { - unsafe fn from_raw_fd(fd: $crate::RawFd) -> Self { - Self { - $inner: $crate::FromRawFd::from_raw_fd(fd), - } - } - } - impl $crate::IntoRawFd for $t { - fn into_raw_fd(self) -> $crate::RawFd { - self.$inner.into_raw_fd() - } - } - }; -} diff --git a/compio-runtime/src/runtime/mod.rs b/compio-runtime/src/runtime/mod.rs index 9e41d980..2d6eb0bd 100644 --- a/compio-runtime/src/runtime/mod.rs +++ b/compio-runtime/src/runtime/mod.rs @@ -3,10 +3,7 @@ use std::{ future::{ready, Future}, io, rc::{Rc, Weak}, - sync::{ - atomic::{AtomicUsize, Ordering}, - Arc, - }, + sync::Arc, task::{Context, Poll, Waker}, time::Duration, }; @@ -41,10 +38,7 @@ impl Default for FutureState { } } -static RUNTIME_COUNTER: AtomicUsize = AtomicUsize::new(0); - pub(crate) struct RuntimeInner { - id: usize, driver: RefCell, runnables: Arc>, op_runtime: RefCell, @@ -55,7 +49,6 @@ pub(crate) struct RuntimeInner { impl RuntimeInner { pub fn new(builder: &ProactorBuilder) -> io::Result { Ok(Self { - id: RUNTIME_COUNTER.fetch_add(1, Ordering::AcqRel), driver: RefCell::new(builder.build()?), runnables: Arc::new(SegQueue::new()), op_runtime: RefCell::default(), diff --git a/compio/examples/dispatcher.rs b/compio/examples/dispatcher.rs index 63e2647c..dc5c6903 100644 --- a/compio/examples/dispatcher.rs +++ b/compio/examples/dispatcher.rs @@ -1,11 +1,10 @@ use std::{num::NonZeroUsize, panic::resume_unwind}; use compio::{ - buf::IntoInner, dispatcher::Dispatcher, io::{AsyncRead, AsyncWriteExt}, net::{TcpListener, TcpStream}, - runtime::{spawn, Unattached}, + runtime::spawn, BufResult, }; use futures_util::{stream::FuturesUnordered, StreamExt}; @@ -34,16 +33,12 @@ async fn main() { .detach(); let mut handles = FuturesUnordered::new(); for _i in 0..CLIENT_NUM { - let (srv, _) = listener.accept().await.unwrap(); - let srv = Unattached::new(srv).unwrap(); + let (mut srv, _) = listener.accept().await.unwrap(); let handle = dispatcher - .dispatch(move || { - let mut srv = srv.into_inner(); - async move { - let BufResult(res, buf) = srv.read(Vec::with_capacity(20)).await; - res.unwrap(); - println!("{}", std::str::from_utf8(&buf).unwrap()); - } + .dispatch(move || async move { + let BufResult(res, buf) = srv.read(Vec::with_capacity(20)).await; + res.unwrap(); + println!("{}", std::str::from_utf8(&buf).unwrap()); }) .unwrap(); handles.push(handle.join()); diff --git a/compio/tests/runtime.rs b/compio/tests/runtime.rs index 080dba9c..83dcd599 100644 --- a/compio/tests/runtime.rs +++ b/compio/tests/runtime.rs @@ -7,7 +7,6 @@ use compio::{ fs::File, io::{AsyncReadAt, AsyncReadExt, AsyncWriteAt, AsyncWriteExt}, net::{TcpListener, TcpStream}, - runtime::Unattached, }; use tempfile::NamedTempFile; @@ -18,14 +17,12 @@ async fn multi_threading() { let listener = TcpListener::bind((Ipv4Addr::LOCALHOST, 0)).await.unwrap(); let addr = listener.local_addr().unwrap(); - let (mut tx, (rx, _)) = + let (mut tx, (mut rx, _)) = futures_util::try_join!(TcpStream::connect(&addr), listener.accept()).unwrap(); tx.write_all(DATA).await.0.unwrap(); - let rx = Unattached::new(rx).unwrap(); if let Err(e) = std::thread::spawn(move || { - let mut rx = rx.into_inner(); compio::runtime::Runtime::new().unwrap().block_on(async { let buffer = Vec::with_capacity(DATA.len()); let ((), buffer) = rx.read_exact(buffer).await.unwrap(); @@ -45,15 +42,13 @@ async fn try_clone() { let listener = TcpListener::bind((Ipv4Addr::LOCALHOST, 0)).await.unwrap(); let addr = listener.local_addr().unwrap(); - let (tx, (rx, _)) = + let (tx, (mut rx, _)) = futures_util::try_join!(TcpStream::connect(&addr), listener.accept()).unwrap(); let mut tx = tx.try_clone().unwrap(); tx.write_all(DATA).await.0.unwrap(); - let rx = Unattached::new(rx.try_clone().unwrap()).unwrap(); if let Err(e) = std::thread::spawn(move || { - let mut rx = rx.into_inner(); compio::runtime::Runtime::new().unwrap().block_on(async { let buffer = Vec::with_capacity(DATA.len()); let ((), buffer) = rx.read_exact(buffer).await.unwrap(); From 5ed954d1a7b2b17c9f058432998d529bc20d3392 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Mon, 29 Jan 2024 10:36:54 +0800 Subject: [PATCH 03/29] refactor(fs,net): impl TryClone --- compio-fs/src/file.rs | 13 +++---------- compio-fs/src/named_pipe.rs | 24 +++++------------------- compio-net/src/socket.rs | 9 +++------ compio-net/src/tcp.rs | 24 +++++------------------- compio-net/src/udp.rs | 13 +++---------- compio-net/src/unix.rs | 24 +++++------------------- compio-runtime/src/attacher.rs | 14 ++++++++++++++ compio/tests/runtime.rs | 1 + 8 files changed, 39 insertions(+), 83 deletions(-) diff --git a/compio-fs/src/file.rs b/compio-fs/src/file.rs index 7b02561b..0b241202 100644 --- a/compio-fs/src/file.rs +++ b/compio-fs/src/file.rs @@ -7,7 +7,7 @@ use compio_driver::{ syscall, AsRawFd, }; use compio_io::{AsyncReadAt, AsyncWriteAt}; -use compio_runtime::{impl_attachable, Attacher, Runtime, TryClone}; +use compio_runtime::{impl_attachable, impl_try_clone, Attacher, Runtime}; #[cfg(unix)] use { compio_buf::{IoVectoredBuf, IoVectoredBufMut}, @@ -70,15 +70,6 @@ impl File { } } - /// Creates a new `File` instance that shares the same underlying file - /// handle as the existing `File` instance. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - let inner = self.inner.try_clone()?; - Ok(Self { inner }) - } - /// Queries metadata about the underlying file. pub async fn metadata(&self) -> io::Result { let op = FileStat::new(self.as_raw_fd()); @@ -227,3 +218,5 @@ impl AsyncWriteAt for &File { impl_raw_fd!(File, inner); impl_attachable!(File, inner); + +impl_try_clone!(File, inner); diff --git a/compio-fs/src/named_pipe.rs b/compio-fs/src/named_pipe.rs index ce8b679a..d4a563ac 100644 --- a/compio-fs/src/named_pipe.rs +++ b/compio-fs/src/named_pipe.rs @@ -9,7 +9,7 @@ use std::{ffi::OsStr, io, ptr::null}; use compio_buf::{BufResult, IoBuf, IoBufMut}; use compio_driver::{impl_raw_fd, op::ConnectNamedPipe, syscall, AsRawFd, FromRawFd, RawFd}; use compio_io::{AsyncRead, AsyncReadAt, AsyncWrite, AsyncWriteAt}; -use compio_runtime::{impl_attachable, Runtime}; +use compio_runtime::{impl_attachable, impl_try_clone, Runtime}; use widestring::U16CString; use windows_sys::Win32::{ Security::SECURITY_ATTRIBUTES, @@ -93,15 +93,6 @@ pub struct NamedPipeServer { } impl NamedPipeServer { - /// Creates a new independently owned handle to the underlying file handle. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - Ok(Self { - handle: self.handle.try_clone()?, - }) - } - /// Retrieves information about the named pipe the server is associated /// with. /// @@ -243,6 +234,8 @@ impl_raw_fd!(NamedPipeServer, handle); impl_attachable!(NamedPipeServer, handle); +impl_try_clone!(NamedPipeServer, handle); + /// A [Windows named pipe] client. /// /// Constructed using [`ClientOptions::open`]. @@ -288,15 +281,6 @@ pub struct NamedPipeClient { } impl NamedPipeClient { - /// Creates a new independently owned handle to the underlying file handle. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - Ok(Self { - handle: self.handle.try_clone()?, - }) - } - /// Retrieves information about the named pipe the client is associated /// with. /// @@ -376,6 +360,8 @@ impl_raw_fd!(NamedPipeClient, handle); impl_attachable!(NamedPipeClient, handle); +impl_try_clone!(NamedPipeClient, handle); + /// A builder structure for construct a named pipe with named pipe-specific /// options. This is required to use for named pipe servers who wants to modify /// pipe-related options. diff --git a/compio-net/src/socket.rs b/compio-net/src/socket.rs index 28217cdb..1c4aecdd 100644 --- a/compio-net/src/socket.rs +++ b/compio-net/src/socket.rs @@ -9,7 +9,7 @@ use compio_driver::{ }, AsRawFd, }; -use compio_runtime::{impl_attachable, Attacher, Runtime, TryClone}; +use compio_runtime::{impl_attachable, impl_try_clone, Attacher, Runtime}; use socket2::{Domain, Protocol, SockAddr, Socket as Socket2, Type}; #[derive(Debug)] @@ -24,11 +24,6 @@ impl Socket { }) } - pub fn try_clone(&self) -> io::Result { - let socket = self.socket.try_clone()?; - Ok(Self { socket }) - } - pub fn peer_addr(&self) -> io::Result { self.socket.peer_addr() } @@ -212,3 +207,5 @@ impl Socket { impl_raw_fd!(Socket, socket); impl_attachable!(Socket, socket); + +impl_try_clone!(Socket, socket); diff --git a/compio-net/src/tcp.rs b/compio-net/src/tcp.rs index 8c436676..5ab77dd0 100644 --- a/compio-net/src/tcp.rs +++ b/compio-net/src/tcp.rs @@ -3,7 +3,7 @@ use std::{future::Future, io, net::SocketAddr}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; use compio_driver::impl_raw_fd; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::impl_attachable; +use compio_runtime::{impl_attachable, impl_try_clone}; use socket2::{Protocol, SockAddr, Type}; use crate::{OwnedReadHalf, OwnedWriteHalf, ReadHalf, Socket, ToSocketAddrsAsync, WriteHalf}; @@ -68,15 +68,6 @@ impl TcpListener { self.inner.close() } - /// Creates a new independently owned handle to the underlying socket. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - Ok(Self { - inner: self.inner.try_clone()?, - }) - } - /// Accepts a new incoming connection from this listener. /// /// This function will yield once a new TCP connection is established. When @@ -122,6 +113,8 @@ impl_raw_fd!(TcpListener, inner); impl_attachable!(TcpListener, inner); +impl_try_clone!(TcpListener, inner); + /// A TCP stream between a local and a remote socket. /// /// A TCP stream can either be created by connecting to an endpoint, via the @@ -182,15 +175,6 @@ impl TcpStream { self.inner.close() } - /// Creates a new independently owned handle to the underlying socket. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - Ok(Self { - inner: self.inner.try_clone()?, - }) - } - /// Returns the socket address of the remote peer of this TCP connection. pub fn peer_addr(&self) -> io::Result { self.inner @@ -296,3 +280,5 @@ impl AsyncWrite for &TcpStream { impl_raw_fd!(TcpStream, inner); impl_attachable!(TcpStream, inner); + +impl_try_clone!(TcpStream, inner); diff --git a/compio-net/src/udp.rs b/compio-net/src/udp.rs index 5e591c8b..9c9e610f 100644 --- a/compio-net/src/udp.rs +++ b/compio-net/src/udp.rs @@ -2,7 +2,7 @@ use std::{future::Future, io, net::SocketAddr}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; use compio_driver::impl_raw_fd; -use compio_runtime::impl_attachable; +use compio_runtime::{impl_attachable, impl_try_clone}; use socket2::{Protocol, SockAddr, Type}; use crate::{Socket, ToSocketAddrsAsync}; @@ -122,15 +122,6 @@ impl UdpSocket { self.inner.close() } - /// Creates a new independently owned handle to the underlying socket. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - Ok(Self { - inner: self.inner.try_clone()?, - }) - } - /// Returns the socket address of the remote peer this socket was connected /// to. /// @@ -263,3 +254,5 @@ impl UdpSocket { impl_raw_fd!(UdpSocket, inner); impl_attachable!(UdpSocket, inner); + +impl_try_clone!(UdpSocket, inner); diff --git a/compio-net/src/unix.rs b/compio-net/src/unix.rs index 059b1404..f8d4f388 100644 --- a/compio-net/src/unix.rs +++ b/compio-net/src/unix.rs @@ -3,7 +3,7 @@ use std::{future::Future, io, path::Path}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; use compio_driver::impl_raw_fd; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::impl_attachable; +use compio_runtime::{impl_attachable, impl_try_clone}; use socket2::{SockAddr, Type}; use crate::{OwnedReadHalf, OwnedWriteHalf, ReadHalf, Socket, WriteHalf}; @@ -71,15 +71,6 @@ impl UnixListener { self.inner.close() } - /// Creates a new independently owned handle to the underlying socket. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - Ok(Self { - inner: self.inner.try_clone()?, - }) - } - /// Accepts a new incoming connection from this listener. /// /// This function will yield once a new Unix domain socket connection @@ -101,6 +92,8 @@ impl_raw_fd!(UnixListener, inner); impl_attachable!(UnixListener, inner); +impl_try_clone!(UnixListener, inner); + /// A Unix stream between two local sockets on Windows & WSL. /// /// A Unix stream can either be created by connecting to an endpoint, via the @@ -165,15 +158,6 @@ impl UnixStream { self.inner.close() } - /// Creates a new independently owned handle to the underlying socket. - /// - /// It does not clear the attach state. - pub fn try_clone(&self) -> io::Result { - Ok(Self { - inner: self.inner.try_clone()?, - }) - } - /// Returns the socket path of the remote peer of this connection. pub fn peer_addr(&self) -> io::Result { #[allow(unused_mut)] @@ -325,3 +309,5 @@ fn fix_unix_socket_length(addr: &mut SockAddr) { addr.set_length(addr_len as _); } } + +impl_try_clone!(UnixStream, inner); diff --git a/compio-runtime/src/attacher.rs b/compio-runtime/src/attacher.rs index 3958fca0..a595d1e7 100644 --- a/compio-runtime/src/attacher.rs +++ b/compio-runtime/src/attacher.rs @@ -162,3 +162,17 @@ macro_rules! impl_attachable { } }; } + +#[macro_export] +#[doc(hidden)] +macro_rules! impl_try_clone { + ($t:ty, $inner:ident) => { + impl $crate::TryClone for $t { + fn try_clone(&self) -> ::std::io::Result { + Ok(Self { + $inner: self.$inner.try_clone()?, + }) + } + } + }; +} diff --git a/compio/tests/runtime.rs b/compio/tests/runtime.rs index 83dcd599..70bd9f5c 100644 --- a/compio/tests/runtime.rs +++ b/compio/tests/runtime.rs @@ -8,6 +8,7 @@ use compio::{ io::{AsyncReadAt, AsyncReadExt, AsyncWriteAt, AsyncWriteExt}, net::{TcpListener, TcpStream}, }; +use compio_runtime::TryClone; use tempfile::NamedTempFile; #[compio_macros::test] From d72cdbf37c89c5c8fccdbba6b4b1a1d16e7b2e9b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Mon, 29 Jan 2024 10:59:01 +0800 Subject: [PATCH 04/29] fix(driver): adapt to new inner API --- compio-driver/src/fusion/mod.rs | 7 ++++++ compio-driver/src/iour/mod.rs | 4 +++ compio-driver/src/lib.rs | 11 ++------- compio-driver/src/poll/mod.rs | 44 ++++++++++----------------------- compio-fs/src/pipe.rs | 27 +++++++++++--------- compio-net/src/socket.rs | 2 +- 6 files changed, 43 insertions(+), 52 deletions(-) diff --git a/compio-driver/src/fusion/mod.rs b/compio-driver/src/fusion/mod.rs index de13546e..65fddd03 100644 --- a/compio-driver/src/fusion/mod.rs +++ b/compio-driver/src/fusion/mod.rs @@ -132,6 +132,13 @@ impl Driver { } } + pub fn create_op(&self, user_data: usize, op: T) -> RawOp { + match &self.fuse { + FuseDriver::Poll(driver) => driver.create_op(user_data, op), + FuseDriver::IoUring(driver) => driver.create_op(user_data, op), + } + } + pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { match &mut self.fuse { FuseDriver::Poll(driver) => driver.attach(fd), diff --git a/compio-driver/src/iour/mod.rs b/compio-driver/src/iour/mod.rs index d6884be7..13df621f 100644 --- a/compio-driver/src/iour/mod.rs +++ b/compio-driver/src/iour/mod.rs @@ -196,6 +196,10 @@ impl Driver { entries.extend(completed_entries); } + pub fn create_op(&self, user_data: usize, op: T) -> RawOp { + RawOp::new(user_data, op) + } + pub fn attach(&mut self, _fd: RawFd) -> io::Result<()> { Ok(()) } diff --git a/compio-driver/src/lib.rs b/compio-driver/src/lib.rs index 5a6080ef..047a4801 100644 --- a/compio-driver/src/lib.rs +++ b/compio-driver/src/lib.rs @@ -177,20 +177,13 @@ impl Proactor { }) } - /// Attach an fd to the driver. It will cause unexpected result to attach - /// the handle with one driver and push an op to another driver. + /// Attach an fd to the driver. /// /// ## Platform specific /// * IOCP: it will be attached to the completion port. An fd could only be /// attached to one driver, and could only be attached once, even if you /// `try_clone` it. - /// * io-uring: it will do nothing and return `Ok(())`. - /// * polling: it will initialize inner queue and register to the driver. On - /// Linux and Android, if the fd is a normal file or a directory, this - /// method will do nothing. For other fd and systems, you should only call - /// this method once for a specific resource. If this method is called - /// twice with the same fd, we assume that the old fd has been closed, and - /// it's a new fd. + /// * io-uring & polling: it will do nothing but return `Ok(())`. pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { self.driver.attach(fd) } diff --git a/compio-driver/src/poll/mod.rs b/compio-driver/src/poll/mod.rs index 4722c4aa..ac8ac38e 100644 --- a/compio-driver/src/poll/mod.rs +++ b/compio-driver/src/poll/mod.rs @@ -138,11 +138,6 @@ impl FdQueue { } None } - - pub fn clear(&mut self) { - self.read_queue.clear(); - self.write_queue.clear(); - } } /// Low-level driver of polling. @@ -187,41 +182,28 @@ impl Driver { }) } + pub fn create_op(&self, user_data: usize, op: T) -> RawOp { + RawOp::new(user_data, op) + } + fn submit(&mut self, user_data: usize, arg: WaitArg) -> io::Result<()> { - let queue = self - .registry - .get_mut(&arg.fd) - .expect("the fd should be attached"); + let need_add = !self.registry.contains_key(&arg.fd); + let queue = self.registry.entry(arg.fd).or_default(); queue.push_back_interest(user_data, arg.interest); // We use fd as the key. let event = queue.event(arg.fd as usize); unsafe { - let fd = BorrowedFd::borrow_raw(arg.fd); - self.poll.modify(fd, event)?; + if need_add { + self.poll.add(arg.fd, event)?; + } else { + let fd = BorrowedFd::borrow_raw(arg.fd); + self.poll.modify(fd, event)?; + } } Ok(()) } - pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { - if cfg!(any(target_os = "linux", target_os = "android")) { - let mut stat = unsafe { std::mem::zeroed() }; - syscall!(libc::fstat(fd, &mut stat))?; - if matches!(stat.st_mode & libc::S_IFMT, libc::S_IFREG | libc::S_IFDIR) { - return Ok(()); - } - } - let queue = self.registry.entry(fd).or_default(); - unsafe { - match self.poll.add(fd, Event::none(0)) { - Ok(()) => {} - Err(e) if e.kind() == io::ErrorKind::AlreadyExists => { - queue.clear(); - let fd = BorrowedFd::borrow_raw(fd); - self.poll.modify(fd, Event::none(0))?; - } - Err(e) => return Err(e), - } - } + pub fn attach(&mut self, _fd: RawFd) -> io::Result<()> { Ok(()) } diff --git a/compio-fs/src/pipe.rs b/compio-fs/src/pipe.rs index 01a376f8..4781d0b1 100644 --- a/compio-fs/src/pipe.rs +++ b/compio-fs/src/pipe.rs @@ -2,13 +2,14 @@ use std::{future::Future, io, path::Path}; -use compio_buf::{buf_try, BufResult, IntoInner, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; +use compio_buf::{BufResult, IntoInner, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; use compio_driver::{ + impl_raw_fd, op::{BufResultExt, Recv, RecvVectored, Send, SendVectored}, - syscall, FromRawFd, IntoRawFd, + syscall, AsRawFd, FromRawFd, IntoRawFd, }; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::{impl_attachable, impl_try_as_raw_fd, Runtime, TryAsRawFd}; +use compio_runtime::{impl_attachable, impl_try_clone, Runtime}; use crate::File; @@ -358,13 +359,13 @@ impl AsyncWrite for Sender { impl AsyncWrite for &Sender { async fn write(&mut self, buffer: T) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = Send::new(fd, buffer); Runtime::current().submit(op).await.into_inner() } async fn write_vectored(&mut self, buffer: T) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = SendVectored::new(fd, buffer); Runtime::current().submit(op).await.into_inner() } @@ -380,10 +381,12 @@ impl AsyncWrite for &Sender { } } -impl_try_as_raw_fd!(Sender, file); +impl_raw_fd!(Sender, file); impl_attachable!(Sender, file); +impl_try_clone!(Sender, file); + /// Reading end of a Unix pipe. /// /// It can be constructed from a FIFO file with [`OpenOptions::open_receiver`]. @@ -483,7 +486,7 @@ impl AsyncRead for Receiver { impl AsyncRead for &Receiver { async fn read(&mut self, buffer: B) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = Recv::new(fd, buffer); Runtime::current() .submit(op) @@ -493,7 +496,7 @@ impl AsyncRead for &Receiver { } async fn read_vectored(&mut self, buffer: V) -> BufResult { - let (fd, buffer) = buf_try!(self.try_as_raw_fd(), buffer); + let fd = self.as_raw_fd(); let op = RecvVectored::new(fd, buffer); Runtime::current() .submit(op) @@ -503,10 +506,12 @@ impl AsyncRead for &Receiver { } } -impl_try_as_raw_fd!(Receiver, file); +impl_raw_fd!(Receiver, file); impl_attachable!(Receiver, file); +impl_try_clone!(Receiver, file); + /// Checks if file is a FIFO async fn is_fifo(file: &File) -> io::Result { use std::os::unix::prelude::FileTypeExt; @@ -515,9 +520,9 @@ async fn is_fifo(file: &File) -> io::Result { } /// Sets file's flags with O_NONBLOCK by fcntl. -fn set_nonblocking(file: &impl TryAsRawFd) -> io::Result<()> { +fn set_nonblocking(file: &impl AsRawFd) -> io::Result<()> { if cfg!(not(all(target_os = "linux", feature = "io-uring"))) { - let fd = file.try_as_raw_fd()?; + let fd = file.as_raw_fd(); let current_flags = syscall!(libc::fcntl(fd, libc::F_GETFL))?; let flags = current_flags | libc::O_NONBLOCK; if flags != current_flags { diff --git a/compio-net/src/socket.rs b/compio-net/src/socket.rs index 1c4aecdd..45755771 100644 --- a/compio-net/src/socket.rs +++ b/compio-net/src/socket.rs @@ -90,7 +90,7 @@ impl Socket { )) { accept_sock.set_nonblocking(true)?; } - let accept_sock = Self::from_socket2(accept_sock); + let accept_sock = Self::from_socket2(accept_sock)?; let addr = op.into_addr(); Ok((accept_sock, addr)) } From f3dacfcc382930533dd47cbec96fed874c011cb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Mon, 29 Jan 2024 11:50:10 +0800 Subject: [PATCH 05/29] fix(fs,windows): attach named pipe server --- compio-fs/src/named_pipe.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/compio-fs/src/named_pipe.rs b/compio-fs/src/named_pipe.rs index d4a563ac..60407a74 100644 --- a/compio-fs/src/named_pipe.rs +++ b/compio-fs/src/named_pipe.rs @@ -301,7 +301,6 @@ impl NamedPipeClient { /// ``` pub fn info(&self) -> io::Result { // Safety: we're ensuring the lifetime of the named pipe. - // Safety: getting info doesn't need to be attached. unsafe { named_pipe_info(self.as_raw_fd()) } } } @@ -994,7 +993,9 @@ impl ServerOptions { ) )?; - Ok(unsafe { NamedPipeServer::from_raw_fd(h as _) }) + Ok(NamedPipeServer { + handle: File::new(unsafe { std::fs::File::from_raw_fd(h as _) })?, + }) } } From 769853a07db9c11a19b71564c2914d181f8c15c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Mon, 29 Jan 2024 11:59:49 +0800 Subject: [PATCH 06/29] test(fs): create server in runtime --- compio-fs/src/named_pipe.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/compio-fs/src/named_pipe.rs b/compio-fs/src/named_pipe.rs index 60407a74..94be7808 100644 --- a/compio-fs/src/named_pipe.rs +++ b/compio-fs/src/named_pipe.rs @@ -394,7 +394,9 @@ impl ServerOptions { /// /// const PIPE_NAME: &str = r"\\.\pipe\compio-named-pipe-new"; /// + /// # compio_runtime::Runtime::new().unwrap().block_on(async move { /// let server = ServerOptions::new().create(PIPE_NAME).unwrap(); + /// # }) /// ``` pub fn new() -> ServerOptions { ServerOptions { From 05e20f2435010ba478bdc18f8ca1a4e3c25c1451 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Wed, 31 Jan 2024 19:32:20 +0800 Subject: [PATCH 07/29] test(lib): use one stream in 2 threads --- compio/tests/runtime.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/compio/tests/runtime.rs b/compio/tests/runtime.rs index 70bd9f5c..965dfb84 100644 --- a/compio/tests/runtime.rs +++ b/compio/tests/runtime.rs @@ -22,11 +22,15 @@ async fn multi_threading() { futures_util::try_join!(TcpStream::connect(&addr), listener.accept()).unwrap(); tx.write_all(DATA).await.0.unwrap(); + tx.write_all(DATA).await.0.unwrap(); + + let (n, buffer) = rx.read_exact(Vec::with_capacity(DATA.len())).await.unwrap(); + assert_eq!(n, buffer.len()); + assert_eq!(DATA, String::from_utf8(buffer).unwrap()); if let Err(e) = std::thread::spawn(move || { compio::runtime::Runtime::new().unwrap().block_on(async { - let buffer = Vec::with_capacity(DATA.len()); - let ((), buffer) = rx.read_exact(buffer).await.unwrap(); + let ((), buffer) = rx.read_exact(Vec::with_capacity(DATA.len())).await.unwrap(); assert_eq!(DATA, String::from_utf8(buffer).unwrap()); }); }) From 67941244fa88b37cf814ae35bc6cfc6bbca07a94 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Wed, 31 Jan 2024 21:33:56 +0800 Subject: [PATCH 08/29] feat(driver,iocp): allow only one thread wait IOCP --- compio-driver/src/iocp/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index 21619c33..fb5449c3 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -164,7 +164,7 @@ struct CompletionPort { impl CompletionPort { pub fn new() -> io::Result { - let port = syscall!(BOOL, CreateIoCompletionPort(INVALID_HANDLE_VALUE, 0, 0, 0))?; + let port = syscall!(BOOL, CreateIoCompletionPort(INVALID_HANDLE_VALUE, 0, 0, 1))?; trace!("new iocp handle: {port}"); let port = unsafe { OwnedHandle::from_raw_handle(port as _) }; Ok(Self { From 883bdb5998e6a57a61f8e4f5d2d692d60a723150 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Wed, 31 Jan 2024 22:22:30 +0800 Subject: [PATCH 09/29] fix(driver,iocp): use Condvar --- compio-driver/Cargo.toml | 2 +- compio-driver/src/iocp/mod.rs | 104 +++++++++++++++++----------------- 2 files changed, 53 insertions(+), 53 deletions(-) diff --git a/compio-driver/Cargo.toml b/compio-driver/Cargo.toml index da33c1ff..dcbd3452 100644 --- a/compio-driver/Cargo.toml +++ b/compio-driver/Cargo.toml @@ -35,7 +35,6 @@ compio-log = { workspace = true } # Utils cfg-if = { workspace = true } crossbeam-channel = { workspace = true } -crossbeam-queue = { workspace = true } slab = { workspace = true } socket2 = { workspace = true } @@ -72,6 +71,7 @@ polling = "3.3.0" os_pipe = { workspace = true } [target.'cfg(unix)'.dependencies] +crossbeam-queue = { workspace = true } libc = { workspace = true } [dev-dependencies] diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index fb5449c3..dff123b7 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -1,7 +1,7 @@ #[cfg(feature = "once_cell_try")] use std::sync::OnceLock; use std::{ - collections::HashSet, + collections::{HashSet, VecDeque}, io, mem::ManuallyDrop, os::windows::prelude::{ @@ -9,10 +9,10 @@ use std::{ OwnedHandle, RawHandle, }, pin::Pin, - ptr::{null, NonNull}, + ptr::NonNull, sync::{ atomic::{AtomicUsize, Ordering}, - Arc, + Arc, Condvar, Mutex, MutexGuard, }, task::Poll, time::Duration, @@ -20,7 +20,6 @@ use std::{ use compio_buf::{arrayvec::ArrayVec, BufResult}; use compio_log::{instrument, trace}; -use crossbeam_queue::SegQueue; use crossbeam_skiplist::SkipMap; #[cfg(not(feature = "once_cell_try"))] use once_cell::sync::OnceCell as OnceLock; @@ -30,13 +29,12 @@ use windows_sys::Win32::{ RtlNtStatusToDosError, ERROR_BAD_COMMAND, ERROR_BUSY, ERROR_HANDLE_EOF, ERROR_IO_INCOMPLETE, ERROR_NO_DATA, ERROR_OPERATION_ABORTED, ERROR_TIMEOUT, FACILITY_NTWIN32, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, STATUS_SUCCESS, - WAIT_OBJECT_0, WAIT_TIMEOUT, }, Networking::WinSock::{WSACleanup, WSAStartup, WSADATA}, Storage::FileSystem::SetFileCompletionNotificationModes, System::{ SystemServices::ERROR_SEVERITY_ERROR, - Threading::{CreateEventW, SetEvent, WaitForSingleObject, INFINITE}, + Threading::INFINITE, WindowsProgramming::{FILE_SKIP_COMPLETION_PORT_ON_SUCCESS, FILE_SKIP_SET_EVENT_ON_HANDLE}, IO::{ CreateIoCompletionPort, GetQueuedCompletionStatusEx, PostQueuedCompletionStatus, @@ -120,46 +118,49 @@ impl IntoRawFd for socket2::Socket { } struct DriverEntry { - queue: SegQueue, - event: OwnedHandle, + queue: Mutex>, + event: Condvar, } impl DriverEntry { - pub fn new() -> io::Result { - let event = syscall!(BOOL, CreateEventW(null(), 0, 0, null()))?; - Ok(Self { - queue: SegQueue::new(), - event: unsafe { OwnedHandle::from_raw_handle(event as _) }, - }) + pub fn new(capacity: usize) -> Self { + Self { + queue: Mutex::new(VecDeque::with_capacity(capacity)), + event: Condvar::new(), + } } - pub fn push(&self, entry: Entry) -> io::Result<()> { - self.queue.push(entry); - syscall!(BOOL, SetEvent(self.event.as_raw_handle() as _))?; - Ok(()) + pub fn push(&self, entry: Entry) { + self.lock().push_back(entry); + self.event.notify_all(); } - pub fn pop(&self) -> Option { - self.queue.pop() + pub fn lock(&self) -> MutexGuard> { + self.queue.lock().unwrap() } - pub fn wait(&self, timeout: Option) -> io::Result<()> { - let timeout = match timeout { - Some(timeout) => timeout.as_millis() as u32, - None => INFINITE, - }; - let res = unsafe { WaitForSingleObject(self.event.as_raw_handle() as _, timeout) }; - match res { - WAIT_OBJECT_0 => Ok(()), - WAIT_TIMEOUT => Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)), - _ => Err(io::Error::last_os_error()), + pub fn wait(&self, timeout: Option) -> io::Result>> { + let guard = self.lock(); + if guard.is_empty() { + if let Some(timeout) = timeout { + let (guard, res) = self.event.wait_timeout(guard, timeout).unwrap(); + if res.timed_out() { + Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)) + } else { + Ok(guard) + } + } else { + Ok(self.event.wait(guard).unwrap()) + } + } else { + Ok(guard) } } } struct CompletionPort { port: OwnedHandle, - drivers: SkipMap, + drivers: SkipMap>, } impl CompletionPort { @@ -173,9 +174,9 @@ impl CompletionPort { }) } - pub fn register(&self, driver: usize) -> io::Result<()> { - self.drivers.insert(driver, DriverEntry::new()?); - Ok(()) + pub fn register(&self, driver: usize, capacity: usize) { + self.drivers + .insert(driver, Arc::new(DriverEntry::new(capacity))); } pub fn attach(&self, fd: RawFd) -> io::Result<()> { @@ -216,18 +217,20 @@ impl CompletionPort { Ok(()) } - pub fn entry(&self, driver: usize) -> crossbeam_skiplist::map::Entry { + pub fn entry(&self, driver: usize) -> Arc { self.drivers .get(&driver) .expect("driver should register first") + .value() + .clone() } - pub fn push(&self, driver: usize, entry: Entry) -> io::Result<()> { - self.entry(driver).value().push(entry) - } - - pub fn wait(&self, driver: usize, timeout: Option) -> io::Result<()> { - self.entry(driver).value().wait(timeout) + pub fn push(&self, driver: usize, entry: Entry) { + self.drivers + .get(&driver) + .expect("driver should register first") + .value() + .push(entry) } } @@ -289,7 +292,7 @@ fn iocp_start() -> io::Result<()> { } }; - port.push(overlapped.driver, Entry::new(overlapped.user_data, res))?; + port.push(overlapped.driver, Entry::new(overlapped.user_data, res)); } } #[allow(unreachable_code)] @@ -360,7 +363,7 @@ impl Driver { IOCP_INIT_ONCE.get_or_try_init(iocp_start)?; let id = DRIVER_COUNTER.fetch_add(1, Ordering::AcqRel); - iocp_port()?.register(id)?; + iocp_port()?.register(id, builder.capacity as _); Ok(Self { id, cancelled: HashSet::default(), @@ -460,15 +463,12 @@ impl Driver { let port = iocp_port()?; - port.wait(self.id, timeout)?; - - { - let driver_entry = port.entry(self.id); - let completed_entries = driver_entry.value(); - entries.extend( - std::iter::from_fn(|| completed_entries.pop()).filter_map(|e| self.create_entry(e)), - ); - } + let driver_entry = port.entry(self.id); + let mut completed_entries = driver_entry.wait(timeout)?; + entries.extend( + std::iter::from_fn(|| completed_entries.pop_front()) + .filter_map(|e| self.create_entry(e)), + ); Ok(()) } From 1eec07fd02545a3b05fecae8e55f924d6ee64870 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Wed, 31 Jan 2024 23:38:04 +0800 Subject: [PATCH 10/29] fix(driver,iocp): reduce clone --- compio-driver/src/iocp/mod.rs | 62 ++++++++++++++--------------------- 1 file changed, 25 insertions(+), 37 deletions(-) diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index dff123b7..e902bcbb 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -131,16 +131,12 @@ impl DriverEntry { } pub fn push(&self, entry: Entry) { - self.lock().push_back(entry); + self.queue.lock().unwrap().push_back(entry); self.event.notify_all(); } - pub fn lock(&self) -> MutexGuard> { - self.queue.lock().unwrap() - } - pub fn wait(&self, timeout: Option) -> io::Result>> { - let guard = self.lock(); + let guard = self.queue.lock().unwrap(); if guard.is_empty() { if let Some(timeout) = timeout { let (guard, res) = self.event.wait_timeout(guard, timeout).unwrap(); @@ -174,9 +170,10 @@ impl CompletionPort { }) } - pub fn register(&self, driver: usize, capacity: usize) { - self.drivers - .insert(driver, Arc::new(DriverEntry::new(capacity))); + pub fn register(&self, driver: usize, capacity: usize) -> Arc { + let driver_entry = Arc::new(DriverEntry::new(capacity)); + self.drivers.insert(driver, driver_entry.clone()); + driver_entry } pub fn attach(&self, fd: RawFd) -> io::Result<()> { @@ -217,14 +214,6 @@ impl CompletionPort { Ok(()) } - pub fn entry(&self, driver: usize) -> Arc { - self.drivers - .get(&driver) - .expect("driver should register first") - .value() - .clone() - } - pub fn push(&self, driver: usize, entry: Entry) { self.drivers .get(&driver) @@ -347,6 +336,7 @@ static IOCP_INIT_ONCE: OnceLock<()> = OnceLock::new(); /// Low-level driver of IOCP. pub(crate) struct Driver { id: usize, + driver_entry: Arc, cancelled: HashSet, pool: AsyncifyPool, notify_overlapped: Arc>, @@ -363,9 +353,10 @@ impl Driver { IOCP_INIT_ONCE.get_or_try_init(iocp_start)?; let id = DRIVER_COUNTER.fetch_add(1, Ordering::AcqRel); - iocp_port()?.register(id, builder.capacity as _); + let driver_entry = iocp_port()?.register(id, builder.capacity as _); Ok(Self { id, + driver_entry, cancelled: HashSet::default(), pool: builder.create_or_get_thread_pool(), notify_overlapped: Arc::new(Overlapped::new(id, Self::NOTIFY, ())), @@ -376,20 +367,6 @@ impl Driver { RawOp::new(self.id, user_data, op) } - fn create_entry(&mut self, entry: Entry) -> Option { - let user_data = entry.user_data(); - if user_data != Self::NOTIFY { - let result = if self.cancelled.remove(&user_data) { - Err(io::Error::from_raw_os_error(ERROR_OPERATION_ABORTED as _)) - } else { - entry.into_result() - }; - Some(Entry::new(user_data, result)) - } else { - None - } - } - pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { iocp_port()?.attach(fd) } @@ -454,6 +431,20 @@ impl Driver { .is_ok()) } + fn create_entry(entry: Entry, cancelled: &mut HashSet) -> Option { + let user_data = entry.user_data(); + if user_data != Self::NOTIFY { + let result = if cancelled.remove(&user_data) { + Err(io::Error::from_raw_os_error(ERROR_OPERATION_ABORTED as _)) + } else { + entry.into_result() + }; + Some(Entry::new(user_data, result)) + } else { + None + } + } + pub unsafe fn poll( &mut self, timeout: Option, @@ -461,13 +452,10 @@ impl Driver { ) -> io::Result<()> { instrument!(compio_log::Level::TRACE, "poll", ?timeout); - let port = iocp_port()?; - - let driver_entry = port.entry(self.id); - let mut completed_entries = driver_entry.wait(timeout)?; + let mut completed_entries = self.driver_entry.wait(timeout)?; entries.extend( std::iter::from_fn(|| completed_entries.pop_front()) - .filter_map(|e| self.create_entry(e)), + .filter_map(|e| Self::create_entry(e, &mut self.cancelled)), ); Ok(()) From acc88f89e78e78a7db69ff5f3e282c300fb9bf9e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Wed, 31 Jan 2024 23:54:32 +0800 Subject: [PATCH 11/29] fix(driver,iocp): use channel instead of handwriting queue --- compio-driver/src/iocp/mod.rs | 97 +++++++++++++++-------------------- 1 file changed, 42 insertions(+), 55 deletions(-) diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index e902bcbb..cc4aa8f7 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -1,7 +1,7 @@ #[cfg(feature = "once_cell_try")] use std::sync::OnceLock; use std::{ - collections::{HashSet, VecDeque}, + collections::HashSet, io, mem::ManuallyDrop, os::windows::prelude::{ @@ -12,7 +12,7 @@ use std::{ ptr::NonNull, sync::{ atomic::{AtomicUsize, Ordering}, - Arc, Condvar, Mutex, MutexGuard, + Arc, }, task::Poll, time::Duration, @@ -20,6 +20,7 @@ use std::{ use compio_buf::{arrayvec::ArrayVec, BufResult}; use compio_log::{instrument, trace}; +use crossbeam_channel::{unbounded, Receiver, RecvTimeoutError, Sender, TryRecvError}; use crossbeam_skiplist::SkipMap; #[cfg(not(feature = "once_cell_try"))] use once_cell::sync::OnceCell as OnceLock; @@ -117,46 +118,9 @@ impl IntoRawFd for socket2::Socket { } } -struct DriverEntry { - queue: Mutex>, - event: Condvar, -} - -impl DriverEntry { - pub fn new(capacity: usize) -> Self { - Self { - queue: Mutex::new(VecDeque::with_capacity(capacity)), - event: Condvar::new(), - } - } - - pub fn push(&self, entry: Entry) { - self.queue.lock().unwrap().push_back(entry); - self.event.notify_all(); - } - - pub fn wait(&self, timeout: Option) -> io::Result>> { - let guard = self.queue.lock().unwrap(); - if guard.is_empty() { - if let Some(timeout) = timeout { - let (guard, res) = self.event.wait_timeout(guard, timeout).unwrap(); - if res.timed_out() { - Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)) - } else { - Ok(guard) - } - } else { - Ok(self.event.wait(guard).unwrap()) - } - } else { - Ok(guard) - } - } -} - struct CompletionPort { port: OwnedHandle, - drivers: SkipMap>, + drivers: SkipMap>, } impl CompletionPort { @@ -170,10 +134,10 @@ impl CompletionPort { }) } - pub fn register(&self, driver: usize, capacity: usize) -> Arc { - let driver_entry = Arc::new(DriverEntry::new(capacity)); - self.drivers.insert(driver, driver_entry.clone()); - driver_entry + pub fn register(&self, driver: usize) -> Receiver { + let (sender, receiver) = unbounded(); + self.drivers.insert(driver, sender); + receiver } pub fn attach(&self, fd: RawFd) -> io::Result<()> { @@ -219,7 +183,8 @@ impl CompletionPort { .get(&driver) .expect("driver should register first") .value() - .push(entry) + .send(entry) + .ok(); // It's OK if the driver has been dropped. } } @@ -336,7 +301,7 @@ static IOCP_INIT_ONCE: OnceLock<()> = OnceLock::new(); /// Low-level driver of IOCP. pub(crate) struct Driver { id: usize, - driver_entry: Arc, + receiver: Receiver, cancelled: HashSet, pool: AsyncifyPool, notify_overlapped: Arc>, @@ -353,10 +318,10 @@ impl Driver { IOCP_INIT_ONCE.get_or_try_init(iocp_start)?; let id = DRIVER_COUNTER.fetch_add(1, Ordering::AcqRel); - let driver_entry = iocp_port()?.register(id, builder.capacity as _); + let receiver = iocp_port()?.register(id); Ok(Self { id, - driver_entry, + receiver, cancelled: HashSet::default(), pool: builder.create_or_get_thread_pool(), notify_overlapped: Arc::new(Overlapped::new(id, Self::NOTIFY, ())), @@ -431,10 +396,10 @@ impl Driver { .is_ok()) } - fn create_entry(entry: Entry, cancelled: &mut HashSet) -> Option { + fn create_entry(&mut self, entry: Entry) -> Option { let user_data = entry.user_data(); if user_data != Self::NOTIFY { - let result = if cancelled.remove(&user_data) { + let result = if self.cancelled.remove(&user_data) { Err(io::Error::from_raw_os_error(ERROR_OPERATION_ABORTED as _)) } else { entry.into_result() @@ -452,11 +417,33 @@ impl Driver { ) -> io::Result<()> { instrument!(compio_log::Level::TRACE, "poll", ?timeout); - let mut completed_entries = self.driver_entry.wait(timeout)?; - entries.extend( - std::iter::from_fn(|| completed_entries.pop_front()) - .filter_map(|e| Self::create_entry(e, &mut self.cancelled)), - ); + let e = if let Some(timeout) = timeout { + match self.receiver.recv_timeout(timeout) { + Ok(e) => e, + Err(e) => match e { + RecvTimeoutError::Timeout => { + return Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)); + } + RecvTimeoutError::Disconnected => { + unreachable!("IOCP thread should not exit") + } + }, + } + } else { + self.receiver.recv().expect("IOCP thread should not exit") + }; + entries.extend(self.create_entry(e)); + + // Query if there are more entries. + loop { + match self.receiver.try_recv() { + Ok(e) => entries.extend(self.create_entry(e)), + Err(e) => match e { + TryRecvError::Empty => break, + TryRecvError::Disconnected => unreachable!("IOCP thread should not exit"), + }, + } + } Ok(()) } From be7cedcde16c61fb2a3abc95fab809e34496975f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sat, 3 Feb 2024 22:20:58 +0800 Subject: [PATCH 12/29] feat(driver,iocp): add iocp-global feature --- compio-driver/Cargo.toml | 5 +- compio-driver/src/iocp/cp/global.rs | 162 ++++++++++++++++++ compio-driver/src/iocp/cp/mod.rs | 155 +++++++++++++++++ compio-driver/src/iocp/cp/multi.rs | 57 +++++++ compio-driver/src/iocp/mod.rs | 248 ++++------------------------ 5 files changed, 407 insertions(+), 220 deletions(-) create mode 100644 compio-driver/src/iocp/cp/global.rs create mode 100644 compio-driver/src/iocp/cp/mod.rs create mode 100644 compio-driver/src/iocp/cp/multi.rs diff --git a/compio-driver/Cargo.toml b/compio-driver/Cargo.toml index dcbd3452..4acf6b84 100644 --- a/compio-driver/Cargo.toml +++ b/compio-driver/Cargo.toml @@ -42,7 +42,7 @@ socket2 = { workspace = true } [target.'cfg(windows)'.dependencies] compio-buf = { workspace = true, features = ["arrayvec"] } aligned-array = "1.0.1" -crossbeam-skiplist = { workspace = true } +crossbeam-skiplist = { workspace = true, optional = true } once_cell = { workspace = true } widestring = { workspace = true } windows-sys = { workspace = true, features = [ @@ -71,6 +71,7 @@ polling = "3.3.0" os_pipe = { workspace = true } [target.'cfg(unix)'.dependencies] +crossbeam-channel = { workspace = true } crossbeam-queue = { workspace = true } libc = { workspace = true } @@ -84,6 +85,8 @@ polling = ["dep:polling", "dep:os_pipe"] io-uring-sqe128 = [] io-uring-cqe32 = [] +iocp-global = ["dep:crossbeam-skiplist"] + # Nightly features once_cell_try = [] nightly = ["once_cell_try"] diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs new file mode 100644 index 00000000..46863d2f --- /dev/null +++ b/compio-driver/src/iocp/cp/global.rs @@ -0,0 +1,162 @@ +#[cfg(feature = "once_cell_try")] +use std::sync::OnceLock; +use std::{ + io, + os::windows::io::{AsRawHandle, RawHandle}, + sync::atomic::{AtomicUsize, Ordering}, + time::Duration, +}; + +use crossbeam_channel::{unbounded, Receiver, RecvTimeoutError, Sender, TryRecvError}; +use crossbeam_skiplist::SkipMap; +#[cfg(not(feature = "once_cell_try"))] +use once_cell::sync::OnceCell as OnceLock; +use windows_sys::Win32::Foundation::ERROR_TIMEOUT; + +use super::CompletionPort; +use crate::{Entry, Overlapped, RawFd}; + +struct GlobalPort { + port: CompletionPort, + drivers: SkipMap>, +} + +impl GlobalPort { + pub fn new() -> io::Result { + Ok(Self { + port: CompletionPort::new()?, + drivers: SkipMap::new(), + }) + } + + pub fn register(&self, driver: usize) -> Receiver { + let (sender, receiver) = unbounded(); + self.drivers.insert(driver, sender); + receiver + } + + pub fn attach(&self, fd: RawFd) -> io::Result<()> { + self.port.attach(fd) + } + + pub fn post( + &self, + res: io::Result, + optr: *mut Overlapped, + ) -> io::Result<()> { + self.port.post(res, optr) + } + + pub fn push(&self, driver: usize, entry: Entry) { + self.drivers + .get(&driver) + .expect("driver should register first") + .value() + .send(entry) + .ok(); // It's OK if the driver has been dropped. + } +} + +impl AsRawHandle for GlobalPort { + fn as_raw_handle(&self) -> RawHandle { + self.port.as_raw_handle() + } +} + +static IOCP_PORT: OnceLock = OnceLock::new(); + +#[inline] +fn iocp_port() -> io::Result<&'static GlobalPort> { + IOCP_PORT.get_or_try_init(GlobalPort::new) +} + +fn iocp_start() -> io::Result<()> { + let port = iocp_port()?; + std::thread::spawn(move || { + loop { + for (driver, entry) in port.port.poll(None)? { + port.push(driver, entry); + } + } + #[allow(unreachable_code)] + io::Result::Ok(()) + }); + Ok(()) +} + +static DRIVER_COUNTER: AtomicUsize = AtomicUsize::new(0); +static IOCP_INIT_ONCE: OnceLock<()> = OnceLock::new(); + +pub struct Port { + id: usize, + port: &'static GlobalPort, + receiver: Receiver, +} + +impl Port { + pub fn new() -> io::Result { + IOCP_INIT_ONCE.get_or_try_init(iocp_start)?; + + let id = DRIVER_COUNTER.fetch_add(1, Ordering::AcqRel); + let port = iocp_port()?; + let receiver = port.register(id); + Ok(Self { id, port, receiver }) + } + + pub fn id(&self) -> usize { + self.id + } + + pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { + self.port.attach(fd) + } + + pub fn handle(&self) -> PortHandle { + PortHandle::new(self.port) + } + + pub fn poll(&self, timeout: Option) -> io::Result + '_> { + let e = if let Some(timeout) = timeout { + match self.receiver.recv_timeout(timeout) { + Ok(e) => e, + Err(e) => match e { + RecvTimeoutError::Timeout => { + return Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)); + } + RecvTimeoutError::Disconnected => { + unreachable!("IOCP thread should not exit") + } + }, + } + } else { + self.receiver.recv().expect("IOCP thread should not exit") + }; + Ok(Some(e) + .into_iter() + .chain(std::iter::from_fn(|| match self.receiver.try_recv() { + Ok(e) => Some(e), + Err(e) => match e { + TryRecvError::Empty => None, + TryRecvError::Disconnected => unreachable!("IOCP thread should not exit"), + }, + }))) + } +} + +pub struct PortHandle { + port: &'static GlobalPort, +} + +impl PortHandle { + fn new(port: &'static GlobalPort) -> Self { + Self { port } + } + + pub fn post( + &self, + res: io::Result, + optr: *mut Overlapped, + ) -> io::Result<()> { + self.port.post(res, optr) + } +} diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs new file mode 100644 index 00000000..f2209525 --- /dev/null +++ b/compio-driver/src/iocp/cp/mod.rs @@ -0,0 +1,155 @@ +use std::{ + io, + os::windows::io::{AsRawHandle, FromRawHandle, OwnedHandle, RawHandle}, + time::Duration, +}; + +use compio_buf::arrayvec::ArrayVec; +use compio_log::*; +use windows_sys::Win32::{ + Foundation::{ + RtlNtStatusToDosError, ERROR_BAD_COMMAND, ERROR_HANDLE_EOF, ERROR_IO_INCOMPLETE, + ERROR_NO_DATA, FACILITY_NTWIN32, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, + STATUS_SUCCESS, + }, + Storage::FileSystem::SetFileCompletionNotificationModes, + System::{ + SystemServices::ERROR_SEVERITY_ERROR, + Threading::INFINITE, + WindowsProgramming::{FILE_SKIP_COMPLETION_PORT_ON_SUCCESS, FILE_SKIP_SET_EVENT_ON_HANDLE}, + IO::{ + CreateIoCompletionPort, GetQueuedCompletionStatusEx, PostQueuedCompletionStatus, + OVERLAPPED_ENTRY, + }, + }, +}; + +use crate::{syscall, Entry, Overlapped, RawFd}; + +cfg_if::cfg_if! { + if #[cfg(feature = "iocp-global")] { + mod global; + pub use global::*; + } else { + mod multi; + pub use multi::*; + } +} + +struct CompletionPort { + port: OwnedHandle, +} + +impl CompletionPort { + pub fn new() -> io::Result { + let port = syscall!(BOOL, CreateIoCompletionPort(INVALID_HANDLE_VALUE, 0, 0, 1))?; + trace!("new iocp handle: {port}"); + let port = unsafe { OwnedHandle::from_raw_handle(port as _) }; + Ok(Self { port }) + } + + pub fn attach(&self, fd: RawFd) -> io::Result<()> { + syscall!( + BOOL, + CreateIoCompletionPort(fd as _, self.port.as_raw_handle() as _, 0, 0) + )?; + syscall!( + BOOL, + SetFileCompletionNotificationModes( + fd as _, + (FILE_SKIP_COMPLETION_PORT_ON_SUCCESS | FILE_SKIP_SET_EVENT_ON_HANDLE) as _ + ) + )?; + Ok(()) + } + + pub fn post( + &self, + res: io::Result, + optr: *mut Overlapped, + ) -> io::Result<()> { + if let Err(e) = &res { + let code = e.raw_os_error().unwrap_or(ERROR_BAD_COMMAND as _); + unsafe { &mut *optr }.base.Internal = ntstatus_from_win32(code) as _; + } + // We have to use CompletionKey to transfer the result because it is large + // enough. It is OK because we set it to zero when attaching handles to IOCP. + syscall!( + BOOL, + PostQueuedCompletionStatus( + self.port.as_raw_handle() as _, + 0, + res.unwrap_or_default(), + optr.cast() + ) + )?; + Ok(()) + } + + pub fn poll( + &self, + timeout: Option, + ) -> io::Result> { + const DEFAULT_CAPACITY: usize = 1024; + + let mut entries = ArrayVec::::new(); + let mut recv_count = 0; + let timeout = match timeout { + Some(timeout) => timeout.as_millis() as u32, + None => INFINITE, + }; + syscall!( + BOOL, + GetQueuedCompletionStatusEx( + self.port.as_raw_handle() as _, + entries.as_mut_ptr(), + DEFAULT_CAPACITY as _, + &mut recv_count, + timeout, + 0 + ) + )?; + trace!("recv_count: {recv_count}"); + unsafe { entries.set_len(recv_count as _) }; + + Ok(entries.into_iter().map(|entry| { + let transferred = entry.dwNumberOfBytesTransferred; + trace!("entry transferred: {transferred}"); + // Any thin pointer is OK because we don't use the type of opcode. + let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); + let overlapped = unsafe { &*overlapped_ptr }; + let res = if matches!( + overlapped.base.Internal as NTSTATUS, + STATUS_SUCCESS | STATUS_PENDING + ) { + if entry.lpCompletionKey != 0 { + Ok(entry.lpCompletionKey) + } else { + Ok(transferred as _) + } + } else { + let error = unsafe { RtlNtStatusToDosError(overlapped.base.Internal as _) }; + match error { + ERROR_IO_INCOMPLETE | ERROR_HANDLE_EOF | ERROR_NO_DATA => Ok(0), + _ => Err(io::Error::from_raw_os_error(error as _)), + } + }; + (overlapped.driver, Entry::new(overlapped.user_data, res)) + })) + } +} + +impl AsRawHandle for CompletionPort { + fn as_raw_handle(&self) -> RawHandle { + self.port.as_raw_handle() + } +} + +#[inline] +fn ntstatus_from_win32(x: i32) -> NTSTATUS { + if x <= 0 { + x + } else { + ((x) & 0x0000FFFF) | (FACILITY_NTWIN32 << 16) as NTSTATUS | ERROR_SEVERITY_ERROR as NTSTATUS + } +} diff --git a/compio-driver/src/iocp/cp/multi.rs b/compio-driver/src/iocp/cp/multi.rs new file mode 100644 index 00000000..2e0f4e5f --- /dev/null +++ b/compio-driver/src/iocp/cp/multi.rs @@ -0,0 +1,57 @@ +use std::{io, os::windows::io::AsRawHandle, sync::Arc, time::Duration}; + +use super::CompletionPort; +use crate::{Entry, Overlapped, RawFd}; + +pub struct Port { + port: Arc, +} + +impl Port { + pub fn new() -> io::Result { + Ok(Self { + port: Arc::new(CompletionPort::new()?), + }) + } + + pub fn id(&self) -> usize { + self.port.as_raw_handle() as _ + } + + pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { + self.port.attach(fd) + } + + pub fn handle(&self) -> PortHandle { + PortHandle::new(self.port.clone()) + } + + pub fn poll(&self, timeout: Option) -> io::Result + '_> { + let current_id = self.id(); + self.port.poll(timeout).map(move |it| { + it.filter_map( + move |(id, entry)| { + if id == current_id { Some(entry) } else { None } + }, + ) + }) + } +} + +pub struct PortHandle { + port: Arc, +} + +impl PortHandle { + fn new(port: Arc) -> Self { + Self { port } + } + + pub fn post( + &self, + res: io::Result, + optr: *mut Overlapped, + ) -> io::Result<()> { + self.port.post(res, optr) + } +} diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index cc4aa8f7..234fd161 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -1,53 +1,33 @@ -#[cfg(feature = "once_cell_try")] -use std::sync::OnceLock; use std::{ collections::HashSet, io, mem::ManuallyDrop, os::windows::prelude::{ AsRawHandle, AsRawSocket, FromRawHandle, FromRawSocket, IntoRawHandle, IntoRawSocket, - OwnedHandle, RawHandle, + RawHandle, }, pin::Pin, ptr::NonNull, - sync::{ - atomic::{AtomicUsize, Ordering}, - Arc, - }, + sync::Arc, task::Poll, time::Duration, }; -use compio_buf::{arrayvec::ArrayVec, BufResult}; +use compio_buf::BufResult; use compio_log::{instrument, trace}; -use crossbeam_channel::{unbounded, Receiver, RecvTimeoutError, Sender, TryRecvError}; -use crossbeam_skiplist::SkipMap; -#[cfg(not(feature = "once_cell_try"))] -use once_cell::sync::OnceCell as OnceLock; use slab::Slab; use windows_sys::Win32::{ - Foundation::{ - RtlNtStatusToDosError, ERROR_BAD_COMMAND, ERROR_BUSY, ERROR_HANDLE_EOF, - ERROR_IO_INCOMPLETE, ERROR_NO_DATA, ERROR_OPERATION_ABORTED, ERROR_TIMEOUT, - FACILITY_NTWIN32, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, STATUS_SUCCESS, - }, + Foundation::{ERROR_BUSY, ERROR_OPERATION_ABORTED}, Networking::WinSock::{WSACleanup, WSAStartup, WSADATA}, - Storage::FileSystem::SetFileCompletionNotificationModes, - System::{ - SystemServices::ERROR_SEVERITY_ERROR, - Threading::INFINITE, - WindowsProgramming::{FILE_SKIP_COMPLETION_PORT_ON_SUCCESS, FILE_SKIP_SET_EVENT_ON_HANDLE}, - IO::{ - CreateIoCompletionPort, GetQueuedCompletionStatusEx, PostQueuedCompletionStatus, - OVERLAPPED, OVERLAPPED_ENTRY, - }, - }, + System::IO::OVERLAPPED, }; use crate::{syscall, AsyncifyPool, Entry, OutEntries, ProactorBuilder}; pub(crate) mod op; +mod cp; + pub(crate) use windows_sys::Win32::Networking::WinSock::{ socklen_t, SOCKADDR_STORAGE as sockaddr_storage, }; @@ -118,143 +98,6 @@ impl IntoRawFd for socket2::Socket { } } -struct CompletionPort { - port: OwnedHandle, - drivers: SkipMap>, -} - -impl CompletionPort { - pub fn new() -> io::Result { - let port = syscall!(BOOL, CreateIoCompletionPort(INVALID_HANDLE_VALUE, 0, 0, 1))?; - trace!("new iocp handle: {port}"); - let port = unsafe { OwnedHandle::from_raw_handle(port as _) }; - Ok(Self { - port, - drivers: SkipMap::new(), - }) - } - - pub fn register(&self, driver: usize) -> Receiver { - let (sender, receiver) = unbounded(); - self.drivers.insert(driver, sender); - receiver - } - - pub fn attach(&self, fd: RawFd) -> io::Result<()> { - syscall!( - BOOL, - CreateIoCompletionPort(fd as _, self.port.as_raw_handle() as _, 0, 0) - )?; - syscall!( - BOOL, - SetFileCompletionNotificationModes( - fd as _, - (FILE_SKIP_COMPLETION_PORT_ON_SUCCESS | FILE_SKIP_SET_EVENT_ON_HANDLE) as _ - ) - )?; - Ok(()) - } - - pub fn post( - &self, - res: io::Result, - optr: *mut Overlapped, - ) -> io::Result<()> { - if let Err(e) = &res { - let code = e.raw_os_error().unwrap_or(ERROR_BAD_COMMAND as _); - unsafe { &mut *optr }.base.Internal = ntstatus_from_win32(code) as _; - } - // We have to use CompletionKey to transfer the result because it is large - // enough. It is OK because we set it to zero when attaching handles to IOCP. - syscall!( - BOOL, - PostQueuedCompletionStatus( - self.port.as_raw_handle() as _, - 0, - res.unwrap_or_default(), - optr.cast() - ) - )?; - Ok(()) - } - - pub fn push(&self, driver: usize, entry: Entry) { - self.drivers - .get(&driver) - .expect("driver should register first") - .value() - .send(entry) - .ok(); // It's OK if the driver has been dropped. - } -} - -impl AsRawHandle for CompletionPort { - fn as_raw_handle(&self) -> RawHandle { - self.port.as_raw_handle() - } -} - -static IOCP_PORT: OnceLock = OnceLock::new(); - -#[inline] -fn iocp_port() -> io::Result<&'static CompletionPort> { - IOCP_PORT.get_or_try_init(CompletionPort::new) -} - -fn iocp_start() -> io::Result<()> { - const DEFAULT_CAPACITY: usize = 1024; - - let port = iocp_port()?; - std::thread::spawn(move || { - let mut entries = ArrayVec::::new(); - loop { - let mut recv_count = 0; - syscall!( - BOOL, - GetQueuedCompletionStatusEx( - port.as_raw_handle() as _, - entries.as_mut_ptr(), - DEFAULT_CAPACITY as _, - &mut recv_count, - INFINITE, - 0 - ) - )?; - trace!("recv_count: {recv_count}"); - unsafe { entries.set_len(recv_count as _) }; - - for entry in entries.drain(..) { - let transferred = entry.dwNumberOfBytesTransferred; - trace!("entry transferred: {transferred}"); - // Any thin pointer is OK because we don't use the type of opcode. - let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); - let overlapped = unsafe { &*overlapped_ptr }; - let res = if matches!( - overlapped.base.Internal as NTSTATUS, - STATUS_SUCCESS | STATUS_PENDING - ) { - if entry.lpCompletionKey != 0 { - Ok(entry.lpCompletionKey) - } else { - Ok(transferred as _) - } - } else { - let error = unsafe { RtlNtStatusToDosError(overlapped.base.Internal as _) }; - match error { - ERROR_IO_INCOMPLETE | ERROR_HANDLE_EOF | ERROR_NO_DATA => Ok(0), - _ => Err(io::Error::from_raw_os_error(error as _)), - } - }; - - port.push(overlapped.driver, Entry::new(overlapped.user_data, res)); - } - } - #[allow(unreachable_code)] - io::Result::Ok(()) - }); - Ok(()) -} - /// Abstraction of IOCP operations. pub trait OpCode { /// Determines that the operation is really overlapped defined by Windows @@ -287,21 +130,9 @@ pub trait OpCode { } } -fn ntstatus_from_win32(x: i32) -> NTSTATUS { - if x <= 0 { - x - } else { - ((x) & 0x0000FFFF) | (FACILITY_NTWIN32 << 16) as NTSTATUS | ERROR_SEVERITY_ERROR as NTSTATUS - } -} - -static DRIVER_COUNTER: AtomicUsize = AtomicUsize::new(0); -static IOCP_INIT_ONCE: OnceLock<()> = OnceLock::new(); - /// Low-level driver of IOCP. pub(crate) struct Driver { - id: usize, - receiver: Receiver, + port: cp::Port, cancelled: HashSet, pool: AsyncifyPool, notify_overlapped: Arc>, @@ -315,13 +146,10 @@ impl Driver { let mut data: WSADATA = unsafe { std::mem::zeroed() }; syscall!(SOCKET, WSAStartup(0x202, &mut data))?; - IOCP_INIT_ONCE.get_or_try_init(iocp_start)?; - - let id = DRIVER_COUNTER.fetch_add(1, Ordering::AcqRel); - let receiver = iocp_port()?.register(id); + let port = cp::Port::new()?; + let id = port.id(); Ok(Self { - id, - receiver, + port, cancelled: HashSet::default(), pool: builder.create_or_get_thread_pool(), notify_overlapped: Arc::new(Overlapped::new(id, Self::NOTIFY, ())), @@ -329,11 +157,11 @@ impl Driver { } pub fn create_op(&self, user_data: usize, op: T) -> RawOp { - RawOp::new(self.id, user_data, op) + RawOp::new(self.port.id(), user_data, op) } pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { - iocp_port()?.attach(fd) + self.port.attach(fd) } pub fn cancel(&mut self, user_data: usize, registry: &mut Slab) { @@ -376,7 +204,7 @@ impl Driver { unsafe impl Send for SendWrapper {} let optr = SendWrapper(NonNull::from(op)); - let port = iocp_port()?; + let port = self.port.handle(); Ok(self .pool .dispatch(move || { @@ -396,10 +224,10 @@ impl Driver { .is_ok()) } - fn create_entry(&mut self, entry: Entry) -> Option { + fn create_entry(cancelled: &mut HashSet, entry: Entry) -> Option { let user_data = entry.user_data(); if user_data != Self::NOTIFY { - let result = if self.cancelled.remove(&user_data) { + let result = if cancelled.remove(&user_data) { Err(io::Error::from_raw_os_error(ERROR_OPERATION_ABORTED as _)) } else { entry.into_result() @@ -417,39 +245,20 @@ impl Driver { ) -> io::Result<()> { instrument!(compio_log::Level::TRACE, "poll", ?timeout); - let e = if let Some(timeout) = timeout { - match self.receiver.recv_timeout(timeout) { - Ok(e) => e, - Err(e) => match e { - RecvTimeoutError::Timeout => { - return Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)); - } - RecvTimeoutError::Disconnected => { - unreachable!("IOCP thread should not exit") - } - }, - } - } else { - self.receiver.recv().expect("IOCP thread should not exit") - }; - entries.extend(self.create_entry(e)); - - // Query if there are more entries. - loop { - match self.receiver.try_recv() { - Ok(e) => entries.extend(self.create_entry(e)), - Err(e) => match e { - TryRecvError::Empty => break, - TryRecvError::Disconnected => unreachable!("IOCP thread should not exit"), - }, - } - } + entries.extend( + self.port + .poll(timeout)? + .filter_map(|e| Self::create_entry(&mut self.cancelled, e)), + ); Ok(()) } pub fn handle(&self) -> io::Result { - Ok(NotifyHandle::new(self.notify_overlapped.clone())) + Ok(NotifyHandle::new( + self.port.handle(), + self.notify_overlapped.clone(), + )) } } @@ -461,17 +270,18 @@ impl Drop for Driver { /// A notify handle to the inner driver. pub struct NotifyHandle { + port: cp::PortHandle, overlapped: Arc>, } impl NotifyHandle { - fn new(overlapped: Arc>) -> Self { - Self { overlapped } + fn new(port: cp::PortHandle, overlapped: Arc>) -> Self { + Self { port, overlapped } } /// Notify the inner driver. pub fn notify(&self) -> io::Result<()> { - iocp_port()?.post( + self.port.post( Ok(0), self.overlapped.as_ref() as *const _ as *mut Overlapped<()> as _, ) From 14231870d7d40c77d243a03e4f90663c2fd518f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sat, 3 Feb 2024 22:23:13 +0800 Subject: [PATCH 13/29] feat(lib): add iocp-global to monocrate --- compio/Cargo.toml | 1 + compio/tests/runtime.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/compio/Cargo.toml b/compio/Cargo.toml index 9c5851ef..f59c849c 100644 --- a/compio/Cargo.toml +++ b/compio/Cargo.toml @@ -73,6 +73,7 @@ libc = { workspace = true } default = ["runtime", "io-uring"] io-uring = ["compio-driver/io-uring"] polling = ["compio-driver/polling"] +iocp-global = ["compio-driver/iocp-global"] io = ["dep:compio-io"] io-compat = ["io", "compio-io/compat"] runtime = ["dep:compio-runtime", "dep:compio-fs", "dep:compio-net", "io"] diff --git a/compio/tests/runtime.rs b/compio/tests/runtime.rs index 965dfb84..dda8a4a1 100644 --- a/compio/tests/runtime.rs +++ b/compio/tests/runtime.rs @@ -12,6 +12,7 @@ use compio_runtime::TryClone; use tempfile::NamedTempFile; #[compio_macros::test] +#[cfg(any(not(windows), feature = "iocp-global"))] async fn multi_threading() { const DATA: &str = "Hello world!"; From be6b299d770c40550d5a08cee8d4a909083ffc01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sat, 3 Feb 2024 22:25:41 +0800 Subject: [PATCH 14/29] ci: add iocp-global to windows target --- .github/workflows/ci_test.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/ci_test.yml b/.github/workflows/ci_test.yml index 579fad74..31ac965c 100644 --- a/.github/workflows/ci_test.yml +++ b/.github/workflows/ci_test.yml @@ -43,6 +43,9 @@ jobs: target: "x86_64-pc-windows-gnu" - os: "windows-latest" target: "i686-pc-windows-msvc" + - os: "windows-latest" + target: "x86_64-pc-windows-msvc" + features: "iocp-global" - os: "macos-12" - os: "macos-13" - os: "macos-14" From ec8b7a6e8880f8977cb60b9e7543bcc7586698ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sat, 3 Feb 2024 22:52:57 +0800 Subject: [PATCH 15/29] fix(driver): repost entry from other IOCP --- compio-driver/src/iocp/cp/global.rs | 2 +- compio-driver/src/iocp/cp/mod.rs | 19 ++++++++++++++++++- compio-driver/src/iocp/cp/multi.rs | 2 +- compio/Cargo.toml | 1 - compio/tests/runtime.rs | 10 +++------- 5 files changed, 23 insertions(+), 11 deletions(-) diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs index 46863d2f..0c1895fa 100644 --- a/compio-driver/src/iocp/cp/global.rs +++ b/compio-driver/src/iocp/cp/global.rs @@ -74,7 +74,7 @@ fn iocp_start() -> io::Result<()> { let port = iocp_port()?; std::thread::spawn(move || { loop { - for (driver, entry) in port.port.poll(None)? { + for (driver, entry) in port.port.poll(None, None)? { port.push(driver, entry); } } diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs index f2209525..54cb12f7 100644 --- a/compio-driver/src/iocp/cp/mod.rs +++ b/compio-driver/src/iocp/cp/mod.rs @@ -86,9 +86,12 @@ impl CompletionPort { Ok(()) } + // If current_driver is specified, any entry that doesn't belong the driver will + // be reposted. The driver id will be used as IOCP handle. pub fn poll( &self, timeout: Option, + current_driver: Option, ) -> io::Result> { const DEFAULT_CAPACITY: usize = 1024; @@ -112,12 +115,26 @@ impl CompletionPort { trace!("recv_count: {recv_count}"); unsafe { entries.set_len(recv_count as _) }; - Ok(entries.into_iter().map(|entry| { + Ok(entries.into_iter().map(move |entry| { let transferred = entry.dwNumberOfBytesTransferred; trace!("entry transferred: {transferred}"); // Any thin pointer is OK because we don't use the type of opcode. let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); let overlapped = unsafe { &*overlapped_ptr }; + if let Some(current_driver) = current_driver { + if overlapped.driver != current_driver { + syscall!( + BOOL, + PostQueuedCompletionStatus( + overlapped.driver as _, + entry.dwNumberOfBytesTransferred, + entry.lpCompletionKey, + entry.lpOverlapped + ) + ) + .ok(); + } + } let res = if matches!( overlapped.base.Internal as NTSTATUS, STATUS_SUCCESS | STATUS_PENDING diff --git a/compio-driver/src/iocp/cp/multi.rs b/compio-driver/src/iocp/cp/multi.rs index 2e0f4e5f..b83db2f9 100644 --- a/compio-driver/src/iocp/cp/multi.rs +++ b/compio-driver/src/iocp/cp/multi.rs @@ -28,7 +28,7 @@ impl Port { pub fn poll(&self, timeout: Option) -> io::Result + '_> { let current_id = self.id(); - self.port.poll(timeout).map(move |it| { + self.port.poll(timeout, Some(current_id)).map(move |it| { it.filter_map( move |(id, entry)| { if id == current_id { Some(entry) } else { None } diff --git a/compio/Cargo.toml b/compio/Cargo.toml index f59c849c..9c5851ef 100644 --- a/compio/Cargo.toml +++ b/compio/Cargo.toml @@ -73,7 +73,6 @@ libc = { workspace = true } default = ["runtime", "io-uring"] io-uring = ["compio-driver/io-uring"] polling = ["compio-driver/polling"] -iocp-global = ["compio-driver/iocp-global"] io = ["dep:compio-io"] io-compat = ["io", "compio-io/compat"] runtime = ["dep:compio-runtime", "dep:compio-fs", "dep:compio-net", "io"] diff --git a/compio/tests/runtime.rs b/compio/tests/runtime.rs index dda8a4a1..852e61ad 100644 --- a/compio/tests/runtime.rs +++ b/compio/tests/runtime.rs @@ -7,12 +7,11 @@ use compio::{ fs::File, io::{AsyncReadAt, AsyncReadExt, AsyncWriteAt, AsyncWriteExt}, net::{TcpListener, TcpStream}, + runtime::TryClone, }; -use compio_runtime::TryClone; use tempfile::NamedTempFile; #[compio_macros::test] -#[cfg(any(not(windows), feature = "iocp-global"))] async fn multi_threading() { const DATA: &str = "Hello world!"; @@ -29,16 +28,13 @@ async fn multi_threading() { assert_eq!(n, buffer.len()); assert_eq!(DATA, String::from_utf8(buffer).unwrap()); - if let Err(e) = std::thread::spawn(move || { + compio::runtime::spawn_blocking(move || { compio::runtime::Runtime::new().unwrap().block_on(async { let ((), buffer) = rx.read_exact(Vec::with_capacity(DATA.len())).await.unwrap(); assert_eq!(DATA, String::from_utf8(buffer).unwrap()); }); }) - .join() - { - std::panic::resume_unwind(e) - } + .await } #[compio_macros::test] From 89bdb863b5536c6a578d06527d6c995a9548643d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sat, 3 Feb 2024 23:14:35 +0800 Subject: [PATCH 16/29] feat(driver): expose PortId instead of usize --- compio-driver/src/iocp/cp/global.rs | 28 +++++++++++++++++++++++----- compio-driver/src/iocp/cp/mod.rs | 16 +++++++--------- compio-driver/src/iocp/cp/multi.rs | 26 +++++++++++++++++++++++--- compio-driver/src/iocp/mod.rs | 6 +++--- 4 files changed, 56 insertions(+), 20 deletions(-) diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs index 0c1895fa..6f255a61 100644 --- a/compio-driver/src/iocp/cp/global.rs +++ b/compio-driver/src/iocp/cp/global.rs @@ -11,10 +11,13 @@ use crossbeam_channel::{unbounded, Receiver, RecvTimeoutError, Sender, TryRecvEr use crossbeam_skiplist::SkipMap; #[cfg(not(feature = "once_cell_try"))] use once_cell::sync::OnceCell as OnceLock; -use windows_sys::Win32::Foundation::ERROR_TIMEOUT; +use windows_sys::Win32::{ + Foundation::ERROR_TIMEOUT, + System::IO::{PostQueuedCompletionStatus, OVERLAPPED}, +}; use super::CompletionPort; -use crate::{Entry, Overlapped, RawFd}; +use crate::{syscall, Entry, Overlapped, RawFd}; struct GlobalPort { port: CompletionPort, @@ -75,7 +78,7 @@ fn iocp_start() -> io::Result<()> { std::thread::spawn(move || { loop { for (driver, entry) in port.port.poll(None, None)? { - port.push(driver, entry); + port.push(driver.0, entry); } } #[allow(unreachable_code)] @@ -103,8 +106,8 @@ impl Port { Ok(Self { id, port, receiver }) } - pub fn id(&self) -> usize { - self.id + pub fn id(&self) -> PortId { + PortId(self.id) } pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { @@ -160,3 +163,18 @@ impl PortHandle { self.port.post(res, optr) } } + +/// The unique ID of IOCP driver. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct PortId(usize); + +impl PortId { + /// Post raw entry to IOCP. + pub fn post_raw(&self, transferred: u32, key: usize, optr: *mut OVERLAPPED) -> io::Result<()> { + syscall!( + BOOL, + PostQueuedCompletionStatus(iocp_port()?.as_raw_handle() as _, transferred, key, optr) + )?; + Ok(()) + } +} diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs index 54cb12f7..ce3a50b9 100644 --- a/compio-driver/src/iocp/cp/mod.rs +++ b/compio-driver/src/iocp/cp/mod.rs @@ -91,8 +91,8 @@ impl CompletionPort { pub fn poll( &self, timeout: Option, - current_driver: Option, - ) -> io::Result> { + current_driver: Option, + ) -> io::Result> { const DEFAULT_CAPACITY: usize = 1024; let mut entries = ArrayVec::::new(); @@ -123,16 +123,14 @@ impl CompletionPort { let overlapped = unsafe { &*overlapped_ptr }; if let Some(current_driver) = current_driver { if overlapped.driver != current_driver { - syscall!( - BOOL, - PostQueuedCompletionStatus( - overlapped.driver as _, + overlapped + .driver + .post_raw( entry.dwNumberOfBytesTransferred, entry.lpCompletionKey, - entry.lpOverlapped + entry.lpOverlapped, ) - ) - .ok(); + .ok(); } } let res = if matches!( diff --git a/compio-driver/src/iocp/cp/multi.rs b/compio-driver/src/iocp/cp/multi.rs index b83db2f9..b37b528a 100644 --- a/compio-driver/src/iocp/cp/multi.rs +++ b/compio-driver/src/iocp/cp/multi.rs @@ -1,7 +1,12 @@ use std::{io, os::windows::io::AsRawHandle, sync::Arc, time::Duration}; +use windows_sys::Win32::{ + Foundation::HANDLE, + System::IO::{PostQueuedCompletionStatus, OVERLAPPED}, +}; + use super::CompletionPort; -use crate::{Entry, Overlapped, RawFd}; +use crate::{syscall, Entry, Overlapped, RawFd}; pub struct Port { port: Arc, @@ -14,8 +19,8 @@ impl Port { }) } - pub fn id(&self) -> usize { - self.port.as_raw_handle() as _ + pub fn id(&self) -> PortId { + PortId(self.port.as_raw_handle() as _) } pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { @@ -55,3 +60,18 @@ impl PortHandle { self.port.post(res, optr) } } + +/// The unique ID of IOCP driver. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct PortId(HANDLE); + +impl PortId { + /// Post raw entry to IOCP. + pub fn post_raw(&self, transferred: u32, key: usize, optr: *mut OVERLAPPED) -> io::Result<()> { + syscall!( + BOOL, + PostQueuedCompletionStatus(self.0, transferred, key, optr) + )?; + Ok(()) + } +} diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index 234fd161..5fb62bb3 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -294,7 +294,7 @@ pub struct Overlapped { /// The base [`OVERLAPPED`]. pub base: OVERLAPPED, /// The unique ID of created driver. - pub driver: usize, + pub driver: cp::PortId, /// The registered user defined data. pub user_data: usize, /// The opcode. @@ -303,7 +303,7 @@ pub struct Overlapped { } impl Overlapped { - pub(crate) fn new(driver: usize, user_data: usize, op: T) -> Self { + pub(crate) fn new(driver: cp::PortId, user_data: usize, op: T) -> Self { Self { base: unsafe { std::mem::zeroed() }, driver, @@ -326,7 +326,7 @@ pub(crate) struct RawOp { } impl RawOp { - pub(crate) fn new(driver: usize, user_data: usize, op: impl OpCode + 'static) -> Self { + pub(crate) fn new(driver: cp::PortId, user_data: usize, op: impl OpCode + 'static) -> Self { let op = Overlapped::new(driver, user_data, op); let op = Box::new(op) as Box>; Self { From 0038769d7609f2599a0706d6def4a2b674027b6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sat, 3 Feb 2024 23:52:50 +0800 Subject: [PATCH 17/29] test(net): try to find out why it blocks Remove the channel and wait for the spawned task directly. --- compio-net/Cargo.toml | 1 - compio-net/tests/tcp_accept.rs | 10 ++++------ compio-net/tests/tcp_connect.rs | 11 ++++------- 3 files changed, 8 insertions(+), 14 deletions(-) diff --git a/compio-net/Cargo.toml b/compio-net/Cargo.toml index d4396955..f88f7207 100644 --- a/compio-net/Cargo.toml +++ b/compio-net/Cargo.toml @@ -39,6 +39,5 @@ libc = { workspace = true } # Shared dev dependencies for all platforms [dev-dependencies] compio-macros = { workspace = true } -futures-channel = { workspace = true } futures-util = { workspace = true } tempfile = { workspace = true } diff --git a/compio-net/tests/tcp_accept.rs b/compio-net/tests/tcp_accept.rs index 2ca12ba5..8a44de0d 100644 --- a/compio-net/tests/tcp_accept.rs +++ b/compio-net/tests/tcp_accept.rs @@ -3,14 +3,12 @@ use compio_net::{TcpListener, TcpStream, ToSocketAddrsAsync}; async fn test_impl(addr: impl ToSocketAddrsAsync) { let listener = TcpListener::bind(addr).await.unwrap(); let addr = listener.local_addr().unwrap(); - let (tx, rx) = futures_channel::oneshot::channel(); - compio_runtime::spawn(async move { + let task = compio_runtime::spawn(async move { let (socket, _) = listener.accept().await.unwrap(); - assert!(tx.send(socket).is_ok()); - }) - .detach(); + socket + }); let cli = TcpStream::connect(&addr).await.unwrap(); - let srv = rx.await.unwrap(); + let srv = task.await; assert_eq!(cli.local_addr().unwrap(), srv.peer_addr().unwrap()); } diff --git a/compio-net/tests/tcp_connect.rs b/compio-net/tests/tcp_connect.rs index ab9bbd81..1057326c 100644 --- a/compio-net/tests/tcp_connect.rs +++ b/compio-net/tests/tcp_connect.rs @@ -10,17 +10,14 @@ async fn test_connect_ip_impl( let addr = listener.local_addr().unwrap(); assert!(assert_fn(&addr)); - let (tx, rx) = futures_channel::oneshot::channel(); - - compio_runtime::spawn(async move { + let task = compio_runtime::spawn(async move { let (socket, addr) = listener.accept().await.unwrap(); assert_eq!(addr, socket.peer_addr().unwrap()); - assert!(tx.send(socket).is_ok()); - }) - .detach(); + socket + }); let mine = TcpStream::connect(&addr).await.unwrap(); - let theirs = rx.await.unwrap(); + let theirs = task.await; assert_eq!(mine.local_addr().unwrap(), theirs.peer_addr().unwrap()); assert_eq!(theirs.local_addr().unwrap(), mine.peer_addr().unwrap()); From 312348dc4c186a22b03fbf728cc460a2b37d4497 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 00:13:29 +0800 Subject: [PATCH 18/29] feat(driver,iocp): deregister from global port on drop --- compio-driver/src/iocp/cp/global.rs | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs index 6f255a61..fd0b4dac 100644 --- a/compio-driver/src/iocp/cp/global.rs +++ b/compio-driver/src/iocp/cp/global.rs @@ -38,6 +38,10 @@ impl GlobalPort { receiver } + pub fn deregister(&self, driver: usize) { + self.drivers.remove(&driver); + } + pub fn attach(&self, fd: RawFd) -> io::Result<()> { self.port.attach(fd) } @@ -51,12 +55,9 @@ impl GlobalPort { } pub fn push(&self, driver: usize, entry: Entry) { - self.drivers - .get(&driver) - .expect("driver should register first") - .value() - .send(entry) - .ok(); // It's OK if the driver has been dropped. + if let Some(e) = self.drivers.get(&driver) { + e.value().send(entry).ok(); // It's OK if the driver has been dropped. + } } } @@ -146,6 +147,12 @@ impl Port { } } +impl Drop for Port { + fn drop(&mut self) { + self.port.deregister(self.id); + } +} + pub struct PortHandle { port: &'static GlobalPort, } From e9b7e6f1461c9a073e48ea6adb718e8640f96071 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 10:57:00 +0800 Subject: [PATCH 19/29] feat(driver,iocp): use IOCP instead of crossbeam channel --- compio-driver/Cargo.toml | 3 +- compio-driver/src/iocp/cp/global.rs | 110 +++++++--------------------- compio-driver/src/iocp/cp/mod.rs | 38 ++++++---- compio-driver/src/iocp/cp/multi.rs | 47 ++++-------- compio-driver/src/iocp/mod.rs | 14 ++-- 5 files changed, 72 insertions(+), 140 deletions(-) diff --git a/compio-driver/Cargo.toml b/compio-driver/Cargo.toml index 4acf6b84..c1fddf52 100644 --- a/compio-driver/Cargo.toml +++ b/compio-driver/Cargo.toml @@ -42,7 +42,6 @@ socket2 = { workspace = true } [target.'cfg(windows)'.dependencies] compio-buf = { workspace = true, features = ["arrayvec"] } aligned-array = "1.0.1" -crossbeam-skiplist = { workspace = true, optional = true } once_cell = { workspace = true } widestring = { workspace = true } windows-sys = { workspace = true, features = [ @@ -85,7 +84,7 @@ polling = ["dep:polling", "dep:os_pipe"] io-uring-sqe128 = [] io-uring-cqe32 = [] -iocp-global = ["dep:crossbeam-skiplist"] +iocp-global = [] # Nightly features once_cell_try = [] diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs index fd0b4dac..4d26fc92 100644 --- a/compio-driver/src/iocp/cp/global.rs +++ b/compio-driver/src/iocp/cp/global.rs @@ -3,45 +3,27 @@ use std::sync::OnceLock; use std::{ io, os::windows::io::{AsRawHandle, RawHandle}, - sync::atomic::{AtomicUsize, Ordering}, time::Duration, }; -use crossbeam_channel::{unbounded, Receiver, RecvTimeoutError, Sender, TryRecvError}; -use crossbeam_skiplist::SkipMap; #[cfg(not(feature = "once_cell_try"))] use once_cell::sync::OnceCell as OnceLock; -use windows_sys::Win32::{ - Foundation::ERROR_TIMEOUT, - System::IO::{PostQueuedCompletionStatus, OVERLAPPED}, -}; +use windows_sys::Win32::System::IO::PostQueuedCompletionStatus; use super::CompletionPort; use crate::{syscall, Entry, Overlapped, RawFd}; struct GlobalPort { port: CompletionPort, - drivers: SkipMap>, } impl GlobalPort { pub fn new() -> io::Result { Ok(Self { port: CompletionPort::new()?, - drivers: SkipMap::new(), }) } - pub fn register(&self, driver: usize) -> Receiver { - let (sender, receiver) = unbounded(); - self.drivers.insert(driver, sender); - receiver - } - - pub fn deregister(&self, driver: usize) { - self.drivers.remove(&driver); - } - pub fn attach(&self, fd: RawFd) -> io::Result<()> { self.port.attach(fd) } @@ -53,12 +35,6 @@ impl GlobalPort { ) -> io::Result<()> { self.port.post(res, optr) } - - pub fn push(&self, driver: usize, entry: Entry) { - if let Some(e) = self.drivers.get(&driver) { - e.value().send(entry).ok(); // It's OK if the driver has been dropped. - } - } } impl AsRawHandle for GlobalPort { @@ -78,8 +54,20 @@ fn iocp_start() -> io::Result<()> { let port = iocp_port()?; std::thread::spawn(move || { loop { - for (driver, entry) in port.port.poll(None, None)? { - port.push(driver.0, entry); + for entry in port.port.poll_raw(None)? { + // Any thin pointer is OK because we don't use the type of opcode. + let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); + let overlapped = unsafe { &*overlapped_ptr }; + syscall!( + BOOL, + PostQueuedCompletionStatus( + overlapped.driver, + entry.dwNumberOfBytesTransferred, + entry.lpCompletionKey, + entry.lpOverlapped, + ) + ) + .ok(); } } #[allow(unreachable_code)] @@ -88,68 +76,39 @@ fn iocp_start() -> io::Result<()> { Ok(()) } -static DRIVER_COUNTER: AtomicUsize = AtomicUsize::new(0); static IOCP_INIT_ONCE: OnceLock<()> = OnceLock::new(); pub struct Port { - id: usize, - port: &'static GlobalPort, - receiver: Receiver, + port: CompletionPort, + global_port: &'static GlobalPort, } impl Port { pub fn new() -> io::Result { IOCP_INIT_ONCE.get_or_try_init(iocp_start)?; - let id = DRIVER_COUNTER.fetch_add(1, Ordering::AcqRel); - let port = iocp_port()?; - let receiver = port.register(id); - Ok(Self { id, port, receiver }) - } - - pub fn id(&self) -> PortId { - PortId(self.id) + Ok(Self { + port: CompletionPort::new()?, + global_port: iocp_port()?, + }) } pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { - self.port.attach(fd) + self.global_port.attach(fd) } pub fn handle(&self) -> PortHandle { - PortHandle::new(self.port) + PortHandle::new(self.global_port) } pub fn poll(&self, timeout: Option) -> io::Result + '_> { - let e = if let Some(timeout) = timeout { - match self.receiver.recv_timeout(timeout) { - Ok(e) => e, - Err(e) => match e { - RecvTimeoutError::Timeout => { - return Err(io::Error::from_raw_os_error(ERROR_TIMEOUT as _)); - } - RecvTimeoutError::Disconnected => { - unreachable!("IOCP thread should not exit") - } - }, - } - } else { - self.receiver.recv().expect("IOCP thread should not exit") - }; - Ok(Some(e) - .into_iter() - .chain(std::iter::from_fn(|| match self.receiver.try_recv() { - Ok(e) => Some(e), - Err(e) => match e { - TryRecvError::Empty => None, - TryRecvError::Disconnected => unreachable!("IOCP thread should not exit"), - }, - }))) + self.port.poll(timeout, None) } } -impl Drop for Port { - fn drop(&mut self) { - self.port.deregister(self.id); +impl AsRawHandle for Port { + fn as_raw_handle(&self) -> RawHandle { + self.port.as_raw_handle() } } @@ -170,18 +129,3 @@ impl PortHandle { self.port.post(res, optr) } } - -/// The unique ID of IOCP driver. -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub struct PortId(usize); - -impl PortId { - /// Post raw entry to IOCP. - pub fn post_raw(&self, transferred: u32, key: usize, optr: *mut OVERLAPPED) -> io::Result<()> { - syscall!( - BOOL, - PostQueuedCompletionStatus(iocp_port()?.as_raw_handle() as _, transferred, key, optr) - )?; - Ok(()) - } -} diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs index ce3a50b9..68a968fc 100644 --- a/compio-driver/src/iocp/cp/mod.rs +++ b/compio-driver/src/iocp/cp/mod.rs @@ -9,7 +9,7 @@ use compio_log::*; use windows_sys::Win32::{ Foundation::{ RtlNtStatusToDosError, ERROR_BAD_COMMAND, ERROR_HANDLE_EOF, ERROR_IO_INCOMPLETE, - ERROR_NO_DATA, FACILITY_NTWIN32, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, + ERROR_NO_DATA, FACILITY_NTWIN32, HANDLE, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, STATUS_SUCCESS, }, Storage::FileSystem::SetFileCompletionNotificationModes, @@ -86,13 +86,10 @@ impl CompletionPort { Ok(()) } - // If current_driver is specified, any entry that doesn't belong the driver will - // be reposted. The driver id will be used as IOCP handle. - pub fn poll( + pub fn poll_raw( &self, timeout: Option, - current_driver: Option, - ) -> io::Result> { + ) -> io::Result> { const DEFAULT_CAPACITY: usize = 1024; let mut entries = ArrayVec::::new(); @@ -115,22 +112,33 @@ impl CompletionPort { trace!("recv_count: {recv_count}"); unsafe { entries.set_len(recv_count as _) }; - Ok(entries.into_iter().map(move |entry| { - let transferred = entry.dwNumberOfBytesTransferred; - trace!("entry transferred: {transferred}"); + Ok(entries.into_iter()) + } + + // If current_driver is specified, any entry that doesn't belong the driver will + // be reposted. The driver id will be used as IOCP handle. + pub fn poll( + &self, + timeout: Option, + current_driver: Option, + ) -> io::Result> { + Ok(self.poll_raw(timeout)?.map(move |entry| { // Any thin pointer is OK because we don't use the type of opcode. let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); let overlapped = unsafe { &*overlapped_ptr }; if let Some(current_driver) = current_driver { if overlapped.driver != current_driver { - overlapped - .driver - .post_raw( + // Repose the entry to correct port. + syscall!( + BOOL, + PostQueuedCompletionStatus( + overlapped.driver, entry.dwNumberOfBytesTransferred, entry.lpCompletionKey, entry.lpOverlapped, ) - .ok(); + ) + .ok(); } } let res = if matches!( @@ -140,7 +148,7 @@ impl CompletionPort { if entry.lpCompletionKey != 0 { Ok(entry.lpCompletionKey) } else { - Ok(transferred as _) + Ok(entry.dwNumberOfBytesTransferred as _) } } else { let error = unsafe { RtlNtStatusToDosError(overlapped.base.Internal as _) }; @@ -149,7 +157,7 @@ impl CompletionPort { _ => Err(io::Error::from_raw_os_error(error as _)), } }; - (overlapped.driver, Entry::new(overlapped.user_data, res)) + Entry::new(overlapped.user_data, res) })) } } diff --git a/compio-driver/src/iocp/cp/multi.rs b/compio-driver/src/iocp/cp/multi.rs index b37b528a..7bdc7eb8 100644 --- a/compio-driver/src/iocp/cp/multi.rs +++ b/compio-driver/src/iocp/cp/multi.rs @@ -1,12 +1,12 @@ -use std::{io, os::windows::io::AsRawHandle, sync::Arc, time::Duration}; - -use windows_sys::Win32::{ - Foundation::HANDLE, - System::IO::{PostQueuedCompletionStatus, OVERLAPPED}, +use std::{ + io, + os::windows::io::{AsRawHandle, RawHandle}, + sync::Arc, + time::Duration, }; use super::CompletionPort; -use crate::{syscall, Entry, Overlapped, RawFd}; +use crate::{Entry, Overlapped, RawFd}; pub struct Port { port: Arc, @@ -19,10 +19,6 @@ impl Port { }) } - pub fn id(&self) -> PortId { - PortId(self.port.as_raw_handle() as _) - } - pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { self.port.attach(fd) } @@ -32,14 +28,14 @@ impl Port { } pub fn poll(&self, timeout: Option) -> io::Result + '_> { - let current_id = self.id(); - self.port.poll(timeout, Some(current_id)).map(move |it| { - it.filter_map( - move |(id, entry)| { - if id == current_id { Some(entry) } else { None } - }, - ) - }) + let current_id = self.as_raw_handle() as _; + self.port.poll(timeout, Some(current_id)) + } +} + +impl AsRawHandle for Port { + fn as_raw_handle(&self) -> RawHandle { + self.port.as_raw_handle() } } @@ -60,18 +56,3 @@ impl PortHandle { self.port.post(res, optr) } } - -/// The unique ID of IOCP driver. -#[derive(Debug, Clone, Copy, PartialEq, Eq)] -pub struct PortId(HANDLE); - -impl PortId { - /// Post raw entry to IOCP. - pub fn post_raw(&self, transferred: u32, key: usize, optr: *mut OVERLAPPED) -> io::Result<()> { - syscall!( - BOOL, - PostQueuedCompletionStatus(self.0, transferred, key, optr) - )?; - Ok(()) - } -} diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index 5fb62bb3..350e7ee0 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -17,7 +17,7 @@ use compio_buf::BufResult; use compio_log::{instrument, trace}; use slab::Slab; use windows_sys::Win32::{ - Foundation::{ERROR_BUSY, ERROR_OPERATION_ABORTED}, + Foundation::{ERROR_BUSY, ERROR_OPERATION_ABORTED, HANDLE}, Networking::WinSock::{WSACleanup, WSAStartup, WSADATA}, System::IO::OVERLAPPED, }; @@ -147,17 +147,17 @@ impl Driver { syscall!(SOCKET, WSAStartup(0x202, &mut data))?; let port = cp::Port::new()?; - let id = port.id(); + let driver = port.as_raw_handle() as _; Ok(Self { port, cancelled: HashSet::default(), pool: builder.create_or_get_thread_pool(), - notify_overlapped: Arc::new(Overlapped::new(id, Self::NOTIFY, ())), + notify_overlapped: Arc::new(Overlapped::new(driver, Self::NOTIFY, ())), }) } pub fn create_op(&self, user_data: usize, op: T) -> RawOp { - RawOp::new(self.port.id(), user_data, op) + RawOp::new(self.port.as_raw_handle() as _, user_data, op) } pub fn attach(&mut self, fd: RawFd) -> io::Result<()> { @@ -294,7 +294,7 @@ pub struct Overlapped { /// The base [`OVERLAPPED`]. pub base: OVERLAPPED, /// The unique ID of created driver. - pub driver: cp::PortId, + pub driver: HANDLE, /// The registered user defined data. pub user_data: usize, /// The opcode. @@ -303,7 +303,7 @@ pub struct Overlapped { } impl Overlapped { - pub(crate) fn new(driver: cp::PortId, user_data: usize, op: T) -> Self { + pub(crate) fn new(driver: HANDLE, user_data: usize, op: T) -> Self { Self { base: unsafe { std::mem::zeroed() }, driver, @@ -326,7 +326,7 @@ pub(crate) struct RawOp { } impl RawOp { - pub(crate) fn new(driver: cp::PortId, user_data: usize, op: impl OpCode + 'static) -> Self { + pub(crate) fn new(driver: HANDLE, user_data: usize, op: impl OpCode + 'static) -> Self { let op = Overlapped::new(driver, user_data, op); let op = Box::new(op) as Box>; Self { From 6ee42aa5c64269f59bd4f1138dfeadbda7c75bf0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 11:06:05 +0800 Subject: [PATCH 20/29] feat(driver): use RawFd instead of HANDLE --- compio-driver/src/iocp/cp/global.rs | 2 +- compio-driver/src/iocp/cp/mod.rs | 6 +++--- compio-driver/src/iocp/cp/multi.rs | 2 +- compio-driver/src/iocp/mod.rs | 8 ++++---- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs index 4d26fc92..b2c488ba 100644 --- a/compio-driver/src/iocp/cp/global.rs +++ b/compio-driver/src/iocp/cp/global.rs @@ -61,7 +61,7 @@ fn iocp_start() -> io::Result<()> { syscall!( BOOL, PostQueuedCompletionStatus( - overlapped.driver, + overlapped.driver as _, entry.dwNumberOfBytesTransferred, entry.lpCompletionKey, entry.lpOverlapped, diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs index 68a968fc..331d9738 100644 --- a/compio-driver/src/iocp/cp/mod.rs +++ b/compio-driver/src/iocp/cp/mod.rs @@ -9,7 +9,7 @@ use compio_log::*; use windows_sys::Win32::{ Foundation::{ RtlNtStatusToDosError, ERROR_BAD_COMMAND, ERROR_HANDLE_EOF, ERROR_IO_INCOMPLETE, - ERROR_NO_DATA, FACILITY_NTWIN32, HANDLE, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, + ERROR_NO_DATA, FACILITY_NTWIN32, INVALID_HANDLE_VALUE, NTSTATUS, STATUS_PENDING, STATUS_SUCCESS, }, Storage::FileSystem::SetFileCompletionNotificationModes, @@ -120,7 +120,7 @@ impl CompletionPort { pub fn poll( &self, timeout: Option, - current_driver: Option, + current_driver: Option, ) -> io::Result> { Ok(self.poll_raw(timeout)?.map(move |entry| { // Any thin pointer is OK because we don't use the type of opcode. @@ -132,7 +132,7 @@ impl CompletionPort { syscall!( BOOL, PostQueuedCompletionStatus( - overlapped.driver, + overlapped.driver as _, entry.dwNumberOfBytesTransferred, entry.lpCompletionKey, entry.lpOverlapped, diff --git a/compio-driver/src/iocp/cp/multi.rs b/compio-driver/src/iocp/cp/multi.rs index 7bdc7eb8..07a820f8 100644 --- a/compio-driver/src/iocp/cp/multi.rs +++ b/compio-driver/src/iocp/cp/multi.rs @@ -28,7 +28,7 @@ impl Port { } pub fn poll(&self, timeout: Option) -> io::Result + '_> { - let current_id = self.as_raw_handle() as _; + let current_id = self.as_raw_handle(); self.port.poll(timeout, Some(current_id)) } } diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index 350e7ee0..a00006aa 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -17,7 +17,7 @@ use compio_buf::BufResult; use compio_log::{instrument, trace}; use slab::Slab; use windows_sys::Win32::{ - Foundation::{ERROR_BUSY, ERROR_OPERATION_ABORTED, HANDLE}, + Foundation::{ERROR_BUSY, ERROR_OPERATION_ABORTED}, Networking::WinSock::{WSACleanup, WSAStartup, WSADATA}, System::IO::OVERLAPPED, }; @@ -294,7 +294,7 @@ pub struct Overlapped { /// The base [`OVERLAPPED`]. pub base: OVERLAPPED, /// The unique ID of created driver. - pub driver: HANDLE, + pub driver: RawFd, /// The registered user defined data. pub user_data: usize, /// The opcode. @@ -303,7 +303,7 @@ pub struct Overlapped { } impl Overlapped { - pub(crate) fn new(driver: HANDLE, user_data: usize, op: T) -> Self { + pub(crate) fn new(driver: RawFd, user_data: usize, op: T) -> Self { Self { base: unsafe { std::mem::zeroed() }, driver, @@ -326,7 +326,7 @@ pub(crate) struct RawOp { } impl RawOp { - pub(crate) fn new(driver: HANDLE, user_data: usize, op: impl OpCode + 'static) -> Self { + pub(crate) fn new(driver: RawFd, user_data: usize, op: impl OpCode + 'static) -> Self { let op = Overlapped::new(driver, user_data, op); let op = Box::new(op) as Box>; Self { From 6b32de31e91b68d74bd6e4494b6b02230bf30fb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 14:17:34 +0800 Subject: [PATCH 21/29] doc(driver): add docs for mod cp --- compio-driver/src/iocp/cp/mod.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs index 331d9738..1597e52f 100644 --- a/compio-driver/src/iocp/cp/mod.rs +++ b/compio-driver/src/iocp/cp/mod.rs @@ -1,3 +1,17 @@ +//! Completion Port +//! +//! This mod contains utilities of IOCP. It provides 2 working modes: +//! IOCP-per-thread, and IOCP-global. +//! +//! ## IOCP-per-thread +//! In `mod multi`. Each driver hosts a seperate port. If the port receives +//! entry that doesn't belong to the current port, it will try to repost it to +//! the correct port. +//! +//! ## IOCP-global +//! In `mod global`. A main port runs in a separate thread, and dispatches all +//! entries to the correct driver. + use std::{ io, os::windows::io::{AsRawHandle, FromRawHandle, OwnedHandle, RawHandle}, From f14d3ef292aabf2645dfde3cd1cbba65af22cd95 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 14:23:48 +0800 Subject: [PATCH 22/29] fix(runtime): docs of Attacher and remove Attachable --- compio-fs/src/file.rs | 4 +--- compio-fs/src/named_pipe.rs | 6 +----- compio-fs/src/pipe.rs | 6 +----- compio-net/src/socket.rs | 4 +--- compio-net/src/tcp.rs | 6 +----- compio-net/src/udp.rs | 4 +--- compio-net/src/unix.rs | 6 +----- compio-runtime/src/attacher.rs | 29 +++-------------------------- 8 files changed, 10 insertions(+), 55 deletions(-) diff --git a/compio-fs/src/file.rs b/compio-fs/src/file.rs index 0b241202..5907e49f 100644 --- a/compio-fs/src/file.rs +++ b/compio-fs/src/file.rs @@ -7,7 +7,7 @@ use compio_driver::{ syscall, AsRawFd, }; use compio_io::{AsyncReadAt, AsyncWriteAt}; -use compio_runtime::{impl_attachable, impl_try_clone, Attacher, Runtime}; +use compio_runtime::{impl_try_clone, Attacher, Runtime}; #[cfg(unix)] use { compio_buf::{IoVectoredBuf, IoVectoredBufMut}, @@ -217,6 +217,4 @@ impl AsyncWriteAt for &File { impl_raw_fd!(File, inner); -impl_attachable!(File, inner); - impl_try_clone!(File, inner); diff --git a/compio-fs/src/named_pipe.rs b/compio-fs/src/named_pipe.rs index 94be7808..6d785c57 100644 --- a/compio-fs/src/named_pipe.rs +++ b/compio-fs/src/named_pipe.rs @@ -9,7 +9,7 @@ use std::{ffi::OsStr, io, ptr::null}; use compio_buf::{BufResult, IoBuf, IoBufMut}; use compio_driver::{impl_raw_fd, op::ConnectNamedPipe, syscall, AsRawFd, FromRawFd, RawFd}; use compio_io::{AsyncRead, AsyncReadAt, AsyncWrite, AsyncWriteAt}; -use compio_runtime::{impl_attachable, impl_try_clone, Runtime}; +use compio_runtime::{impl_try_clone, Runtime}; use widestring::U16CString; use windows_sys::Win32::{ Security::SECURITY_ATTRIBUTES, @@ -232,8 +232,6 @@ impl AsyncWrite for &NamedPipeServer { impl_raw_fd!(NamedPipeServer, handle); -impl_attachable!(NamedPipeServer, handle); - impl_try_clone!(NamedPipeServer, handle); /// A [Windows named pipe] client. @@ -357,8 +355,6 @@ impl AsyncWrite for &NamedPipeClient { impl_raw_fd!(NamedPipeClient, handle); -impl_attachable!(NamedPipeClient, handle); - impl_try_clone!(NamedPipeClient, handle); /// A builder structure for construct a named pipe with named pipe-specific diff --git a/compio-fs/src/pipe.rs b/compio-fs/src/pipe.rs index 4781d0b1..51adeba2 100644 --- a/compio-fs/src/pipe.rs +++ b/compio-fs/src/pipe.rs @@ -9,7 +9,7 @@ use compio_driver::{ syscall, AsRawFd, FromRawFd, IntoRawFd, }; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::{impl_attachable, impl_try_clone, Runtime}; +use compio_runtime::{impl_try_clone, Runtime}; use crate::File; @@ -383,8 +383,6 @@ impl AsyncWrite for &Sender { impl_raw_fd!(Sender, file); -impl_attachable!(Sender, file); - impl_try_clone!(Sender, file); /// Reading end of a Unix pipe. @@ -508,8 +506,6 @@ impl AsyncRead for &Receiver { impl_raw_fd!(Receiver, file); -impl_attachable!(Receiver, file); - impl_try_clone!(Receiver, file); /// Checks if file is a FIFO diff --git a/compio-net/src/socket.rs b/compio-net/src/socket.rs index 45755771..873f2228 100644 --- a/compio-net/src/socket.rs +++ b/compio-net/src/socket.rs @@ -9,7 +9,7 @@ use compio_driver::{ }, AsRawFd, }; -use compio_runtime::{impl_attachable, impl_try_clone, Attacher, Runtime}; +use compio_runtime::{impl_try_clone, Attacher, Runtime}; use socket2::{Domain, Protocol, SockAddr, Socket as Socket2, Type}; #[derive(Debug)] @@ -206,6 +206,4 @@ impl Socket { impl_raw_fd!(Socket, socket); -impl_attachable!(Socket, socket); - impl_try_clone!(Socket, socket); diff --git a/compio-net/src/tcp.rs b/compio-net/src/tcp.rs index 5ab77dd0..7700650a 100644 --- a/compio-net/src/tcp.rs +++ b/compio-net/src/tcp.rs @@ -3,7 +3,7 @@ use std::{future::Future, io, net::SocketAddr}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; use compio_driver::impl_raw_fd; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::{impl_attachable, impl_try_clone}; +use compio_runtime::impl_try_clone; use socket2::{Protocol, SockAddr, Type}; use crate::{OwnedReadHalf, OwnedWriteHalf, ReadHalf, Socket, ToSocketAddrsAsync, WriteHalf}; @@ -111,8 +111,6 @@ impl TcpListener { impl_raw_fd!(TcpListener, inner); -impl_attachable!(TcpListener, inner); - impl_try_clone!(TcpListener, inner); /// A TCP stream between a local and a remote socket. @@ -279,6 +277,4 @@ impl AsyncWrite for &TcpStream { impl_raw_fd!(TcpStream, inner); -impl_attachable!(TcpStream, inner); - impl_try_clone!(TcpStream, inner); diff --git a/compio-net/src/udp.rs b/compio-net/src/udp.rs index 9c9e610f..e865f07b 100644 --- a/compio-net/src/udp.rs +++ b/compio-net/src/udp.rs @@ -2,7 +2,7 @@ use std::{future::Future, io, net::SocketAddr}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; use compio_driver::impl_raw_fd; -use compio_runtime::{impl_attachable, impl_try_clone}; +use compio_runtime::impl_try_clone; use socket2::{Protocol, SockAddr, Type}; use crate::{Socket, ToSocketAddrsAsync}; @@ -253,6 +253,4 @@ impl UdpSocket { impl_raw_fd!(UdpSocket, inner); -impl_attachable!(UdpSocket, inner); - impl_try_clone!(UdpSocket, inner); diff --git a/compio-net/src/unix.rs b/compio-net/src/unix.rs index f8d4f388..89928428 100644 --- a/compio-net/src/unix.rs +++ b/compio-net/src/unix.rs @@ -3,7 +3,7 @@ use std::{future::Future, io, path::Path}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; use compio_driver::impl_raw_fd; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::{impl_attachable, impl_try_clone}; +use compio_runtime::impl_try_clone; use socket2::{SockAddr, Type}; use crate::{OwnedReadHalf, OwnedWriteHalf, ReadHalf, Socket, WriteHalf}; @@ -90,8 +90,6 @@ impl UnixListener { impl_raw_fd!(UnixListener, inner); -impl_attachable!(UnixListener, inner); - impl_try_clone!(UnixListener, inner); /// A Unix stream between two local sockets on Windows & WSL. @@ -264,8 +262,6 @@ impl AsyncWrite for &UnixStream { impl_raw_fd!(UnixStream, inner); -impl_attachable!(UnixStream, inner); - #[cfg(windows)] #[inline] fn empty_unix_socket() -> SockAddr { diff --git a/compio-runtime/src/attacher.rs b/compio-runtime/src/attacher.rs index a595d1e7..7c533321 100644 --- a/compio-runtime/src/attacher.rs +++ b/compio-runtime/src/attacher.rs @@ -30,7 +30,8 @@ pub struct Attacher { } impl Attacher { - /// Create [`Attacher`]. + /// Create [`Attacher`]. It tries to attach the source, and will return + /// [`Err`] if it fails. pub fn new(source: S) -> io::Result { let this = Self { source, @@ -41,7 +42,7 @@ impl Attacher { } /// Attach the source. This method could be called many times, but if the - /// action fails, the error will only return once. + /// action fails, it will try to attach the source during each call. fn attach(&self) -> io::Result<()> { let r = Runtime::current(); let inner = r.inner(); @@ -51,12 +52,6 @@ impl Attacher { } } -impl Attachable for Attacher { - fn is_attached(&self) -> bool { - self.once.get().is_some() - } -} - impl IntoRawFd for Attacher { fn into_raw_fd(self) -> RawFd { self.source.into_raw_fd() @@ -106,12 +101,6 @@ impl DerefMut for Attacher { } } -/// Represents an attachable resource to driver. -pub trait Attachable { - /// Check if [`Attachable::attach`] has been called. - fn is_attached(&self) -> bool; -} - /// Duplicatable file or socket. pub trait TryClone: Sized { /// Duplicate the source. @@ -151,18 +140,6 @@ impl TryClone for OwnedFd { } } -#[macro_export] -#[doc(hidden)] -macro_rules! impl_attachable { - ($t:ty, $inner:ident) => { - impl $crate::Attachable for $t { - fn is_attached(&self) -> bool { - self.$inner.is_attached() - } - } - }; -} - #[macro_export] #[doc(hidden)] macro_rules! impl_try_clone { From 1fbac8f4a16e816cfd38f44dc58813d9f07ad7a8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 14:24:10 +0800 Subject: [PATCH 23/29] fix(Cargo.toml): remove useless skiplist dep --- Cargo.toml | 1 - 1 file changed, 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index e847af71..dcb21bba 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -39,7 +39,6 @@ cfg-if = "1.0.0" criterion = "0.5.1" crossbeam-channel = "0.5.8" crossbeam-queue = "0.3.8" -crossbeam-skiplist = "0.1.3" futures-channel = "0.3.29" futures-util = "0.3.29" libc = "0.2.149" From c90f23c2d4012c95a7bd8ac0cc4973562dbced7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 14:29:29 +0800 Subject: [PATCH 24/29] test(lib): fix for read_exact --- compio/tests/runtime.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/compio/tests/runtime.rs b/compio/tests/runtime.rs index 852e61ad..b3522835 100644 --- a/compio/tests/runtime.rs +++ b/compio/tests/runtime.rs @@ -24,8 +24,7 @@ async fn multi_threading() { tx.write_all(DATA).await.0.unwrap(); tx.write_all(DATA).await.0.unwrap(); - let (n, buffer) = rx.read_exact(Vec::with_capacity(DATA.len())).await.unwrap(); - assert_eq!(n, buffer.len()); + let ((), buffer) = rx.read_exact(Vec::with_capacity(DATA.len())).await.unwrap(); assert_eq!(DATA, String::from_utf8(buffer).unwrap()); compio::runtime::spawn_blocking(move || { From a70d5d7d519705761e8239cbb196c80852324cc0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 14:47:54 +0800 Subject: [PATCH 25/29] fix(driver): log post error --- compio-driver/src/iocp/cp/global.rs | 16 +++++++++++++--- compio-driver/src/iocp/cp/mod.rs | 14 +++++++++++--- 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs index b2c488ba..cc76dbf0 100644 --- a/compio-driver/src/iocp/cp/global.rs +++ b/compio-driver/src/iocp/cp/global.rs @@ -6,6 +6,7 @@ use std::{ time::Duration, }; +use compio_log::*; #[cfg(not(feature = "once_cell_try"))] use once_cell::sync::OnceCell as OnceLock; use windows_sys::Win32::System::IO::PostQueuedCompletionStatus; @@ -53,12 +54,13 @@ fn iocp_port() -> io::Result<&'static GlobalPort> { fn iocp_start() -> io::Result<()> { let port = iocp_port()?; std::thread::spawn(move || { + instrument!(compio_log::Level::TRACE, "iocp_start"); loop { for entry in port.port.poll_raw(None)? { // Any thin pointer is OK because we don't use the type of opcode. let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); let overlapped = unsafe { &*overlapped_ptr }; - syscall!( + if let Err(e) = syscall!( BOOL, PostQueuedCompletionStatus( overlapped.driver as _, @@ -66,8 +68,16 @@ fn iocp_start() -> io::Result<()> { entry.lpCompletionKey, entry.lpOverlapped, ) - ) - .ok(); + ) { + error!( + "fail to dispatch entry ({}, {}, {:p}) to driver {:p}: {:?}", + entry.dwNumberOfBytesTransferred, + entry.lpCompletionKey, + entry.lpOverlapped, + overlapped.driver, + e + ); + } } } #[allow(unreachable_code)] diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs index 1597e52f..814b35cb 100644 --- a/compio-driver/src/iocp/cp/mod.rs +++ b/compio-driver/src/iocp/cp/mod.rs @@ -143,7 +143,7 @@ impl CompletionPort { if let Some(current_driver) = current_driver { if overlapped.driver != current_driver { // Repose the entry to correct port. - syscall!( + if let Err(e) = syscall!( BOOL, PostQueuedCompletionStatus( overlapped.driver as _, @@ -151,8 +151,16 @@ impl CompletionPort { entry.lpCompletionKey, entry.lpOverlapped, ) - ) - .ok(); + ) { + error!( + "fail to repost entry ({}, {}, {:p}) to driver {:p}: {:?}", + entry.dwNumberOfBytesTransferred, + entry.lpCompletionKey, + entry.lpOverlapped, + overlapped.driver, + e + ); + } } } let res = if matches!( From fd5510ecc7cd76af9391c1691738bc9df86f95fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Sun, 4 Feb 2024 14:51:33 +0800 Subject: [PATCH 26/29] fix(driver,poll): mark submit unsafe --- compio-driver/src/poll/mod.rs | 21 ++++++++++++--------- 1 file changed, 12 insertions(+), 9 deletions(-) diff --git a/compio-driver/src/poll/mod.rs b/compio-driver/src/poll/mod.rs index ac8ac38e..a979c87d 100644 --- a/compio-driver/src/poll/mod.rs +++ b/compio-driver/src/poll/mod.rs @@ -186,19 +186,19 @@ impl Driver { RawOp::new(user_data, op) } - fn submit(&mut self, user_data: usize, arg: WaitArg) -> io::Result<()> { + /// # Safety + /// The input fd should be valid. + unsafe fn submit(&mut self, user_data: usize, arg: WaitArg) -> io::Result<()> { let need_add = !self.registry.contains_key(&arg.fd); let queue = self.registry.entry(arg.fd).or_default(); queue.push_back_interest(user_data, arg.interest); // We use fd as the key. let event = queue.event(arg.fd as usize); - unsafe { - if need_add { - self.poll.add(arg.fd, event)?; - } else { - let fd = BorrowedFd::borrow_raw(arg.fd); - self.poll.modify(fd, event)?; - } + if need_add { + self.poll.add(arg.fd, event)?; + } else { + let fd = BorrowedFd::borrow_raw(arg.fd); + self.poll.modify(fd, event)?; } Ok(()) } @@ -218,7 +218,10 @@ impl Driver { let op_pin = op.as_pin(); match op_pin.pre_submit() { Ok(Decision::Wait(arg)) => { - self.submit(user_data, arg)?; + // SAFETY: fd is from the OpCode. + unsafe { + self.submit(user_data, arg)?; + } Poll::Pending } Ok(Decision::Completed(res)) => Poll::Ready(Ok(res)), From b1408282622fb3999ad1a7d41475e15933307ae3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Fri, 16 Feb 2024 19:03:03 +0800 Subject: [PATCH 27/29] fix(driver): use InternalHigh as op result --- compio-driver/src/iocp/cp/global.rs | 12 ++++++++-- compio-driver/src/iocp/cp/mod.rs | 37 +++++++++++++++-------------- compio-driver/src/iocp/cp/multi.rs | 4 ++++ compio-driver/src/iocp/mod.rs | 5 +--- 4 files changed, 34 insertions(+), 24 deletions(-) diff --git a/compio-driver/src/iocp/cp/global.rs b/compio-driver/src/iocp/cp/global.rs index cc76dbf0..fd093215 100644 --- a/compio-driver/src/iocp/cp/global.rs +++ b/compio-driver/src/iocp/cp/global.rs @@ -36,6 +36,10 @@ impl GlobalPort { ) -> io::Result<()> { self.port.post(res, optr) } + + pub fn post_raw(&self, optr: *const Overlapped) -> io::Result<()> { + self.port.post_raw(optr) + } } impl AsRawHandle for GlobalPort { @@ -60,7 +64,7 @@ fn iocp_start() -> io::Result<()> { // Any thin pointer is OK because we don't use the type of opcode. let overlapped_ptr: *mut Overlapped<()> = entry.lpOverlapped.cast(); let overlapped = unsafe { &*overlapped_ptr }; - if let Err(e) = syscall!( + if let Err(_e) = syscall!( BOOL, PostQueuedCompletionStatus( overlapped.driver as _, @@ -75,7 +79,7 @@ fn iocp_start() -> io::Result<()> { entry.lpCompletionKey, entry.lpOverlapped, overlapped.driver, - e + _e ); } } @@ -138,4 +142,8 @@ impl PortHandle { ) -> io::Result<()> { self.port.post(res, optr) } + + pub fn post_raw(&self, optr: *const Overlapped) -> io::Result<()> { + self.port.post_raw(optr) + } } diff --git a/compio-driver/src/iocp/cp/mod.rs b/compio-driver/src/iocp/cp/mod.rs index 814b35cb..f78972f0 100644 --- a/compio-driver/src/iocp/cp/mod.rs +++ b/compio-driver/src/iocp/cp/mod.rs @@ -82,20 +82,25 @@ impl CompletionPort { res: io::Result, optr: *mut Overlapped, ) -> io::Result<()> { - if let Err(e) = &res { - let code = e.raw_os_error().unwrap_or(ERROR_BAD_COMMAND as _); - unsafe { &mut *optr }.base.Internal = ntstatus_from_win32(code) as _; + if let Some(overlapped) = unsafe { optr.as_mut() } { + match &res { + Ok(transferred) => { + overlapped.base.Internal = STATUS_SUCCESS as _; + overlapped.base.InternalHigh = *transferred; + } + Err(e) => { + let code = e.raw_os_error().unwrap_or(ERROR_BAD_COMMAND as _); + overlapped.base.Internal = ntstatus_from_win32(code) as _; + } + } } - // We have to use CompletionKey to transfer the result because it is large - // enough. It is OK because we set it to zero when attaching handles to IOCP. + self.post_raw(optr) + } + + pub fn post_raw(&self, optr: *const Overlapped) -> io::Result<()> { syscall!( BOOL, - PostQueuedCompletionStatus( - self.port.as_raw_handle() as _, - 0, - res.unwrap_or_default(), - optr.cast() - ) + PostQueuedCompletionStatus(self.port.as_raw_handle() as _, 0, 0, optr.cast()) )?; Ok(()) } @@ -143,7 +148,7 @@ impl CompletionPort { if let Some(current_driver) = current_driver { if overlapped.driver != current_driver { // Repose the entry to correct port. - if let Err(e) = syscall!( + if let Err(_e) = syscall!( BOOL, PostQueuedCompletionStatus( overlapped.driver as _, @@ -158,7 +163,7 @@ impl CompletionPort { entry.lpCompletionKey, entry.lpOverlapped, overlapped.driver, - e + _e ); } } @@ -167,11 +172,7 @@ impl CompletionPort { overlapped.base.Internal as NTSTATUS, STATUS_SUCCESS | STATUS_PENDING ) { - if entry.lpCompletionKey != 0 { - Ok(entry.lpCompletionKey) - } else { - Ok(entry.dwNumberOfBytesTransferred as _) - } + Ok(overlapped.base.InternalHigh) } else { let error = unsafe { RtlNtStatusToDosError(overlapped.base.Internal as _) }; match error { diff --git a/compio-driver/src/iocp/cp/multi.rs b/compio-driver/src/iocp/cp/multi.rs index 07a820f8..5f99956c 100644 --- a/compio-driver/src/iocp/cp/multi.rs +++ b/compio-driver/src/iocp/cp/multi.rs @@ -55,4 +55,8 @@ impl PortHandle { ) -> io::Result<()> { self.port.post(res, optr) } + + pub fn post_raw(&self, optr: *const Overlapped) -> io::Result<()> { + self.port.post_raw(optr) + } } diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index a00006aa..ee2707a1 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -281,10 +281,7 @@ impl NotifyHandle { /// Notify the inner driver. pub fn notify(&self) -> io::Result<()> { - self.port.post( - Ok(0), - self.overlapped.as_ref() as *const _ as *mut Overlapped<()> as _, - ) + self.port.post_raw(self.overlapped.as_ref()) } } From b039ac5a14326cdad9fb726d540720c5c51b821f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Mon, 4 Mar 2024 16:11:56 +0800 Subject: [PATCH 28/29] fix: resolve conflict on unix --- compio-driver/src/lib.rs | 6 ++++++ compio-fs/src/stdio/unix.rs | 33 ++++++++++--------------------- compio-runtime/src/runtime/mod.rs | 20 ++++++++++++++----- 3 files changed, 31 insertions(+), 28 deletions(-) diff --git a/compio-driver/src/lib.rs b/compio-driver/src/lib.rs index 047a4801..39ce24f9 100644 --- a/compio-driver/src/lib.rs +++ b/compio-driver/src/lib.rs @@ -272,6 +272,12 @@ impl Proactor { } } +impl AsRawFd for Proactor { + fn as_raw_fd(&self) -> RawFd { + self.driver.as_raw_fd() + } +} + /// An completed entry returned from kernel. #[derive(Debug)] pub(crate) struct Entry { diff --git a/compio-fs/src/stdio/unix.rs b/compio-fs/src/stdio/unix.rs index f883f926..fe9c359e 100644 --- a/compio-fs/src/stdio/unix.rs +++ b/compio-fs/src/stdio/unix.rs @@ -1,9 +1,8 @@ use std::{io, mem::ManuallyDrop}; use compio_buf::{BufResult, IoBuf, IoBufMut, IoVectoredBuf, IoVectoredBufMut}; -use compio_driver::{FromRawFd, RawFd}; +use compio_driver::{AsRawFd, FromRawFd, RawFd}; use compio_io::{AsyncRead, AsyncWrite}; -use compio_runtime::TryAsRawFd; use crate::pipe::{Receiver, Sender}; @@ -31,13 +30,9 @@ impl AsyncRead for Stdin { } } -impl TryAsRawFd for Stdin { - fn try_as_raw_fd(&self) -> io::Result { - self.0.try_as_raw_fd() - } - - unsafe fn as_raw_fd_unchecked(&self) -> RawFd { - self.0.as_raw_fd_unchecked() +impl AsRawFd for Stdin { + fn as_raw_fd(&self) -> RawFd { + self.0.as_raw_fd() } } @@ -73,13 +68,9 @@ impl AsyncWrite for Stdout { } } -impl TryAsRawFd for Stdout { - fn try_as_raw_fd(&self) -> io::Result { - self.0.try_as_raw_fd() - } - - unsafe fn as_raw_fd_unchecked(&self) -> RawFd { - self.0.as_raw_fd_unchecked() +impl AsRawFd for Stdout { + fn as_raw_fd(&self) -> RawFd { + self.0.as_raw_fd() } } @@ -115,12 +106,8 @@ impl AsyncWrite for Stderr { } } -impl TryAsRawFd for Stderr { - fn try_as_raw_fd(&self) -> io::Result { - self.0.try_as_raw_fd() - } - - unsafe fn as_raw_fd_unchecked(&self) -> RawFd { - self.0.as_raw_fd_unchecked() +impl AsRawFd for Stderr { + fn as_raw_fd(&self) -> RawFd { + self.0.as_raw_fd() } } diff --git a/compio-runtime/src/runtime/mod.rs b/compio-runtime/src/runtime/mod.rs index 2d6eb0bd..83d36257 100644 --- a/compio-runtime/src/runtime/mod.rs +++ b/compio-runtime/src/runtime/mod.rs @@ -10,7 +10,9 @@ use std::{ use async_task::{Runnable, Task}; use compio_buf::IntoInner; -use compio_driver::{op::Asyncify, Key, OpCode, Proactor, ProactorBuilder, PushEntry, RawFd}; +use compio_driver::{ + op::Asyncify, AsRawFd, Key, OpCode, Proactor, ProactorBuilder, PushEntry, RawFd, +}; use compio_log::{debug, instrument}; use crossbeam_queue::SegQueue; use futures_util::{future::Either, FutureExt}; @@ -57,10 +59,6 @@ impl RuntimeInner { }) } - pub fn id(&self) -> usize { - self.id - } - // Safety: be careful about the captured lifetime. pub unsafe fn spawn_unchecked(&self, future: F) -> Task { let runnables = self.runnables.clone(); @@ -229,6 +227,12 @@ impl RuntimeInner { } } +impl AsRawFd for RuntimeInner { + fn as_raw_fd(&self) -> RawFd { + self.driver.borrow().as_raw_fd() + } +} + struct RuntimeContext { depth: usize, ptr: Weak, @@ -408,6 +412,12 @@ impl Runtime { } } +impl AsRawFd for Runtime { + fn as_raw_fd(&self) -> RawFd { + self.inner.as_raw_fd() + } +} + #[cfg(feature = "criterion")] impl criterion::async_executor::AsyncExecutor for Runtime { fn block_on(&self, future: impl Future) -> T { From a9af8946584db53bbed73e69a5395793070e9db6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=AE=87=E9=80=B8?= Date: Mon, 4 Mar 2024 16:17:09 +0800 Subject: [PATCH 29/29] fix(driver,iocp): readd AsRawFd impl --- compio-driver/src/iocp/mod.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/compio-driver/src/iocp/mod.rs b/compio-driver/src/iocp/mod.rs index ee2707a1..4f900d65 100644 --- a/compio-driver/src/iocp/mod.rs +++ b/compio-driver/src/iocp/mod.rs @@ -262,6 +262,12 @@ impl Driver { } } +impl AsRawFd for Driver { + fn as_raw_fd(&self) -> RawFd { + self.port.as_raw_handle() + } +} + impl Drop for Driver { fn drop(&mut self) { syscall!(SOCKET, WSACleanup()).ok();