Browse Source

feat(mpsc): add waiting `send`/`send_ref` (#7)

This branch adds implementations of `Sender::send` and
`Sender::send_ref` methods for the MPSC channels. This allows a sender
to wait for capacity when the channel is full, rather than returning an
error.

This took a lot of work to get working, and there were some potential
deadlocks that had to be fixed. However, it's passing all the loom tests
now (finally)!

For the blocking version, some `loom` tests currently fail due to an
upstream issue (tokio-rs/loom#246). However, the async versions of the
same tests (which are not effected by the loom bug) do pass, so we can
assume that it's probably correct.

Currently, the senders queue themselves to wait using a spinlock around
a `VecDeque`, which I don't love at all. But, this works for now, and we
can consider switching to an intrusive list strategy later.

Signed-off-by: Eliza Weisman <[email protected]>
Eliza Weisman 3 years ago
parent
commit
76df064cbf
18 changed files with 1472 additions and 767 deletions
  1. 4 1
      Cargo.toml
  2. 104 49
      src/lib.rs
  3. 15 7
      src/loom.rs
  4. 15 0
      src/macros.rs
  5. 190 12
      src/mpsc.rs
  6. 46 46
      src/mpsc/async_impl.rs
  7. 42 37
      src/mpsc/sync.rs
  8. 126 5
      src/mpsc/tests/mpsc_async.rs
  9. 144 7
      src/mpsc/tests/mpsc_sync.rs
  10. 5 2
      src/static_thingbuf.rs
  11. 0 211
      src/sync_mpsc.rs
  12. 7 4
      src/thingbuf.rs
  13. 17 2
      src/util.rs
  14. 25 384
      src/util/wait.rs
  15. 388 0
      src/util/wait/wait_cell.rs
  16. 248 0
      src/util/wait/wait_queue.rs
  17. 45 0
      tests/mpsc_async.rs
  18. 51 0
      tests/mpsc_sync.rs

+ 4 - 1
Cargo.toml

@@ -21,7 +21,7 @@ futures = "0.3"
 criterion = { version = "0.3.5", features = ["async_tokio"] }
 # for comparison benchmarks
 crossbeam = "0.8.1"
-tokio = { version = "1.14.0", features = ["rt", "sync"] }
+tokio = { version = "1.14.0", features = ["rt", "rt-multi-thread", "sync", "macros"] }
 async-std = "1"
 
 [profile.test]
@@ -33,3 +33,6 @@ harness = false
 [[bench]]
 name = "async_mpsc"
 harness = false
+
+[patch.crates-io]
+loom = { git = "https://github.com/tokio-rs/loom", branch = "eliza/fix-double-panic-in-drop" }

+ 104 - 49
src/lib.rs

@@ -1,6 +1,6 @@
 #![cfg_attr(not(feature = "std"), no_std)]
 #![cfg_attr(docsrs, feature(doc_cfg))]
-use core::{fmt, mem::MaybeUninit, ops::Index};
+use core::{cmp, fmt, mem::MaybeUninit, ops::Index};
 
 #[macro_use]
 mod macros;
@@ -26,7 +26,7 @@ pub use self::static_thingbuf::StaticThingBuf;
 
 use crate::{
     loom::{
-        atomic::{AtomicUsize, Ordering},
+        atomic::{AtomicUsize, Ordering::*},
         UnsafeCell,
     },
     util::{Backoff, CachePadded},
@@ -46,6 +46,7 @@ struct Core {
     gen: usize,
     gen_mask: usize,
     idx_mask: usize,
+    closed: usize,
     capacity: usize,
 }
 
@@ -57,14 +58,16 @@ struct Slot<T> {
 impl Core {
     #[cfg(not(test))]
     const fn new(capacity: usize) -> Self {
-        let gen = (capacity + 1).next_power_of_two();
-        let idx_mask = gen - 1;
-        let gen_mask = !(gen - 1);
+        let closed = (capacity + 1).next_power_of_two();
+        let idx_mask = closed - 1;
+        let gen = closed << 1;
+        let gen_mask = !(closed | idx_mask);
         Self {
             head: CachePadded(AtomicUsize::new(0)),
             tail: CachePadded(AtomicUsize::new(0)),
             gen,
             gen_mask,
+            closed,
             idx_mask,
             capacity,
         }
@@ -72,13 +75,15 @@ impl Core {
 
     #[cfg(test)]
     fn new(capacity: usize) -> Self {
-        let gen = (capacity + 1).next_power_of_two();
-        let idx_mask = gen - 1;
-        let gen_mask = !(gen - 1);
+        let closed = (capacity + 1).next_power_of_two();
+        let idx_mask = closed - 1;
+        let gen = closed << 1;
+        let gen_mask = !(closed | idx_mask);
         Self {
             head: CachePadded(AtomicUsize::new(0)),
             tail: CachePadded(AtomicUsize::new(0)),
             gen,
+            closed,
             gen_mask,
             idx_mask,
             capacity,
@@ -109,34 +114,50 @@ impl Core {
         self.capacity
     }
 
-    fn push_ref<'slots, T, S>(&self, slots: &'slots S) -> Result<Ref<'slots, T>, Full<()>>
+    fn close(&self) -> bool {
+        if std::thread::panicking() {
+            return false;
+        }
+        test_dbg!(self.tail.fetch_or(self.closed, SeqCst) & self.closed == 0)
+    }
+
+    fn push_ref<'slots, T, S>(
+        &self,
+        slots: &'slots S,
+    ) -> Result<Ref<'slots, T>, mpsc::TrySendError<()>>
     where
         T: Default,
         S: Index<usize, Output = Slot<T>> + ?Sized,
     {
         test_println!("push_ref");
         let mut backoff = Backoff::new();
-        let mut tail = self.tail.load(Ordering::Relaxed);
+        let mut tail = self.tail.load(Relaxed);
 
         loop {
+            if test_dbg!(tail & self.closed != 0) {
+                return Err(mpsc::TrySendError::Closed(()));
+            }
             let (idx, gen) = self.idx_gen(tail);
             test_dbg!(idx);
             test_dbg!(gen);
             let slot = &slots[idx];
-            let state = slot.state.load(Ordering::Acquire);
+            let actual_state = test_dbg!(slot.state.load(Acquire));
+            let state = if actual_state == EMPTY_STATE {
+                idx
+            } else {
+                actual_state
+            };
 
-            if state == tail || (state == 0 && gen == 0) {
+            if test_dbg!(state == tail) || test_dbg!(actual_state == EMPTY_STATE && gen == 0) {
                 // Move the tail index forward by 1.
                 let next_tail = self.next(idx, gen);
-                match self.tail.compare_exchange_weak(
-                    tail,
-                    next_tail,
-                    Ordering::AcqRel,
-                    Ordering::Relaxed,
-                ) {
+                match test_dbg!(self
+                    .tail
+                    .compare_exchange_weak(tail, next_tail, SeqCst, Acquire))
+                {
                     Ok(_) => {
                         // We got the slot! It's now okay to write to it
-                        test_println!("claimed tail slot");
+                        test_println!("claimed tail slot [{}]", idx);
                         if gen == 0 {
                             slot.value.with_mut(|value| unsafe {
                                 // Safety: we have just claimed exclusive ownership over
@@ -161,9 +182,18 @@ impl Core {
                 }
             }
 
-            if state.wrapping_add(self.gen) == tail + 1 {
-                if self.head.load(Ordering::SeqCst).wrapping_add(self.gen) == tail {
-                    return Err(Full(()));
+            if test_dbg!(state.wrapping_add(self.gen) == tail + 1) {
+                // fake RMW op to placate loom. this should be equivalent to
+                // doing a relaxed load after a SeqCst fence (per Godbolt
+                // https://godbolt.org/z/zb15qfEa9), however, loom understands
+                // this correctly, while it does not understand an explicit
+                // SeqCst fence and a load.
+                // XXX(eliza): this makes me DEEPLY UNCOMFORTABLE but if it's a
+                // load it gets reordered differently in the model checker lmao...
+                let head = test_dbg!(self.head.fetch_or(0, SeqCst));
+                if test_dbg!(head.wrapping_add(self.gen) == tail) {
+                    test_println!("channel full");
+                    return Err(mpsc::TrySendError::Full(()));
                 }
 
                 backoff.spin();
@@ -171,17 +201,17 @@ impl Core {
                 backoff.spin_yield();
             }
 
-            tail = self.tail.load(Ordering::Relaxed)
+            tail = test_dbg!(self.tail.load(Acquire));
         }
     }
 
-    fn pop_ref<'slots, T, S>(&self, slots: &'slots S) -> Option<Ref<'slots, T>>
+    fn pop_ref<'slots, T, S>(&self, slots: &'slots S) -> Result<Ref<'slots, T>, mpsc::TrySendError>
     where
         S: Index<usize, Output = Slot<T>> + ?Sized,
     {
         test_println!("pop_ref");
         let mut backoff = Backoff::new();
-        let mut head = self.head.load(Ordering::Relaxed);
+        let mut head = self.head.load(Relaxed);
 
         loop {
             test_dbg!(head);
@@ -189,22 +219,20 @@ impl Core {
             test_dbg!(idx);
             test_dbg!(gen);
             let slot = &slots[idx];
-            let state = slot.state.load(Ordering::Acquire);
-            test_dbg!(state);
+            let state = test_dbg!(slot.state.load(Acquire));
+            let state = if state == EMPTY_STATE { idx } else { state };
 
             // If the slot's state is ahead of the head index by one, we can pop
             // it.
             if test_dbg!(state == head + 1) {
                 let next_head = self.next(idx, gen);
-                match self.head.compare_exchange(
-                    head,
-                    next_head,
-                    Ordering::SeqCst,
-                    Ordering::Relaxed,
-                ) {
+                match test_dbg!(self
+                    .head
+                    .compare_exchange_weak(head, next_head, SeqCst, Acquire))
+                {
                     Ok(_) => {
-                        test_println!("claimed head slot");
-                        return Some(Ref {
+                        test_println!("claimed head slot [{}]", idx);
+                        return Ok(Ref {
                             new_state: head.wrapping_add(self.gen),
                             slot,
                         });
@@ -218,10 +246,26 @@ impl Core {
             }
 
             if test_dbg!(state == head) {
-                let tail = self.tail.load(Ordering::SeqCst);
+                // fake RMW op to placate loom. this should be equivalent to
+                // doing a relaxed load after a SeqCst fence (per Godbolt
+                // https://godbolt.org/z/zb15qfEa9), however, loom understands
+                // this correctly, while it does not understand an explicit
+                // SeqCst fence and a load.
+                // XXX(eliza): this makes me DEEPLY UNCOMFORTABLE but if it's a
+                // load it gets reordered differently in the model checker lmao...
+
+                let tail = test_dbg!(self.tail.fetch_or(0, SeqCst));
+
+                if test_dbg!(tail & !self.closed == head) {
+                    return if test_dbg!(tail & self.closed != 0) {
+                        Err(mpsc::TrySendError::Closed(()))
+                    } else {
+                        Err(mpsc::TrySendError::Full(()))
+                    };
+                }
 
-                if test_dbg!(tail == head) {
-                    return None;
+                if test_dbg!(backoff.done_spinning()) {
+                    return Err(mpsc::TrySendError::Full(()));
                 }
 
                 backoff.spin();
@@ -229,17 +273,16 @@ impl Core {
                 backoff.spin_yield();
             }
 
-            head = self.head.load(Ordering::Relaxed);
+            head = test_dbg!(self.head.load(Acquire));
         }
     }
 
     fn len(&self) -> usize {
-        use std::cmp;
         loop {
-            let tail = self.tail.load(Ordering::SeqCst);
-            let head = self.head.load(Ordering::SeqCst);
+            let tail = self.tail.load(SeqCst);
+            let head = self.head.load(SeqCst);
 
-            if self.tail.load(Ordering::SeqCst) == tail {
+            if self.tail.load(SeqCst) == tail {
                 let (head_idx, _) = self.idx_gen(head);
                 let (tail_idx, _) = self.idx_gen(tail);
                 return match head_idx.cmp(&tail_idx) {
@@ -256,6 +299,8 @@ impl Core {
 // === impl Ref ===
 
 impl<T> Ref<'_, T> {
+    const RELEASED: usize = usize::MAX;
+
     #[inline]
     pub fn with<U>(&self, f: impl FnOnce(&T) -> U) -> U {
         self.slot.value.with(|value| unsafe {
@@ -278,15 +323,23 @@ impl<T> Ref<'_, T> {
             f(&mut *(&mut *value).as_mut_ptr())
         })
     }
+
+    pub(crate) fn release(&mut self) {
+        if self.new_state == Self::RELEASED {
+            test_println!("release_ref; already released");
+            return;
+        }
+
+        test_println!("release_ref");
+        test_dbg!(self.slot.state.store(test_dbg!(self.new_state), Release));
+        self.new_state = Self::RELEASED;
+    }
 }
 
 impl<T> Drop for Ref<'_, T> {
     #[inline]
     fn drop(&mut self) {
-        test_println!("drop_ref");
-        self.slot
-            .state
-            .store(test_dbg!(self.new_state), Ordering::Release);
+        self.release();
     }
 }
 
@@ -321,12 +374,14 @@ impl<T: fmt::Write> fmt::Write for Ref<'_, T> {
 
 // === impl Slot ===
 
+const EMPTY_STATE: usize = usize::MAX;
+
 impl<T> Slot<T> {
     #[cfg(not(test))]
     const fn empty() -> Self {
         Self {
             value: UnsafeCell::new(MaybeUninit::uninit()),
-            state: AtomicUsize::new(0),
+            state: AtomicUsize::new(EMPTY_STATE),
         }
     }
 
@@ -334,7 +389,7 @@ impl<T> Slot<T> {
     fn empty() -> Self {
         Self {
             value: UnsafeCell::new(MaybeUninit::uninit()),
-            state: AtomicUsize::new(0),
+            state: AtomicUsize::new(EMPTY_STATE),
         }
     }
 }

+ 15 - 7
src/loom.rs

@@ -2,6 +2,7 @@ pub(crate) use self::inner::*;
 
 #[cfg(test)]
 mod inner {
+
     pub(crate) mod atomic {
         pub use loom::sync::atomic::*;
         pub use std::sync::atomic::Ordering;
@@ -10,6 +11,11 @@ mod inner {
     pub(crate) use loom::{cell::UnsafeCell, future, hint, sync, thread};
     use std::{cell::RefCell, fmt::Write};
 
+    pub(crate) mod model {
+        #[allow(unused_imports)]
+        pub(crate) use loom::model::Builder;
+    }
+
     std::thread_local! {
         static TRACE_BUF: RefCell<String> = RefCell::new(String::new());
     }
@@ -25,6 +31,7 @@ mod inner {
             .unwrap_or_else(|_| println!("{}", args.take().unwrap()))
     }
 
+    #[track_caller]
     pub(crate) fn run_builder(
         builder: loom::model::Builder,
         model: impl Fn() + Sync + Send + std::panic::UnwindSafe + 'static,
@@ -98,11 +105,15 @@ mod inner {
         // wrap the loom model with `catch_unwind` to avoid potentially losing
         // test output on double panics.
         let current_iteration = std::sync::Arc::new(AtomicUsize::new(1));
+        let test_name = match std::thread::current().name() {
+            Some("main") | None => "test".to_string(),
+            Some(name) => name.to_string(),
+        };
         builder.check(move || {
+            let iteration = current_iteration.fetch_add(1, Ordering::Relaxed);
             traceln(format_args!(
                 "\n---- {} iteration {} ----",
-                std::thread::current().name().unwrap_or("<unknown test>"),
-                current_iteration.fetch_add(1, Ordering::Relaxed)
+                test_name, iteration,
             ));
 
             model();
@@ -112,12 +123,9 @@ mod inner {
         });
     }
 
+    #[track_caller]
     pub(crate) fn model(model: impl Fn() + std::panic::UnwindSafe + Sync + Send + 'static) {
-        let mut builder = loom::model::Builder::default();
-        // // A couple of our tests will hit the max number of branches riiiiight
-        // // before they should complete. Double it so this stops happening.
-        builder.max_branches *= 2;
-        run_builder(builder, model)
+        run_builder(Default::default(), model)
     }
 
     pub(crate) mod alloc {

+ 15 - 0
src/macros.rs

@@ -42,6 +42,21 @@ macro_rules! feature {
     }
 }
 
+macro_rules! fmt_bits {
+    ($self: expr, $f: expr, $has_states: ident, $($name: ident),+) => {
+        $(
+            if $self.contains(Self::$name) {
+                if $has_states {
+                    $f.write_str(" | ")?;
+                }
+                $f.write_str(stringify!($name))?;
+                $has_states = true;
+            }
+        )+
+
+    };
+}
+
 #[allow(unused_macros)]
 macro_rules! unreachable_unchecked {
     ($($arg:tt)+) => {

+ 190 - 12
src/mpsc.rs

@@ -11,11 +11,18 @@
 //! can be constructed using the [`sync::channel`] function.
 
 use crate::{
-    loom::atomic::AtomicUsize,
-    util::wait::{Notify, WaitCell},
+    loom::{atomic::AtomicUsize, hint},
+    util::{
+        wait::{Notify, WaitCell, WaitResult},
+        Backoff,
+    },
     Ref, ThingBuf,
 };
 use core::fmt;
+use core::task::Poll;
+
+#[cfg(feature = "alloc")]
+use crate::util::wait::{NotifyOnDrop, WaitQueue};
 
 #[derive(Debug)]
 #[non_exhaustive]
@@ -25,10 +32,15 @@ pub enum TrySendError<T = ()> {
 }
 
 #[derive(Debug)]
-struct Inner<T, N> {
+pub struct Closed<T = ()>(T);
+
+#[derive(Debug)]
+struct Inner<T, N: Notify> {
     thingbuf: ThingBuf<T>,
     rx_wait: WaitCell<N>,
     tx_count: AtomicUsize,
+    #[cfg(feature = "alloc")]
+    tx_wait: WaitQueue<NotifyOnDrop<N>>,
 }
 
 struct SendRefInner<'a, T, N: Notify> {
@@ -48,20 +60,32 @@ impl TrySendError {
 }
 
 // ==== impl Inner ====
+impl<T, N: Notify> Inner<T, N> {
+    fn new(thingbuf: ThingBuf<T>) -> Self {
+        Self {
+            thingbuf,
+            rx_wait: WaitCell::new(),
+            tx_count: AtomicUsize::new(1),
+            #[cfg(feature = "alloc")]
+            tx_wait: WaitQueue::new(),
+        }
+    }
+
+    fn close_rx(&self) {
+        if self.thingbuf.core.close() {
+            crate::loom::hint::spin_loop();
+            test_println!("draining_queue");
+            self.tx_wait.drain();
+        }
+    }
+}
 
 impl<T: Default, N: Notify> Inner<T, N> {
     fn try_send_ref(&self) -> Result<SendRefInner<'_, T, N>, TrySendError> {
         self.thingbuf
-            .push_ref()
+            .core
+            .push_ref(self.thingbuf.slots.as_ref())
             .map(|slot| SendRefInner { inner: self, slot })
-            .map_err(|_| {
-                if self.rx_wait.is_rx_closed() {
-                    TrySendError::Closed(())
-                } else {
-                    self.rx_wait.notify();
-                    TrySendError::Full(())
-                }
-            })
     }
 
     fn try_send(&self, val: T) -> Result<(), TrySendError<T>> {
@@ -73,6 +97,101 @@ impl<T: Default, N: Notify> Inner<T, N> {
             Err(e) => Err(e.with_value(val)),
         }
     }
+
+    /// Performs one iteration of the `send_ref` loop.
+    ///
+    /// The loop itself has to be written in the actual `send` method's
+    /// implementation, rather than on `inner`, because it might be async and
+    /// may yield, or might park the thread.
+    fn poll_send_ref(
+        &self,
+        mk_waiter: impl Fn() -> N,
+    ) -> Poll<Result<SendRefInner<'_, T, N>, Closed>> {
+        let mut backoff = Backoff::new();
+        // try to send a few times in a loop, in case the receiver notifies us
+        // right before we park.
+        loop {
+            // try to reserve a send slot, returning if we succeeded or if the
+            // queue was closed.
+            match self.try_send_ref() {
+                Ok(slot) => return Poll::Ready(Ok(slot)),
+                Err(TrySendError::Closed(_)) => return Poll::Ready(Err(Closed(()))),
+                Err(_) => {}
+            }
+
+            // try to push a waiter
+            let pushed_waiter = self.tx_wait.push_waiter(|| {
+                let current = mk_waiter();
+                test_println!("parking sender ({:?})", current);
+                NotifyOnDrop::new(current)
+            });
+
+            match test_dbg!(pushed_waiter) {
+                WaitResult::TxClosed => {
+                    // the channel closed while we were registering the waiter!
+                    return Poll::Ready(Err(Closed(())));
+                }
+                WaitResult::Wait => {
+                    // okay, we are now queued to wait. gotosleep!
+                    return Poll::Pending;
+                }
+                WaitResult::Notified => {
+                    // we consumed a queued notification. try again...
+                    backoff.spin_yield();
+                }
+            }
+        }
+    }
+
+    /// Performs one iteration of the `recv_ref` loop.
+    ///
+    /// The loop itself has to be written in the actual `send` method's
+    /// implementation, rather than on `inner`, because it might be async and
+    /// may yield, or might park the thread.
+    fn poll_recv_ref(&self, mk_waiter: impl Fn() -> N) -> Poll<Option<Ref<'_, T>>> {
+        macro_rules! try_poll_recv {
+            () => {
+                // If we got a value, return it!
+                match self.thingbuf.core.pop_ref(self.thingbuf.slots.as_ref()) {
+                    Ok(slot) => return Poll::Ready(Some(slot)),
+                    Err(TrySendError::Closed(_)) => return Poll::Ready(None),
+                    _ => {}
+                }
+            };
+        }
+
+        test_println!("poll_recv_ref");
+        loop {
+            test_println!("poll_recv_ref => loop");
+
+            // try to receive a reference, returning if we succeeded or the
+            // channel is closed.
+            try_poll_recv!();
+
+            // otherwise, gotosleep
+            match test_dbg!(self.rx_wait.wait_with(&mk_waiter)) {
+                WaitResult::Wait => {
+                    // we successfully registered a waiter! try polling again,
+                    // just in case someone sent a message while we were
+                    // registering the waiter.
+                    try_poll_recv!();
+                    test_dbg!(self.tx_wait.notify());
+                    return Poll::Pending;
+                }
+                WaitResult::TxClosed => {
+                    // the channel is closed (all the receivers are dropped).
+                    // however, there may be messages left in the queue. try
+                    // popping from the queue until it's empty.
+                    return Poll::Ready(self.thingbuf.pop_ref());
+                }
+                WaitResult::Notified => {
+                    // we were notified while we were trying to register the
+                    // waiter. loop and try polling again.
+                    hint::spin_loop();
+                }
+            }
+        }
+    }
 }
 
 impl<T, N: Notify> SendRefInner<'_, T, N> {
@@ -91,6 +210,7 @@ impl<T, N: Notify> Drop for SendRefInner<'_, T, N> {
     #[inline]
     fn drop(&mut self) {
         test_println!("drop SendRef<T, {}>", std::any::type_name::<N>());
+        self.slot.release();
         self.inner.rx_wait.notify();
     }
 }
@@ -171,6 +291,64 @@ macro_rules! impl_send_ref {
     };
 }
 
+macro_rules! impl_recv_ref {
+    (pub struct $name:ident<$notify:ty>;) => {
+        pub struct $name<'recv, T> {
+            slot: Ref<'recv, T>,
+            inner: &'recv Inner<T, $notify>,
+        }
+
+        impl<T> $name<'_, T> {
+            #[inline]
+            pub fn with<U>(&self, f: impl FnOnce(&T) -> U) -> U {
+                self.slot.with(f)
+            }
+
+            #[inline]
+            pub fn with_mut<U>(&mut self, f: impl FnOnce(&mut T) -> U) -> U {
+                self.slot.with_mut(f)
+            }
+        }
+
+        impl<T: fmt::Debug> fmt::Debug for $name<'_, T> {
+            fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+                self.slot.fmt(f)
+            }
+        }
+
+        impl<T: fmt::Display> fmt::Display for $name<'_, T> {
+            fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+                self.slot.fmt(f)
+            }
+        }
+
+        impl<T: fmt::Write> fmt::Write for $name<'_, T> {
+            #[inline]
+            fn write_str(&mut self, s: &str) -> fmt::Result {
+                self.slot.write_str(s)
+            }
+
+            #[inline]
+            fn write_char(&mut self, c: char) -> fmt::Result {
+                self.slot.write_char(c)
+            }
+
+            #[inline]
+            fn write_fmt(&mut self, f: fmt::Arguments<'_>) -> fmt::Result {
+                self.slot.write_fmt(f)
+            }
+        }
+
+        impl<T> Drop for RecvRef<'_, T> {
+            fn drop(&mut self) {
+                test_println!("drop RecvRef<T, {}>", stringify!($notify));
+                self.slot.release();
+                self.inner.tx_wait.notify();
+            }
+        }
+    };
+}
+
 mod async_impl;
 pub use self::async_impl::*;
 

+ 46 - 46
src/mpsc/async_impl.rs

@@ -1,11 +1,9 @@
 use super::*;
 use crate::{
     loom::{
-        self,
-        atomic::{AtomicUsize, Ordering},
+        atomic::{self, Ordering},
         sync::Arc,
     },
-    util::wait::{WaitCell, WaitResult},
     Ref, ThingBuf,
 };
 use core::{
@@ -17,11 +15,7 @@ use core::{
 
 /// Returns a new synchronous multi-producer, single consumer channel.
 pub fn channel<T>(thingbuf: ThingBuf<T>) -> (Sender<T>, Receiver<T>) {
-    let inner = Arc::new(Inner {
-        thingbuf,
-        rx_wait: WaitCell::new(),
-        tx_count: AtomicUsize::new(1),
-    });
+    let inner = Arc::new(Inner::new(thingbuf));
     let tx = Sender {
         inner: inner.clone(),
     };
@@ -43,6 +37,10 @@ impl_send_ref! {
     pub struct SendRef<Waker>;
 }
 
+impl_recv_ref! {
+    pub struct RecvRef<Waker>;
+}
+
 /// A [`Future`] that tries to receive a reference from a [`Receiver`].
 ///
 /// This type is returned by [`Receiver::recv_ref`].
@@ -73,6 +71,35 @@ impl<T: Default> Sender<T> {
     pub fn try_send(&self, val: T) -> Result<(), TrySendError<T>> {
         self.inner.try_send(val)
     }
+
+    pub async fn send_ref(&self) -> Result<SendRef<'_, T>, Closed> {
+        // This future is private because if we replace the waiter queue thing with an
+        // intrusive list, we won't want to expose the future type publicly, for safety reasons.
+        struct SendRefFuture<'sender, T>(&'sender Sender<T>);
+        impl<'sender, T: Default + 'sender> Future for SendRefFuture<'sender, T> {
+            type Output = Result<SendRef<'sender, T>, Closed>;
+
+            fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
+                // perform one send ref loop iteration
+                self.0
+                    .inner
+                    .poll_send_ref(|| cx.waker().clone())
+                    .map(|ok| ok.map(SendRef))
+            }
+        }
+
+        SendRefFuture(self).await
+    }
+
+    pub async fn send(&self, val: T) -> Result<(), Closed<T>> {
+        match self.send_ref().await {
+            Err(Closed(())) => Err(Closed(val)),
+            Ok(mut slot) => {
+                slot.with_mut(|slot| *slot = val);
+                Ok(())
+            }
+        }
+    }
 }
 
 impl<T> Clone for Sender<T> {
@@ -91,7 +118,8 @@ impl<T> Drop for Sender<T> {
         }
 
         // if we are the last sender, synchronize
-        test_dbg!(self.inner.tx_count.load(Ordering::SeqCst));
+        test_dbg!(atomic::fence(Ordering::SeqCst));
+        self.inner.thingbuf.core.close();
         self.inner.rx_wait.close_tx();
     }
 }
@@ -120,37 +148,13 @@ impl<T: Default> Receiver<T> {
     /// sender, or when the channel is closed.  Note that on multiple calls to
     /// `poll_recv_ref`, only the [`Waker`] from the [`Context`] passed to the most
     /// recent call is scheduled to receive a wakeup.
-    pub fn poll_recv_ref(&self, cx: &mut Context<'_>) -> Poll<Option<Ref<'_, T>>> {
-        loop {
-            if let Some(r) = self.try_recv_ref() {
-                return Poll::Ready(Some(r));
-            }
-
-            // Okay, no value is ready --- try to wait.
-            match test_dbg!(self.inner.rx_wait.wait_with(|| cx.waker().clone())) {
-                WaitResult::TxClosed => {
-                    // All senders have been dropped, but the channel might
-                    // still have messages in it. Return `Ready`; if the recv'd ref
-                    // is `None` then we've popped everything.
-                    return Poll::Ready(self.try_recv_ref());
-                }
-                WaitResult::Wait => {
-                    // make sure nobody sent a message while we were registering
-                    // the waiter...
-                    // XXX(eliza): a nicer solution _might_ just be to pack the
-                    // waiter state into the tail idx or something or something
-                    // but that kind of defeats the purpose of just having a
-                    // nice "wrap a queue into a channel" API...
-                    if let Some(val) = self.try_recv_ref() {
-                        return Poll::Ready(Some(val));
-                    }
-                    return Poll::Pending;
-                }
-                WaitResult::Notified => {
-                    loom::hint::spin_loop();
-                }
-            };
-        }
+    pub fn poll_recv_ref(&self, cx: &mut Context<'_>) -> Poll<Option<RecvRef<'_, T>>> {
+        self.inner.poll_recv_ref(|| cx.waker().clone()).map(|some| {
+            some.map(|slot| RecvRef {
+                slot,
+                inner: &*self.inner,
+            })
+        })
     }
 
     /// # Returns
@@ -171,10 +175,6 @@ impl<T: Default> Receiver<T> {
             .map(|opt| opt.map(|mut r| r.with_mut(core::mem::take)))
     }
 
-    fn try_recv_ref(&self) -> Option<Ref<'_, T>> {
-        self.inner.thingbuf.pop_ref()
-    }
-
     pub fn is_closed(&self) -> bool {
         test_dbg!(self.inner.tx_count.load(Ordering::SeqCst)) <= 1
     }
@@ -182,14 +182,14 @@ impl<T: Default> Receiver<T> {
 
 impl<T> Drop for Receiver<T> {
     fn drop(&mut self) {
-        self.inner.rx_wait.close_rx();
+        self.inner.close_rx();
     }
 }
 
 // === impl RecvRefFuture ===
 
 impl<'a, T: Default> Future for RecvRefFuture<'a, T> {
-    type Output = Option<Ref<'a, T>>;
+    type Output = Option<RecvRef<'a, T>>;
 
     fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
         self.rx.poll_recv_ref(cx)

+ 42 - 37
src/mpsc/sync.rs

@@ -6,23 +6,17 @@
 use super::*;
 use crate::{
     loom::{
-        self,
-        atomic::{AtomicUsize, Ordering},
+        atomic::{self, Ordering},
         sync::Arc,
         thread::{self, Thread},
     },
-    util::wait::{WaitCell, WaitResult},
     Ref, ThingBuf,
 };
 use core::fmt;
 
 /// Returns a new asynchronous multi-producer, single consumer channel.
 pub fn channel<T>(thingbuf: ThingBuf<T>) -> (Sender<T>, Receiver<T>) {
-    let inner = Arc::new(Inner {
-        thingbuf,
-        rx_wait: WaitCell::new(),
-        tx_count: AtomicUsize::new(1),
-    });
+    let inner = Arc::new(Inner::new(thingbuf));
     let tx = Sender {
         inner: inner.clone(),
     };
@@ -44,6 +38,10 @@ impl_send_ref! {
     pub struct SendRef<Thread>;
 }
 
+impl_recv_ref! {
+    pub struct RecvRef<Thread>;
+}
+
 // === impl Sender ===
 
 impl<T: Default> Sender<T> {
@@ -54,6 +52,28 @@ impl<T: Default> Sender<T> {
     pub fn try_send(&self, val: T) -> Result<(), TrySendError<T>> {
         self.inner.try_send(val)
     }
+
+    pub fn send_ref(&self) -> Result<SendRef<'_, T>, Closed> {
+        loop {
+            // perform one send ref loop iteration
+            if let Poll::Ready(result) = self.inner.poll_send_ref(thread::current) {
+                return result.map(SendRef);
+            }
+
+            // if that iteration failed, park the thread.
+            thread::park();
+        }
+    }
+
+    pub fn send(&self, val: T) -> Result<(), Closed<T>> {
+        match self.send_ref() {
+            Err(Closed(())) => Err(Closed(val)),
+            Ok(mut slot) => {
+                slot.with_mut(|slot| *slot = val);
+                Ok(())
+            }
+        }
+    }
 }
 
 impl<T> Clone for Sender<T> {
@@ -72,44 +92,29 @@ impl<T> Drop for Sender<T> {
         }
 
         // if we are the last sender, synchronize
-        test_dbg!(self.inner.tx_count.load(Ordering::SeqCst));
-        self.inner.rx_wait.close_tx();
+        test_dbg!(atomic::fence(Ordering::SeqCst));
+        if self.inner.thingbuf.core.close() {
+            self.inner.rx_wait.close_tx();
+        }
     }
 }
 
 // === impl Receiver ===
 
 impl<T: Default> Receiver<T> {
-    pub fn recv_ref(&self) -> Option<Ref<'_, T>> {
+    pub fn recv_ref(&self) -> Option<RecvRef<'_, T>> {
         loop {
-            // If we got a value, return it!
-            if let Some(r) = self.inner.thingbuf.pop_ref() {
-                return Some(r);
-            }
-
-            // otherwise, gotosleep
-            match test_dbg!(self.inner.rx_wait.wait_with(thread::current)) {
-                WaitResult::TxClosed => {
-                    // All senders have been dropped, but the channel might
-                    // still have messages in it...
-                    return self.inner.thingbuf.pop_ref();
+            match self.inner.poll_recv_ref(thread::current) {
+                Poll::Ready(r) => {
+                    return r.map(|slot| RecvRef {
+                        slot,
+                        inner: &*self.inner,
+                    })
                 }
-                WaitResult::Wait => {
-                    // make sure nobody sent a message while we were registering
-                    // the waiter...
-                    // XXX(eliza): a nicer solution _might_ just be to pack the
-                    // waiter state into the tail idx or something or something
-                    // but that kind of defeats the purpose of just having a
-                    // nice "wrap a queue into a channel" API...
-                    if let Some(val) = self.inner.thingbuf.pop_ref() {
-                        return Some(val);
-                    }
+                Poll::Pending => {
                     test_println!("parking ({:?})", thread::current());
                     thread::park();
                 }
-                WaitResult::Notified => {
-                    loom::hint::spin_loop();
-                }
             }
         }
     }
@@ -129,7 +134,7 @@ impl<T: Default> Receiver<T> {
 }
 
 impl<'a, T: Default> Iterator for &'a Receiver<T> {
-    type Item = Ref<'a, T>;
+    type Item = RecvRef<'a, T>;
 
     fn next(&mut self) -> Option<Self::Item> {
         self.recv_ref()
@@ -138,6 +143,6 @@ impl<'a, T: Default> Iterator for &'a Receiver<T> {
 
 impl<T> Drop for Receiver<T> {
     fn drop(&mut self) {
-        self.inner.rx_wait.close_rx();
+        self.inner.close_rx();
     }
 }

+ 126 - 5
src/mpsc/tests/mpsc_async.rs

@@ -5,20 +5,19 @@ use crate::{
 };
 
 #[test]
-fn basically_works() {
+fn mpsc_try_send_recv() {
     loom::model(|| {
-        let (tx, rx) = channel(ThingBuf::new(4));
+        let (tx, rx) = channel(ThingBuf::new(3));
 
         let p1 = {
             let tx = tx.clone();
             thread::spawn(move || {
                 tx.try_send_ref().unwrap().with_mut(|val| *val = 1);
-                tx.try_send_ref().unwrap().with_mut(|val| *val = 2);
             })
         };
         let p2 = thread::spawn(move || {
+            tx.try_send(2).unwrap();
             tx.try_send(3).unwrap();
-            tx.try_send(4).unwrap();
         });
 
         let mut vals = future::block_on(async move {
@@ -30,7 +29,7 @@ fn basically_works() {
         });
 
         vals.sort_unstable();
-        assert_eq!(vals, vec![1, 2, 3, 4]);
+        assert_eq!(vals, vec![1, 2, 3]);
 
         p1.join().unwrap();
         p2.join().unwrap();
@@ -122,6 +121,128 @@ fn spsc_recv_then_send_then_close() {
     })
 }
 
+#[test]
+fn spsc_send_recv_in_order_no_wrap() {
+    const N_SENDS: usize = 4;
+    loom::model(|| {
+        let (tx, rx) = channel(ThingBuf::<usize>::new(N_SENDS));
+        let consumer = thread::spawn(move || {
+            future::block_on(async move {
+                for i in 1..=N_SENDS {
+                    assert_eq!(rx.recv().await, Some(i));
+                }
+                assert_eq!(rx.recv().await, None);
+            })
+        });
+        future::block_on(async move {
+            for i in 1..=N_SENDS {
+                tx.send(i).await.unwrap()
+            }
+        });
+        consumer.join().unwrap();
+    })
+}
+
+#[test]
+fn spsc_send_recv_in_order_wrap() {
+    const N_SENDS: usize = 2;
+    loom::model(|| {
+        let (tx, rx) = channel(ThingBuf::<usize>::new(N_SENDS / 2));
+        let consumer = thread::spawn(move || {
+            future::block_on(async move {
+                for i in 1..=N_SENDS {
+                    assert_eq!(rx.recv().await, Some(i));
+                }
+                assert_eq!(rx.recv().await, None);
+            })
+        });
+        future::block_on(async move {
+            for i in 1..=N_SENDS {
+                tx.send(i).await.unwrap()
+            }
+        });
+        consumer.join().unwrap();
+    })
+}
+
+#[test]
+fn mpsc_send_recv_wrap() {
+    loom::model(|| {
+        let (tx, rx) = channel(ThingBuf::<usize>::new(1));
+        let producer1 = do_producer(tx.clone(), 10);
+        let producer2 = do_producer(tx, 20);
+
+        let results = future::block_on(async move {
+            let mut results = Vec::new();
+            while let Some(val) = rx.recv().await {
+                test_println!("RECEIVED {:?}", val);
+                results.push(val);
+            }
+            results
+        });
+
+        producer1.join().expect("producer 1 panicked");
+        producer2.join().expect("producer 2 panicked");
+
+        assert_eq!(results.len(), 2);
+        assert!(
+            results.contains(&10),
+            "missing value from producer 1; results={:?}",
+            results
+        );
+
+        assert!(
+            results.contains(&20),
+            "missing value from producer 2; results={:?}",
+            results
+        );
+    })
+}
+
+#[test]
+fn mpsc_send_recv_no_wrap() {
+    loom::model(|| {
+        let (tx, rx) = channel(ThingBuf::<usize>::new(2));
+        let producer1 = do_producer(tx.clone(), 10);
+        let producer2 = do_producer(tx, 20);
+
+        let results = future::block_on(async move {
+            let mut results = Vec::new();
+            while let Some(val) = rx.recv().await {
+                test_println!("RECEIVED {:?}", val);
+                results.push(val);
+            }
+            results
+        });
+
+        producer1.join().expect("producer 1 panicked");
+        producer2.join().expect("producer 2 panicked");
+
+        assert_eq!(results.len(), 2);
+        assert!(
+            results.contains(&10),
+            "missing value from producer 1; results={:?}",
+            results
+        );
+
+        assert!(
+            results.contains(&20),
+            "missing value from producer 2; results={:?}",
+            results
+        );
+    })
+}
+
+fn do_producer(tx: Sender<usize>, tag: usize) -> thread::JoinHandle<()> {
+    thread::spawn(move || {
+        future::block_on(async move {
+            test_println!("SENDING {:?}", tag);
+            tx.send(tag).await.unwrap();
+            test_println!("SENT {:?}", tag);
+        })
+    })
+}
+
 #[test]
 fn tx_close_wakes() {
     loom::model(|| {

+ 144 - 7
src/mpsc/tests/mpsc_sync.rs

@@ -5,20 +5,28 @@ use crate::{
 };
 
 #[test]
-fn basically_works() {
+// This test currently fails because `loom` implements the wrong semantics for
+// `Thread::unpark()`/`thread::park` (see
+// https://github.com/tokio-rs/loom/issues/246).
+// However, it implements the correct semantics for async `Waker`s (which
+// _should_ be the same as park/unpark), so the async version of this test more
+// or less verifies that the algorithm here is correct.
+//
+// TODO(eliza): when tokio-rs/loom#246 is fixed, we can re-enable this test!
+#[ignore]
+fn mpsc_try_send_recv() {
     loom::model(|| {
-        let (tx, rx) = sync::channel(ThingBuf::new(4));
+        let (tx, rx) = sync::channel(ThingBuf::new(3));
 
         let p1 = {
             let tx = tx.clone();
             thread::spawn(move || {
                 tx.try_send_ref().unwrap().with_mut(|val| *val = 1);
-                tx.try_send_ref().unwrap().with_mut(|val| *val = 2);
             })
         };
         let p2 = thread::spawn(move || {
+            tx.try_send(2).unwrap();
             tx.try_send(3).unwrap();
-            tx.try_send(4).unwrap();
         });
 
         let mut vals = Vec::new();
@@ -28,7 +36,7 @@ fn basically_works() {
         }
 
         vals.sort_unstable();
-        assert_eq!(vals, vec![1, 2, 3, 4]);
+        assert_eq!(vals, vec![1, 2, 3]);
 
         p1.join().unwrap();
         p2.join().unwrap();
@@ -43,6 +51,7 @@ fn rx_closes() {
 
         let producer = thread::spawn(move || {
             'iters: for i in 0..=ITERATIONS {
+                test_println!("sending {}", i);
                 'send: loop {
                     match tx.try_send(i) {
                         Ok(_) => break 'send,
@@ -67,7 +76,7 @@ fn rx_closes() {
 }
 
 #[test]
-fn spsc_recv_then_send() {
+fn spsc_recv_then_try_send() {
     loom::model(|| {
         let (tx, rx) = sync::channel(ThingBuf::<i32>::new(4));
         let consumer = thread::spawn(move || {
@@ -94,7 +103,7 @@ fn spsc_recv_then_close() {
 }
 
 #[test]
-fn spsc_recv_then_send_then_close() {
+fn spsc_recv_then_try_send_then_close() {
     loom::model(|| {
         let (tx, rx) = sync::channel(ThingBuf::<i32>::new(2));
         let consumer = thread::spawn(move || {
@@ -110,6 +119,134 @@ fn spsc_recv_then_send_then_close() {
     })
 }
 
+#[test]
+// This test currently fails because `loom` implements the wrong semantics for
+// `Thread::unpark()`/`thread::park` (see
+// https://github.com/tokio-rs/loom/issues/246).
+// However, it implements the correct semantics for async `Waker`s (which
+// _should_ be the same as park/unpark), so the async version of this test more
+// or less verifies that the algorithm here is correct.
+//
+// TODO(eliza): when tokio-rs/loom#246 is fixed, we can re-enable this test!
+#[ignore]
+fn mpsc_send_recv_wrap() {
+    loom::model(|| {
+        let (tx, rx) = sync::channel(ThingBuf::<usize>::new(1));
+        let producer1 = do_producer(tx.clone(), 10);
+        let producer2 = do_producer(tx, 20);
+
+        let mut results = Vec::new();
+        while let Some(val) = rx.recv() {
+            test_println!("RECEIVED {:?}", val);
+            results.push(val);
+        }
+
+        producer1.join().expect("producer 1 panicked");
+        producer2.join().expect("producer 2 panicked");
+
+        assert_eq!(results.len(), 2);
+        assert!(
+            results.contains(&10),
+            "missing value from producer 1; results={:?}",
+            results
+        );
+
+        assert!(
+            results.contains(&20),
+            "missing value from producer 2; results={:?}",
+            results
+        );
+    })
+}
+
+#[test]
+fn mpsc_send_recv_no_wrap() {
+    loom::model(|| {
+        let (tx, rx) = sync::channel(ThingBuf::<usize>::new(2));
+        let producer1 = do_producer(tx.clone(), 10);
+        let producer2 = do_producer(tx, 20);
+
+        let mut results = Vec::new();
+        while let Some(val) = rx.recv() {
+            test_println!("RECEIVED {:?}", val);
+            results.push(val);
+        }
+
+        producer1.join().expect("producer 1 panicked");
+        producer2.join().expect("producer 2 panicked");
+
+        assert_eq!(results.len(), 2);
+        assert!(
+            results.contains(&10),
+            "missing value from producer 1; results={:?}",
+            results
+        );
+
+        assert!(
+            results.contains(&20),
+            "missing value from producer 2; results={:?}",
+            results
+        );
+    })
+}
+
+fn do_producer(tx: sync::Sender<usize>, tag: usize) -> thread::JoinHandle<()> {
+    thread::spawn(move || {
+        test_println!("SENDING {:?}", tag);
+        tx.send(tag).unwrap();
+        test_println!("SENT {:?}", tag);
+    })
+}
+
+#[test]
+fn spsc_send_recv_in_order_no_wrap() {
+    const N_SENDS: usize = 4;
+    loom::model(|| {
+        let (tx, rx) = sync::channel(ThingBuf::<usize>::new(N_SENDS));
+        let consumer = thread::spawn(move || {
+            for i in 1..=N_SENDS {
+                assert_eq!(rx.recv(), Some(i));
+            }
+            assert_eq!(rx.recv(), None);
+        });
+
+        for i in 1..=N_SENDS {
+            tx.send(i).unwrap()
+        }
+        drop(tx);
+        consumer.join().unwrap();
+    })
+}
+
+#[test]
+// This test currently fails because `loom` implements the wrong semantics for
+// `Thread::unpark()`/`thread::park` (see
+// https://github.com/tokio-rs/loom/issues/246).
+// However, it implements the correct semantics for async `Waker`s (which
+// _should_ be the same as park/unpark), so the async version of this test more
+// or less verifies that the algorithm here is correct.
+//
+// TODO(eliza): when tokio-rs/loom#246 is fixed, we can re-enable this test!
+#[ignore]
+fn spsc_send_recv_in_order_wrap() {
+    const N_SENDS: usize = 2;
+    loom::model(|| {
+        let (tx, rx) = sync::channel(ThingBuf::<usize>::new(N_SENDS / 2));
+        let consumer = thread::spawn(move || {
+            for i in 1..=N_SENDS {
+                assert_eq!(rx.recv(), Some(i));
+            }
+            assert_eq!(rx.recv(), None);
+        });
+
+        for i in 1..=N_SENDS {
+            tx.send(i).unwrap()
+        }
+        drop(tx);
+        consumer.join().unwrap();
+    })
+}
+
 #[test]
 fn tx_close_wakes() {
     loom::model(|| {

+ 5 - 2
src/static_thingbuf.rs

@@ -40,7 +40,10 @@ impl<T, const CAP: usize> StaticThingBuf<T, CAP> {
 
 impl<T: Default, const CAP: usize> StaticThingBuf<T, CAP> {
     pub fn push_ref(&self) -> Result<Ref<'_, T>, Full> {
-        self.core.push_ref(&self.slots)
+        self.core.push_ref(&self.slots).map_err(|e| match e {
+            crate::mpsc::TrySendError::Full(()) => Full(()),
+            _ => unreachable!(),
+        })
     }
 
     #[inline]
@@ -49,7 +52,7 @@ impl<T: Default, const CAP: usize> StaticThingBuf<T, CAP> {
     }
 
     pub fn pop_ref(&self) -> Option<Ref<'_, T>> {
-        self.core.pop_ref(&self.slots)
+        self.core.pop_ref(&self.slots).ok()
     }
 
     #[inline]

+ 0 - 211
src/sync_mpsc.rs

@@ -1,211 +0,0 @@
-use super::*;
-use crate::{
-    error::TrySendError,
-    loom::{
-        atomic::{AtomicUsize, Ordering},
-        sync::Arc,
-        thread::{self, Thread},
-    },
-    util::wait::{WaitCell, WaitResult},
-};
-
-#[cfg(test)]
-mod tests;
-
-pub fn channel<T>(thingbuf: ThingBuf<T>) -> (Sender<T>, Receiver<T>) {
-    let inner = Arc::new(Inner {
-        thingbuf,
-        rx_wait: WaitCell::new(),
-        tx_count: AtomicUsize::new(1),
-    });
-    let tx = Sender {
-        inner: inner.clone(),
-    };
-    let rx = Receiver { inner };
-    (tx, rx)
-}
-
-pub struct Sender<T> {
-    inner: Arc<Inner<T>>,
-}
-
-pub struct Receiver<T> {
-    inner: Arc<Inner<T>>,
-}
-
-pub struct SendRef<'a, T> {
-    inner: &'a Inner<T>,
-    slot: Ref<'a, T>,
-}
-
-struct Inner<T> {
-    thingbuf: ThingBuf<T>,
-    rx_wait: WaitCell<Thread>,
-    tx_count: AtomicUsize,
-}
-
-// === impl Sender ===
-
-impl<T: Default> Sender<T> {
-    pub fn try_send_ref(&self) -> Result<SendRef<'_, T>, TrySendError> {
-        self.inner
-            .thingbuf
-            .push_ref()
-            .map(|slot| SendRef {
-                inner: &*self.inner,
-                slot,
-            })
-            .map_err(|e| {
-                if self.inner.rx_wait.is_rx_closed() {
-                    TrySendError::Closed(error::Closed(()))
-                } else {
-                    self.inner.rx_wait.notify();
-                    TrySendError::AtCapacity(e)
-                }
-            })
-    }
-
-    pub fn try_send(&self, val: T) -> Result<(), TrySendError> {
-        self.try_send_ref()?.with_mut(|slot| {
-            *slot = val;
-        });
-        Ok(())
-    }
-}
-
-impl<T> Clone for Sender<T> {
-    fn clone(&self) -> Self {
-        test_dbg!(self.inner.tx_count.fetch_add(1, Ordering::Relaxed));
-        Self {
-            inner: self.inner.clone(),
-        }
-    }
-}
-
-impl<T> Drop for Sender<T> {
-    fn drop(&mut self) {
-        if test_dbg!(self.inner.tx_count.fetch_sub(1, Ordering::Release)) > 1 {
-            return;
-        }
-
-        // if we are the last sender, synchronize
-        test_dbg!(self.inner.tx_count.load(Ordering::SeqCst));
-        self.inner.rx_wait.close_tx();
-    }
-}
-
-// === impl Receiver ===
-
-impl<T: Default> Receiver<T> {
-    pub fn recv_ref(&self) -> Option<Ref<'_, T>> {
-        loop {
-            // If we got a value, return it!
-            if let Some(r) = self.inner.thingbuf.pop_ref() {
-                return Some(r);
-            }
-
-            // otherwise, gotosleep
-            match test_dbg!(self.inner.rx_wait.wait_with(thread::current)) {
-                WaitResult::TxClosed => {
-                    // All senders have been dropped, but the channel might
-                    // still have messages in it...
-                    return self.inner.thingbuf.pop_ref();
-                }
-                WaitResult::Wait => {
-                    // make sure nobody sent a message while we were registering
-                    // the waiter...
-                    // XXX(eliza): a nicer solution _might_ just be to pack the
-                    // waiter state into the tail idx or something or something
-                    // but that kind of defeats the purpose of just having a
-                    // nice "wrap a queue into a channel" API...
-                    if let Some(val) = self.inner.thingbuf.pop_ref() {
-                        return Some(val);
-                    }
-                    test_println!("parking ({:?})", thread::current());
-                    thread::park();
-                }
-                WaitResult::Notified => {
-                    loom::hint::spin_loop();
-                }
-            }
-        }
-    }
-
-    pub fn try_recv_ref(&self) -> Option<Ref<'_, T>> {
-        self.inner.thingbuf.pop_ref()
-    }
-
-    pub fn recv(&self) -> Option<T> {
-        let val = self.recv_ref()?.with_mut(core::mem::take);
-        Some(val)
-    }
-
-    pub fn is_closed(&self) -> bool {
-        test_dbg!(self.inner.tx_count.load(Ordering::SeqCst)) <= 1
-    }
-}
-
-impl<'a, T: Default> Iterator for &'a Receiver<T> {
-    type Item = Ref<'a, T>;
-
-    fn next(&mut self) -> Option<Self::Item> {
-        self.recv_ref()
-    }
-}
-
-impl<T> Drop for Receiver<T> {
-    fn drop(&mut self) {
-        self.inner.rx_wait.close_rx();
-    }
-}
-
-// === impl SendRef ===
-
-impl<T> SendRef<'_, T> {
-    #[inline]
-    pub fn with<U>(&self, f: impl FnOnce(&T) -> U) -> U {
-        self.slot.with(f)
-    }
-
-    #[inline]
-    pub fn with_mut<U>(&mut self, f: impl FnOnce(&mut T) -> U) -> U {
-        self.slot.with_mut(f)
-    }
-}
-
-impl<T> Drop for SendRef<'_, T> {
-    #[inline]
-    fn drop(&mut self) {
-        test_println!("drop SendRef");
-        self.inner.rx_wait.notify();
-    }
-}
-
-impl<T: fmt::Debug> fmt::Debug for SendRef<'_, T> {
-    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
-        self.with(|val| fmt::Debug::fmt(val, f))
-    }
-}
-
-impl<T: fmt::Display> fmt::Display for SendRef<'_, T> {
-    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
-        self.with(|val| fmt::Display::fmt(val, f))
-    }
-}
-
-impl<T: fmt::Write> fmt::Write for SendRef<'_, T> {
-    #[inline]
-    fn write_str(&mut self, s: &str) -> fmt::Result {
-        self.with_mut(|val| val.write_str(s))
-    }
-
-    #[inline]
-    fn write_char(&mut self, c: char) -> fmt::Result {
-        self.with_mut(|val| val.write_char(c))
-    }
-
-    #[inline]
-    fn write_fmt(&mut self, f: fmt::Arguments<'_>) -> fmt::Result {
-        self.with_mut(|val| val.write_fmt(f))
-    }
-}

+ 7 - 4
src/thingbuf.rs

@@ -7,8 +7,8 @@ use core::{fmt, ptr};
 mod tests;
 
 pub struct ThingBuf<T> {
-    core: Core,
-    slots: Box<[Slot<T>]>,
+    pub(crate) core: Core,
+    pub(crate) slots: Box<[Slot<T>]>,
 }
 
 // === impl ThingBuf ===
@@ -24,7 +24,10 @@ impl<T: Default> ThingBuf<T> {
     }
 
     pub fn push_ref(&self) -> Result<Ref<'_, T>, Full> {
-        self.core.push_ref(&*self.slots)
+        self.core.push_ref(&*self.slots).map_err(|e| match e {
+            crate::mpsc::TrySendError::Full(()) => Full(()),
+            _ => unreachable!(),
+        })
     }
 
     #[inline]
@@ -33,7 +36,7 @@ impl<T: Default> ThingBuf<T> {
     }
 
     pub fn pop_ref(&self) -> Option<Ref<'_, T>> {
-        self.core.pop_ref(&*self.slots)
+        self.core.pop_ref(&*self.slots).ok()
     }
 
     #[inline]

+ 17 - 2
src/util.rs

@@ -21,8 +21,8 @@ pub(crate) struct CachePadded<T>(pub(crate) T);
 // === impl Backoff ===
 
 impl Backoff {
-    const MAX_SPINS: u8 = 6;
-    const MAX_YIELDS: u8 = 10;
+    const MAX_SPINS: u8 = 3;
+    const MAX_YIELDS: u8 = 6;
     #[inline]
     pub(crate) fn new() -> Self {
         Self(0)
@@ -30,10 +30,17 @@ impl Backoff {
 
     #[inline]
     pub(crate) fn spin(&mut self) {
+        #[cfg(not(test))]
         for _ in 0..test_dbg!(1 << self.0.min(Self::MAX_SPINS)) {
             loom::hint::spin_loop();
         }
 
+        #[cfg(test)]
+        {
+            test_println!("hint::spin_loop() (x{})", 1 << self.0.min(Self::MAX_SPINS));
+            loom::hint::spin_loop();
+        }
+
         if self.0 <= Self::MAX_SPINS {
             self.0 += 1;
         }
@@ -42,11 +49,15 @@ impl Backoff {
     #[inline]
     pub(crate) fn spin_yield(&mut self) {
         if self.0 <= Self::MAX_SPINS || cfg!(not(any(feature = "std", test))) {
+            #[cfg(not(test))]
             for _ in 0..1 << self.0 {
                 loom::hint::spin_loop();
             }
+
+            test_println!("hint::spin_loop() (x{})", 1 << self.0);
         }
 
+        test_println!("thread::yield_now()");
         #[cfg(any(test, feature = "std"))]
         loom::thread::yield_now();
 
@@ -54,6 +65,10 @@ impl Backoff {
             self.0 += 1;
         }
     }
+
+    pub(crate) fn done_spinning(&self) -> bool {
+        self.0 >= Self::MAX_SPINS
+    }
 }
 
 // === impl CachePadded ===

+ 25 - 384
src/util/wait.rs

@@ -1,207 +1,32 @@
-use crate::{
-    loom::{
-        atomic::{
-            AtomicUsize,
-            Ordering::{self, *},
-        },
-        UnsafeCell,
-    },
-    util::panic::{self, RefUnwindSafe, UnwindSafe},
-};
-use core::{fmt, ops, task::Waker};
+use crate::util::panic::UnwindSafe;
+use core::{fmt, task::Waker};
 
-#[cfg(feature = "std")]
-use crate::loom::thread;
+mod wait_cell;
+pub(crate) use self::wait_cell::WaitCell;
 
-/// An atomically registered waiter ([`Waker`] or [`Thread`]).
-///
-/// This is inspired by the [`AtomicWaker` type] used in Tokio's
-/// synchronization primitives, with the following modifications:
-///
-/// - Unlike [`AtomicWaker`], a `WaitCell` is generic over the type of the
-///   waiting value. This means it can be used in both asynchronous code (with
-///   [`Waker`]s), or in synchronous, multi-threaded code (with a [`Thread`]).
-/// - An additional bit of state is added to allow setting a "close" bit. This
-///   is so that closing a channel can be tracked in the same atomic as the
-///   receiver's notification state, reducing the number of separate atomic RMW
-///   ops that have to be synchronized between.
-/// - A `WaitCell` is always woken by value. This is just because I didn't
-///   actually need separate "take waiter" and "wake" steps for any of the uses
-///   in `ThingBuf`...
-///
-/// [`AtomicWaker`]: https://github.com/tokio-rs/tokio/blob/09b770c5db31a1f35631600e1d239679354da2dd/tokio/src/sync/task/atomic_waker.rs
-pub(crate) struct WaitCell<T> {
-    lock: AtomicUsize,
-    waiter: UnsafeCell<Option<T>>,
+feature! {
+    #![feature = "alloc"]
+    pub(crate) mod wait_queue;
+    pub(crate) use self::wait_queue::WaitQueue;
 }
 
+#[cfg(feature = "std")]
+use crate::loom::thread;
+
 #[derive(Debug, Eq, PartialEq)]
 pub(crate) enum WaitResult {
     Wait,
-    Notified,
     TxClosed,
+    Notified,
 }
 
+#[derive(Debug)]
+pub(crate) struct NotifyOnDrop<T: Notify>(Option<T>);
+
 pub(crate) trait Notify: UnwindSafe + fmt::Debug {
     fn notify(self);
 }
 
-#[derive(Eq, PartialEq, Copy, Clone)]
-struct State(usize);
-
-// === impl WaitCell ===
-
-impl<T: Notify> WaitCell<T> {
-    pub(crate) fn new() -> Self {
-        Self {
-            lock: AtomicUsize::new(State::WAITING.0),
-            waiter: UnsafeCell::new(None),
-        }
-    }
-
-    pub(crate) fn close_rx(&self) {
-        test_dbg!(self.fetch_or(State::RX_CLOSED, AcqRel));
-    }
-
-    pub(crate) fn is_rx_closed(&self) -> bool {
-        test_dbg!(self.current_state().contains(State::RX_CLOSED))
-    }
-
-    pub(crate) fn wait_with(&self, f: impl FnOnce() -> T) -> WaitResult {
-        test_println!("registering waiter");
-
-        // this is based on tokio's AtomicWaker synchronization strategy
-        match test_dbg!(self.compare_exchange(State::WAITING, State::PARKING)) {
-            // someone else is notifying the receiver, so don't park!
-            Err(actual) if test_dbg!(actual.contains(State::TX_CLOSED)) => {
-                return WaitResult::TxClosed;
-            }
-            Err(actual) if test_dbg!(actual.contains(State::NOTIFYING)) => {
-                // f().notify();
-                // loom::hint::spin_loop();
-                return WaitResult::Notified;
-            }
-
-            Err(actual) => {
-                debug_assert!(
-                    actual == State::PARKING || actual == State::PARKING | State::NOTIFYING
-                );
-                return WaitResult::Wait;
-            }
-            Ok(_) => {}
-        }
-
-        test_println!("-> locked!");
-        let (panicked, prev_waiter) = match panic::catch_unwind(panic::AssertUnwindSafe(f)) {
-            Ok(new_waiter) => {
-                let new_waiter = test_dbg!(new_waiter);
-                let prev_waiter = self
-                    .waiter
-                    .with_mut(|waiter| unsafe { (*waiter).replace(new_waiter) });
-                (None, test_dbg!(prev_waiter))
-            }
-            Err(panic) => (Some(panic), None),
-        };
-
-        let result = match test_dbg!(self.compare_exchange(State::PARKING, State::WAITING)) {
-            Ok(_) => {
-                let _ = panic::catch_unwind(move || drop(prev_waiter));
-
-                WaitResult::Wait
-            }
-            Err(actual) => {
-                test_println!("-> was notified; state={:?}", actual);
-                let waiter = self.waiter.with_mut(|waiter| unsafe { (*waiter).take() });
-                // Reset to the WAITING state by clearing everything *except*
-                // the closed bits (which must remain set).
-                let state = test_dbg!(self.fetch_and(State::TX_CLOSED | State::RX_CLOSED, AcqRel));
-                // The only valid state transition while we were parking is to
-                // add the TX_CLOSED bit.
-                debug_assert!(
-                    state == actual || state == actual | State::TX_CLOSED,
-                    "state changed unexpectedly while parking!"
-                );
-
-                if let Some(prev_waiter) = prev_waiter {
-                    let _ = panic::catch_unwind(move || {
-                        prev_waiter.notify();
-                    });
-                }
-
-                if let Some(waiter) = waiter {
-                    debug_assert!(panicked.is_none());
-                    waiter.notify();
-                }
-
-                if test_dbg!(state.contains(State::TX_CLOSED)) {
-                    WaitResult::TxClosed
-                } else {
-                    WaitResult::Notified
-                }
-            }
-        };
-
-        if let Some(panic) = panicked {
-            panic::resume_unwind(panic);
-        }
-
-        result
-    }
-
-    pub(crate) fn notify(&self) {
-        self.notify2(false)
-    }
-
-    pub(crate) fn close_tx(&self) {
-        self.notify2(true)
-    }
-
-    fn notify2(&self, close: bool) {
-        test_println!("notifying; close={:?};", close);
-        let bits = if close {
-            State::NOTIFYING | State::TX_CLOSED
-        } else {
-            State::NOTIFYING
-        };
-        test_dbg!(bits);
-        if test_dbg!(self.fetch_or(bits, AcqRel)) == State::WAITING {
-            // we have the lock!
-            let waiter = self.waiter.with_mut(|thread| unsafe { (*thread).take() });
-
-            test_dbg!(self.fetch_and(!State::NOTIFYING, AcqRel));
-
-            if let Some(waiter) = test_dbg!(waiter) {
-                waiter.notify();
-            }
-        }
-    }
-}
-
-impl<T> WaitCell<T> {
-    #[inline(always)]
-    fn compare_exchange(&self, State(curr): State, State(new): State) -> Result<State, State> {
-        self.lock
-            .compare_exchange(curr, new, AcqRel, Acquire)
-            .map(State)
-            .map_err(State)
-    }
-
-    #[inline(always)]
-    fn fetch_and(&self, State(state): State, order: Ordering) -> State {
-        State(self.lock.fetch_and(state, order))
-    }
-
-    #[inline(always)]
-    fn fetch_or(&self, State(state): State, order: Ordering) -> State {
-        State(self.lock.fetch_or(state, order))
-    }
-
-    #[inline(always)]
-    fn current_state(&self) -> State {
-        State(self.lock.load(Acquire))
-    }
-}
-
 #[cfg(feature = "std")]
 impl Notify for thread::Thread {
     fn notify(self) {
@@ -217,206 +42,22 @@ impl Notify for Waker {
     }
 }
 
-impl<T: UnwindSafe> UnwindSafe for WaitCell<T> {}
-impl<T: RefUnwindSafe> RefUnwindSafe for WaitCell<T> {}
-unsafe impl<T: Send> Send for WaitCell<T> {}
-unsafe impl<T: Send> Sync for WaitCell<T> {}
-
-impl<T> fmt::Debug for WaitCell<T> {
-    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
-        f.debug_struct("WaitCell")
-            .field("state", &self.current_state())
-            .finish()
-    }
-}
-
-// === impl State ===
-
-impl State {
-    const WAITING: Self = Self(0b00);
-    const PARKING: Self = Self(0b01);
-    const NOTIFYING: Self = Self(0b10);
-    const TX_CLOSED: Self = Self(0b100);
-    const RX_CLOSED: Self = Self(0b1000);
-
-    fn contains(self, Self(state): Self) -> bool {
-        self.0 & state == state
-    }
-}
-
-impl ops::BitOr for State {
-    type Output = Self;
-
-    fn bitor(self, Self(rhs): Self) -> Self::Output {
-        Self(self.0 | rhs)
+impl<T: Notify> NotifyOnDrop<T> {
+    pub(crate) fn new(notify: T) -> Self {
+        Self(Some(notify))
     }
 }
 
-impl ops::Not for State {
-    type Output = Self;
-
-    fn not(self) -> Self::Output {
-        Self(!self.0)
+impl<T: Notify> Notify for NotifyOnDrop<T> {
+    fn notify(self) {
+        drop(self)
     }
 }
 
-impl fmt::Debug for State {
-    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
-        let mut has_states = false;
-        macro_rules! f_bits {
-            ($self: expr, $f: expr, $has_states: ident, $($name: ident),+) => {
-                $(
-                    if $self.contains(Self::$name) {
-                        if $has_states {
-                            $f.write_str(" | ")?;
-                        }
-                        $f.write_str(stringify!($name))?;
-                        $has_states = true;
-                    }
-                )+
-
-            };
-        }
-
-        f_bits!(self, f, has_states, PARKING, NOTIFYING, TX_CLOSED, RX_CLOSED);
-
-        if !has_states {
-            if *self == Self::WAITING {
-                return f.write_str("WAITING");
-            }
-
-            f.debug_tuple("UnknownState")
-                .field(&format_args!("{:#b}", self.0))
-                .finish()?;
+impl<T: Notify> Drop for NotifyOnDrop<T> {
+    fn drop(&mut self) {
+        if let Some(notify) = self.0.take() {
+            notify.notify();
         }
-
-        Ok(())
-    }
-}
-
-#[cfg(test)]
-mod tests {
-    use super::*;
-    use crate::loom::{
-        self, future,
-        sync::atomic::{AtomicUsize, Ordering::Relaxed},
-        thread,
-    };
-    #[cfg(feature = "alloc")]
-    use alloc::sync::Arc;
-    use core::task::{Poll, Waker};
-
-    struct Chan {
-        num: AtomicUsize,
-        task: WaitCell<Waker>,
-    }
-
-    const NUM_NOTIFY: usize = 2;
-
-    async fn wait_on(chan: Arc<Chan>) {
-        futures::future::poll_fn(move |cx| {
-            let res = test_dbg!(chan.task.wait_with(|| cx.waker().clone()));
-
-            if NUM_NOTIFY == chan.num.load(Relaxed) {
-                return Poll::Ready(());
-            }
-
-            if res == WaitResult::Notified || res == WaitResult::TxClosed {
-                return Poll::Ready(());
-            }
-
-            Poll::Pending
-        })
-        .await
-    }
-
-    #[test]
-    #[cfg(feature = "alloc")]
-    fn basic_notification() {
-        loom::model(|| {
-            let chan = Arc::new(Chan {
-                num: AtomicUsize::new(0),
-                task: WaitCell::new(),
-            });
-
-            for _ in 0..NUM_NOTIFY {
-                let chan = chan.clone();
-
-                thread::spawn(move || {
-                    chan.num.fetch_add(1, Relaxed);
-                    chan.task.notify();
-                });
-            }
-
-            future::block_on(wait_on(chan));
-        });
-    }
-
-    #[test]
-    #[cfg(feature = "alloc")]
-    fn tx_close() {
-        loom::model(|| {
-            let chan = Arc::new(Chan {
-                num: AtomicUsize::new(0),
-                task: WaitCell::new(),
-            });
-
-            thread::spawn({
-                let chan = chan.clone();
-                move || {
-                    chan.num.fetch_add(1, Relaxed);
-                    chan.task.notify();
-                }
-            });
-
-            thread::spawn({
-                let chan = chan.clone();
-                move || {
-                    chan.num.fetch_add(1, Relaxed);
-                    chan.task.close_tx();
-                }
-            });
-
-            future::block_on(wait_on(chan));
-        });
-    }
-
-    #[test]
-    #[cfg(feature = "std")]
-    fn test_panicky_waker() {
-        use std::panic;
-        use std::ptr;
-        use std::task::{RawWaker, RawWakerVTable, Waker};
-
-        static PANICKING_VTABLE: RawWakerVTable =
-            RawWakerVTable::new(|_| panic!("clone"), |_| (), |_| (), |_| ());
-
-        let panicking = unsafe { Waker::from_raw(RawWaker::new(ptr::null(), &PANICKING_VTABLE)) };
-
-        loom::model(move || {
-            let chan = Arc::new(Chan {
-                num: AtomicUsize::new(0),
-                task: WaitCell::new(),
-            });
-
-            for _ in 0..NUM_NOTIFY {
-                let chan = chan.clone();
-
-                thread::spawn(move || {
-                    chan.num.fetch_add(1, Relaxed);
-                    chan.task.notify();
-                });
-            }
-
-            // Note: this panic should have no effect on the overall state of the
-            // waker and it should proceed as normal.
-            //
-            // A thread above might race to flag a wakeup, and a WAKING state will
-            // be preserved if this expected panic races with that so the below
-            // procedure should be allowed to continue uninterrupted.
-            let _ = panic::catch_unwind(|| chan.task.wait_with(|| panicking.clone()));
-
-            future::block_on(wait_on(chan));
-        });
     }
 }

+ 388 - 0
src/util/wait/wait_cell.rs

@@ -0,0 +1,388 @@
+use super::{Notify, WaitResult};
+use crate::{
+    loom::{
+        atomic::{
+            AtomicUsize,
+            Ordering::{self, *},
+        },
+        UnsafeCell,
+    },
+    util::panic::{self, RefUnwindSafe, UnwindSafe},
+};
+use core::{fmt, ops};
+
+/// An atomically registered waiter ([`Waker`] or [`Thread`]).
+///
+/// This is inspired by the [`AtomicWaker` type] used in Tokio's
+/// synchronization primitives, with the following modifications:
+///
+/// - Unlike [`AtomicWaker`], a `WaitCell` is generic over the type of the
+///   waiting value. This means it can be used in both asynchronous code (with
+///   [`Waker`]s), or in synchronous, multi-threaded code (with a [`Thread`]).
+/// - An additional bit of state is added to allow setting a "close" bit. This
+///   is so that closing a channel can be tracked in the same atomic as the
+///   receiver's notification state, reducing the number of separate atomic RMW
+///   ops that have to be synchronized between.
+/// - A `WaitCell` is always woken by value. This is just because I didn't
+///   actually need separate "take waiter" and "wake" steps for any of the uses
+///   in `ThingBuf`...
+///
+/// [`AtomicWaker`]: https://github.com/tokio-rs/tokio/blob/09b770c5db31a1f35631600e1d239679354da2dd/tokio/src/sync/task/atomic_waker.rs
+pub(crate) struct WaitCell<T> {
+    lock: AtomicUsize,
+    waiter: UnsafeCell<Option<T>>,
+}
+
+#[derive(Eq, PartialEq, Copy, Clone)]
+struct State(usize);
+
+// === impl WaitCell ===
+impl<T> WaitCell<T> {
+    #[cfg(not(test))]
+    pub(crate) const fn new() -> Self {
+        Self {
+            lock: AtomicUsize::new(State::WAITING.0),
+            waiter: UnsafeCell::new(None),
+        }
+    }
+
+    #[cfg(test)]
+    pub(crate) fn new() -> Self {
+        Self {
+            lock: AtomicUsize::new(State::WAITING.0),
+            waiter: UnsafeCell::new(None),
+        }
+    }
+}
+
+impl<T: Notify> WaitCell<T> {
+    pub(crate) fn wait_with(&self, f: impl FnOnce() -> T) -> WaitResult {
+        test_println!("registering waiter");
+
+        // this is based on tokio's AtomicWaker synchronization strategy
+        match test_dbg!(self.compare_exchange(State::WAITING, State::PARKING, Acquire)) {
+            // someone else is notifying the receiver, so don't park!
+            Err(actual) if test_dbg!(actual.contains(State::TX_CLOSED)) => {
+                return WaitResult::TxClosed;
+            }
+            Err(actual) if test_dbg!(actual.contains(State::NOTIFYING)) => {
+                f().notify();
+                crate::loom::hint::spin_loop();
+                return WaitResult::Notified;
+            }
+
+            Err(actual) => {
+                debug_assert!(
+                    actual == State::PARKING || actual == State::PARKING | State::NOTIFYING
+                );
+                return WaitResult::Wait;
+            }
+            Ok(_) => {}
+        }
+
+        test_println!("-> locked!");
+        let (panicked, prev_waiter) = match panic::catch_unwind(panic::AssertUnwindSafe(f)) {
+            Ok(new_waiter) => {
+                let new_waiter = test_dbg!(new_waiter);
+                let prev_waiter = self
+                    .waiter
+                    .with_mut(|waiter| unsafe { (*waiter).replace(new_waiter) });
+                (None, test_dbg!(prev_waiter))
+            }
+            Err(panic) => (Some(panic), None),
+        };
+
+        let result = match test_dbg!(self.compare_exchange(State::PARKING, State::WAITING, AcqRel))
+        {
+            Ok(_) => {
+                let _ = panic::catch_unwind(move || drop(prev_waiter));
+
+                WaitResult::Wait
+            }
+            Err(actual) => {
+                test_println!("-> was notified; state={:?}", actual);
+                let waiter = self.waiter.with_mut(|waiter| unsafe { (*waiter).take() });
+                // Reset to the WAITING state by clearing everything *except*
+                // the closed bits (which must remain set).
+                let state = test_dbg!(self.fetch_and(State::TX_CLOSED, AcqRel));
+                // The only valid state transition while we were parking is to
+                // add the TX_CLOSED bit.
+                debug_assert!(
+                    state == actual || state == actual | State::TX_CLOSED,
+                    "state changed unexpectedly while parking!"
+                );
+
+                if let Some(prev_waiter) = prev_waiter {
+                    let _ = panic::catch_unwind(move || {
+                        prev_waiter.notify();
+                    });
+                }
+
+                if let Some(waiter) = waiter {
+                    debug_assert!(panicked.is_none());
+                    waiter.notify();
+                }
+
+                if test_dbg!(state.contains(State::TX_CLOSED)) {
+                    WaitResult::TxClosed
+                } else {
+                    WaitResult::Notified
+                }
+            }
+        };
+
+        if let Some(panic) = panicked {
+            panic::resume_unwind(panic);
+        }
+
+        result
+    }
+
+    pub(crate) fn notify(&self) -> bool {
+        self.notify2(false)
+    }
+
+    pub(crate) fn close_tx(&self) {
+        self.notify2(true);
+    }
+
+    fn notify2(&self, close: bool) -> bool {
+        test_println!("notifying; close={:?};", close);
+        let bits = if close {
+            State::NOTIFYING | State::TX_CLOSED
+        } else {
+            State::NOTIFYING
+        };
+        test_dbg!(bits);
+        if test_dbg!(self.fetch_or(bits, AcqRel)) == State::WAITING {
+            // we have the lock!
+            let waiter = self.waiter.with_mut(|thread| unsafe { (*thread).take() });
+
+            test_dbg!(self.fetch_and(!State::NOTIFYING, AcqRel));
+
+            if let Some(waiter) = test_dbg!(waiter) {
+                waiter.notify();
+                return true;
+            }
+        }
+        false
+    }
+}
+
+impl<T> WaitCell<T> {
+    #[inline(always)]
+    fn compare_exchange(
+        &self,
+        State(curr): State,
+        State(new): State,
+        success: Ordering,
+    ) -> Result<State, State> {
+        self.lock
+            .compare_exchange(curr, new, success, Acquire)
+            .map(State)
+            .map_err(State)
+    }
+
+    #[inline(always)]
+    fn fetch_and(&self, State(state): State, order: Ordering) -> State {
+        State(self.lock.fetch_and(state, order))
+    }
+
+    #[inline(always)]
+    fn fetch_or(&self, State(state): State, order: Ordering) -> State {
+        State(self.lock.fetch_or(state, order))
+    }
+
+    #[inline(always)]
+    fn current_state(&self) -> State {
+        State(self.lock.load(Acquire))
+    }
+}
+
+impl<T: UnwindSafe> UnwindSafe for WaitCell<T> {}
+impl<T: RefUnwindSafe> RefUnwindSafe for WaitCell<T> {}
+unsafe impl<T: Send> Send for WaitCell<T> {}
+unsafe impl<T: Send> Sync for WaitCell<T> {}
+
+impl<T> fmt::Debug for WaitCell<T> {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        f.debug_struct("WaitCell")
+            .field("state", &self.current_state())
+            .finish()
+    }
+}
+
+// === impl State ===
+
+impl State {
+    const WAITING: Self = Self(0b00);
+    const PARKING: Self = Self(0b01);
+    const NOTIFYING: Self = Self(0b10);
+    const TX_CLOSED: Self = Self(0b100);
+
+    fn contains(self, Self(state): Self) -> bool {
+        self.0 & state == state
+    }
+}
+
+impl ops::BitOr for State {
+    type Output = Self;
+
+    fn bitor(self, Self(rhs): Self) -> Self::Output {
+        Self(self.0 | rhs)
+    }
+}
+
+impl ops::Not for State {
+    type Output = Self;
+
+    fn not(self) -> Self::Output {
+        Self(!self.0)
+    }
+}
+
+impl fmt::Debug for State {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        let mut has_states = false;
+
+        fmt_bits!(self, f, has_states, PARKING, NOTIFYING, TX_CLOSED);
+
+        if !has_states {
+            if *self == Self::WAITING {
+                return f.write_str("WAITING");
+            }
+
+            f.debug_tuple("UnknownState")
+                .field(&format_args!("{:#b}", self.0))
+                .finish()?;
+        }
+
+        Ok(())
+    }
+}
+#[cfg(test)]
+mod tests {
+    use super::*;
+    use crate::loom::{
+        future,
+        sync::atomic::{AtomicUsize, Ordering::Relaxed},
+        thread,
+    };
+    #[cfg(feature = "alloc")]
+    use alloc::sync::Arc;
+    use core::task::{Poll, Waker};
+
+    struct Chan {
+        num: AtomicUsize,
+        task: WaitCell<Waker>,
+    }
+
+    const NUM_NOTIFY: usize = 2;
+
+    async fn wait_on(chan: Arc<Chan>) {
+        futures::future::poll_fn(move |cx| {
+            let res = test_dbg!(chan.task.wait_with(|| cx.waker().clone()));
+
+            if NUM_NOTIFY == chan.num.load(Relaxed) {
+                return Poll::Ready(());
+            }
+
+            if res == WaitResult::Notified || res == WaitResult::TxClosed {
+                return Poll::Ready(());
+            }
+
+            Poll::Pending
+        })
+        .await
+    }
+
+    #[test]
+    #[cfg(feature = "alloc")]
+    fn basic_notification() {
+        crate::loom::model(|| {
+            let chan = Arc::new(Chan {
+                num: AtomicUsize::new(0),
+                task: WaitCell::new(),
+            });
+
+            for _ in 0..NUM_NOTIFY {
+                let chan = chan.clone();
+
+                thread::spawn(move || {
+                    chan.num.fetch_add(1, Relaxed);
+                    chan.task.notify();
+                });
+            }
+
+            future::block_on(wait_on(chan));
+        });
+    }
+
+    #[test]
+    #[cfg(feature = "alloc")]
+    fn tx_close() {
+        crate::loom::model(|| {
+            let chan = Arc::new(Chan {
+                num: AtomicUsize::new(0),
+                task: WaitCell::new(),
+            });
+
+            thread::spawn({
+                let chan = chan.clone();
+                move || {
+                    chan.num.fetch_add(1, Relaxed);
+                    chan.task.notify();
+                }
+            });
+
+            thread::spawn({
+                let chan = chan.clone();
+                move || {
+                    chan.num.fetch_add(1, Relaxed);
+                    chan.task.close_tx();
+                }
+            });
+
+            future::block_on(wait_on(chan));
+        });
+    }
+
+    // #[test]
+    // #[cfg(feature = "std")]
+    // fn test_panicky_waker() {
+    //     use std::panic;
+    //     use std::ptr;
+    //     use std::task::{RawWaker, RawWakerVTable, Waker};
+
+    //     static PANICKING_VTABLE: RawWakerVTable =
+    //         RawWakerVTable::new(|_| panic!("clone"), |_| (), |_| (), |_| ());
+
+    //     let panicking = unsafe { Waker::from_raw(RawWaker::new(ptr::null(), &PANICKING_VTABLE)) };
+
+    //     loom::model(move || {
+    //         let chan = Arc::new(Chan {
+    //             num: AtomicUsize::new(0),
+    //             task: WaitCell::new(),
+    //         });
+
+    //         for _ in 0..NUM_NOTIFY {
+    //             let chan = chan.clone();
+
+    //             thread::spawn(move || {
+    //                 chan.num.fetch_add(1, Relaxed);
+    //                 chan.task.notify();
+    //             });
+    //         }
+
+    //         // Note: this panic should have no effect on the overall state of the
+    //         // waker and it should proceed as normal.
+    //         //
+    //         // A thread above might race to flag a wakeup, and a WAKING state will
+    //         // be preserved if this expected panic races with that so the below
+    //         // procedure should be allowed to continue uninterrupted.
+    //         let _ = panic::catch_unwind(|| chan.task.wait_with(|| panicking.clone()));
+
+    //         future::block_on(wait_on(chan));
+    //     });
+    // }
+}

+ 248 - 0
src/util/wait/wait_queue.rs

@@ -0,0 +1,248 @@
+use super::{Notify, WaitResult};
+use crate::{
+    loom::{
+        atomic::{
+            AtomicUsize,
+            Ordering::{self, *},
+        },
+        UnsafeCell,
+    },
+    util::{panic, Backoff, CachePadded},
+};
+use alloc::collections::VecDeque;
+use core::fmt;
+
+/// A mediocre wait queue, implemented as a spinlock around a `VecDeque` of
+/// waiters.
+// TODO(eliza): this can almost certainly be replaced with an intrusive list of
+// some kind, but crossbeam uses a spinlock + vec, so it's _probably_ fine...
+// XXX(eliza): the biggest downside of this is that it can't be used without
+// `liballoc`, which is sad for `no-std` async-await users...
+pub(crate) struct WaitQueue<T> {
+    locked: CachePadded<AtomicUsize>,
+    queue: UnsafeCell<VecDeque<T>>,
+}
+
+pub(crate) struct Locked<'a, T> {
+    queue: &'a WaitQueue<T>,
+    state: State,
+}
+
+#[derive(Copy, Clone)]
+struct State(usize);
+
+impl<T> WaitQueue<T> {
+    pub(crate) fn new() -> Self {
+        Self {
+            locked: CachePadded(AtomicUsize::new(State::UNLOCKED.0 | State::EMPTY.0)),
+            queue: UnsafeCell::new(VecDeque::new()),
+        }
+    }
+
+    fn compare_exchange_weak(
+        &self,
+        curr: State,
+        next: State,
+        success: Ordering,
+        failure: Ordering,
+    ) -> Result<State, State> {
+        let res = self
+            .locked
+            .compare_exchange_weak(curr.0, next.0, success, failure)
+            .map(State)
+            .map_err(State);
+        test_println!(
+            "self.state.compare_exchange_weak({:?}, {:?}, {:?}, {:?}) = {:?}",
+            curr,
+            next,
+            success,
+            failure,
+            res
+        );
+        res
+    }
+
+    fn fetch_clear(&self, state: State, order: Ordering) -> State {
+        let res = State(self.locked.fetch_and(!state.0, order));
+        test_println!(
+            "self.state.fetch_clear({:?}, {:?}) = {:?}",
+            state,
+            order,
+            res
+        );
+        res
+    }
+
+    fn lock(&self) -> Result<Locked<'_, T>, State> {
+        let mut backoff = Backoff::new();
+        let mut state = State(self.locked.load(Ordering::Relaxed));
+        loop {
+            test_dbg!(&state);
+            if test_dbg!(state.contains(State::CLOSED)) {
+                return Err(state);
+            }
+
+            if !test_dbg!(state.contains(State::LOCKED)) {
+                match self.compare_exchange_weak(
+                    state,
+                    State(state.0 | State::LOCKED.0),
+                    AcqRel,
+                    Acquire,
+                ) {
+                    Ok(_) => return Ok(Locked { queue: self, state }),
+                    Err(actual) => {
+                        state = actual;
+                        backoff.spin();
+                    }
+                }
+            } else {
+                state = State(self.locked.load(Ordering::Relaxed));
+                backoff.spin_yield();
+            }
+        }
+    }
+
+    pub(crate) fn push_waiter(&self, mk_waiter: impl FnOnce() -> T) -> WaitResult {
+        if let Ok(mut lock) = self.lock() {
+            if lock.state.queued() > 0 {
+                lock.state = lock.state.sub_queued();
+                return WaitResult::Notified;
+            }
+            lock.queue.queue.with_mut(|q| unsafe {
+                (*q).push_back(mk_waiter());
+            });
+            WaitResult::Wait
+        } else {
+            WaitResult::TxClosed
+        }
+    }
+
+    pub(crate) fn drain(&self) {
+        if let Ok(lock) = self.lock() {
+            // if test_dbg!(lock.state.contains(State::EMPTY)) {
+            //     return;
+            // }
+            lock.queue.queue.with_mut(|q| {
+                let waiters = unsafe { (*q).drain(..) };
+                for waiter in waiters {
+                    drop(waiter);
+                }
+            })
+        }
+    }
+}
+
+impl<T: Notify> WaitQueue<T> {
+    pub(crate) fn notify(&self) -> bool {
+        test_println!("notifying tx");
+
+        if let Ok(mut lock) = self.lock() {
+            return lock.notify();
+        }
+
+        false
+    }
+}
+
+impl<T> Drop for WaitQueue<T> {
+    fn drop(&mut self) {
+        self.drain();
+    }
+}
+
+impl<T> fmt::Debug for WaitQueue<T> {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        f.write_str("WaitQueue(..)")
+    }
+}
+
+impl<T: Notify> Locked<'_, T> {
+    fn notify(&mut self) -> bool {
+        // if test_dbg!(self.state.contains(State::EMPTY)) {
+        //     self.state = self.state.add_queued();
+        //     return false;
+        // }
+        self.queue.queue.with_mut(|q| {
+            let q = unsafe { &mut *q };
+            if let Some(waiter) = q.pop_front() {
+                waiter.notify();
+                if q.is_empty() {
+                    self.queue.fetch_clear(State::EMPTY, Release);
+                }
+                true
+            } else {
+                self.state = self.state.add_queued();
+                false
+            }
+        })
+    }
+
+    // TODO(eliza): future cancellation nonsense...
+    #[allow(dead_code)]
+    pub(crate) fn remove(&mut self, i: usize) -> Option<T> {
+        self.queue.queue.with_mut(|q| unsafe { (*q).remove(i) })
+    }
+}
+
+impl<T> Drop for Locked<'_, T> {
+    fn drop(&mut self) {
+        test_dbg!(State(self.queue.locked.swap(self.state.0, Release)));
+    }
+}
+
+impl<T: panic::UnwindSafe> panic::UnwindSafe for WaitQueue<T> {}
+impl<T: panic::RefUnwindSafe> panic::RefUnwindSafe for WaitQueue<T> {}
+unsafe impl<T: Send> Send for WaitQueue<T> {}
+unsafe impl<T: Send> Sync for WaitQueue<T> {}
+
+// === impl State ===
+
+impl State {
+    const UNLOCKED: Self = Self(0b00);
+    const LOCKED: Self = Self(0b01);
+    const EMPTY: Self = Self(0b10);
+    const CLOSED: Self = Self(0b100);
+
+    const FLAG_BITS: usize = Self::LOCKED.0 | Self::EMPTY.0 | Self::CLOSED.0;
+    const QUEUED_SHIFT: usize = Self::FLAG_BITS.trailing_ones() as usize;
+    const QUEUED_ONE: usize = 1 << Self::QUEUED_SHIFT;
+
+    fn queued(self) -> usize {
+        self.0 >> Self::QUEUED_SHIFT
+    }
+
+    fn add_queued(self) -> Self {
+        Self(self.0 + Self::QUEUED_ONE)
+    }
+
+    fn contains(self, Self(state): Self) -> bool {
+        self.0 & state == state
+    }
+
+    fn sub_queued(self) -> Self {
+        let flags = self.0 & Self::FLAG_BITS;
+        Self(self.0 & (!Self::FLAG_BITS).saturating_sub(Self::QUEUED_ONE) | flags)
+    }
+}
+
+impl fmt::Debug for State {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        f.write_str("State(")?;
+        let mut has_flags = false;
+
+        fmt_bits!(self, f, has_flags, LOCKED, EMPTY, CLOSED);
+
+        if !has_flags {
+            f.write_str("UNLOCKED")?;
+        }
+
+        let queued = self.queued();
+        if queued > 0 {
+            write!(f, ", queued: {})", queued)?;
+        } else {
+            f.write_str(")")?;
+        }
+
+        Ok(())
+    }
+}

+ 45 - 0
tests/mpsc_async.rs

@@ -0,0 +1,45 @@
+use thingbuf::{mpsc, ThingBuf};
+
+#[tokio::test(flavor = "multi_thread")]
+async fn basically_works() {
+    use std::collections::HashSet;
+
+    const N_SENDS: usize = 10;
+    const N_PRODUCERS: usize = 10;
+
+    async fn do_producer(tx: mpsc::Sender<usize>, n: usize) {
+        let tag = n * N_SENDS;
+        for i in 0..N_SENDS {
+            let msg = i + tag;
+            println!("sending {}...", msg);
+            tx.send(msg).await.unwrap();
+            println!("sent {}!", msg);
+        }
+        println!("PRODUCER {} DONE!", n);
+    }
+
+    let (tx, rx) = mpsc::channel(ThingBuf::new(N_SENDS / 2));
+    for n in 0..N_PRODUCERS {
+        tokio::spawn(do_producer(tx.clone(), n));
+    }
+    drop(tx);
+
+    let mut results = HashSet::new();
+    while let Some(val) = {
+        println!("receiving...");
+        rx.recv().await
+    } {
+        println!("received {}!", val);
+        results.insert(val);
+    }
+
+    let results = dbg!(results);
+
+    for n in 0..N_PRODUCERS {
+        let tag = n * N_SENDS;
+        for i in 0..N_SENDS {
+            let msg = i + tag;
+            assert!(results.contains(&msg), "missing message {:?}", msg);
+        }
+    }
+}

+ 51 - 0
tests/mpsc_sync.rs

@@ -0,0 +1,51 @@
+use std::thread;
+use thingbuf::{mpsc::sync, ThingBuf};
+
+#[test]
+fn basically_works() {
+    use std::collections::HashSet;
+
+    const N_SENDS: usize = 10;
+    const N_PRODUCERS: usize = 10;
+
+    fn start_producer(tx: sync::Sender<usize>, n: usize) -> thread::JoinHandle<()> {
+        let tag = n * N_SENDS;
+        thread::Builder::new()
+            .name(format!("producer {}", n))
+            .spawn(move || {
+                for i in 0..N_SENDS {
+                    let msg = i + tag;
+                    println!("[producer {}] sending {}...", n, msg);
+                    tx.send(msg).unwrap();
+                    println!("[producer {}] sent {}!", n, msg);
+                }
+                println!("[producer {}] DONE!", n);
+            })
+            .expect("spawning threads should succeed")
+    }
+
+    let (tx, rx) = sync::channel(ThingBuf::new(N_SENDS / 2));
+    for n in 0..N_PRODUCERS {
+        start_producer(tx.clone(), n);
+    }
+    drop(tx);
+
+    let mut results = HashSet::new();
+    while let Some(val) = {
+        println!("receiving...");
+        rx.recv()
+    } {
+        println!("received {}!", val);
+        results.insert(val);
+    }
+
+    let results = dbg!(results);
+
+    for n in 0..N_PRODUCERS {
+        let tag = n * N_SENDS;
+        for i in 0..N_SENDS {
+            let msg = i + tag;
+            assert!(results.contains(&msg), "missing message {:?}", msg);
+        }
+    }
+}