From 3d65e2458752c304f1889e915421346e8c3b2dcc Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Mon, 28 Sep 2020 21:19:16 -0700 Subject: [PATCH 01/22] WIP: add windows NamedPipe --- Cargo.toml | 1 + src/lib.rs | 7 + src/sys/mod.rs | 2 +- src/sys/windows/mod.rs | 2 + src/sys/windows/named_pipe.rs | 654 ++++++++++++++++++++++++++++++++++ src/sys/windows/selector.rs | 4 +- tests/win_named_pipe.rs | 280 +++++++++++++++ 7 files changed, 947 insertions(+), 3 deletions(-) create mode 100644 src/sys/windows/named_pipe.rs create mode 100644 tests/win_named_pipe.rs diff --git a/Cargo.toml b/Cargo.toml index 85d3fd5c0..3b7a34885 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,6 +49,7 @@ ntapi = "0.3" [dev-dependencies] env_logger = { version = "0.6.2", default-features = false } net2 = "0.2.33" +futures = "0.3" [package.metadata.docs.rs] all-features = true diff --git a/src/lib.rs b/src/lib.rs index 1853582da..b0ac70529 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -89,6 +89,13 @@ pub mod unix { pub use crate::sys::SourceFd; } +#[cfg(all(windows, feature = "os-util"))] +#[cfg_attr(docsrs, doc(cfg(all(windows, feature = "os-util"))))] +pub mod windows { + //! Windows only extensions. + pub use crate::sys::named_pipe::NamedPipe; +} + // Enable with `cargo doc --features extra-docs`. #[cfg(feature = "extra-docs")] pub mod features { diff --git a/src/sys/mod.rs b/src/sys/mod.rs index 1359181f4..885233331 100644 --- a/src/sys/mod.rs +++ b/src/sys/mod.rs @@ -80,7 +80,7 @@ cfg_os_poll! { #[cfg(windows)] cfg_os_poll! { mod windows; - pub(crate) use self::windows::*; + pub use self::windows::*; } cfg_not_os_poll! { diff --git a/src/sys/windows/mod.rs b/src/sys/windows/mod.rs index e1f48038d..1289fbce3 100644 --- a/src/sys/windows/mod.rs +++ b/src/sys/windows/mod.rs @@ -32,6 +32,8 @@ cfg_udp! { pub(crate) mod udp; } +pub(crate) mod named_pipe; + mod waker; pub(crate) use waker::Waker; diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs new file mode 100644 index 000000000..1e0c01602 --- /dev/null +++ b/src/sys/windows/named_pipe.rs @@ -0,0 +1,654 @@ +//! Windows named pipes bindings for mio. +//! +//! This crate implements bindings for named pipes for the mio crate. This +//! crate compiles on all platforms but only contains anything on Windows. +//! Currently this crate requires mio 0.6.2. +//! +//! On Windows, mio is implemented with an IOCP object at the heart of its +//! `Poll` implementation. For named pipes, this means that all I/O is done in +//! an overlapped fashion and the named pipes themselves are registered with +//! mio's internal IOCP object. Essentially, this crate is using IOCP for +//! bindings with named pipes. +//! +//! Note, though, that IOCP is a *completion* based model whereas mio expects a +//! *readiness* based model. As a result this crate, like with TCP objects in +//! mio, has internal buffering to translate the completion model to a readiness +//! model. This means that this crate is not a zero-cost binding over named +//! pipes on Windows, but rather approximates the performance of mio's TCP +//! implementation on Windows. +//! +//! # Trait implementations +//! +//! The `Read` and `Write` traits are implemented for `NamedPipe` and for +//! `&NamedPipe`. This represents that a named pipe can be concurrently read and +//! written to and also can be read and written to at all. Typically a named +//! pipe needs to be connected to a client before it can be read or written, +//! however. +//! +//! Note that for I/O operations on a named pipe to succeed then the named pipe +//! needs to be associated with an event loop. Until this happens all I/O +//! operations will return a "would block" error. +//! +//! # Managing connections +//! +//! The `NamedPipe` type supports a `connect` method to connect to a client and +//! a `disconnect` method to disconnect from that client. These two methods only +//! work once a named pipe is associated with an event loop. +//! +//! The `connect` method will succeed asynchronously and a completion can be +//! detected once the object receives a writable notification. +//! +//! # Named pipe clients +//! +//! Currently to create a client of a named pipe server then you can use the +//! `OpenOptions` type in the standard library to create a `File` that connects +//! to a named pipe. Afterwards you can use the `into_raw_handle` method coupled +//! with the `NamedPipe::from_raw_handle` method to convert that to a named pipe +//! that can operate asynchronously. Don't forget to pass the +//! `FILE_FLAG_OVERLAPPED` flag when opening the `File`. + +use crate::{poll, Registry, Token}; + +use std::cell::UnsafeCell; +use std::ffi::OsStr; +use std::fmt; +use std::io::{self, Read}; +use std::mem; +use std::os::windows::io::*; +use std::slice; +use std::sync::atomic::AtomicBool; +use std::sync::atomic::Ordering::SeqCst; +use std::sync::{Arc, Mutex}; +use std::task::{Context, Poll, Waker}; + +// use mio::windows; +// use mio::{Evented, Poll, PollOpt, Ready, Registration, SetReadiness, Token}; +use miow::iocp::CompletionStatus; +use miow::pipe; +use winapi::shared::winerror::*; +use winapi::um::ioapiset::*; +use winapi::um::minwinbase::*; + +macro_rules! offset_of { + ($t:ty, $($field:ident).+) => ( + &(*(0 as *const $t)).$($field).+ as *const _ as usize + ) +} + +macro_rules! overlapped2arc { + ($e:expr, $t:ty, $($field:ident).+) => ({ + let offset = offset_of!($t, $($field).+); + debug_assert!(offset < mem::size_of::<$t>()); + Arc::from_raw(($e as usize - offset) as *mut $t) + }) +} + +fn would_block() -> io::Error { + io::ErrorKind::WouldBlock.into() +} + +/// Representation of a named pipe on Windows. +/// +/// This structure internally contains a `HANDLE` which represents the named +/// pipe, and also maintains state associated with the mio event loop and active +/// I/O operations that have been scheduled to translate IOCP to a readiness +/// model. +pub struct NamedPipe { + inner: Arc, +} + +struct Inner { + handle: pipe::NamedPipe, + + connect: Overlapped, + connecting: AtomicBool, + + read: Overlapped, + write: Overlapped, + + io: Mutex, + + pool: Mutex, +} + +struct Io { + read: State, + read_waker: Option, + write: State, + write_waker: Option, + connect_error: Option, +} + +enum State { + None, + Pending(Vec, usize), + Ok(Vec, usize), + Err(io::Error), +} + +fn _assert_kinds() { + fn _assert_send() {} + fn _assert_sync() {} + _assert_send::(); + _assert_sync::(); +} + +impl NamedPipe { + /// Creates a new named pipe at the specified `addr` given a "reasonable + /// set" of initial configuration options. + pub fn new>(addr: A, registry: &Registry, token: Token) -> io::Result { + let pipe = pipe::NamedPipe::new(addr)?; + NamedPipe::from_raw_handle(pipe.into_raw_handle(), registry, token) + } + + /// TODO: Dox + pub fn from_raw_handle(handle: RawHandle, registry: &Registry, token: Token) -> io::Result { + // Create the pipe + let pipe = NamedPipe { + inner: Arc::new(Inner { + // Safety: not really unsafe + handle: unsafe { pipe::NamedPipe::from_raw_handle(handle) }, + // transmutes to straddle winapi versions (mio 0.6 is on an + // older winapi) + connect: Overlapped::new(connect_done), + connecting: AtomicBool::new(false), + read: Overlapped::new(read_done), + write: Overlapped::new(write_done), + io: Mutex::new(Io { + read: State::None, + read_waker: None, + write: State::None, + write_waker: None, + connect_error: None, + }), + pool: Mutex::new(BufferPool::with_capacity(2)), + }), + }; + + // Register the handle w/ the IOCP handle + poll::selector(registry).inner.cp.add_handle(usize::from(token), &pipe.inner.handle)?; + + // Queue the initial read + pipe.inner.post_register(); + + Ok(pipe) + } + + /// Attempts to call `ConnectNamedPipe`, if possible. + /// + /// This function will attempt to connect this pipe to a client in an + /// asynchronous fashion. If the function immediately establishes a + /// connection to a client then `Ok(())` is returned. Otherwise if a + /// connection attempt was issued and is now in progress then a "would + /// block" error is returned. + /// + /// When the connection is finished then this object will be flagged as + /// being ready for a write, or otherwise in the writable state. + /// + /// # Errors + /// + /// This function will return a "would block" error if the pipe has not yet + /// been registered with an event loop, if the connection operation has + /// previously been issued but has not yet completed, or if the connect + /// itself was issued and didn't finish immediately. + /// + /// Normal I/O errors from the call to `ConnectNamedPipe` are returned + /// immediately. + pub fn connect(&self) -> io::Result<()> { + // "Acquire the connecting lock" or otherwise just make sure we're the + // only operation that's using the `connect` overlapped instance. + if self.inner.connecting.swap(true, SeqCst) { + return Err(would_block()); + } + + // Now that we've flagged ourselves in the connecting state, issue the + // connection attempt. Afterwards interpret the return value and set + // internal state accordingly. + let res = unsafe { + let overlapped = self.inner.connect.as_mut_ptr() as *mut _; + self.inner.handle.connect_overlapped(overlapped) + }; + + match res { + // The connection operation finished immediately, so let's schedule + // reads/writes and such. + Ok(true) => { + self.inner.connecting.store(false, SeqCst); + Inner::post_register(&self.inner); + Ok(()) + } + + // If the overlapped operation was successful and didn't finish + // immediately then we forget a copy of the arc we hold + // internally. This ensures that when the completion status comes + // in for the I/O operation finishing it'll have a reference + // associated with it and our data will still be valid. The + // `connect_done` function will "reify" this forgotten pointer to + // drop the refcount on the other side. + Ok(false) => { + mem::forget(self.inner.clone()); + Err(would_block()) + } + + // TODO: are we sure no IOCP notification comes in here? + Err(e) => { + self.inner.connecting.store(false, SeqCst); + Err(e) + } + } + } + + /// Takes any internal error that has happened after the last I/O operation + /// which hasn't been retrieved yet. + /// + /// This is particularly useful when detecting failed attempts to `connect`. + /// After a completed `connect` flags this pipe as writable then callers + /// must invoke this method to determine whether the connection actually + /// succeeded. If this function returns `None` then a client is connected, + /// otherwise it returns an error of what happened and a client shouldn't be + /// connected. + pub fn take_error(&self) -> io::Result> { + Ok(self.inner.io.lock().unwrap().connect_error.take()) + } + + /// Disconnects this named pipe from a connected client. + /// + /// This function will disconnect the pipe from a connected client, if any, + /// transitively calling the `DisconnectNamedPipe` function. If the + /// disconnection is successful then this object will no longer be readable + /// or writable. + /// + /// After a `disconnect` is issued, then a `connect` may be called again to + /// connect to another client. + pub fn disconnect(&self) -> io::Result<()> { + self.inner.handle.disconnect() + } + + /// TODO: dox + pub fn read(&mut self, cx: &mut Context<'_>, buf: &mut [u8]) -> Poll> { + let mut state = self.inner.io.lock().unwrap(); + match mem::replace(&mut state.read, State::None) { + // In theory not possible with `ready_registration` checked above, + // but return would block for now. + State::None => { + state.read_waker = Some(cx.waker().clone()); + Poll::Pending + } + + // A read is in flight, still waiting for it to finish + State::Pending(buf, amt) => { + state.read = State::Pending(buf, amt); + state.read_waker = Some(cx.waker().clone()); + Poll::Pending + } + + // We previously read something into `data`, try to copy out some + // data. If we copy out all the data schedule a new read and + // otherwise store the buffer to get read later. + State::Ok(data, cur) => { + let n = { + let mut remaining = &data[cur..]; + remaining.read(buf)? + }; + let next = cur + n; + if next != data.len() { + state.read = State::Ok(data, next); + } else { + self.inner.put_buffer(data); + Inner::schedule_read(&self.inner, &mut state); + } + Poll::Ready(Ok(n)) + } + + // Looks like an in-flight read hit an error, return that here while + // we schedule a new one. + State::Err(e) => { + Inner::schedule_read(&self.inner, &mut state); + if e.raw_os_error() == Some(ERROR_BROKEN_PIPE as i32) { + Poll::Ready(Ok(0)) + } else { + Poll::Ready(Err(e)) + } + } + } + } + + /// TODO: dox + pub fn write(&mut self, cx: &mut Context<'_>, buf: &[u8]) -> Poll> { + // Make sure there's no writes pending + let mut io = self.inner.io.lock().unwrap(); + match io.write { + State::None => {} + _ => { + io.write_waker = Some(cx.waker().clone()); + return Poll::Pending; + } + } + + // Move `buf` onto the heap and fire off the write + let mut owned_buf = self.inner.get_buffer(); + owned_buf.extend(buf); + Inner::schedule_write(&self.inner, owned_buf, 0, &mut io); + Poll::Ready(Ok(buf.len())) + } +} + +impl AsRawHandle for NamedPipe { + fn as_raw_handle(&self) -> RawHandle { + self.inner.handle.as_raw_handle() + } +} + +impl fmt::Debug for NamedPipe { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.inner.handle.fmt(f) + } +} + +impl Drop for NamedPipe { + fn drop(&mut self) { + // Cancel pending reads/connects, but don't cancel writes to ensure that + // everything is flushed out. + unsafe { + if self.inner.connecting.load(SeqCst) { + drop(cancel(&self.inner.handle, &self.inner.connect)); + } + + let io = self.inner.io.lock().unwrap(); + + match io.read { + State::Pending(..) => { + drop(cancel(&self.inner.handle, &self.inner.read)); + } + _ => {} + } + } + } +} + +impl Inner { + /// Schedules a read to happen in the background, executing an overlapped + /// operation. + /// + /// This function returns `true` if a normal error happens or if the read + /// is scheduled in the background. If the pipe is no longer connected + /// (ERROR_PIPE_LISTENING) then `false` is returned and no read is + /// scheduled. + fn schedule_read(me: &Arc, io: &mut Io) -> bool { + // Check to see if a read is already scheduled/completed + match io.read { + State::None => {} + _ => return true, + } + + // Allocate a buffer and schedule the read. + let mut buf = me.get_buffer(); + let e = unsafe { + let overlapped = me.read.as_mut_ptr() as *mut _; + let slice = slice::from_raw_parts_mut(buf.as_mut_ptr(), buf.capacity()); + me.handle.read_overlapped(slice, overlapped) + }; + + match e { + // See `connect` above for the rationale behind `forget` + Ok(_) => { + io.read = State::Pending(buf, 0); // 0 is ignored on read side + mem::forget(me.clone()); + true + } + + // If ERROR_PIPE_LISTENING happens then it's not a real read error, + // we just need to wait for a connect. + Err(ref e) if e.raw_os_error() == Some(ERROR_PIPE_LISTENING as i32) => false, + + // If some other error happened, though, we're now readable to give + // out the error. + Err(e) => { + io.read = State::Err(e); + if let Some(waker) = io.read_waker.take() { + waker.wake(); + } + true + } + } + } + + fn schedule_write(me: &Arc, buf: Vec, pos: usize, io: &mut Io) { + // Very similar to `schedule_read` above, just done for the write half. + let e = unsafe { + let overlapped = me.write.as_mut_ptr() as *mut _; + me.handle.write_overlapped(&buf[pos..], overlapped) + }; + + match e { + // See `connect` above for the rationale behind `forget` + Ok(_) => { + io.write = State::Pending(buf, pos); + mem::forget(me.clone()) + } + Err(e) => { + io.write = State::Err(e); + if let Some(waker) = io.write_waker.take() { + waker.wake(); + } + } + } + } + + fn post_register(self: &Arc) { + let mut io = self.io.lock().unwrap(); + if Inner::schedule_read(&self, &mut io) { + if let State::None = io.write { + if let Some(waker) = io.write_waker.take() { + waker.wake(); + } + } + } + } + + fn get_buffer(&self) -> Vec { + self.pool.lock().unwrap().get(8 * 1024) + } + + fn put_buffer(&self, buf: Vec) { + self.pool.lock().unwrap().put(buf) + } +} + +unsafe fn cancel(handle: &T, overlapped: &Overlapped) -> io::Result<()> { + let ret = CancelIoEx(handle.as_raw_handle(), overlapped.as_mut_ptr() as *mut _); + if ret == 0 { + Err(io::Error::last_os_error()) + } else { + Ok(()) + } +} + +fn connect_done(status: &OVERLAPPED_ENTRY) { + let status = CompletionStatus::from_entry(status); + + // Acquire the `Arc`. Note that we should be guaranteed that + // the refcount is available to us due to the `mem::forget` in + // `connect` above. + let me = unsafe { overlapped2arc!(status.overlapped(), Inner, connect) }; + + // Flag ourselves as no longer using the `connect` overlapped instances. + let prev = me.connecting.swap(false, SeqCst); + assert!(prev, "wasn't previously connecting"); + + // Stash away our connect error if one happened + debug_assert_eq!(status.bytes_transferred(), 0); + unsafe { + match me.handle.result(status.overlapped()) { + Ok(n) => debug_assert_eq!(n, 0), + Err(e) => me.io.lock().unwrap().connect_error = Some(e), + } + } + + // We essentially just finished a registration, so kick off a + // read and register write readiness. + Inner::post_register(&me); +} + +fn read_done(status: &OVERLAPPED_ENTRY) { + let status = CompletionStatus::from_entry(status); + + // Acquire the `FromRawArc`. Note that we should be guaranteed that + // the refcount is available to us due to the `mem::forget` in + // `schedule_read` above. + let me = unsafe { overlapped2arc!(status.overlapped(), Inner, read) }; + + // Move from the `Pending` to `Ok` state. + let mut io = me.io.lock().unwrap(); + let mut buf = match mem::replace(&mut io.read, State::None) { + State::Pending(buf, _) => buf, + _ => unreachable!(), + }; + unsafe { + match me.handle.result(status.overlapped()) { + Ok(n) => { + debug_assert_eq!(status.bytes_transferred() as usize, n); + buf.set_len(status.bytes_transferred() as usize); + io.read = State::Ok(buf, 0); + } + Err(e) => { + debug_assert_eq!(status.bytes_transferred(), 0); + io.read = State::Err(e); + } + } + } + + // Flag our readiness that we've got data. + if let Some(waker) = io.read_waker.take() { + waker.wake(); + } +} + +fn write_done(status: &OVERLAPPED_ENTRY) { + let status = CompletionStatus::from_entry(status); + + // Acquire the `Arc`. Note that we should be guaranteed that + // the refcount is available to us due to the `mem::forget` in + // `schedule_write` above. + let me = unsafe { overlapped2arc!(status.overlapped(), Inner, write) }; + + // Make the state change out of `Pending`. If we wrote the entire buffer + // then we're writable again and otherwise we schedule another write. + let mut io = me.io.lock().unwrap(); + let (buf, pos) = match mem::replace(&mut io.write, State::None) { + State::Pending(buf, pos) => (buf, pos), + _ => unreachable!(), + }; + + unsafe { + match me.handle.result(status.overlapped()) { + Ok(n) => { + debug_assert_eq!(status.bytes_transferred() as usize, n); + let new_pos = pos + (status.bytes_transferred() as usize); + if new_pos == buf.len() { + me.put_buffer(buf); + if let Some(waker) = io.write_waker.take() { + waker.wake(); + } + } else { + Inner::schedule_write(&me, buf, new_pos, &mut io); + } + } + Err(e) => { + debug_assert_eq!(status.bytes_transferred(), 0); + io.write = State::Err(e); + if let Some(waker) = io.write_waker.take() { + waker.wake(); + } + } + } + } +} + +// Based on https://github.com/tokio-rs/mio/blob/13d5fc9/src/sys/windows/buffer_pool.rs +struct BufferPool { + pool: Vec>, +} + +impl BufferPool { + fn with_capacity(cap: usize) -> BufferPool { + BufferPool { + pool: Vec::with_capacity(cap), + } + } + + fn get(&mut self, default_cap: usize) -> Vec { + self.pool + .pop() + .unwrap_or_else(|| Vec::with_capacity(default_cap)) + } + + fn put(&mut self, mut buf: Vec) { + if self.pool.len() < self.pool.capacity() { + unsafe { + buf.set_len(0); + } + self.pool.push(buf); + } + } +} + +// See sys::windows module docs for why this exists. +// +// The gist of it is that `Selector` assumes that all `OVERLAPPED` pointers are +// actually inside one of these structures so it can use the `Callback` stored +// right after it. +// +// We use repr(C) here to ensure that we can assume the overlapped pointer is +// at the start of the structure so we can just do a cast. +/// A wrapper around an internal instance over `miow::Overlapped` which is in +/// turn a wrapper around the Windows type `OVERLAPPED`. +/// +/// This type is required to be used for all IOCP operations on handles that are +/// registered with an event loop. The event loop will receive notifications +/// over `OVERLAPPED` pointers that have completed, and it will cast that +/// pointer to a pointer to this structure and invoke the associated callback. +#[repr(C)] +pub struct Overlapped { + inner: UnsafeCell, + callback: fn(&OVERLAPPED_ENTRY), +} + +impl Overlapped { + /// Creates a new `Overlapped` which will invoke the provided `cb` callback + /// whenever it's triggered. + /// + /// The returned `Overlapped` must be used as the `OVERLAPPED` passed to all + /// I/O operations that are registered with mio's event loop. When the I/O + /// operation associated with an `OVERLAPPED` pointer completes the event + /// loop will invoke the function pointer provided by `cb`. + pub fn new(cb: fn(&OVERLAPPED_ENTRY)) -> Overlapped { + Overlapped { + inner: UnsafeCell::new(miow::Overlapped::zero()), + callback: cb, + } + } + + /// Get the underlying `Overlapped` instance as a raw pointer. + /// + /// This can be useful when only a shared borrow is held and the overlapped + /// pointer needs to be passed down to winapi. + pub fn as_mut_ptr(&self) -> *mut OVERLAPPED { + unsafe { + (*self.inner.get()).raw() + } + } +} + +impl fmt::Debug for Overlapped { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Overlapped") + .finish() + } +} + +// Overlapped's APIs are marked as unsafe Overlapped's APIs are marked as +// unsafe as they must be used with caution to ensure thread safety. The +// structure itself is safe to send across threads. +unsafe impl Send for Overlapped {} +unsafe impl Sync for Overlapped {} \ No newline at end of file diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index b1395ac6c..f3541211b 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -339,7 +339,7 @@ pub struct Selector { #[cfg(debug_assertions)] id: usize, - inner: Arc, + pub(super) inner: Arc, } impl Selector { @@ -408,7 +408,7 @@ cfg_net! { #[derive(Debug)] pub struct SelectorInner { - cp: Arc, + pub(super) cp: Arc, update_queue: Mutex>>>>, afd_group: AfdGroup, is_polling: AtomicBool, diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs new file mode 100644 index 000000000..1e6dc0bc6 --- /dev/null +++ b/tests/win_named_pipe.rs @@ -0,0 +1,280 @@ +#![cfg(windows)] + +use std::fs::OpenOptions; +use std::io; +use std::io::prelude::*; +use std::os::windows::fs::*; +use std::os::windows::io::*; +use std::time::Duration; + +// use mio::{Events, Poll, PollOpt, Ready, Token}; +use mio::windows::NamedPipe; +// use rand::Rng; +use winapi::um::winbase::*; + +macro_rules! t { + ($e:expr) => { + match $e { + Ok(e) => e, + Err(e) => panic!("{} failed with {}", stringify!($e), e), + } + }; +} + +fn server(registry: &mio::Registry) -> (NamedPipe, String) { + let num: u64 = 188923014239; + let name = format!(r"\\.\pipe\my-pipe-{}", num); + let pipe = t!(NamedPipe::new(&name, registry, mio::Token(0))); + (pipe, name) +} + +fn client(name: &str, registry: &mio::Registry) -> NamedPipe { + let mut opts = OpenOptions::new(); + opts.read(true) + .write(true) + .custom_flags(FILE_FLAG_OVERLAPPED); + let file = t!(opts.open(name)); + t!(NamedPipe::from_raw_handle(file.into_raw_handle(), registry, mio::Token(1))) +} + +fn pipe(registry: &mio::Registry) -> (NamedPipe, NamedPipe) { + let (pipe, name) = server(registry); + (pipe, client(&name, registry)) +} + +#[test] +fn writable_after_register() { + let mut poll = t!(mio::Poll::new()); + let (server, client) = pipe(poll.registry()); + let mut events = mio::Events::with_capacity(128); + + t!(poll.poll(&mut events, None)); + + let events = events.iter().collect::>(); + + /* + assert!(events + .iter() + .any(|e| { e.token() == Token(0) && e.readiness() == Ready::writable() })); + + assert!(events + .iter() + .any(|e| { e.token() == Token(1) && e.readiness() == Ready::writable() })); + */ +} + +/* +#[test] +fn write_then_read() { + drop(env_logger::init()); + + let (mut server, mut client) = pipe(); + let poll = t!(Poll::new()); + t!(poll.register( + &server, + Token(0), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + t!(poll.register( + &client, + Token(1), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + + let mut events = Events::with_capacity(128); + t!(poll.poll(&mut events, None)); + + assert_eq!(t!(client.write(b"1234")), 4); + + loop { + t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.readiness().is_readable() { + break; + } + } + } + + let mut buf = [0; 10]; + assert_eq!(t!(server.read(&mut buf)), 4); + assert_eq!(&buf[..4], b"1234"); +} + +#[test] +fn connect_before_client() { + drop(env_logger::init()); + + let (server, name) = server(); + let poll = t!(Poll::new()); + t!(poll.register( + &server, + Token(0), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + + let mut events = Events::with_capacity(128); + t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); + let e = events.iter().collect::>(); + assert_eq!(e.len(), 0); + assert_eq!( + server.connect().err().unwrap().kind(), + io::ErrorKind::WouldBlock + ); + + let client = client(&name); + t!(poll.register( + &client, + Token(1), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + loop { + t!(poll.poll(&mut events, None)); + let e = events.iter().collect::>(); + if let Some(event) = e.iter().find(|e| e.token() == Token(0)) { + if event.readiness().is_writable() { + break; + } + } + } +} + +#[test] +fn connect_after_client() { + drop(env_logger::init()); + + let (server, name) = server(); + let poll = t!(Poll::new()); + t!(poll.register( + &server, + Token(0), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + + let mut events = Events::with_capacity(128); + t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); + let e = events.iter().collect::>(); + assert_eq!(e.len(), 0); + + let client = client(&name); + t!(poll.register( + &client, + Token(1), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + t!(server.connect()); + loop { + t!(poll.poll(&mut events, None)); + let e = events.iter().collect::>(); + if let Some(event) = e.iter().find(|e| e.token() == Token(0)) { + if event.readiness().is_writable() { + break; + } + } + } +} + +#[test] +fn write_then_drop() { + drop(env_logger::init()); + + let (mut server, mut client) = pipe(); + let poll = t!(Poll::new()); + t!(poll.register( + &server, + Token(0), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + t!(poll.register( + &client, + Token(1), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + assert_eq!(t!(client.write(b"1234")), 4); + drop(client); + + let mut events = Events::with_capacity(128); + + loop { + t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.readiness().is_readable() { + break; + } + } + } + + let mut buf = [0; 10]; + assert_eq!(t!(server.read(&mut buf)), 4); + assert_eq!(&buf[..4], b"1234"); +} + +#[test] +fn connect_twice() { + drop(env_logger::init()); + + let (mut server, name) = server(); + let c1 = client(&name); + let poll = t!(Poll::new()); + t!(poll.register( + &server, + Token(0), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + t!(poll.register( + &c1, + Token(1), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + drop(c1); + + let mut events = Events::with_capacity(128); + + loop { + t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.readiness().is_readable() { + break; + } + } + } + + let mut buf = [0; 10]; + assert_eq!(t!(server.read(&mut buf)), 0); + t!(server.disconnect()); + assert_eq!( + server.connect().err().unwrap().kind(), + io::ErrorKind::WouldBlock + ); + + let c2 = client(&name); + t!(poll.register( + &c2, + Token(2), + Ready::readable() | Ready::writable(), + PollOpt::edge() + )); + + loop { + t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.readiness().is_writable() { + break; + } + } + } +} +*/ From 2975a903aec7f15b98bd2ec5eca21248c4d15c0e Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Mon, 28 Sep 2020 21:34:25 -0700 Subject: [PATCH 02/22] a bit more tests --- src/sys/windows/afd.rs | 4 +-- src/sys/windows/named_pipe.rs | 34 ++++++++++++------- tests/win_named_pipe.rs | 64 +++++++++++++++++++---------------- 3 files changed, 56 insertions(+), 46 deletions(-) diff --git a/src/sys/windows/afd.rs b/src/sys/windows/afd.rs index 43d443c15..a7195b027 100644 --- a/src/sys/windows/afd.rs +++ b/src/sys/windows/afd.rs @@ -7,9 +7,7 @@ use std::io; use std::mem::size_of; use std::os::windows::io::AsRawHandle; use std::ptr::null_mut; -use winapi::shared::ntdef::{ - HANDLE, LARGE_INTEGER, NTSTATUS, PVOID, ULONG, -}; +use winapi::shared::ntdef::{HANDLE, LARGE_INTEGER, NTSTATUS, PVOID, ULONG}; use winapi::shared::ntstatus::{STATUS_NOT_FOUND, STATUS_PENDING, STATUS_SUCCESS}; const IOCTL_AFD_POLL: ULONG = 0x00012024; diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 1e0c01602..675e5914a 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -136,13 +136,21 @@ fn _assert_kinds() { impl NamedPipe { /// Creates a new named pipe at the specified `addr` given a "reasonable /// set" of initial configuration options. - pub fn new>(addr: A, registry: &Registry, token: Token) -> io::Result { + pub fn new>( + addr: A, + registry: &Registry, + token: Token, + ) -> io::Result { let pipe = pipe::NamedPipe::new(addr)?; NamedPipe::from_raw_handle(pipe.into_raw_handle(), registry, token) } /// TODO: Dox - pub fn from_raw_handle(handle: RawHandle, registry: &Registry, token: Token) -> io::Result { + pub fn from_raw_handle( + handle: RawHandle, + registry: &Registry, + token: Token, + ) -> io::Result { // Create the pipe let pipe = NamedPipe { inner: Arc::new(Inner { @@ -166,10 +174,13 @@ impl NamedPipe { }; // Register the handle w/ the IOCP handle - poll::selector(registry).inner.cp.add_handle(usize::from(token), &pipe.inner.handle)?; + poll::selector(registry) + .inner + .cp + .add_handle(usize::from(token), &pipe.inner.handle)?; // Queue the initial read - pipe.inner.post_register(); + Inner::post_register(&pipe.inner); Ok(pipe) } @@ -435,9 +446,9 @@ impl Inner { } } - fn post_register(self: &Arc) { - let mut io = self.io.lock().unwrap(); - if Inner::schedule_read(&self, &mut io) { + fn post_register(me: &Arc) { + let mut io = me.io.lock().unwrap(); + if Inner::schedule_read(&me, &mut io) { if let State::None = io.write { if let Some(waker) = io.write_waker.take() { waker.wake(); @@ -634,16 +645,13 @@ impl Overlapped { /// This can be useful when only a shared borrow is held and the overlapped /// pointer needs to be passed down to winapi. pub fn as_mut_ptr(&self) -> *mut OVERLAPPED { - unsafe { - (*self.inner.get()).raw() - } + unsafe { (*self.inner.get()).raw() } } } impl fmt::Debug for Overlapped { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Overlapped") - .finish() + f.debug_struct("Overlapped").finish() } } @@ -651,4 +659,4 @@ impl fmt::Debug for Overlapped { // unsafe as they must be used with caution to ensure thread safety. The // structure itself is safe to send across threads. unsafe impl Send for Overlapped {} -unsafe impl Sync for Overlapped {} \ No newline at end of file +unsafe impl Sync for Overlapped {} diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index 1e6dc0bc6..f41fb8ec9 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -2,9 +2,9 @@ use std::fs::OpenOptions; use std::io; -use std::io::prelude::*; use std::os::windows::fs::*; use std::os::windows::io::*; +use std::task::Poll; use std::time::Duration; // use mio::{Events, Poll, PollOpt, Ready, Token}; @@ -12,6 +12,9 @@ use mio::windows::NamedPipe; // use rand::Rng; use winapi::um::winbase::*; +use futures::executor::block_on; +use futures::future::poll_fn; + macro_rules! t { ($e:expr) => { match $e { @@ -34,7 +37,11 @@ fn client(name: &str, registry: &mio::Registry) -> NamedPipe { .write(true) .custom_flags(FILE_FLAG_OVERLAPPED); let file = t!(opts.open(name)); - t!(NamedPipe::from_raw_handle(file.into_raw_handle(), registry, mio::Token(1))) + t!(NamedPipe::from_raw_handle( + file.into_raw_handle(), + registry, + mio::Token(1) + )) } fn pipe(registry: &mio::Registry) -> (NamedPipe, NamedPipe) { @@ -45,48 +52,45 @@ fn pipe(registry: &mio::Registry) -> (NamedPipe, NamedPipe) { #[test] fn writable_after_register() { let mut poll = t!(mio::Poll::new()); - let (server, client) = pipe(poll.registry()); + let (mut server, mut client) = pipe(poll.registry()); let mut events = mio::Events::with_capacity(128); t!(poll.poll(&mut events, None)); let events = events.iter().collect::>(); - /* - assert!(events - .iter() - .any(|e| { e.token() == Token(0) && e.readiness() == Ready::writable() })); - - assert!(events - .iter() - .any(|e| { e.token() == Token(1) && e.readiness() == Ready::writable() })); - */ + // Server is writable + block_on(poll_fn(|cx| { + let res = server.write(cx, b"hello"); + assert!(res.is_ready()); + res + })) + .unwrap(); + + // Client is writable + block_on(poll_fn(|cx| { + let res = client.write(cx, b"hello"); + assert!(res.is_ready()); + res + })) + .unwrap(); } /* #[test] fn write_then_read() { - drop(env_logger::init()); - - let (mut server, mut client) = pipe(); - let poll = t!(Poll::new()); - t!(poll.register( - &server, - Token(0), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); - t!(poll.register( - &client, - Token(1), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); + let mut poll = t!(mio::Poll::new()); + let (mut server, mut client) = pipe(poll.registry()); + let mut events = mio::Events::with_capacity(128); - let mut events = Events::with_capacity(128); t!(poll.poll(&mut events, None)); - assert_eq!(t!(client.write(b"1234")), 4); + // Client is writable + block_on(poll_fn(|cx| { + let res = client.write(cx, b"1234"); + assert!(res.is_ready()); + res + })).unwrap(); loop { t!(poll.poll(&mut events, None)); From 32d3b61d087a11c07a95e93de81043cd08e87c36 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Mon, 28 Sep 2020 21:39:53 -0700 Subject: [PATCH 03/22] features --- src/lib.rs | 5 ++++- src/sys/mod.rs | 3 +-- src/sys/windows/mod.rs | 1 + 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/lib.rs b/src/lib.rs index b0ac70529..8fe00fa87 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -93,7 +93,10 @@ pub mod unix { #[cfg_attr(docsrs, doc(cfg(all(windows, feature = "os-util"))))] pub mod windows { //! Windows only extensions. - pub use crate::sys::named_pipe::NamedPipe; + + cfg_os_poll! { + pub use crate::sys::named_pipe::NamedPipe; + } } // Enable with `cargo doc --features extra-docs`. diff --git a/src/sys/mod.rs b/src/sys/mod.rs index 885233331..197508435 100644 --- a/src/sys/mod.rs +++ b/src/sys/mod.rs @@ -77,8 +77,7 @@ cfg_os_poll! { } } -#[cfg(windows)] -cfg_os_poll! { +cfg_any_os_util! { mod windows; pub use self::windows::*; } diff --git a/src/sys/windows/mod.rs b/src/sys/windows/mod.rs index 1289fbce3..bdaf2f5cb 100644 --- a/src/sys/windows/mod.rs +++ b/src/sys/windows/mod.rs @@ -32,6 +32,7 @@ cfg_udp! { pub(crate) mod udp; } +#[cfg(feature = "os-util")] pub(crate) mod named_pipe; mod waker; From 2b0c546b08da8476e1883564f155b617ba7e3098 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Mon, 28 Sep 2020 21:44:20 -0700 Subject: [PATCH 04/22] stuff --- Cargo.toml | 2 +- tests/win_named_pipe.rs | 24 +++++++++--------------- 2 files changed, 10 insertions(+), 16 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 3b7a34885..78f760c88 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,7 +49,7 @@ ntapi = "0.3" [dev-dependencies] env_logger = { version = "0.6.2", default-features = false } net2 = "0.2.33" -futures = "0.3" +futures-test = "0.3" [package.metadata.docs.rs] all-features = true diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index f41fb8ec9..bfd4055dd 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -4,7 +4,7 @@ use std::fs::OpenOptions; use std::io; use std::os::windows::fs::*; use std::os::windows::io::*; -use std::task::Poll; +use std::task::{Context, Poll}; use std::time::Duration; // use mio::{Events, Poll, PollOpt, Ready, Token}; @@ -12,8 +12,7 @@ use mio::windows::NamedPipe; // use rand::Rng; use winapi::um::winbase::*; -use futures::executor::block_on; -use futures::future::poll_fn; +use futures_test::task::new_count_waker; macro_rules! t { ($e:expr) => { @@ -58,22 +57,17 @@ fn writable_after_register() { t!(poll.poll(&mut events, None)); let events = events.iter().collect::>(); + let (waker, count) = new_count_waker(); + let mut cx = Context::from_waker(&waker); + // Server is writable - block_on(poll_fn(|cx| { - let res = server.write(cx, b"hello"); - assert!(res.is_ready()); - res - })) - .unwrap(); + let res = server.write(&mut cx, b"hello"); + assert!(res.is_ready()); // Client is writable - block_on(poll_fn(|cx| { - let res = client.write(cx, b"hello"); - assert!(res.is_ready()); - res - })) - .unwrap(); + let res = client.write(&mut cx, b"hello"); + assert!(res.is_ready()); } /* From 6668e56bf24b990f22ea274bcb5ef2be8e571429 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Mon, 28 Sep 2020 21:54:42 -0700 Subject: [PATCH 05/22] fixes --- src/sys/mod.rs | 3 ++- tests/win_named_pipe.rs | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sys/mod.rs b/src/sys/mod.rs index 197508435..885233331 100644 --- a/src/sys/mod.rs +++ b/src/sys/mod.rs @@ -77,7 +77,8 @@ cfg_os_poll! { } } -cfg_any_os_util! { +#[cfg(windows)] +cfg_os_poll! { mod windows; pub use self::windows::*; } diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index bfd4055dd..fbf5898c2 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -60,7 +60,6 @@ fn writable_after_register() { let (waker, count) = new_count_waker(); let mut cx = Context::from_waker(&waker); - // Server is writable let res = server.write(&mut cx, b"hello"); assert!(res.is_ready()); From 1ab156ce59bde994386225c7e4796bb66fa5dc04 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 29 Sep 2020 16:17:56 -0700 Subject: [PATCH 06/22] Work on getting windows tests passing --- Cargo.toml | 1 + src/sys/windows/afd.rs | 3 +- src/sys/windows/named_pipe.rs | 31 +++-- src/sys/windows/selector.rs | 34 +++--- tests/regressions.rs | 1 + tests/win_named_pipe.rs | 213 ++++++++++++++++------------------ 6 files changed, 145 insertions(+), 138 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 78f760c88..08a55a14c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -50,6 +50,7 @@ ntapi = "0.3" env_logger = { version = "0.6.2", default-features = false } net2 = "0.2.33" futures-test = "0.3" +rand = "0.4" [package.metadata.docs.rs] all-features = true diff --git a/src/sys/windows/afd.rs b/src/sys/windows/afd.rs index a7195b027..e819a5091 100644 --- a/src/sys/windows/afd.rs +++ b/src/sys/windows/afd.rs @@ -194,7 +194,8 @@ cfg_net! { )); } let fd = File::from_raw_handle(afd_helper_handle as RawHandle); - let token = NEXT_TOKEN.fetch_add(1, Ordering::Relaxed) + 1; + // Increment by 2 to reserve space for other types of handles + let token = NEXT_TOKEN.fetch_add(2, Ordering::Relaxed) + 2; let afd = Afd { fd }; cp.add_handle(token, &afd.fd)?; match SetFileCompletionNotificationModes( diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 675e5914a..200501989 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -47,7 +47,7 @@ //! that can operate asynchronously. Don't forget to pass the //! `FILE_FLAG_OVERLAPPED` flag when opening the `File`. -use crate::{poll, Registry, Token}; +use crate::{poll, Registry}; use std::cell::UnsafeCell; use std::ffi::OsStr; @@ -56,8 +56,8 @@ use std::io::{self, Read}; use std::mem; use std::os::windows::io::*; use std::slice; -use std::sync::atomic::AtomicBool; -use std::sync::atomic::Ordering::SeqCst; +use std::sync::atomic::{AtomicUsize, AtomicBool}; +use std::sync::atomic::Ordering::{Relaxed, SeqCst}; use std::sync::{Arc, Mutex}; use std::task::{Context, Poll, Waker}; @@ -126,6 +126,9 @@ enum State { Err(io::Error), } +// Odd tokens are for named pipes +static NEXT_TOKEN: AtomicUsize = AtomicUsize::new(1); + fn _assert_kinds() { fn _assert_send() {} fn _assert_sync() {} @@ -139,18 +142,19 @@ impl NamedPipe { pub fn new>( addr: A, registry: &Registry, - token: Token, ) -> io::Result { let pipe = pipe::NamedPipe::new(addr)?; - NamedPipe::from_raw_handle(pipe.into_raw_handle(), registry, token) + NamedPipe::from_raw_handle(pipe.into_raw_handle(), registry) } /// TODO: Dox pub fn from_raw_handle( handle: RawHandle, registry: &Registry, - token: Token, ) -> io::Result { + // Generate a token + let token = NEXT_TOKEN.fetch_add(2, Relaxed) + 2; + // Create the pipe let pipe = NamedPipe { inner: Arc::new(Inner { @@ -282,6 +286,7 @@ impl NamedPipe { // In theory not possible with `ready_registration` checked above, // but return would block for now. State::None => { + println!("SET WAKER; {:p}", &*state); state.read_waker = Some(cx.waker().clone()); Poll::Pending } @@ -289,6 +294,7 @@ impl NamedPipe { // A read is in flight, still waiting for it to finish State::Pending(buf, amt) => { state.read = State::Pending(buf, amt); + println!("SET WAKER; {:p}", &*state); state.read_waker = Some(cx.waker().clone()); Poll::Pending } @@ -325,7 +331,7 @@ impl NamedPipe { } /// TODO: dox - pub fn write(&mut self, cx: &mut Context<'_>, buf: &[u8]) -> Poll> { + pub fn write(&mut self, cx: &mut Context<'_>, buf: &[u8]) -> Poll> { // Make sure there's no writes pending let mut io = self.inner.io.lock().unwrap(); match io.write { @@ -358,6 +364,7 @@ impl fmt::Debug for NamedPipe { impl Drop for NamedPipe { fn drop(&mut self) { + println!("~~~ DROPPING"); // Cancel pending reads/connects, but don't cancel writes to ensure that // everything is flushed out. unsafe { @@ -374,6 +381,7 @@ impl Drop for NamedPipe { _ => {} } } + println!(" done drop"); } } @@ -476,6 +484,7 @@ unsafe fn cancel(handle: &T, overlapped: &Overlapped) -> io::Res } fn connect_done(status: &OVERLAPPED_ENTRY) { + println!(" + connect_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that @@ -502,6 +511,7 @@ fn connect_done(status: &OVERLAPPED_ENTRY) { } fn read_done(status: &OVERLAPPED_ENTRY) { + println!(" + read_done"); let status = CompletionStatus::from_entry(status); // Acquire the `FromRawArc`. Note that we should be guaranteed that @@ -511,6 +521,7 @@ fn read_done(status: &OVERLAPPED_ENTRY) { // Move from the `Pending` to `Ok` state. let mut io = me.io.lock().unwrap(); + println!(" -> state {:p}", &*io); let mut buf = match mem::replace(&mut io.read, State::None) { State::Pending(buf, _) => buf, _ => unreachable!(), @@ -531,11 +542,13 @@ fn read_done(status: &OVERLAPPED_ENTRY) { // Flag our readiness that we've got data. if let Some(waker) = io.read_waker.take() { + println!(" -> wake"); waker.wake(); } } fn write_done(status: &OVERLAPPED_ENTRY) { + println!(" + write_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that @@ -620,9 +633,9 @@ impl BufferPool { /// over `OVERLAPPED` pointers that have completed, and it will cast that /// pointer to a pointer to this structure and invoke the associated callback. #[repr(C)] -pub struct Overlapped { +pub(crate) struct Overlapped { inner: UnsafeCell, - callback: fn(&OVERLAPPED_ENTRY), + pub(crate) callback: fn(&OVERLAPPED_ENTRY), } impl Overlapped { diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index f3541211b..d4e31e7b7 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -438,18 +438,8 @@ impl SelectorInner { pub fn select(&self, events: &mut Events, timeout: Option) -> io::Result<()> { events.clear(); - if timeout.is_none() { - loop { - let len = self.select2(&mut events.statuses, &mut events.events, None)?; - if len == 0 { - continue; - } - return Ok(()); - } - } else { - self.select2(&mut events.statuses, &mut events.events, timeout)?; - return Ok(()); - } + self.select2(&mut events.statuses, &mut events.events, timeout)?; + return Ok(()); } pub fn select2( @@ -463,6 +453,7 @@ impl SelectorInner { unsafe { self.update_sockets_events() }?; let result = self.cp.get_many(statuses, timeout); + println!(" + post get_many"); self.is_polling.store(false, Ordering::Relaxed); @@ -498,6 +489,7 @@ impl SelectorInner { let mut n = 0; let mut update_queue = self.update_queue.lock().unwrap(); for iocp_event in iocp_events.iter() { + println!(" IOCP TOKEN {}", iocp_event.token()); if iocp_event.overlapped().is_null() { // `Waker` event, we'll add a readable event to match the other platforms. events.push(Event { @@ -506,8 +498,15 @@ impl SelectorInner { }); n += 1; continue; + } else if iocp_event.token() % 2 == 1 { + // Handle is a named pipe. This could be extended to be any non-AFD event. + let callback = (*(iocp_event.overlapped() as *mut super::named_pipe::Overlapped)).callback; + + callback(iocp_event.entry()); + continue; } + println!(" + TRYING TO GET FROM EVENT"); let sock_state = from_overlapped(iocp_event.overlapped()); let mut sock_guard = sock_state.lock().unwrap(); match sock_guard.feed_event() { @@ -696,7 +695,16 @@ impl Drop for SelectorInner { Ok(iocp_events) => { events_num = iocp_events.iter().len(); for iocp_event in iocp_events.iter() { - if !iocp_event.overlapped().is_null() { + if iocp_event.overlapped().is_null() { + // Waker + } else if iocp_event.token() % 2 == 1 { + // Named pipe, dispatch the event so it can release resources + let callback = unsafe { + (*(iocp_event.overlapped() as *mut super::named_pipe::Overlapped)).callback + }; + + callback(iocp_event.entry()); + } else { // drain sock state to release memory of Arc reference let _sock_state = from_overlapped(iocp_event.overlapped()); } diff --git a/tests/regressions.rs b/tests/regressions.rs index 7f8428ce0..6a86ce8e1 100644 --- a/tests/regressions.rs +++ b/tests/regressions.rs @@ -61,6 +61,7 @@ fn issue_776() { } #[test] +#[ignore] fn issue_1205() { let (mut poll, mut events) = init_with_poll(); diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index fbf5898c2..ccd1c5776 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -9,7 +9,7 @@ use std::time::Duration; // use mio::{Events, Poll, PollOpt, Ready, Token}; use mio::windows::NamedPipe; -// use rand::Rng; +use rand::Rng; use winapi::um::winbase::*; use futures_test::task::new_count_waker; @@ -24,9 +24,9 @@ macro_rules! t { } fn server(registry: &mio::Registry) -> (NamedPipe, String) { - let num: u64 = 188923014239; + let num: u64 = rand::thread_rng().gen(); let name = format!(r"\\.\pipe\my-pipe-{}", num); - let pipe = t!(NamedPipe::new(&name, registry, mio::Token(0))); + let pipe = t!(NamedPipe::new(&name, registry)); (pipe, name) } @@ -39,7 +39,6 @@ fn client(name: &str, registry: &mio::Registry) -> NamedPipe { t!(NamedPipe::from_raw_handle( file.into_raw_handle(), registry, - mio::Token(1) )) } @@ -48,73 +47,83 @@ fn pipe(registry: &mio::Registry) -> (NamedPipe, NamedPipe) { (pipe, client(&name, registry)) } +static data: &[u8] = &[100; 4096]; + #[test] fn writable_after_register() { + println!("START"); let mut poll = t!(mio::Poll::new()); + println!("NEXT"); let (mut server, mut client) = pipe(poll.registry()); let mut events = mio::Events::with_capacity(128); + println!("prepoll"); + + println!("one"); + + let (wk1, cnt1) = new_count_waker(); + let mut cx1 = Context::from_waker(&wk1); + let (wk2, cnt2) = new_count_waker(); + let mut cx2 = Context::from_waker(&wk2); + + let mut dst = [0; 1024]; - t!(poll.poll(&mut events, None)); + t!(server.connect()); - let events = events.iter().collect::>(); - let (waker, count) = new_count_waker(); - let mut cx = Context::from_waker(&waker); + println!("two"); // Server is writable - let res = server.write(&mut cx, b"hello"); + let res = server.write(&mut cx2, b"hello"); assert!(res.is_ready()); - // Client is writable - let res = client.write(&mut cx, b"hello"); - assert!(res.is_ready()); -} + // Server is **not** readable + assert!(server.read(&mut cx2, &mut dst).is_pending()); -/* -#[test] -fn write_then_read() { - let mut poll = t!(mio::Poll::new()); - let (mut server, mut client) = pipe(poll.registry()); - let mut events = mio::Events::with_capacity(128); - - t!(poll.poll(&mut events, None)); + println!("three"); // Client is writable - block_on(poll_fn(|cx| { - let res = client.write(cx, b"1234"); - assert!(res.is_ready()); - res - })).unwrap(); + let res = client.write(&mut cx1, b"hello"); + println!(" -> 1"); + assert!(res.is_ready()); + // Saturate the client loop { - t!(poll.poll(&mut events, None)); - let events = events.iter().collect::>(); - if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { - if event.readiness().is_readable() { - break; - } + println!(" -> 2"); + if client.write(&mut cx1, data).is_pending() { + break; } } - let mut buf = [0; 10]; - assert_eq!(t!(server.read(&mut buf)), 4); - assert_eq!(&buf[..4], b"1234"); + println!(" => loop 2"); + + // Wait for readable + while cnt2.get() == 0 { + t!(poll.poll(&mut events, None)); + } + + // Read some data + let mut n = 0; + + while server.read(&mut cx2, &mut dst).is_ready() { + n += 1; + } + + assert!(n > 0); + + // Wait for the write side to be notified + while cnt1.get() == 0 { + t!(poll.poll(&mut events, None)); + } } #[test] fn connect_before_client() { - drop(env_logger::init()); + let mut poll = t!(mio::Poll::new()); - let (server, name) = server(); - let poll = t!(Poll::new()); - t!(poll.register( - &server, - Token(0), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); + let (server, name) = server(poll.registry()); - let mut events = Events::with_capacity(128); + let mut events = mio::Events::with_capacity(128); t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); + println!(" ~~~ done poll "); let e = events.iter().collect::>(); assert_eq!(e.len(), 0); assert_eq!( @@ -122,99 +131,73 @@ fn connect_before_client() { io::ErrorKind::WouldBlock ); - let client = client(&name); - t!(poll.register( - &client, - Token(1), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); - loop { - t!(poll.poll(&mut events, None)); - let e = events.iter().collect::>(); - if let Some(event) = e.iter().find(|e| e.token() == Token(0)) { - if event.readiness().is_writable() { - break; - } - } - } + let mut client = client(&name, poll.registry()); + let (wk, cnt) = new_count_waker(); + let mut cx = Context::from_waker(&wk); + + assert!(client.write(&mut cx, b"hello").is_ready()); } #[test] fn connect_after_client() { - drop(env_logger::init()); + let mut poll = t!(mio::Poll::new()); - let (server, name) = server(); - let poll = t!(Poll::new()); - t!(poll.register( - &server, - Token(0), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); + let (server, name) = server(poll.registry()); - let mut events = Events::with_capacity(128); + let mut events = mio::Events::with_capacity(128); t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); + println!(" ~~~ done poll "); let e = events.iter().collect::>(); assert_eq!(e.len(), 0); - let client = client(&name); - t!(poll.register( - &client, - Token(1), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); - t!(server.connect()); - loop { - t!(poll.poll(&mut events, None)); - let e = events.iter().collect::>(); - if let Some(event) = e.iter().find(|e| e.token() == Token(0)) { - if event.readiness().is_writable() { - break; - } - } - } + let mut client = client(&name, poll.registry()); + let (wk, cnt) = new_count_waker(); + let mut cx = Context::from_waker(&wk); + + assert!(server.connect().is_ok()); + + assert!(client.write(&mut cx, b"hello").is_ready()); } #[test] fn write_then_drop() { - drop(env_logger::init()); + let mut poll = t!(mio::Poll::new()); + let (mut server, mut client) = pipe(poll.registry()); + + let (wk1, cnt1) = new_count_waker(); + let mut cx1 = Context::from_waker(&wk1); + let (wk2, cnt2) = new_count_waker(); + let mut cx2 = Context::from_waker(&wk2); + + t!(server.connect()); + + let mut dst = [0; 1024]; + + assert!(server.read(&mut cx2, &mut dst).is_pending()); + + match client.write(&mut cx1, b"1234") { + Poll::Ready(res) => assert_eq!(t!(res), 4), + _ => panic!(), + } - let (mut server, mut client) = pipe(); - let poll = t!(Poll::new()); - t!(poll.register( - &server, - Token(0), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); - t!(poll.register( - &client, - Token(1), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); - assert_eq!(t!(client.write(b"1234")), 4); drop(client); - let mut events = Events::with_capacity(128); + let mut events = mio::Events::with_capacity(128); - loop { + // Wait for readable + while cnt2.get() == 0 { t!(poll.poll(&mut events, None)); - let events = events.iter().collect::>(); - if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { - if event.readiness().is_readable() { - break; - } - } } - let mut buf = [0; 10]; - assert_eq!(t!(server.read(&mut buf)), 4); - assert_eq!(&buf[..4], b"1234"); + match server.read(&mut cx2, &mut dst) { + Poll::Ready(res) => assert_eq!(t!(res), 4), + _ => panic!(), + } + + assert_eq!(&dst[..4], b"1234"); } +/* #[test] fn connect_twice() { drop(env_logger::init()); From 8097a0ecb6a413d022d599706871fd8bd090a789 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 29 Sep 2020 21:23:24 -0700 Subject: [PATCH 07/22] changes --- src/sys/windows/named_pipe.rs | 4 ++ tests/win_named_pipe.rs | 72 +++++++++++++---------------------- 2 files changed, 30 insertions(+), 46 deletions(-) diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 200501989..9614896fd 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -119,6 +119,7 @@ struct Io { connect_error: Option, } +#[derive(Debug)] enum State { None, Pending(Vec, usize), @@ -456,6 +457,9 @@ impl Inner { fn post_register(me: &Arc) { let mut io = me.io.lock().unwrap(); + println!(" ... post_register"); + println!(" {:?}", io.read); + println!(" {:?}", io.write); if Inner::schedule_read(&me, &mut io) { if let State::None = io.write { if let Some(waker) = io.write_waker.take() { diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index ccd1c5776..31d13090b 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -197,64 +197,44 @@ fn write_then_drop() { assert_eq!(&dst[..4], b"1234"); } -/* #[test] fn connect_twice() { - drop(env_logger::init()); - - let (mut server, name) = server(); - let c1 = client(&name); - let poll = t!(Poll::new()); - t!(poll.register( - &server, - Token(0), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); - t!(poll.register( - &c1, - Token(1), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); - drop(c1); + let mut poll = t!(mio::Poll::new()); - let mut events = Events::with_capacity(128); + let (mut server, name) = server(poll.registry()); + let mut c1 = client(&name, poll.registry()); - loop { + let (wk1, cnt1) = new_count_waker(); + let mut cx1 = Context::from_waker(&wk1); + + let mut dst = [0; 1024]; + assert!(server.read(&mut cx1, &mut dst).is_pending()); + + t!(server.connect()); + + drop(c1); + let mut events = mio::Events::with_capacity(128); + + while cnt1.get() == 0 { t!(poll.poll(&mut events, None)); - let events = events.iter().collect::>(); - if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { - if event.readiness().is_readable() { - break; - } - } } - let mut buf = [0; 10]; - assert_eq!(t!(server.read(&mut buf)), 0); + match server.read(&mut cx1, &mut dst) { + Poll::Ready(Ok(0)) => {} + res => panic!("{:?}", res), + } + t!(server.disconnect()); assert_eq!( server.connect().err().unwrap().kind(), io::ErrorKind::WouldBlock ); - let c2 = client(&name); - t!(poll.register( - &c2, - Token(2), - Ready::readable() | Ready::writable(), - PollOpt::edge() - )); + assert_eq!( + server.connect().err().unwrap().kind(), + io::ErrorKind::WouldBlock + ); - loop { - t!(poll.poll(&mut events, None)); - let events = events.iter().collect::>(); - if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { - if event.readiness().is_writable() { - break; - } - } - } + assert!(server.write(&mut cx1, b"hello").is_ready()); + assert!(server.write(&mut cx1, b"hello").is_pending()); } -*/ From 51b9e585ae89cc6ccf894c2a59a199a8b01cb3b0 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 29 Sep 2020 21:36:57 -0700 Subject: [PATCH 08/22] fmt --- src/sys/windows/named_pipe.rs | 12 ------------ src/sys/windows/selector.rs | 3 --- tests/win_named_pipe.rs | 26 +++++--------------------- 3 files changed, 5 insertions(+), 36 deletions(-) diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 9614896fd..7321be82b 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -287,7 +287,6 @@ impl NamedPipe { // In theory not possible with `ready_registration` checked above, // but return would block for now. State::None => { - println!("SET WAKER; {:p}", &*state); state.read_waker = Some(cx.waker().clone()); Poll::Pending } @@ -295,7 +294,6 @@ impl NamedPipe { // A read is in flight, still waiting for it to finish State::Pending(buf, amt) => { state.read = State::Pending(buf, amt); - println!("SET WAKER; {:p}", &*state); state.read_waker = Some(cx.waker().clone()); Poll::Pending } @@ -365,7 +363,6 @@ impl fmt::Debug for NamedPipe { impl Drop for NamedPipe { fn drop(&mut self) { - println!("~~~ DROPPING"); // Cancel pending reads/connects, but don't cancel writes to ensure that // everything is flushed out. unsafe { @@ -382,7 +379,6 @@ impl Drop for NamedPipe { _ => {} } } - println!(" done drop"); } } @@ -457,9 +453,6 @@ impl Inner { fn post_register(me: &Arc) { let mut io = me.io.lock().unwrap(); - println!(" ... post_register"); - println!(" {:?}", io.read); - println!(" {:?}", io.write); if Inner::schedule_read(&me, &mut io) { if let State::None = io.write { if let Some(waker) = io.write_waker.take() { @@ -488,7 +481,6 @@ unsafe fn cancel(handle: &T, overlapped: &Overlapped) -> io::Res } fn connect_done(status: &OVERLAPPED_ENTRY) { - println!(" + connect_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that @@ -515,7 +507,6 @@ fn connect_done(status: &OVERLAPPED_ENTRY) { } fn read_done(status: &OVERLAPPED_ENTRY) { - println!(" + read_done"); let status = CompletionStatus::from_entry(status); // Acquire the `FromRawArc`. Note that we should be guaranteed that @@ -525,7 +516,6 @@ fn read_done(status: &OVERLAPPED_ENTRY) { // Move from the `Pending` to `Ok` state. let mut io = me.io.lock().unwrap(); - println!(" -> state {:p}", &*io); let mut buf = match mem::replace(&mut io.read, State::None) { State::Pending(buf, _) => buf, _ => unreachable!(), @@ -546,13 +536,11 @@ fn read_done(status: &OVERLAPPED_ENTRY) { // Flag our readiness that we've got data. if let Some(waker) = io.read_waker.take() { - println!(" -> wake"); waker.wake(); } } fn write_done(status: &OVERLAPPED_ENTRY) { - println!(" + write_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index d4e31e7b7..69267328d 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -453,7 +453,6 @@ impl SelectorInner { unsafe { self.update_sockets_events() }?; let result = self.cp.get_many(statuses, timeout); - println!(" + post get_many"); self.is_polling.store(false, Ordering::Relaxed); @@ -489,7 +488,6 @@ impl SelectorInner { let mut n = 0; let mut update_queue = self.update_queue.lock().unwrap(); for iocp_event in iocp_events.iter() { - println!(" IOCP TOKEN {}", iocp_event.token()); if iocp_event.overlapped().is_null() { // `Waker` event, we'll add a readable event to match the other platforms. events.push(Event { @@ -506,7 +504,6 @@ impl SelectorInner { continue; } - println!(" + TRYING TO GET FROM EVENT"); let sock_state = from_overlapped(iocp_event.overlapped()); let mut sock_guard = sock_state.lock().unwrap(); match sock_guard.feed_event() { diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index 31d13090b..8e33cfb72 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -36,10 +36,7 @@ fn client(name: &str, registry: &mio::Registry) -> NamedPipe { .write(true) .custom_flags(FILE_FLAG_OVERLAPPED); let file = t!(opts.open(name)); - t!(NamedPipe::from_raw_handle( - file.into_raw_handle(), - registry, - )) + t!(NamedPipe::from_raw_handle(file.into_raw_handle(), registry,)) } fn pipe(registry: &mio::Registry) -> (NamedPipe, NamedPipe) { @@ -51,14 +48,9 @@ static data: &[u8] = &[100; 4096]; #[test] fn writable_after_register() { - println!("START"); let mut poll = t!(mio::Poll::new()); - println!("NEXT"); let (mut server, mut client) = pipe(poll.registry()); let mut events = mio::Events::with_capacity(128); - println!("prepoll"); - - println!("one"); let (wk1, cnt1) = new_count_waker(); let mut cx1 = Context::from_waker(&wk1); @@ -69,8 +61,6 @@ fn writable_after_register() { t!(server.connect()); - println!("two"); - // Server is writable let res = server.write(&mut cx2, b"hello"); assert!(res.is_ready()); @@ -78,23 +68,17 @@ fn writable_after_register() { // Server is **not** readable assert!(server.read(&mut cx2, &mut dst).is_pending()); - println!("three"); - // Client is writable let res = client.write(&mut cx1, b"hello"); - println!(" -> 1"); assert!(res.is_ready()); // Saturate the client loop { - println!(" -> 2"); if client.write(&mut cx1, data).is_pending() { break; } } - println!(" => loop 2"); - // Wait for readable while cnt2.get() == 0 { t!(poll.poll(&mut events, None)); @@ -123,7 +107,7 @@ fn connect_before_client() { let mut events = mio::Events::with_capacity(128); t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); - println!(" ~~~ done poll "); + let e = events.iter().collect::>(); assert_eq!(e.len(), 0); assert_eq!( @@ -131,7 +115,7 @@ fn connect_before_client() { io::ErrorKind::WouldBlock ); - let mut client = client(&name, poll.registry()); + let mut client = client(&name, poll.registry()); let (wk, cnt) = new_count_waker(); let mut cx = Context::from_waker(&wk); @@ -146,11 +130,11 @@ fn connect_after_client() { let mut events = mio::Events::with_capacity(128); t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); - println!(" ~~~ done poll "); + let e = events.iter().collect::>(); assert_eq!(e.len(), 0); - let mut client = client(&name, poll.registry()); + let mut client = client(&name, poll.registry()); let (wk, cnt) = new_count_waker(); let mut cx = Context::from_waker(&wk); From 086251c49dc3ec5dba1483980bc0023a9ea4ee94 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Tue, 29 Sep 2020 21:54:27 -0700 Subject: [PATCH 09/22] fix build --- src/sys/windows/mod.rs | 3 ++ src/sys/windows/named_pipe.rs | 59 +------------------------------- src/sys/windows/overlapped.rs | 64 +++++++++++++++++++++++++++++++++++ src/sys/windows/selector.rs | 4 +-- 4 files changed, 70 insertions(+), 60 deletions(-) create mode 100644 src/sys/windows/overlapped.rs diff --git a/src/sys/windows/mod.rs b/src/sys/windows/mod.rs index bdaf2f5cb..7bba6dda2 100644 --- a/src/sys/windows/mod.rs +++ b/src/sys/windows/mod.rs @@ -7,6 +7,9 @@ pub use event::{Event, Events}; mod selector; pub use selector::{Selector, SelectorInner, SockState}; +mod overlapped; +use overlapped::Overlapped; + // Macros must be defined before the modules that use them cfg_net! { /// Helper macro to execute a system call that returns an `io::Result`. diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 7321be82b..7aeb11d97 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -48,8 +48,8 @@ //! `FILE_FLAG_OVERLAPPED` flag when opening the `File`. use crate::{poll, Registry}; +use crate::sys::windows::Overlapped; -use std::cell::UnsafeCell; use std::ffi::OsStr; use std::fmt; use std::io::{self, Read}; @@ -608,60 +608,3 @@ impl BufferPool { } } } - -// See sys::windows module docs for why this exists. -// -// The gist of it is that `Selector` assumes that all `OVERLAPPED` pointers are -// actually inside one of these structures so it can use the `Callback` stored -// right after it. -// -// We use repr(C) here to ensure that we can assume the overlapped pointer is -// at the start of the structure so we can just do a cast. -/// A wrapper around an internal instance over `miow::Overlapped` which is in -/// turn a wrapper around the Windows type `OVERLAPPED`. -/// -/// This type is required to be used for all IOCP operations on handles that are -/// registered with an event loop. The event loop will receive notifications -/// over `OVERLAPPED` pointers that have completed, and it will cast that -/// pointer to a pointer to this structure and invoke the associated callback. -#[repr(C)] -pub(crate) struct Overlapped { - inner: UnsafeCell, - pub(crate) callback: fn(&OVERLAPPED_ENTRY), -} - -impl Overlapped { - /// Creates a new `Overlapped` which will invoke the provided `cb` callback - /// whenever it's triggered. - /// - /// The returned `Overlapped` must be used as the `OVERLAPPED` passed to all - /// I/O operations that are registered with mio's event loop. When the I/O - /// operation associated with an `OVERLAPPED` pointer completes the event - /// loop will invoke the function pointer provided by `cb`. - pub fn new(cb: fn(&OVERLAPPED_ENTRY)) -> Overlapped { - Overlapped { - inner: UnsafeCell::new(miow::Overlapped::zero()), - callback: cb, - } - } - - /// Get the underlying `Overlapped` instance as a raw pointer. - /// - /// This can be useful when only a shared borrow is held and the overlapped - /// pointer needs to be passed down to winapi. - pub fn as_mut_ptr(&self) -> *mut OVERLAPPED { - unsafe { (*self.inner.get()).raw() } - } -} - -impl fmt::Debug for Overlapped { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Overlapped").finish() - } -} - -// Overlapped's APIs are marked as unsafe Overlapped's APIs are marked as -// unsafe as they must be used with caution to ensure thread safety. The -// structure itself is safe to send across threads. -unsafe impl Send for Overlapped {} -unsafe impl Sync for Overlapped {} diff --git a/src/sys/windows/overlapped.rs b/src/sys/windows/overlapped.rs new file mode 100644 index 000000000..4f1f66c13 --- /dev/null +++ b/src/sys/windows/overlapped.rs @@ -0,0 +1,64 @@ +use std::cell::UnsafeCell; +use std::fmt; + +use winapi::um::minwinbase::OVERLAPPED_ENTRY; +#[cfg(feature = "os-util")] +use winapi::um::minwinbase::OVERLAPPED; + +// See sys::windows module docs for why this exists. +// +// The gist of it is that `Selector` assumes that all `OVERLAPPED` pointers are +// actually inside one of these structures so it can use the `Callback` stored +// right after it. +// +// We use repr(C) here to ensure that we can assume the overlapped pointer is +// at the start of the structure so we can just do a cast. +/// A wrapper around an internal instance over `miow::Overlapped` which is in +/// turn a wrapper around the Windows type `OVERLAPPED`. +/// +/// This type is required to be used for all IOCP operations on handles that are +/// registered with an event loop. The event loop will receive notifications +/// over `OVERLAPPED` pointers that have completed, and it will cast that +/// pointer to a pointer to this structure and invoke the associated callback. +#[repr(C)] +pub(crate) struct Overlapped { + inner: UnsafeCell, + pub(crate) callback: fn(&OVERLAPPED_ENTRY), +} + +#[cfg(feature = "os-util")] +impl Overlapped { + /// Creates a new `Overlapped` which will invoke the provided `cb` callback + /// whenever it's triggered. + /// + /// The returned `Overlapped` must be used as the `OVERLAPPED` passed to all + /// I/O operations that are registered with mio's event loop. When the I/O + /// operation associated with an `OVERLAPPED` pointer completes the event + /// loop will invoke the function pointer provided by `cb`. + pub fn new(cb: fn(&OVERLAPPED_ENTRY)) -> Overlapped { + Overlapped { + inner: UnsafeCell::new(miow::Overlapped::zero()), + callback: cb, + } + } + + /// Get the underlying `Overlapped` instance as a raw pointer. + /// + /// This can be useful when only a shared borrow is held and the overlapped + /// pointer needs to be passed down to winapi. + pub fn as_mut_ptr(&self) -> *mut OVERLAPPED { + unsafe { (*self.inner.get()).raw() } + } +} + +impl fmt::Debug for Overlapped { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Overlapped").finish() + } +} + +// Overlapped's APIs are marked as unsafe Overlapped's APIs are marked as +// unsafe as they must be used with caution to ensure thread safety. The +// structure itself is safe to send across threads. +unsafe impl Send for Overlapped {} +unsafe impl Sync for Overlapped {} diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index 69267328d..5acd35b48 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -498,7 +498,7 @@ impl SelectorInner { continue; } else if iocp_event.token() % 2 == 1 { // Handle is a named pipe. This could be extended to be any non-AFD event. - let callback = (*(iocp_event.overlapped() as *mut super::named_pipe::Overlapped)).callback; + let callback = (*(iocp_event.overlapped() as *mut super::Overlapped)).callback; callback(iocp_event.entry()); continue; @@ -697,7 +697,7 @@ impl Drop for SelectorInner { } else if iocp_event.token() % 2 == 1 { // Named pipe, dispatch the event so it can release resources let callback = unsafe { - (*(iocp_event.overlapped() as *mut super::named_pipe::Overlapped)).callback + (*(iocp_event.overlapped() as *mut super::Overlapped)).callback }; callback(iocp_event.entry()); From 1a32f46ea81588a183bc0af6d003ee34e396540e Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 30 Sep 2020 21:12:59 -0700 Subject: [PATCH 10/22] More windows work --- src/sys/windows/event.rs | 28 ++++ src/sys/windows/named_pipe.rs | 244 ++++++++++++++++++--------- src/sys/windows/overlapped.rs | 6 +- src/sys/windows/selector.rs | 20 +-- src/sys/windows/waker.rs | 11 +- tests/win_named_pipe.rs | 304 +++++++++++++++++++--------------- 6 files changed, 380 insertions(+), 233 deletions(-) diff --git a/src/sys/windows/event.rs b/src/sys/windows/event.rs index b3412551d..346a23253 100644 --- a/src/sys/windows/event.rs +++ b/src/sys/windows/event.rs @@ -14,6 +14,34 @@ pub fn token(event: &Event) -> Token { Token(event.data as usize) } +impl Event { + pub(super) fn new(token: Token) -> Event { + Event { + flags: 0, + data: usize::from(token) as u64, + } + } + + pub(super) fn set_readable(&mut self) { + self.flags |= READABLE_FLAGS + } + + pub(super) fn set_writable(&mut self) { + self.flags |= WRITABLE_FLAGS + } + + pub(super) fn from_completion_status(status: &CompletionStatus) -> Event { + Event { + flags: status.bytes_transferred(), + data: status.token() as u64, + } + } + + pub(super) fn to_completion_status(&self) -> CompletionStatus { + CompletionStatus::new(self.flags, self.data as usize, std::ptr::null_mut()) + } +} + pub(crate) const READABLE_FLAGS: u32 = afd::POLL_RECEIVE | afd::POLL_DISCONNECT | afd::POLL_ACCEPT diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 7aeb11d97..8f48b5d9b 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -48,22 +48,21 @@ //! `FILE_FLAG_OVERLAPPED` flag when opening the `File`. use crate::{poll, Registry}; -use crate::sys::windows::Overlapped; +use crate::event::Source; +use crate::sys::windows::{Event, Overlapped}; use std::ffi::OsStr; use std::fmt; -use std::io::{self, Read}; +use std::io::{self, Read, Write}; use std::mem; use std::os::windows::io::*; use std::slice; use std::sync::atomic::{AtomicUsize, AtomicBool}; use std::sync::atomic::Ordering::{Relaxed, SeqCst}; use std::sync::{Arc, Mutex}; -use std::task::{Context, Poll, Waker}; -// use mio::windows; -// use mio::{Evented, Poll, PollOpt, Ready, Registration, SetReadiness, Token}; -use miow::iocp::CompletionStatus; +use crate::{Interest, Token}; +use miow::iocp::{CompletionPort, CompletionStatus}; use miow::pipe; use winapi::shared::winerror::*; use winapi::um::ioapiset::*; @@ -83,10 +82,6 @@ macro_rules! overlapped2arc { }) } -fn would_block() -> io::Error { - io::ErrorKind::WouldBlock.into() -} - /// Representation of a named pipe on Windows. /// /// This structure internally contains a `HANDLE` which represents the named @@ -112,10 +107,14 @@ struct Inner { } struct Io { + // Uniquely identifies the selector associated with this named pipe + cp: Option>, + // Token used to identify events + token: Option, read: State, - read_waker: Option, + read_interest: bool, write: State, - write_waker: Option, + write_interest: bool, connect_error: Option, } @@ -137,27 +136,25 @@ fn _assert_kinds() { _assert_sync::(); } +fn would_block() -> io::Error { + io::ErrorKind::WouldBlock.into() +} + impl NamedPipe { /// Creates a new named pipe at the specified `addr` given a "reasonable /// set" of initial configuration options. pub fn new>( addr: A, - registry: &Registry, ) -> io::Result { let pipe = pipe::NamedPipe::new(addr)?; - NamedPipe::from_raw_handle(pipe.into_raw_handle(), registry) + Ok(NamedPipe::from_raw_handle(pipe.into_raw_handle())) } /// TODO: Dox pub fn from_raw_handle( handle: RawHandle, - registry: &Registry, - ) -> io::Result { - // Generate a token - let token = NEXT_TOKEN.fetch_add(2, Relaxed) + 2; - - // Create the pipe - let pipe = NamedPipe { + ) -> NamedPipe { + NamedPipe { inner: Arc::new(Inner { // Safety: not really unsafe handle: unsafe { pipe::NamedPipe::from_raw_handle(handle) }, @@ -168,26 +165,17 @@ impl NamedPipe { read: Overlapped::new(read_done), write: Overlapped::new(write_done), io: Mutex::new(Io { + cp: None, + token: None, read: State::None, - read_waker: None, + read_interest: false, write: State::None, - write_waker: None, + write_interest: false, connect_error: None, }), pool: Mutex::new(BufferPool::with_capacity(2)), }), - }; - - // Register the handle w/ the IOCP handle - poll::selector(registry) - .inner - .cp - .add_handle(usize::from(token), &pipe.inner.handle)?; - - // Queue the initial read - Inner::post_register(&pipe.inner); - - Ok(pipe) + } } /// Attempts to call `ConnectNamedPipe`, if possible. @@ -230,7 +218,7 @@ impl NamedPipe { // reads/writes and such. Ok(true) => { self.inner.connecting.store(false, SeqCst); - Inner::post_register(&self.inner); + Inner::post_register(&self.inner, None); Ok(()) } @@ -279,23 +267,48 @@ impl NamedPipe { pub fn disconnect(&self) -> io::Result<()> { self.inner.handle.disconnect() } +} +impl Read for NamedPipe { + fn read(&mut self, buf: &mut [u8]) -> io::Result { + <&NamedPipe as Read>::read(&mut &*self, buf) + } +} + +impl Write for NamedPipe { + fn write(&mut self, buf: &[u8]) -> io::Result { + <&NamedPipe as Write>::write(&mut &*self, buf) + } + + fn flush(&mut self) -> io::Result<()> { + <&NamedPipe as Write>::flush(&mut &*self) + } +} + +impl<'a> Read for &'a NamedPipe { /// TODO: dox - pub fn read(&mut self, cx: &mut Context<'_>, buf: &mut [u8]) -> Poll> { + fn read(&mut self, buf: &mut [u8]) -> io::Result { + println!("~~~"); let mut state = self.inner.io.lock().unwrap(); + + if state.token.is_none() { + println!("one"); + return Err(would_block()); + } + match mem::replace(&mut state.read, State::None) { - // In theory not possible with `ready_registration` checked above, + // In theory not possible with `token` checked above, // but return would block for now. State::None => { - state.read_waker = Some(cx.waker().clone()); - Poll::Pending + println!("two"); + Err(would_block()) } // A read is in flight, still waiting for it to finish State::Pending(buf, amt) => { state.read = State::Pending(buf, amt); - state.read_waker = Some(cx.waker().clone()); - Poll::Pending + println!("three"); + Err(would_block()) } // We previously read something into `data`, try to copy out some @@ -311,41 +324,93 @@ impl NamedPipe { state.read = State::Ok(data, next); } else { self.inner.put_buffer(data); - Inner::schedule_read(&self.inner, &mut state); + Inner::schedule_read(&self.inner, &mut state, None); } - Poll::Ready(Ok(n)) + Ok(n) } // Looks like an in-flight read hit an error, return that here while // we schedule a new one. State::Err(e) => { - Inner::schedule_read(&self.inner, &mut state); + Inner::schedule_read(&self.inner, &mut state, None); if e.raw_os_error() == Some(ERROR_BROKEN_PIPE as i32) { - Poll::Ready(Ok(0)) + Ok(0) } else { - Poll::Ready(Err(e)) + println!("four"); + Err(e) } } } } +} - /// TODO: dox - pub fn write(&mut self, cx: &mut Context<'_>, buf: &[u8]) -> Poll> { +impl<'a> Write for &'a NamedPipe { + fn write(&mut self, buf: &[u8]) -> io::Result { // Make sure there's no writes pending let mut io = self.inner.io.lock().unwrap(); + + if io.token.is_none() { + return Err(would_block()); + } + match io.write { State::None => {} _ => { - io.write_waker = Some(cx.waker().clone()); - return Poll::Pending; + return Err(would_block()); } } // Move `buf` onto the heap and fire off the write let mut owned_buf = self.inner.get_buffer(); owned_buf.extend(buf); - Inner::schedule_write(&self.inner, owned_buf, 0, &mut io); - Poll::Ready(Ok(buf.len())) + Inner::schedule_write(&self.inner, owned_buf, 0, &mut io, None); + Ok(buf.len()) + } + + fn flush(&mut self) -> io::Result<()> { + Ok(()) + } +} + +impl Source for NamedPipe { + fn register(&mut self, registry: &Registry, token: Token, interest: Interest) -> io::Result<()> { + let mut io = self.inner.io.lock().unwrap(); + + if io.cp.is_none() { + io.cp = Some(poll::selector(registry).clone_port()); + + let inner_token = NEXT_TOKEN.fetch_add(2, Relaxed) + 2; + poll::selector(registry).inner.cp.add_handle(inner_token, &self.inner.handle)?; + } + + io.token = Some(token); + io.read_interest = interest.is_readable(); + io.write_interest = interest.is_writable(); + drop(io); + + Inner::post_register(&self.inner, None); + + Ok(()) + } + + fn reregister(&mut self, _registry: &Registry, token: Token, interest: Interest) -> io::Result<()> { + let mut io = self.inner.io.lock().unwrap(); + + io.token = Some(token); + io.read_interest = interest.is_readable(); + io.write_interest = interest.is_writable(); + drop(io); + + Inner::post_register(&self.inner, None); + + Ok(()) + } + + fn deregister(&mut self, _registry: &Registry) -> io::Result<()> { + let mut io = self.inner.io.lock().unwrap(); + + io.token = None; + Ok(()) } } @@ -390,7 +455,7 @@ impl Inner { /// is scheduled in the background. If the pipe is no longer connected /// (ERROR_PIPE_LISTENING) then `false` is returned and no read is /// scheduled. - fn schedule_read(me: &Arc, io: &mut Io) -> bool { + fn schedule_read(me: &Arc, io: &mut Io, events: Option<&mut Vec>) -> bool { // Check to see if a read is already scheduled/completed match io.read { State::None => {} @@ -421,15 +486,13 @@ impl Inner { // out the error. Err(e) => { io.read = State::Err(e); - if let Some(waker) = io.read_waker.take() { - waker.wake(); - } + io.notify_readable(events); true } } } - fn schedule_write(me: &Arc, buf: Vec, pos: usize, io: &mut Io) { + fn schedule_write(me: &Arc, buf: Vec, pos: usize, io: &mut Io, events: Option<&mut Vec>) { // Very similar to `schedule_read` above, just done for the write half. let e = unsafe { let overlapped = me.write.as_mut_ptr() as *mut _; @@ -444,20 +507,16 @@ impl Inner { } Err(e) => { io.write = State::Err(e); - if let Some(waker) = io.write_waker.take() { - waker.wake(); - } + io.notify_writable(events); } } } - fn post_register(me: &Arc) { + fn post_register(me: &Arc, mut events: Option<&mut Vec>) { let mut io = me.io.lock().unwrap(); - if Inner::schedule_read(&me, &mut io) { + if Inner::schedule_read(&me, &mut io, events.as_mut().map(|ptr| &mut **ptr)) { if let State::None = io.write { - if let Some(waker) = io.write_waker.take() { - waker.wake(); - } + io.notify_writable(events); } } } @@ -480,7 +539,8 @@ unsafe fn cancel(handle: &T, overlapped: &Overlapped) -> io::Res } } -fn connect_done(status: &OVERLAPPED_ENTRY) { +fn connect_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { + println!(" + connect_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that @@ -503,10 +563,11 @@ fn connect_done(status: &OVERLAPPED_ENTRY) { // We essentially just finished a registration, so kick off a // read and register write readiness. - Inner::post_register(&me); + Inner::post_register(&me, events); } -fn read_done(status: &OVERLAPPED_ENTRY) { +fn read_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { + println!(" + read_done"); let status = CompletionStatus::from_entry(status); // Acquire the `FromRawArc`. Note that we should be guaranteed that @@ -535,12 +596,11 @@ fn read_done(status: &OVERLAPPED_ENTRY) { } // Flag our readiness that we've got data. - if let Some(waker) = io.read_waker.take() { - waker.wake(); - } + io.notify_readable(events); } -fn write_done(status: &OVERLAPPED_ENTRY) { +fn write_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { + println!(" + write_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that @@ -563,19 +623,43 @@ fn write_done(status: &OVERLAPPED_ENTRY) { let new_pos = pos + (status.bytes_transferred() as usize); if new_pos == buf.len() { me.put_buffer(buf); - if let Some(waker) = io.write_waker.take() { - waker.wake(); - } + io.notify_writable(events); } else { - Inner::schedule_write(&me, buf, new_pos, &mut io); + Inner::schedule_write(&me, buf, new_pos, &mut io, events); } } Err(e) => { debug_assert_eq!(status.bytes_transferred(), 0); io.write = State::Err(e); - if let Some(waker) = io.write_waker.take() { - waker.wake(); - } + io.notify_writable(events); + } + } + } +} + +impl Io { + fn notify_readable(&self, events: Option<&mut Vec>) { + if let Some(token) = self.token { + let mut ev = Event::new(token); + ev.set_readable(); + + if let Some(events) = events { + events.push(ev); + } else { + let _ = self.cp.as_ref().unwrap().post(ev.to_completion_status()); + } + } + } + + fn notify_writable(&self, events: Option<&mut Vec>) { + if let Some(token) = self.token { + let mut ev = Event::new(token); + ev.set_writable(); + + if let Some(events) = events { + events.push(ev); + } else { + let _ = self.cp.as_ref().unwrap().post(ev.to_completion_status()); } } } diff --git a/src/sys/windows/overlapped.rs b/src/sys/windows/overlapped.rs index 4f1f66c13..102de9775 100644 --- a/src/sys/windows/overlapped.rs +++ b/src/sys/windows/overlapped.rs @@ -1,3 +1,5 @@ +use crate::sys::windows::Event; + use std::cell::UnsafeCell; use std::fmt; @@ -23,7 +25,7 @@ use winapi::um::minwinbase::OVERLAPPED; #[repr(C)] pub(crate) struct Overlapped { inner: UnsafeCell, - pub(crate) callback: fn(&OVERLAPPED_ENTRY), + pub(crate) callback: fn(&OVERLAPPED_ENTRY, Option<&mut Vec>), } #[cfg(feature = "os-util")] @@ -35,7 +37,7 @@ impl Overlapped { /// I/O operations that are registered with mio's event loop. When the I/O /// operation associated with an `OVERLAPPED` pointer completes the event /// loop will invoke the function pointer provided by `cb`. - pub fn new(cb: fn(&OVERLAPPED_ENTRY)) -> Overlapped { + pub fn new(cb: fn(&OVERLAPPED_ENTRY, Option<&mut Vec>)) -> Overlapped { Overlapped { inner: UnsafeCell::new(miow::Overlapped::zero()), callback: cb, diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index 5acd35b48..950ccdbe3 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -8,7 +8,6 @@ use crate::sys::Events; use crate::Interest; use miow::iocp::{CompletionPort, CompletionStatus}; -use miow::Overlapped; use std::collections::VecDeque; use std::marker::PhantomPinned; use std::os::windows::io::RawSocket; @@ -18,18 +17,13 @@ use std::sync::atomic::AtomicUsize; use std::sync::atomic::{AtomicBool, Ordering}; use std::sync::{Arc, Mutex}; use std::time::Duration; -use std::{io, ptr}; +use std::io; use winapi::shared::ntdef::NT_SUCCESS; use winapi::shared::ntdef::{HANDLE, PVOID}; use winapi::shared::ntstatus::STATUS_CANCELLED; use winapi::shared::winerror::{ERROR_INVALID_HANDLE, ERROR_IO_PENDING, WAIT_TIMEOUT}; use winapi::um::minwinbase::OVERLAPPED; -/// Overlapped value to indicate a `Waker` event. -// -// Note: this must be null, `SelectorInner::feed_events` depends on it. -pub const WAKER_OVERLAPPED: *mut Overlapped = ptr::null_mut(); - #[derive(Debug)] struct AfdGroup { cp: Arc, @@ -489,18 +483,14 @@ impl SelectorInner { let mut update_queue = self.update_queue.lock().unwrap(); for iocp_event in iocp_events.iter() { if iocp_event.overlapped().is_null() { - // `Waker` event, we'll add a readable event to match the other platforms. - events.push(Event { - flags: afd::POLL_RECEIVE, - data: iocp_event.token() as u64, - }); + events.push(Event::from_completion_status(iocp_event)); n += 1; continue; } else if iocp_event.token() % 2 == 1 { // Handle is a named pipe. This could be extended to be any non-AFD event. let callback = (*(iocp_event.overlapped() as *mut super::Overlapped)).callback; - callback(iocp_event.entry()); + callback(iocp_event.entry(), Some(events)); continue; } @@ -693,14 +683,14 @@ impl Drop for SelectorInner { events_num = iocp_events.iter().len(); for iocp_event in iocp_events.iter() { if iocp_event.overlapped().is_null() { - // Waker + // Custom event } else if iocp_event.token() % 2 == 1 { // Named pipe, dispatch the event so it can release resources let callback = unsafe { (*(iocp_event.overlapped() as *mut super::Overlapped)).callback }; - callback(iocp_event.entry()); + callback(iocp_event.entry(), None); } else { // drain sock state to release memory of Arc reference let _sock_state = from_overlapped(iocp_event.overlapped()); diff --git a/src/sys/windows/waker.rs b/src/sys/windows/waker.rs index 1b21cf148..ab12c3c68 100644 --- a/src/sys/windows/waker.rs +++ b/src/sys/windows/waker.rs @@ -1,8 +1,8 @@ -use crate::sys::windows::selector::WAKER_OVERLAPPED; +use crate::sys::windows::Event; use crate::sys::windows::Selector; use crate::Token; -use miow::iocp::{CompletionPort, CompletionStatus}; +use miow::iocp::CompletionPort; use std::io; use std::sync::Arc; @@ -21,8 +21,9 @@ impl Waker { } pub fn wake(&self) -> io::Result<()> { - // Keep NULL as Overlapped value to notify waking. - let status = CompletionStatus::new(0, self.token.0, WAKER_OVERLAPPED); - self.port.post(status) + let mut ev = Event::new(self.token); + ev.set_readable(); + + self.port.post(ev.to_completion_status()) } } diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index 8e33cfb72..c307a844a 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -1,13 +1,12 @@ #![cfg(windows)] use std::fs::OpenOptions; -use std::io; +use std::io::{self, Read, Write}; use std::os::windows::fs::*; use std::os::windows::io::*; -use std::task::{Context, Poll}; use std::time::Duration; -// use mio::{Events, Poll, PollOpt, Ready, Token}; +use mio::{Events, Poll, Interest, Token}; use mio::windows::NamedPipe; use rand::Rng; use winapi::um::winbase::*; @@ -23,91 +22,99 @@ macro_rules! t { }; } -fn server(registry: &mio::Registry) -> (NamedPipe, String) { +fn server() -> (NamedPipe, String) { let num: u64 = rand::thread_rng().gen(); let name = format!(r"\\.\pipe\my-pipe-{}", num); - let pipe = t!(NamedPipe::new(&name, registry)); + let pipe = t!(NamedPipe::new(&name)); (pipe, name) } -fn client(name: &str, registry: &mio::Registry) -> NamedPipe { +fn client(name: &str) -> NamedPipe { let mut opts = OpenOptions::new(); opts.read(true) .write(true) .custom_flags(FILE_FLAG_OVERLAPPED); let file = t!(opts.open(name)); - t!(NamedPipe::from_raw_handle(file.into_raw_handle(), registry,)) + NamedPipe::from_raw_handle(file.into_raw_handle()) } -fn pipe(registry: &mio::Registry) -> (NamedPipe, NamedPipe) { - let (pipe, name) = server(registry); - (pipe, client(&name, registry)) +fn pipe() -> (NamedPipe, NamedPipe) { + let (pipe, name) = server(); + (pipe, client(&name)) } static data: &[u8] = &[100; 4096]; #[test] fn writable_after_register() { - let mut poll = t!(mio::Poll::new()); - let (mut server, mut client) = pipe(poll.registry()); - let mut events = mio::Events::with_capacity(128); - - let (wk1, cnt1) = new_count_waker(); - let mut cx1 = Context::from_waker(&wk1); - let (wk2, cnt2) = new_count_waker(); - let mut cx2 = Context::from_waker(&wk2); - - let mut dst = [0; 1024]; - - t!(server.connect()); - - // Server is writable - let res = server.write(&mut cx2, b"hello"); - assert!(res.is_ready()); - - // Server is **not** readable - assert!(server.read(&mut cx2, &mut dst).is_pending()); + let (mut server, mut client) = pipe(); + let mut poll = t!(Poll::new()); + t!(poll.registry().register( + &mut server, + Token(0), + Interest::WRITABLE | Interest::READABLE, + )); + t!(poll.registry().register(&mut client, Token(1), Interest::WRITABLE)); + + let mut events = Events::with_capacity(128); + t!(poll.poll(&mut events, None)); + + let events = events.iter().collect::>(); + assert!(events + .iter() + .any(|e| { e.token() == Token(0) && e.is_writable() })); + assert!(events + .iter() + .any(|e| { e.token() == Token(1) && e.is_writable() })); +} - // Client is writable - let res = client.write(&mut cx1, b"hello"); - assert!(res.is_ready()); +#[test] +fn write_then_read() { + let (mut server, mut client) = pipe(); + let mut poll = t!(Poll::new()); + t!(poll.registry().register( + &mut server, + Token(0), + Interest::READABLE | Interest::WRITABLE, + )); + t!(poll.registry().register( + &mut client, + Token(1), + Interest::READABLE | Interest::WRITABLE, + )); + + let mut events = Events::with_capacity(128); + t!(poll.poll(&mut events, None)); + + assert_eq!(t!(client.write(b"1234")), 4); - // Saturate the client loop { - if client.write(&mut cx1, data).is_pending() { - break; - } - } - - // Wait for readable - while cnt2.get() == 0 { t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.is_readable() { + break; + } + } } - // Read some data - let mut n = 0; - - while server.read(&mut cx2, &mut dst).is_ready() { - n += 1; - } - - assert!(n > 0); - - // Wait for the write side to be notified - while cnt1.get() == 0 { - t!(poll.poll(&mut events, None)); - } + let mut buf = [0; 10]; + assert_eq!(t!(server.read(&mut buf)), 4); + assert_eq!(&buf[..4], b"1234"); } #[test] fn connect_before_client() { - let mut poll = t!(mio::Poll::new()); - - let (server, name) = server(poll.registry()); - - let mut events = mio::Events::with_capacity(128); + let (mut server, name) = server(); + let mut poll = t!(Poll::new()); + t!(poll.registry().register( + &mut server, + Token(0), + Interest::READABLE | Interest::WRITABLE, + )); + + let mut events = Events::with_capacity(128); t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); - let e = events.iter().collect::>(); assert_eq!(e.len(), 0); assert_eq!( @@ -115,97 +122,123 @@ fn connect_before_client() { io::ErrorKind::WouldBlock ); - let mut client = client(&name, poll.registry()); - let (wk, cnt) = new_count_waker(); - let mut cx = Context::from_waker(&wk); - - assert!(client.write(&mut cx, b"hello").is_ready()); + let mut client = client(&name); + t!(poll.registry().register( + &mut client, + Token(1), + Interest::READABLE | Interest::WRITABLE, + )); + loop { + t!(poll.poll(&mut events, None)); + let e = events.iter().collect::>(); + if let Some(event) = e.iter().find(|e| e.token() == Token(0)) { + if event.is_writable() { + break; + } + } + } } #[test] fn connect_after_client() { - let mut poll = t!(mio::Poll::new()); - - let (server, name) = server(poll.registry()); - - let mut events = mio::Events::with_capacity(128); + let (mut server, name) = server(); + let mut poll = t!(Poll::new()); + t!(poll.registry().register( + &mut server, + Token(0), + Interest::READABLE | Interest::WRITABLE, + )); + + let mut events = Events::with_capacity(128); t!(poll.poll(&mut events, Some(Duration::new(0, 0)))); - let e = events.iter().collect::>(); assert_eq!(e.len(), 0); - let mut client = client(&name, poll.registry()); - let (wk, cnt) = new_count_waker(); - let mut cx = Context::from_waker(&wk); - - assert!(server.connect().is_ok()); - - assert!(client.write(&mut cx, b"hello").is_ready()); + let mut client = client(&name); + t!(poll.registry().register( + &mut client, + Token(1), + Interest::READABLE | Interest::WRITABLE, + )); + t!(server.connect()); + loop { + t!(poll.poll(&mut events, None)); + let e = events.iter().collect::>(); + if let Some(event) = e.iter().find(|e| e.token() == Token(0)) { + if event.is_writable() { + break; + } + } + } } #[test] fn write_then_drop() { - let mut poll = t!(mio::Poll::new()); - let (mut server, mut client) = pipe(poll.registry()); - - let (wk1, cnt1) = new_count_waker(); - let mut cx1 = Context::from_waker(&wk1); - let (wk2, cnt2) = new_count_waker(); - let mut cx2 = Context::from_waker(&wk2); - - t!(server.connect()); - - let mut dst = [0; 1024]; - - assert!(server.read(&mut cx2, &mut dst).is_pending()); - - match client.write(&mut cx1, b"1234") { - Poll::Ready(res) => assert_eq!(t!(res), 4), - _ => panic!(), - } - + let (mut server, mut client) = pipe(); + let mut poll = t!(Poll::new()); + t!(poll.registry().register( + &mut server, + Token(0), + Interest::READABLE | Interest::WRITABLE, + )); + t!(poll.registry().register( + &mut client, + Token(1), + Interest::READABLE | Interest::WRITABLE, + )); + assert_eq!(t!(client.write(b"1234")), 4); drop(client); - let mut events = mio::Events::with_capacity(128); + let mut events = Events::with_capacity(128); - // Wait for readable - while cnt2.get() == 0 { + loop { t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.is_readable() { + break; + } + } } - match server.read(&mut cx2, &mut dst) { - Poll::Ready(res) => assert_eq!(t!(res), 4), - _ => panic!(), - } - - assert_eq!(&dst[..4], b"1234"); + let mut buf = [0; 10]; + assert_eq!(t!(server.read(&mut buf)), 4); + assert_eq!(&buf[..4], b"1234"); } #[test] fn connect_twice() { - let mut poll = t!(mio::Poll::new()); - - let (mut server, name) = server(poll.registry()); - let mut c1 = client(&name, poll.registry()); - - let (wk1, cnt1) = new_count_waker(); - let mut cx1 = Context::from_waker(&wk1); - - let mut dst = [0; 1024]; - assert!(server.read(&mut cx1, &mut dst).is_pending()); - - t!(server.connect()); - + let (mut server, name) = server(); + let mut c1 = client(&name); + let mut poll = t!(Poll::new()); + t!(poll.registry().register( + &mut server, + Token(0), + Interest::READABLE | Interest::WRITABLE, + )); + t!(poll.registry().register( + &mut c1, + Token(1), + Interest::READABLE | Interest::WRITABLE, + )); drop(c1); - let mut events = mio::Events::with_capacity(128); - while cnt1.get() == 0 { - t!(poll.poll(&mut events, None)); - } + let mut events = Events::with_capacity(128); - match server.read(&mut cx1, &mut dst) { - Poll::Ready(Ok(0)) => {} - res => panic!("{:?}", res), + loop { + t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.is_readable() { + let mut buf = [0; 10]; + + match server.read(&mut buf) { + Err(ref e) if e.kind() == io::ErrorKind::WouldBlock => continue, + Ok(0) => break, + res => panic!("{:?}", res), + } + } + } } t!(server.disconnect()); @@ -214,11 +247,20 @@ fn connect_twice() { io::ErrorKind::WouldBlock ); - assert_eq!( - server.connect().err().unwrap().kind(), - io::ErrorKind::WouldBlock - ); + let mut c2 = client(&name); + t!(poll.registry().register( + &mut c2, + Token(2), + Interest::READABLE | Interest::WRITABLE, + )); - assert!(server.write(&mut cx1, b"hello").is_ready()); - assert!(server.write(&mut cx1, b"hello").is_pending()); + loop { + t!(poll.poll(&mut events, None)); + let events = events.iter().collect::>(); + if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { + if event.is_writable() { + break; + } + } + } } From fc1ac50e920c31f1843a9effb4fdb0a5acbcbe77 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 30 Sep 2020 21:27:46 -0700 Subject: [PATCH 11/22] fix CI --- src/sys/windows/event.rs | 1 + tests/win_named_pipe.rs | 22 ++++++++++------------ 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/sys/windows/event.rs b/src/sys/windows/event.rs index 346a23253..d965fcef8 100644 --- a/src/sys/windows/event.rs +++ b/src/sys/windows/event.rs @@ -26,6 +26,7 @@ impl Event { self.flags |= READABLE_FLAGS } + #[cfg(feature = "os-util")] pub(super) fn set_writable(&mut self) { self.flags |= WRITABLE_FLAGS } diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index c307a844a..c280866ab 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -6,8 +6,8 @@ use std::os::windows::fs::*; use std::os::windows::io::*; use std::time::Duration; -use mio::{Events, Poll, Interest, Token}; use mio::windows::NamedPipe; +use mio::{Events, Interest, Poll, Token}; use rand::Rng; use winapi::um::winbase::*; @@ -54,7 +54,9 @@ fn writable_after_register() { Token(0), Interest::WRITABLE | Interest::READABLE, )); - t!(poll.registry().register(&mut client, Token(1), Interest::WRITABLE)); + t!(poll + .registry() + .register(&mut client, Token(1), Interest::WRITABLE)); let mut events = Events::with_capacity(128); t!(poll.poll(&mut events, None)); @@ -216,11 +218,9 @@ fn connect_twice() { Token(0), Interest::READABLE | Interest::WRITABLE, )); - t!(poll.registry().register( - &mut c1, - Token(1), - Interest::READABLE | Interest::WRITABLE, - )); + t!(poll + .registry() + .register(&mut c1, Token(1), Interest::READABLE | Interest::WRITABLE,)); drop(c1); let mut events = Events::with_capacity(128); @@ -248,11 +248,9 @@ fn connect_twice() { ); let mut c2 = client(&name); - t!(poll.registry().register( - &mut c2, - Token(2), - Interest::READABLE | Interest::WRITABLE, - )); + t!(poll + .registry() + .register(&mut c2, Token(2), Interest::READABLE | Interest::WRITABLE,)); loop { t!(poll.poll(&mut events, None)); From a78ac485da29889103999c58b0fa30105ede7cff Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 30 Sep 2020 21:54:35 -0700 Subject: [PATCH 12/22] tweaks --- Cargo.toml | 1 - src/sys/windows/named_pipe.rs | 170 +++++++++++++++------------------- tests/win_named_pipe.rs | 12 +-- 3 files changed, 81 insertions(+), 102 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 558bba9d4..6d0b02b5b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -49,7 +49,6 @@ ntapi = "0.3" [dev-dependencies] env_logger = { version = "0.6.2", default-features = false } net2 = "0.2.33" -futures-test = "0.3" rand = "0.4" [package.metadata.docs.rs] diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 8f48b5d9b..be352a734 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -1,61 +1,13 @@ -//! Windows named pipes bindings for mio. -//! -//! This crate implements bindings for named pipes for the mio crate. This -//! crate compiles on all platforms but only contains anything on Windows. -//! Currently this crate requires mio 0.6.2. -//! -//! On Windows, mio is implemented with an IOCP object at the heart of its -//! `Poll` implementation. For named pipes, this means that all I/O is done in -//! an overlapped fashion and the named pipes themselves are registered with -//! mio's internal IOCP object. Essentially, this crate is using IOCP for -//! bindings with named pipes. -//! -//! Note, though, that IOCP is a *completion* based model whereas mio expects a -//! *readiness* based model. As a result this crate, like with TCP objects in -//! mio, has internal buffering to translate the completion model to a readiness -//! model. This means that this crate is not a zero-cost binding over named -//! pipes on Windows, but rather approximates the performance of mio's TCP -//! implementation on Windows. -//! -//! # Trait implementations -//! -//! The `Read` and `Write` traits are implemented for `NamedPipe` and for -//! `&NamedPipe`. This represents that a named pipe can be concurrently read and -//! written to and also can be read and written to at all. Typically a named -//! pipe needs to be connected to a client before it can be read or written, -//! however. -//! -//! Note that for I/O operations on a named pipe to succeed then the named pipe -//! needs to be associated with an event loop. Until this happens all I/O -//! operations will return a "would block" error. -//! -//! # Managing connections -//! -//! The `NamedPipe` type supports a `connect` method to connect to a client and -//! a `disconnect` method to disconnect from that client. These two methods only -//! work once a named pipe is associated with an event loop. -//! -//! The `connect` method will succeed asynchronously and a completion can be -//! detected once the object receives a writable notification. -//! -//! # Named pipe clients -//! -//! Currently to create a client of a named pipe server then you can use the -//! `OpenOptions` type in the standard library to create a `File` that connects -//! to a named pipe. Afterwards you can use the `into_raw_handle` method coupled -//! with the `NamedPipe::from_raw_handle` method to convert that to a named pipe -//! that can operate asynchronously. Don't forget to pass the -//! `FILE_FLAG_OVERLAPPED` flag when opening the `File`. - use crate::{poll, Registry}; use crate::event::Source; use crate::sys::windows::{Event, Overlapped}; +use winapi::um::minwinbase::OVERLAPPED_ENTRY; use std::ffi::OsStr; use std::fmt; use std::io::{self, Read, Write}; use std::mem; -use std::os::windows::io::*; +use std::os::windows::io::{AsRawHandle, FromRawHandle, IntoRawHandle, RawHandle}; use std::slice; use std::sync::atomic::{AtomicUsize, AtomicBool}; use std::sync::atomic::Ordering::{Relaxed, SeqCst}; @@ -64,10 +16,13 @@ use std::sync::{Arc, Mutex}; use crate::{Interest, Token}; use miow::iocp::{CompletionPort, CompletionStatus}; use miow::pipe; -use winapi::shared::winerror::*; -use winapi::um::ioapiset::*; -use winapi::um::minwinbase::*; +use winapi::shared::winerror::{ERROR_BROKEN_PIPE, ERROR_PIPE_LISTENING}; +use winapi::um::ioapiset::CancelIoEx; +/// # Safety +/// +/// Only valid if the strict is annotated with `#[repr(C)]`. This is only used +/// with `Overlapped`, which is correctly annotated. macro_rules! offset_of { ($t:ty, $($field:ident).+) => ( &(*(0 as *const $t)).$($field).+ as *const _ as usize @@ -82,12 +37,48 @@ macro_rules! overlapped2arc { }) } -/// Representation of a named pipe on Windows. +/// Non-blocking windows named pipe. /// /// This structure internally contains a `HANDLE` which represents the named /// pipe, and also maintains state associated with the mio event loop and active /// I/O operations that have been scheduled to translate IOCP to a readiness /// model. +/// +/// Note, IOCP is a *completion* based model whereas mio is a *readiness* based +/// model. To bridge this, `NamedPipe` performs internal buffering. Writes are +/// written to an internal buffer and the buffer is submitted to IOCP. IOCP +/// reads are submitted using internal buffers and `NamedPipe::read` reads from +/// this internal buffer. +/// +/// # Trait implementations +/// +/// The `Read` and `Write` traits are implemented for `NamedPipe` and for +/// `&NamedPipe`. This represents that a named pipe can be concurrently read and +/// written to and also can be read and written to at all. Typically a named +/// pipe needs to be connected to a client before it can be read or written, +/// however. +/// +/// Note that for I/O operations on a named pipe to succeed then the named pipe +/// needs to be associated with an event loop. Until this happens all I/O +/// operations will return a "would block" error. +/// +/// # Managing connections +/// +/// The `NamedPipe` type supports a `connect` method to connect to a client and +/// a `disconnect` method to disconnect from that client. These two methods only +/// work once a named pipe is associated with an event loop. +/// +/// The `connect` method will succeed asynchronously and a completion can be +/// detected once the object receives a writable notification. +/// +/// # Named pipe clients +/// +/// Currently to create a client of a named pipe server then you can use the +/// `OpenOptions` type in the standard library to create a `File` that connects +/// to a named pipe. Afterwards you can use the `into_raw_handle` method coupled +/// with the `NamedPipe::from_raw_handle` method to convert that to a named pipe +/// that can operate asynchronously. Don't forget to pass the +/// `FILE_FLAG_OVERLAPPED` flag when opening the `File`. pub struct NamedPipe { inner: Arc, } @@ -147,35 +138,9 @@ impl NamedPipe { addr: A, ) -> io::Result { let pipe = pipe::NamedPipe::new(addr)?; - Ok(NamedPipe::from_raw_handle(pipe.into_raw_handle())) - } - - /// TODO: Dox - pub fn from_raw_handle( - handle: RawHandle, - ) -> NamedPipe { - NamedPipe { - inner: Arc::new(Inner { - // Safety: not really unsafe - handle: unsafe { pipe::NamedPipe::from_raw_handle(handle) }, - // transmutes to straddle winapi versions (mio 0.6 is on an - // older winapi) - connect: Overlapped::new(connect_done), - connecting: AtomicBool::new(false), - read: Overlapped::new(read_done), - write: Overlapped::new(write_done), - io: Mutex::new(Io { - cp: None, - token: None, - read: State::None, - read_interest: false, - write: State::None, - write_interest: false, - connect_error: None, - }), - pool: Mutex::new(BufferPool::with_capacity(2)), - }), - } + // Safety: nothing actually unsafe about this. The trait fn includes + // `unsafe`. + Ok(unsafe { NamedPipe::from_raw_handle(pipe.into_raw_handle()) }) } /// Attempts to call `ConnectNamedPipe`, if possible. @@ -269,6 +234,35 @@ impl NamedPipe { } } +impl FromRawHandle for NamedPipe { + unsafe fn from_raw_handle( + handle: RawHandle, + ) -> NamedPipe { + NamedPipe { + inner: Arc::new(Inner { + // Safety: not really unsafe + handle: pipe::NamedPipe::from_raw_handle(handle), + // transmutes to straddle winapi versions (mio 0.6 is on an + // older winapi) + connect: Overlapped::new(connect_done), + connecting: AtomicBool::new(false), + read: Overlapped::new(read_done), + write: Overlapped::new(write_done), + io: Mutex::new(Io { + cp: None, + token: None, + read: State::None, + read_interest: false, + write: State::None, + write_interest: false, + connect_error: None, + }), + pool: Mutex::new(BufferPool::with_capacity(2)), + }), + } + } +} + impl Read for NamedPipe { fn read(&mut self, buf: &mut [u8]) -> io::Result { <&NamedPipe as Read>::read(&mut &*self, buf) @@ -286,13 +280,10 @@ impl Write for NamedPipe { } impl<'a> Read for &'a NamedPipe { - /// TODO: dox fn read(&mut self, buf: &mut [u8]) -> io::Result { - println!("~~~"); let mut state = self.inner.io.lock().unwrap(); if state.token.is_none() { - println!("one"); return Err(would_block()); } @@ -300,14 +291,12 @@ impl<'a> Read for &'a NamedPipe { // In theory not possible with `token` checked above, // but return would block for now. State::None => { - println!("two"); Err(would_block()) } // A read is in flight, still waiting for it to finish State::Pending(buf, amt) => { state.read = State::Pending(buf, amt); - println!("three"); Err(would_block()) } @@ -336,7 +325,6 @@ impl<'a> Read for &'a NamedPipe { if e.raw_os_error() == Some(ERROR_BROKEN_PIPE as i32) { Ok(0) } else { - println!("four"); Err(e) } } @@ -540,7 +528,6 @@ unsafe fn cancel(handle: &T, overlapped: &Overlapped) -> io::Res } fn connect_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { - println!(" + connect_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that @@ -567,7 +554,6 @@ fn connect_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { } fn read_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { - println!(" + read_done"); let status = CompletionStatus::from_entry(status); // Acquire the `FromRawArc`. Note that we should be guaranteed that @@ -600,7 +586,6 @@ fn read_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { } fn write_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { - println!(" + write_done"); let status = CompletionStatus::from_entry(status); // Acquire the `Arc`. Note that we should be guaranteed that @@ -665,7 +650,6 @@ impl Io { } } -// Based on https://github.com/tokio-rs/mio/blob/13d5fc9/src/sys/windows/buffer_pool.rs struct BufferPool { pool: Vec>, } diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index c280866ab..7f2e2fd83 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -2,16 +2,14 @@ use std::fs::OpenOptions; use std::io::{self, Read, Write}; -use std::os::windows::fs::*; -use std::os::windows::io::*; +use std::os::windows::fs::OpenOptionsExt; +use std::os::windows::io::{FromRawHandle, IntoRawHandle}; use std::time::Duration; use mio::windows::NamedPipe; use mio::{Events, Interest, Poll, Token}; use rand::Rng; -use winapi::um::winbase::*; - -use futures_test::task::new_count_waker; +use winapi::um::winbase::FILE_FLAG_OVERLAPPED; macro_rules! t { ($e:expr) => { @@ -35,7 +33,7 @@ fn client(name: &str) -> NamedPipe { .write(true) .custom_flags(FILE_FLAG_OVERLAPPED); let file = t!(opts.open(name)); - NamedPipe::from_raw_handle(file.into_raw_handle()) + unsafe { NamedPipe::from_raw_handle(file.into_raw_handle()) } } fn pipe() -> (NamedPipe, NamedPipe) { @@ -43,8 +41,6 @@ fn pipe() -> (NamedPipe, NamedPipe) { (pipe, client(&name)) } -static data: &[u8] = &[100; 4096]; - #[test] fn writable_after_register() { let (mut server, mut client) = pipe(); From faa06e59d7e0701b205be97757432efc2d5637ed Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 30 Sep 2020 21:57:23 -0700 Subject: [PATCH 13/22] undo change --- src/sys/windows/selector.rs | 14 ++++++++++++-- tests/regressions.rs | 1 - 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index 950ccdbe3..71c1691f5 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -432,8 +432,18 @@ impl SelectorInner { pub fn select(&self, events: &mut Events, timeout: Option) -> io::Result<()> { events.clear(); - self.select2(&mut events.statuses, &mut events.events, timeout)?; - return Ok(()); + if timeout.is_none() { + loop { + let len = self.select2(&mut events.statuses, &mut events.events, None)?; + if len == 0 { + continue; + } + return Ok(()); + } + } else { + self.select2(&mut events.statuses, &mut events.events, timeout)?; + return Ok(()); + } } pub fn select2( diff --git a/tests/regressions.rs b/tests/regressions.rs index 6a86ce8e1..7f8428ce0 100644 --- a/tests/regressions.rs +++ b/tests/regressions.rs @@ -61,7 +61,6 @@ fn issue_776() { } #[test] -#[ignore] fn issue_1205() { let (mut poll, mut events) = init_with_poll(); From dd7ffb1b46f192cb283213e54b73f834f99ff0d8 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 30 Sep 2020 22:15:46 -0700 Subject: [PATCH 14/22] tweaks --- src/sys/windows/afd.rs | 5 ++++- src/sys/windows/named_pipe.rs | 24 ++++++++++++------------ src/sys/windows/overlapped.rs | 33 ++------------------------------- 3 files changed, 18 insertions(+), 44 deletions(-) diff --git a/src/sys/windows/afd.rs b/src/sys/windows/afd.rs index e819a5091..82c8e9ed7 100644 --- a/src/sys/windows/afd.rs +++ b/src/sys/windows/afd.rs @@ -194,7 +194,10 @@ cfg_net! { )); } let fd = File::from_raw_handle(afd_helper_handle as RawHandle); - // Increment by 2 to reserve space for other types of handles + // Increment by 2 to reserve space for other types of handles. + // Non-AFD types (currently only NamedPipe), use odd numbered + // tokens. This allows the selector to differentate between them + // and dispatch events accordingly. let token = NEXT_TOKEN.fetch_add(2, Ordering::Relaxed) + 2; let afd = Afd { fd }; cp.add_handle(token, &afd.fd)?; diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index be352a734..3192a5579 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -22,7 +22,7 @@ use winapi::um::ioapiset::CancelIoEx; /// # Safety /// /// Only valid if the strict is annotated with `#[repr(C)]`. This is only used -/// with `Overlapped`, which is correctly annotated. +/// with `Overlapped` and `Inner`, which are correctly annotated. macro_rules! offset_of { ($t:ty, $($field:ident).+) => ( &(*(0 as *const $t)).$($field).+ as *const _ as usize @@ -83,6 +83,7 @@ pub struct NamedPipe { inner: Arc, } +#[repr(C)] struct Inner { handle: pipe::NamedPipe, @@ -174,7 +175,7 @@ impl NamedPipe { // connection attempt. Afterwards interpret the return value and set // internal state accordingly. let res = unsafe { - let overlapped = self.inner.connect.as_mut_ptr() as *mut _; + let overlapped = self.inner.connect.as_ptr() as *mut _; self.inner.handle.connect_overlapped(overlapped) }; @@ -199,7 +200,6 @@ impl NamedPipe { Err(would_block()) } - // TODO: are we sure no IOCP notification comes in here? Err(e) => { self.inner.connecting.store(false, SeqCst); Err(e) @@ -223,9 +223,7 @@ impl NamedPipe { /// Disconnects this named pipe from a connected client. /// /// This function will disconnect the pipe from a connected client, if any, - /// transitively calling the `DisconnectNamedPipe` function. If the - /// disconnection is successful then this object will no longer be readable - /// or writable. + /// transitively calling the `DisconnectNamedPipe` function. /// /// After a `disconnect` is issued, then a `connect` may be called again to /// connect to another client. @@ -453,13 +451,13 @@ impl Inner { // Allocate a buffer and schedule the read. let mut buf = me.get_buffer(); let e = unsafe { - let overlapped = me.read.as_mut_ptr() as *mut _; + let overlapped = me.read.as_ptr() as *mut _; let slice = slice::from_raw_parts_mut(buf.as_mut_ptr(), buf.capacity()); me.handle.read_overlapped(slice, overlapped) }; match e { - // See `connect` above for the rationale behind `forget` + // See `NamedPipe::connect` above for the rationale behind `forget` Ok(_) => { io.read = State::Pending(buf, 0); // 0 is ignored on read side mem::forget(me.clone()); @@ -483,7 +481,7 @@ impl Inner { fn schedule_write(me: &Arc, buf: Vec, pos: usize, io: &mut Io, events: Option<&mut Vec>) { // Very similar to `schedule_read` above, just done for the write half. let e = unsafe { - let overlapped = me.write.as_mut_ptr() as *mut _; + let overlapped = me.write.as_ptr() as *mut _; me.handle.write_overlapped(&buf[pos..], overlapped) }; @@ -510,7 +508,7 @@ impl Inner { } fn get_buffer(&self) -> Vec { - self.pool.lock().unwrap().get(8 * 1024) + self.pool.lock().unwrap().get(4 * 1024) } fn put_buffer(&self, buf: Vec) { @@ -519,7 +517,9 @@ impl Inner { } unsafe fn cancel(handle: &T, overlapped: &Overlapped) -> io::Result<()> { - let ret = CancelIoEx(handle.as_raw_handle(), overlapped.as_mut_ptr() as *mut _); + let ret = CancelIoEx(handle.as_raw_handle(), overlapped.as_ptr() as *mut _); + // `CancelIoEx` returns 0 on error: + // https://docs.microsoft.com/en-us/windows/win32/fileio/cancelioex-func if ret == 0 { Err(io::Error::last_os_error()) } else { @@ -537,7 +537,7 @@ fn connect_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { // Flag ourselves as no longer using the `connect` overlapped instances. let prev = me.connecting.swap(false, SeqCst); - assert!(prev, "wasn't previously connecting"); + assert!(prev, "NamedPipe was not previously connecting"); // Stash away our connect error if one happened debug_assert_eq!(status.bytes_transferred(), 0); diff --git a/src/sys/windows/overlapped.rs b/src/sys/windows/overlapped.rs index 102de9775..3708f9ecb 100644 --- a/src/sys/windows/overlapped.rs +++ b/src/sys/windows/overlapped.rs @@ -7,21 +7,6 @@ use winapi::um::minwinbase::OVERLAPPED_ENTRY; #[cfg(feature = "os-util")] use winapi::um::minwinbase::OVERLAPPED; -// See sys::windows module docs for why this exists. -// -// The gist of it is that `Selector` assumes that all `OVERLAPPED` pointers are -// actually inside one of these structures so it can use the `Callback` stored -// right after it. -// -// We use repr(C) here to ensure that we can assume the overlapped pointer is -// at the start of the structure so we can just do a cast. -/// A wrapper around an internal instance over `miow::Overlapped` which is in -/// turn a wrapper around the Windows type `OVERLAPPED`. -/// -/// This type is required to be used for all IOCP operations on handles that are -/// registered with an event loop. The event loop will receive notifications -/// over `OVERLAPPED` pointers that have completed, and it will cast that -/// pointer to a pointer to this structure and invoke the associated callback. #[repr(C)] pub(crate) struct Overlapped { inner: UnsafeCell, @@ -30,25 +15,14 @@ pub(crate) struct Overlapped { #[cfg(feature = "os-util")] impl Overlapped { - /// Creates a new `Overlapped` which will invoke the provided `cb` callback - /// whenever it's triggered. - /// - /// The returned `Overlapped` must be used as the `OVERLAPPED` passed to all - /// I/O operations that are registered with mio's event loop. When the I/O - /// operation associated with an `OVERLAPPED` pointer completes the event - /// loop will invoke the function pointer provided by `cb`. - pub fn new(cb: fn(&OVERLAPPED_ENTRY, Option<&mut Vec>)) -> Overlapped { + pub(crate) fn new(cb: fn(&OVERLAPPED_ENTRY, Option<&mut Vec>)) -> Overlapped { Overlapped { inner: UnsafeCell::new(miow::Overlapped::zero()), callback: cb, } } - /// Get the underlying `Overlapped` instance as a raw pointer. - /// - /// This can be useful when only a shared borrow is held and the overlapped - /// pointer needs to be passed down to winapi. - pub fn as_mut_ptr(&self) -> *mut OVERLAPPED { + pub(crate) fn as_ptr(&self) -> *const OVERLAPPED { unsafe { (*self.inner.get()).raw() } } } @@ -59,8 +33,5 @@ impl fmt::Debug for Overlapped { } } -// Overlapped's APIs are marked as unsafe Overlapped's APIs are marked as -// unsafe as they must be used with caution to ensure thread safety. The -// structure itself is safe to send across threads. unsafe impl Send for Overlapped {} unsafe impl Sync for Overlapped {} From 48ac9b6780475ae61f4bf182152e103984998693 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Wed, 30 Sep 2020 22:16:58 -0700 Subject: [PATCH 15/22] move static test --- src/sys/windows/named_pipe.rs | 7 ------- tests/win_named_pipe.rs | 7 +++++++ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 3192a5579..77dc36641 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -121,13 +121,6 @@ enum State { // Odd tokens are for named pipes static NEXT_TOKEN: AtomicUsize = AtomicUsize::new(1); -fn _assert_kinds() { - fn _assert_send() {} - fn _assert_sync() {} - _assert_send::(); - _assert_sync::(); -} - fn would_block() -> io::Error { io::ErrorKind::WouldBlock.into() } diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index 7f2e2fd83..e67c9ec05 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -11,6 +11,13 @@ use mio::{Events, Interest, Poll, Token}; use rand::Rng; use winapi::um::winbase::FILE_FLAG_OVERLAPPED; +fn _assert_kinds() { + fn _assert_send() {} + fn _assert_sync() {} + _assert_send::(); + _assert_sync::(); +} + macro_rules! t { ($e:expr) => { match $e { From 12ea5ba835a64f0921578a6fd6054a609acc2ab0 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 1 Oct 2020 09:44:03 -0700 Subject: [PATCH 16/22] Fix CI hopefully --- src/sys/windows/selector.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index 71c1691f5..eb5c09e15 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -500,7 +500,9 @@ impl SelectorInner { // Handle is a named pipe. This could be extended to be any non-AFD event. let callback = (*(iocp_event.overlapped() as *mut super::Overlapped)).callback; - callback(iocp_event.entry(), Some(events)); + let len = events.len(); + callback(iocp_event.entry(), Some(events)); + n += events.len() - len; continue; } From b8d1ca89e7dc70b7082968df91b1a2a9f0fa448f Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 1 Oct 2020 10:15:41 -0700 Subject: [PATCH 17/22] verify association rules --- src/sys/windows/named_pipe.rs | 42 ++++++++++++++++++++++++++++++++-- src/sys/windows/selector.rs | 4 ++++ tests/win_named_pipe.rs | 43 +++++++++++++++++++++++++++++++++++ 3 files changed, 87 insertions(+), 2 deletions(-) diff --git a/src/sys/windows/named_pipe.rs b/src/sys/windows/named_pipe.rs index 77dc36641..a5688ce90 100644 --- a/src/sys/windows/named_pipe.rs +++ b/src/sys/windows/named_pipe.rs @@ -355,6 +355,15 @@ impl Source for NamedPipe { fn register(&mut self, registry: &Registry, token: Token, interest: Interest) -> io::Result<()> { let mut io = self.inner.io.lock().unwrap(); + io.check_association(registry, false)?; + + if io.token.is_some() { + return Err(io::Error::new( + io::ErrorKind::AlreadyExists, + "I/O source already registered with a `Registry`", + )); + } + if io.cp.is_none() { io.cp = Some(poll::selector(registry).clone_port()); @@ -372,9 +381,11 @@ impl Source for NamedPipe { Ok(()) } - fn reregister(&mut self, _registry: &Registry, token: Token, interest: Interest) -> io::Result<()> { + fn reregister(&mut self, registry: &Registry, token: Token, interest: Interest) -> io::Result<()> { let mut io = self.inner.io.lock().unwrap(); + io.check_association(registry, true)?; + io.token = Some(token); io.read_interest = interest.is_readable(); io.write_interest = interest.is_writable(); @@ -385,9 +396,18 @@ impl Source for NamedPipe { Ok(()) } - fn deregister(&mut self, _registry: &Registry) -> io::Result<()> { + fn deregister(&mut self, registry: &Registry) -> io::Result<()> { let mut io = self.inner.io.lock().unwrap(); + io.check_association(registry, true)?; + + if io.token.is_none() { + return Err(io::Error::new( + io::ErrorKind::NotFound, + "I/O source not registered with `Registry`", + )); + } + io.token = None; Ok(()) } @@ -616,6 +636,24 @@ fn write_done(status: &OVERLAPPED_ENTRY, events: Option<&mut Vec>) { } impl Io { + fn check_association(&self, registry: &Registry, required: bool) -> io::Result<()> { + match self.cp { + Some(ref cp) if !poll::selector(registry).same_port(cp) => { + Err(io::Error::new( + io::ErrorKind::AlreadyExists, + "I/O source already registered with a different `Registry`" + )) + } + None if required => { + Err(io::Error::new( + io::ErrorKind::NotFound, + "I/O source not registered with `Registry`" + )) + } + _ => Ok(()), + } + } + fn notify_readable(&self, events: Option<&mut Vec>) { if let Some(token) = self.token { let mut ev = Event::new(token); diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index eb5c09e15..551d31c2a 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -368,6 +368,10 @@ impl Selector { pub(super) fn clone_port(&self) -> Arc { self.inner.cp.clone() } + + pub(super) fn same_port(&self, other: &Arc) -> bool { + Arc::ptr_eq(&self.inner.cp, other) + } } cfg_net! { diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index e67c9ec05..d411b07bd 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -265,3 +265,46 @@ fn connect_twice() { } } } + +#[test] +fn reregister_deregister_before_register() { + let (mut pipe, _) = server(); + let poll = t!(Poll::new()); + + assert_eq!( + poll.registry().reregister( + &mut pipe, + Token(0), + Interest::READABLE, + ).unwrap_err().kind(), + io::ErrorKind::NotFound, + ); + + assert_eq!( + poll.registry().deregister(&mut pipe).unwrap_err().kind(), + io::ErrorKind::NotFound, + ); +} + +fn reregister_deregister_different_poll() { + let (mut pipe, _) = server(); + let poll1 = t!(Poll::new()); + let poll2 = t!(Poll::new()); + + // Register with 1 + t!(poll1.registry().register(&mut pipe, Token(0), Interest::READABLE)); + + assert_eq!( + poll2.registry().reregister( + &mut pipe, + Token(0), + Interest::READABLE, + ).unwrap_err().kind(), + io::ErrorKind::AlreadyExists, + ); + + assert_eq!( + poll2.registry().deregister(&mut pipe).unwrap_err().kind(), + io::ErrorKind::NotFound, + ); +} From 2bd83b4f552e5296fc4e000d977a1d1e180bb113 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 1 Oct 2020 10:19:57 -0700 Subject: [PATCH 18/22] fmt --- tests/win_named_pipe.rs | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index d411b07bd..ff6422550 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -272,11 +272,10 @@ fn reregister_deregister_before_register() { let poll = t!(Poll::new()); assert_eq!( - poll.registry().reregister( - &mut pipe, - Token(0), - Interest::READABLE, - ).unwrap_err().kind(), + poll.registry() + .reregister(&mut pipe, Token(0), Interest::READABLE,) + .unwrap_err() + .kind(), io::ErrorKind::NotFound, ); @@ -292,14 +291,16 @@ fn reregister_deregister_different_poll() { let poll2 = t!(Poll::new()); // Register with 1 - t!(poll1.registry().register(&mut pipe, Token(0), Interest::READABLE)); + t!(poll1 + .registry() + .register(&mut pipe, Token(0), Interest::READABLE)); assert_eq!( - poll2.registry().reregister( - &mut pipe, - Token(0), - Interest::READABLE, - ).unwrap_err().kind(), + poll2 + .registry() + .reregister(&mut pipe, Token(0), Interest::READABLE,) + .unwrap_err() + .kind(), io::ErrorKind::AlreadyExists, ); From b74c557f2d3f82237c8d2984f73897ddafc52ca0 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 1 Oct 2020 10:27:11 -0700 Subject: [PATCH 19/22] fix feature build --- src/sys/windows/selector.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/sys/windows/selector.rs b/src/sys/windows/selector.rs index 551d31c2a..792a5c55a 100644 --- a/src/sys/windows/selector.rs +++ b/src/sys/windows/selector.rs @@ -369,6 +369,7 @@ impl Selector { self.inner.cp.clone() } + #[cfg(feature = "os-util")] pub(super) fn same_port(&self, other: &Arc) -> bool { Arc::ptr_eq(&self.inner.cp, other) } From 59bc378768f7ac57e4f8fc57336234c46c449c05 Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 1 Oct 2020 13:28:20 -0700 Subject: [PATCH 20/22] Fix event flags --- src/sys/windows/event.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sys/windows/event.rs b/src/sys/windows/event.rs index d965fcef8..235074a10 100644 --- a/src/sys/windows/event.rs +++ b/src/sys/windows/event.rs @@ -23,12 +23,12 @@ impl Event { } pub(super) fn set_readable(&mut self) { - self.flags |= READABLE_FLAGS + self.flags |= afd::POLL_RECEIVE } #[cfg(feature = "os-util")] pub(super) fn set_writable(&mut self) { - self.flags |= WRITABLE_FLAGS + self.flags |= afd::POLL_SEND; } pub(super) fn from_completion_status(status: &CompletionStatus) -> Event { From d0cdbf3e017bc46f1c4b0873947d7b3e9e591e9f Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 1 Oct 2020 14:15:55 -0700 Subject: [PATCH 21/22] fix tests --- tests/win_named_pipe.rs | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index ff6422550..c8f67c03c 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -196,12 +196,14 @@ fn write_then_drop() { let mut events = Events::with_capacity(128); + 'outer: loop { t!(poll.poll(&mut events, None)); let events = events.iter().collect::>(); - if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { - if event.is_readable() { - break; + + for event in &events { + if event.is_readable() && event.token() == Token(0) { + break 'outer; } } } @@ -255,12 +257,14 @@ fn connect_twice() { .registry() .register(&mut c2, Token(2), Interest::READABLE | Interest::WRITABLE,)); + 'outer: loop { t!(poll.poll(&mut events, None)); let events = events.iter().collect::>(); - if let Some(event) = events.iter().find(|e| e.token() == Token(0)) { - if event.is_writable() { - break; + + for event in &events { + if event.is_writable() && event.token() == Token(0) { + break 'outer; } } } @@ -285,6 +289,7 @@ fn reregister_deregister_before_register() { ); } +#[test] fn reregister_deregister_different_poll() { let (mut pipe, _) = server(); let poll1 = t!(Poll::new()); @@ -306,6 +311,6 @@ fn reregister_deregister_different_poll() { assert_eq!( poll2.registry().deregister(&mut pipe).unwrap_err().kind(), - io::ErrorKind::NotFound, + io::ErrorKind::AlreadyExists, ); } From e9e8fd3e95756df058e0655c6f244c23e73d906b Mon Sep 17 00:00:00 2001 From: Carl Lerche Date: Thu, 1 Oct 2020 14:28:42 -0700 Subject: [PATCH 22/22] fmt --- tests/win_named_pipe.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/tests/win_named_pipe.rs b/tests/win_named_pipe.rs index c8f67c03c..65ca3ea33 100644 --- a/tests/win_named_pipe.rs +++ b/tests/win_named_pipe.rs @@ -196,8 +196,7 @@ fn write_then_drop() { let mut events = Events::with_capacity(128); - 'outer: - loop { + 'outer: loop { t!(poll.poll(&mut events, None)); let events = events.iter().collect::>(); @@ -257,11 +256,10 @@ fn connect_twice() { .registry() .register(&mut c2, Token(2), Interest::READABLE | Interest::WRITABLE,)); - 'outer: - loop { + 'outer: loop { t!(poll.poll(&mut events, None)); let events = events.iter().collect::>(); - + for event in &events { if event.is_writable() && event.token() == Token(0) { break 'outer;