|
@@ -1,6 +1,6 @@
|
|
|
use crate::{
|
|
|
loom::{
|
|
|
- atomic::{AtomicBool, AtomicUsize, Ordering::*},
|
|
|
+ atomic::{AtomicUsize, Ordering::*},
|
|
|
cell::UnsafeCell,
|
|
|
},
|
|
|
util::{mutex::Mutex, CachePadded},
|
|
@@ -57,13 +57,27 @@ use core::{fmt, marker::PhantomPinned, pin::Pin, ptr::NonNull};
|
|
|
/// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue
|
|
|
#[derive(Debug)]
|
|
|
pub(crate) struct WaitQueue<T> {
|
|
|
- /// The wait queue's state variable. The first bit indicates whether the
|
|
|
- /// queue is closed; the remainder is a counter of notifications assigned to
|
|
|
- /// the queue because no waiters were currently available to be woken.
|
|
|
+ /// The wait queue's state variable.
|
|
|
///
|
|
|
- /// These stored notifications are "consumed" when new waiters are
|
|
|
- /// registered; those waiters will be woken immediately rather than being
|
|
|
- /// enqueued to wait.
|
|
|
+ /// The queue is always in one of the following states:
|
|
|
+ ///
|
|
|
+ /// - `EMPTY`: No waiters are queued, and there is no pending notification.
|
|
|
+ /// Waiting while the queue is in this state will enqueue the waiter;
|
|
|
+ /// notifying while in this state will store a pending notification in the
|
|
|
+ /// queue, transitioning to the `WAKING` state.
|
|
|
+ ///
|
|
|
+ /// - `WAITING`: There are one or more waiters in the queue. Waiting while
|
|
|
+ /// the queue is in this state will not transition the state. Waking while
|
|
|
+ /// in this state will wake the first waiter in the queue; if this empties
|
|
|
+ /// the queue, then the queue will transition to the `EMPTY` state.
|
|
|
+ ///
|
|
|
+ /// - `WAKING`: The queue has a stored notification. Waiting while the queue
|
|
|
+ /// is in this state will consume the pending notification *without*
|
|
|
+ /// enqueueing the waiter and transition the queue to the `EMPTY` state.
|
|
|
+ /// Waking while in this state will leave the queue in this state.
|
|
|
+ ///
|
|
|
+ /// - `CLOSED`: The queue is closed. Waiting while in this state will return
|
|
|
+ /// [`WaitResult::Closed`] without transitioning the queue's state.
|
|
|
state: CachePadded<AtomicUsize>,
|
|
|
/// The linked list of waiters.
|
|
|
///
|
|
@@ -77,16 +91,44 @@ pub(crate) struct WaitQueue<T> {
|
|
|
/// modified through the list, so the lock must be held when modifying the
|
|
|
/// node.
|
|
|
///
|
|
|
- /// A spinlock is used on `no_std` platforms; [`std::sync::Mutex`] is used
|
|
|
- /// when the standard library is available.
|
|
|
+ /// A spinlock is used on `no_std` platforms; [`std::sync::Mutex`] or
|
|
|
+ /// `parking_lot::Mutex` are used when the standard library is available
|
|
|
+ /// (depending on feature flags).
|
|
|
list: Mutex<List<T>>,
|
|
|
}
|
|
|
|
|
|
/// A waiter node which may be linked into a wait queue.
|
|
|
#[derive(Debug)]
|
|
|
pub(crate) struct Waiter<T> {
|
|
|
+ /// The waiter's state variable.
|
|
|
+ ///
|
|
|
+ /// A waiter is always in one of the following states:
|
|
|
+ ///
|
|
|
+ /// - `EMPTY`: The waiter is not linked in the queue, and does not have a
|
|
|
+ /// `Thread`/`Waker`.
|
|
|
+ ///
|
|
|
+ /// - `WAITING`: The waiter is linked in the queue and has a
|
|
|
+ /// `Thread`/`Waker`.
|
|
|
+ ///
|
|
|
+ /// - `WAKING`: The waiter has been notified by the wait queue. If it is in
|
|
|
+ /// this state, it is *not* linked into the queue, and does not have a
|
|
|
+ /// `Thread`/`Waker`.
|
|
|
+ ///
|
|
|
+ /// - `WAKING`: The waiter has been notified because the wait queue closed.
|
|
|
+ /// If it is in this state, it is *not* linked into the queue, and does
|
|
|
+ /// not have a `Thread`/`Waker`.
|
|
|
+ ///
|
|
|
+ /// This may be inspected without holding the lock; it can be used to
|
|
|
+ /// determine whether the lock must be acquired.
|
|
|
+ state: CachePadded<AtomicUsize>,
|
|
|
+
|
|
|
+ /// The linked list node and stored `Thread`/`Waker`.
|
|
|
+ ///
|
|
|
+ /// # Safety
|
|
|
+ ///
|
|
|
+ /// This `UnsafeCell` may only be accessed while holding the `Mutex` around
|
|
|
+ /// the wait queue's linked list!
|
|
|
node: UnsafeCell<Node<T>>,
|
|
|
- pub(crate) was_woken_from_queue: AtomicBool,
|
|
|
}
|
|
|
|
|
|
#[derive(Debug)]
|
|
@@ -109,9 +151,10 @@ struct List<T> {
|
|
|
tail: Link<Waiter<T>>,
|
|
|
}
|
|
|
|
|
|
-const CLOSED: usize = 1;
|
|
|
-const ONE_QUEUED: usize = 2;
|
|
|
const EMPTY: usize = 0;
|
|
|
+const WAITING: usize = 1;
|
|
|
+const WAKING: usize = 2;
|
|
|
+const CLOSED: usize = 3;
|
|
|
|
|
|
impl<T: Notify + Unpin> WaitQueue<T> {
|
|
|
pub(crate) fn new() -> Self {
|
|
@@ -121,87 +164,186 @@ impl<T: Notify + Unpin> WaitQueue<T> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- pub(crate) fn wait(
|
|
|
- &self,
|
|
|
- waiter: &mut Option<Pin<&mut Waiter<T>>>,
|
|
|
- register: impl FnOnce(&mut Option<T>),
|
|
|
- ) -> WaitResult {
|
|
|
- test_println!("WaitQueue::push_waiter()");
|
|
|
-
|
|
|
- // First, go ahead and check if the queue has been closed. This is
|
|
|
- // necessary even if `waiter` is `None`, as the waiter may already be
|
|
|
- // queued, and just checking if the list was closed.
|
|
|
- // TODO(eliza): that actually kind of sucks lol...
|
|
|
- // Is there at least one queued notification assigned to the wait
|
|
|
- // queue? If so, try to consume that now, rather than waiting.
|
|
|
- match test_dbg!(self
|
|
|
- .state
|
|
|
- .compare_exchange(ONE_QUEUED, EMPTY, AcqRel, Acquire))
|
|
|
- {
|
|
|
+ /// Start waiting for a notification.
|
|
|
+ ///
|
|
|
+ /// If the queue has a stored notification, this consumes it and returns
|
|
|
+ /// [`WaitResult::Notified`] without adding the waiter to the queue. If the
|
|
|
+ /// queue is closed, this returns [`WaitResult::Closed`] without adding the
|
|
|
+ /// waiter to the queue. Otherwise, the waiter is enqueued, and this returns
|
|
|
+ /// [`WaitResult::Wait`].
|
|
|
+ #[inline(always)]
|
|
|
+ pub(crate) fn start_wait(&self, node: Pin<&mut Waiter<T>>, waiter: &T) -> WaitResult {
|
|
|
+ test_println!("WaitQueue::start_wait({:p})", node);
|
|
|
+
|
|
|
+ // Optimistically, acquire a stored notification before trying to lock
|
|
|
+ // the wait list.
|
|
|
+ match test_dbg!(self.state.compare_exchange(WAKING, EMPTY, SeqCst, SeqCst)) {
|
|
|
Ok(_) => return WaitResult::Notified,
|
|
|
Err(CLOSED) => return WaitResult::Closed,
|
|
|
- Err(_state) => debug_assert_eq!(_state, EMPTY),
|
|
|
+ Err(_) => {}
|
|
|
}
|
|
|
|
|
|
- // If we were actually called with a real waiter, try to queue the node.
|
|
|
- if test_dbg!(waiter.is_some()) {
|
|
|
- // There are no queued notifications to consume, and the queue is
|
|
|
- // still open. Therefore, it's time to actually push the waiter to
|
|
|
- // the queue...finally lol :)
|
|
|
+ // Slow path: the queue is not closed, and we failed to consume a stored
|
|
|
+ // notification. We need to acquire the lock and enqueue the waiter.
|
|
|
+ self.start_wait_slow(node, waiter)
|
|
|
+ }
|
|
|
|
|
|
- // Grab the lock...
|
|
|
- let mut list = self.list.lock();
|
|
|
+ /// Slow path of `start_wait`: acquires the linked list lock, and adds the
|
|
|
+ /// waiter to the queue.
|
|
|
+ #[cold]
|
|
|
+ #[inline(never)]
|
|
|
+ fn start_wait_slow(&self, node: Pin<&mut Waiter<T>>, waiter: &T) -> WaitResult {
|
|
|
+ test_println!("WaitQueue::start_wait_slow({:p})", node);
|
|
|
+ // There are no queued notifications to consume, and the queue is
|
|
|
+ // still open. Therefore, it's time to actually push the waiter to
|
|
|
+ // the queue...finally lol :)
|
|
|
+
|
|
|
+ // Grab the lock...
|
|
|
+ let mut list = self.list.lock();
|
|
|
+ // Reload the queue's state, as it may have changed while we were
|
|
|
+ // waiting to lock the linked list.
|
|
|
+ let mut state = self.state.load(Acquire);
|
|
|
+
|
|
|
+ loop {
|
|
|
+ match test_dbg!(state) {
|
|
|
+ // The queue is empty: transition the state to WAITING, as we
|
|
|
+ // are adding a waiter.
|
|
|
+ EMPTY => {
|
|
|
+ match test_dbg!(self
|
|
|
+ .state
|
|
|
+ .compare_exchange_weak(EMPTY, WAITING, SeqCst, SeqCst))
|
|
|
+ {
|
|
|
+ Ok(_) => break,
|
|
|
+ Err(actual) => {
|
|
|
+ debug_assert!(actual == WAKING || actual == CLOSED);
|
|
|
+ state = actual;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- // Okay, we have the lock...but what if someone changed the state
|
|
|
- // WHILE we were waiting to acquire the lock? isn't concurrent
|
|
|
- // programming great? :) :) :) :) :)
|
|
|
- // Try to consume a queued notification *again* in case any were
|
|
|
- // assigned to the queue while we were waiting to acquire the lock.
|
|
|
- match test_dbg!(self
|
|
|
- .state
|
|
|
- .compare_exchange(ONE_QUEUED, EMPTY, AcqRel, Acquire))
|
|
|
- {
|
|
|
- Ok(_) => return WaitResult::Notified,
|
|
|
- Err(CLOSED) => return WaitResult::Closed,
|
|
|
- Err(_state) => debug_assert_eq!(_state, EMPTY),
|
|
|
+ // The queue was woken while we were waiting to acquire the
|
|
|
+ // lock. Attempt to consume the wakeup.
|
|
|
+ WAKING => {
|
|
|
+ match test_dbg!(self
|
|
|
+ .state
|
|
|
+ .compare_exchange_weak(WAKING, EMPTY, SeqCst, SeqCst))
|
|
|
+ {
|
|
|
+ // Consumed the wakeup!
|
|
|
+ Ok(_) => return WaitResult::Notified,
|
|
|
+ Err(actual) => {
|
|
|
+ debug_assert!(actual == EMPTY || actual == CLOSED);
|
|
|
+ state = actual;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // The queue closed while we were waiting to acquire the lock;
|
|
|
+ // we're done here!
|
|
|
+ CLOSED => return WaitResult::Closed,
|
|
|
+
|
|
|
+ // The queue is already in the WAITING state, so we don't need
|
|
|
+ // to mess with it.
|
|
|
+ _state => {
|
|
|
+ debug_assert_eq!(_state, WAITING,
|
|
|
+ "start_wait_slow: unexpected state value {:?} (expected WAITING). this is a bug!",
|
|
|
+ _state,
|
|
|
+ );
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Time to wait! Store the waiter in the node, advance the node's state
|
|
|
+ // to Waiting, and add it to the queue.
|
|
|
+
|
|
|
+ node.with_node(&mut *list, |node| {
|
|
|
+ let _prev = node.waiter.replace(waiter.clone());
|
|
|
+ debug_assert!(
|
|
|
+ _prev.is_none(),
|
|
|
+ "start_wait_slow: called with a node that already had a waiter!"
|
|
|
+ );
|
|
|
+ });
|
|
|
+
|
|
|
+ let _prev_state = test_dbg!(node.state.swap(WAITING, Release));
|
|
|
+ debug_assert!(
|
|
|
+ _prev_state == EMPTY || _prev_state == WAKING,
|
|
|
+ "start_wait_slow: called with a node that was not empty ({}) or woken ({})! actual={}",
|
|
|
+ EMPTY,
|
|
|
+ WAKING,
|
|
|
+ _prev_state,
|
|
|
+ );
|
|
|
+ list.enqueue(node);
|
|
|
+
|
|
|
+ WaitResult::Wait
|
|
|
+ }
|
|
|
+
|
|
|
+ /// Continue waiting for a notification.
|
|
|
+ ///
|
|
|
+ /// This is called when a waiter has been woken. It determines if the
|
|
|
+ /// node was woken from the queue, or if the wakeup was spurious. If the
|
|
|
+ /// wakeup was from the queue, this returns [`WaitResult::Notified`] or
|
|
|
+ /// [`WaitResult::Closed`]. Otherwise, if the wakeup was spurious, this will
|
|
|
+ /// lock the queue and check if the node's waiter needs to be updated.
|
|
|
+ #[inline(always)]
|
|
|
+ pub(crate) fn continue_wait(&self, node: Pin<&mut Waiter<T>>, my_waiter: &T) -> WaitResult {
|
|
|
+ test_println!("WaitQueue::continue_wait({:p})", node);
|
|
|
+
|
|
|
+ // Fast path: check if the node was woken from the queue.
|
|
|
+ let state = test_dbg!(node.state.load(Acquire));
|
|
|
+ match state {
|
|
|
+ WAKING => return WaitResult::Notified,
|
|
|
+ CLOSED => return WaitResult::Closed,
|
|
|
+ _state => {
|
|
|
+ debug_assert_eq!(
|
|
|
+ _state, WAITING,
|
|
|
+ "continue_wait should not be called unless the node has been enqueued"
|
|
|
+ );
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Slow path: received a spurious wakeup. We must lock the queue so that
|
|
|
+ // we can potentially modify the node's waiter.
|
|
|
+ self.continue_wait_slow(node, my_waiter)
|
|
|
+ }
|
|
|
+
|
|
|
+ /// Slow path for `continue_wait`: locks the linked list and updates the
|
|
|
+ /// node with a new waiter.
|
|
|
+ #[cold]
|
|
|
+ #[inline(never)]
|
|
|
+ fn continue_wait_slow(&self, node: Pin<&mut Waiter<T>>, my_waiter: &T) -> WaitResult {
|
|
|
+ test_println!("WaitQueue::continue_wait_slow({:p})", node);
|
|
|
+
|
|
|
+ // If the waiting task/thread was woken but no wakeup was assigned to
|
|
|
+ // the node, we may need to update the node with a new waiter.
|
|
|
+ // Therefore, lock the queue in order to modify the node.
|
|
|
+ let mut list = self.list.lock();
|
|
|
+
|
|
|
+ // The node may have been woken while we were waiting to acquire the
|
|
|
+ // lock. If so, check the new state.
|
|
|
+ match test_dbg!(node.state.load(Acquire)) {
|
|
|
+ WAKING => return WaitResult::Notified,
|
|
|
+ CLOSED => return WaitResult::Closed,
|
|
|
+ _state => {
|
|
|
+ debug_assert_eq!(
|
|
|
+ _state, WAITING,
|
|
|
+ "continue_wait_slow should not be called unless the node has been enqueued"
|
|
|
+ );
|
|
|
}
|
|
|
+ }
|
|
|
|
|
|
- // We didn't consume a queued notification. it is now, finally, time
|
|
|
- // to actually put the waiter in the linked list. wasn't that fun?
|
|
|
-
|
|
|
- if let Some(waiter) = waiter.take() {
|
|
|
- // Now that we have the lock, register the `Waker` or `Thread`
|
|
|
- // to
|
|
|
- let should_queue = unsafe {
|
|
|
- test_println!("WaitQueue::push_waiter -> registering {:p}", waiter);
|
|
|
- // Safety: the waker can only be registered while holding
|
|
|
- // the wait queue lock. We are holding the lock, so no one
|
|
|
- // else will try to touch the waker until we're done.
|
|
|
- waiter.with_node(|node| {
|
|
|
- // Does the node need to be added to the wait queue? If
|
|
|
- // it currently has a waiter (prior to registering),
|
|
|
- // then we know it's already in the queue. Otherwise, if
|
|
|
- // it doesn't have a waiter, it is either waiting for
|
|
|
- // the first time, or it is re-registering after a
|
|
|
- // notification that it wasn't able to consume (for some
|
|
|
- // reason).
|
|
|
- let should_queue = node.waiter.is_none();
|
|
|
- register(&mut node.waiter);
|
|
|
- should_queue
|
|
|
- })
|
|
|
- };
|
|
|
- if test_dbg!(should_queue) {
|
|
|
- test_println!("WaitQueue::push_waiter -> pushing {:p}", waiter);
|
|
|
- test_dbg!(waiter.was_woken_from_queue.swap(false, AcqRel));
|
|
|
- list.push_front(waiter);
|
|
|
+ // Okay, we were not woken and need to continue waiting. It may be
|
|
|
+ // necessary to update the waiter with a new waiter (in practice, this
|
|
|
+ // is only necessary in async).
|
|
|
+ node.with_node(&mut *list, |node| {
|
|
|
+ if let Some(ref mut waiter) = node.waiter {
|
|
|
+ if !waiter.same(my_waiter) {
|
|
|
+ *waiter = my_waiter.clone();
|
|
|
}
|
|
|
} else {
|
|
|
- // XXX(eliza): in practice we can't ever get here because of the
|
|
|
- // `if` above. this should probably be `unreachable_unchecked`
|
|
|
- // but i'm a coward...
|
|
|
- unreachable!("this could be unchecked...")
|
|
|
+ // XXX(eliza): This branch should _probably_ never occur...
|
|
|
+ node.waiter = Some(my_waiter.clone());
|
|
|
}
|
|
|
- }
|
|
|
+ });
|
|
|
|
|
|
WaitResult::Wait
|
|
|
}
|
|
@@ -211,29 +353,82 @@ impl<T: Notify + Unpin> WaitQueue<T> {
|
|
|
///
|
|
|
/// If a waiter was popped from the queue, returns `true`. Otherwise, if the
|
|
|
/// notification was assigned to the queue, returns `false`.
|
|
|
+ #[inline(always)]
|
|
|
pub(crate) fn notify(&self) -> bool {
|
|
|
test_println!("WaitQueue::notify()");
|
|
|
+
|
|
|
+ // Fast path: If the queue is empty, we can simply assign the
|
|
|
+ // notification to the queue.
|
|
|
+ let mut state = self.state.load(Acquire);
|
|
|
+
|
|
|
+ while test_dbg!(state) == WAKING || state == EMPTY {
|
|
|
+ match test_dbg!(self
|
|
|
+ .state
|
|
|
+ .compare_exchange_weak(state, WAKING, SeqCst, SeqCst))
|
|
|
+ {
|
|
|
+ // No waiters are currently waiting, assign the notification to
|
|
|
+ // the queue to be consumed by the next wait attempt.
|
|
|
+ Ok(_) => return false,
|
|
|
+ Err(actual) => state = actual,
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Slow path: there are waiters in the queue, so we must acquire the
|
|
|
+ // lock and wake one of them.
|
|
|
+ self.notify_slow(state)
|
|
|
+ }
|
|
|
+
|
|
|
+ /// Slow path for `notify`: acquire the lock on the linked list, dequeue a
|
|
|
+ /// waiter, and notify it.
|
|
|
+ #[cold]
|
|
|
+ #[inline(never)]
|
|
|
+ fn notify_slow(&self, state: usize) -> bool {
|
|
|
+ test_println!("WaitQueue::notify_slow(state: {})", state);
|
|
|
+
|
|
|
let mut list = self.list.lock();
|
|
|
- if let Some(node) = list.pop_back() {
|
|
|
- drop(list);
|
|
|
- test_println!("notifying {:?}", node);
|
|
|
- node.notify();
|
|
|
- true
|
|
|
- } else {
|
|
|
- test_println!("no waiters to notify...");
|
|
|
- // This can be relaxed because we're holding the lock.
|
|
|
- test_dbg!(self.state.store(ONE_QUEUED, Relaxed));
|
|
|
- false
|
|
|
+ match state {
|
|
|
+ EMPTY | WAKING => {
|
|
|
+ if let Err(actual) = self.state.compare_exchange(state, WAKING, SeqCst, SeqCst) {
|
|
|
+ debug_assert!(actual == EMPTY || actual == WAKING);
|
|
|
+ self.state.store(WAKING, SeqCst);
|
|
|
+ }
|
|
|
+ false
|
|
|
+ }
|
|
|
+ WAITING => {
|
|
|
+ let waiter = list.dequeue(WAKING);
|
|
|
+ debug_assert!(waiter.is_some(), "if we were in the `WAITING` state, there must be a waiter in the queue!\nself={:#?}", self);
|
|
|
+
|
|
|
+ // If we popped the last node, transition back to the empty
|
|
|
+ // state.
|
|
|
+ if test_dbg!(list.is_empty()) {
|
|
|
+ self.state.store(EMPTY, SeqCst);
|
|
|
+ }
|
|
|
+
|
|
|
+ // drop the lock
|
|
|
+ drop(list);
|
|
|
+
|
|
|
+ // wake the waiter
|
|
|
+ if let Some(waiter) = waiter {
|
|
|
+ waiter.notify();
|
|
|
+ true
|
|
|
+ } else {
|
|
|
+ false
|
|
|
+ }
|
|
|
+ }
|
|
|
+ weird => unreachable!("notify_slow: unexpected state value {:?}", weird),
|
|
|
}
|
|
|
}
|
|
|
|
|
|
/// Close the queue, notifying all waiting tasks.
|
|
|
pub(crate) fn close(&self) {
|
|
|
test_println!("WaitQueue::close()");
|
|
|
- test_dbg!(self.state.store(CLOSED, Release));
|
|
|
+
|
|
|
+ test_dbg!(self.state.store(CLOSED, SeqCst));
|
|
|
let mut list = self.list.lock();
|
|
|
- while let Some(node) = list.pop_back() {
|
|
|
- node.notify();
|
|
|
+ while !list.is_empty() {
|
|
|
+ if let Some(waiter) = list.dequeue(CLOSED) {
|
|
|
+ waiter.notify();
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -243,49 +438,76 @@ impl<T: Notify + Unpin> WaitQueue<T> {
|
|
|
impl<T: Notify> Waiter<T> {
|
|
|
pub(crate) fn new() -> Self {
|
|
|
Self {
|
|
|
+ state: CachePadded(AtomicUsize::new(EMPTY)),
|
|
|
node: UnsafeCell::new(Node {
|
|
|
next: None,
|
|
|
prev: None,
|
|
|
waiter: None,
|
|
|
_pin: PhantomPinned,
|
|
|
}),
|
|
|
- was_woken_from_queue: AtomicBool::new(false),
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ #[inline(never)]
|
|
|
+ pub(crate) fn remove(self: Pin<&mut Self>, q: &WaitQueue<T>) {
|
|
|
+ test_println!("Waiter::remove({:p})", self);
|
|
|
+ let mut list = q.list.lock();
|
|
|
+ unsafe {
|
|
|
+ // Safety: removing a node is unsafe even when the list is locked,
|
|
|
+ // because there's no way to guarantee that the node is part of
|
|
|
+ // *this* list. However, the potential callers of this method will
|
|
|
+ // never have access to any other linked lists, so we can just kind
|
|
|
+ // of assume that this is safe.
|
|
|
+ list.remove(self);
|
|
|
+ }
|
|
|
+ if test_dbg!(list.is_empty()) {
|
|
|
+ let _ = test_dbg!(q.state.compare_exchange(WAITING, EMPTY, SeqCst, SeqCst));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ #[inline]
|
|
|
+ pub(crate) fn is_linked(&self) -> bool {
|
|
|
+ test_dbg!(self.state.load(Acquire)) == WAITING
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
impl<T> Waiter<T> {
|
|
|
- unsafe fn with_node<U>(&self, f: impl FnOnce(&mut Node<T>) -> U) -> U {
|
|
|
- self.node.with_mut(|node| f(&mut *node))
|
|
|
+ /// # Safety
|
|
|
+ ///
|
|
|
+ /// This is only safe to call while the list is locked. The dummy `_list`
|
|
|
+ /// parameter ensures this method is only called while holding the lock, so
|
|
|
+ /// this can be safe.
|
|
|
+ #[inline(always)]
|
|
|
+ fn with_node<U>(&self, _list: &mut List<T>, f: impl FnOnce(&mut Node<T>) -> U) -> U {
|
|
|
+ self.node.with_mut(|node| unsafe {
|
|
|
+ // Safety: the dummy `_list` argument ensures that the caller has
|
|
|
+ // the right to mutate the list (e.g. the list is locked).
|
|
|
+ f(&mut *node)
|
|
|
+ })
|
|
|
}
|
|
|
|
|
|
+ /// # Safety
|
|
|
+ ///
|
|
|
+ /// This is only safe to call while the list is locked.
|
|
|
unsafe fn set_prev(&mut self, prev: Option<NonNull<Waiter<T>>>) {
|
|
|
self.node.with_mut(|node| (*node).prev = prev);
|
|
|
}
|
|
|
|
|
|
+ /// # Safety
|
|
|
+ ///
|
|
|
+ /// This is only safe to call while the list is locked.
|
|
|
unsafe fn take_prev(&mut self) -> Option<NonNull<Waiter<T>>> {
|
|
|
self.node.with_mut(|node| (*node).prev.take())
|
|
|
}
|
|
|
|
|
|
+ /// # Safety
|
|
|
+ ///
|
|
|
+ /// This is only safe to call while the list is locked.
|
|
|
unsafe fn take_next(&mut self) -> Option<NonNull<Waiter<T>>> {
|
|
|
self.node.with_mut(|node| (*node).next.take())
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-impl<T: Notify> Waiter<T> {
|
|
|
- pub(crate) fn remove(self: Pin<&mut Self>, q: &WaitQueue<T>) {
|
|
|
- test_println!("Waiter::remove({:p})", self);
|
|
|
- unsafe {
|
|
|
- // Safety: removing a node is unsafe even when the list is locked,
|
|
|
- // because there's no way to guarantee that the node is part of
|
|
|
- // *this* list. However, the potential callers of this method will
|
|
|
- // never have access to any other linked lists, so we can just kind
|
|
|
- // of assume that this is safe.
|
|
|
- q.list.lock().remove(self);
|
|
|
- }
|
|
|
- }
|
|
|
-}
|
|
|
-
|
|
|
unsafe impl<T: Send> Send for Waiter<T> {}
|
|
|
unsafe impl<T: Send> Sync for Waiter<T> {}
|
|
|
|
|
@@ -299,18 +521,24 @@ impl<T> List<T> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- fn push_front(&mut self, waiter: Pin<&mut Waiter<T>>) {
|
|
|
- unsafe {
|
|
|
- waiter.with_node(|node| {
|
|
|
- node.next = self.head;
|
|
|
- node.prev = None;
|
|
|
- })
|
|
|
- }
|
|
|
+ fn enqueue(&mut self, waiter: Pin<&mut Waiter<T>>) {
|
|
|
+ test_println!("List::enqueue({:p})", waiter);
|
|
|
+
|
|
|
+ let node = unsafe { Pin::into_inner_unchecked(waiter) };
|
|
|
+ let ptr = NonNull::from(&*node);
|
|
|
+ debug_assert_ne!(
|
|
|
+ self.head,
|
|
|
+ Some(ptr),
|
|
|
+ "tried to enqueue the same waiter twice!"
|
|
|
+ );
|
|
|
|
|
|
- let ptr = unsafe { NonNull::from(Pin::into_inner_unchecked(waiter)) };
|
|
|
+ let head = self.head.replace(ptr);
|
|
|
+ node.with_node(self, |node| {
|
|
|
+ node.next = head;
|
|
|
+ node.prev = None;
|
|
|
+ });
|
|
|
|
|
|
- debug_assert_ne!(self.head, Some(ptr), "tried to push the same waiter twice!");
|
|
|
- if let Some(mut head) = self.head.replace(ptr) {
|
|
|
+ if let Some(mut head) = head {
|
|
|
unsafe {
|
|
|
head.as_mut().set_prev(Some(ptr));
|
|
|
}
|
|
@@ -321,36 +549,41 @@ impl<T> List<T> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- fn pop_back(&mut self) -> Option<T> {
|
|
|
+ fn dequeue(&mut self, new_state: usize) -> Option<T> {
|
|
|
let mut last = self.tail?;
|
|
|
- test_println!("List::pop_back() -> {:p}", last);
|
|
|
-
|
|
|
- unsafe {
|
|
|
- let last = last.as_mut();
|
|
|
- let prev = last.take_prev();
|
|
|
+ test_println!("List::dequeue({:?}) -> {:p}", new_state, last);
|
|
|
+
|
|
|
+ let last = unsafe { last.as_mut() };
|
|
|
+ let _prev_state = test_dbg!(last.state.swap(new_state, Release));
|
|
|
+ debug_assert_eq!(_prev_state, WAITING);
|
|
|
+
|
|
|
+ let (prev, waiter) = last.with_node(self, |node| {
|
|
|
+ node.next = None;
|
|
|
+ (node.prev.take(), node.waiter.take())
|
|
|
+ });
|
|
|
+
|
|
|
+ match prev {
|
|
|
+ Some(mut prev) => unsafe {
|
|
|
+ let _ = prev.as_mut().take_next();
|
|
|
+ },
|
|
|
+ None => self.head = None,
|
|
|
+ }
|
|
|
|
|
|
- match prev {
|
|
|
- Some(mut prev) => {
|
|
|
- let _ = prev.as_mut().take_next();
|
|
|
- }
|
|
|
- None => self.head = None,
|
|
|
- }
|
|
|
+ self.tail = prev;
|
|
|
|
|
|
- self.tail = prev;
|
|
|
- last.take_next();
|
|
|
- last.was_woken_from_queue.store(true, Relaxed);
|
|
|
- last.with_node(|node| node.waiter.take())
|
|
|
- }
|
|
|
+ waiter
|
|
|
}
|
|
|
|
|
|
unsafe fn remove(&mut self, node: Pin<&mut Waiter<T>>) {
|
|
|
+ test_println!("List::remove({:p})", node);
|
|
|
+
|
|
|
let node_ref = node.get_unchecked_mut();
|
|
|
let prev = node_ref.take_prev();
|
|
|
let next = node_ref.take_next();
|
|
|
let ptr = NonNull::from(node_ref);
|
|
|
|
|
|
if let Some(mut prev) = prev {
|
|
|
- prev.as_mut().with_node(|prev| {
|
|
|
+ prev.as_mut().with_node(self, |prev| {
|
|
|
debug_assert_eq!(prev.next, Some(ptr));
|
|
|
prev.next = next;
|
|
|
});
|
|
@@ -359,7 +592,7 @@ impl<T> List<T> {
|
|
|
}
|
|
|
|
|
|
if let Some(mut next) = next {
|
|
|
- next.as_mut().with_node(|next| {
|
|
|
+ next.as_mut().with_node(self, |next| {
|
|
|
debug_assert_eq!(next.prev, Some(ptr));
|
|
|
next.prev = prev;
|
|
|
});
|
|
@@ -367,6 +600,10 @@ impl<T> List<T> {
|
|
|
self.tail = prev;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ fn is_empty(&self) -> bool {
|
|
|
+ self.head == None && self.tail == None
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
impl<T> fmt::Debug for List<T> {
|
|
@@ -374,6 +611,7 @@ impl<T> fmt::Debug for List<T> {
|
|
|
f.debug_struct("List")
|
|
|
.field("head", &self.head)
|
|
|
.field("tail", &self.tail)
|
|
|
+ .field("is_empty", &self.is_empty())
|
|
|
.finish()
|
|
|
}
|
|
|
}
|