From eb8ac57224b720e4919b070e9d216cfb2dda14a2 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 9 Jul 2024 09:48:15 -0700 Subject: [PATCH 01/41] feat(maitake-sync): optional `lock_api` support Still needs docs, and plumbing through all the async primitives... --- Cargo.lock | 17 + Cargo.toml | 2 +- maitake-sync/Cargo.toml | 1 + maitake-sync/src/blocking.rs | 31 ++ maitake-sync/src/blocking/mutex.rs | 376 ++++++++++++++++++ maitake-sync/src/{spin => blocking}/rwlock.rs | 356 ++++++++++------- maitake-sync/src/lib.rs | 1 + maitake-sync/src/spin.rs | 211 +++++++++- 8 files changed, 842 insertions(+), 153 deletions(-) create mode 100644 maitake-sync/src/blocking.rs create mode 100644 maitake-sync/src/blocking/mutex.rs rename maitake-sync/src/{spin => blocking}/rwlock.rs (67%) diff --git a/Cargo.lock b/Cargo.lock index 7b4cc772..5cda26da 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1389,6 +1389,16 @@ dependencies = [ "json", ] +[[package]] +name = "lock_api" +version = "0.4.12" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" +dependencies = [ + "autocfg", + "scopeguard", +] + [[package]] name = "log" version = "0.4.20" @@ -1440,6 +1450,7 @@ dependencies = [ "cordyceps", "futures", "futures-util", + "lock_api", "loom", "mycelium-bitfield", "pin-project", @@ -2200,6 +2211,12 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" +[[package]] +name = "scopeguard" +version = "1.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" + [[package]] name = "semver" version = "1.0.21" diff --git a/Cargo.toml b/Cargo.toml index 10ef8e5d..f1ca61c1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -104,4 +104,4 @@ tracing-core = { git = "https://github.com/tokio-rs/tracing" } [profile.loom] inherits = "test" lto = true -opt-level = 3 \ No newline at end of file +opt-level = 3 diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index bce67c7b..c11ac8d8 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -29,6 +29,7 @@ core-error = [] [dependencies] mycelium-bitfield = { version = "0.1.3", path = "../bitfield" } cordyceps = { version = "0.3.0", path = "../cordyceps" } +lock_api = { version = "0.4", default-features = false, optional = true } pin-project = "1" portable-atomic = "1.2" tracing = { version = "0.1", default_features = false, optional = true } diff --git a/maitake-sync/src/blocking.rs b/maitake-sync/src/blocking.rs new file mode 100644 index 00000000..33d9a3ee --- /dev/null +++ b/maitake-sync/src/blocking.rs @@ -0,0 +1,31 @@ +//! Synchronous (blocking) synchronization primitives. +//! +//! The synchronization primitives in `maitake-sync` are _asynchronous_. They +//! are designed to be used with [`core::task`] and [`core::future`], and when +//! it is necessary to wait for another task to complete some work for the +//! current task to proceed, `maitake`'s synchronization primitives wait by +//! *yielding* to the asynchronous task scheduler to allow another task to +//! proceed. +//! +//! This module, on the other hand, provides _synchronous_ (or _blocking_) +//! synchronization primitives. Rather than yielding to the runtime, these +//! synchronization primitives will block the current CPU core (or thread, if +//! running in an environment with threads) until they are woken by other cores. +//! This is performed by *spinning*: issuing yield or pause instructions in a +//! loop until some value changes. These synchronization primitives are, in some +//! cases, necessary to implement the async synchronization primitives that form +//! `maitake-sync`'s core APIs. They are also exposed publicly so they can be +//! used in other projects, when a spinlock-based synchronization primitive is +//! needed. +//! +//! This module provides the following APIs: +//! +//! - [`Mutex`]: a synchronous [mutual exclusion] lock. +//! - [`RwLock`]: a synchronous [reader-writer] lock. + +//! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion +//! [reader-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock +mod mutex; +mod rwlock; + +pub use self::{mutex::*, rwlock::*}; diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs new file mode 100644 index 00000000..727de636 --- /dev/null +++ b/maitake-sync/src/blocking/mutex.rs @@ -0,0 +1,376 @@ +use crate::{ + loom::cell::{MutPtr, UnsafeCell}, + spin::Spinlock, + util::fmt, +}; +use core::ops::{Deref, DerefMut}; + +/// A blocking mutual exclusion lock for protecting shared data. +/// Each mutex has a type parameter which represents +/// the data that it is protecting. The data can only be accessed through the +/// RAII guards returned from [`lock`] and [`try_lock`], which guarantees that +/// the data is only ever accessed when the mutex is locked. +/// +/// # Fairness +/// +/// This is *not* a fair mutex. +/// +/// # Loom-specific behavior +/// +/// When `cfg(loom)` is enabled, this mutex will use Loom's simulated atomics, +/// checked `UnsafeCell`, and simulated spin loop hints. +/// +/// [`lock`]: Mutex::lock +/// [`try_lock`]: Mutex::try_lock +pub struct Mutex { + lock: Lock, + data: UnsafeCell, +} + +/// An RAII implementation of a "scoped lock" of a mutex. When this structure is +/// dropped (falls out of scope), the lock will be unlocked. +/// +/// The data protected by the mutex can be accessed through this guard via its +/// [`Deref`] and [`DerefMut`] implementations. +/// +/// This structure is created by the [`lock`] and [`try_lock`] methods on +/// [`Mutex`]. +/// +/// [`lock`]: Mutex::lock +/// [`try_lock`]: Mutex::try_lock +#[must_use = "if unused, the `Mutex` will immediately unlock"] +pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { + ptr: MutPtr, + lock: &'a Lock, +} + +pub unsafe trait RawMutex { + type GuardMarker; + + fn lock(&self); + + fn try_lock(&self) -> bool; + + unsafe fn unlock(&self); + + fn is_locked(&self) -> bool; +} + +#[cfg(feature = "lock_api")] +unsafe impl RawMutex for T { + type GuardMarker = ::GuardMarker; + + #[inline] + #[track_caller] + fn lock(&self) { + lock_api::RawMutex::lock(self); + } + + #[inline] + #[track_caller] + fn try_lock(&self) -> bool { + lock_api::RawMutex::try_lock(self) + } + + #[inline] + #[track_caller] + unsafe fn unlock(&self) { + lock_api::RawMutex::unlock(self); + } + + #[inline] + #[track_caller] + fn is_locked(&self) -> bool { + lock_api::RawMutex::is_locked(self) + } +} + +impl Mutex { + loom_const_fn! { + /// Returns a new `Mutex` protecting the provided `data`. + /// + /// The returned `Mutex` is in an unlocked state, ready for use. + /// + /// # Examples + /// + /// ``` + /// use maitake_sync::spin::Mutex; + /// + /// let mutex = Mutex::new(0); + /// ``` + #[must_use] + pub fn new(data: T) -> Self { + Self { + lock: Spinlock::new(), + data: UnsafeCell::new(data), + } + } + } +} + +#[cfg(feature = "lock_api")] +impl Mutex +where + Lock: lock_api::RawMutex, +{ + #[must_use] + pub const fn with_raw_mutex(data: T) -> Self { + Self { + lock: Lock::INIT, + data: UnsafeCell::new(data), + } + } +} + +impl Mutex +where + Lock: RawMutex, +{ + fn guard<'mutex>(&'mutex self) -> MutexGuard<'mutex, T, Lock> { + MutexGuard { + ptr: self.data.get_mut(), + lock: &self.lock, + } + } + + /// Attempts to acquire this lock without spinning + /// + /// If the lock could not be acquired at this time, then [`None`] is returned. + /// Otherwise, an RAII guard is returned. The lock will be unlocked when the + /// guard is dropped. + /// + /// This function will never spin. + #[must_use] + #[cfg_attr(test, track_caller)] + pub fn try_lock(&self) -> Option> { + if self.lock.try_lock() { + Some(self.guard()) + } else { + None + } + } + + /// Acquires a mutex, spinning until it is locked. + /// + /// This function will spin until the mutex is available to lock. Upon + /// returning, the thread is the only thread with the lock + /// held. An RAII guard is returned to allow scoped unlock of the lock. When + /// the guard goes out of scope, the mutex will be unlocked. + #[cfg_attr(test, track_caller)] + pub fn lock(&self) -> MutexGuard<'_, T, Lock> { + self.lock.lock(); + self.guard() + } + + /// Forcibly unlock the mutex. + /// + /// If a lock is currently held, it will be released, regardless of who's + /// holding it. Of course, this is **outrageously, disgustingly unsafe** and + /// you should never do it. + /// + /// # Safety + /// + /// This deliberately violates mutual exclusion. + /// + /// Only call this method when it is _guaranteed_ that no stack frame that + /// has previously locked the mutex will ever continue executing. + /// Essentially, this is only okay to call when the kernel is oopsing and + /// all code running on other cores has already been killed. + pub unsafe fn force_unlock(&self) { + self.lock.unlock() + } + + /// Consumes this `Mutex`, returning the guarded data. + #[inline] + #[must_use] + pub fn into_inner(self) -> T { + self.data.into_inner() + } + + /// Returns a mutable reference to the underlying data. + /// + /// Since this call borrows the `Mutex` mutably, no actual locking needs to + /// take place -- the mutable borrow statically guarantees no locks exist. + /// + /// # Examples + /// + /// ``` + /// let mut lock = maitake_sync::spin::Mutex::new(0); + /// *lock.get_mut() = 10; + /// assert_eq!(*lock.lock(), 10); + /// ``` + pub fn get_mut(&mut self) -> &mut T { + unsafe { + // Safety: since this call borrows the `Mutex` mutably, no actual + // locking needs to take place -- the mutable borrow statically + // guarantees no locks exist. + self.data.with_mut(|data| &mut *data) + } + } +} + +impl Default for Mutex { + fn default() -> Self { + Self { + lock: Default::default(), + data: Default::default(), + } + } +} + +impl fmt::Debug for Mutex { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Mutex") + .field("data", &fmt::opt(&self.try_lock()).or_else("")) + .finish() + } +} + +unsafe impl Send for Mutex {} +unsafe impl Sync for Mutex {} + +// === impl MutexGuard === + +impl Deref for MutexGuard<'_, T, Lock> { + type Target = T; + #[inline] + fn deref(&self) -> &Self::Target { + unsafe { + // Safety: we are holding the lock, so it is okay to dereference the + // mut pointer. + &*self.ptr.deref() + } + } +} + +impl DerefMut for MutexGuard<'_, T, Lock> { + #[inline] + fn deref_mut(&mut self) -> &mut Self::Target { + unsafe { + // Safety: we are holding the lock, so it is okay to dereference the + // mut pointer. + self.ptr.deref() + } + } +} + +impl AsRef for MutexGuard<'_, T, Lock> +where + T: AsRef, + Lock: RawMutex, +{ + #[inline] + fn as_ref(&self) -> &R { + self.deref().as_ref() + } +} + +impl AsMut for MutexGuard<'_, T, Lock> +where + T: AsMut, + Lock: RawMutex, +{ + #[inline] + fn as_mut(&mut self) -> &mut R { + self.deref_mut().as_mut() + } +} + +impl Drop for MutexGuard<'_, T, Lock> +where + Lock: RawMutex, +{ + #[inline] + #[cfg_attr(test, track_caller)] + fn drop(&mut self) { + unsafe { self.lock.unlock() } + } +} + +impl fmt::Debug for MutexGuard<'_, T, Lock> +where + T: fmt::Display, + Lock: RawMutex, +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.deref().fmt(f) + } +} + +impl fmt::Display for MutexGuard<'_, T, Lock> +where + T: fmt::Display, + Lock: RawMutex, +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.deref().fmt(f) + } +} + +unsafe impl Send for MutexGuard<'_, T, Lock> +where + T: Send, + Lock: RawMutex, + Lock::GuardMarker: Send, +{ +} +unsafe impl Sync for MutexGuard<'_, T, Lock> +where + T: Send, + Lock: RawMutex, + Lock::GuardMarker: Send, +{ +} + +#[cfg(test)] +mod tests { + use crate::loom::{self, thread}; + use std::prelude::v1::*; + use std::sync::Arc; + + use super::*; + + #[test] + fn multithreaded() { + loom::model(|| { + let mutex = Arc::new(Mutex::new(String::new())); + let mutex2 = mutex.clone(); + + let t1 = thread::spawn(move || { + tracing::info!("t1: locking..."); + let mut lock = mutex2.lock(); + tracing::info!("t1: locked"); + lock.push_str("bbbbb"); + tracing::info!("t1: dropping..."); + }); + + { + tracing::info!("t2: locking..."); + let mut lock = mutex.lock(); + tracing::info!("t2: locked"); + lock.push_str("bbbbb"); + tracing::info!("t2: dropping..."); + } + t1.join().unwrap(); + }); + } + + #[test] + fn try_lock() { + loom::model(|| { + let mutex = Mutex::new(42); + // First lock succeeds + let a = mutex.try_lock(); + assert_eq!(a.as_ref().map(|r| **r), Some(42)); + + // Additional lock failes + let b = mutex.try_lock(); + assert!(b.is_none()); + + // After dropping lock, it succeeds again + ::core::mem::drop(a); + let c = mutex.try_lock(); + assert_eq!(c.as_ref().map(|r| **r), Some(42)); + }); + } +} diff --git a/maitake-sync/src/spin/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs similarity index 67% rename from maitake-sync/src/spin/rwlock.rs rename to maitake-sync/src/blocking/rwlock.rs index 41dfb915..7402a866 100644 --- a/maitake-sync/src/spin/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -4,11 +4,9 @@ /// /// [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock use crate::{ - loom::{ - cell::{ConstPtr, MutPtr, UnsafeCell}, - sync::atomic::{AtomicUsize, Ordering::*}, - }, - util::{fmt, Backoff}, + loom::cell::{ConstPtr, MutPtr, UnsafeCell}, + spin, + util::fmt, }; use core::ops::{Deref, DerefMut}; @@ -35,8 +33,8 @@ use core::ops::{Deref, DerefMut}; /// /// [`spin::Mutex`]: crate::spin::Mutex /// [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock -pub struct RwLock { - state: AtomicUsize, +pub struct RwLock { + lock: Lock, data: UnsafeCell, } @@ -52,9 +50,9 @@ pub struct RwLock { /// [`read`]: RwLock::read /// [`try_read`]: RwLock::try_read #[must_use = "if unused, the `RwLock` will immediately unlock"] -pub struct RwLockReadGuard<'lock, T: ?Sized> { +pub struct RwLockReadGuard<'lock, T: ?Sized, Lock: RawRwLock = spin::RwSpinlock> { ptr: ConstPtr, - state: &'lock AtomicUsize, + lock: &'lock Lock, } /// An RAII implementation of a "scoped write lock" of a [`RwLock`]. When this @@ -69,9 +67,82 @@ pub struct RwLockReadGuard<'lock, T: ?Sized> { /// [`write`]: RwLock::write /// [`try_write`]: RwLock::try_write #[must_use = "if unused, the `RwLock` will immediately unlock"] -pub struct RwLockWriteGuard<'lock, T: ?Sized> { +pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawRwLock = spin::RwSpinlock> { ptr: MutPtr, - state: &'lock AtomicUsize, + lock: &'lock Lock, +} + +pub unsafe trait RawRwLock { + type GuardMarker; + + fn lock_shared(&self); + + fn try_lock_shared(&self) -> bool; + + unsafe fn unlock_shared(&self); + + fn lock_exclusive(&self); + + fn try_lock_exclusive(&self) -> bool; + + unsafe fn unlock_exclusive(&self); + + fn is_locked(&self) -> bool; + + fn is_locked_exclusive(&self) -> bool; +} + +#[cfg(feature = "lock_api")] +unsafe impl RawRwLock for T { + type GuardMarker = ::GuardMarker; + + #[inline] + #[track_caller] + fn lock_shared(&self) { + lock_api::RawRwLock::lock_shared(self) + } + + #[inline] + #[track_caller] + fn try_lock_shared(&self) -> bool { + lock_api::RawRwLock::try_lock_shared(self) + } + + #[inline] + #[track_caller] + unsafe fn unlock_shared(&self) { + lock_api::RawRwLock::unlock_shared(self) + } + + #[inline] + #[track_caller] + fn lock_exclusive(&self) { + lock_api::RawRwLock::lock_exclusive(self) + } + + #[inline] + #[track_caller] + fn try_lock_exclusive(&self) -> bool { + lock_api::RawRwLock::try_lock_exclusive(self) + } + + #[inline] + #[track_caller] + unsafe fn unlock_exclusive(&self) { + lock_api::RawRwLock::unlock_exclusive(self) + } + + #[inline] + #[track_caller] + fn is_locked(&self) -> bool { + lock_api::RawRwLock::is_locked(self) + } + + #[inline] + #[track_caller] + fn is_locked_exclusive(&self) -> bool { + lock_api::RawRwLock::is_locked_exclusive(self) + } } const UNLOCKED: usize = 0; @@ -93,14 +164,53 @@ impl RwLock { #[must_use] pub fn new(data: T) -> Self { Self { - state: AtomicUsize::new(0), + lock: todo!(), data: UnsafeCell::new(data), } } } + + /// Returns the current number of readers holding a read lock. + /// + /// # Note + /// + /// This method is not synchronized with attempts to increment the reader + /// count, and its value may become out of date as soon as it is read. This + /// is **not** intended to be used for synchronization purposes! It is + /// intended only for debugging purposes or for use as a heuristic. + #[inline] + #[must_use] + pub fn reader_count(&self) -> usize { + self.lock.reader_count() + } +} + +#[cfg(feature = "lock_api")] +impl RwLock { + /// Creates a new instance of an `RwLock` which is unlocked. + pub const fn with_raw_mutex(data: T) -> Self { + RwLock { + data: UnsafeCell::new(data), + lock: Lock::INIT, + } + } } -impl RwLock { +impl RwLock { + fn read_guard(&self) -> RwLockReadGuard<'_, T, Lock> { + RwLockReadGuard { + ptr: self.data.get(), + lock: &self.lock, + } + } + + fn write_guard(&self) -> RwLockWriteGuard<'_, T, Lock> { + RwLockWriteGuard { + ptr: self.data.get_mut(), + lock: &self.lock, + } + } + /// Locks this `RwLock` for shared read access, spinning until it can be /// acquired. /// @@ -112,14 +222,10 @@ impl RwLock { /// /// Returns an RAII guard which will release this thread's shared access /// once it is dropped. - pub fn read(&self) -> RwLockReadGuard<'_, T> { - let mut backoff = Backoff::new(); - loop { - if let Some(guard) = self.try_read() { - return guard; - } - backoff.spin(); - } + #[cfg_attr(test, track_caller)] + pub fn read(&self) -> RwLockReadGuard<'_, T, Lock> { + self.lock.lock_shared(); + self.read_guard() } /// Attempts to acquire this `RwLock` for shared read access. @@ -130,27 +236,12 @@ impl RwLock { /// /// This function does not spin. #[cfg_attr(test, track_caller)] - pub fn try_read(&self) -> Option> { - // Add a reader. - let state = test_dbg!(self.state.fetch_add(READER, Acquire)); - - // Ensure we don't overflow the reader count and clobber the lock's - // state. - assert!( - state < usize::MAX - (READER * 2), - "read lock counter overflow! this is very bad" - ); - - // Is the write lock held? If so, undo the increment and bail. - if state & WRITER == 1 { - test_dbg!(self.state.fetch_sub(READER, Release)); - return None; + pub fn try_read(&self) -> Option> { + if self.lock.try_lock_shared() { + Some(self.read_guard()) + } else { + None } - - Some(RwLockReadGuard { - ptr: self.data.get(), - state: &self.state, - }) } /// Locks this `RwLock` for exclusive write access, spinning until write @@ -161,43 +252,10 @@ impl RwLock { /// /// Returns an RAII guard which will drop the write access of this `RwLock` /// when dropped. - pub fn write(&self) -> RwLockWriteGuard<'_, T> { - let mut backoff = Backoff::new(); - - // Wait for the lock to become available and set the `WRITER` bit. - // - // Note that, unlike the `read` method, we don't use the `try_write` - // method here, as we would like to use `compare_exchange_weak` to allow - // spurious failures for improved performance. The `try_write` method - // cannot use `compare_exchange_weak`, because it will never retry, and - // a spurious failure means we would incorrectly fail to lock the RwLock - // when we should have successfully locked it. - while test_dbg!(self - .state - .compare_exchange_weak(UNLOCKED, WRITER, Acquire, Relaxed)) - .is_err() - { - test_dbg!(backoff.spin()); - } - - RwLockWriteGuard { - ptr: self.data.get_mut(), - state: &self.state, - } - } - - /// Returns the current number of readers holding a read lock. - /// - /// # Note - /// - /// This method is not synchronized with attempts to increment the reader - /// count, and its value may become out of date as soon as it is read. This - /// is **not** intended to be used for synchronization purposes! It is - /// intended only for debugging purposes or for use as a heuristic. - #[inline] - #[must_use] - pub fn reader_count(&self) -> usize { - self.state.load(Relaxed) >> 1 + #[cfg_attr(test, track_caller)] + pub fn write(&self) -> RwLockWriteGuard<'_, T, Lock> { + self.lock.lock_exclusive(); + self.write_guard() } /// Returns `true` if there is currently a writer holding a write lock. @@ -211,7 +269,7 @@ impl RwLock { #[inline] #[must_use] pub fn has_writer(&self) -> bool { - self.state.load(Relaxed) & WRITER == 1 + self.lock.is_locked_exclusive() } /// Attempts to acquire this `RwLock` for exclusive write access. @@ -221,19 +279,12 @@ impl RwLock { /// RAII guard is returned. The write access is released when it is dropped. /// /// This function does not spin. - pub fn try_write(&self) -> Option> { - if test_dbg!(self - .state - .compare_exchange(UNLOCKED, WRITER, Acquire, Relaxed)) - .is_ok() - { - return Some(RwLockWriteGuard { - ptr: self.data.get_mut(), - state: &self.state, - }); + pub fn try_write(&self) -> Option> { + if self.lock.try_lock_exclusive() { + Some(self.write_guard()) + } else { + None } - - None } /// Returns a mutable reference to the underlying data. @@ -256,9 +307,7 @@ impl RwLock { self.data.with_mut(|data| &mut *data) } } -} -impl RwLock { /// Consumes this `RwLock`, returning the guarded data. #[inline] #[must_use] @@ -267,37 +316,13 @@ impl RwLock { } } -impl fmt::Debug for RwLock { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - let state = &self.state.load(Relaxed); - f.debug_struct("RwLock") - // N.B.: this does *not* use the `reader_count` and `has_writer` - // methods *intentionally*, because those two methods perform - // independent reads of the lock's state, and may race with other - // lock operations that occur concurrently. If, for example, we read - // a non-zero reader count, and then read the state again to check - // for a writer, the reader may have been released and a write lock - // acquired between the two reads, resulting in the `Debug` impl - // displaying an invalid state when the lock was not actually *in* - // such a state! - // - // Therefore, we instead perform a single load to snapshot the state - // and unpack both the reader count and the writer count from the - // lock. - .field("readers", &(state >> 1)) - .field("writer", &(state & WRITER)) - .field( - "data", - &fmt::opt(&self.try_read()).or_else(""), - ) - .finish() - } -} - -impl Default for RwLock { +impl Default for RwLock { /// Creates a new `RwLock`, with the `Default` value for T. fn default() -> RwLock { - RwLock::new(Default::default()) + RwLock { + data: UnsafeCell::new(Default::default()), + lock: Default::default(), + } } } @@ -309,12 +334,12 @@ impl From for RwLock { } } -unsafe impl Send for RwLock {} -unsafe impl Sync for RwLock {} +unsafe impl Send for RwLock {} +unsafe impl Sync for RwLock {} // === impl RwLockReadGuard === -impl Deref for RwLockReadGuard<'_, T> { +impl Deref for RwLockReadGuard<'_, T, Lock> { type Target = T; #[inline] fn deref(&self) -> &Self::Target { @@ -326,9 +351,10 @@ impl Deref for RwLockReadGuard<'_, T> { } } -impl AsRef for RwLockReadGuard<'_, T> +impl AsRef for RwLockReadGuard<'_, T, Lock> where T: AsRef, + Lock: RawRwLock, { #[inline] fn as_ref(&self) -> &R { @@ -336,24 +362,29 @@ where } } -impl Drop for RwLockReadGuard<'_, T> { +impl Drop for RwLockReadGuard<'_, T, Lock> { + #[inline] + #[cfg_attr(test, track_caller)] fn drop(&mut self) { - let _val = test_dbg!(self.state.fetch_sub(READER, Release)); - debug_assert_eq!( - _val & WRITER, - 0, - "tried to drop a read guard while write locked, something is Very Wrong!" - ) + unsafe { self.lock.unlock_shared() } } } -impl fmt::Debug for RwLockReadGuard<'_, T> { +impl fmt::Debug for RwLockReadGuard<'_, T, Lock> +where + T: ?Sized + fmt::Debug, + Lock: RawRwLock, +{ fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.deref().fmt(f) } } -impl fmt::Display for RwLockReadGuard<'_, T> { +impl fmt::Display for RwLockReadGuard<'_, T, Lock> +where + T: ?Sized + fmt::Debug, + Lock: RawRwLock, +{ fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.deref().fmt(f) } @@ -365,12 +396,23 @@ impl fmt::Display for RwLockReadGuard<'_, T> { /// (requiring `T: Sync`), but it *cannot* be used to move ownership of a `T` /// across thread boundaries, as the `T` cannot be taken out of the lock through /// a `RwLockReadGuard`. -unsafe impl Send for RwLockReadGuard<'_, T> {} -unsafe impl Sync for RwLockReadGuard<'_, T> {} +unsafe impl Send for RwLockReadGuard<'_, T, Lock> +where + T: ?Sized + Sync, + Lock: RawRwLock, + Lock::GuardMarker: Send, +{ +} +unsafe impl Sync for RwLockReadGuard<'_, T, Lock> +where + T: ?Sized + Sync, + Lock: RawRwLock, +{ +} // === impl RwLockWriteGuard === -impl Deref for RwLockWriteGuard<'_, T> { +impl Deref for RwLockWriteGuard<'_, T, Lock> { type Target = T; #[inline] fn deref(&self) -> &Self::Target { @@ -382,7 +424,7 @@ impl Deref for RwLockWriteGuard<'_, T> { } } -impl DerefMut for RwLockWriteGuard<'_, T> { +impl DerefMut for RwLockWriteGuard<'_, T, Lock> { #[inline] fn deref_mut(&mut self) -> &mut Self::Target { unsafe { @@ -393,9 +435,10 @@ impl DerefMut for RwLockWriteGuard<'_, T> { } } -impl AsRef for RwLockWriteGuard<'_, T> +impl AsRef for RwLockWriteGuard<'_, T, Lock> where T: AsRef, + Lock: RawRwLock, { #[inline] fn as_ref(&self) -> &R { @@ -403,19 +446,29 @@ where } } -impl Drop for RwLockWriteGuard<'_, T> { +impl Drop for RwLockWriteGuard<'_, T, Lock> { + #[inline] + #[cfg_attr(test, track_caller)] fn drop(&mut self) { - let _val = test_dbg!(self.state.swap(UNLOCKED, Release)); + unsafe { self.lock.unlock_exclusive() } } } -impl fmt::Debug for RwLockWriteGuard<'_, T> { +impl fmt::Debug for RwLockWriteGuard<'_, T, Lock> +where + T: ?Sized + fmt::Debug, + Lock: RawRwLock, +{ fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.deref().fmt(f) } } -impl fmt::Display for RwLockWriteGuard<'_, T> { +impl fmt::Display for RwLockWriteGuard<'_, T, Lock> +where + T: ?Sized + fmt::Display, + Lock: RawRwLock, +{ fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.deref().fmt(f) } @@ -425,13 +478,24 @@ impl fmt::Display for RwLockWriteGuard<'_, T> { /// because it can be used to *move* a `T` across thread boundaries, as it /// allows mutable access to the `T` that can be used with /// [`core::mem::replace`] or [`core::mem::swap`]. -unsafe impl Send for RwLockWriteGuard<'_, T> {} +unsafe impl Send for RwLockWriteGuard<'_, T, Lock> +where + T: ?Sized + Send + Sync, + Lock: RawRwLock, + Lock::GuardMarker: Send, +{ +} /// A [`RwLockWriteGuard`] is only [`Sync`] if `T` is [`Send`] and [`Sync`], /// because it can be used to *move* a `T` across thread boundaries, as it /// allows mutable access to the `T` that can be used with /// [`core::mem::replace`] or [`core::mem::swap`]. -unsafe impl Sync for RwLockWriteGuard<'_, T> {} +unsafe impl Sync for RwLockWriteGuard<'_, T, Lock> +where + T: ?Sized + Send + Sync, + Lock: RawRwLock, +{ +} #[cfg(test)] mod tests { diff --git a/maitake-sync/src/lib.rs b/maitake-sync/src/lib.rs index 5eb6fea8..c8b341d6 100644 --- a/maitake-sync/src/lib.rs +++ b/maitake-sync/src/lib.rs @@ -13,6 +13,7 @@ pub(crate) mod loom; #[macro_use] pub mod util; +pub mod blocking; pub mod mutex; pub mod rwlock; pub mod semaphore; diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index dda7276d..5203a329 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -30,12 +30,211 @@ //! //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion //! [reader-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock -mod mutex; pub mod once; -mod rwlock; -pub use self::{ - mutex::*, - once::{InitOnce, Lazy}, - rwlock::*, +pub use self::once::{InitOnce, Lazy}; +pub use crate::blocking::*; +use crate::{ + loom::sync::atomic::{AtomicBool, AtomicUsize, Ordering::*}, + util::{fmt, Backoff}, }; + +/// A spinlock-based [`RawMutex`] implementation. +/// +/// This mutex will spin with an exponential backoff while waiting for the lock +/// to become available. +#[derive(Debug)] +pub struct Spinlock { + locked: AtomicBool, +} + +/// A spinlock-based [`RawRwLock`] implementation. +pub struct RwSpinlock { + state: AtomicUsize, +} + +// === impl RawSpinlock === + +impl Spinlock { + loom_const_fn! { + pub(crate) fn new() -> Self { + Self { locked: AtomicBool::new(false) } + } + } +} + +impl Default for Spinlock { + fn default() -> Self { + Self::new() + } +} + +unsafe impl RawMutex for Spinlock { + type GuardMarker = (); + + #[cfg_attr(test, track_caller)] + fn lock(&self) { + let mut boff = Backoff::default(); + while test_dbg!(self + .locked + .compare_exchange(false, true, Acquire, Acquire) + .is_err()) + { + while test_dbg!(self.is_locked()) { + boff.spin(); + } + } + } + + #[cfg_attr(test, track_caller)] + #[inline] + fn try_lock(&self) -> bool { + test_dbg!(self + .locked + .compare_exchange(false, true, Acquire, Acquire) + .is_ok()) + } + + #[cfg_attr(test, track_caller)] + #[inline] + unsafe fn unlock(&self) { + test_dbg!(self.locked.store(false, Release)); + } + + #[inline] + fn is_locked(&self) -> bool { + self.locked.load(Relaxed) + } +} + +const UNLOCKED: usize = 0; +const WRITER: usize = 1 << 0; +const READER: usize = 1 << 1; + +impl RwSpinlock { + loom_const_fn! { + pub(crate) fn new() -> Self { + Self { + state: AtomicUsize::new(UNLOCKED), + } + } + } + + pub(crate) fn reader_count(&self) -> usize { + self.state.load(Relaxed) >> 1 + } +} + +unsafe impl RawRwLock for RwSpinlock { + type GuardMarker = (); + + #[cfg_attr(test, track_caller)] + fn lock_shared(&self) { + let mut boff = Backoff::new(); + while !self.try_lock_shared() { + boff.spin(); + } + } + + #[cfg_attr(test, track_caller)] + fn try_lock_shared(&self) -> bool { + // Add a reader. + let state = test_dbg!(self.state.fetch_add(READER, Acquire)); + + // Ensure we don't overflow the reader count and clobber the lock's + // state. + assert!( + state < usize::MAX - (READER * 2), + "read lock counter overflow! this is very bad" + ); + + // Is the write lock held? If so, undo the increment and bail. + if state & WRITER == 1 { + test_dbg!(self.state.fetch_sub(READER, Release)); + false + } else { + true + } + } + + #[cfg_attr(test, track_caller)] + #[inline] + unsafe fn unlock_shared(&self) { + let _val = test_dbg!(self.state.fetch_sub(READER, Release)); + debug_assert_eq!( + _val & WRITER, + 0, + "tried to drop a read guard while write locked, something is Very Wrong!" + ) + } + + #[cfg_attr(test, track_caller)] + fn lock_exclusive(&self) { + let mut backoff = Backoff::new(); + + // Wait for the lock to become available and set the `WRITER` bit. + // + // Note that, unlike the `lock_shared` method, we don't use the + // `try_lock_exclusive` method here, as we would like to use + // `compare_exchange_weak` to allow spurious failures for improved + // performance. The `try_lock_exclusive` method cannot use + // `compare_exchange_weak`, because it will never retry, and + // a spurious failure means we would incorrectly fail to lock the RwLock + // when we should have successfully locked it. + while test_dbg!(self + .state + .compare_exchange_weak(UNLOCKED, WRITER, Acquire, Relaxed)) + .is_err() + { + test_dbg!(backoff.spin()); + } + } + + #[cfg_attr(test, track_caller)] + #[inline] + fn try_lock_exclusive(&self) -> bool { + test_dbg!(self + .state + .compare_exchange(UNLOCKED, WRITER, Acquire, Relaxed)) + .is_ok() + } + + #[cfg_attr(test, track_caller)] + #[inline] + unsafe fn unlock_exclusive(&self) { + let _val = test_dbg!(self.state.swap(UNLOCKED, Release)); + } + + #[inline] + fn is_locked(&self) -> bool { + self.state.load(Relaxed) & (WRITER | READER) != 0 + } + + #[inline] + fn is_locked_exclusive(&self) -> bool { + self.state.load(Relaxed) & WRITER == 1 + } +} + +impl fmt::Debug for RwSpinlock { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + let state = &self.state.load(Relaxed); + f.debug_struct("RawSpinRwLock") + // N.B.: this does *not* use the `reader_count` and `has_writer` + // methods *intentionally*, because those two methods perform + // independent reads of the lock's state, and may race with other + // lock operations that occur concurrently. If, for example, we read + // a non-zero reader count, and then read the state again to check + // for a writer, the reader may have been released and a write lock + // acquired between the two reads, resulting in the `Debug` impl + // displaying an invalid state when the lock was not actually *in* + // such a state! + // + // Therefore, we instead perform a single load to snapshot the state + // and unpack both the reader count and the writer count from the + // lock. + .field("readers", &(state >> 1)) + .field("writer", &(state & WRITER)) + .finish() + } +} From 4ca4d63d4a29a9a888c9f46073a3f7267621a8a0 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Tue, 9 Jul 2024 11:41:48 -0700 Subject: [PATCH 02/41] feat(maitake-sync): update WaitQueue to use RawMutex --- maitake-sync/src/blocking/mutex.rs | 19 +- maitake-sync/src/blocking/rwlock.rs | 36 +++- maitake-sync/src/loom.rs | 60 +++++- maitake-sync/src/spin/mutex.rs | 300 ---------------------------- maitake-sync/src/util/fmt.rs | 2 +- maitake-sync/src/wait_queue.rs | 91 +++++---- 6 files changed, 146 insertions(+), 362 deletions(-) delete mode 100644 maitake-sync/src/spin/mutex.rs diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 727de636..daefde19 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -126,7 +126,7 @@ impl Mutex where Lock: RawMutex, { - fn guard<'mutex>(&'mutex self) -> MutexGuard<'mutex, T, Lock> { + fn guard(&self) -> MutexGuard<'_, T, Lock> { MutexGuard { ptr: self.data.get_mut(), lock: &self.lock, @@ -213,25 +213,30 @@ impl Default for Mutex { fn default() -> Self { Self { lock: Default::default(), - data: Default::default(), + data: UnsafeCell::new(Default::default()), } } } -impl fmt::Debug for Mutex { +impl fmt::Debug for Mutex +where + T: fmt::Debug, + Lock: fmt::Debug + RawMutex, +{ fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("Mutex") .field("data", &fmt::opt(&self.try_lock()).or_else("")) + .field("lock", &self.lock) .finish() } } -unsafe impl Send for Mutex {} -unsafe impl Sync for Mutex {} +unsafe impl Send for Mutex {} +unsafe impl Sync for Mutex {} // === impl MutexGuard === -impl Deref for MutexGuard<'_, T, Lock> { +impl Deref for MutexGuard<'_, T, Lock> { type Target = T; #[inline] fn deref(&self) -> &Self::Target { @@ -289,7 +294,7 @@ where impl fmt::Debug for MutexGuard<'_, T, Lock> where - T: fmt::Display, + T: fmt::Debug, Lock: RawMutex, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { diff --git a/maitake-sync/src/blocking/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs index 7402a866..4ea34cfe 100644 --- a/maitake-sync/src/blocking/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -5,7 +5,7 @@ /// [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock use crate::{ loom::cell::{ConstPtr, MutPtr, UnsafeCell}, - spin, + spin::RwSpinlock, util::fmt, }; use core::ops::{Deref, DerefMut}; @@ -33,7 +33,7 @@ use core::ops::{Deref, DerefMut}; /// /// [`spin::Mutex`]: crate::spin::Mutex /// [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock -pub struct RwLock { +pub struct RwLock { lock: Lock, data: UnsafeCell, } @@ -50,7 +50,7 @@ pub struct RwLock { /// [`read`]: RwLock::read /// [`try_read`]: RwLock::try_read #[must_use = "if unused, the `RwLock` will immediately unlock"] -pub struct RwLockReadGuard<'lock, T: ?Sized, Lock: RawRwLock = spin::RwSpinlock> { +pub struct RwLockReadGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { ptr: ConstPtr, lock: &'lock Lock, } @@ -67,7 +67,7 @@ pub struct RwLockReadGuard<'lock, T: ?Sized, Lock: RawRwLock = spin::RwSpinlock> /// [`write`]: RwLock::write /// [`try_write`]: RwLock::try_write #[must_use = "if unused, the `RwLock` will immediately unlock"] -pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawRwLock = spin::RwSpinlock> { +pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { ptr: MutPtr, lock: &'lock Lock, } @@ -145,10 +145,6 @@ unsafe impl RawRwLock for T { } } -const UNLOCKED: usize = 0; -const WRITER: usize = 1 << 0; -const READER: usize = 1 << 1; - impl RwLock { loom_const_fn! { /// Creates a new, unlocked `RwLock` protecting the provided `data`. @@ -164,7 +160,7 @@ impl RwLock { #[must_use] pub fn new(data: T) -> Self { Self { - lock: todo!(), + lock: RwSpinlock::new(), data: UnsafeCell::new(data), } } @@ -307,7 +303,9 @@ impl RwLock { self.data.with_mut(|data| &mut *data) } } +} +impl RwLock { /// Consumes this `RwLock`, returning the guarded data. #[inline] #[must_use] @@ -318,7 +316,7 @@ impl RwLock { impl Default for RwLock { /// Creates a new `RwLock`, with the `Default` value for T. - fn default() -> RwLock { + fn default() -> RwLock { RwLock { data: UnsafeCell::new(Default::default()), lock: Default::default(), @@ -334,6 +332,22 @@ impl From for RwLock { } } +impl fmt::Debug for RwLock +where + T: fmt::Debug, + Lock: fmt::Debug + RawRwLock, +{ + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("RwLock") + .field( + "data", + &fmt::opt(&self.try_read()).or_else(""), + ) + .field("lock", &self.lock) + .finish() + } +} + unsafe impl Send for RwLock {} unsafe impl Sync for RwLock {} @@ -382,7 +396,7 @@ where impl fmt::Display for RwLockReadGuard<'_, T, Lock> where - T: ?Sized + fmt::Debug, + T: ?Sized + fmt::Display, Lock: RawRwLock, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index fa37206b..c4e9e739 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -73,28 +73,70 @@ mod inner { pub(crate) use loom::sync::*; pub(crate) mod spin { - pub(crate) use loom::sync::MutexGuard; + use core::{ + marker::PhantomData, + ops::{Deref, DerefMut}, + }; + + use alloc::fmt; /// Mock version of mycelium's spinlock, but using /// `loom::sync::Mutex`. The API is slightly different, since the /// mycelium mutex does not support poisoning. - #[derive(Debug)] - pub(crate) struct Mutex(loom::sync::Mutex); + pub(crate) struct Mutex( + loom::sync::Mutex, + PhantomData, + ); - impl Mutex { + pub(crate) struct MutexGuard<'a, T, Lock = crate::spin::Spinlock>( + loom::sync::MutexGuard<'a, T>, + PhantomData, + ); + + impl Mutex { #[track_caller] pub(crate) fn new(t: T) -> Self { - Self(loom::sync::Mutex::new(t)) + Self(loom::sync::Mutex::new(t), PhantomData) } #[track_caller] - pub fn try_lock(&self) -> Option> { - self.0.try_lock().ok() + pub fn try_lock(&self) -> Option> { + self.0.try_lock().map(|x| MutexGuard(x, PhantomData)).ok() } #[track_caller] - pub fn lock(&self) -> MutexGuard<'_, T> { - self.0.lock().expect("loom mutex will never poison") + pub fn lock(&self) -> MutexGuard<'_, T, Lock> { + self.0 + .lock() + .map(|x| MutexGuard(x, PhantomData)) + .expect("loom mutex will never poison") + } + } + + impl fmt::Debug for Mutex { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.0.fmt(f) + } + } + + impl Deref for MutexGuard<'_, T, Lock> { + type Target = T; + #[inline] + fn deref(&self) -> &Self::Target { + self.0.deref() + } + } + + impl DerefMut for MutexGuard<'_, T, Lock> { + #[inline] + fn deref_mut(&mut self) -> &mut Self::Target { + self.0.deref_mut() + } + } + + impl fmt::Debug for MutexGuard<'_, T, Lock> { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + self.0.fmt(f) } } } diff --git a/maitake-sync/src/spin/mutex.rs b/maitake-sync/src/spin/mutex.rs deleted file mode 100644 index e5ada4c3..00000000 --- a/maitake-sync/src/spin/mutex.rs +++ /dev/null @@ -1,300 +0,0 @@ -use crate::{ - loom::{ - cell::{MutPtr, UnsafeCell}, - sync::atomic::{AtomicBool, Ordering::*}, - }, - util::{fmt, Backoff}, -}; -use core::ops::{Deref, DerefMut}; - -/// A spinlock-based mutual exclusion lock for protecting shared data -/// -/// This mutex will spin with an exponential backoff while waiting for the lock -/// to become available. Each mutex has a type parameter which represents -/// the data that it is protecting. The data can only be accessed through the -/// RAII guards returned from [`lock`] and [`try_lock`], which guarantees that -/// the data is only ever accessed when the mutex is locked. -/// -/// # Fairness -/// -/// This is *not* a fair mutex. -/// -/// # Loom-specific behavior -/// -/// When `cfg(loom)` is enabled, this mutex will use Loom's simulated atomics, -/// checked `UnsafeCell`, and simulated spin loop hints. -/// -/// [`lock`]: Mutex::lock -/// [`try_lock`]: Mutex::try_lock -pub struct Mutex { - locked: AtomicBool, - data: UnsafeCell, -} - -/// An RAII implementation of a "scoped lock" of a mutex. When this structure is -/// dropped (falls out of scope), the lock will be unlocked. -/// -/// The data protected by the mutex can be accessed through this guard via its -/// [`Deref`] and [`DerefMut`] implementations. -/// -/// This structure is created by the [`lock`] and [`try_lock`] methods on -/// [`Mutex`]. -/// -/// [`lock`]: Mutex::lock -/// [`try_lock`]: Mutex::try_lock -#[must_use = "if unused, the `Mutex` will immediately unlock"] -pub struct MutexGuard<'a, T> { - ptr: MutPtr, - locked: &'a AtomicBool, -} - -impl Mutex { - loom_const_fn! { - /// Returns a new `Mutex` protecting the provided `data`. - /// - /// The returned `Mutex` is in an unlocked state, ready for use. - /// - /// # Examples - /// - /// ``` - /// use maitake_sync::spin::Mutex; - /// - /// let mutex = Mutex::new(0); - /// ``` - #[must_use] - pub fn new(data: T) -> Self { - Self { - locked: AtomicBool::new(false), - data: UnsafeCell::new(data), - } - } - } - - /// Attempts to acquire this lock without spinning - /// - /// If the lock could not be acquired at this time, then [`None`] is returned. - /// Otherwise, an RAII guard is returned. The lock will be unlocked when the - /// guard is dropped. - /// - /// This function will never spin. - #[must_use] - #[cfg_attr(test, track_caller)] - pub fn try_lock(&self) -> Option> { - if test_dbg!(self - .locked - .compare_exchange(false, true, Acquire, Acquire) - .is_ok()) - { - Some(MutexGuard { - ptr: self.data.get_mut(), - locked: &self.locked, - }) - } else { - None - } - } - - /// Acquires a mutex, spinning until it is locked. - /// - /// This function will spin until the mutex is available to lock. Upon - /// returning, the thread is the only thread with the lock - /// held. An RAII guard is returned to allow scoped unlock of the lock. When - /// the guard goes out of scope, the mutex will be unlocked. - #[cfg_attr(test, track_caller)] - pub fn lock(&self) -> MutexGuard<'_, T> { - let mut boff = Backoff::default(); - while test_dbg!(self - .locked - .compare_exchange(false, true, Acquire, Acquire) - .is_err()) - { - while test_dbg!(self.locked.load(Relaxed)) { - boff.spin(); - } - } - - MutexGuard { - ptr: self.data.get_mut(), - locked: &self.locked, - } - } - - /// Forcibly unlock the mutex. - /// - /// If a lock is currently held, it will be released, regardless of who's - /// holding it. Of course, this is **outrageously, disgustingly unsafe** and - /// you should never do it. - /// - /// # Safety - /// - /// This deliberately violates mutual exclusion. - /// - /// Only call this method when it is _guaranteed_ that no stack frame that - /// has previously locked the mutex will ever continue executing. - /// Essentially, this is only okay to call when the kernel is oopsing and - /// all code running on other cores has already been killed. - pub unsafe fn force_unlock(&self) { - self.locked.store(false, Release); - } - - /// Consumes this `Mutex`, returning the guarded data. - #[inline] - #[must_use] - pub fn into_inner(self) -> T { - self.data.into_inner() - } - - /// Returns a mutable reference to the underlying data. - /// - /// Since this call borrows the `Mutex` mutably, no actual locking needs to - /// take place -- the mutable borrow statically guarantees no locks exist. - /// - /// # Examples - /// - /// ``` - /// let mut lock = maitake_sync::spin::Mutex::new(0); - /// *lock.get_mut() = 10; - /// assert_eq!(*lock.lock(), 10); - /// ``` - pub fn get_mut(&mut self) -> &mut T { - unsafe { - // Safety: since this call borrows the `Mutex` mutably, no actual - // locking needs to take place -- the mutable borrow statically - // guarantees no locks exist. - self.data.with_mut(|data| &mut *data) - } - } -} - -impl Default for Mutex { - fn default() -> Self { - Self::new(Default::default()) - } -} - -impl fmt::Debug for Mutex { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Mutex") - .field("data", &fmt::opt(&self.try_lock()).or_else("")) - .finish() - } -} - -unsafe impl Send for Mutex {} -unsafe impl Sync for Mutex {} - -// === impl MutexGuard === - -impl<'a, T> Deref for MutexGuard<'a, T> { - type Target = T; - #[inline] - fn deref(&self) -> &Self::Target { - unsafe { - // Safety: we are holding the lock, so it is okay to dereference the - // mut pointer. - &*self.ptr.deref() - } - } -} - -impl<'a, T> DerefMut for MutexGuard<'a, T> { - #[inline] - fn deref_mut(&mut self) -> &mut Self::Target { - unsafe { - // Safety: we are holding the lock, so it is okay to dereference the - // mut pointer. - self.ptr.deref() - } - } -} - -impl<'a, T, R: ?Sized> AsRef for MutexGuard<'a, T> -where - T: AsRef, -{ - #[inline] - fn as_ref(&self) -> &R { - self.deref().as_ref() - } -} - -impl<'a, T, R: ?Sized> AsMut for MutexGuard<'a, T> -where - T: AsMut, -{ - #[inline] - fn as_mut(&mut self) -> &mut R { - self.deref_mut().as_mut() - } -} - -impl<'a, T> Drop for MutexGuard<'a, T> { - fn drop(&mut self) { - test_dbg!(self.locked.store(false, Release)); - } -} - -impl<'a, T: fmt::Debug> fmt::Debug for MutexGuard<'a, T> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.deref().fmt(f) - } -} - -impl<'a, T: fmt::Display> fmt::Display for MutexGuard<'a, T> { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.deref().fmt(f) - } -} - -#[cfg(test)] -mod tests { - use crate::loom::{self, thread}; - use std::prelude::v1::*; - use std::sync::Arc; - - use super::*; - - #[test] - fn multithreaded() { - loom::model(|| { - let mutex = Arc::new(Mutex::new(String::new())); - let mutex2 = mutex.clone(); - - let t1 = thread::spawn(move || { - tracing::info!("t1: locking..."); - let mut lock = mutex2.lock(); - tracing::info!("t1: locked"); - lock.push_str("bbbbb"); - tracing::info!("t1: dropping..."); - }); - - { - tracing::info!("t2: locking..."); - let mut lock = mutex.lock(); - tracing::info!("t2: locked"); - lock.push_str("bbbbb"); - tracing::info!("t2: dropping..."); - } - t1.join().unwrap(); - }); - } - - #[test] - fn try_lock() { - loom::model(|| { - let mutex = Mutex::new(42); - // First lock succeeds - let a = mutex.try_lock(); - assert_eq!(a.as_ref().map(|r| **r), Some(42)); - - // Additional lock failes - let b = mutex.try_lock(); - assert!(b.is_none()); - - // After dropping lock, it succeeds again - ::core::mem::drop(a); - let c = mutex.try_lock(); - assert_eq!(c.as_ref().map(|r| **r), Some(42)); - }); - } -} diff --git a/maitake-sync/src/util/fmt.rs b/maitake-sync/src/util/fmt.rs index e42da133..59a7bffb 100644 --- a/maitake-sync/src/util/fmt.rs +++ b/maitake-sync/src/util/fmt.rs @@ -71,7 +71,7 @@ impl Debug for FmtOption<'_, T> { #[inline] fn fmt(&self, f: &mut Formatter<'_>) -> Result { match self.opt { - Some(val) => val.fmt(f), + Some(val) => Debug::fmt(val, f), None => f.write_str(self.or_else), } } diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index 6298c48b..fb021e42 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -5,6 +5,7 @@ #[cfg(any(test, maitake_ultraverbose))] use crate::util::fmt; use crate::{ + blocking::RawMutex, loom::{ cell::UnsafeCell, sync::{ @@ -12,6 +13,7 @@ use crate::{ spin::{Mutex, MutexGuard}, }, }, + spin::Spinlock, util::{CachePadded, WakeBatch}, WaitResult, }; @@ -175,7 +177,7 @@ mod tests; /// [mutex]: crate::Mutex /// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue #[derive(Debug)] -pub struct WaitQueue { +pub struct WaitQueue { /// The wait queue's state variable. state: CachePadded, @@ -191,11 +193,10 @@ pub struct WaitQueue { /// lock must be held when modifying the /// node. /// - /// A spinlock (from `mycelium_util`) is used here, in order to support - /// `no_std` platforms; when running `loom` tests, a `loom` mutex is used - /// instead to simulate the spinlock, because loom doesn't play nice with - /// real spinlocks. - queue: Mutex>, + /// A spinlock is used here, in order to support `no_std` platforms; when + /// running `loom` tests, a `loom` mutex is used instead to simulate the + /// spinlock, because loom doesn't play nice with real spinlocks. + queue: Mutex, Lock>, } /// Future returned from [`WaitQueue::wait()`]. @@ -220,9 +221,9 @@ pub struct WaitQueue { #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Wait<'a> { +pub struct Wait<'a, Lock: RawMutex = Spinlock> { /// The [`WaitQueue`] being waited on. - queue: &'a WaitQueue, + queue: &'a WaitQueue, /// Entry in the wait queue linked list. #[pin] @@ -359,7 +360,7 @@ impl WaitQueue { /// Returns a new `WaitQueue`. #[must_use] pub fn new() -> Self { - Self::new_with_state(State::Empty) + Self::make(State::Empty, Mutex::new(List::new())) } } @@ -372,16 +373,32 @@ impl WaitQueue { // TODO(eliza): should this be a public API? #[must_use] pub(crate) fn new_woken() -> Self { - Self::new_with_state(State::Woken) + Self::make(State::Woken, Mutex::new(List::new())) } } +} + +#[cfg(all(feature = "lock_api", not(loom)))] +impl WaitQueue +where + Lock: lock_api::RawMutex, +{ + #[must_use] + pub const fn with_raw_mutex() -> Self { + Self::make(State::Empty, Mutex::with_raw_mutex(List::new())) + } +} +impl WaitQueue +where + Lock: RawMutex, +{ loom_const_fn! { #[must_use] - fn new_with_state(state: State) -> Self { + fn make(state: State, queue: Mutex, Lock>) -> Self { Self { state: CachePadded::new(AtomicUsize::new(state.into_usize())), - queue: Mutex::new(List::new()), + queue, } } } @@ -659,7 +676,7 @@ impl WaitQueue { /// [`wake()`]: Self::wake /// [`wake_all()`]: Self::wake_all /// [`Closed`]: crate::Closed - pub fn wait(&self) -> Wait<'_> { + pub fn wait(&self) -> Wait<'_, Lock> { Wait { queue: self, waiter: self.waiter(), @@ -977,9 +994,9 @@ impl WaitQueue { fn drain_to_wake_batch<'q>( &'q self, batch: &mut WakeBatch, - mut queue: MutexGuard<'q, List>, + mut queue: MutexGuard<'q, List, Lock>, wakeup: Wakeup, - ) -> MutexGuard<'q, List> { + ) -> MutexGuard<'q, List, Lock> { while let Some(node) = queue.pop_back() { let Some(waker) = Waiter::wake(node, &mut queue, wakeup.clone()) else { // this waiter was enqueued by `Wait::register` and doesn't have @@ -1058,11 +1075,14 @@ impl Waiter { } } - fn poll_wait( + fn poll_wait( mut self: Pin<&mut Self>, - queue: &WaitQueue, + queue: &WaitQueue, waker: Option<&Waker>, - ) -> Poll> { + ) -> Poll> + where + Lock: RawMutex, + { test_debug!(ptr = ?fmt::ptr(self.as_mut()), "Waiter::poll_wait"); let mut this = self.as_mut().project(); @@ -1188,10 +1208,13 @@ impl Waiter { /// /// This is called from the `drop` implementation for the [`Wait`] and /// [`WaitOwned`] futures. - fn release(mut self: Pin<&mut Self>, queue: &WaitQueue) { + fn release(mut self: Pin<&mut Self>, queue: &WaitQueue) + where + Lock: RawMutex, + { let state = *(self.as_mut().project().state); let ptr = NonNull::from(unsafe { Pin::into_inner_unchecked(self) }); - test_debug!(self = ?fmt::ptr(ptr), ?state, ?queue, "Waiter::release"); + test_debug!(self = ?fmt::ptr(ptr), ?state, ?queue.state, "Waiter::release"); // if we're not enqueued, we don't have to do anything else. if state.get(WaitStateBits::STATE) != WaitState::Waiting { @@ -1251,7 +1274,7 @@ unsafe impl Linked> for Waiter { // === impl Wait === -impl Wait<'_> { +impl Wait<'_, Lock> { /// Returns `true` if this `Wait` future is waiting for a notification from /// the provided [`WaitQueue`]. /// @@ -1269,7 +1292,7 @@ impl Wait<'_> { /// ``` #[inline] #[must_use] - pub fn waits_on(&self, queue: &WaitQueue) -> bool { + pub fn waits_on(&self, queue: &WaitQueue) -> bool { ptr::eq(self.queue, queue) } @@ -1306,7 +1329,7 @@ impl Wait<'_> { /// ``` #[inline] #[must_use] - pub fn same_queue(&self, other: &Wait<'_>) -> bool { + pub fn same_queue(&self, other: &Wait<'_, Lock>) -> bool { ptr::eq(self.queue, other.queue) } @@ -1363,7 +1386,7 @@ impl Wait<'_> { } } -impl Future for Wait<'_> { +impl Future for Wait<'_, Lock> { type Output = WaitResult<()>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1373,7 +1396,7 @@ impl Future for Wait<'_> { } #[pinned_drop] -impl PinnedDrop for Wait<'_> { +impl PinnedDrop for Wait<'_, Lock> { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(this.queue); @@ -1449,16 +1472,16 @@ feature! { /// ``` #[derive(Debug)] #[pin_project(PinnedDrop)] - pub struct WaitOwned { + pub struct WaitOwned { /// The `WaitQueue` being waited on. - queue: Arc, + queue: Arc>, /// Entry in the wait queue. #[pin] waiter: Waiter, } - impl WaitQueue { + impl WaitQueue { /// Wait to be woken up by this queue, returning a future that's valid /// for the `'static` lifetime. /// @@ -1493,7 +1516,7 @@ feature! { /// [`wait()`]: Self::wait /// [closed]: Self::close /// [`Closed`]: crate::Closed - pub fn wait_owned(self: &Arc) -> WaitOwned { + pub fn wait_owned(self: &Arc) -> WaitOwned { let waiter = self.waiter(); let queue = self.clone(); WaitOwned { queue, waiter } @@ -1502,7 +1525,7 @@ feature! { // === impl WaitOwned === - impl WaitOwned { + impl WaitOwned { /// Returns `true` if this `WaitOwned` future is waiting for a /// notification from the provided [`WaitQueue`]. /// @@ -1521,7 +1544,7 @@ feature! { /// ``` #[inline] #[must_use] - pub fn waits_on(&self, queue: &WaitQueue) -> bool { + pub fn waits_on(&self, queue: &WaitQueue) -> bool { ptr::eq(&*self.queue, queue) } @@ -1562,7 +1585,7 @@ feature! { /// ``` #[inline] #[must_use] - pub fn same_queue(&self, other: &WaitOwned) -> bool { + pub fn same_queue(&self, other: &WaitOwned) -> bool { Arc::ptr_eq(&self.queue, &other.queue) } @@ -1621,7 +1644,7 @@ feature! { } } - impl Future for WaitOwned { + impl Future for WaitOwned { type Output = WaitResult<()>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1631,7 +1654,7 @@ feature! { } #[pinned_drop] - impl PinnedDrop for WaitOwned { + impl PinnedDrop for WaitOwned { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(&*this.queue); From 13cd5aa075c619039d8b5b2527a59d54529ca158 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 10 Jul 2024 10:55:38 -0700 Subject: [PATCH 03/41] feat(maitake-sync): make `Semaphore` generic over RawMutex --- maitake-sync/src/rwlock.rs | 2 +- maitake-sync/src/rwlock/owned.rs | 4 +- maitake-sync/src/semaphore.rs | 111 ++++++++++++++++++++----------- 3 files changed, 75 insertions(+), 42 deletions(-) diff --git a/maitake-sync/src/rwlock.rs b/maitake-sync/src/rwlock.rs index dc192b1b..41cd7bd4 100644 --- a/maitake-sync/src/rwlock.rs +++ b/maitake-sync/src/rwlock.rs @@ -202,7 +202,7 @@ impl RwLock { } impl RwLock { - const MAX_READERS: usize = Semaphore::MAX_PERMITS; + const MAX_READERS: usize = semaphore::MAX_PERMITS; /// Locks this `RwLock` with shared read access, causing the current task /// to yield until the lock has been acquired. diff --git a/maitake-sync/src/rwlock/owned.rs b/maitake-sync/src/rwlock/owned.rs index c30cac34..9f801ec7 100644 --- a/maitake-sync/src/rwlock/owned.rs +++ b/maitake-sync/src/rwlock/owned.rs @@ -1,5 +1,5 @@ use super::*; -use crate::Semaphore; +use crate::semaphore; use alloc::sync::Arc; /// Owned [RAII] structure used to release the shared read access of a @@ -83,7 +83,7 @@ pub struct OwnedRwLockWriteGuard { /// `Drop` impl, but Loom considers us to be "accessing" it as long as the /// `MutPtr` exists. data: cell::MutPtr, - _lock: AddPermits<{ Semaphore::MAX_PERMITS }, T>, + _lock: AddPermits<{ semaphore::MAX_PERMITS }, T>, } /// A wrapper around an `RwLock` `Arc` clone that releases a fixed number of diff --git a/maitake-sync/src/semaphore.rs b/maitake-sync/src/semaphore.rs index 7f3f1ee7..59d8a996 100644 --- a/maitake-sync/src/semaphore.rs +++ b/maitake-sync/src/semaphore.rs @@ -5,6 +5,7 @@ //! //! [counting semaphore]: https://en.wikipedia.org/wiki/Semaphore_(programming) use crate::{ + blocking::RawMutex, loom::{ cell::UnsafeCell, sync::{ @@ -12,6 +13,7 @@ use crate::{ spin::{Mutex, MutexGuard}, }, }, + spin::Spinlock, util::{fmt, CachePadded, WakeBatch}, WaitResult, }; @@ -167,7 +169,7 @@ mod tests; /// [counting semaphore]: https://en.wikipedia.org/wiki/Semaphore_(programming) /// [`acquire`]: Semaphore::acquire #[derive(Debug)] -pub struct Semaphore { +pub struct Semaphore { /// The number of permits in the semaphore (or [`usize::MAX] if the /// semaphore is closed. permits: CachePadded, @@ -178,7 +180,7 @@ pub struct Semaphore { /// `no_std` platforms; when running `loom` tests, a `loom` mutex is used /// instead to simulate the spinlock, because loom doesn't play nice with /// real spinlocks. - waiters: Mutex, + waiters: Mutex, } /// A [RAII guard] representing one or more permits acquired from a @@ -193,9 +195,9 @@ pub struct Semaphore { /// [RAII guard]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[derive(Debug)] #[must_use = "dropping a `Permit` releases the acquired permits back to the `Semaphore`"] -pub struct Permit<'sem> { +pub struct Permit<'sem, Lock: RawMutex = Spinlock> { permits: usize, - semaphore: &'sem Semaphore, + semaphore: &'sem Semaphore, } /// The future returned by the [`Semaphore::acquire`] method. @@ -217,8 +219,8 @@ pub struct Permit<'sem> { #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Acquire<'sem> { - semaphore: &'sem Semaphore, +pub struct Acquire<'sem, Lock: RawMutex = Spinlock> { + semaphore: &'sem Semaphore, queued: bool, permits: usize, #[pin] @@ -293,11 +295,6 @@ struct Node { // === impl Semaphore === impl Semaphore { - /// The maximum number of permits a `Semaphore` may contain. - pub const MAX_PERMITS: usize = usize::MAX - 1; - - const CLOSED: usize = usize::MAX; - loom_const_fn! { /// Returns a new `Semaphore` with `permits` permits available. /// @@ -308,20 +305,45 @@ impl Semaphore { /// [`MAX_PERMITS`]: Self::MAX_PERMITS #[must_use] pub fn new(permits: usize) -> Self { + Self::make( + permits, + Mutex::new(SemQueue::new()) + ) + } + } +} + +#[cfg(feature = "lock_api")] +impl Semaphore { + #[must_use] + pub const fn with_raw_mutex(permits: usize) -> Self { + Self::make(permits, Mutex::with_raw_mutex(SemQueue::new())) + } +} + +// This is factored out as a free constant in this module so that `RwLock` can +// depend on it without having to specify `Semaphore`'s type parameters. This is +// a little annoying but whatever. +pub(crate) const MAX_PERMITS: usize = usize::MAX - 1; + +impl Semaphore { + /// The maximum number of permits a `Semaphore` may contain. + pub const MAX_PERMITS: usize = MAX_PERMITS; + + const CLOSED: usize = usize::MAX; + + loom_const_fn! { + fn make(permits: usize, waiters: Mutex) -> Self { assert!( permits <= Self::MAX_PERMITS, "a semaphore may not have more than Semaphore::MAX_PERMITS permits", - ); + ); Self { permits: CachePadded::new(AtomicUsize::new(permits)), - waiters: Mutex::new(SemQueue { - queue: List::new(), - closed: false, - }), + waiters, } } } - /// Returns the number of permits currently available in this semaphore, or /// 0 if the semaphore is [closed]. /// @@ -352,7 +374,7 @@ impl Semaphore { /// /// [`Closed`]: crate::Closed /// [closed]: Semaphore::close - pub fn acquire(&self, permits: usize) -> Acquire<'_> { + pub fn acquire(&self, permits: usize) -> Acquire<'_, Lock> { Acquire { semaphore: self, queued: false, @@ -398,7 +420,7 @@ impl Semaphore { /// /// [`Closed`]: crate::Closed /// [closed]: Semaphore::close - pub fn try_acquire(&self, permits: usize) -> Result, TryAcquireError> { + pub fn try_acquire(&self, permits: usize) -> Result, TryAcquireError> { trace!(permits, "Semaphore::try_acquire"); self.try_acquire_inner(permits).map(|_| Permit { permits, @@ -595,7 +617,7 @@ impl Semaphore { fn add_permits_locked<'sem>( &'sem self, mut permits: usize, - mut waiters: MutexGuard<'sem, SemQueue>, + mut waiters: MutexGuard<'sem, SemQueue, Lock>, ) { trace!(permits, "Semaphore::add_permits"); if waiters.closed { @@ -734,11 +756,22 @@ impl Semaphore { } } } +// === impl SemQueue === + +impl SemQueue { + #[must_use] + const fn new() -> Self { + Self { + queue: List::new(), + closed: false, + } + } +} // === impl Acquire === -impl<'sem> Future for Acquire<'sem> { - type Output = WaitResult>; +impl<'sem, Lock: RawMutex> Future for Acquire<'sem, Lock> { + type Output = WaitResult>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.project(); let poll = this @@ -754,7 +787,7 @@ impl<'sem> Future for Acquire<'sem> { } #[pinned_drop] -impl PinnedDrop for Acquire<'_> { +impl PinnedDrop for Acquire<'_, Lock> { fn drop(self: Pin<&mut Self>) { let this = self.project(); trace!(?this.queued, "Acquire::drop"); @@ -768,11 +801,11 @@ impl PinnedDrop for Acquire<'_> { // the `Acquire` future will only access this `UnsafeCell` when mutably borrowed // (when polling or dropping the future), so the future itself is safe to share // immutably between threads. -unsafe impl Sync for Acquire<'_> {} +unsafe impl Sync for Acquire<'_, Lock> {} // === impl Permit === -impl Permit<'_> { +impl Permit<'_, Lock> { /// Forget this permit, dropping it *without* returning the number of /// acquired permits to the semaphore. /// @@ -790,7 +823,7 @@ impl Permit<'_> { } } -impl Drop for Permit<'_> { +impl Drop for Permit<'_, Lock> { fn drop(&mut self) { trace!(?self.permits, "Permit::drop"); self.semaphore.add_permits(self.permits); @@ -844,8 +877,8 @@ feature! { #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] - pub struct AcquireOwned { - semaphore: Arc, + pub struct AcquireOwned { + semaphore: Arc>, queued: bool, permits: usize, #[pin] @@ -868,12 +901,12 @@ feature! { /// [RAII guard]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[derive(Debug)] #[must_use = "dropping an `OwnedPermit` releases the acquired permits back to the `Semaphore`"] - pub struct OwnedPermit { + pub struct OwnedPermit { permits: usize, - semaphore: Arc, + semaphore: Arc>, } - impl Semaphore { + impl Semaphore { /// Acquire `permits` permits from the `Semaphore`, waiting asynchronously /// if there are insufficient permits currently available, and returning /// an [`OwnedPermit`]. @@ -899,7 +932,7 @@ feature! { /// [`acquire`]: Semaphore::acquire /// [`Closed`]: crate::Closed /// [closed]: Semaphore::close - pub fn acquire_owned(self: &Arc, permits: usize) -> AcquireOwned { + pub fn acquire_owned(self: &Arc, permits: usize) -> AcquireOwned { AcquireOwned { semaphore: self.clone(), queued: false, @@ -929,7 +962,7 @@ feature! { /// [`try_acquire`]: Semaphore::try_acquire /// [`Closed`]: crate::Closed /// [closed]: Semaphore::close - pub fn try_acquire_owned(self: &Arc, permits: usize) -> Result { + pub fn try_acquire_owned(self: &Arc, permits: usize) -> Result, TryAcquireError> { trace!(permits, "Semaphore::try_acquire_owned"); self.try_acquire_inner(permits).map(|_| OwnedPermit { permits, @@ -940,8 +973,8 @@ feature! { // === impl AcquireOwned === - impl Future for AcquireOwned { - type Output = WaitResult; + impl Future for AcquireOwned { + type Output = WaitResult>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { let this = self.project(); @@ -960,7 +993,7 @@ feature! { } #[pinned_drop] - impl PinnedDrop for AcquireOwned { + impl PinnedDrop for AcquireOwned { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); trace!(?this.queued, "AcquireOwned::drop"); @@ -971,11 +1004,11 @@ feature! { // safety: this is safe for the same reasons as the `Sync` impl for the // `Acquire` future. - unsafe impl Sync for AcquireOwned {} + unsafe impl Sync for AcquireOwned {} // === impl OwnedPermit === - impl OwnedPermit { + impl OwnedPermit { /// Forget this permit, dropping it *without* returning the number of /// acquired permits to the semaphore. /// @@ -993,7 +1026,7 @@ feature! { } } - impl Drop for OwnedPermit { + impl Drop for OwnedPermit { fn drop(&mut self) { trace!(?self.permits, "OwnedPermit::drop"); self.semaphore.add_permits(self.permits); From d86b684c824578f3ca422d147fc0db2ac1b59651 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Wed, 10 Jul 2024 12:42:39 -0700 Subject: [PATCH 04/41] feat(maitake-sync): make `RwLock` generic over RawMutex --- maitake-sync/src/blocking/rwlock.rs | 4 +- maitake-sync/src/rwlock.rs | 79 +++++++++++++++++++++-------- 2 files changed, 61 insertions(+), 22 deletions(-) diff --git a/maitake-sync/src/blocking/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs index 4ea34cfe..f3416922 100644 --- a/maitake-sync/src/blocking/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -348,8 +348,8 @@ where } } -unsafe impl Send for RwLock {} -unsafe impl Sync for RwLock {} +unsafe impl Send for RwLock {} +unsafe impl Sync for RwLock {} // === impl RwLockReadGuard === diff --git a/maitake-sync/src/rwlock.rs b/maitake-sync/src/rwlock.rs index 41cd7bd4..f2d38011 100644 --- a/maitake-sync/src/rwlock.rs +++ b/maitake-sync/src/rwlock.rs @@ -5,7 +5,9 @@ //! [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock use super::semaphore::{self, Semaphore}; use crate::{ + blocking::RawMutex, loom::cell::{self, UnsafeCell}, + spin::Spinlock, util::fmt, }; use core::ops::{Deref, DerefMut}; @@ -89,12 +91,12 @@ mod tests; /// [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock /// [_write-preferring_]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock#Priority_policies /// [`std::sync::RwLock`]: https://doc.rust-lang.org/stable/std/sync/struct.RwLock.html -pub struct RwLock { +pub struct RwLock { /// The semaphore used to control access to `data`. /// /// To read `data`, a single permit must be acquired. To write to `data`, /// all the permits in the semaphore must be acquired. - sem: Semaphore, + sem: Semaphore, /// The data protected by the lock. data: UnsafeCell, @@ -116,7 +118,7 @@ pub struct RwLock { /// [`read`]: RwLock::read /// [`try_read`]: RwLock::try_read #[must_use = "if unused, the `RwLock` will immediately unlock"] -pub struct RwLockReadGuard<'lock, T: ?Sized> { +pub struct RwLockReadGuard<'lock, T: ?Sized, Lock: RawMutex = Spinlock> { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important for Loom tests; the `ConstPtr` @@ -124,7 +126,7 @@ pub struct RwLockReadGuard<'lock, T: ?Sized> { /// another task that wants to access the cell, and Loom will still consider the data to /// be "accessed" until the `ConstPtr` is dropped. data: cell::ConstPtr, - _permit: semaphore::Permit<'lock>, + _permit: semaphore::Permit<'lock, Lock>, } /// [RAII] structure used to release the exclusive write access of a [`RwLock`] when @@ -143,7 +145,7 @@ pub struct RwLockReadGuard<'lock, T: ?Sized> { /// [`write`]: RwLock::write /// [`try_write`]: RwLock::try_write #[must_use = "if unused, the `RwLock` will immediately unlock"] -pub struct RwLockWriteGuard<'lock, T: ?Sized> { +pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawMutex = Spinlock> { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important for Loom tests; the `MutPtr` @@ -151,7 +153,7 @@ pub struct RwLockWriteGuard<'lock, T: ?Sized> { /// another task that wants to access the cell, and Loom will still consider /// the data to be "accessed mutably" until the `MutPtr` is dropped. data: cell::MutPtr, - _permit: semaphore::Permit<'lock>, + _permit: semaphore::Permit<'lock, Lock>, } feature! { @@ -192,7 +194,20 @@ impl RwLock { } } } +} + +#[cfg(feature = "lock_api")] +impl RwLock { + #[must_use] + pub const fn with_raw_mutex(data: T) -> Self { + Self { + sem: Semaphore::with_raw_mutex(Self::MAX_READERS), + data: UnsafeCell::new(data), + } + } +} +impl RwLock { /// Consumes this `RwLock`, returning the guarded data. #[inline] #[must_use] @@ -201,7 +216,7 @@ impl RwLock { } } -impl RwLock { +impl RwLock { const MAX_READERS: usize = semaphore::MAX_PERMITS; /// Locks this `RwLock` with shared read access, causing the current task @@ -265,7 +280,7 @@ impl RwLock { /// /// [priority policy]: Self#priority-policy /// [an RAII guard]: - pub async fn read(&self) -> RwLockReadGuard<'_, T> { + pub async fn read(&self) -> RwLockReadGuard<'_, T, Lock> { let _permit = self .sem .acquire(1) @@ -317,7 +332,7 @@ impl RwLock { /// # } /// # test(); /// ``` - pub async fn write(&self) -> RwLockWriteGuard<'_, T> { + pub async fn write(&self) -> RwLockWriteGuard<'_, T, Lock> { let _permit = self .sem .acquire(Self::MAX_READERS) @@ -356,7 +371,7 @@ impl RwLock { /// ``` /// /// [an RAII guard]: RwLockReadGuard - pub fn try_read(&self) -> Option> { + pub fn try_read(&self) -> Option> { match self.sem.try_acquire(1) { Ok(_permit) => Some(RwLockReadGuard { data: self.data.get(), @@ -396,7 +411,7 @@ impl RwLock { /// ``` /// /// [an RAII guard]: RwLockWriteGuard - pub fn try_write(&self) -> Option> { + pub fn try_write(&self) -> Option> { match self.sem.try_acquire(Self::MAX_READERS) { Ok(_permit) => Some(RwLockWriteGuard { data: self.data.get_mut(), @@ -433,11 +448,15 @@ impl RwLock { impl Default for RwLock { fn default() -> Self { - Self::new(Default::default()) + Self::new(T::default()) } } -impl fmt::Debug for RwLock { +impl fmt::Debug for RwLock +where + T: ?Sized + fmt::Debug, + Lock: RawMutex + fmt::Debug, +{ fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let Self { sem, data: _ } = self; f.debug_struct("RwLock") @@ -450,12 +469,22 @@ impl fmt::Debug for RwLock { // Safety: if `T` is `Send + Sync`, an `RwLock` can safely be sent or shared // between threads. If `T` wasn't `Send`, this would be unsafe, since the // `RwLock` exposes access to the `T`. -unsafe impl Send for RwLock where T: ?Sized + Send {} -unsafe impl Sync for RwLock where T: ?Sized + Send + Sync {} +unsafe impl Send for RwLock +where + T: ?Sized + Send, + Lock: RawMutex + Send, +{ +} +unsafe impl Sync for RwLock +where + T: ?Sized + Send + Sync, + Lock: RawMutex + Sync, +{ +} // === impl RwLockReadGuard === -impl Deref for RwLockReadGuard<'_, T> { +impl Deref for RwLockReadGuard<'_, T, Lock> { type Target = T; #[inline] @@ -468,7 +497,7 @@ impl Deref for RwLockReadGuard<'_, T> { } } -impl fmt::Debug for RwLockReadGuard<'_, T> { +impl fmt::Debug for RwLockReadGuard<'_, T, Lock> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.deref().fmt(f) } @@ -477,12 +506,22 @@ impl fmt::Debug for RwLockReadGuard<'_, T> { // Safety: A read guard can be shared or sent between threads as long as `T` is // `Sync`. It can implement `Send` even if `T` does not implement `Send`, as // long as `T` is `Sync`, because the read guard only permits borrowing the `T`. -unsafe impl Send for RwLockReadGuard<'_, T> where T: ?Sized + Sync {} -unsafe impl Sync for RwLockReadGuard<'_, T> where T: ?Sized + Send + Sync {} +unsafe impl Send for RwLockReadGuard<'_, T, Lock> +where + T: ?Sized + Sync, + Lock: RawMutex + Sync, +{ +} +unsafe impl Sync for RwLockReadGuard<'_, T, Lock> +where + T: ?Sized + Send + Sync, + Lock: RawMutex + Sync, +{ +} // === impl RwLockWriteGuard === -impl Deref for RwLockWriteGuard<'_, T> { +impl Deref for RwLockWriteGuard<'_, T, Lock> { type Target = T; #[inline] From 6e215d770e97c4b7e0c7d23bfb456ed8900fcd6f Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 11 Jul 2024 09:48:11 -0700 Subject: [PATCH 05/41] feat(maitake-sync): lock api in async mutex --- maitake-sync/src/blocking/mutex.rs | 2 + maitake-sync/src/mutex.rs | 135 +++++++++++++++++++++-------- 2 files changed, 103 insertions(+), 34 deletions(-) diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index daefde19..d2ff308e 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -44,6 +44,8 @@ pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { lock: &'a Lock, } +/// Trait abstracting over blocking [`Mutex`] implementations (`maitake-sync`'s +/// version). pub unsafe trait RawMutex { type GuardMarker; diff --git a/maitake-sync/src/mutex.rs b/maitake-sync/src/mutex.rs index dd94c9f1..722a9a9d 100644 --- a/maitake-sync/src/mutex.rs +++ b/maitake-sync/src/mutex.rs @@ -4,7 +4,9 @@ //! //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion use crate::{ + blocking::RawMutex, loom::cell::{MutPtr, UnsafeCell}, + spin::Spinlock, util::fmt, wait_queue::{self, WaitQueue}, }; @@ -84,8 +86,8 @@ mod tests; /// [storage]: https://mycelium.elizas.website/maitake/task/trait.Storage.html /// [no-unwinding]: https://mycelium.elizas.website/maitake/index.html#maitake-does-not-support-unwinding -pub struct Mutex { - wait: WaitQueue, +pub struct Mutex { + wait: WaitQueue, data: UnsafeCell, } @@ -105,12 +107,12 @@ pub struct Mutex { /// [`try_lock`]: Mutex::try_lock /// [RAII]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[must_use = "if unused, the `Mutex` will immediately unlock"] -pub struct MutexGuard<'a, T: ?Sized> { +pub struct MutexGuard<'a, T: ?Sized, L: RawMutex = Spinlock> { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important. data: MutPtr, - _wake: WakeOnDrop<'a, T>, + _wake: WakeOnDrop<'a, T, L>, } /// A [future] returned by the [`Mutex::lock`] method. @@ -134,15 +136,15 @@ pub struct MutexGuard<'a, T: ?Sized> { #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] #[pin_project] #[derive(Debug)] -pub struct Lock<'a, T: ?Sized> { +pub struct Lock<'a, T: ?Sized, L: RawMutex = Spinlock> { #[pin] - wait: wait_queue::Wait<'a>, - mutex: &'a Mutex, + wait: wait_queue::Wait<'a, L>, + mutex: &'a Mutex, } /// This is used in order to ensure that the wakeup is performed only *after* /// the data ptr is dropped, in order to keep `loom` happy. -struct WakeOnDrop<'a, T: ?Sized>(&'a Mutex); +struct WakeOnDrop<'a, T: ?Sized, L: RawMutex>(&'a Mutex); // === impl Mutex === @@ -178,7 +180,9 @@ impl Mutex { } } } +} +impl Mutex { /// Consumes this `Mutex`, returning the guarded data. #[inline] #[must_use] @@ -187,7 +191,7 @@ impl Mutex { } } -impl Mutex { +impl Mutex { /// Locks this mutex. /// /// This returns a [`Lock`] future that will wait until no other task is @@ -207,7 +211,7 @@ impl Mutex { /// *guard = 2; /// } /// ``` - pub fn lock(&self) -> Lock<'_, T> { + pub fn lock(&self) -> Lock<'_, T, L> { Lock { wait: self.wait.wait(), mutex: self, @@ -236,7 +240,7 @@ impl Mutex { /// # Some(()) /// # } /// ``` - pub fn try_lock(&self) -> Option> { + pub fn try_lock(&self) -> Option> { match self.wait.try_wait() { Poll::Pending => None, Poll::Ready(Ok(_)) => Some(unsafe { @@ -275,7 +279,7 @@ impl Mutex { /// # Safety /// /// This may only be called once a lock has been acquired. - unsafe fn guard(&self) -> MutexGuard<'_, T> { + unsafe fn guard(&self) -> MutexGuard<'_, T, L> { MutexGuard { _wake: WakeOnDrop(self), data: self.data.get_mut(), @@ -289,7 +293,11 @@ impl Default for Mutex { } } -impl fmt::Debug for Mutex { +impl fmt::Debug for Mutex +where + T: ?Sized + fmt::Debug, + L: RawMutex + fmt::Debug, +{ fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let Self { data: _, wait } = self; f.debug_struct("Mutex") @@ -299,8 +307,18 @@ impl fmt::Debug for Mutex { } } -unsafe impl Send for Mutex where T: Send {} -unsafe impl Sync for Mutex where T: Send {} +unsafe impl Send for Mutex +where + T: ?Sized + Send, + L: Send, +{ +} +unsafe impl Sync for Mutex +where + T: ?Sized + Send, + L: Sync, +{ +} // === impl Lock === @@ -328,7 +346,11 @@ impl<'a, T> Future for Lock<'a, T> { // === impl MutexGuard === -impl<'a, T: ?Sized> Deref for MutexGuard<'a, T> { +impl Deref for MutexGuard<'_, T, L> +where + T: ?Sized, + L: RawMutex, +{ type Target = T; #[inline] @@ -340,7 +362,11 @@ impl<'a, T: ?Sized> Deref for MutexGuard<'a, T> { } } -impl DerefMut for MutexGuard<'_, T> { +impl DerefMut for MutexGuard<'_, T, L> +where + T: ?Sized, + L: RawMutex, +{ #[inline] fn deref_mut(&mut self) -> &mut Self::Target { unsafe { @@ -350,17 +376,32 @@ impl DerefMut for MutexGuard<'_, T> { } } -impl fmt::Debug for MutexGuard<'_, T> { +impl fmt::Debug for MutexGuard<'_, T, L> +where + T: ?Sized + fmt::Debug, + L: RawMutex, +{ #[inline] fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.deref().fmt(f) } } -unsafe impl Send for MutexGuard<'_, T> where T: Send {} -unsafe impl Sync for MutexGuard<'_, T> where T: Send + Sync {} +unsafe impl Send for MutexGuard<'_, T, L> +where + T: ?Sized + Send, + L: RawMutex + Sync, +{ +} +unsafe impl Sync for MutexGuard<'_, T, L> +where + T: ?Sized + Send + Sync, + // A `MutexGuard`` has a reference to a `L`-typed RawMutex in it, so `` + L: RawMutex + Sync, +{ +} -impl<'a, T: ?Sized> Drop for WakeOnDrop<'a, T> { +impl Drop for WakeOnDrop<'_, T, L> { fn drop(&mut self) { self.0.wait.wake() } @@ -393,15 +434,15 @@ feature! { /// [`try_lock_owned`]: Mutex::try_lock_owned /// [RAII]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[must_use = "if unused, the Mutex will immediately unlock"] - pub struct OwnedMutexGuard { + pub struct OwnedMutexGuard { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important. data: MutPtr, - _wake: WakeArcOnDrop, + _wake: WakeArcOnDrop, } - impl Mutex { + impl Mutex { /// Locks this mutex, returning an [owned RAII guard][`OwnedMutexGuard`]. /// @@ -436,7 +477,7 @@ feature! { /// } /// # } /// ``` - pub async fn lock_owned(self: Arc) -> OwnedMutexGuard { + pub async fn lock_owned(self: Arc) -> OwnedMutexGuard { self.wait.wait().await.unwrap(); unsafe { // safety: we have just acquired the lock @@ -482,7 +523,7 @@ feature! { /// } /// # } /// ``` - pub fn try_lock_owned(self: Arc) -> Result, Arc> { + pub fn try_lock_owned(self: Arc) -> Result, Arc> { match self.wait.try_wait() { Poll::Pending => Err(self), Poll::Ready(Ok(_)) => Ok(unsafe { @@ -500,7 +541,7 @@ feature! { /// # Safety /// /// This may only be called once a lock has been acquired. - unsafe fn owned_guard(self: Arc) -> OwnedMutexGuard { + unsafe fn owned_guard(self: Arc) -> OwnedMutexGuard { let data = self.data.get_mut(); OwnedMutexGuard { _wake: WakeArcOnDrop(self), @@ -509,11 +550,15 @@ feature! { } } - struct WakeArcOnDrop(Arc>); + struct WakeArcOnDrop(Arc>); // === impl OwnedMutexGuard === - impl Deref for OwnedMutexGuard { + impl Deref for OwnedMutexGuard + where + T: ?Sized, + L: RawMutex, + { type Target = T; #[inline] @@ -525,7 +570,11 @@ feature! { } } - impl DerefMut for OwnedMutexGuard { + impl DerefMut for OwnedMutexGuard + where + T: ?Sized, + L: RawMutex, + { #[inline] fn deref_mut(&mut self) -> &mut Self::Target { unsafe { @@ -535,17 +584,35 @@ feature! { } } - impl fmt::Debug for OwnedMutexGuard { + impl fmt::Debug for OwnedMutexGuard + where + T: ?Sized + fmt::Debug, + L: RawMutex, + { #[inline] fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.deref().fmt(f) } } - unsafe impl Send for OwnedMutexGuard where T: Send {} - unsafe impl Sync for OwnedMutexGuard where T: Send + Sync {} + unsafe impl Send for OwnedMutexGuard + where + T: ?Sized + Send, + L: RawMutex + Sync, + { + } + unsafe impl Sync for OwnedMutexGuard + where + T: ?Sized + Send + Sync, + L: RawMutex + Sync, + { + } - impl Drop for WakeArcOnDrop { + impl Drop for WakeArcOnDrop + where + T: ?Sized, + L: RawMutex, + { fn drop(&mut self) { self.0.wait.wake() } From 5af051edc1246a8e7fe3d49b332d5ca844aa8dae Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 15 Jul 2024 10:18:28 -0700 Subject: [PATCH 06/41] docs --- maitake-sync/README.md | 20 ++++++++++++- maitake-sync/src/blocking.rs | 44 +++++++++++++++++++---------- maitake-sync/src/blocking/mutex.rs | 44 +++++++++++++++++++++++++++++ maitake-sync/src/blocking/rwlock.rs | 41 +++++++++++++++++++++++++++ 4 files changed, 133 insertions(+), 16 deletions(-) diff --git a/maitake-sync/README.md b/maitake-sync/README.md index f552793f..9ead4496 100644 --- a/maitake-sync/README.md +++ b/maitake-sync/README.md @@ -157,6 +157,15 @@ critical section, are described [here][interrupt-cfgs]. [single-core]: https://docs.rs/portable-atomic/latest/portable_atomic/#optional-cfg [interrupt-cfgs]: https://github.com/taiki-e/portable-atomic/blob/HEAD/src/imp/interrupt/README.md +### `lock_api` support + +In addition to async locks, `maitake-sync` also provides a [`blocking`] module, +which contains blocking `Mutex` and `RwLock` implementations. Many of +`maitake-sync`'s async synchronization primitives, including `WaitQueue`, +`Mutex`, `RwLock`, and `Semaphore`, internally use the [`blocking::Mutex`] type +for wait-list synchronization. + + ## features The following features are available (this list is incomplete; you can help by [expanding it].) @@ -169,8 +178,17 @@ The following features are available (this list is incomplete; you can help by [ | `no-cache-pad` | `false` | Inhibits cache padding for the [`CachePadded`] struct. When this feature is NOT enabled, the size will be determined based on target platform. | | `tracing` | `false` | Enables support for [`tracing`] diagnostics. Requires `liballoc`.| | `core-error` | `false` | Enables implementations of the [`core::error::Error` trait][core-error] for `maitake-sync`'s error types. *Requires a nightly Rust toolchain*. | +| `lock_api` | `false` | Enables support for the [`lock_api`] crate, which can be used to provide custom blocking `Mutex` implementations. See [the section on `lock_api` support](#lock_api-support) for details. | [`liballoc`]: https://doc.rust-lang.org/alloc/ [`CachePadded`]: https://docs.rs/maitake-sync/latest/maitake_sync/util/struct.CachePadded.html [`tracing`]: https://crates.io/crates/tracing -[core-error]: https://doc.rust-lang.org/stable/core/error/index.html \ No newline at end of file +[core-error]: https://doc.rust-lang.org/stable/core/error/index.html +[`lock_api`]: https://docs.rs/lock_api +[`blocking`]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html +[`blocking::Mutex`]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.Mutex.html +[`spin`]: https://docs.rs/maitake-sync/latest/maitake_sync/spin/index.html +[`Spinlock`]: + https://docs.rs/maitake-sync/latest/maitake_sync/spin/struct.Spinlock.html diff --git a/maitake-sync/src/blocking.rs b/maitake-sync/src/blocking.rs index 33d9a3ee..807b69a6 100644 --- a/maitake-sync/src/blocking.rs +++ b/maitake-sync/src/blocking.rs @@ -1,30 +1,44 @@ //! Synchronous (blocking) synchronization primitives. //! -//! The synchronization primitives in `maitake-sync` are _asynchronous_. They -//! are designed to be used with [`core::task`] and [`core::future`], and when -//! it is necessary to wait for another task to complete some work for the -//! current task to proceed, `maitake`'s synchronization primitives wait by -//! *yielding* to the asynchronous task scheduler to allow another task to -//! proceed. +//! The core synchronization primitives in `maitake-sync`, such as +//! [`Mutex`](crate::Mutex), [`RwLock`](crate::RwLock), and +//! [`WaitQueue`](crate::WaitQueue) are _asynchronous_. They are designed to be +//! used with [`core::task`] and [`core::future`], and when it is necessary to +//! wait for another task to complete some work for the current task to proceed, +//! `maitake`'s synchronization primitives wait by *yielding* to the +//! asynchronous task scheduler to allow another task to proceed. //! //! This module, on the other hand, provides _synchronous_ (or _blocking_) //! synchronization primitives. Rather than yielding to the runtime, these //! synchronization primitives will block the current CPU core (or thread, if //! running in an environment with threads) until they are woken by other cores. -//! This is performed by *spinning*: issuing yield or pause instructions in a -//! loop until some value changes. These synchronization primitives are, in some -//! cases, necessary to implement the async synchronization primitives that form -//! `maitake-sync`'s core APIs. They are also exposed publicly so they can be -//! used in other projects, when a spinlock-based synchronization primitive is -//! needed. +//! These synchronization primitives are, in some cases, necessary to implement +//! the async synchronization primitives that form `maitake-sync`'s core APIs. +//! They are also exposed publicly in this module so that they can be used in +//! other projects when a blocking-based synchronization primitive is needed.\ //! -//! This module provides the following APIs: +//! This module provides the following synchronization primitive types: //! //! - [`Mutex`]: a synchronous [mutual exclusion] lock. //! - [`RwLock`]: a synchronous [reader-writer] lock. - +//! +//! # `lock_api` support +//! +//! By default, the [`Mutex`] and [`RwLock`] types are implemented using simple +//! _[spinlocks_]_, which wait for the lock to become available by _spinning_: +//! repeatedly checking an atomic value in a loop, executing [spin-loop hint +//! instructions] until the lock value changes. These spinlock implementations +//! are represented by the [`Spinlock`] and [`RwSpinlock`] types in the +//! [`spin`] module. Spinlocks are a +//! +//! //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion -//! [reader-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock +//! [reader-writer lock]: +//! https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock +//! [spinlocks]: https://en.wikipedia.org/wiki/Spinlock +//! [spin-loop hint instructions]: core::hint::spin_loop +//! [`Spinlock`]: crate::spin::Spinlock +//! [`RwSpinlock`]: crate::spin::RwSpinlock mod mutex; mod rwlock; diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index d2ff308e..98ddef85 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -46,15 +46,48 @@ pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { /// Trait abstracting over blocking [`Mutex`] implementations (`maitake-sync`'s /// version). +/// +/// This trait is essentially a duplicate of the [`lock_api::RawMutex`] trait. +/// `maitake-sync` defines its own version of `RawMutex` so that the `lock_api` +/// dependency can be optional, and can be disabled when only using +/// `maitake-sync`'s spinlocks. When the "lock_api" feature flag is enabled, +/// this trait will be implemented for all types implementing +/// [`lock_api::RawMutex`]. Users who wish to provide their own `RawMutex` +/// implementations should implement the [`lock_api::RawMutex`] trait, *not* +/// this trait. +/// +/// # Safety +/// +/// Implementations of this trait must ensure that the mutex is actually +/// exclusive: a lock can't be acquired while the mutex is already locked. pub unsafe trait RawMutex { + /// Marker type which determines whether a lock guard should be [`Send`]. + /// + /// Implementations should use one of the [`lock_api::GuardSend`] or + /// [`lock_api::GuardNoSend`] helper types here. type GuardMarker; + /// Acquires this mutex, blocking the current thread/CPU core until it is + /// able to do so. fn lock(&self); + /// Attempts to acquire this mutex without blocking. Returns `true` + /// if the lock was successfully acquired and `false` otherwise. fn try_lock(&self) -> bool; + /// Unlocks this mutex. + /// + /// # Safety + /// + /// This method may only be called if the mutex is held in the current + /// context, i.e. it must be paired with a successful call to [`lock`] or + /// [`try_lock`]. + /// + /// [`lock`]: RawMutex::lock + /// [`try_lock`]: RawMutex::try_lock unsafe fn unlock(&self); + /// Returns `true` if the mutex is currently locked. fn is_locked(&self) -> bool; } @@ -115,6 +148,17 @@ impl Mutex where Lock: lock_api::RawMutex, { + /// Returns a new `Mutex` protecting the provided `data`, using the + /// `Lock` type parameter as the raw mutex implementation. + /// + /// This constructor is used to override the internal implementation of + /// mutex operations, with an implementation of the [`lock_api::RawMutex`] + /// trait. By default, the [`Mutex::new`] constructor uses a [`Spinlock`] as + /// the underlying raw mutex implementation, which will spin until the mutex + /// is unlocked, without using platform-specific or OS-specific blocking + /// mechanisms. + /// + /// The returned `Mutex` is in an unlocked state, ready for use. #[must_use] pub const fn with_raw_mutex(data: T) -> Self { Self { diff --git a/maitake-sync/src/blocking/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs index f3416922..fdf1fe49 100644 --- a/maitake-sync/src/blocking/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -72,23 +72,64 @@ pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { lock: &'lock Lock, } +/// Trait abstracting over blocking [`RwLock`] implementations (`maitake-sync`'s +/// version). +/// +/// This trait is essentially a duplicate of the [`lock_api::RawRwLock`] trait. +/// `maitake-sync` defines its own version of `RawRwLock` so that the `lock_api` +/// dependency can be optional, and can be disabled when only using +/// `maitake-sync`'s spinlocks. When the "lock_api" feature flag is enabled, +/// this trait will be implemented for all types implementing +/// [`lock_api::RawRwLock`]. Users who wish to provide their own `RawRwLock` +/// implementations should implement the [`lock_api::RawRwLock`] trait, *not* +/// this trait. +/// +/// # Safety +/// +/// Implementations of this trait must ensure that the `RwLock` is actually +/// exclusive: an exclusive lock can't be acquired while an exclusive or shared +/// lock exists, and a shared lock can't be acquire while an exclusive lock +/// exists. pub unsafe trait RawRwLock { + /// Marker type which determines whether a lock guard should be [`Send`]. + /// + /// Implementations should use one of the [`lock_api::GuardSend`] or + /// [`lock_api::GuardNoSend`] helper types here. type GuardMarker; + /// Acquires a shared lock, blocking the current thread/CPU core until it is + /// able to do so. fn lock_shared(&self); + /// Attempts to acquire a shared lock without blocking. fn try_lock_shared(&self) -> bool; + /// Releases a shared lock. + /// + /// # Safety + /// + /// This method may only be called if a shared lock is held in the current context. unsafe fn unlock_shared(&self); + /// Acquires an exclusive lock, blocking the current thread/CPU core until + /// it is able to do so. fn lock_exclusive(&self); + /// Attempts to acquire an exclusive lock without blocking. fn try_lock_exclusive(&self) -> bool; + /// Releases an exclusive lock. + /// + /// # Safety + /// + /// This method may only be called if an exclusive lock is held in the + /// current context. unsafe fn unlock_exclusive(&self); + /// Returns `true` if this `RwLock` is currently locked in any way. fn is_locked(&self) -> bool; + /// Returns `true` if this `RwLock` is currently locked exclusively. fn is_locked_exclusive(&self) -> bool; } From 99cfb40dbd9b2ad086972c7f37abb9ade1367152 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 18 Jul 2024 12:29:56 -0700 Subject: [PATCH 07/41] wip closure based API (broken) --- maitake-sync/src/blocking/mutex.rs | 84 ++++++- maitake-sync/src/loom.rs | 16 ++ maitake-sync/src/spin.rs | 8 +- maitake-sync/src/wait_queue.rs | 376 ++++++++++++++++------------- 4 files changed, 309 insertions(+), 175 deletions(-) diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 98ddef85..b72fe408 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -44,6 +44,15 @@ pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { lock: &'a Lock, } +pub unsafe trait RawScopedMutex { + fn with(&self, f: impl FnOnce() -> T) -> T; + + fn try_with(&self, f: impl FnOnce() -> T) -> Option; + + /// Returns `true` if the mutex is currently locked. + fn is_locked(&self) -> bool; +} + /// Trait abstracting over blocking [`Mutex`] implementations (`maitake-sync`'s /// version). /// @@ -91,6 +100,33 @@ pub unsafe trait RawMutex { fn is_locked(&self) -> bool; } +unsafe impl RawScopedMutex for L +where + L: RawMutex, +{ + fn with(&self, f: impl FnOnce() -> T) -> T { + // TODO(eliza): RAIIify + self.lock(); + let ret = f(); + unsafe { self.unlock() } + ret + } + + fn try_with(&self, f: impl FnOnce() -> T) -> Option { + if !self.try_lock() { + return None; + } + let ret = f(); + unsafe { self.unlock() } + Some(ret) + } + + /// Returns `true` if the mutex is currently locked. + fn is_locked(&self) -> bool { + RawMutex::is_locked(self) + } +} + #[cfg(feature = "lock_api")] unsafe impl RawMutex for T { type GuardMarker = ::GuardMarker; @@ -143,6 +179,29 @@ impl Mutex { } } +impl Mutex { + loom_const_fn! { + /// Returns a new `Mutex` protecting the provided `data`, using the + /// `Lock` type parameter as the raw mutex implementation. + /// + /// This constructor is used to override the internal implementation of + /// mutex operations, with an implementation of the [`lock_api::RawMutex`] + /// trait. By default, the [`Mutex::new`] constructor uses a [`Spinlock`] as + /// the underlying raw mutex implementation, which will spin until the mutex + /// is unlocked, without using platform-specific or OS-specific blocking + /// mechanisms. + /// + /// The returned `Mutex` is in an unlocked state, ready for use. + #[must_use] + pub fn with_raw_mutex(data: T, lock: Lock) -> Self { + Self { + lock, + data: UnsafeCell::new(data), + } + } + } +} + #[cfg(feature = "lock_api")] impl Mutex where @@ -160,11 +219,32 @@ where /// /// The returned `Mutex` is in an unlocked state, ready for use. #[must_use] - pub const fn with_raw_mutex(data: T) -> Self { + pub const fn with_lock_api(data: T) -> Self { Self { lock: Lock::INIT, data: UnsafeCell::new(data), - } + } /// Returns a new `Mutex` protecting the provided `data`, using the + /// `Lock` type parameter as the raw mutex implementation. + /// + /// This constructor is used to override the internal implementation of + /// mutex operations, with an implementation of the [`lock_api::RawMutex`] + /// trait. By default, the [`Mutex::new`] constructor uses a [`Spinlock`] as + /// the underlying raw mutex implementation, which will spin until the mutex + /// is unlocked, without using platform-specific or OS-specific blocking + /// mechanisms. + /// + /// The returned `Mutex` is in an unlocked state, ready for use. + } +} + +impl Mutex { + pub fn with(&self, f: impl FnOnce(&mut T) -> U) -> U { + self.lock.with(|| { + self.data.with_mut(|data| unsafe { + // Safety: we just locked the mutex. + f(&mut *data) + }) + }) } } diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index c4e9e739..9a84ab9f 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -99,6 +99,22 @@ mod inner { Self(loom::sync::Mutex::new(t), PhantomData) } + #[track_caller] + pub(crate) fn with_raw_mutex(t: T, _: Lock) -> Self { + Self::new(t) + } + + #[track_caller] + pub fn with(&self, f: impl FnOnce(&mut T) -> U) -> U { + let location = core::panic::Location::caller(); + tracing::debug!( %location, "Mutex::with: locking...",); + let mut guard = self.lock(); + tracing::debug!(%location, "Mutex::with: -> locked!",); + let res = f(&mut *guard); + tracing::debug!(%location, "Mutex::with: unlocking...",); + res + } + #[track_caller] pub fn try_lock(&self) -> Option> { self.0.try_lock().map(|x| MutexGuard(x, PhantomData)).ok() diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index 5203a329..d7d5cf25 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -61,6 +61,12 @@ impl Spinlock { Self { locked: AtomicBool::new(false) } } } + + #[inline] + #[must_use] + fn is_locked(&self) -> bool { + self.locked.load(Relaxed) + } } impl Default for Spinlock { @@ -103,7 +109,7 @@ unsafe impl RawMutex for Spinlock { #[inline] fn is_locked(&self) -> bool { - self.locked.load(Relaxed) + Spinlock::is_locked(self) } } diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index fb021e42..75113364 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -5,12 +5,12 @@ #[cfg(any(test, maitake_ultraverbose))] use crate::util::fmt; use crate::{ - blocking::RawMutex, + blocking::RawScopedMutex, loom::{ cell::UnsafeCell, sync::{ atomic::{AtomicUsize, Ordering::*}, - spin::{Mutex, MutexGuard}, + spin::Mutex, }, }, spin::Spinlock, @@ -177,7 +177,7 @@ mod tests; /// [mutex]: crate::Mutex /// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue #[derive(Debug)] -pub struct WaitQueue { +pub struct WaitQueue { /// The wait queue's state variable. state: CachePadded, @@ -221,7 +221,7 @@ pub struct WaitQueue { #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Wait<'a, Lock: RawMutex = Spinlock> { +pub struct Wait<'a, Lock: RawScopedMutex = Spinlock> { /// The [`WaitQueue`] being waited on. queue: &'a WaitQueue, @@ -360,7 +360,7 @@ impl WaitQueue { /// Returns a new `WaitQueue`. #[must_use] pub fn new() -> Self { - Self::make(State::Empty, Mutex::new(List::new())) + Self::make(State::Empty, Spinlock::new()) } } @@ -373,32 +373,31 @@ impl WaitQueue { // TODO(eliza): should this be a public API? #[must_use] pub(crate) fn new_woken() -> Self { - Self::make(State::Woken, Mutex::new(List::new())) + Self::make(State::Woken, Spinlock::new()) } } } #[cfg(all(feature = "lock_api", not(loom)))] +impl WaitQueue where Lock: lock_api::RawMutex {} + impl WaitQueue where - Lock: lock_api::RawMutex, + Lock: RawScopedMutex, { - #[must_use] - pub const fn with_raw_mutex() -> Self { - Self::make(State::Empty, Mutex::with_raw_mutex(List::new())) + loom_const_fn! { + #[must_use] + pub fn with_raw_mutex(lock: Lock) -> Self { + Self::make(State::Empty, lock) + } } -} -impl WaitQueue -where - Lock: RawMutex, -{ loom_const_fn! { #[must_use] - fn make(state: State, queue: Mutex, Lock>) -> Self { + fn make(state: State, lock: Lock) -> Self { Self { state: CachePadded::new(AtomicUsize::new(state.into_usize())), - queue, + queue: Mutex::with_raw_mutex(List::new(), lock) } } } @@ -472,15 +471,19 @@ where // okay, there are tasks waiting on the queue; we must acquire the lock // on the linked list and wake the next task from the queue. - let mut queue = self.queue.lock(); - test_debug!("wake: -> locked"); - - // the queue's state may have changed while we were waiting to acquire - // the lock, so we need to acquire a new snapshot. - state = self.load(); - - if let Some(waker) = self.wake_locked(&mut queue, state) { - drop(queue); + let waker = self.queue.with(|queue| { + test_debug!("wake: -> locked"); + + // the queue's state may have changed while we were waiting to acquire + // the lock, so we need to acquire a new snapshot before we take the + // waker. + state = self.load(); + self.wake_locked(queue, state) + }); + + //now that we've released the lock, wake the waiting task (if we + //actually deuqueued one). + if let Some(waker) = waker { waker.wake(); } } @@ -546,36 +549,58 @@ where /// [`wake()`]: Self::wake /// [`wait()`]: Self::wait pub fn wake_all(&self) { - let mut queue = self.queue.lock(); - let state = self.load(); - - match state.get(QueueState::STATE) { - // if the queue is closed, bail. - State::Closed => return, + let mut batch = WakeBatch::new(); + let mut waking = true; + let (current_state, done) = self.queue.with(|queue| { + let current_state = self.load(); + let state = current_state.get(QueueState::STATE); // if there are no waiters in the queue, increment the number of - // `wake_all` calls and return. - State::Woken | State::Empty => { + // `wake_all` calls and return. incrementing the `wake_all` count + // must be performed inside the lock, so we do it here. + + if state == State::Waiting { + // if the queue is closed, or there are no wakers, nothing left to + // do here. + return (current_state, true); + } else if let State::Woken | State::Empty = state { self.state.fetch_add(QueueState::ONE_WAKE_ALL, SeqCst); - return; + return (current_state, true); } - State::Waiting => {} + waking = Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All); + if !waking { + let next_state = QueueState::new().with_state(State::Empty).with( + QueueState::WAKE_ALLS, + current_state.get(QueueState::WAKE_ALLS) + 1, + ); + self.compare_exchange(current_state, next_state) + .expect("state should not have transitioned while locked"); + } + + (current_state, false) + }); + + if done { + return; } - let mut batch = WakeBatch::new(); - queue = self.drain_to_wake_batch(&mut batch, queue, Wakeup::All); - - // now that the queue has been drained, transition to the empty state, - // and increment the wake_all count. - let next_state = QueueState::new() - .with_state(State::Empty) - .with(QueueState::WAKE_ALLS, state.get(QueueState::WAKE_ALLS) + 1); - self.compare_exchange(state, next_state) - .expect("state should not have transitioned while locked"); - - // wake any tasks that were woken in the last iteration of the batch loop. - drop(queue); batch.wake_all(); + while waking { + self.queue.with(|queue| { + if waking && batch.can_add_waker() { + waking = Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All); + } + if !waking { + let next_state = QueueState::new().with_state(State::Empty).with( + QueueState::WAKE_ALLS, + current_state.get(QueueState::WAKE_ALLS) + 1, + ); + self.compare_exchange(current_state, next_state) + .expect("state should not have transitioned while locked"); + } + }); + batch.wake_all(); + } } /// Close the queue, indicating that it may no longer be used. @@ -596,10 +621,13 @@ where } let mut batch = WakeBatch::new(); - let _ = self.drain_to_wake_batch(&mut batch, self.queue.lock(), Wakeup::Closed); - - // wake any tasks that were woken in the last iteration of the batch loop. - batch.wake_all(); + let mut waking = true; + while waking { + waking = self + .queue + .with(|queue| Self::drain_to_wake_batch(&mut batch, queue, Wakeup::Closed)); + batch.wake_all(); + } } /// Wait to be woken up by this queue. @@ -991,18 +1019,19 @@ where /// batch. Instead, it returns the [`MutexGuard`], in case additional /// operations must be performed with the lock held before waking the final /// batch of waiters. - fn drain_to_wake_batch<'q>( - &'q self, + fn drain_to_wake_batch( batch: &mut WakeBatch, - mut queue: MutexGuard<'q, List, Lock>, + queue: &mut List, wakeup: Wakeup, - ) -> MutexGuard<'q, List, Lock> { + ) -> bool { + let mut enqueued_any = false; while let Some(node) = queue.pop_back() { - let Some(waker) = Waiter::wake(node, &mut queue, wakeup.clone()) else { + let Some(waker) = Waiter::wake(node, queue, wakeup.clone()) else { // this waiter was enqueued by `Wait::register` and doesn't have // a waker, just keep going. continue; }; + enqueued_any = true; if batch.add_waker(waker) { // there's still room in the wake set, just keep adding to it. @@ -1010,14 +1039,10 @@ where } // wake set is full, drop the lock and wake everyone! - drop(queue); - batch.wake_all(); - - // reacquire the lock and continue waking - queue = self.queue.lock(); + break; } - queue + enqueued_any } } @@ -1081,14 +1106,15 @@ impl Waiter { waker: Option<&Waker>, ) -> Poll> where - Lock: RawMutex, + Lock: RawScopedMutex, { test_debug!(ptr = ?fmt::ptr(self.as_mut()), "Waiter::poll_wait"); + let ptr = unsafe { NonNull::from(Pin::into_inner_unchecked(self.as_mut())) }; let mut this = self.as_mut().project(); match test_dbg!(this.state.get(WaitStateBits::STATE)) { WaitState::Start => { - let mut queue_state = queue.load(); + let queue_state = queue.load(); // can we consume a pending wakeup? if queue @@ -1103,101 +1129,104 @@ impl Waiter { } // okay, no pending wakeups. try to wait... + test_debug!("poll_wait: locking..."); - let mut waiters = queue.queue.lock(); - test_debug!("poll_wait: -> locked"); - queue_state = queue.load(); - - // the whole queue was woken while we were trying to acquire - // the lock! - if queue_state.get(QueueState::WAKE_ALLS) - != this.state.get(WaitStateBits::WAKE_ALLS) - { - this.state.set(WaitStateBits::STATE, WaitState::Woken); - return Poll::Ready(Ok(())); - } + queue.queue.with(move |waiters| { + test_debug!("poll_wait: -> locked"); + let mut queue_state = queue.load(); + + // the whole queue was woken while we were trying to acquire + // the lock! + if queue_state.get(QueueState::WAKE_ALLS) + != this.state.get(WaitStateBits::WAKE_ALLS) + { + this.state.set(WaitStateBits::STATE, WaitState::Woken); + return Poll::Ready(Ok(())); + } - // transition the queue to the waiting state - 'to_waiting: loop { - match test_dbg!(queue_state.get(QueueState::STATE)) { - // the queue is `Empty`, transition to `Waiting` - State::Empty => { - match queue.compare_exchange( - queue_state, - queue_state.with_state(State::Waiting), - ) { - Ok(_) => break 'to_waiting, - Err(actual) => queue_state = actual, + // transition the queue to the waiting state + 'to_waiting: loop { + match test_dbg!(queue_state.get(QueueState::STATE)) { + // the queue is `Empty`, transition to `Waiting` + State::Empty => { + match queue.compare_exchange( + queue_state, + queue_state.with_state(State::Waiting), + ) { + Ok(_) => break 'to_waiting, + Err(actual) => queue_state = actual, + } } - } - // the queue is already `Waiting` - State::Waiting => break 'to_waiting, - // the queue was woken, consume the wakeup. - State::Woken => { - match queue - .compare_exchange(queue_state, queue_state.with_state(State::Empty)) - { - Ok(_) => { - this.state.set(WaitStateBits::STATE, WaitState::Woken); - return Poll::Ready(Ok(())); + // the queue is already `Waiting` + State::Waiting => break 'to_waiting, + // the queue was woken, consume the wakeup. + State::Woken => { + match queue.compare_exchange( + queue_state, + queue_state.with_state(State::Empty), + ) { + Ok(_) => { + this.state.set(WaitStateBits::STATE, WaitState::Woken); + return Poll::Ready(Ok(())); + } + Err(actual) => queue_state = actual, } - Err(actual) => queue_state = actual, } + State::Closed => return crate::closed(), } - State::Closed => return crate::closed(), } - } - // enqueue the node - this.state.set(WaitStateBits::STATE, WaitState::Waiting); - if let Some(waker) = waker { - this.node.as_mut().with_mut(|node| { - unsafe { - // safety: we may mutate the node because we are - // holding the lock. - debug_assert!(matches!((*node).waker, Wakeup::Empty)); - (*node).waker = Wakeup::Waiting(waker.clone()); - } - }); - } + // enqueue the node + this.state.set(WaitStateBits::STATE, WaitState::Waiting); + if let Some(waker) = waker { + this.node.as_mut().with_mut(|node| { + unsafe { + // safety: we may mutate the node because we are + // holding the lock. + debug_assert!(matches!((*node).waker, Wakeup::Empty)); + (*node).waker = Wakeup::Waiting(waker.clone()); + } + }); + } - let ptr = unsafe { NonNull::from(Pin::into_inner_unchecked(self)) }; - waiters.push_front(ptr); + waiters.push_front(ptr); - Poll::Pending + Poll::Pending + }) } WaitState::Waiting => { - let mut _waiters = queue.queue.lock(); - this.node.with_mut(|node| unsafe { - // safety: we may mutate the node because we are - // holding the lock. - let node = &mut *node; - match node.waker { - Wakeup::Waiting(ref mut curr_waker) => { - match waker { - Some(waker) if !curr_waker.will_wake(waker) => { - *curr_waker = waker.clone() + queue.queue.with(|_waiters| { + this.node.with_mut(|node| unsafe { + // safety: we may mutate the node because we are + // holding the lock. + let node = &mut *node; + match node.waker { + Wakeup::Waiting(ref mut curr_waker) => { + match waker { + Some(waker) if !curr_waker.will_wake(waker) => { + *curr_waker = waker.clone() + } + _ => {} } - _ => {} + Poll::Pending } - Poll::Pending - } - Wakeup::All | Wakeup::One => { - this.state.set(WaitStateBits::STATE, WaitState::Woken); - Poll::Ready(Ok(())) - } - Wakeup::Closed => { - this.state.set(WaitStateBits::STATE, WaitState::Woken); - crate::closed() - } - Wakeup::Empty => { - if let Some(waker) = waker { - node.waker = Wakeup::Waiting(waker.clone()); + Wakeup::All | Wakeup::One => { + this.state.set(WaitStateBits::STATE, WaitState::Woken); + Poll::Ready(Ok(())) } + Wakeup::Closed => { + this.state.set(WaitStateBits::STATE, WaitState::Woken); + crate::closed() + } + Wakeup::Empty => { + if let Some(waker) = waker { + node.waker = Wakeup::Waiting(waker.clone()); + } - Poll::Pending + Poll::Pending + } } - } + }) }) } WaitState::Woken => Poll::Ready(Ok(())), @@ -1210,7 +1239,7 @@ impl Waiter { /// [`WaitOwned`] futures. fn release(mut self: Pin<&mut Self>, queue: &WaitQueue) where - Lock: RawMutex, + Lock: RawScopedMutex, { let state = *(self.as_mut().project().state); let ptr = NonNull::from(unsafe { Pin::into_inner_unchecked(self) }); @@ -1221,28 +1250,31 @@ impl Waiter { return; } - let mut waiters = queue.queue.lock(); - let state = queue.load(); + let next_waiter = queue.queue.with(|waiters| { + let state = queue.load(); + // remove the node + unsafe { + // safety: we have the lock on the queue, so this is safe. + waiters.remove(ptr); + }; - // remove the node - unsafe { - // safety: we have the lock on the queue, so this is safe. - waiters.remove(ptr); - }; - - // if we removed the last waiter from the queue, transition the state to - // `Empty`. - if test_dbg!(waiters.is_empty()) && state.get(QueueState::STATE) == State::Waiting { - queue.store(state.with_state(State::Empty)); - } + // if we removed the last waiter from the queue, transition the state to + // `Empty`. + if test_dbg!(waiters.is_empty()) && state.get(QueueState::STATE) == State::Waiting { + queue.store(state.with_state(State::Empty)); + } - // if the node has an unconsumed wakeup, it must be assigned to the next - // node in the queue. - if Waiter::with_node(ptr, &mut waiters, |node| matches!(&node.waker, Wakeup::One)) { - if let Some(waker) = queue.wake_locked(&mut waiters, state) { - drop(waiters); - waker.wake() + // if the node has an unconsumed wakeup, it must be assigned to the next + // node in the queue. + if Waiter::with_node(ptr, waiters, |node| matches!(&node.waker, Wakeup::One)) { + queue.wake_locked(waiters, state) + } else { + None } + }); + + if let Some(next) = next_waiter { + next.wake(); } } } @@ -1274,7 +1306,7 @@ unsafe impl Linked> for Waiter { // === impl Wait === -impl Wait<'_, Lock> { +impl Wait<'_, Lock> { /// Returns `true` if this `Wait` future is waiting for a notification from /// the provided [`WaitQueue`]. /// @@ -1386,7 +1418,7 @@ impl Wait<'_, Lock> { } } -impl Future for Wait<'_, Lock> { +impl Future for Wait<'_, Lock> { type Output = WaitResult<()>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1396,7 +1428,7 @@ impl Future for Wait<'_, Lock> { } #[pinned_drop] -impl PinnedDrop for Wait<'_, Lock> { +impl PinnedDrop for Wait<'_, Lock> { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(this.queue); @@ -1472,7 +1504,7 @@ feature! { /// ``` #[derive(Debug)] #[pin_project(PinnedDrop)] - pub struct WaitOwned { + pub struct WaitOwned { /// The `WaitQueue` being waited on. queue: Arc>, @@ -1481,7 +1513,7 @@ feature! { waiter: Waiter, } - impl WaitQueue { + impl WaitQueue { /// Wait to be woken up by this queue, returning a future that's valid /// for the `'static` lifetime. /// @@ -1525,7 +1557,7 @@ feature! { // === impl WaitOwned === - impl WaitOwned { + impl WaitOwned { /// Returns `true` if this `WaitOwned` future is waiting for a /// notification from the provided [`WaitQueue`]. /// @@ -1644,7 +1676,7 @@ feature! { } } - impl Future for WaitOwned { + impl Future for WaitOwned { type Output = WaitResult<()>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1654,7 +1686,7 @@ feature! { } #[pinned_drop] - impl PinnedDrop for WaitOwned { + impl PinnedDrop for WaitOwned { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(&*this.queue); From ccc47d83e1bfc2ab0bb085e84b0538846de1055f Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 20 Jul 2024 10:32:18 -0700 Subject: [PATCH 08/41] make WaitQueue actually work --- maitake-sync/src/blocking/mutex.rs | 12 +----------- maitake-sync/src/wait_queue.rs | 27 +++++++-------------------- 2 files changed, 8 insertions(+), 31 deletions(-) diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index b72fe408..4f07cf00 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -223,17 +223,7 @@ where Self { lock: Lock::INIT, data: UnsafeCell::new(data), - } /// Returns a new `Mutex` protecting the provided `data`, using the - /// `Lock` type parameter as the raw mutex implementation. - /// - /// This constructor is used to override the internal implementation of - /// mutex operations, with an implementation of the [`lock_api::RawMutex`] - /// trait. By default, the [`Mutex::new`] constructor uses a [`Spinlock`] as - /// the underlying raw mutex implementation, which will spin until the mutex - /// is unlocked, without using platform-specific or OS-specific blocking - /// mechanisms. - /// - /// The returned `Mutex` is in an unlocked state, ready for use. + } } } diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index 75113364..abaf7e01 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -551,6 +551,7 @@ where pub fn wake_all(&self) { let mut batch = WakeBatch::new(); let mut waking = true; + let (current_state, done) = self.queue.with(|queue| { let current_state = self.load(); let state = current_state.get(QueueState::STATE); @@ -558,16 +559,11 @@ where // if there are no waiters in the queue, increment the number of // `wake_all` calls and return. incrementing the `wake_all` count // must be performed inside the lock, so we do it here. - - if state == State::Waiting { - // if the queue is closed, or there are no wakers, nothing left to - // do here. - return (current_state, true); - } else if let State::Woken | State::Empty = state { + if let State::Woken | State::Empty = state { self.state.fetch_add(QueueState::ONE_WAKE_ALL, SeqCst); return (current_state, true); } - waking = Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All); + waking = test_dbg!(Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All)); if !waking { let next_state = QueueState::new().with_state(State::Empty).with( QueueState::WAKE_ALLS, @@ -1009,29 +1005,20 @@ where waker } - /// Drain the queue of all waiters, and push them to `batch`. - /// - /// When the [`WakeBatch`] is full, this function drops the lock, wakes the - /// current contents of the [`WakeBatch`] before reacquiring the lock and - /// continuing. - /// - /// Note that this will *not* wake the final batch of waiters added to the - /// batch. Instead, it returns the [`MutexGuard`], in case additional - /// operations must be performed with the lock held before waking the final - /// batch of waiters. + /// Drain waiters from `queue` and add them to `batch`. Returns `true` if + /// the batch was filled while more waiters remain in the queue, indicating + /// that this function must be called again to wake all waiters. fn drain_to_wake_batch( batch: &mut WakeBatch, queue: &mut List, wakeup: Wakeup, ) -> bool { - let mut enqueued_any = false; while let Some(node) = queue.pop_back() { let Some(waker) = Waiter::wake(node, queue, wakeup.clone()) else { // this waiter was enqueued by `Wait::register` and doesn't have // a waker, just keep going. continue; }; - enqueued_any = true; if batch.add_waker(waker) { // there's still room in the wake set, just keep adding to it. @@ -1042,7 +1029,7 @@ where break; } - enqueued_any + !queue.is_empty() } } From db1890bf7ca6ea8193f0ca203432e2437ecc8cae Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 20 Jul 2024 10:42:36 -0700 Subject: [PATCH 09/41] simplify wake_all implementation --- maitake-sync/src/wait_queue.rs | 70 ++++++++++++++++++---------------- 1 file changed, 37 insertions(+), 33 deletions(-) diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index abaf7e01..a3595b77 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -550,30 +550,42 @@ where /// [`wait()`]: Self::wait pub fn wake_all(&self) { let mut batch = WakeBatch::new(); - let mut waking = true; + let mut waiters_remaining = true; + + // This is a little bit contorted: we must load the state inside the + // lock, but for all states except for `Waiting`, we just need to bail + // out...but we can't `return` from the outer function inside the lock + // closure. Therefore, we just return a `bool` and, if it's `true`, + // return instead of doing more work. + let done = self.queue.with(|queue| { + let state = self.load(); + + match test_dbg!(state.get(QueueState::STATE)) { + // If there are no waiters in the queue, increment the number of + // `wake_all` calls and return. incrementing the `wake_all` count + // must be performed inside the lock, so we do it here. + State::Woken | State::Empty => { + self.state.fetch_add(QueueState::ONE_WAKE_ALL, SeqCst); + true + } + // If the queue is already closed, this is a no-op. Just bail. + State::Closed => true, + // Okay, there are waiters in the queue. Transition to the empty + // state inside the lock and start draining the queue. + State::Waiting => { + let next_state = QueueState::new() + .with_state(State::Empty) + .with(QueueState::WAKE_ALLS, state.get(QueueState::WAKE_ALLS) + 1); + self.compare_exchange(state, next_state) + .expect("state should not have transitioned while locked"); - let (current_state, done) = self.queue.with(|queue| { - let current_state = self.load(); - let state = current_state.get(QueueState::STATE); + // Drain the first batch of waiters from the queue. + waiters_remaining = + test_dbg!(Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All)); - // if there are no waiters in the queue, increment the number of - // `wake_all` calls and return. incrementing the `wake_all` count - // must be performed inside the lock, so we do it here. - if let State::Woken | State::Empty = state { - self.state.fetch_add(QueueState::ONE_WAKE_ALL, SeqCst); - return (current_state, true); - } - waking = test_dbg!(Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All)); - if !waking { - let next_state = QueueState::new().with_state(State::Empty).with( - QueueState::WAKE_ALLS, - current_state.get(QueueState::WAKE_ALLS) + 1, - ); - self.compare_exchange(current_state, next_state) - .expect("state should not have transitioned while locked"); + false + } } - - (current_state, false) }); if done { @@ -581,19 +593,11 @@ where } batch.wake_all(); - while waking { + // As long as there are waiters remaining to wake, lock the queue, drain + // another batch, release the lock, and wake them. + while waiters_remaining { self.queue.with(|queue| { - if waking && batch.can_add_waker() { - waking = Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All); - } - if !waking { - let next_state = QueueState::new().with_state(State::Empty).with( - QueueState::WAKE_ALLS, - current_state.get(QueueState::WAKE_ALLS) + 1, - ); - self.compare_exchange(current_state, next_state) - .expect("state should not have transitioned while locked"); - } + waiters_remaining = Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All); }); batch.wake_all(); } From 791c2d4fb852e0056828cba803e82b48c6414bde Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 20 Jul 2024 11:04:35 -0700 Subject: [PATCH 10/41] update async `Mutex` to use scoped mutex API This is trivial as it's just a waitqueue wrapper --- maitake-sync/src/mutex.rs | 52 +++++++++++++++++------------------ maitake-sync/src/semaphore.rs | 4 +-- 2 files changed, 28 insertions(+), 28 deletions(-) diff --git a/maitake-sync/src/mutex.rs b/maitake-sync/src/mutex.rs index 722a9a9d..fe8859c9 100644 --- a/maitake-sync/src/mutex.rs +++ b/maitake-sync/src/mutex.rs @@ -4,7 +4,7 @@ //! //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion use crate::{ - blocking::RawMutex, + blocking::RawScopedMutex, loom::cell::{MutPtr, UnsafeCell}, spin::Spinlock, util::fmt, @@ -86,7 +86,7 @@ mod tests; /// [storage]: https://mycelium.elizas.website/maitake/task/trait.Storage.html /// [no-unwinding]: https://mycelium.elizas.website/maitake/index.html#maitake-does-not-support-unwinding -pub struct Mutex { +pub struct Mutex { wait: WaitQueue, data: UnsafeCell, } @@ -107,7 +107,7 @@ pub struct Mutex { /// [`try_lock`]: Mutex::try_lock /// [RAII]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[must_use = "if unused, the `Mutex` will immediately unlock"] -pub struct MutexGuard<'a, T: ?Sized, L: RawMutex = Spinlock> { +pub struct MutexGuard<'a, T: ?Sized, L: RawScopedMutex = Spinlock> { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important. @@ -136,7 +136,7 @@ pub struct MutexGuard<'a, T: ?Sized, L: RawMutex = Spinlock> { #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] #[pin_project] #[derive(Debug)] -pub struct Lock<'a, T: ?Sized, L: RawMutex = Spinlock> { +pub struct Lock<'a, T: ?Sized, L: RawScopedMutex = Spinlock> { #[pin] wait: wait_queue::Wait<'a, L>, mutex: &'a Mutex, @@ -144,7 +144,7 @@ pub struct Lock<'a, T: ?Sized, L: RawMutex = Spinlock> { /// This is used in order to ensure that the wakeup is performed only *after* /// the data ptr is dropped, in order to keep `loom` happy. -struct WakeOnDrop<'a, T: ?Sized, L: RawMutex>(&'a Mutex); +struct WakeOnDrop<'a, T: ?Sized, L: RawScopedMutex>(&'a Mutex); // === impl Mutex === @@ -182,7 +182,7 @@ impl Mutex { } } -impl Mutex { +impl Mutex { /// Consumes this `Mutex`, returning the guarded data. #[inline] #[must_use] @@ -191,7 +191,7 @@ impl Mutex { } } -impl Mutex { +impl Mutex { /// Locks this mutex. /// /// This returns a [`Lock`] future that will wait until no other task is @@ -296,7 +296,7 @@ impl Default for Mutex { impl fmt::Debug for Mutex where T: ?Sized + fmt::Debug, - L: RawMutex + fmt::Debug, + L: RawScopedMutex + fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let Self { data: _, wait } = self; @@ -307,13 +307,13 @@ where } } -unsafe impl Send for Mutex +unsafe impl Send for Mutex where T: ?Sized + Send, L: Send, { } -unsafe impl Sync for Mutex +unsafe impl Sync for Mutex where T: ?Sized + Send, L: Sync, @@ -349,7 +349,7 @@ impl<'a, T> Future for Lock<'a, T> { impl Deref for MutexGuard<'_, T, L> where T: ?Sized, - L: RawMutex, + L: RawScopedMutex, { type Target = T; @@ -365,7 +365,7 @@ where impl DerefMut for MutexGuard<'_, T, L> where T: ?Sized, - L: RawMutex, + L: RawScopedMutex, { #[inline] fn deref_mut(&mut self) -> &mut Self::Target { @@ -379,7 +379,7 @@ where impl fmt::Debug for MutexGuard<'_, T, L> where T: ?Sized + fmt::Debug, - L: RawMutex, + L: RawScopedMutex, { #[inline] fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { @@ -390,18 +390,18 @@ where unsafe impl Send for MutexGuard<'_, T, L> where T: ?Sized + Send, - L: RawMutex + Sync, + L: RawScopedMutex + Sync, { } unsafe impl Sync for MutexGuard<'_, T, L> where T: ?Sized + Send + Sync, - // A `MutexGuard`` has a reference to a `L`-typed RawMutex in it, so `` - L: RawMutex + Sync, + // A `MutexGuard`` has a reference to a `L`-typed RawScopedMutex in it, so `` + L: RawScopedMutex + Sync, { } -impl Drop for WakeOnDrop<'_, T, L> { +impl Drop for WakeOnDrop<'_, T, L> { fn drop(&mut self) { self.0.wait.wake() } @@ -434,7 +434,7 @@ feature! { /// [`try_lock_owned`]: Mutex::try_lock_owned /// [RAII]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[must_use = "if unused, the Mutex will immediately unlock"] - pub struct OwnedMutexGuard { + pub struct OwnedMutexGuard { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important. @@ -442,7 +442,7 @@ feature! { _wake: WakeArcOnDrop, } - impl Mutex { + impl Mutex { /// Locks this mutex, returning an [owned RAII guard][`OwnedMutexGuard`]. /// @@ -550,14 +550,14 @@ feature! { } } - struct WakeArcOnDrop(Arc>); + struct WakeArcOnDrop(Arc>); // === impl OwnedMutexGuard === impl Deref for OwnedMutexGuard where T: ?Sized, - L: RawMutex, + L: RawScopedMutex, { type Target = T; @@ -573,7 +573,7 @@ feature! { impl DerefMut for OwnedMutexGuard where T: ?Sized, - L: RawMutex, + L: RawScopedMutex, { #[inline] fn deref_mut(&mut self) -> &mut Self::Target { @@ -587,7 +587,7 @@ feature! { impl fmt::Debug for OwnedMutexGuard where T: ?Sized + fmt::Debug, - L: RawMutex, + L: RawScopedMutex, { #[inline] fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { @@ -598,20 +598,20 @@ feature! { unsafe impl Send for OwnedMutexGuard where T: ?Sized + Send, - L: RawMutex + Sync, + L: RawScopedMutex + Sync, { } unsafe impl Sync for OwnedMutexGuard where T: ?Sized + Send + Sync, - L: RawMutex + Sync, + L: RawScopedMutex + Sync, { } impl Drop for WakeArcOnDrop where T: ?Sized, - L: RawMutex, + L: RawScopedMutex, { fn drop(&mut self) { self.0.wait.wake() diff --git a/maitake-sync/src/semaphore.rs b/maitake-sync/src/semaphore.rs index 59d8a996..2bb20d97 100644 --- a/maitake-sync/src/semaphore.rs +++ b/maitake-sync/src/semaphore.rs @@ -316,8 +316,8 @@ impl Semaphore { #[cfg(feature = "lock_api")] impl Semaphore { #[must_use] - pub const fn with_raw_mutex(permits: usize) -> Self { - Self::make(permits, Mutex::with_raw_mutex(SemQueue::new())) + pub const fn with_lock_api(permits: usize) -> Self { + Self::make(permits, Mutex::with_lock_api(SemQueue::new())) } } From 03c9f0a68c4827c423cdcb326495d5575eae5cfe Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 20 Jul 2024 11:13:38 -0700 Subject: [PATCH 11/41] add `try_with` to `blocking::Mutex`, `Debug` --- maitake-sync/src/blocking/mutex.rs | 27 ++++++++++++++++++++++----- 1 file changed, 22 insertions(+), 5 deletions(-) diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 4f07cf00..eb453386 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -236,6 +236,15 @@ impl Mutex { }) }) } + + pub fn try_with(&self, f: impl FnOnce(&mut T) -> U) -> Option { + self.lock.try_with(|| { + self.data.with_mut(|data| unsafe { + // Safety: we just locked the mutex. + f(&mut *data) + }) + }) + } } impl Mutex @@ -337,13 +346,21 @@ impl Default for Mutex { impl fmt::Debug for Mutex where T: fmt::Debug, - Lock: fmt::Debug + RawMutex, + Lock: RawScopedMutex + fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Mutex") - .field("data", &fmt::opt(&self.try_lock()).or_else("")) - .field("lock", &self.lock) - .finish() + self.try_with(|data| { + f.debug_struct("Mutex") + .field("data", data) + .field("lock", &self.lock) + .finish() + }) + .unwrap_or_else(|| { + f.debug_struct("Mutex") + .field("data", &format_args!("")) + .field("lock", &self.lock) + .finish() + }) } } From 6f42ce371cba9a2d2bd904a31db49dd33f9888fe Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 20 Jul 2024 13:40:33 -0700 Subject: [PATCH 12/41] also do waitqueue --- maitake-sync/src/wait_map.rs | 353 +++++++++++++++++++-------------- maitake-sync/src/wait_queue.rs | 10 +- 2 files changed, 214 insertions(+), 149 deletions(-) diff --git a/maitake-sync/src/wait_map.rs b/maitake-sync/src/wait_map.rs index e8924358..8833d641 100644 --- a/maitake-sync/src/wait_map.rs +++ b/maitake-sync/src/wait_map.rs @@ -3,13 +3,15 @@ //! //! See the documentation for the [`WaitMap`] type for details. use crate::{ + blocking::RawScopedMutex, loom::{ cell::UnsafeCell, sync::{ atomic::{AtomicUsize, Ordering::*}, - spin::{Mutex, MutexGuard}, + spin::Mutex, }, }, + spin::Spinlock, util::{fmt, CachePadded, WakeBatch}, }; use cordyceps::{ @@ -176,7 +178,7 @@ const fn notified(data: T) -> Poll> { /// [ilist]: cordyceps::List /// [intrusive]: https://fuchsia.dev/fuchsia-src/development/languages/c-cpp/fbl_containers_guide/introduction /// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue -pub struct WaitMap { +pub struct WaitMap { /// The wait queue's state variable. state: CachePadded, @@ -196,10 +198,14 @@ pub struct WaitMap { /// `no_std` platforms; when running `loom` tests, a `loom` mutex is used /// instead to simulate the spinlock, because loom doesn't play nice with /// real spinlocks. - queue: Mutex>>, + queue: Mutex>, Lock>, } -impl Debug for WaitMap { +impl Debug for WaitMap +where + K: PartialEq, + Lock: RawScopedMutex + fmt::Debug, +{ fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { f.debug_struct("WaitMap") .field("state", &self.state) @@ -230,16 +236,16 @@ impl Debug for WaitMap { #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Wait<'a, K: PartialEq, V> { +pub struct Wait<'a, K: PartialEq, V, Lock: RawScopedMutex = Spinlock> { /// The [`WaitMap`] being waited on from. - queue: &'a WaitMap, + queue: &'a WaitMap, /// Entry in the wait queue linked list. #[pin] waiter: Waiter, } -impl<'map, 'wait, K: PartialEq, V> Wait<'map, K, V> { +impl<'map, 'wait, K: PartialEq, V, Lock: RawScopedMutex> Wait<'map, K, V, Lock> { /// Returns a future that completes when the `Wait` item has been /// added to the [`WaitMap`], and is ready to receive data /// @@ -285,7 +291,7 @@ impl<'map, 'wait, K: PartialEq, V> Wait<'map, K, V> { /// /// assert!(matches!(q.wake(&0, 100), WakeOutcome::Woke)); /// ``` - pub fn subscribe(self: Pin<&'wait mut Self>) -> Subscribe<'wait, 'map, K, V> { + pub fn subscribe(self: Pin<&'wait mut Self>) -> Subscribe<'wait, 'map, K, V, Lock> { Subscribe { wait: self } } @@ -295,7 +301,7 @@ impl<'map, 'wait, K: PartialEq, V> Wait<'map, K, V> { note = "renamed to `subscribe` for consistency, use that instead" )] #[allow(deprecated)] // let us use the deprecated type alias - pub fn enqueue(self: Pin<&'wait mut Self>) -> EnqueueWait<'wait, 'map, K, V> { + pub fn enqueue(self: Pin<&'wait mut Self>) -> EnqueueWait<'wait, 'map, K, V, Lock> { self.subscribe() } } @@ -455,6 +461,30 @@ impl WaitMap { queue: Mutex::new(List::new()), } } +} + +#[cfg(all(feature = "lock_api", not(loom)))] +impl WaitMap +where + Lock: lock_api::RawMutex, + K: PartialEq, +{ + #[must_use] + pub const fn with_lock_api() -> Self { + Self::with_raw_mutex(Lock::INIT) + } +} + +impl WaitMap { + loom_const_fn! { + #[must_use] + pub fn with_raw_mutex(lock: Lock) -> Self { + Self { + state: CachePadded::new(AtomicUsize::new(State::Empty.into_usize())), + queue: Mutex::with_raw_mutex(List::new(), lock), + } + } + } /// Wake a certain task in the queue. /// @@ -483,21 +513,22 @@ impl WaitMap { // okay, there are tasks waiting on the queue; we must acquire the lock // on the linked list and wake the next task from the queue. - let mut queue = self.queue.lock(); - test_debug!("wake: -> locked"); - - // the queue's state may have changed while we were waiting to acquire - // the lock, so we need to acquire a new snapshot. - state = self.load(); - - if let Some(node) = self.node_match_locked(key, &mut *queue, state) { - let waker = Waiter::::wake(node, &mut *queue, Wakeup::DataReceived(val)); - drop(queue); - waker.wake(); - WakeOutcome::Woke - } else { - WakeOutcome::NoMatch(val) - } + self.queue.with(|queue| { + test_debug!("wake: -> locked"); + + // the queue's state may have changed while we were waiting to acquire + // the lock, so we need to acquire a new snapshot. + state = self.load(); + + if let Some(node) = self.node_match_locked(key, &mut *queue, state) { + let waker = Waiter::::wake(node, &mut *queue, Wakeup::DataReceived(val)); + drop(queue); + waker.wake(); + WakeOutcome::Woke + } else { + WakeOutcome::NoMatch(val) + } + }) } /// Returns `true` if this `WaitMap` is [closed](Self::close). @@ -523,26 +554,21 @@ impl WaitMap { return; } - let mut queue = self.queue.lock(); let mut batch = WakeBatch::new(); - while let Some(node) = queue.pop_back() { - let waker = Waiter::wake(node, &mut queue, Wakeup::Closed); - if batch.add_waker(waker) { - // there's still room in the wake set, just keep adding to it. - continue; - } - - // wake set is full, drop the lock and wake everyone! - drop(queue); + let mut waiters_remaining = true; + while waiters_remaining { + waiters_remaining = self.queue.with(|waiters| { + while let Some(node) = waiters.pop_back() { + let waker = Waiter::wake(node, waiters, Wakeup::Closed); + if !batch.add_waker(waker) { + // there's still room in the wake set, just keep adding to it. + return true; + } + } + false + }); batch.wake_all(); - - // reacquire the lock and continue waking - queue = self.queue.lock(); } - - // drop the lock and wake the final batch of waiters in the `WakeBatch`. - drop(queue); - batch.wake_all(); } /// Wait to be woken up by this queue. @@ -555,7 +581,7 @@ impl WaitMap { /// `WaitMap`, the future will resolve to an Error the first time it is polled /// /// [`wake`]: Self::wake - pub fn wait(&self, key: K) -> Wait<'_, K, V> { + pub fn wait(&self, key: K) -> Wait<'_, K, V, Lock> { Wait { queue: self, waiter: self.waiter(key), @@ -675,11 +701,19 @@ feature! { /// See [`Wait::subscribe`] for more information and usage example. #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] #[derive(Debug)] -pub struct Subscribe<'a, 'b, K: PartialEq, V> { - wait: Pin<&'a mut Wait<'b, K, V>>, +pub struct Subscribe<'a, 'b, K, V, Lock = Spinlock> +where + K: PartialEq, + Lock: RawScopedMutex, +{ + wait: Pin<&'a mut Wait<'b, K, V, Lock>>, } -impl<'a, 'b, K: PartialEq, V> Future for Subscribe<'a, 'b, K, V> { +impl<'a, 'b, K, V, Lock> Future for Subscribe<'a, 'b, K, V, Lock> +where + K: PartialEq, + Lock: RawScopedMutex, +{ type Output = WaitResult<()>; fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -698,7 +732,7 @@ impl<'a, 'b, K: PartialEq, V> Future for Subscribe<'a, 'b, K, V> { since = "0.1.3", note = "renamed to `Subscribe` for consistency, use that instead" )] -pub type EnqueueWait<'a, 'b, K, V> = Subscribe<'a, 'b, K, V>; +pub type EnqueueWait<'a, 'b, K, V, Lock> = Subscribe<'a, 'b, K, V, Lock>; impl Waiter { /// Wake the task that owns this `Waiter`. @@ -748,11 +782,14 @@ impl Waiter { /// Moves a `Wait` from the `Start` condition. /// /// Caller MUST ensure the `Wait` is in the start condition before calling. - fn start_to_wait( + fn start_to_wait( mut self: Pin<&mut Self>, - queue: &WaitMap, + queue: &WaitMap, cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll> + where + Lock: RawScopedMutex, + { let mut this = self.as_mut().project(); debug_assert!( @@ -762,54 +799,58 @@ impl Waiter { // Try to wait... test_debug!("poll_wait: locking..."); - let mut waiters = queue.queue.lock(); - test_debug!("poll_wait: -> locked"); - let mut queue_state = queue.load(); - - // transition the queue to the waiting state - 'to_waiting: loop { - match test_dbg!(queue_state) { - // the queue is `Empty`, transition to `Waiting` - State::Empty => match queue.compare_exchange(queue_state, State::Waiting) { - Ok(_) => break 'to_waiting, - Err(actual) => queue_state = actual, - }, - // the queue is already `Waiting` - State::Waiting => break 'to_waiting, - State::Closed => return closed(), + queue.queue.with(|waiters| { + test_debug!("poll_wait: -> locked"); + let mut queue_state = queue.load(); + + // transition the queue to the waiting state + 'to_waiting: loop { + match test_dbg!(queue_state) { + // the queue is `Empty`, transition to `Waiting` + State::Empty => match queue.compare_exchange(queue_state, State::Waiting) { + Ok(_) => break 'to_waiting, + Err(actual) => queue_state = actual, + }, + // the queue is already `Waiting` + State::Waiting => break 'to_waiting, + State::Closed => return closed(), + } } - } - - // Check if key already exists - // - // Note: It's okay not to re-update the state here, if we were empty - // this check will never trigger, if we are already waiting, we should - // still be waiting. - let mut cursor = waiters.cursor_front_mut(); - if cursor.any(|n| &n.key == this.key) { - return duplicate(); - } - // enqueue the node - *this.state = WaitState::Waiting; - this.node.as_mut().with_mut(|node| { - unsafe { - // safety: we may mutate the node because we are - // holding the lock. - (*node).waker = Wakeup::Waiting(cx.waker().clone()); + // Check if key already exists + // + // Note: It's okay not to re-update the state here, if we were empty + // this check will never trigger, if we are already waiting, we should + // still be waiting. + let mut cursor = waiters.cursor_front_mut(); + if cursor.any(|n| &n.key == this.key) { + return duplicate(); } - }); - let ptr = unsafe { NonNull::from(Pin::into_inner_unchecked(self)) }; - waiters.push_front(ptr); - Poll::Ready(Ok(())) + // enqueue the node + *this.state = WaitState::Waiting; + this.node.as_mut().with_mut(|node| { + unsafe { + // safety: we may mutate the node because we are + // holding the lock. + (*node).waker = Wakeup::Waiting(cx.waker().clone()); + } + }); + let ptr = unsafe { NonNull::from(Pin::into_inner_unchecked(self)) }; + waiters.push_front(ptr); + + Poll::Ready(Ok(())) + }) } - fn poll_wait( + fn poll_wait( mut self: Pin<&mut Self>, - queue: &WaitMap, + queue: &WaitMap, cx: &mut Context<'_>, - ) -> Poll> { + ) -> Poll> + where + Lock: RawScopedMutex, + { test_debug!(ptr = ?fmt::ptr(self.as_mut()), "Waiter::poll_wait"); let this = self.as_mut().project(); @@ -819,45 +860,49 @@ impl Waiter { Poll::Pending } WaitState::Waiting => { - let mut _waiters = queue.queue.lock(); - this.node.with_mut(|node| unsafe { - // safety: we may mutate the node because we are - // holding the lock. - let node = &mut *node; - let result; - node.waker = match mem::replace(&mut node.waker, Wakeup::Empty) { - // We already had a waker, but are now getting another one. - // Store the new one, droping the old one - Wakeup::Waiting(waker) => { - result = Poll::Pending; - if !waker.will_wake(cx.waker()) { - Wakeup::Waiting(cx.waker().clone()) - } else { - Wakeup::Waiting(waker) + // We must lock the linked list in order to safely mutate our node in + // the list. We don't actually need the mutable reference to the + // queue here, though. + queue.queue.with(|_waiters| { + this.node.with_mut(|node| unsafe { + // safety: we may mutate the node because we are + // holding the lock. + let node = &mut *node; + let result; + node.waker = match mem::replace(&mut node.waker, Wakeup::Empty) { + // We already had a waker, but are now getting another one. + // Store the new one, droping the old one + Wakeup::Waiting(waker) => { + result = Poll::Pending; + if !waker.will_wake(cx.waker()) { + Wakeup::Waiting(cx.waker().clone()) + } else { + Wakeup::Waiting(waker) + } + } + // We have received the data, take the data out of the + // future, and provide it to the poller + Wakeup::DataReceived(val) => { + result = notified(val); + Wakeup::Retreived } - } - // We have received the data, take the data out of the - // future, and provide it to the poller - Wakeup::DataReceived(val) => { - result = notified(val); - Wakeup::Retreived - } - Wakeup::Retreived => { - result = consumed(); - Wakeup::Retreived - } - - Wakeup::Closed => { - *this.state = WaitState::Completed; - result = closed(); - Wakeup::Closed - } - Wakeup::Empty => { - result = never_added(); - Wakeup::Closed - } - }; - result + Wakeup::Retreived => { + result = consumed(); + Wakeup::Retreived + } + + Wakeup::Closed => { + *this.state = WaitState::Completed; + result = closed(); + Wakeup::Closed + } + Wakeup::Empty => { + result = never_added(); + Wakeup::Closed + } + }; + result + }) }) } WaitState::Completed => consumed(), @@ -868,7 +913,10 @@ impl Waiter { /// /// This is called from the `drop` implementation for the [`Wait`] and /// [`WaitOwned`] futures. - fn release(mut self: Pin<&mut Self>, queue: &WaitMap) { + fn release(mut self: Pin<&mut Self>, queue: &WaitMap) + where + Lock: RawScopedMutex, + { let state = *(self.as_mut().project().state); let ptr = NonNull::from(unsafe { Pin::into_inner_unchecked(self) }); test_debug!(self = ?fmt::ptr(ptr), ?state, ?queue, "Waiter::release"); @@ -878,20 +926,21 @@ impl Waiter { return; } - let mut waiters: MutexGuard>> = queue.queue.lock(); - let state = queue.load(); + queue.queue.with(|waiters| { + let state = queue.load(); - // remove the node - unsafe { - // safety: we have the lock on the queue, so this is safe. - waiters.remove(ptr); - }; - - // if we removed the last waiter from the queue, transition the state to - // `Empty`. - if test_dbg!(waiters.is_empty()) && state == State::Waiting { - queue.store(State::Empty); - } + // remove the node + unsafe { + // safety: we have the lock on the queue, so this is safe. + waiters.remove(ptr); + }; + + // if we removed the last waiter from the queue, transition the state to + // `Empty`. + if test_dbg!(waiters.is_empty()) && state == State::Waiting { + queue.store(State::Empty); + } + }) } } @@ -932,7 +981,11 @@ impl Future for Wait<'_, K, V> { } #[pinned_drop] -impl PinnedDrop for Wait<'_, K, V> { +impl PinnedDrop for Wait<'_, K, V, Lock> +where + K: PartialEq, + Lock: RawScopedMutex, +{ fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(this.queue); @@ -1006,16 +1059,16 @@ feature! { /// assert_unpin::>(); #[derive(Debug)] #[pin_project(PinnedDrop)] - pub struct WaitOwned { + pub struct WaitOwned { /// The `WaitMap` being waited on. - queue: Arc>, + queue: Arc>, /// Entry in the wait queue. #[pin] waiter: Waiter, } - impl WaitMap { + impl WaitMap { /// Wait to be woken up by this queue, returning a future that's valid /// for the `'static` lifetime. /// @@ -1032,14 +1085,14 @@ feature! { /// /// [`wake`]: Self::wake /// [`wait`]: Self::wait - pub fn wait_owned(self: &Arc, key: K) -> WaitOwned { + pub fn wait_owned(self: &Arc, key: K) -> WaitOwned { let waiter = self.waiter(key); let queue = self.clone(); WaitOwned { queue, waiter } } } - impl Future for WaitOwned { + impl Future for WaitOwned { type Output = WaitResult; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1049,7 +1102,11 @@ feature! { } #[pinned_drop] - impl PinnedDrop for WaitOwned { + impl PinnedDrop for WaitOwned + where + K: PartialEq, + Lock: RawScopedMutex, + { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(&*this.queue); diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index a3595b77..feeddaf0 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -379,7 +379,15 @@ impl WaitQueue { } #[cfg(all(feature = "lock_api", not(loom)))] -impl WaitQueue where Lock: lock_api::RawMutex {} +impl WaitQueue +where + Lock: lock_api::RawMutex, +{ + #[must_use] + pub const fn with_lock_api() -> Self { + Self::make(State::Empty, Lock::INIT) + } +} impl WaitQueue where From 86a933cfff9538ac7d9a0e4a32ef5e5058363e14 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 20 Jul 2024 13:42:05 -0700 Subject: [PATCH 13/41] rename closure lock methods to `with_lock` --- maitake-sync/src/blocking/mutex.rs | 18 +++++++++--------- maitake-sync/src/wait_map.rs | 10 +++++----- maitake-sync/src/wait_queue.rs | 14 +++++++------- 3 files changed, 21 insertions(+), 21 deletions(-) diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index eb453386..479a57f7 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -45,9 +45,9 @@ pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { } pub unsafe trait RawScopedMutex { - fn with(&self, f: impl FnOnce() -> T) -> T; + fn with_lock(&self, f: impl FnOnce() -> T) -> T; - fn try_with(&self, f: impl FnOnce() -> T) -> Option; + fn try_with_lock(&self, f: impl FnOnce() -> T) -> Option; /// Returns `true` if the mutex is currently locked. fn is_locked(&self) -> bool; @@ -104,7 +104,7 @@ unsafe impl RawScopedMutex for L where L: RawMutex, { - fn with(&self, f: impl FnOnce() -> T) -> T { + fn with_lock(&self, f: impl FnOnce() -> T) -> T { // TODO(eliza): RAIIify self.lock(); let ret = f(); @@ -112,7 +112,7 @@ where ret } - fn try_with(&self, f: impl FnOnce() -> T) -> Option { + fn try_with_lock(&self, f: impl FnOnce() -> T) -> Option { if !self.try_lock() { return None; } @@ -228,8 +228,8 @@ where } impl Mutex { - pub fn with(&self, f: impl FnOnce(&mut T) -> U) -> U { - self.lock.with(|| { + pub fn with_lock(&self, f: impl FnOnce(&mut T) -> U) -> U { + self.lock.with_lock(|| { self.data.with_mut(|data| unsafe { // Safety: we just locked the mutex. f(&mut *data) @@ -237,8 +237,8 @@ impl Mutex { }) } - pub fn try_with(&self, f: impl FnOnce(&mut T) -> U) -> Option { - self.lock.try_with(|| { + pub fn try_with_lock(&self, f: impl FnOnce(&mut T) -> U) -> Option { + self.lock.try_with_lock(|| { self.data.with_mut(|data| unsafe { // Safety: we just locked the mutex. f(&mut *data) @@ -349,7 +349,7 @@ where Lock: RawScopedMutex + fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.try_with(|data| { + self.try_with_lock(|data| { f.debug_struct("Mutex") .field("data", data) .field("lock", &self.lock) diff --git a/maitake-sync/src/wait_map.rs b/maitake-sync/src/wait_map.rs index 8833d641..9c844501 100644 --- a/maitake-sync/src/wait_map.rs +++ b/maitake-sync/src/wait_map.rs @@ -513,7 +513,7 @@ impl WaitMap { // okay, there are tasks waiting on the queue; we must acquire the lock // on the linked list and wake the next task from the queue. - self.queue.with(|queue| { + self.queue.with_lock(|queue| { test_debug!("wake: -> locked"); // the queue's state may have changed while we were waiting to acquire @@ -557,7 +557,7 @@ impl WaitMap { let mut batch = WakeBatch::new(); let mut waiters_remaining = true; while waiters_remaining { - waiters_remaining = self.queue.with(|waiters| { + waiters_remaining = self.queue.with_lock(|waiters| { while let Some(node) = waiters.pop_back() { let waker = Waiter::wake(node, waiters, Wakeup::Closed); if !batch.add_waker(waker) { @@ -799,7 +799,7 @@ impl Waiter { // Try to wait... test_debug!("poll_wait: locking..."); - queue.queue.with(|waiters| { + queue.queue.with_lock(|waiters| { test_debug!("poll_wait: -> locked"); let mut queue_state = queue.load(); @@ -863,7 +863,7 @@ impl Waiter { // We must lock the linked list in order to safely mutate our node in // the list. We don't actually need the mutable reference to the // queue here, though. - queue.queue.with(|_waiters| { + queue.queue.with_lock(|_waiters| { this.node.with_mut(|node| unsafe { // safety: we may mutate the node because we are // holding the lock. @@ -926,7 +926,7 @@ impl Waiter { return; } - queue.queue.with(|waiters| { + queue.queue.with_lock(|waiters| { let state = queue.load(); // remove the node diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index feeddaf0..dc211b42 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -479,7 +479,7 @@ where // okay, there are tasks waiting on the queue; we must acquire the lock // on the linked list and wake the next task from the queue. - let waker = self.queue.with(|queue| { + let waker = self.queue.with_lock(|queue| { test_debug!("wake: -> locked"); // the queue's state may have changed while we were waiting to acquire @@ -565,7 +565,7 @@ where // out...but we can't `return` from the outer function inside the lock // closure. Therefore, we just return a `bool` and, if it's `true`, // return instead of doing more work. - let done = self.queue.with(|queue| { + let done = self.queue.with_lock(|queue| { let state = self.load(); match test_dbg!(state.get(QueueState::STATE)) { @@ -604,7 +604,7 @@ where // As long as there are waiters remaining to wake, lock the queue, drain // another batch, release the lock, and wake them. while waiters_remaining { - self.queue.with(|queue| { + self.queue.with_lock(|queue| { waiters_remaining = Self::drain_to_wake_batch(&mut batch, queue, Wakeup::All); }); batch.wake_all(); @@ -633,7 +633,7 @@ where while waking { waking = self .queue - .with(|queue| Self::drain_to_wake_batch(&mut batch, queue, Wakeup::Closed)); + .with_lock(|queue| Self::drain_to_wake_batch(&mut batch, queue, Wakeup::Closed)); batch.wake_all(); } } @@ -1130,7 +1130,7 @@ impl Waiter { // okay, no pending wakeups. try to wait... test_debug!("poll_wait: locking..."); - queue.queue.with(move |waiters| { + queue.queue.with_lock(move |waiters| { test_debug!("poll_wait: -> locked"); let mut queue_state = queue.load(); @@ -1194,7 +1194,7 @@ impl Waiter { }) } WaitState::Waiting => { - queue.queue.with(|_waiters| { + queue.queue.with_lock(|_waiters| { this.node.with_mut(|node| unsafe { // safety: we may mutate the node because we are // holding the lock. @@ -1249,7 +1249,7 @@ impl Waiter { return; } - let next_waiter = queue.queue.with(|waiters| { + let next_waiter = queue.queue.with_lock(|waiters| { let state = queue.load(); // remove the node unsafe { From b3346c4bdf4479b7559ddf1ac5e9c29f1f344dd5 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 22 Jul 2024 08:42:19 -0700 Subject: [PATCH 14/41] start on `ScopedMutexTraits` integration --- Cargo.lock | 6 +++ Cargo.toml | 1 + maitake-sync/Cargo.toml | 1 + maitake-sync/src/blocking/mutex.rs | 76 +++++++++++++++--------------- maitake-sync/src/mutex.rs | 52 ++++++++++---------- maitake-sync/src/spin.rs | 52 ++++++++++++++++++++ maitake-sync/src/wait_map.rs | 57 +++++++++++----------- maitake-sync/src/wait_queue.rs | 28 +++++------ 8 files changed, 167 insertions(+), 106 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5cda26da..cd169dc2 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1456,6 +1456,7 @@ dependencies = [ "pin-project", "portable-atomic", "proptest", + "scoped-mutex-traits", "tokio", "tokio-test", "tracing 0.1.40", @@ -2205,6 +2206,11 @@ version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" +[[package]] +name = "scoped-mutex-traits" +version = "0.1.0" +source = "git+https://github.com/tosc-rs/scoped-mutex#13eebe716e9b5945c21fa652ed3adfc7d6ca01ac" + [[package]] name = "scoped-tls" version = "1.0.1" diff --git a/Cargo.toml b/Cargo.toml index f1ca61c1..81f34436 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -98,6 +98,7 @@ dynamic-range-start = "0xFFFF_8000_0000_0000" # use `tracing` 0.2 from git tracing = { git = "https://github.com/tokio-rs/tracing" } tracing-core = { git = "https://github.com/tokio-rs/tracing" } +scoped-mutex-traits = { git = "https://github.com/tosc-rs/scoped-mutex" } # Custom profile for Loom tests: enable release optimizations so that the loom # tests are less slow, but don't disable debug assertions. diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index c11ac8d8..412c3443 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -29,6 +29,7 @@ core-error = [] [dependencies] mycelium-bitfield = { version = "0.1.3", path = "../bitfield" } cordyceps = { version = "0.3.0", path = "../cordyceps" } +scoped-mutex-traits = "0.1.0" lock_api = { version = "0.4", default-features = false, optional = true } pin-project = "1" portable-atomic = "1.2" diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 479a57f7..b4fb4235 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -4,6 +4,7 @@ use crate::{ util::fmt, }; use core::ops::{Deref, DerefMut}; +pub use scoped_mutex_traits::ScopedRawMutex; /// A blocking mutual exclusion lock for protecting shared data. /// Each mutex has a type parameter which represents @@ -44,15 +45,6 @@ pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { lock: &'a Lock, } -pub unsafe trait RawScopedMutex { - fn with_lock(&self, f: impl FnOnce() -> T) -> T; - - fn try_with_lock(&self, f: impl FnOnce() -> T) -> Option; - - /// Returns `true` if the mutex is currently locked. - fn is_locked(&self) -> bool; -} - /// Trait abstracting over blocking [`Mutex`] implementations (`maitake-sync`'s /// version). /// @@ -100,33 +92,6 @@ pub unsafe trait RawMutex { fn is_locked(&self) -> bool; } -unsafe impl RawScopedMutex for L -where - L: RawMutex, -{ - fn with_lock(&self, f: impl FnOnce() -> T) -> T { - // TODO(eliza): RAIIify - self.lock(); - let ret = f(); - unsafe { self.unlock() } - ret - } - - fn try_with_lock(&self, f: impl FnOnce() -> T) -> Option { - if !self.try_lock() { - return None; - } - let ret = f(); - unsafe { self.unlock() } - Some(ret) - } - - /// Returns `true` if the mutex is currently locked. - fn is_locked(&self) -> bool { - RawMutex::is_locked(self) - } -} - #[cfg(feature = "lock_api")] unsafe impl RawMutex for T { type GuardMarker = ::GuardMarker; @@ -227,7 +192,21 @@ where } } -impl Mutex { +impl Mutex { + /// Lock this `Mutex`, blocking if it is not currently unlocked, and call + /// `f()` with the locked data once the lock is acquired. + /// + /// When the `Mutex` is unlocked, this method locks it, calls `f()` with the + /// data protected by the `Mutex`, and then unlocks the `Mutex` and returns + /// the result of `f()`. If the `Mutex` is locked, this method blocks until + /// it is unlocked, and then takes the lock. + /// + /// To return immediately rather than blocking, use [`Mutex::try_with_lock`] + /// instead. + /// + /// This method is available when the `Mutex`'s `Lock` type parameter + /// implements the [`ScopedRawMutex`] trait. + #[track_caller] pub fn with_lock(&self, f: impl FnOnce(&mut T) -> U) -> U { self.lock.with_lock(|| { self.data.with_mut(|data| unsafe { @@ -237,6 +216,27 @@ impl Mutex { }) } + /// Attempt to lock this `Mutex` without blocking and call `f()` with the + /// locked data if the lock is acquired. + /// + /// If the `Mutex` is unlocked, this method locks it, calls `f()` with the + /// data protected by the `Mutex`, and then unlocks the `Mutex` and returns + /// [`Some`]`(U)`. Otherwise, if the lock is already held, this method + /// returns `None` immediately, without blocking. + /// + /// To block until the `Mutex` is unlocked instead of returning `None`, use + /// [`Mutex::with_lock`] instead. + /// + /// This method is available when the `Mutex`'s `Lock` type parameter + /// implements the [`ScopedRawMutex`] trait. + /// + /// # Returns + /// + /// - [`Some`]`(U)` if the lock was acquired, containing the result of + /// `f()`. + /// - [`None`] if the lock is currently held and could not be acquired + /// without blocking. + #[track_caller] pub fn try_with_lock(&self, f: impl FnOnce(&mut T) -> U) -> Option { self.lock.try_with_lock(|| { self.data.with_mut(|data| unsafe { @@ -346,7 +346,7 @@ impl Default for Mutex { impl fmt::Debug for Mutex where T: fmt::Debug, - Lock: RawScopedMutex + fmt::Debug, + Lock: ScopedRawMutex + fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.try_with_lock(|data| { diff --git a/maitake-sync/src/mutex.rs b/maitake-sync/src/mutex.rs index fe8859c9..55f221c1 100644 --- a/maitake-sync/src/mutex.rs +++ b/maitake-sync/src/mutex.rs @@ -4,7 +4,7 @@ //! //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion use crate::{ - blocking::RawScopedMutex, + blocking::ScopedRawMutex, loom::cell::{MutPtr, UnsafeCell}, spin::Spinlock, util::fmt, @@ -86,7 +86,7 @@ mod tests; /// [storage]: https://mycelium.elizas.website/maitake/task/trait.Storage.html /// [no-unwinding]: https://mycelium.elizas.website/maitake/index.html#maitake-does-not-support-unwinding -pub struct Mutex { +pub struct Mutex { wait: WaitQueue, data: UnsafeCell, } @@ -107,7 +107,7 @@ pub struct Mutex { /// [`try_lock`]: Mutex::try_lock /// [RAII]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[must_use = "if unused, the `Mutex` will immediately unlock"] -pub struct MutexGuard<'a, T: ?Sized, L: RawScopedMutex = Spinlock> { +pub struct MutexGuard<'a, T: ?Sized, L: ScopedRawMutex = Spinlock> { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important. @@ -136,7 +136,7 @@ pub struct MutexGuard<'a, T: ?Sized, L: RawScopedMutex = Spinlock> { #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] #[pin_project] #[derive(Debug)] -pub struct Lock<'a, T: ?Sized, L: RawScopedMutex = Spinlock> { +pub struct Lock<'a, T: ?Sized, L: ScopedRawMutex = Spinlock> { #[pin] wait: wait_queue::Wait<'a, L>, mutex: &'a Mutex, @@ -144,7 +144,7 @@ pub struct Lock<'a, T: ?Sized, L: RawScopedMutex = Spinlock> { /// This is used in order to ensure that the wakeup is performed only *after* /// the data ptr is dropped, in order to keep `loom` happy. -struct WakeOnDrop<'a, T: ?Sized, L: RawScopedMutex>(&'a Mutex); +struct WakeOnDrop<'a, T: ?Sized, L: ScopedRawMutex>(&'a Mutex); // === impl Mutex === @@ -182,7 +182,7 @@ impl Mutex { } } -impl Mutex { +impl Mutex { /// Consumes this `Mutex`, returning the guarded data. #[inline] #[must_use] @@ -191,7 +191,7 @@ impl Mutex { } } -impl Mutex { +impl Mutex { /// Locks this mutex. /// /// This returns a [`Lock`] future that will wait until no other task is @@ -296,7 +296,7 @@ impl Default for Mutex { impl fmt::Debug for Mutex where T: ?Sized + fmt::Debug, - L: RawScopedMutex + fmt::Debug, + L: ScopedRawMutex + fmt::Debug, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let Self { data: _, wait } = self; @@ -307,13 +307,13 @@ where } } -unsafe impl Send for Mutex +unsafe impl Send for Mutex where T: ?Sized + Send, L: Send, { } -unsafe impl Sync for Mutex +unsafe impl Sync for Mutex where T: ?Sized + Send, L: Sync, @@ -349,7 +349,7 @@ impl<'a, T> Future for Lock<'a, T> { impl Deref for MutexGuard<'_, T, L> where T: ?Sized, - L: RawScopedMutex, + L: ScopedRawMutex, { type Target = T; @@ -365,7 +365,7 @@ where impl DerefMut for MutexGuard<'_, T, L> where T: ?Sized, - L: RawScopedMutex, + L: ScopedRawMutex, { #[inline] fn deref_mut(&mut self) -> &mut Self::Target { @@ -379,7 +379,7 @@ where impl fmt::Debug for MutexGuard<'_, T, L> where T: ?Sized + fmt::Debug, - L: RawScopedMutex, + L: ScopedRawMutex, { #[inline] fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { @@ -390,18 +390,18 @@ where unsafe impl Send for MutexGuard<'_, T, L> where T: ?Sized + Send, - L: RawScopedMutex + Sync, + L: ScopedRawMutex + Sync, { } unsafe impl Sync for MutexGuard<'_, T, L> where T: ?Sized + Send + Sync, - // A `MutexGuard`` has a reference to a `L`-typed RawScopedMutex in it, so `` - L: RawScopedMutex + Sync, + // A `MutexGuard`` has a reference to a `L`-typed ScopedRawMutex in it, so `` + L: ScopedRawMutex + Sync, { } -impl Drop for WakeOnDrop<'_, T, L> { +impl Drop for WakeOnDrop<'_, T, L> { fn drop(&mut self) { self.0.wait.wake() } @@ -434,7 +434,7 @@ feature! { /// [`try_lock_owned`]: Mutex::try_lock_owned /// [RAII]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[must_use = "if unused, the Mutex will immediately unlock"] - pub struct OwnedMutexGuard { + pub struct OwnedMutexGuard { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important. @@ -442,7 +442,7 @@ feature! { _wake: WakeArcOnDrop, } - impl Mutex { + impl Mutex { /// Locks this mutex, returning an [owned RAII guard][`OwnedMutexGuard`]. /// @@ -550,14 +550,14 @@ feature! { } } - struct WakeArcOnDrop(Arc>); + struct WakeArcOnDrop(Arc>); // === impl OwnedMutexGuard === impl Deref for OwnedMutexGuard where T: ?Sized, - L: RawScopedMutex, + L: ScopedRawMutex, { type Target = T; @@ -573,7 +573,7 @@ feature! { impl DerefMut for OwnedMutexGuard where T: ?Sized, - L: RawScopedMutex, + L: ScopedRawMutex, { #[inline] fn deref_mut(&mut self) -> &mut Self::Target { @@ -587,7 +587,7 @@ feature! { impl fmt::Debug for OwnedMutexGuard where T: ?Sized + fmt::Debug, - L: RawScopedMutex, + L: ScopedRawMutex, { #[inline] fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { @@ -598,20 +598,20 @@ feature! { unsafe impl Send for OwnedMutexGuard where T: ?Sized + Send, - L: RawScopedMutex + Sync, + L: ScopedRawMutex + Sync, { } unsafe impl Sync for OwnedMutexGuard where T: ?Sized + Send + Sync, - L: RawScopedMutex + Sync, + L: ScopedRawMutex + Sync, { } impl Drop for WakeArcOnDrop where T: ?Sized, - L: RawScopedMutex, + L: ScopedRawMutex, { fn drop(&mut self) { self.0.wait.wake() diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index d7d5cf25..b1862970 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -67,6 +67,28 @@ impl Spinlock { fn is_locked(&self) -> bool { self.locked.load(Relaxed) } + + /// Returns a RAII guard that unlocks this spinlock. This is like a simpler + /// version of a real life `MutexGuard` that ensures a `ScopedRawMutex` impl + /// is unlocked in the event of a panic + /// + /// # Safety + /// + /// The spinlock must be locked. + unsafe fn unlock_on_drop(&self) -> impl Drop + '_ { + struct Unlock<'lock>(&'lock Spinlock); + impl Drop for Unlock<'_> { + #[inline(always)] + #[track_caller] + fn drop(&mut self) { + debug_assert!(self.is_locked()); + unsafe { self.0.unlock() } + } + } + + debug_assert!(self.is_locked()); + Unlock(self) + } } impl Default for Spinlock { @@ -75,6 +97,36 @@ impl Default for Spinlock { } } +unsafe impl ScopedRawMutex for Spinlock { + #[inline] + fn with_lock(&self, f: impl FnOnce() -> R) -> R { + self.lock(); + let _unlock = unsafe { + // Safety: we just took the lock. + self.unlock_on_drop() + }; + f() + } + + #[inline] + fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { + if self.try_lock() { + let _unlock = unsafe { + // Safety: we just took the lock. + self.unlock_on_drop() + }; + Some(f()) + } else { + None + } + } + + #[inline] + fn is_locked(&self) -> bool { + Spinlock::is_locked(self) + } +} + unsafe impl RawMutex for Spinlock { type GuardMarker = (); diff --git a/maitake-sync/src/wait_map.rs b/maitake-sync/src/wait_map.rs index 9c844501..00fac0e3 100644 --- a/maitake-sync/src/wait_map.rs +++ b/maitake-sync/src/wait_map.rs @@ -3,7 +3,7 @@ //! //! See the documentation for the [`WaitMap`] type for details. use crate::{ - blocking::RawScopedMutex, + blocking::ScopedRawMutex, loom::{ cell::UnsafeCell, sync::{ @@ -178,7 +178,7 @@ const fn notified(data: T) -> Poll> { /// [ilist]: cordyceps::List /// [intrusive]: https://fuchsia.dev/fuchsia-src/development/languages/c-cpp/fbl_containers_guide/introduction /// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue -pub struct WaitMap { +pub struct WaitMap { /// The wait queue's state variable. state: CachePadded, @@ -204,7 +204,7 @@ pub struct WaitMap { impl Debug for WaitMap where K: PartialEq, - Lock: RawScopedMutex + fmt::Debug, + Lock: ScopedRawMutex + fmt::Debug, { fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { f.debug_struct("WaitMap") @@ -236,7 +236,7 @@ where #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Wait<'a, K: PartialEq, V, Lock: RawScopedMutex = Spinlock> { +pub struct Wait<'a, K: PartialEq, V, Lock: ScopedRawMutex = Spinlock> { /// The [`WaitMap`] being waited on from. queue: &'a WaitMap, @@ -245,7 +245,7 @@ pub struct Wait<'a, K: PartialEq, V, Lock: RawScopedMutex = Spinlock> { waiter: Waiter, } -impl<'map, 'wait, K: PartialEq, V, Lock: RawScopedMutex> Wait<'map, K, V, Lock> { +impl<'map, 'wait, K: PartialEq, V, Lock: ScopedRawMutex> Wait<'map, K, V, Lock> { /// Returns a future that completes when the `Wait` item has been /// added to the [`WaitMap`], and is ready to receive data /// @@ -463,19 +463,20 @@ impl WaitMap { } } -#[cfg(all(feature = "lock_api", not(loom)))] -impl WaitMap -where - Lock: lock_api::RawMutex, - K: PartialEq, -{ - #[must_use] - pub const fn with_lock_api() -> Self { - Self::with_raw_mutex(Lock::INIT) - } -} - -impl WaitMap { +// TODO(eliza): figure out what to do about `lock_api` not impling `RawScopedMutex`... +// #[cfg(all(feature = "lock_api", not(loom)))] +// impl WaitMap +// where +// Lock: lock_api::RawMutex, +// K: PartialEq, +// { +// #[must_use] +// pub const fn with_lock_api() -> Self { +// Self::with_raw_mutex(Lock::INIT) +// } +// } + +impl WaitMap { loom_const_fn! { #[must_use] pub fn with_raw_mutex(lock: Lock) -> Self { @@ -704,7 +705,7 @@ feature! { pub struct Subscribe<'a, 'b, K, V, Lock = Spinlock> where K: PartialEq, - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { wait: Pin<&'a mut Wait<'b, K, V, Lock>>, } @@ -712,7 +713,7 @@ where impl<'a, 'b, K, V, Lock> Future for Subscribe<'a, 'b, K, V, Lock> where K: PartialEq, - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { type Output = WaitResult<()>; @@ -788,7 +789,7 @@ impl Waiter { cx: &mut Context<'_>, ) -> Poll> where - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { let mut this = self.as_mut().project(); @@ -849,7 +850,7 @@ impl Waiter { cx: &mut Context<'_>, ) -> Poll> where - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { test_debug!(ptr = ?fmt::ptr(self.as_mut()), "Waiter::poll_wait"); let this = self.as_mut().project(); @@ -915,7 +916,7 @@ impl Waiter { /// [`WaitOwned`] futures. fn release(mut self: Pin<&mut Self>, queue: &WaitMap) where - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { let state = *(self.as_mut().project().state); let ptr = NonNull::from(unsafe { Pin::into_inner_unchecked(self) }); @@ -984,7 +985,7 @@ impl Future for Wait<'_, K, V> { impl PinnedDrop for Wait<'_, K, V, Lock> where K: PartialEq, - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); @@ -1059,7 +1060,7 @@ feature! { /// assert_unpin::>(); #[derive(Debug)] #[pin_project(PinnedDrop)] - pub struct WaitOwned { + pub struct WaitOwned { /// The `WaitMap` being waited on. queue: Arc>, @@ -1068,7 +1069,7 @@ feature! { waiter: Waiter, } - impl WaitMap { + impl WaitMap { /// Wait to be woken up by this queue, returning a future that's valid /// for the `'static` lifetime. /// @@ -1092,7 +1093,7 @@ feature! { } } - impl Future for WaitOwned { + impl Future for WaitOwned { type Output = WaitResult; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1105,7 +1106,7 @@ feature! { impl PinnedDrop for WaitOwned where K: PartialEq, - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index dc211b42..709fb50d 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -5,7 +5,7 @@ #[cfg(any(test, maitake_ultraverbose))] use crate::util::fmt; use crate::{ - blocking::RawScopedMutex, + blocking::ScopedRawMutex, loom::{ cell::UnsafeCell, sync::{ @@ -177,7 +177,7 @@ mod tests; /// [mutex]: crate::Mutex /// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue #[derive(Debug)] -pub struct WaitQueue { +pub struct WaitQueue { /// The wait queue's state variable. state: CachePadded, @@ -221,7 +221,7 @@ pub struct WaitQueue { #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Wait<'a, Lock: RawScopedMutex = Spinlock> { +pub struct Wait<'a, Lock: ScopedRawMutex = Spinlock> { /// The [`WaitQueue`] being waited on. queue: &'a WaitQueue, @@ -391,7 +391,7 @@ where impl WaitQueue where - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { loom_const_fn! { #[must_use] @@ -1105,7 +1105,7 @@ impl Waiter { waker: Option<&Waker>, ) -> Poll> where - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { test_debug!(ptr = ?fmt::ptr(self.as_mut()), "Waiter::poll_wait"); let ptr = unsafe { NonNull::from(Pin::into_inner_unchecked(self.as_mut())) }; @@ -1238,7 +1238,7 @@ impl Waiter { /// [`WaitOwned`] futures. fn release(mut self: Pin<&mut Self>, queue: &WaitQueue) where - Lock: RawScopedMutex, + Lock: ScopedRawMutex, { let state = *(self.as_mut().project().state); let ptr = NonNull::from(unsafe { Pin::into_inner_unchecked(self) }); @@ -1305,7 +1305,7 @@ unsafe impl Linked> for Waiter { // === impl Wait === -impl Wait<'_, Lock> { +impl Wait<'_, Lock> { /// Returns `true` if this `Wait` future is waiting for a notification from /// the provided [`WaitQueue`]. /// @@ -1417,7 +1417,7 @@ impl Wait<'_, Lock> { } } -impl Future for Wait<'_, Lock> { +impl Future for Wait<'_, Lock> { type Output = WaitResult<()>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1427,7 +1427,7 @@ impl Future for Wait<'_, Lock> { } #[pinned_drop] -impl PinnedDrop for Wait<'_, Lock> { +impl PinnedDrop for Wait<'_, Lock> { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(this.queue); @@ -1503,7 +1503,7 @@ feature! { /// ``` #[derive(Debug)] #[pin_project(PinnedDrop)] - pub struct WaitOwned { + pub struct WaitOwned { /// The `WaitQueue` being waited on. queue: Arc>, @@ -1512,7 +1512,7 @@ feature! { waiter: Waiter, } - impl WaitQueue { + impl WaitQueue { /// Wait to be woken up by this queue, returning a future that's valid /// for the `'static` lifetime. /// @@ -1556,7 +1556,7 @@ feature! { // === impl WaitOwned === - impl WaitOwned { + impl WaitOwned { /// Returns `true` if this `WaitOwned` future is waiting for a /// notification from the provided [`WaitQueue`]. /// @@ -1675,7 +1675,7 @@ feature! { } } - impl Future for WaitOwned { + impl Future for WaitOwned { type Output = WaitResult<()>; fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { @@ -1685,7 +1685,7 @@ feature! { } #[pinned_drop] - impl PinnedDrop for WaitOwned { + impl PinnedDrop for WaitOwned { fn drop(mut self: Pin<&mut Self>) { let this = self.project(); this.waiter.release(&*this.queue); From b8b32a12c6ae46de647cea5cade9555ff94bc288 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Thu, 25 Jul 2024 09:37:38 -0700 Subject: [PATCH 15/41] update everything to `mutex-traits` --- Cargo.lock | 12 +-- Cargo.toml | 2 +- maitake-sync/Cargo.toml | 2 +- maitake-sync/src/blocking.rs | 4 +- maitake-sync/src/blocking/mutex.rs | 116 ++--------------------------- maitake-sync/src/rwlock.rs | 11 --- maitake-sync/src/semaphore.rs | 8 -- maitake-sync/src/spin.rs | 56 +------------- maitake-sync/src/wait_map.rs | 87 ++++++++++++---------- maitake-sync/src/wait_queue.rs | 26 +++---- 10 files changed, 81 insertions(+), 243 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cd169dc2..2ef515f9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1452,11 +1452,11 @@ dependencies = [ "futures-util", "lock_api", "loom", + "mutex-traits", "mycelium-bitfield", "pin-project", "portable-atomic", "proptest", - "scoped-mutex-traits", "tokio", "tokio-test", "tracing 0.1.40", @@ -1541,6 +1541,11 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "mutex-traits" +version = "0.999.0" +source = "git+https://github.com/tosc-rs/scoped-mutex#67dedfa152e553ccee1ae78342eae7f6bb67cb96" + [[package]] name = "mycelium-alloc" version = "0.1.0" @@ -2206,11 +2211,6 @@ version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f98d2aa92eebf49b69786be48e4477826b256916e84a57ff2a4f21923b48eb4c" -[[package]] -name = "scoped-mutex-traits" -version = "0.1.0" -source = "git+https://github.com/tosc-rs/scoped-mutex#13eebe716e9b5945c21fa652ed3adfc7d6ca01ac" - [[package]] name = "scoped-tls" version = "1.0.1" diff --git a/Cargo.toml b/Cargo.toml index 81f34436..704f7a9b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -98,7 +98,7 @@ dynamic-range-start = "0xFFFF_8000_0000_0000" # use `tracing` 0.2 from git tracing = { git = "https://github.com/tokio-rs/tracing" } tracing-core = { git = "https://github.com/tokio-rs/tracing" } -scoped-mutex-traits = { git = "https://github.com/tosc-rs/scoped-mutex" } +mutex-traits = { git = "https://github.com/tosc-rs/scoped-mutex" } # Custom profile for Loom tests: enable release optimizations so that the loom # tests are less slow, but don't disable debug assertions. diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index 412c3443..4385b519 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -28,8 +28,8 @@ core-error = [] [dependencies] mycelium-bitfield = { version = "0.1.3", path = "../bitfield" } +mutex-traits = "0.999.0" cordyceps = { version = "0.3.0", path = "../cordyceps" } -scoped-mutex-traits = "0.1.0" lock_api = { version = "0.4", default-features = false, optional = true } pin-project = "1" portable-atomic = "1.2" diff --git a/maitake-sync/src/blocking.rs b/maitake-sync/src/blocking.rs index 807b69a6..5acb7a1b 100644 --- a/maitake-sync/src/blocking.rs +++ b/maitake-sync/src/blocking.rs @@ -39,7 +39,7 @@ //! [spin-loop hint instructions]: core::hint::spin_loop //! [`Spinlock`]: crate::spin::Spinlock //! [`RwSpinlock`]: crate::spin::RwSpinlock -mod mutex; -mod rwlock; +pub(crate) mod mutex; +pub(crate) mod rwlock; pub use self::{mutex::*, rwlock::*}; diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index b4fb4235..53cd5bd9 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -4,7 +4,8 @@ use crate::{ util::fmt, }; use core::ops::{Deref, DerefMut}; -pub use scoped_mutex_traits::ScopedRawMutex; + +pub use mutex_traits::{RawMutex, ScopedRawMutex}; /// A blocking mutual exclusion lock for protecting shared data. /// Each mutex has a type parameter which represents @@ -45,82 +46,6 @@ pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { lock: &'a Lock, } -/// Trait abstracting over blocking [`Mutex`] implementations (`maitake-sync`'s -/// version). -/// -/// This trait is essentially a duplicate of the [`lock_api::RawMutex`] trait. -/// `maitake-sync` defines its own version of `RawMutex` so that the `lock_api` -/// dependency can be optional, and can be disabled when only using -/// `maitake-sync`'s spinlocks. When the "lock_api" feature flag is enabled, -/// this trait will be implemented for all types implementing -/// [`lock_api::RawMutex`]. Users who wish to provide their own `RawMutex` -/// implementations should implement the [`lock_api::RawMutex`] trait, *not* -/// this trait. -/// -/// # Safety -/// -/// Implementations of this trait must ensure that the mutex is actually -/// exclusive: a lock can't be acquired while the mutex is already locked. -pub unsafe trait RawMutex { - /// Marker type which determines whether a lock guard should be [`Send`]. - /// - /// Implementations should use one of the [`lock_api::GuardSend`] or - /// [`lock_api::GuardNoSend`] helper types here. - type GuardMarker; - - /// Acquires this mutex, blocking the current thread/CPU core until it is - /// able to do so. - fn lock(&self); - - /// Attempts to acquire this mutex without blocking. Returns `true` - /// if the lock was successfully acquired and `false` otherwise. - fn try_lock(&self) -> bool; - - /// Unlocks this mutex. - /// - /// # Safety - /// - /// This method may only be called if the mutex is held in the current - /// context, i.e. it must be paired with a successful call to [`lock`] or - /// [`try_lock`]. - /// - /// [`lock`]: RawMutex::lock - /// [`try_lock`]: RawMutex::try_lock - unsafe fn unlock(&self); - - /// Returns `true` if the mutex is currently locked. - fn is_locked(&self) -> bool; -} - -#[cfg(feature = "lock_api")] -unsafe impl RawMutex for T { - type GuardMarker = ::GuardMarker; - - #[inline] - #[track_caller] - fn lock(&self) { - lock_api::RawMutex::lock(self); - } - - #[inline] - #[track_caller] - fn try_lock(&self) -> bool { - lock_api::RawMutex::try_lock(self) - } - - #[inline] - #[track_caller] - unsafe fn unlock(&self) { - lock_api::RawMutex::unlock(self); - } - - #[inline] - #[track_caller] - fn is_locked(&self) -> bool { - lock_api::RawMutex::is_locked(self) - } -} - impl Mutex { loom_const_fn! { /// Returns a new `Mutex` protecting the provided `data`. @@ -144,7 +69,7 @@ impl Mutex { } } -impl Mutex { +impl Mutex { loom_const_fn! { /// Returns a new `Mutex` protecting the provided `data`, using the /// `Lock` type parameter as the raw mutex implementation. @@ -158,40 +83,15 @@ impl Mutex { /// /// The returned `Mutex` is in an unlocked state, ready for use. #[must_use] - pub fn with_raw_mutex(data: T, lock: Lock) -> Self { + pub fn with_raw_mutex(data: T) -> Self { Self { - lock, + lock: Lock::INIT, data: UnsafeCell::new(data), } } } } -#[cfg(feature = "lock_api")] -impl Mutex -where - Lock: lock_api::RawMutex, -{ - /// Returns a new `Mutex` protecting the provided `data`, using the - /// `Lock` type parameter as the raw mutex implementation. - /// - /// This constructor is used to override the internal implementation of - /// mutex operations, with an implementation of the [`lock_api::RawMutex`] - /// trait. By default, the [`Mutex::new`] constructor uses a [`Spinlock`] as - /// the underlying raw mutex implementation, which will spin until the mutex - /// is unlocked, without using platform-specific or OS-specific blocking - /// mechanisms. - /// - /// The returned `Mutex` is in an unlocked state, ready for use. - #[must_use] - pub const fn with_lock_api(data: T) -> Self { - Self { - lock: Lock::INIT, - data: UnsafeCell::new(data), - } - } -} - impl Mutex { /// Lock this `Mutex`, blocking if it is not currently unlocked, and call /// `f()` with the locked data once the lock is acquired. @@ -346,19 +246,19 @@ impl Default for Mutex { impl fmt::Debug for Mutex where T: fmt::Debug, - Lock: ScopedRawMutex + fmt::Debug, + Lock: ScopedRawMutex, { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { self.try_with_lock(|data| { f.debug_struct("Mutex") .field("data", data) - .field("lock", &self.lock) + .field("lock", &format_args!("{}", core::any::type_name::())) .finish() }) .unwrap_or_else(|| { f.debug_struct("Mutex") .field("data", &format_args!("")) - .field("lock", &self.lock) + .field("lock", &format_args!("{}", core::any::type_name::())) .finish() }) } diff --git a/maitake-sync/src/rwlock.rs b/maitake-sync/src/rwlock.rs index f2d38011..d0f28006 100644 --- a/maitake-sync/src/rwlock.rs +++ b/maitake-sync/src/rwlock.rs @@ -196,17 +196,6 @@ impl RwLock { } } -#[cfg(feature = "lock_api")] -impl RwLock { - #[must_use] - pub const fn with_raw_mutex(data: T) -> Self { - Self { - sem: Semaphore::with_raw_mutex(Self::MAX_READERS), - data: UnsafeCell::new(data), - } - } -} - impl RwLock { /// Consumes this `RwLock`, returning the guarded data. #[inline] diff --git a/maitake-sync/src/semaphore.rs b/maitake-sync/src/semaphore.rs index 2bb20d97..4e8fda49 100644 --- a/maitake-sync/src/semaphore.rs +++ b/maitake-sync/src/semaphore.rs @@ -313,14 +313,6 @@ impl Semaphore { } } -#[cfg(feature = "lock_api")] -impl Semaphore { - #[must_use] - pub const fn with_lock_api(permits: usize) -> Self { - Self::make(permits, Mutex::with_lock_api(SemQueue::new())) - } -} - // This is factored out as a free constant in this module so that `RwLock` can // depend on it without having to specify `Semaphore`'s type parameters. This is // a little annoying but whatever. diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index b1862970..a8dcb674 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -33,6 +33,10 @@ pub mod once; pub use self::once::{InitOnce, Lazy}; +#[deprecated( + since = "0.1.3", + note = "moved to the `blocking` module, prefer importing from there" +)] pub use crate::blocking::*; use crate::{ loom::sync::atomic::{AtomicBool, AtomicUsize, Ordering::*}, @@ -67,28 +71,6 @@ impl Spinlock { fn is_locked(&self) -> bool { self.locked.load(Relaxed) } - - /// Returns a RAII guard that unlocks this spinlock. This is like a simpler - /// version of a real life `MutexGuard` that ensures a `ScopedRawMutex` impl - /// is unlocked in the event of a panic - /// - /// # Safety - /// - /// The spinlock must be locked. - unsafe fn unlock_on_drop(&self) -> impl Drop + '_ { - struct Unlock<'lock>(&'lock Spinlock); - impl Drop for Unlock<'_> { - #[inline(always)] - #[track_caller] - fn drop(&mut self) { - debug_assert!(self.is_locked()); - unsafe { self.0.unlock() } - } - } - - debug_assert!(self.is_locked()); - Unlock(self) - } } impl Default for Spinlock { @@ -97,36 +79,6 @@ impl Default for Spinlock { } } -unsafe impl ScopedRawMutex for Spinlock { - #[inline] - fn with_lock(&self, f: impl FnOnce() -> R) -> R { - self.lock(); - let _unlock = unsafe { - // Safety: we just took the lock. - self.unlock_on_drop() - }; - f() - } - - #[inline] - fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { - if self.try_lock() { - let _unlock = unsafe { - // Safety: we just took the lock. - self.unlock_on_drop() - }; - Some(f()) - } else { - None - } - } - - #[inline] - fn is_locked(&self) -> bool { - Spinlock::is_locked(self) - } -} - unsafe impl RawMutex for Spinlock { type GuardMarker = (); diff --git a/maitake-sync/src/wait_map.rs b/maitake-sync/src/wait_map.rs index 00fac0e3..a75209af 100644 --- a/maitake-sync/src/wait_map.rs +++ b/maitake-sync/src/wait_map.rs @@ -204,7 +204,7 @@ pub struct WaitMap { impl Debug for WaitMap where K: PartialEq, - Lock: ScopedRawMutex + fmt::Debug, + Lock: ScopedRawMutex, { fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { f.debug_struct("WaitMap") @@ -447,7 +447,7 @@ impl WaitMap { #[cfg(not(loom))] pub const fn new() -> Self { Self { - state: CachePadded::new(AtomicUsize::new(State::Empty.into_usize())), + state: Self::mk_state(), queue: Mutex::new(List::new()), } } @@ -457,35 +457,34 @@ impl WaitMap { #[cfg(loom)] pub fn new() -> Self { Self { - state: CachePadded::new(AtomicUsize::new(State::Empty.into_usize())), + state: Self::mk_state(), queue: Mutex::new(List::new()), } } } -// TODO(eliza): figure out what to do about `lock_api` not impling `RawScopedMutex`... -// #[cfg(all(feature = "lock_api", not(loom)))] -// impl WaitMap -// where -// Lock: lock_api::RawMutex, -// K: PartialEq, -// { -// #[must_use] -// pub const fn with_lock_api() -> Self { -// Self::with_raw_mutex(Lock::INIT) -// } -// } - -impl WaitMap { +impl WaitMap +where + K: PartialEq, + Lock: ScopedRawMutex + mutex_traits::ConstInit, +{ loom_const_fn! { #[must_use] - pub fn with_raw_mutex(lock: Lock) -> Self { + pub fn with_raw_mutex() -> Self { Self { - state: CachePadded::new(AtomicUsize::new(State::Empty.into_usize())), - queue: Mutex::with_raw_mutex(List::new(), lock), + state: Self::mk_state(), + queue: Mutex::with_raw_mutex(List::new()), } } } +} + +impl WaitMap { + loom_const_fn! { + fn mk_state() -> CachePadded { + CachePadded::new(AtomicUsize::new(State::Empty.into_usize())) + } + } /// Wake a certain task in the queue. /// @@ -514,22 +513,34 @@ impl WaitMap { // okay, there are tasks waiting on the queue; we must acquire the lock // on the linked list and wake the next task from the queue. - self.queue.with_lock(|queue| { + let mut val = Some(val); + let maybe_waker = self.queue.with_lock(|queue| { test_debug!("wake: -> locked"); // the queue's state may have changed while we were waiting to acquire // the lock, so we need to acquire a new snapshot. state = self.load(); - if let Some(node) = self.node_match_locked(key, &mut *queue, state) { - let waker = Waiter::::wake(node, &mut *queue, Wakeup::DataReceived(val)); - drop(queue); - waker.wake(); - WakeOutcome::Woke - } else { - WakeOutcome::NoMatch(val) - } - }) + let node = self.node_match_locked(key, &mut *queue, state)?; + // if there's a node, give it the value and take the waker and + // return it. we return the waker from this closure rather than + // waking it, because we need to release the lock before waking the + // task. + let val = val + .take() + .expect("value is only taken elsewhere if there is no waker, but there is one"); + let waker = Waiter::::wake(node, &mut *queue, Wakeup::DataReceived(val)); + Some(waker) + }); + + if let Some(waker) = maybe_waker { + waker.wake(); + WakeOutcome::Woke + } else { + let val = + val.expect("value is only taken elsewhere if there is a waker, and there isn't"); + WakeOutcome::NoMatch(val) + } } /// Returns `true` if this `WaitMap` is [closed](Self::close). @@ -791,17 +802,17 @@ impl Waiter { where Lock: ScopedRawMutex, { - let mut this = self.as_mut().project(); - - debug_assert!( - matches!(this.state, WaitState::Start), - "start_to_wait should ONLY be called from the Start state!" - ); - // Try to wait... test_debug!("poll_wait: locking..."); - queue.queue.with_lock(|waiters| { + queue.queue.with_lock(move |waiters| { test_debug!("poll_wait: -> locked"); + let mut this = self.as_mut().project(); + + debug_assert!( + matches!(this.state, WaitState::Start), + "start_to_wait should ONLY be called from the Start state!" + ); + let mut queue_state = queue.load(); // transition the queue to the waiting state diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index 709fb50d..4069b989 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -360,7 +360,7 @@ impl WaitQueue { /// Returns a new `WaitQueue`. #[must_use] pub fn new() -> Self { - Self::make(State::Empty, Spinlock::new()) + Self::make(State::Empty, Mutex::new(List::new())) } } @@ -373,19 +373,20 @@ impl WaitQueue { // TODO(eliza): should this be a public API? #[must_use] pub(crate) fn new_woken() -> Self { - Self::make(State::Woken, Spinlock::new()) + Self::make(State::Woken, Mutex::new(List::new())) } } } -#[cfg(all(feature = "lock_api", not(loom)))] impl WaitQueue where - Lock: lock_api::RawMutex, + Lock: ScopedRawMutex + mutex_traits::ConstInit, { - #[must_use] - pub const fn with_lock_api() -> Self { - Self::make(State::Empty, Lock::INIT) + loom_const_fn! { + #[must_use] + pub fn with_raw_mutex() -> Self { + Self::make(State::Empty, Mutex::with_raw_mutex(List::new())) + } } } @@ -395,17 +396,10 @@ where { loom_const_fn! { #[must_use] - pub fn with_raw_mutex(lock: Lock) -> Self { - Self::make(State::Empty, lock) - } - } - - loom_const_fn! { - #[must_use] - fn make(state: State, lock: Lock) -> Self { + fn make(state: State, queue: Mutex, Lock>) -> Self { Self { state: CachePadded::new(AtomicUsize::new(state.into_usize())), - queue: Mutex::with_raw_mutex(List::new(), lock) + queue, } } } From b33c33b9e1c6880168fb7bd39e71a0b56ca07ff0 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 27 Jul 2024 09:23:04 -0700 Subject: [PATCH 16/41] fix simulated loom mutex --- maitake-sync/src/loom.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index 9a84ab9f..26f15866 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -100,18 +100,18 @@ mod inner { } #[track_caller] - pub(crate) fn with_raw_mutex(t: T, _: Lock) -> Self { + pub(crate) fn with_raw_mutex(t: T) -> Self { Self::new(t) } #[track_caller] - pub fn with(&self, f: impl FnOnce(&mut T) -> U) -> U { + pub fn with_lock(&self, f: impl FnOnce(&mut T) -> U) -> U { let location = core::panic::Location::caller(); - tracing::debug!( %location, "Mutex::with: locking...",); + tracing::debug!(%location, "Mutex::with: locking...",); let mut guard = self.lock(); tracing::debug!(%location, "Mutex::with: -> locked!",); let res = f(&mut *guard); - tracing::debug!(%location, "Mutex::with: unlocking...",); + tracing::debug!(%location, "Mutex::with: -> unlocking...",); res } From 75b0a7ffeb2d4ab3ec1e3a25d632e2f7e486a800 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 27 Jul 2024 09:36:45 -0700 Subject: [PATCH 17/41] add nicer debug logs to loom mutex --- maitake-sync/src/loom.rs | 66 +++++++++++++++++++++++++++++++--------- 1 file changed, 51 insertions(+), 15 deletions(-) diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index 26f15866..143c47c7 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -76,6 +76,7 @@ mod inner { use core::{ marker::PhantomData, ops::{Deref, DerefMut}, + panic::Location, }; use alloc::fmt; @@ -88,10 +89,11 @@ mod inner { PhantomData, ); - pub(crate) struct MutexGuard<'a, T, Lock = crate::spin::Spinlock>( - loom::sync::MutexGuard<'a, T>, - PhantomData, - ); + pub(crate) struct MutexGuard<'a, T, Lock = crate::spin::Spinlock> { + guard: loom::sync::MutexGuard<'a, T>, + location: &'static Location<'static>, + _p: PhantomData, + } impl Mutex { #[track_caller] @@ -106,26 +108,49 @@ mod inner { #[track_caller] pub fn with_lock(&self, f: impl FnOnce(&mut T) -> U) -> U { - let location = core::panic::Location::caller(); - tracing::debug!(%location, "Mutex::with: locking...",); let mut guard = self.lock(); - tracing::debug!(%location, "Mutex::with: -> locked!",); let res = f(&mut *guard); - tracing::debug!(%location, "Mutex::with: -> unlocking...",); res } #[track_caller] pub fn try_lock(&self) -> Option> { - self.0.try_lock().map(|x| MutexGuard(x, PhantomData)).ok() + let location = Location::caller(); + tracing::debug!(%location, "Mutex::try_lock"); + + match self.0.try_lock() { + Ok(guard) => { + tracing::debug!(%location, "Mutex::try_lock -> locked!"); + Some(MutexGuard { + guard, + location, + _p: PhantomData, + }) + } + Err(_) => { + tracing::debug!(%location, "Mutex::try_lock -> already locked"); + None + } + } } #[track_caller] pub fn lock(&self) -> MutexGuard<'_, T, Lock> { - self.0 + let location = Location::caller(); + tracing::debug!(%location, "Mutex::lock"); + + let guard = self + .0 .lock() - .map(|x| MutexGuard(x, PhantomData)) - .expect("loom mutex will never poison") + .map(|guard| MutexGuard { + guard, + location, + _p: PhantomData, + }) + .expect("loom mutex will never poison"); + + tracing::debug!(%location, "Mutex::lock -> locked"); + guard } } @@ -139,20 +164,31 @@ mod inner { type Target = T; #[inline] fn deref(&self) -> &Self::Target { - self.0.deref() + self.guard.deref() } } impl DerefMut for MutexGuard<'_, T, Lock> { #[inline] fn deref_mut(&mut self) -> &mut Self::Target { - self.0.deref_mut() + self.guard.deref_mut() } } impl fmt::Debug for MutexGuard<'_, T, Lock> { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - self.0.fmt(f) + self.guard.fmt(f) + } + } + + impl Drop for MutexGuard<'_, T, Lock> { + #[track_caller] + fn drop(&mut self) { + tracing::debug!( + location.dropped = %Location::caller(), + location.locked = %self.location, + "MutexGuard::drop: unlocking", + ); } } } From 3a1b1854d6bc841a080a3e45e2470e19c45ffc88 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 27 Jul 2024 10:53:45 -0700 Subject: [PATCH 18/41] remove `lock_api` dep --- Cargo.lock | 17 ----------------- maitake-sync/Cargo.toml | 1 - maitake-sync/README.md | 1 - 3 files changed, 19 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2ef515f9..c38d4902 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1389,16 +1389,6 @@ dependencies = [ "json", ] -[[package]] -name = "lock_api" -version = "0.4.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" -dependencies = [ - "autocfg", - "scopeguard", -] - [[package]] name = "log" version = "0.4.20" @@ -1450,7 +1440,6 @@ dependencies = [ "cordyceps", "futures", "futures-util", - "lock_api", "loom", "mutex-traits", "mycelium-bitfield", @@ -2217,12 +2206,6 @@ version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" -[[package]] -name = "scopeguard" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" - [[package]] name = "semver" version = "1.0.21" diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index 4385b519..53b0cfa1 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -30,7 +30,6 @@ core-error = [] mycelium-bitfield = { version = "0.1.3", path = "../bitfield" } mutex-traits = "0.999.0" cordyceps = { version = "0.3.0", path = "../cordyceps" } -lock_api = { version = "0.4", default-features = false, optional = true } pin-project = "1" portable-atomic = "1.2" tracing = { version = "0.1", default_features = false, optional = true } diff --git a/maitake-sync/README.md b/maitake-sync/README.md index 9ead4496..9ae18ead 100644 --- a/maitake-sync/README.md +++ b/maitake-sync/README.md @@ -178,7 +178,6 @@ The following features are available (this list is incomplete; you can help by [ | `no-cache-pad` | `false` | Inhibits cache padding for the [`CachePadded`] struct. When this feature is NOT enabled, the size will be determined based on target platform. | | `tracing` | `false` | Enables support for [`tracing`] diagnostics. Requires `liballoc`.| | `core-error` | `false` | Enables implementations of the [`core::error::Error` trait][core-error] for `maitake-sync`'s error types. *Requires a nightly Rust toolchain*. | -| `lock_api` | `false` | Enables support for the [`lock_api`] crate, which can be used to provide custom blocking `Mutex` implementations. See [the section on `lock_api` support](#lock_api-support) for details. | [`liballoc`]: https://doc.rust-lang.org/alloc/ [`CachePadded`]: https://docs.rs/maitake-sync/latest/maitake_sync/util/struct.CachePadded.html From 6f97043a827a264450f349c328b9657e04137c30 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 27 Jul 2024 11:28:46 -0700 Subject: [PATCH 19/41] huge docs pass + missing constructors --- maitake-sync/README.md | 56 +++++++++---- maitake-sync/src/blocking.rs | 124 ++++++++++++++++++++++++++-- maitake-sync/src/blocking/mutex.rs | 72 ++++++++++++---- maitake-sync/src/blocking/rwlock.rs | 12 --- maitake-sync/src/mutex.rs | 51 ++++++++++-- maitake-sync/src/rwlock.rs | 44 +++++++++- maitake-sync/src/semaphore.rs | 37 +++++++++ maitake-sync/src/wait_map.rs | 35 ++++++++ maitake-sync/src/wait_queue.rs | 38 +++++++++ 9 files changed, 413 insertions(+), 56 deletions(-) diff --git a/maitake-sync/README.md b/maitake-sync/README.md index 9ae18ead..ee005958 100644 --- a/maitake-sync/README.md +++ b/maitake-sync/README.md @@ -76,8 +76,8 @@ The following synchronization primitives are provided: can be woken by its key In addition, the [`util` module] contains a collection of general-purpose -utilities for implementing synchronization primitives, and the [`spin` module] -contains implementations of *non-async*, spinning-based synchronization +utilities for implementing synchronization primitives, and the [`blocking` +module] contains implementations of *non-async* blocking synchronization primitives. [`core::task`]: https://doc.rust-lang.org/stable/core/task/index.html @@ -100,8 +100,8 @@ primitives. https://docs.rs/maitake-sync/latest/maitake_sync/struct.WaitMap.html [`util` module]: https://docs.rs/maitake-sync/latest/maitake_sync/util/index.html -[`spin` module]: - https://docs.rs/maitake-sync/latest/maitake_sync/spin/index.html +[`blocking` module]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html ## usage considerations @@ -157,14 +157,44 @@ critical section, are described [here][interrupt-cfgs]. [single-core]: https://docs.rs/portable-atomic/latest/portable_atomic/#optional-cfg [interrupt-cfgs]: https://github.com/taiki-e/portable-atomic/blob/HEAD/src/imp/interrupt/README.md -### `lock_api` support +### overriding blocking mutex implementations In addition to async locks, `maitake-sync` also provides a [`blocking`] module, -which contains blocking `Mutex` and `RwLock` implementations. Many of -`maitake-sync`'s async synchronization primitives, including `WaitQueue`, -`Mutex`, `RwLock`, and `Semaphore`, internally use the [`blocking::Mutex`] type -for wait-list synchronization. +which contains blocking [`blocking::Mutex`] and [`blocking::RwLock`] types. Many of +`maitake-sync`'s async synchronization primitives, including [`WaitQueue`], +[`Mutex`], [`RwLock`], and [`Semaphore`], internally use the [`blocking::Mutex`] +type for wait-list synchronization. By default, the [`blocking::Mutex`] type is +implemented using an atomic [spinlock]. + +However, for many applications, a generic spinlock is not the ideal blocking +mechanism. Therefore, `maitake-sync`'s [`blocking::Mutex`] type, and the +async synchronization primitives that depend on it, are generic over a `Lock` +type parameter which may be overridden using the [`RawMutex`] and +[`ScopedRawMutex`] traits from the [`mutex-traits`] crate, allowing alternative +blocking mutex implementations to be used with `maitake-sync`. Using the +[`mutex-traits`] adapters in the [`mutex`] crate, `maitake-sync`'s types may +also be used with raw mutex implementations that implement traits from the +[`lock_api`] and [`critical-section`] crates. + +See [the documentation on overriding mutex implementations][overriding] for more +details. +[`blocking`]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html +[`blocking::Mutex`]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.Mutex.html +[`blocking::RwLock`]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.RwLock.html +[spinlock]: https://en.wikipedia.org/wiki/Spinlock +[`RawMutex`]: + https://docs.rs/mutex-traitsc/latest/mutex_traits/trait.RawMutex.html +[`ScopedRawMutex`]: + https://docs.rs/mutex-traitsc/latest/mutex_traits/trait.ScopedRawMutex.html +[`mutex-traits`]: https://crates.io/crates/mutex-traits +[`lock_api`]: https://crates.io/crates/lock_api +[`critical-section`]: https://crates.io/crates/critical-section +[overriding]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html#overriding-mutex-implementations ## features @@ -183,11 +213,3 @@ The following features are available (this list is incomplete; you can help by [ [`CachePadded`]: https://docs.rs/maitake-sync/latest/maitake_sync/util/struct.CachePadded.html [`tracing`]: https://crates.io/crates/tracing [core-error]: https://doc.rust-lang.org/stable/core/error/index.html -[`lock_api`]: https://docs.rs/lock_api -[`blocking`]: - https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html -[`blocking::Mutex`]: - https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.Mutex.html -[`spin`]: https://docs.rs/maitake-sync/latest/maitake_sync/spin/index.html -[`Spinlock`]: - https://docs.rs/maitake-sync/latest/maitake_sync/spin/struct.Spinlock.html diff --git a/maitake-sync/src/blocking.rs b/maitake-sync/src/blocking.rs index 5acb7a1b..85730803 100644 --- a/maitake-sync/src/blocking.rs +++ b/maitake-sync/src/blocking.rs @@ -15,22 +15,120 @@ //! These synchronization primitives are, in some cases, necessary to implement //! the async synchronization primitives that form `maitake-sync`'s core APIs. //! They are also exposed publicly in this module so that they can be used in -//! other projects when a blocking-based synchronization primitive is needed.\ +//! other projects when a blocking-based synchronization primitive is needed. //! //! This module provides the following synchronization primitive types: //! -//! - [`Mutex`]: a synchronous [mutual exclusion] lock. -//! - [`RwLock`]: a synchronous [reader-writer] lock. +//! - [`Mutex`]: a synchronous [mutual exclusion lock]. +//! - [`RwLock`]: a synchronous [reader-writer lock]. //! -//! # `lock_api` support +//! # overriding mutex implementations //! //! By default, the [`Mutex`] and [`RwLock`] types are implemented using simple -//! _[spinlocks_]_, which wait for the lock to become available by _spinning_: +//! _[spinlocks]_, which wait for the lock to become available by _spinning_: //! repeatedly checking an atomic value in a loop, executing [spin-loop hint //! instructions] until the lock value changes. These spinlock implementations //! are represented by the [`Spinlock`] and [`RwSpinlock`] types in the -//! [`spin`] module. Spinlocks are a +//! [`spin`](crate::spin) module. //! +//! Spinlocks are simple to implement and, thanks to the Rust standard library +//! abstracting over atomic operations, portable. The default spinlock will work +//! on any platform with support for atomic compare-and-swap operations.[^1] +//! However, there are a number of reasons why a generic spinlock may not be +//! desirable for all use-cases. For example: +//! +//! - On single-core platforms, there *is* no concurrent thread of execution +//! which can acquire and release a lock. If code running on a single-core +//! system attempts to acquire a lock and finds that it is already locked, +//! waiting for the lock to be released will never work, since the *same* +//! thread of execution is holding the lock already. On such systems, any +//! attempt to lock a mutex that is locked is guaranteed to be a deadlock. +//! Therefore, code which can guarantee that it will only run on single-core +//! CPUs may prefer to avoid the complexity of a "real" lock implementation +//! altogether, and panic (or otherwise alert the programmer) rather than +//! deadlocking when attempting to acquire a mutex that is already locked. +//! - In bare-metal code, the data protected by a mutex may be shared not only +//! with concurrent threads of execution, but with [interrupt handlers] as +//! well. When this is the case, it may be necessary for acquiring a lock to +//! also disable interrupts while the lock is held (and re-enable then when +//! the lock is released) to avoid racing with code that runs in an interrupt +//! handler. +//! - While spinlocks are simple and portable, they are [not the most +//! *efficient*][busy-wait]. A CPU core waiting on a spinlock draws power +//! while in a spin loop, and is not executing any other tasks. Systems with a +//! notion of a scheduler, whether provided by an operating system or +//! implemented directly within a bare-metal program, may prefer to yield to +//! the scheduler when waiting for a lock. +//! - Some hardware platforms provide mechanisms to optimize the performance of +//! spinlocks, such as [Hardware Lock Elision] on x86 CPUs. When such features +//! are available, using them can improve performance and efficiency. +//! +//! However, all of these alternative waiting strategies knowledge of either the +//! underlying hardware platform, the specific details of the system, or both. A +//! single-core "fake" spinlock naturally requires the knowledge that the +//! hardware platform is single-core, and hardware lock elision similarly +//! requires knowledge about platform-specific features. Similarly, disabling +//! interrupts may require application-specific as well as hardware-specific +//! code, especially if only certain interrupts need to be disabled. And, of +//! course, yielding to a scheduler requires scheduler-specific code. Since +//! `maitake-sync` is a platform-agnostic, generic library, it does not provide +//! its own implementations of such behaviors. Instead, users which need a +//! blocking strategy other than the default spinlock may *override* the default +//! behavior using the traits provided by the [`mutex-traits`] crate. +//! +//! The [`Mutex`] type in this module is generic over an additional `Lock` type +//! parameter, which represents the actual raw mutex implementation. This type +//! parameter defaults to the [`Spinlock`] type, but it can be overridden to an +//! arbitrary user-provided type. The [`mutex-traits`] crate provides two +//! traits representing a raw mutex, [`mutex_traits::ScopedRawMutex`] and +//! [`mutex_traits::RawMutex`]. These can be implemented to provide a custom +//! lock implementation. [`mutex_traits::RawMutex`] represents a generic +//! mutual-exclusion lock that can be freely locked and unlocked at any time, +//! while [`mutex_traits::ScopedRawMutex`] is a subset of [`RawMutex`] for which +//! locks can only be acquired and released for the duration of a closure. As +//! the functionality of [`RawMutex`] is a superset of [`ScopedRawMutex`], all +//! types which implement [`RawMutex`] also implement [`ScopedRawMutex`]. In +//! general, it is recommended for user lock types implement the more flexible +//! [`RawMutex`] trait rather than [`ScopedRawMutex`], if possible: +//! [`ScopedRawMutex`] exists to support more restricted lock types which +//! require scoped lock-and-unlock operations. Finally, the [`ConstInit`] trait +//! abstracts over `const fn` initialization of a raw mutex type, and is +//! required for `const fn` constructors with a custom raw mutex type. +//! +//! When the `Lock` type parameter implements [`ScopedRawMutex`], the [`Mutex`] +//! type provides the [`Mutex::with_lock`] and [`Mutex::try_with_lock`] methods, +//! which execute a closure with the mutex locked, and release the lock when the +//! closure returns. When the `Lock` type parameter also implements +//! [`RawMutex`], the [`Mutex`] type provides [`Mutex::lock`] and +//! [`Mutex::try_lock`] methods, which return a RAII [`MutexGuard`], similar to +//! the interface provided by [`std::sync::Mutex`]. The [`Mutex::new`] function +//! returns a [`Mutex`] using the default spinlock. To instead construct a +//! [`Mutex`] with a custom [`RawMutex`] implementation, use the +//! [`Mutex::with_raw_mutex`] function. +//! +//! Furthermore, many *async* synchronization primitives provided by this crate, +//! such as the [async `Mutex`](crate::Mutex), [async `RwLock`], [`WaitQueue`], +//! [`WaitMap`], and [`Semaphore`], internally depend on the blocking `Mutex` +//! for wait list synchronization. These types are *also* generic over a `Lock` +//! type parameter, and also provide `with_raw_mutex` constructors, such as +//! [`WaitQueue::with_raw_mutex`](crate::WaitQueue::with_raw_mutex). This allows +//! the blocking mutex used by these types to be overridden. The majority +//! `maitake-sync`'s async synchronization types only require the `Lock` type to +//! implement [`ScopedRawMutex`]. However, the [`Semaphore`] and [async +//! `RwLock`] require the more permissive [`RawMutex`] trait. +//! +//! The [`mutex` crate] provides a number of types implementing [`RawMutex`] and +//! [`ScopedRawMutex`], including adapters for compatibility with the +//! [`lock_api`] and [`critical-section`] crates. +//! +//! Similarly to the [`RawMutex`] trait, the blocking [`RwLock`] type in this +//! module is generic over a `Lock` type parameter, which must implement the +//! [`RawRwLock`] trait. This allows the `RwLock`'s blocking behavior to be +//! overridden similarly to [`Mutex`]. +//! +//! [^1]: Including those where "atomics" are implemented by the +//! `portable-atomic` crate, as described +//! [here](crate#support-for-atomic-operations). //! //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion //! [reader-writer lock]: @@ -39,7 +137,21 @@ //! [spin-loop hint instructions]: core::hint::spin_loop //! [`Spinlock`]: crate::spin::Spinlock //! [`RwSpinlock`]: crate::spin::RwSpinlock +//! [interrupt handler]: https://en.wikipedia.org/wiki/Interrupt_handler +//! [busy-wait]: https://en.wikipedia.org/wiki/Spinlock#Alternatives +//! [Hardware Lock Elision]: +//! https://en.wikipedia.org/wiki/Transactional_Synchronization_Extensions#HLE +//! [`WaitQueue`]: crate::WaitQueue +//! [`WaitMap`]: crate::WaitMap +//! [`Semaphore`]: crate::Semaphore +//! [async `RwLock`]: crate::RwLock +//! [`mutex-traits`]: https://docs.rs/mutex-traits +//! [`mutex` crate]: https://crates.io/crates/mutex +//! [`lock_api`]: https://crates.io/crates/lock_api +//! [`critical-section`]: https://crates.io/crates/critical-section +//! [`std::sync::Mutex`]: https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html pub(crate) mod mutex; pub(crate) mod rwlock; pub use self::{mutex::*, rwlock::*}; +pub use mutex_traits::ConstInit; diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 53cd5bd9..106a3444 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -10,13 +10,31 @@ pub use mutex_traits::{RawMutex, ScopedRawMutex}; /// A blocking mutual exclusion lock for protecting shared data. /// Each mutex has a type parameter which represents /// the data that it is protecting. The data can only be accessed through the -/// RAII guards returned from [`lock`] and [`try_lock`], which guarantees that +/// RAII guards returned from [`lock`] and [`try_lock`], or within the closures +/// passed to [`with_lock`] and [`try_with_lock`], which guarantees that /// the data is only ever accessed when the mutex is locked. /// /// # Fairness /// /// This is *not* a fair mutex. /// +/// # Overriding mutex implementations +/// +/// This type is generic over a `Lock` type parameter which represents a raw +/// mutex implementation. By default, this is a [`Spinlock`]. To construct a new +/// `Mutex` with an alternative raw mutex implementation, use the +/// [`Mutex::with_raw_mutex`] cosntructor. See the [module-level documentation +/// on overriding mutex +/// implementations](crate::blocking#overriding-mutex-implementations) for +/// more details. +/// +/// When `Lock` implements the [`RawMutex`] trait, the [`Mutex`] type provides +/// the [`lock`] and [`try_lock`] methods, which return a RAII [`MutexGuard`], +/// similar to the [`std::sync::Mutex`] API, in addition to the scoped +/// [`with_lock`] and [`try_with_lock`] methods. When `Lock` only implements +/// [`ScopedRawMutex`], the [`Mutex`] type provides only the scoped +/// [`with_lock`] and [`try_with_lock`] methods. +/// /// # Loom-specific behavior /// /// When `cfg(loom)` is enabled, this mutex will use Loom's simulated atomics, @@ -24,6 +42,9 @@ pub use mutex_traits::{RawMutex, ScopedRawMutex}; /// /// [`lock`]: Mutex::lock /// [`try_lock`]: Mutex::try_lock +/// [`with_lock`]: Mutex::with_lock +/// [`try_with_lock`]: Mutex::try_with_lock +/// [`std::sync::Mutex`]: https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html pub struct Mutex { lock: Lock, data: UnsafeCell, @@ -74,12 +95,9 @@ impl Mutex { /// Returns a new `Mutex` protecting the provided `data`, using the /// `Lock` type parameter as the raw mutex implementation. /// - /// This constructor is used to override the internal implementation of - /// mutex operations, with an implementation of the [`lock_api::RawMutex`] - /// trait. By default, the [`Mutex::new`] constructor uses a [`Spinlock`] as - /// the underlying raw mutex implementation, which will spin until the mutex - /// is unlocked, without using platform-specific or OS-specific blocking - /// mechanisms. + /// See the [module-level documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) for + /// more details. /// /// The returned `Mutex` is in an unlocked state, ready for use. #[must_use] @@ -104,8 +122,11 @@ impl Mutex { /// To return immediately rather than blocking, use [`Mutex::try_with_lock`] /// instead. /// - /// This method is available when the `Mutex`'s `Lock` type parameter - /// implements the [`ScopedRawMutex`] trait. + /// This method is available as long as the `Mutex`'s `Lock` type parameter + /// implements the [`ScopedRawMutex`] trait. See the [module-level + /// documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) for + /// more details. #[track_caller] pub fn with_lock(&self, f: impl FnOnce(&mut T) -> U) -> U { self.lock.with_lock(|| { @@ -127,8 +148,11 @@ impl Mutex { /// To block until the `Mutex` is unlocked instead of returning `None`, use /// [`Mutex::with_lock`] instead. /// - /// This method is available when the `Mutex`'s `Lock` type parameter - /// implements the [`ScopedRawMutex`] trait. + /// This method is available as long as the `Mutex`'s `Lock` type parameter + /// implements the [`ScopedRawMutex`] trait. See the [module-level + /// documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) for + /// more details. /// /// # Returns /// @@ -158,13 +182,19 @@ where } } - /// Attempts to acquire this lock without spinning + /// Attempts to acquire this lock without blocking /// /// If the lock could not be acquired at this time, then [`None`] is returned. /// Otherwise, an RAII guard is returned. The lock will be unlocked when the /// guard is dropped. /// - /// This function will never spin. + /// This function will never block. + /// + /// This method is only availble if the `Mutex`'s `Lock` type parameter + /// implements the [`RawMutex`] trait. See the [module-level documentation + /// on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) for + /// more details. #[must_use] #[cfg_attr(test, track_caller)] pub fn try_lock(&self) -> Option> { @@ -175,12 +205,18 @@ where } } - /// Acquires a mutex, spinning until it is locked. + /// Acquires a mutex, blocking until it is locked. /// - /// This function will spin until the mutex is available to lock. Upon + /// This function will block until the mutex is available to lock. Upon /// returning, the thread is the only thread with the lock /// held. An RAII guard is returned to allow scoped unlock of the lock. When /// the guard goes out of scope, the mutex will be unlocked. + /// + /// This method is only availble if the `Mutex`'s `Lock` type parameter + /// implements the [`RawMutex`] trait. See the [module-level documentation + /// on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) for + /// more details. #[cfg_attr(test, track_caller)] pub fn lock(&self) -> MutexGuard<'_, T, Lock> { self.lock.lock(); @@ -193,6 +229,12 @@ where /// holding it. Of course, this is **outrageously, disgustingly unsafe** and /// you should never do it. /// + /// This method is only availble if the `Mutex`'s `Lock` type parameter + /// implements the [`RawMutex`] trait. See the [module-level documentation + /// on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) for + /// more details. + /// /// # Safety /// /// This deliberately violates mutual exclusion. diff --git a/maitake-sync/src/blocking/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs index fdf1fe49..45fa945f 100644 --- a/maitake-sync/src/blocking/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -75,15 +75,6 @@ pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { /// Trait abstracting over blocking [`RwLock`] implementations (`maitake-sync`'s /// version). /// -/// This trait is essentially a duplicate of the [`lock_api::RawRwLock`] trait. -/// `maitake-sync` defines its own version of `RawRwLock` so that the `lock_api` -/// dependency can be optional, and can be disabled when only using -/// `maitake-sync`'s spinlocks. When the "lock_api" feature flag is enabled, -/// this trait will be implemented for all types implementing -/// [`lock_api::RawRwLock`]. Users who wish to provide their own `RawRwLock` -/// implementations should implement the [`lock_api::RawRwLock`] trait, *not* -/// this trait. -/// /// # Safety /// /// Implementations of this trait must ensure that the `RwLock` is actually @@ -92,9 +83,6 @@ pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { /// exists. pub unsafe trait RawRwLock { /// Marker type which determines whether a lock guard should be [`Send`]. - /// - /// Implementations should use one of the [`lock_api::GuardSend`] or - /// [`lock_api::GuardNoSend`] helper types here. type GuardMarker; /// Acquires a shared lock, blocking the current thread/CPU core until it is diff --git a/maitake-sync/src/mutex.rs b/maitake-sync/src/mutex.rs index 55f221c1..1dc617c5 100644 --- a/maitake-sync/src/mutex.rs +++ b/maitake-sync/src/mutex.rs @@ -16,8 +16,8 @@ use core::{ pin::Pin, task::{Context, Poll}, }; +use mutex_traits::ConstInit; use pin_project::pin_project; - #[cfg(test)] mod tests; @@ -33,7 +33,7 @@ mod tests; /// [`lock`] method will wait by causing the current [task] to yield until the /// shared data is available. This is in contrast to *blocking* mutices, such as /// [`std::sync::Mutex`], which wait by blocking the current thread[^1], or -/// *spinlock* based mutices, such as [`spin::Mutex`], which wait by spinning +/// *spinlock* based mutices, such as [`blocking::Mutex`], which wait by spinning /// in a busy loop. /// /// The [`futures-util`] crate also provides an implementation of an asynchronous @@ -52,12 +52,22 @@ mod tests; /// will not acquire the lock until every other task ahead of it in the queue /// has had a chance to lock the shared data. Again, this is in contrast to /// [`std::sync::Mutex`], where fairness depends on the underlying OS' locking -/// primitives; and [`spin::Mutex`] and [`futures_util::lock::Mutex`], which +/// primitives; and [`blocking::Mutex`] and [`futures_util::lock::Mutex`], which /// will never guarantee fairness. /// /// Finally, this mutex does not implement [poisoning][^3], unlike /// [`std::sync::Mutex`]. /// +/// # Overriding the blocking mutex +/// +/// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to +/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To +/// use an alternative [`ScopedRawMutex`] implementation, use the +/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// on overriding mutex +/// implementations](crate::blocking#overriding-mutex-implementations) for more +/// details. +/// /// [^1]: And therefore require an operating system to manage threading. /// /// [^2]: The [tasks](core::task) themselves must, of course, be stored @@ -77,7 +87,7 @@ mod tests; /// [task]: core::task /// [fairly queued]: https://en.wikipedia.org/wiki/Unbounded_nondeterminism#Fairness /// [`std::sync::Mutex`]: https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html -/// [`spin::Mutex`]: crate::spin::Mutex +/// [`blocking::Mutex`]: crate::blocking::Mutex /// [`futures-util`]: https://crates.io/crate/futures-util /// [`futures_util::lock::Mutex`]: https://docs.rs/futures-util/latest/futures_util/lock/struct.Mutex.html /// [intrusive linked list]: crate::WaitQueue#implementation-notes @@ -155,6 +165,13 @@ impl Mutex { /// The returned `Mutex` will be in the unlocked state and is ready for /// use. /// + /// This constructor returns a [`Mutex`] that uses a [`Spinlock`] as the + /// underlying blocking mutex implementation. To use an alternative + /// [`ScopedRawMutex`] implementation, use the [`Mutex::with_raw_mutex`] + /// constructor instead. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + /// /// # Examples /// /// ``` @@ -172,7 +189,7 @@ impl Mutex { #[must_use] pub fn new(data: T) -> Self { Self { - // The queue must start with a single store d wakeup, so that the + // The queue must start with a single stored wakeup, so that the // first task that tries to acquire the lock will succeed // immediately. wait: WaitQueue::new_woken(), @@ -182,6 +199,30 @@ impl Mutex { } } +#[cfg(not(loom))] +impl Mutex +where + L: ScopedRawMutex + ConstInit, +{ + /// Returns a new `Mutex` protecting the provided `data`, using the provided + /// [`ScopedRawMutex`] implementation as the raw mutex. + /// + /// The returned `Mutex` will be in the unlocked state and is ready for + /// use. + /// + /// This constructor allows a [`Mutex`] to be constructed with any type that + /// implements [`ScopedRawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + pub const fn with_raw_mutex(data: T) -> Self { + Self { + wait: WaitQueue::new_woken_with_raw_mutex(), + data: UnsafeCell::new(data), + } + } +} + impl Mutex { /// Consumes this `Mutex`, returning the guarded data. #[inline] diff --git a/maitake-sync/src/rwlock.rs b/maitake-sync/src/rwlock.rs index d0f28006..2f0962f6 100644 --- a/maitake-sync/src/rwlock.rs +++ b/maitake-sync/src/rwlock.rs @@ -59,6 +59,19 @@ mod tests; /// in contrast to the Rust standard library's [`std::sync::RwLock`], where the /// priority policy is dependent on the operating system's implementation. /// +/// # Overriding the blocking mutex +/// +/// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to +/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To +/// use an alternative [`RawMutex`] implementation, use the +/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// on overriding mutex +/// implementations](crate::blocking#overriding-mutex-implementations) for more +/// details. +/// +/// Note that this type currently requires that the raw mutex implement +/// [`RawMutex`] rather than [`mutex_traits::ScopedRawMutex`]! +/// /// # Examples /// /// ``` @@ -90,7 +103,8 @@ mod tests; /// [`write`]: Self::write /// [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock /// [_write-preferring_]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock#Priority_policies -/// [`std::sync::RwLock`]: https://doc.rust-lang.org/stable/std/sync/struct.RwLock.html +/// [`std::sync::RwLock`]: +/// https://doc.rust-lang.org/stable/std/sync/struct.RwLock.html pub struct RwLock { /// The semaphore used to control access to `data`. /// @@ -169,6 +183,13 @@ impl RwLock { /// Returns a new `RwLock` protecting the provided `data`, in an /// unlocked state. /// + /// This constructor returns a `RwLock` that uses a [`Spinlock`] as the + /// underlying blocking mutex implementation. To use an alternative + /// [`RawMutex`] implementation, use the [`RwLock::with_raw_mutex`] + /// constructor instead. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + /// /// # Examples /// /// ``` @@ -196,6 +217,27 @@ impl RwLock { } } +#[cfg(not(loom))] +impl RwLock +where + Lock: RawMutex + mutex_traits::ConstInit, +{ + /// Returns a new `RwLock` protecting the provided `data`, in an + /// unlocked state, using the provided [`RawMutex`] implementation. + /// + /// This constructor allows a [`RwLock`] to be constructed with any type that + /// implements [`RawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + pub const fn with_raw_mutex(data: T) -> Self { + Self { + sem: Semaphore::with_raw_mutex(Self::MAX_READERS), + data: UnsafeCell::new(data), + } + } +} + impl RwLock { /// Consumes this `RwLock`, returning the guarded data. #[inline] diff --git a/maitake-sync/src/semaphore.rs b/maitake-sync/src/semaphore.rs index 4e8fda49..f598f532 100644 --- a/maitake-sync/src/semaphore.rs +++ b/maitake-sync/src/semaphore.rs @@ -59,6 +59,19 @@ mod tests; /// semaphore remains fair even when a call to `acquire` requests more than one /// permit at a time. /// +/// # Overriding the blocking mutex +/// +/// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to +/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To +/// use an alternative [`RawMutex`] implementation, use the +/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// on overriding mutex +/// implementations](crate::blocking#overriding-mutex-implementations) for more +/// details. +/// +/// Note that this type currently requires that the raw mutex implement +/// [`RawMutex`] rather than [`mutex_traits::ScopedRawMutex`]! +/// /// # Examples /// /// Using a semaphore to limit concurrency: @@ -313,6 +326,30 @@ impl Semaphore { } } +#[cfg(not(loom))] +impl Semaphore +where + Lock: RawMutex + mutex_traits::ConstInit, +{ + /// Returns a new `Semaphore` with `permits` permits available, using the + /// provided [`RawMutex`] implementation. + /// + /// This constructor allows a [`Semaphore`] to be constructed with any type that + /// implements [`RawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + /// + /// # Panics + /// + /// If `permits` is less than [`MAX_PERMITS`] ([`usize::MAX`] - 1). + /// + /// [`MAX_PERMITS`]: Self::MAX_PERMITS + pub const fn with_raw_mutex(permits: usize) -> Self { + Self::make(permits, Mutex::with_raw_mutex(SemQueue::new())) + } +} + // This is factored out as a free constant in this module so that `RwLock` can // depend on it without having to specify `Semaphore`'s type parameters. This is // a little annoying but whatever. diff --git a/maitake-sync/src/wait_map.rs b/maitake-sync/src/wait_map.rs index a75209af..ed44e9b9 100644 --- a/maitake-sync/src/wait_map.rs +++ b/maitake-sync/src/wait_map.rs @@ -87,6 +87,16 @@ const fn notified(data: T) -> Poll> { /// be used to provide the task with the desired data, as well as wake /// the task for further processing. /// +/// # Overriding the blocking mutex +/// +/// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to +/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To +/// use an alternative [`ScopedRawMutex`] implementation, use the +/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// on overriding mutex +/// implementations](crate::blocking#overriding-mutex-implementations) for more +/// details. +/// /// # Examples /// /// Waking a single task at a time by calling [`wake`][wake]: @@ -443,6 +453,14 @@ enum Wakeup { impl WaitMap { /// Returns a new `WaitMap`. + /// + /// This constructor returns a `WaitMap` that uses a [`Spinlock`] as + /// the [`ScopedRawMutex`] implementation for wait list synchronization. + /// To use a different [`ScopedRawMutex`] implementation, use the + /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See + /// [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. #[must_use] #[cfg(not(loom))] pub const fn new() -> Self { @@ -453,6 +471,14 @@ impl WaitMap { } /// Returns a new `WaitMap`. + /// + /// This constructor returns a `WaitMap` that uses a [`Spinlock`] as + /// the [`ScopedRawMutex`] implementation for wait list synchronization. + /// To use a different [`ScopedRawMutex`] implementation, use the + /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See + /// [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. #[must_use] #[cfg(loom)] pub fn new() -> Self { @@ -469,6 +495,15 @@ where Lock: ScopedRawMutex + mutex_traits::ConstInit, { loom_const_fn! { + + /// Returns a new `WaitMap`, using the provided [`ScopedRawMutex`] + /// implementation for wait-list synchronization. + /// + /// This constructor allows a `WaitMap` to be constructed with any type that + /// implements [`ScopedRawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. #[must_use] pub fn with_raw_mutex() -> Self { Self { diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index 4069b989..9c83be88 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -50,6 +50,16 @@ mod tests; /// synchronization primitive on its own: sometimes, you just need to have a /// bunch of tasks wait for something and then wake them all up. /// +/// # Overriding the blocking mutex +/// +/// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to +/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To +/// use an alternative [`ScopedRawMutex`] implementation, use the +/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// on overriding mutex +/// implementations](crate::blocking#overriding-mutex-implementations) for more +/// details. +/// /// # Examples /// /// Waking a single task at a time by calling [`wake`][wake]: @@ -358,6 +368,14 @@ enum Wakeup { impl WaitQueue { loom_const_fn! { /// Returns a new `WaitQueue`. + /// + /// This constructor returns a `WaitQueue` that uses a [`Spinlock`] as + /// the [`ScopedRawMutex`] implementation for wait list synchronization. + /// To use a different [`ScopedRawMutex`] implementation, use the + /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See + /// [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. #[must_use] pub fn new() -> Self { Self::make(State::Empty, Mutex::new(List::new())) @@ -383,11 +401,31 @@ where Lock: ScopedRawMutex + mutex_traits::ConstInit, { loom_const_fn! { + /// Returns a new `WaitQueue`, using the provided [`ScopedRawMutex`] + /// implementation for wait-list synchronization. + /// + /// This constructor allows a `WaitQueue` to be constructed with any type that + /// implements [`ScopedRawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. #[must_use] pub fn with_raw_mutex() -> Self { Self::make(State::Empty, Mutex::with_raw_mutex(List::new())) } } + + loom_const_fn! { + /// Returns a new `WaitQueue` with a single stored wakeup. + /// + /// The first call to [`wait`] on this queue will immediately succeed. + /// + /// [`wait`]: Self::wait + #[must_use] + pub(crate) fn new_woken_with_raw_mutex() -> Self { + Self::make(State::Woken, Mutex::with_raw_mutex(List::new())) + } + } } impl WaitQueue From 2cea7c13d45ecd3512581e1967bb9824c8bf634b Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 27 Jul 2024 12:03:31 -0700 Subject: [PATCH 20/41] fix loom tests not compiling with no-default-features --- maitake-sync/src/lib.rs | 2 +- maitake-sync/src/loom.rs | 20 ++++++++++++++++++-- maitake-sync/src/mutex.rs | 3 +-- maitake-sync/src/wait_map/tests/loom.rs | 1 + maitake-sync/src/wait_queue.rs | 19 +++++++++---------- maitake-sync/src/wait_queue/tests/loom.rs | 2 ++ 6 files changed, 32 insertions(+), 15 deletions(-) diff --git a/maitake-sync/src/lib.rs b/maitake-sync/src/lib.rs index c8b341d6..e30b71a2 100644 --- a/maitake-sync/src/lib.rs +++ b/maitake-sync/src/lib.rs @@ -5,7 +5,7 @@ #![cfg_attr(feature = "core-error", feature(error_in_core))] #![warn(missing_docs, missing_debug_implementations)] -#[cfg(feature = "alloc")] +#[cfg(any(feature = "alloc", test))] extern crate alloc; pub(crate) mod loom; diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index 143c47c7..25c992a4 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -76,10 +76,12 @@ mod inner { use core::{ marker::PhantomData, ops::{Deref, DerefMut}, - panic::Location, }; - use alloc::fmt; + #[cfg(feature = "tracing")] + use core::panic::Location; + + use core::fmt; /// Mock version of mycelium's spinlock, but using /// `loom::sync::Mutex`. The API is slightly different, since the @@ -91,6 +93,7 @@ mod inner { pub(crate) struct MutexGuard<'a, T, Lock = crate::spin::Spinlock> { guard: loom::sync::MutexGuard<'a, T>, + #[cfg(feature = "tracing")] location: &'static Location<'static>, _p: PhantomData, } @@ -115,19 +118,25 @@ mod inner { #[track_caller] pub fn try_lock(&self) -> Option> { + #[cfg(feature = "tracing")] let location = Location::caller(); + #[cfg(feature = "tracing")] tracing::debug!(%location, "Mutex::try_lock"); match self.0.try_lock() { Ok(guard) => { + #[cfg(feature = "tracing")] tracing::debug!(%location, "Mutex::try_lock -> locked!"); Some(MutexGuard { guard, + + #[cfg(feature = "tracing")] location, _p: PhantomData, }) } Err(_) => { + #[cfg(feature = "tracing")] tracing::debug!(%location, "Mutex::try_lock -> already locked"); None } @@ -136,7 +145,10 @@ mod inner { #[track_caller] pub fn lock(&self) -> MutexGuard<'_, T, Lock> { + #[cfg(feature = "tracing")] let location = Location::caller(); + + #[cfg(feature = "tracing")] tracing::debug!(%location, "Mutex::lock"); let guard = self @@ -144,11 +156,14 @@ mod inner { .lock() .map(|guard| MutexGuard { guard, + + #[cfg(feature = "tracing")] location, _p: PhantomData, }) .expect("loom mutex will never poison"); + #[cfg(feature = "tracing")] tracing::debug!(%location, "Mutex::lock -> locked"); guard } @@ -184,6 +199,7 @@ mod inner { impl Drop for MutexGuard<'_, T, Lock> { #[track_caller] fn drop(&mut self) { + #[cfg(feature = "tracing")] tracing::debug!( location.dropped = %Location::caller(), location.locked = %self.location, diff --git a/maitake-sync/src/mutex.rs b/maitake-sync/src/mutex.rs index 1dc617c5..1eb85ea9 100644 --- a/maitake-sync/src/mutex.rs +++ b/maitake-sync/src/mutex.rs @@ -16,7 +16,6 @@ use core::{ pin::Pin, task::{Context, Poll}, }; -use mutex_traits::ConstInit; use pin_project::pin_project; #[cfg(test)] mod tests; @@ -202,7 +201,7 @@ impl Mutex { #[cfg(not(loom))] impl Mutex where - L: ScopedRawMutex + ConstInit, + L: ScopedRawMutex + mutex_traits::ConstInit, { /// Returns a new `Mutex` protecting the provided `data`, using the provided /// [`ScopedRawMutex`] implementation as the raw mutex. diff --git a/maitake-sync/src/wait_map/tests/loom.rs b/maitake-sync/src/wait_map/tests/loom.rs index 11f72dc8..46602b0e 100644 --- a/maitake-sync/src/wait_map/tests/loom.rs +++ b/maitake-sync/src/wait_map/tests/loom.rs @@ -50,6 +50,7 @@ fn wake_two_sequential() { } #[test] +#[cfg(feature = "alloc")] fn wake_close() { use ::alloc::sync::Arc; diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index 9c83be88..391e8a13 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -415,16 +415,15 @@ where } } - loom_const_fn! { - /// Returns a new `WaitQueue` with a single stored wakeup. - /// - /// The first call to [`wait`] on this queue will immediately succeed. - /// - /// [`wait`]: Self::wait - #[must_use] - pub(crate) fn new_woken_with_raw_mutex() -> Self { - Self::make(State::Woken, Mutex::with_raw_mutex(List::new())) - } + /// Returns a new `WaitQueue` with a single stored wakeup. + /// + /// The first call to [`wait`] on this queue will immediately succeed. + /// + /// [`wait`]: Self::wait + #[must_use] + #[cfg(not(loom))] + pub(crate) const fn new_woken_with_raw_mutex() -> Self { + Self::make(State::Woken, Mutex::with_raw_mutex(List::new())) } } diff --git a/maitake-sync/src/wait_queue/tests/loom.rs b/maitake-sync/src/wait_queue/tests/loom.rs index eb13d1a0..d096c313 100644 --- a/maitake-sync/src/wait_queue/tests/loom.rs +++ b/maitake-sync/src/wait_queue/tests/loom.rs @@ -43,6 +43,7 @@ fn wake_all_sequential() { } #[test] +#[cfg(feature = "alloc")] fn wake_all_concurrent() { use alloc::sync::Arc; // must be higher than the number of threads in a `WakeBatch`, but below @@ -108,6 +109,7 @@ fn wake_all_reregistering() { } #[test] +#[cfg(feature = "alloc")] fn wake_close() { use alloc::sync::Arc; From c7b9c0b204288372ded06fa7673c110f57502b13 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sun, 28 Jul 2024 09:04:54 -0700 Subject: [PATCH 21/41] sketch out `DefaultMutex` --- Cargo.lock | 7 + maitake-sync/Cargo.toml | 2 + maitake-sync/src/blocking.rs | 2 + maitake-sync/src/blocking/default_mutex.rs | 199 +++++++++++++++++++++ maitake-sync/src/lib.rs | 2 +- 5 files changed, 211 insertions(+), 1 deletion(-) create mode 100644 maitake-sync/src/blocking/default_mutex.rs diff --git a/Cargo.lock b/Cargo.lock index c38d4902..31960cd1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -623,6 +623,12 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "critical-section" +version = "1.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7059fff8937831a9ae6f0fe4d658ffabf58f2ca96aa9dec1c889f936f705f216" + [[package]] name = "crossbeam-channel" version = "0.5.11" @@ -1438,6 +1444,7 @@ name = "maitake-sync" version = "0.1.2" dependencies = [ "cordyceps", + "critical-section", "futures", "futures-util", "loom", diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index 53b0cfa1..0a604265 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -23,6 +23,7 @@ rust-version = "1.61.0" [features] default = ["alloc"] alloc = ["cordyceps/alloc"] +std = ["alloc"] no-cache-pad = ["cordyceps/no-cache-pad"] core-error = [] @@ -30,6 +31,7 @@ core-error = [] mycelium-bitfield = { version = "0.1.3", path = "../bitfield" } mutex-traits = "0.999.0" cordyceps = { version = "0.3.0", path = "../cordyceps" } +critical-section = { version = "1.1", optional = true } pin-project = "1" portable-atomic = "1.2" tracing = { version = "0.1", default_features = false, optional = true } diff --git a/maitake-sync/src/blocking.rs b/maitake-sync/src/blocking.rs index 85730803..1e2f0a29 100644 --- a/maitake-sync/src/blocking.rs +++ b/maitake-sync/src/blocking.rs @@ -150,8 +150,10 @@ //! [`lock_api`]: https://crates.io/crates/lock_api //! [`critical-section`]: https://crates.io/crates/critical-section //! [`std::sync::Mutex`]: https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html +mod default_mutex; pub(crate) mod mutex; pub(crate) mod rwlock; pub use self::{mutex::*, rwlock::*}; +pub use default_mutex::DefaultMutex; pub use mutex_traits::ConstInit; diff --git a/maitake-sync/src/blocking/default_mutex.rs b/maitake-sync/src/blocking/default_mutex.rs new file mode 100644 index 00000000..4941b734 --- /dev/null +++ b/maitake-sync/src/blocking/default_mutex.rs @@ -0,0 +1,199 @@ +//! Default "chef's choice" [`mutex_traits::ScopedRawMutex`] implementation. +//! +//! This type is what users will get when they don't override the `Lock` type +//! parameter for `maitake-sync`'s synchronziation primitives. Therefore, we +//! make a best-effort attempt to Do The Right Thing based on the current +//! feature flag combination. In particular, here's what we currently give you: +//! +//! - **If `cfg(loom)` is enabled, then the `DefaultMutex` is a `loom` mutex** +//! so that `maitake-sync` primitives work nicely in `loom` tests +//! +//! - **If the `std` feature is enabled, then the `DefaultMutex` is a +//! `std::sync::Mutex`**, so that `std` users get an OS mutex rather than a +//! spinlock. +//! +//! - **If the `critical-section` feature is enabled, then the `DefaultMutex` is +//! a spinlock that acquires a critical section once locked.**. This ensures +//! that bare-metal users who have enabled `critical-section` get a mutex that +//! disables IRQs when locked. +//! +//! - **Otherwise, the `DefaultMutex` is a spinlock**. This is the default +//! behavior and will at least work on all platforms, but may not be the most +//! efficient, and may not be IRQ-safe. +//! +//! +//! # Notes +//! +//! - The `DefaultMutex` cannot ever implement `RawMutex`, only +//! `ScopedRawMutex`. This is because it's impossible for a +//! `critical-section`-based implementation to implement `RawMutex`, due to +//! [`critical-section`'s safety requirements][cs-reqs], which we can't uphold +//! in a RAII situation with multiple locks. If we implemented `RawMutex` for +//! the non-`critical-section` implementations, then the `critical-section` +//! feature flag would *take away* methods that would otherwise be available, +//! making it non-additive, which is a BIG NO-NO for feature flags. +//! +//! - On the other hand, it *is* okay to have `cfg(loom)` not implement +//! `ConstInit` where every other implementation does. This is because `cfg(loom)` +//! is a `RUSTFLAGS` cfg rather than a feature flag, and therefore can only be +//! enabled by the top-level build. It can't be enabled by a dependency and +//! suddenly make your code not compile. Loom users are already used to stuff +//! being const-initializable in real life, but not in loom tests, so this is +//! more okay. +//! +//! [cs-reqs]: https://docs.rs/critical-section/latest/critical_section/fn.acquire.html#safety +#[cfg(loom)] +pub use loom_impl::DefaultMutex; + +#[cfg(all(not(loom), feature = "std"))] +pub use std_impl::DefaultMutex; + +#[cfg(all(not(loom), not(feature = "std"), feature = "critical-section"))] +pub use cs_impl::DefaultMutex; + +#[cfg(all(not(loom), not(feature = "std"), not(feature = "critical-section")))] +pub use spin_impl::DefaultMutex; + +#[cfg(loom)] +mod loom_impl { + #[cfg(feature = "tracing")] + use core::panic::Location; + use mutex_traits::ScopedRawMutex; + + pub struct DefaultMutex(loom::sync::Mutex<()>); + + unsafe impl ScopedRawMutex for DefaultMutex { + #[track_caller] + fn with_lock(&self, f: impl FnOnce() -> R) -> R { + #[cfg(feature = "tracing")] + let location = Location::caller(); + #[cfg(feature = "tracing")] + tracing::debug!(%location, "DefaultMutex::with_lock()"); + + let guard = self.0.lock(); + tracing::debug!(%location, "DefaultMutex::with_lock() -> locked"); + + let result = f(); + drop(guard); + + #[cfg(feature = "tracing")] + tracing::debug!(%location, "DefaultMutex::with_lock() -> unlocked"); + + result + } + + #[track_caller] + fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { + #[cfg(feature = "tracing")] + let location = Location::caller(); + #[cfg(feature = "tracing")] + tracing::debug!(%location, "DefaultMutex::try_with_lock()"); + + match self.0.try_lock() { + Ok(guard) => { + #[cfg(feature = "tracing")] + tracing::debug!(%location, "DefaultMutex::try_with_lock() -> locked"); + + let result = f(); + drop(guard); + + #[cfg(feature = "tracing")] + tracing::debug!(%location, "DefaultMutex::try_with_lock() -> unlocked"); + + Some(result) + } + None => { + #[cfg(feature = "tracing")] + tracing::debug!(%location, "DefaultMutex::try_with_lock() -> already locked"); + + None + } + } + } + + fn is_locked(&self) -> bool { + self.0.try_lock().is_none() + } + } +} + +#[cfg(all(not(loom), feature = "std"))] +mod std_impl { + use mutex_traits::ScopedRawMutex; + + pub struct DefaultMutex(std::sync::Mutex<()>); + + unsafe impl ScopedRawMutex for DefaultMutex { + #[track_caller] + #[inline] + fn with_lock(&self, f: impl FnOnce() -> R) -> R { + let _guard = self.0.lock().unwrap(); + f() + } + + #[track_caller] + fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { + let _guard = self.0.try_lock().ok()?; + Some(f()) + } + + #[inline] + fn is_locked(&self) -> bool { + self.0.try_lock().is_ok() + } + } +} + +#[cfg(all(not(loom), not(feature = "std"), feature = "critical-section"))] +mod cs_impl { + use crate::spin::Spinlock; + use mutex_traits::ScopedRawMutex; + + pub struct DefaultMutex(Spinlock); + + unsafe impl ScopedRawMutex for DefaultMutex { + #[track_caller] + #[inline] + fn with_lock(&self, f: impl FnOnce() -> R) -> R { + self.0.with_lock(|| critical_section::with(|| f())) + } + + #[track_caller] + #[inline] + fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { + self.0.try_with_lock(|| critical_section::with(|| f())) + } + + #[inline] + fn is_locked(&self) -> bool { + self.0.is_locked() + } + } +} + +#[cfg(all(not(loom), not(feature = "std"), not(feature = "critical-section")))] +mod spin_impl { + use crate::spin::Spinlock; + use mutex_traits::ScopedRawMutex; + + pub struct DefaultMutex(Spinlock); + + unsafe impl ScopedRawMutex for DefaultMutex { + #[track_caller] + #[inline] + fn with_lock(&self, f: impl FnOnce() -> R) -> R { + self.0.with_lock(|| critical_section::with(|| f())) + } + + #[track_caller] + #[inline] + fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { + self.0.try_with_lock(|| critical_section::with(|| f())) + } + + #[inline] + fn is_locked(&self) -> bool { + self.0.is_locked() + } + } +} diff --git a/maitake-sync/src/lib.rs b/maitake-sync/src/lib.rs index e30b71a2..6354efbb 100644 --- a/maitake-sync/src/lib.rs +++ b/maitake-sync/src/lib.rs @@ -1,7 +1,7 @@ #![doc = include_str!("../README.md")] #![cfg_attr(docsrs, feature(doc_cfg, doc_auto_cfg, doc_cfg_hide))] #![cfg_attr(docsrs, doc(cfg_hide(docsrs, loom)))] -#![cfg_attr(not(test), no_std)] +#![cfg_attr(not(any(test, feature = "std")), no_std)] #![cfg_attr(feature = "core-error", feature(error_in_core))] #![warn(missing_docs, missing_debug_implementations)] From 1da40956ba4f16e0337869e014f99a72859979da Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sun, 28 Jul 2024 09:35:10 -0700 Subject: [PATCH 22/41] add `DefaultMutex` tests and fix some broken stuff --- maitake-sync/src/blocking/default_mutex.rs | 107 +++++++++++++++++++-- maitake-sync/src/loom.rs | 2 +- 2 files changed, 101 insertions(+), 8 deletions(-) diff --git a/maitake-sync/src/blocking/default_mutex.rs b/maitake-sync/src/blocking/default_mutex.rs index 4941b734..c2b4f3fb 100644 --- a/maitake-sync/src/blocking/default_mutex.rs +++ b/maitake-sync/src/blocking/default_mutex.rs @@ -60,6 +60,7 @@ mod loom_impl { use core::panic::Location; use mutex_traits::ScopedRawMutex; + #[derive(Debug, Default)] pub struct DefaultMutex(loom::sync::Mutex<()>); unsafe impl ScopedRawMutex for DefaultMutex { @@ -119,10 +120,32 @@ mod loom_impl { #[cfg(all(not(loom), feature = "std"))] mod std_impl { - use mutex_traits::ScopedRawMutex; + use mutex_traits::{ConstInit, ScopedRawMutex}; + #[derive(Debug)] + #[must_use] pub struct DefaultMutex(std::sync::Mutex<()>); + impl DefaultMutex { + #[inline] + pub const fn new() -> Self { + Self(std::sync::Mutex::new(())) + } + } + + impl ConstInit for DefaultMutex { + // As is traditional, clippy is wrong about this. + #[allow(clippy::declare_interior_mutable_const)] + const INIT: Self = Self::new(); + } + + impl Default for DefaultMutex { + #[inline] + fn default() -> Self { + Self::new() + } + } + unsafe impl ScopedRawMutex for DefaultMutex { #[track_caller] #[inline] @@ -147,21 +170,40 @@ mod std_impl { #[cfg(all(not(loom), not(feature = "std"), feature = "critical-section"))] mod cs_impl { use crate::spin::Spinlock; - use mutex_traits::ScopedRawMutex; + use mutex_traits::{ConstInit, ScopedRawMutex}; + #[derive(Debug)] pub struct DefaultMutex(Spinlock); + impl DefaultMutex { + #[inline] + pub const fn new() -> Self { + Self(Spinlock::new()) + } + } + + impl ConstInit for DefaultMutex { + const INIT: Self = Self::new(); + } + + impl Default for DefaultMutex { + #[inline] + fn default() -> Self { + Self::new() + } + } + unsafe impl ScopedRawMutex for DefaultMutex { #[track_caller] #[inline] fn with_lock(&self, f: impl FnOnce() -> R) -> R { - self.0.with_lock(|| critical_section::with(|| f())) + self.0.with_lock(|| critical_section::with(|_cs| f())) } #[track_caller] #[inline] fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { - self.0.try_with_lock(|| critical_section::with(|| f())) + self.0.try_with_lock(|| critical_section::with(|_cs| f())) } #[inline] @@ -174,21 +216,40 @@ mod cs_impl { #[cfg(all(not(loom), not(feature = "std"), not(feature = "critical-section")))] mod spin_impl { use crate::spin::Spinlock; - use mutex_traits::ScopedRawMutex; + use mutex_traits::{ConstInit, ScopedRawMutex}; + #[derive(Debug)] pub struct DefaultMutex(Spinlock); + impl DefaultMutex { + #[inline] + pub const fn new() -> Self { + Self(Spinlock::new()) + } + } + + impl ConstInit for DefaultMutex { + const INIT: Self = Self::new(); + } + + impl Default for DefaultMutex { + #[inline] + fn default() -> Self { + Self::new() + } + } + unsafe impl ScopedRawMutex for DefaultMutex { #[track_caller] #[inline] fn with_lock(&self, f: impl FnOnce() -> R) -> R { - self.0.with_lock(|| critical_section::with(|| f())) + self.0.with_lock(f) } #[track_caller] #[inline] fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { - self.0.try_with_lock(|| critical_section::with(|| f())) + self.0.try_with_lock(f) } #[inline] @@ -197,3 +258,35 @@ mod spin_impl { } } } + +#[cfg(test)] +mod test { + use super::DefaultMutex; + + // Check that a `DefaultMutex` will always implement the traits we expect it + // to. + #[test] + fn default_mutex_trait_impls() { + fn assert_scoped_raw_mutex() {} + fn assert_send_and_sync() {} + fn assert_default() {} + fn assert_debug() {} + + assert_scoped_raw_mutex::(); + assert_send_and_sync::(); + assert_default::(); + assert_debug::(); + } + + // Check that a non-`loom` `DefaultMutex` has a const-fn constructor, and + // implements `ConstInit`. + #[cfg(not(loom))] + #[test] + fn const_constructor() { + fn assert_const_init() {} + + assert_const_init::(); + + static _MY_COOL_MUTEX: DefaultMutex = DefaultMutex::new(); + } +} diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index 25c992a4..fa8114e9 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -215,7 +215,7 @@ mod inner { mod inner { #![allow(dead_code, unused_imports)] pub(crate) mod sync { - #[cfg(feature = "alloc")] + #[cfg(any(feature = "alloc", test))] pub use alloc::sync::*; pub use core::sync::*; From 28e3226b4d586897090d4448cffa78188bf7eeee Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sun, 28 Jul 2024 11:29:51 -0700 Subject: [PATCH 23/41] plumbing, docs, and redo constructors --- maitake-sync/README.md | 18 +- maitake-sync/src/blocking.rs | 22 +- maitake-sync/src/blocking/default_mutex.rs | 323 ++++++++++++++------- maitake-sync/src/blocking/mutex.rs | 93 +++--- maitake-sync/src/loom.rs | 4 +- maitake-sync/src/mutex.rs | 69 ++--- maitake-sync/src/rwlock.rs | 40 ++- maitake-sync/src/semaphore.rs | 50 ++-- maitake-sync/src/spin.rs | 32 +- maitake-sync/src/wait_map.rs | 79 ++--- maitake-sync/src/wait_queue.rs | 68 ++--- 11 files changed, 435 insertions(+), 363 deletions(-) diff --git a/maitake-sync/README.md b/maitake-sync/README.md index ee005958..2cf49289 100644 --- a/maitake-sync/README.md +++ b/maitake-sync/README.md @@ -163,11 +163,13 @@ In addition to async locks, `maitake-sync` also provides a [`blocking`] module, which contains blocking [`blocking::Mutex`] and [`blocking::RwLock`] types. Many of `maitake-sync`'s async synchronization primitives, including [`WaitQueue`], [`Mutex`], [`RwLock`], and [`Semaphore`], internally use the [`blocking::Mutex`] -type for wait-list synchronization. By default, the [`blocking::Mutex`] type is -implemented using an atomic [spinlock]. +type for wait-list synchronization. By default, this type uses a +[`blocking::DefaultMutex`][`DefaultMutex`] as the underlying mutex +implementation, which attempts to provide the best generic mutex implementation +based on the currently enabled feature flags. -However, for many applications, a generic spinlock is not the ideal blocking -mechanism. Therefore, `maitake-sync`'s [`blocking::Mutex`] type, and the +However, in some cases, it may be desirable to provide a custom mutex +implementation. Therefore, `maitake-sync`'s [`blocking::Mutex`] type, and the async synchronization primitives that depend on it, are generic over a `Lock` type parameter which may be overridden using the [`RawMutex`] and [`ScopedRawMutex`] traits from the [`mutex-traits`] crate, allowing alternative @@ -177,7 +179,7 @@ also be used with raw mutex implementations that implement traits from the [`lock_api`] and [`critical-section`] crates. See [the documentation on overriding mutex implementations][overriding] for more -details. +details. [`blocking`]: https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html @@ -185,6 +187,8 @@ details. https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.Mutex.html [`blocking::RwLock`]: https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.RwLock.html +[`DefaultMutex`]: + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.DefaultMutex.html [spinlock]: https://en.wikipedia.org/wiki/Spinlock [`RawMutex`]: https://docs.rs/mutex-traitsc/latest/mutex_traits/trait.RawMutex.html @@ -194,7 +198,7 @@ details. [`lock_api`]: https://crates.io/crates/lock_api [`critical-section`]: https://crates.io/crates/critical-section [overriding]: - https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html#overriding-mutex-implementations + https://docs.rs/maitake-sync/latest/maitake_sync/blocking/index.html#overriding-mutex-implementations ## features @@ -205,6 +209,8 @@ The following features are available (this list is incomplete; you can help by [ | Feature | Default | Explanation | | :--- | :--- | :--- | | `alloc` | `true` | Enables [`liballoc`] dependency | +| `std` | `false` | Enables the Rust standard library, disabling `#![no-std]`. This implies the `alloc` feature. | +| `critical-section` | `false` | Enables a variant of the [`DefaultMutex`] type that uses the [`critical-section`] crate. | | `no-cache-pad` | `false` | Inhibits cache padding for the [`CachePadded`] struct. When this feature is NOT enabled, the size will be determined based on target platform. | | `tracing` | `false` | Enables support for [`tracing`] diagnostics. Requires `liballoc`.| | `core-error` | `false` | Enables implementations of the [`core::error::Error` trait][core-error] for `maitake-sync`'s error types. *Requires a nightly Rust toolchain*. | diff --git a/maitake-sync/src/blocking.rs b/maitake-sync/src/blocking.rs index 1e2f0a29..49039bf4 100644 --- a/maitake-sync/src/blocking.rs +++ b/maitake-sync/src/blocking.rs @@ -24,12 +24,14 @@ //! //! # overriding mutex implementations //! -//! By default, the [`Mutex`] and [`RwLock`] types are implemented using simple -//! _[spinlocks]_, which wait for the lock to become available by _spinning_: -//! repeatedly checking an atomic value in a loop, executing [spin-loop hint -//! instructions] until the lock value changes. These spinlock implementations -//! are represented by the [`Spinlock`] and [`RwSpinlock`] types in the -//! [`spin`](crate::spin) module. +//! By default, the [`Mutex`] type uses a [`DefaultMutex`] as the underlying +//! blocking strategy. This type attempts to choose a suitable implementation +//! for the blocking mutex based on the currently available [feature flags]. When +//! the `std` feature is not enabled, this is typically a _[spinlock]_, which +//! waits for the lock to become available by _spinning_: repeatedly checking an +//! atomic value in a loop, executing [spin-loop hint instructions] until the +//! lock value changes. These spinlock implementations are represented by the +//! [`Spinlock`] and [`RwSpinlock`] types in the [`spin`](crate::spin) module. //! //! Spinlocks are simple to implement and, thanks to the Rust standard library //! abstracting over atomic operations, portable. The default spinlock will work @@ -133,7 +135,7 @@ //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion //! [reader-writer lock]: //! https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock -//! [spinlocks]: https://en.wikipedia.org/wiki/Spinlock +//! [spinlock]: https://en.wikipedia.org/wiki/Spinlock //! [spin-loop hint instructions]: core::hint::spin_loop //! [`Spinlock`]: crate::spin::Spinlock //! [`RwSpinlock`]: crate::spin::RwSpinlock @@ -149,11 +151,15 @@ //! [`mutex` crate]: https://crates.io/crates/mutex //! [`lock_api`]: https://crates.io/crates/lock_api //! [`critical-section`]: https://crates.io/crates/critical-section -//! [`std::sync::Mutex`]: https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html +//! [`std::sync::Mutex`]: +//! https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html +//! [feature flags]: crate#features mod default_mutex; pub(crate) mod mutex; pub(crate) mod rwlock; pub use self::{mutex::*, rwlock::*}; + pub use default_mutex::DefaultMutex; + pub use mutex_traits::ConstInit; diff --git a/maitake-sync/src/blocking/default_mutex.rs b/maitake-sync/src/blocking/default_mutex.rs index c2b4f3fb..30dbcbd3 100644 --- a/maitake-sync/src/blocking/default_mutex.rs +++ b/maitake-sync/src/blocking/default_mutex.rs @@ -1,26 +1,7 @@ -//! Default "chef's choice" [`mutex_traits::ScopedRawMutex`] implementation. +//! Default "chef's choice" [`ScopedRawMutex`] implementation. //! //! This type is what users will get when they don't override the `Lock` type -//! parameter for `maitake-sync`'s synchronziation primitives. Therefore, we -//! make a best-effort attempt to Do The Right Thing based on the current -//! feature flag combination. In particular, here's what we currently give you: -//! -//! - **If `cfg(loom)` is enabled, then the `DefaultMutex` is a `loom` mutex** -//! so that `maitake-sync` primitives work nicely in `loom` tests -//! -//! - **If the `std` feature is enabled, then the `DefaultMutex` is a -//! `std::sync::Mutex`**, so that `std` users get an OS mutex rather than a -//! spinlock. -//! -//! - **If the `critical-section` feature is enabled, then the `DefaultMutex` is -//! a spinlock that acquires a critical section once locked.**. This ensures -//! that bare-metal users who have enabled `critical-section` get a mutex that -//! disables IRQs when locked. -//! -//! - **Otherwise, the `DefaultMutex` is a spinlock**. This is the default -//! behavior and will at least work on all platforms, but may not be the most -//! efficient, and may not be IRQ-safe. -//! +//! parameter for `maitake-sync`'s synchronziation primitives. //! //! # Notes //! @@ -41,71 +22,237 @@ //! being const-initializable in real life, but not in loom tests, so this is //! more okay. //! -//! [cs-reqs]: https://docs.rs/critical-section/latest/critical_section/fn.acquire.html#safety +//! [cs-reqs]: +//! https://docs.rs/critical-section/latest/critical_section/fn.acquire.html#safety +#[cfg(not(loom))] +use super::ConstInit; +use super::ScopedRawMutex; + +/// Default, best-effort [`ScopedRawMutex`] implementation. +/// +/// This is the default `Lock` type parameter for the [`Mutex`](crate::Mutex) +/// type, and for the async synchronization primitives that use the blocking +/// `Mutex`. This type makes a best-effort attempt to Do The Right Thing based +/// on the currently enabled [feature flags]. In particular, here's what we +/// currently give you: +/// +/// - **If `cfg(loom)` is enabled, then the `DefaultMutex` is a [`loom` mutex]** +/// so that `maitake-sync` primitives work nicely in `loom` tests +/// +/// - **If the `std` feature is enabled, then the `DefaultMutex` is a +/// [`std::sync::Mutex`]**, so that `std` users get an OS mutex rather than a +/// spinlock. +/// +/// - **If the `critical-section` feature is enabled, then the `DefaultMutex` is +/// a spinlock that [acquires a critical section][cs] once locked**. This +/// ensures that bare-metal users who have enabled `critical-section` get a +/// mutex that disables IRQs when locked. +/// +/// - **Otherwise, the `DefaultMutex` is a [`Spinlock`]**. This is the default +/// behavior and will at least work on all platforms, but may not be the most +/// efficient, and may not be IRQ-safe. +/// +/// # Notes +/// +/// - Regardless of feature flags, this type implements the [`ScopedRawMutex`] +/// trait, *not* the [`RawMutex`] trait. In order to use methods or types that +/// require a [`RawMutex`], you must [provide your own `RawMutex` +/// type][overriding]. +/// - :warning: If the `critical-section` feature is enabled, you **MUST** +/// provide a `critical-section` implementation. See the [`critical-section` +/// documentation][cs-providing] for details on how to select an +/// implementation. If you don't provide an implementation, you'll get a +/// [linker error][cs-link-err] when compiling your code. +/// - This type has a `const fn new()` constructor and implements the +/// [`ConstInit` trait](super::ConstInit) *except* when `cfg(loom)` is +/// enabled. +/// +/// Loom users are probably already aware that `loom`'s simulated +/// types cannot be const initialized, as they must bind to the *current* test +/// iteration when constructed. This is not a non-additive feature flag, +/// because `loom` support can only be enabled by a `RUSTFLAGS` cfg set by the +/// top-level build, and not by a dependency.`s` +/// +/// [feature flags]: crate#features +/// [`loom` mutex]: https://docs.rs/loom/latest/loom/sync/struct.Mutex.html +/// [cs]: https://docs.rs/critical-section/latest/critical_section/fn.with.html +/// [`Spinlock`]: crate::spin::Spinlock +/// [overriding]: crate::blocking#overriding-mutex-implementations +/// [`RawMutex`]: mutex_traits::RawMutex +/// [cs-providing]: +/// https://docs.rs/critical-section/latest/critical_section/index.html#usage-in-no-std-binaries +/// [cs-link-err]: +/// https://docs.rs/critical-section/latest/critical_section/index.html#undefined-reference-errors +/// +// N.B. that this is a wrapper type around the various impls, rather than just a +// re-export, because I didn't want to duplicate the docs for all the impls... +#[must_use = "why create a `DefaultMutex` if you're not going to lock it?"] +pub struct DefaultMutex(Inner); + +impl DefaultMutex { + loom_const_fn! { + /// Returns a new `DefaultMutex`. + /// + /// See the [type-level documentation](Self) for details on how to use a `DefaultMutex`. + // loom::sync::Mutex`'s constructor captures the location it's + // constructed, so that we can track where it came from in test output. + // That's nice, let's not break it! + #[track_caller] + #[inline] + pub fn new() -> Self { + Self(Inner::new()) + } + } +} + +impl Default for DefaultMutex { + #[track_caller] // again, for Loom Reasons + fn default() -> Self { + Self::new() + } +} + +impl core::fmt::Debug for DefaultMutex { + #[inline] + fn fmt(&self, f: &mut core::fmt::Formatter<'_>) -> core::fmt::Result { + self.0.fmt(f) + } +} + +#[cfg(not(loom))] +impl ConstInit for DefaultMutex { + // As is traditional, clippy is wrong about this. + #[allow(clippy::declare_interior_mutable_const)] + const INIT: Self = Self::new(); +} + +unsafe impl ScopedRawMutex for DefaultMutex { + #[track_caller] + fn with_lock(&self, f: impl FnOnce() -> R) -> R { + self.0.with_lock(f) + } + + #[track_caller] + fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { + self.0.try_with_lock(f) + } + + #[inline] + fn is_locked(&self) -> bool { + self.0.is_locked() + } +} + #[cfg(loom)] -pub use loom_impl::DefaultMutex; +use loom_impl::LoomDefaultMutex as Inner; #[cfg(all(not(loom), feature = "std"))] -pub use std_impl::DefaultMutex; +use std_impl::StdDefaultMutex as Inner; #[cfg(all(not(loom), not(feature = "std"), feature = "critical-section"))] -pub use cs_impl::DefaultMutex; +use cs_impl::CriticalSectionDefaultMutex as Inner; #[cfg(all(not(loom), not(feature = "std"), not(feature = "critical-section")))] -pub use spin_impl::DefaultMutex; +use spin_impl::SpinDefaultMutex as Inner; #[cfg(loom)] mod loom_impl { - #[cfg(feature = "tracing")] + use super::ScopedRawMutex; + #[cfg(any(feature = "tracing", test))] use core::panic::Location; - use mutex_traits::ScopedRawMutex; - #[derive(Debug, Default)] - pub struct DefaultMutex(loom::sync::Mutex<()>); + #[derive(Debug)] + pub(super) struct LoomDefaultMutex(loom::sync::Mutex<()>); - unsafe impl ScopedRawMutex for DefaultMutex { + impl LoomDefaultMutex { + // loom::sync::Mutex`'s constructor captures the location it's + // constructed, so that we can track where it came from in test output. + // That's nice, let's not break it! #[track_caller] + pub(super) fn new() -> Self { + Self(loom::sync::Mutex::new(())) + } + } + + unsafe impl ScopedRawMutex for LoomDefaultMutex { + #[track_caller] + #[inline] fn with_lock(&self, f: impl FnOnce() -> R) -> R { - #[cfg(feature = "tracing")] + #[cfg(any(feature = "tracing", test))] let location = Location::caller(); - #[cfg(feature = "tracing")] - tracing::debug!(%location, "DefaultMutex::with_lock()"); + #[cfg(any(feature = "tracing", test))] + tracing::trace!( + target: "maitake_sync::blocking", + %location, + "DefaultMutex::with_lock()", + ); let guard = self.0.lock(); - tracing::debug!(%location, "DefaultMutex::with_lock() -> locked"); + let _span = tracing::debug_span!( + target: "maitake_sync::blocking", + "locked", + %location, + ) + .entered(); + #[cfg(any(feature = "tracing", test))] + tracing::debug!( + target: "maitake_sync::blocking", + "DefaultMutex::with_lock() -> locked", + ); let result = f(); drop(guard); - #[cfg(feature = "tracing")] - tracing::debug!(%location, "DefaultMutex::with_lock() -> unlocked"); + #[cfg(any(feature = "tracing", test))] + tracing::debug!( + target: "maitake_sync::blocking", + "DefaultMutex::with_lock() -> unlocked", + ); result } #[track_caller] + #[inline] fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { - #[cfg(feature = "tracing")] + #[cfg(any(feature = "tracing", test))] let location = Location::caller(); - #[cfg(feature = "tracing")] - tracing::debug!(%location, "DefaultMutex::try_with_lock()"); + #[cfg(any(feature = "tracing", test))] + tracing::trace!( + target: "maitake_sync::blocking", + %location, + "DefaultMutex::try_with_lock()", + ); match self.0.try_lock() { Ok(guard) => { - #[cfg(feature = "tracing")] - tracing::debug!(%location, "DefaultMutex::try_with_lock() -> locked"); + let _span = + tracing::debug_span!(target: "maitake_sync::blocking", "locked", %location) + .entered(); + #[cfg(any(feature = "tracing", test))] + tracing::debug!( + target: "maitake_sync::blocking", + "DefaultMutex::try_with_lock() -> locked", + ); let result = f(); drop(guard); - #[cfg(feature = "tracing")] - tracing::debug!(%location, "DefaultMutex::try_with_lock() -> unlocked"); + #[cfg(any(feature = "tracing", test))] + tracing::debug!( + target: "maitake_sync::blocking", + "DefaultMutex::try_with_lock() -> unlocked", + ); Some(result) } - None => { - #[cfg(feature = "tracing")] - tracing::debug!(%location, "DefaultMutex::try_with_lock() -> already locked"); + Err(_) => { + #[cfg(any(feature = "tracing", test))] + tracing::debug!( + target: "maitake_sync::blocking", + %location, + "DefaultMutex::try_with_lock() -> already locked", + ); None } @@ -113,54 +260,40 @@ mod loom_impl { } fn is_locked(&self) -> bool { - self.0.try_lock().is_none() + self.0.try_lock().is_err() } } } #[cfg(all(not(loom), feature = "std"))] mod std_impl { - use mutex_traits::{ConstInit, ScopedRawMutex}; - + use super::ScopedRawMutex; #[derive(Debug)] #[must_use] - pub struct DefaultMutex(std::sync::Mutex<()>); + pub(super) struct StdDefaultMutex(std::sync::Mutex<()>); - impl DefaultMutex { + impl StdDefaultMutex { #[inline] - pub const fn new() -> Self { + pub(super) const fn new() -> Self { Self(std::sync::Mutex::new(())) } } - impl ConstInit for DefaultMutex { - // As is traditional, clippy is wrong about this. - #[allow(clippy::declare_interior_mutable_const)] - const INIT: Self = Self::new(); - } - - impl Default for DefaultMutex { - #[inline] - fn default() -> Self { - Self::new() - } - } - - unsafe impl ScopedRawMutex for DefaultMutex { + unsafe impl ScopedRawMutex for StdDefaultMutex { #[track_caller] - #[inline] + #[inline(always)] fn with_lock(&self, f: impl FnOnce() -> R) -> R { let _guard = self.0.lock().unwrap(); f() } #[track_caller] + #[inline(always)] fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { let _guard = self.0.try_lock().ok()?; Some(f()) } - #[inline] fn is_locked(&self) -> bool { self.0.try_lock().is_ok() } @@ -169,39 +302,28 @@ mod std_impl { #[cfg(all(not(loom), not(feature = "std"), feature = "critical-section"))] mod cs_impl { + use super::ScopedRawMutex; use crate::spin::Spinlock; - use mutex_traits::{ConstInit, ScopedRawMutex}; #[derive(Debug)] - pub struct DefaultMutex(Spinlock); + pub(super) struct CriticalSectionDefaultMutex(Spinlock); - impl DefaultMutex { + impl CriticalSectionDefaultMutex { #[inline] - pub const fn new() -> Self { + pub(super) const fn new() -> Self { Self(Spinlock::new()) } } - impl ConstInit for DefaultMutex { - const INIT: Self = Self::new(); - } - - impl Default for DefaultMutex { - #[inline] - fn default() -> Self { - Self::new() - } - } - - unsafe impl ScopedRawMutex for DefaultMutex { + unsafe impl ScopedRawMutex for CriticalSectionDefaultMutex { #[track_caller] - #[inline] + #[inline(always)] fn with_lock(&self, f: impl FnOnce() -> R) -> R { self.0.with_lock(|| critical_section::with(|_cs| f())) } #[track_caller] - #[inline] + #[inline(always)] fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { self.0.try_with_lock(|| critical_section::with(|_cs| f())) } @@ -215,44 +337,33 @@ mod cs_impl { #[cfg(all(not(loom), not(feature = "std"), not(feature = "critical-section")))] mod spin_impl { + use super::ScopedRawMutex; use crate::spin::Spinlock; - use mutex_traits::{ConstInit, ScopedRawMutex}; #[derive(Debug)] - pub struct DefaultMutex(Spinlock); + pub(super) struct SpinDefaultMutex(Spinlock); - impl DefaultMutex { + impl SpinDefaultMutex { #[inline] - pub const fn new() -> Self { + pub(super) const fn new() -> Self { Self(Spinlock::new()) } } - impl ConstInit for DefaultMutex { - const INIT: Self = Self::new(); - } - - impl Default for DefaultMutex { - #[inline] - fn default() -> Self { - Self::new() - } - } - - unsafe impl ScopedRawMutex for DefaultMutex { + unsafe impl ScopedRawMutex for SpinDefaultMutex { #[track_caller] - #[inline] + #[inline(always)] fn with_lock(&self, f: impl FnOnce() -> R) -> R { self.0.with_lock(f) } #[track_caller] - #[inline] + #[inline(always)] fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { self.0.try_with_lock(f) } - #[inline] + #[inline(always)] fn is_locked(&self) -> bool { self.0.is_locked() } diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 106a3444..03bfdb08 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -1,6 +1,6 @@ use crate::{ + blocking::DefaultMutex, loom::cell::{MutPtr, UnsafeCell}, - spin::Spinlock, util::fmt, }; use core::ops::{Deref, DerefMut}; @@ -21,8 +21,8 @@ pub use mutex_traits::{RawMutex, ScopedRawMutex}; /// # Overriding mutex implementations /// /// This type is generic over a `Lock` type parameter which represents a raw -/// mutex implementation. By default, this is a [`Spinlock`]. To construct a new -/// `Mutex` with an alternative raw mutex implementation, use the +/// mutex implementation. By default, this is the [`DefaultMutex`]. To construct +/// a new `Mutex` with an alternative raw mutex implementation, use the /// [`Mutex::with_raw_mutex`] cosntructor. See the [module-level documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for @@ -35,6 +35,10 @@ pub use mutex_traits::{RawMutex, ScopedRawMutex}; /// [`ScopedRawMutex`], the [`Mutex`] type provides only the scoped /// [`with_lock`] and [`try_with_lock`] methods. /// +/// :warning: Note that [`DefaultMutex`] does *not* implement `RawMutex`, so +/// using the [`lock`] and [`try_lock`] RAII API requires selecting an +/// alternative [`RawMutex`] implementation. +/// /// # Loom-specific behavior /// /// When `cfg(loom)` is enabled, this mutex will use Loom's simulated atomics, @@ -45,7 +49,7 @@ pub use mutex_traits::{RawMutex, ScopedRawMutex}; /// [`with_lock`]: Mutex::with_lock /// [`try_with_lock`]: Mutex::try_with_lock /// [`std::sync::Mutex`]: https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html -pub struct Mutex { +pub struct Mutex { lock: Lock, data: UnsafeCell, } @@ -62,7 +66,7 @@ pub struct Mutex { /// [`lock`]: Mutex::lock /// [`try_lock`]: Mutex::try_lock #[must_use = "if unused, the `Mutex` will immediately unlock"] -pub struct MutexGuard<'a, T, Lock: RawMutex = Spinlock> { +pub struct MutexGuard<'a, T, Lock: RawMutex> { ptr: MutPtr, lock: &'a Lock, } @@ -73,27 +77,31 @@ impl Mutex { /// /// The returned `Mutex` is in an unlocked state, ready for use. /// + /// This constructor returns a mutex that uses the [`DefaultMutex`] + /// implementation. To use an alternative `RawMutex` type, use the + /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. + /// /// # Examples /// /// ``` - /// use maitake_sync::spin::Mutex; + /// use maitake_sync::blocking::Mutex; /// /// let mutex = Mutex::new(0); /// ``` #[must_use] pub fn new(data: T) -> Self { Self { - lock: Spinlock::new(), + lock: DefaultMutex::new(), data: UnsafeCell::new(data), } } } } -impl Mutex { +impl Mutex { loom_const_fn! { - /// Returns a new `Mutex` protecting the provided `data`, using the - /// `Lock` type parameter as the raw mutex implementation. + /// Returns a new `Mutex` protecting the provided `data`, using + /// `lock` type parameter as the raw mutex implementation. /// /// See the [module-level documentation on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for @@ -101,13 +109,41 @@ impl Mutex { /// /// The returned `Mutex` is in an unlocked state, ready for use. #[must_use] - pub fn with_raw_mutex(data: T) -> Self { + pub fn with_raw_mutex(data: T, lock: Lock) -> Self { Self { - lock: Lock::INIT, + lock, data: UnsafeCell::new(data), } } } + + /// Consumes this `Mutex`, returning the guarded data. + #[inline] + #[must_use] + pub fn into_inner(self) -> T { + self.data.into_inner() + } + + /// Returns a mutable reference to the underlying data. + /// + /// Since this call borrows the `Mutex` mutably, no actual locking needs to + /// take place -- the mutable borrow statically guarantees no locks exist. + /// + /// # Examples + /// + /// ``` + /// let mut lock = maitake_sync::blocking::Mutex::new(0); + /// lock.with_lock(|data| *data = 10); + /// assert_eq!(*lock.get_mut(), 10); + /// ``` + pub fn get_mut(&mut self) -> &mut T { + unsafe { + // Safety: since this call borrows the `Mutex` mutably, no actual + // locking needs to take place -- the mutable borrow statically + // guarantees no locks exist. + self.data.with_mut(|data| &mut *data) + } + } } impl Mutex { @@ -246,34 +282,6 @@ where pub unsafe fn force_unlock(&self) { self.lock.unlock() } - - /// Consumes this `Mutex`, returning the guarded data. - #[inline] - #[must_use] - pub fn into_inner(self) -> T { - self.data.into_inner() - } - - /// Returns a mutable reference to the underlying data. - /// - /// Since this call borrows the `Mutex` mutably, no actual locking needs to - /// take place -- the mutable borrow statically guarantees no locks exist. - /// - /// # Examples - /// - /// ``` - /// let mut lock = maitake_sync::spin::Mutex::new(0); - /// *lock.get_mut() = 10; - /// assert_eq!(*lock.lock(), 10); - /// ``` - pub fn get_mut(&mut self) -> &mut T { - unsafe { - // Safety: since this call borrows the `Mutex` mutably, no actual - // locking needs to take place -- the mutable borrow statically - // guarantees no locks exist. - self.data.with_mut(|data| &mut *data) - } - } } impl Default for Mutex { @@ -405,6 +413,7 @@ where #[cfg(test)] mod tests { use crate::loom::{self, thread}; + use crate::spin::Spinlock; use std::prelude::v1::*; use std::sync::Arc; @@ -413,7 +422,7 @@ mod tests { #[test] fn multithreaded() { loom::model(|| { - let mutex = Arc::new(Mutex::new(String::new())); + let mutex = Arc::new(Mutex::with_raw_mutex(String::new(), Spinlock::new())); let mutex2 = mutex.clone(); let t1 = thread::spawn(move || { @@ -438,7 +447,7 @@ mod tests { #[test] fn try_lock() { loom::model(|| { - let mutex = Mutex::new(42); + let mutex = Mutex::with_raw_mutex(42, Spinlock::new()); // First lock succeeds let a = mutex.try_lock(); assert_eq!(a.as_ref().map(|r| **r), Some(42)); diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index fa8114e9..4f7cf559 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -72,7 +72,7 @@ mod inner { pub(crate) mod sync { pub(crate) use loom::sync::*; - pub(crate) mod spin { + pub(crate) mod blocking { use core::{ marker::PhantomData, ops::{Deref, DerefMut}, @@ -219,7 +219,7 @@ mod inner { pub use alloc::sync::*; pub use core::sync::*; - pub use crate::spin; + pub use crate::blocking; } pub(crate) mod atomic { diff --git a/maitake-sync/src/mutex.rs b/maitake-sync/src/mutex.rs index 1eb85ea9..66cf987d 100644 --- a/maitake-sync/src/mutex.rs +++ b/maitake-sync/src/mutex.rs @@ -4,9 +4,8 @@ //! //! [mutual exclusion lock]: https://en.wikipedia.org/wiki/Mutual_exclusion use crate::{ - blocking::ScopedRawMutex, + blocking::{DefaultMutex, ScopedRawMutex}, loom::cell::{MutPtr, UnsafeCell}, - spin::Spinlock, util::fmt, wait_queue::{self, WaitQueue}, }; @@ -60,8 +59,9 @@ mod tests; /// # Overriding the blocking mutex /// /// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to -/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To -/// use an alternative [`ScopedRawMutex`] implementation, use the +/// synchronize access to its wait list. By default, the [`DefaultMutex`] type +/// is used as the underlying mutex implementation. To use an alternative +/// [`ScopedRawMutex`] implementation, use the /// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for more @@ -95,7 +95,7 @@ mod tests; /// [storage]: https://mycelium.elizas.website/maitake/task/trait.Storage.html /// [no-unwinding]: https://mycelium.elizas.website/maitake/index.html#maitake-does-not-support-unwinding -pub struct Mutex { +pub struct Mutex { wait: WaitQueue, data: UnsafeCell, } @@ -116,7 +116,7 @@ pub struct Mutex { /// [`try_lock`]: Mutex::try_lock /// [RAII]: https://rust-unofficial.github.io/patterns/patterns/behavioural/RAII.html #[must_use = "if unused, the `Mutex` will immediately unlock"] -pub struct MutexGuard<'a, T: ?Sized, L: ScopedRawMutex = Spinlock> { +pub struct MutexGuard<'a, T: ?Sized, L: ScopedRawMutex = DefaultMutex> { /// /!\ WARNING: semi-load-bearing drop order /!\ /// /// This struct's field ordering is important. @@ -145,7 +145,7 @@ pub struct MutexGuard<'a, T: ?Sized, L: ScopedRawMutex = Spinlock> { #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] #[pin_project] #[derive(Debug)] -pub struct Lock<'a, T: ?Sized, L: ScopedRawMutex = Spinlock> { +pub struct Lock<'a, T: ?Sized, L: ScopedRawMutex = DefaultMutex> { #[pin] wait: wait_queue::Wait<'a, L>, mutex: &'a Mutex, @@ -164,7 +164,7 @@ impl Mutex { /// The returned `Mutex` will be in the unlocked state and is ready for /// use. /// - /// This constructor returns a [`Mutex`] that uses a [`Spinlock`] as the + /// This constructor returns a [`Mutex`] that uses a [`DefaultMutex`] as the /// underlying blocking mutex implementation. To use an alternative /// [`ScopedRawMutex`] implementation, use the [`Mutex::with_raw_mutex`] /// constructor instead. See [the documentation on overriding mutex @@ -187,41 +187,36 @@ impl Mutex { /// ``` #[must_use] pub fn new(data: T) -> Self { + Self::with_raw_mutex(data, DefaultMutex::new()) + } + } +} + +impl Mutex { + loom_const_fn! { + /// Returns a new `Mutex` protecting the provided `data`, using the provided + /// [`ScopedRawMutex`] implementation as the raw mutex. + /// + /// The returned `Mutex` will be in the unlocked state and is ready for + /// use. + /// + /// This constructor allows a [`Mutex`] to be constructed with any type that + /// implements [`ScopedRawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + pub fn with_raw_mutex(data: T, lock: L) -> Self { Self { // The queue must start with a single stored wakeup, so that the // first task that tries to acquire the lock will succeed // immediately. - wait: WaitQueue::new_woken(), + wait: WaitQueue::::new_woken(lock), data: UnsafeCell::new(data), } } } } -#[cfg(not(loom))] -impl Mutex -where - L: ScopedRawMutex + mutex_traits::ConstInit, -{ - /// Returns a new `Mutex` protecting the provided `data`, using the provided - /// [`ScopedRawMutex`] implementation as the raw mutex. - /// - /// The returned `Mutex` will be in the unlocked state and is ready for - /// use. - /// - /// This constructor allows a [`Mutex`] to be constructed with any type that - /// implements [`ScopedRawMutex`] as the underlying raw blocking mutex - /// implementation. See [the documentation on overriding mutex - /// implementations](crate::blocking#overriding-mutex-implementations) - /// for more details. - pub const fn with_raw_mutex(data: T) -> Self { - Self { - wait: WaitQueue::new_woken_with_raw_mutex(), - data: UnsafeCell::new(data), - } - } -} - impl Mutex { /// Consumes this `Mutex`, returning the guarded data. #[inline] @@ -297,14 +292,6 @@ impl Mutex { /// /// Since this call borrows the `Mutex` mutably, no actual locking needs to /// take place -- the mutable borrow statically guarantees no locks exist. - /// - /// # Examples - /// - /// ``` - /// let mut lock = maitake_sync::spin::Mutex::new(0); - /// *lock.get_mut() = 10; - /// assert_eq!(*lock.try_lock().unwrap(), 10); - /// ``` pub fn get_mut(&mut self) -> &mut T { unsafe { // Safety: since this call borrows the `Mutex` mutably, no actual diff --git a/maitake-sync/src/rwlock.rs b/maitake-sync/src/rwlock.rs index 2f0962f6..41b1e294 100644 --- a/maitake-sync/src/rwlock.rs +++ b/maitake-sync/src/rwlock.rs @@ -209,36 +209,30 @@ impl RwLock { /// ``` #[must_use] pub fn new(data: T) -> Self { - Self { - sem: Semaphore::new(Self::MAX_READERS), - data: UnsafeCell::new(data), - } + Self::with_raw_mutex(data, Spinlock::new()) } } } -#[cfg(not(loom))] -impl RwLock -where - Lock: RawMutex + mutex_traits::ConstInit, -{ - /// Returns a new `RwLock` protecting the provided `data`, in an - /// unlocked state, using the provided [`RawMutex`] implementation. - /// - /// This constructor allows a [`RwLock`] to be constructed with any type that - /// implements [`RawMutex`] as the underlying raw blocking mutex - /// implementation. See [the documentation on overriding mutex - /// implementations](crate::blocking#overriding-mutex-implementations) - /// for more details. - pub const fn with_raw_mutex(data: T) -> Self { - Self { - sem: Semaphore::with_raw_mutex(Self::MAX_READERS), - data: UnsafeCell::new(data), +impl RwLock { + loom_const_fn! { + /// Returns a new `RwLock` protecting the provided `data`, in an + /// unlocked state, using the provided [`RawMutex`] implementation. + /// + /// This constructor allows a [`RwLock`] to be constructed with any type that + /// implements [`RawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + #[must_use] + pub fn with_raw_mutex(data: T, lock: Lock) -> Self { + Self { + sem: Semaphore::with_raw_mutex(Self::MAX_READERS, lock), + data: UnsafeCell::new(data), + } } } -} -impl RwLock { /// Consumes this `RwLock`, returning the guarded data. #[inline] #[must_use] diff --git a/maitake-sync/src/semaphore.rs b/maitake-sync/src/semaphore.rs index f598f532..6797383c 100644 --- a/maitake-sync/src/semaphore.rs +++ b/maitake-sync/src/semaphore.rs @@ -10,7 +10,7 @@ use crate::{ cell::UnsafeCell, sync::{ atomic::{AtomicUsize, Ordering::*}, - spin::{Mutex, MutexGuard}, + blocking::{Mutex, MutexGuard}, }, }, spin::Spinlock, @@ -318,38 +318,11 @@ impl Semaphore { /// [`MAX_PERMITS`]: Self::MAX_PERMITS #[must_use] pub fn new(permits: usize) -> Self { - Self::make( - permits, - Mutex::new(SemQueue::new()) - ) + Self::with_raw_mutex(permits, Spinlock::new()) } } } -#[cfg(not(loom))] -impl Semaphore -where - Lock: RawMutex + mutex_traits::ConstInit, -{ - /// Returns a new `Semaphore` with `permits` permits available, using the - /// provided [`RawMutex`] implementation. - /// - /// This constructor allows a [`Semaphore`] to be constructed with any type that - /// implements [`RawMutex`] as the underlying raw blocking mutex - /// implementation. See [the documentation on overriding mutex - /// implementations](crate::blocking#overriding-mutex-implementations) - /// for more details. - /// - /// # Panics - /// - /// If `permits` is less than [`MAX_PERMITS`] ([`usize::MAX`] - 1). - /// - /// [`MAX_PERMITS`]: Self::MAX_PERMITS - pub const fn with_raw_mutex(permits: usize) -> Self { - Self::make(permits, Mutex::with_raw_mutex(SemQueue::new())) - } -} - // This is factored out as a free constant in this module so that `RwLock` can // depend on it without having to specify `Semaphore`'s type parameters. This is // a little annoying but whatever. @@ -362,17 +335,32 @@ impl Semaphore { const CLOSED: usize = usize::MAX; loom_const_fn! { - fn make(permits: usize, waiters: Mutex) -> Self { + /// Returns a new `Semaphore` with `permits` permits available, using the + /// provided [`RawMutex`] implementation. + /// + /// This constructor allows a [`Semaphore`] to be constructed with any type that + /// implements [`RawMutex`] as the underlying raw blocking mutex + /// implementation. See [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + /// + /// # Panics + /// + /// If `permits` is less than [`MAX_PERMITS`] ([`usize::MAX`] - 1). + /// + /// [`MAX_PERMITS`]: Self::MAX_PERMITS + pub fn with_raw_mutex(permits: usize, lock: Lock) -> Self { assert!( permits <= Self::MAX_PERMITS, "a semaphore may not have more than Semaphore::MAX_PERMITS permits", ); Self { permits: CachePadded::new(AtomicUsize::new(permits)), - waiters, + waiters: Mutex::with_raw_mutex(SemQueue::new(), lock) } } } + /// Returns the number of permits currently available in this semaphore, or /// 0 if the semaphore is [closed]. /// diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index a8dcb674..67396dd5 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -33,16 +33,38 @@ pub mod once; pub use self::once::{InitOnce, Lazy}; -#[deprecated( - since = "0.1.3", - note = "moved to the `blocking` module, prefer importing from there" -)] -pub use crate::blocking::*; + use crate::{ + blocking::{self, RawMutex, RawRwLock}, loom::sync::atomic::{AtomicBool, AtomicUsize, Ordering::*}, util::{fmt, Backoff}, }; +/// A type alias for a [`blocking::Mutex`] which explicitly uses a [`Spinlock`] +/// for synchronization. See the [`blocking::Mutex`] type's documentation for +/// details. +pub type Mutex = blocking::Mutex; + +/// A type alias for a [`blocking::MutexGuard`] returned by a +/// [`spin::Mutex`](super::Mutex). See the [`blocking::MutexGuard`] type's +/// documentation for details. +pub type MutexGuard<'a, T> = blocking::MutexGuard<'a, T, Spinlock>; + +/// A type alias for a [`blocking::RwLock`] which explicitly uses a [`RwSpinlock`] +/// for synchronization. See the [`blocking::RwLock`] type's documentation for +/// details. +pub type RwLock = blocking::RwLock; + +/// A type alias for a [`blocking::RwLockReadGuard`] returned by a +/// [`spin::RwLock`](super::RwLock). See the [`blocking::RwLockReadGuard`] type's +/// documentation for details. +pub type RwLockReadGuard<'a, T> = blocking::RwLockReadGuard<'a, T, RwSpinlock>; + +/// A type alias for a [`blocking::RwLockWriteGuard`] returned by a +/// [`spin::RwLock`](super::RwLock). See the [`blocking::RwLockWriteGuard`] type's +/// documentation for details. +pub type RwLockWriteGuard<'a, T> = blocking::RwLockWriteGuard<'a, T, RwSpinlock>; + /// A spinlock-based [`RawMutex`] implementation. /// /// This mutex will spin with an exponential backoff while waiting for the lock diff --git a/maitake-sync/src/wait_map.rs b/maitake-sync/src/wait_map.rs index ed44e9b9..e9e54507 100644 --- a/maitake-sync/src/wait_map.rs +++ b/maitake-sync/src/wait_map.rs @@ -3,15 +3,11 @@ //! //! See the documentation for the [`WaitMap`] type for details. use crate::{ - blocking::ScopedRawMutex, + blocking::{DefaultMutex, Mutex, ScopedRawMutex}, loom::{ cell::UnsafeCell, - sync::{ - atomic::{AtomicUsize, Ordering::*}, - spin::Mutex, - }, + sync::atomic::{AtomicUsize, Ordering::*}, }, - spin::Spinlock, util::{fmt, CachePadded, WakeBatch}, }; use cordyceps::{ @@ -90,7 +86,7 @@ const fn notified(data: T) -> Poll> { /// # Overriding the blocking mutex /// /// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to -/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To +/// synchronize access to its wait list. By default, this is a [`DefaultMutex`]. To /// use an alternative [`ScopedRawMutex`] implementation, use the /// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation /// on overriding mutex @@ -188,7 +184,7 @@ const fn notified(data: T) -> Poll> { /// [ilist]: cordyceps::List /// [intrusive]: https://fuchsia.dev/fuchsia-src/development/languages/c-cpp/fbl_containers_guide/introduction /// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue -pub struct WaitMap { +pub struct WaitMap { /// The wait queue's state variable. state: CachePadded, @@ -246,7 +242,7 @@ where #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Wait<'a, K: PartialEq, V, Lock: ScopedRawMutex = Spinlock> { +pub struct Wait<'a, K: PartialEq, V, Lock: ScopedRawMutex = DefaultMutex> { /// The [`WaitMap`] being waited on from. queue: &'a WaitMap, @@ -452,39 +448,19 @@ enum Wakeup { // === impl WaitMap === impl WaitMap { - /// Returns a new `WaitMap`. - /// - /// This constructor returns a `WaitMap` that uses a [`Spinlock`] as - /// the [`ScopedRawMutex`] implementation for wait list synchronization. - /// To use a different [`ScopedRawMutex`] implementation, use the - /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See - /// [the documentation on overriding mutex - /// implementations](crate::blocking#overriding-mutex-implementations) - /// for more details. - #[must_use] - #[cfg(not(loom))] - pub const fn new() -> Self { - Self { - state: Self::mk_state(), - queue: Mutex::new(List::new()), - } - } - - /// Returns a new `WaitMap`. - /// - /// This constructor returns a `WaitMap` that uses a [`Spinlock`] as - /// the [`ScopedRawMutex`] implementation for wait list synchronization. - /// To use a different [`ScopedRawMutex`] implementation, use the - /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See - /// [the documentation on overriding mutex - /// implementations](crate::blocking#overriding-mutex-implementations) - /// for more details. - #[must_use] - #[cfg(loom)] - pub fn new() -> Self { - Self { - state: Self::mk_state(), - queue: Mutex::new(List::new()), + loom_const_fn! { + /// Returns a new `WaitMap`. + /// + /// This constructor returns a `WaitMap` that uses a [`DefaultMutex`] as + /// the [`ScopedRawMutex`] implementation for wait list synchronization. + /// To use a different [`ScopedRawMutex`] implementation, use the + /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See + /// [the documentation on overriding mutex + /// implementations](crate::blocking#overriding-mutex-implementations) + /// for more details. + #[must_use] + pub fn new() -> Self { + Self::with_raw_mutex(DefaultMutex::new()) } } } @@ -492,10 +468,9 @@ impl WaitMap { impl WaitMap where K: PartialEq, - Lock: ScopedRawMutex + mutex_traits::ConstInit, + Lock: ScopedRawMutex, { loom_const_fn! { - /// Returns a new `WaitMap`, using the provided [`ScopedRawMutex`] /// implementation for wait-list synchronization. /// @@ -505,22 +480,16 @@ where /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. #[must_use] - pub fn with_raw_mutex() -> Self { + pub fn with_raw_mutex(lock: Lock) -> Self { Self { - state: Self::mk_state(), - queue: Mutex::with_raw_mutex(List::new()), + state: CachePadded::new(AtomicUsize::new(State::Empty.into_usize())), + queue: Mutex::with_raw_mutex(List::new(), lock), } } } } impl WaitMap { - loom_const_fn! { - fn mk_state() -> CachePadded { - CachePadded::new(AtomicUsize::new(State::Empty.into_usize())) - } - } - /// Wake a certain task in the queue. /// /// If the queue is empty, a wakeup is stored in the `WaitMap`, and the @@ -748,7 +717,7 @@ feature! { /// See [`Wait::subscribe`] for more information and usage example. #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] #[derive(Debug)] -pub struct Subscribe<'a, 'b, K, V, Lock = Spinlock> +pub struct Subscribe<'a, 'b, K, V, Lock = DefaultMutex> where K: PartialEq, Lock: ScopedRawMutex, @@ -1106,7 +1075,7 @@ feature! { /// assert_unpin::>(); #[derive(Debug)] #[pin_project(PinnedDrop)] - pub struct WaitOwned { + pub struct WaitOwned { /// The `WaitMap` being waited on. queue: Arc>, diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index 391e8a13..26a55f10 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -5,15 +5,11 @@ #[cfg(any(test, maitake_ultraverbose))] use crate::util::fmt; use crate::{ - blocking::ScopedRawMutex, + blocking::{DefaultMutex, Mutex, ScopedRawMutex}, loom::{ cell::UnsafeCell, - sync::{ - atomic::{AtomicUsize, Ordering::*}, - spin::Mutex, - }, + sync::atomic::{AtomicUsize, Ordering::*}, }, - spin::Spinlock, util::{CachePadded, WakeBatch}, WaitResult, }; @@ -53,7 +49,7 @@ mod tests; /// # Overriding the blocking mutex /// /// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to -/// synchronize access to its wait list. By default, this is a [`Spinlock`]. To +/// synchronize access to its wait list. By default, this is a [`DefaultMutex`]. To /// use an alternative [`ScopedRawMutex`] implementation, use the /// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation /// on overriding mutex @@ -187,7 +183,7 @@ mod tests; /// [mutex]: crate::Mutex /// [2]: https://www.1024cores.net/home/lock-free-algorithms/queues/intrusive-mpsc-node-based-queue #[derive(Debug)] -pub struct WaitQueue { +pub struct WaitQueue { /// The wait queue's state variable. state: CachePadded, @@ -231,7 +227,7 @@ pub struct WaitQueue { #[derive(Debug)] #[pin_project(PinnedDrop)] #[must_use = "futures do nothing unless `.await`ed or `poll`ed"] -pub struct Wait<'a, Lock: ScopedRawMutex = Spinlock> { +pub struct Wait<'a, Lock: ScopedRawMutex = DefaultMutex> { /// The [`WaitQueue`] being waited on. queue: &'a WaitQueue, @@ -369,7 +365,7 @@ impl WaitQueue { loom_const_fn! { /// Returns a new `WaitQueue`. /// - /// This constructor returns a `WaitQueue` that uses a [`Spinlock`] as + /// This constructor returns a `WaitQueue` that uses a [`DefaultMutex`] as /// the [`ScopedRawMutex`] implementation for wait list synchronization. /// To use a different [`ScopedRawMutex`] implementation, use the /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See @@ -378,27 +374,14 @@ impl WaitQueue { /// for more details. #[must_use] pub fn new() -> Self { - Self::make(State::Empty, Mutex::new(List::new())) - } - } - - loom_const_fn! { - /// Returns a new `WaitQueue` with a single stored wakeup. - /// - /// The first call to [`wait`] on this queue will immediately succeed. - /// - /// [`wait`]: Self::wait - // TODO(eliza): should this be a public API? - #[must_use] - pub(crate) fn new_woken() -> Self { - Self::make(State::Woken, Mutex::new(List::new())) + Self::with_raw_mutex(DefaultMutex::new()) } } } impl WaitQueue where - Lock: ScopedRawMutex + mutex_traits::ConstInit, + Lock: ScopedRawMutex, { loom_const_fn! { /// Returns a new `WaitQueue`, using the provided [`ScopedRawMutex`] @@ -410,33 +393,30 @@ where /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. #[must_use] - pub fn with_raw_mutex() -> Self { - Self::make(State::Empty, Mutex::with_raw_mutex(List::new())) + pub fn with_raw_mutex(lock: Lock) -> Self { + Self::make(State::Empty, lock) } } - /// Returns a new `WaitQueue` with a single stored wakeup. - /// - /// The first call to [`wait`] on this queue will immediately succeed. - /// - /// [`wait`]: Self::wait - #[must_use] - #[cfg(not(loom))] - pub(crate) const fn new_woken_with_raw_mutex() -> Self { - Self::make(State::Woken, Mutex::with_raw_mutex(List::new())) + loom_const_fn! { + /// Returns a new `WaitQueue` with a single stored wakeup. + /// + /// The first call to [`wait`] on this queue will immediately succeed. + /// + /// [`wait`]: Self::wait + // TODO(eliza): should this be a public API? + #[must_use] + pub(crate) fn new_woken(lock: Lock) -> Self { + Self::make(State::Woken, lock) + } } -} -impl WaitQueue -where - Lock: ScopedRawMutex, -{ loom_const_fn! { #[must_use] - fn make(state: State, queue: Mutex, Lock>) -> Self { + fn make(state: State, lock: Lock) -> Self { Self { state: CachePadded::new(AtomicUsize::new(state.into_usize())), - queue, + queue: Mutex::with_raw_mutex(List::new(), lock), } } } @@ -1534,7 +1514,7 @@ feature! { /// ``` #[derive(Debug)] #[pin_project(PinnedDrop)] - pub struct WaitOwned { + pub struct WaitOwned { /// The `WaitQueue` being waited on. queue: Arc>, From 45804c10748298a0e0296d15f686413dd036d8b9 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sun, 28 Jul 2024 12:06:25 -0700 Subject: [PATCH 24/41] make mycelium compile with breaking changes maybe `new` should infer the lock type idk... --- alloc/src/buddy.rs | 114 +++++++++++++++++--------------- hal-x86_64/src/interrupt.rs | 7 +- hal-x86_64/src/serial.rs | 2 +- hal-x86_64/src/time/pit.rs | 4 +- hal-x86_64/src/vga.rs | 15 +++-- maitake-sync/src/spin.rs | 3 +- maitake/src/time/timer.rs | 36 ++++------ maitake/src/time/timer/sleep.rs | 38 ++++++----- src/arch/x86_64/framebuf.rs | 8 ++- src/drivers/ps2_keyboard.rs | 8 ++- util/src/sync.rs | 6 +- 11 files changed, 127 insertions(+), 114 deletions(-) diff --git a/alloc/src/buddy.rs b/alloc/src/buddy.rs index 96648271..6d6239b7 100644 --- a/alloc/src/buddy.rs +++ b/alloc/src/buddy.rs @@ -14,7 +14,7 @@ use mycelium_util::intrusive::{list, Linked, List}; use mycelium_util::math::Logarithm; use mycelium_util::sync::{ atomic::{AtomicUsize, Ordering::*}, - spin, + blocking::Mutex, }; #[derive(Debug)] @@ -41,7 +41,7 @@ pub struct Alloc { /// Array of free lists by "order". The order of an block is the number /// of times the minimum page size must be doubled to reach that block's /// size. - free_lists: [spin::Mutex>; FREE_LISTS], + free_lists: [Mutex>; FREE_LISTS], } type Result = core::result::Result; @@ -65,7 +65,7 @@ impl Alloc { // // see https://github.com/rust-lang/rust-clippy/issues/7665 #[allow(clippy::declare_interior_mutable_const)] - const ONE_FREE_LIST: spin::Mutex> = spin::Mutex::new(List::new()); + const ONE_FREE_LIST: Mutex> = Mutex::new(List::new()); // ensure we don't split memory into regions too small to fit the free // block header in them. @@ -190,16 +190,12 @@ impl Alloc { let _span = tracing::debug_span!("free_list", order, size = self.size_for_order(order),) .entered(); - match list.try_lock() { - Some(list) => { - for entry in list.iter() { - tracing::debug!("entry={entry:?}"); - } - } - None => { - tracing::debug!(""); + list.try_with_lock(|list| { + for entry in list.iter() { + tracing::debug!("entry={entry:?}"); } - } + }) + .unwrap_or_else(|| tracing::debug!("")) } } @@ -283,29 +279,35 @@ impl Alloc { tracing::trace!(curr_order = idx + order); // Is there an available block on this free list? - let mut free_list = free_list.lock(); - if let Some(mut block) = free_list.pop_back() { - let block = unsafe { block.as_mut() }; - tracing::trace!(?block, ?free_list, "found"); - - // Unless this is the free list on which we'd expect to find a - // block of the requested size (the first free list we checked), - // the block is larger than the requested allocation. In that - // case, we'll need to split it down and push the remainder onto - // the appropriate free lists. - if idx > 0 { - let curr_order = idx + order; - tracing::trace!(?curr_order, ?order, "split down"); - self.split_down(block, curr_order, order); - } + let allocated = free_list.with_lock(|free_list| { + if let Some(mut block) = free_list.pop_back() { + let block = unsafe { block.as_mut() }; + tracing::trace!(?block, ?free_list, "found"); + + // Unless this is the free list on which we'd expect to find a + // block of the requested size (the first free list we checked), + // the block is larger than the requested allocation. In that + // case, we'll need to split it down and push the remainder onto + // the appropriate free lists. + if idx > 0 { + let curr_order = idx + order; + tracing::trace!(?curr_order, ?order, "split down"); + self.split_down(block, curr_order, order); + } - // Change the block's magic to indicate that it is allocated, so - // that we can avoid checking the free list if we try to merge - // it before the first word is written to. - block.make_busy(); - tracing::trace!(?block, "made busy"); - self.allocated_size.fetch_add(block.size(), Release); - return Some(block.into()); + // Change the block's magic to indicate that it is allocated, so + // that we can avoid checking the free list if we try to merge + // it before the first word is written to. + block.make_busy(); + tracing::trace!(?block, "made busy"); + self.allocated_size.fetch_add(block.size(), Release); + Some(block.into()) + } else { + None + } + }); + if let Some(block) = allocated { + return Some(block); } } None @@ -334,25 +336,29 @@ impl Alloc { // Starting at the minimum order on which the freed range will fit for (idx, free_list) in self.free_lists.as_ref()[min_order..].iter().enumerate() { let curr_order = idx + min_order; - let mut free_list = free_list.lock(); - - // Is there a free buddy block at this order? - if let Some(mut buddy) = unsafe { self.take_buddy(block, curr_order, &mut free_list) } { - // Okay, merge the blocks, and try the next order! - if buddy < block { - mem::swap(&mut block, &mut buddy); - } - unsafe { - block.as_mut().merge(buddy.as_mut()); + let done = free_list.with_lock(|free_list| { + // Is there a free buddy block at this order? + if let Some(mut buddy) = unsafe { self.take_buddy(block, curr_order, free_list) } { + // Okay, merge the blocks, and try the next order! + if buddy < block { + mem::swap(&mut block, &mut buddy); + } + unsafe { + block.as_mut().merge(buddy.as_mut()); + } + tracing::trace!(?buddy, ?block, "merged with buddy"); + // Keep merging! + false + } else { + // Okay, we can't keep merging, so push the block on the current + // free list. + free_list.push_front(block); + tracing::trace!("deallocated block"); + self.allocated_size.fetch_sub(size, Release); + true } - tracing::trace!(?buddy, ?block, "merged with buddy"); - // Keep merging! - } else { - // Okay, we can't keep merging, so push the block on the current - // free list. - free_list.push_front(block); - tracing::trace!("deallocated block"); - self.allocated_size.fetch_sub(size, Release); + }); + if done { return Ok(()); } } @@ -369,7 +375,7 @@ impl Alloc { if order > free_lists.len() { todo!("(eliza): choppity chop chop down the block!"); } - free_lists[order].lock().push_front(block); + free_lists[order].with_lock(|list| list.push_front(block)); let mut sz = self.heap_size.load(Acquire); while let Err(actual) = // TODO(eliza): if this overflows that's bad news lol... @@ -473,7 +479,7 @@ impl Alloc { .split_back(size, self.offset()) .expect("block too small to split!"); tracing::trace!(?block, ?new_block); - free_lists[order].lock().push_front(new_block); + free_lists[order].with_lock(|list| list.push_front(new_block)); } } } diff --git a/hal-x86_64/src/interrupt.rs b/hal-x86_64/src/interrupt.rs index 47ec5d43..366abff1 100644 --- a/hal-x86_64/src/interrupt.rs +++ b/hal-x86_64/src/interrupt.rs @@ -127,7 +127,8 @@ pub struct Registers { _pad2: [u16; 3], } -static IDT: spin::Mutex = spin::Mutex::new(idt::Idt::new()); +static IDT: spin::Mutex = + spin::Mutex::with_raw_mutex(idt::Idt::new(), spin::Spinlock::new()); static INTERRUPT_CONTROLLER: InitOnce = InitOnce::uninitialized(); impl Controller { @@ -207,7 +208,7 @@ impl Controller { InterruptModel::Apic { local, - io: spin::Mutex::new(io), + io: spin::Mutex::with_raw_mutex(io, spin::Spinlock::new()), } } model => { @@ -225,7 +226,7 @@ impl Controller { // clear for you, the reader, that at this point they are definitely intentionally enabled. pics.enable(); } - InterruptModel::Pic(spin::Mutex::new(pics)) + InterruptModel::Pic(spin::Mutex::with_raw_mutex(pics, spin::Spinlock::new())) } }; tracing::trace!(interrupt_model = ?model); diff --git a/hal-x86_64/src/serial.rs b/hal-x86_64/src/serial.rs index d13d6841..2e33bfd7 100644 --- a/hal-x86_64/src/serial.rs +++ b/hal-x86_64/src/serial.rs @@ -110,7 +110,7 @@ impl Port { })?; Ok(Self { - inner: spin::Mutex::new(registers), + inner: spin::Mutex::with_raw_mutex(registers, spin::Spinlock::new()), }) } diff --git a/hal-x86_64/src/time/pit.rs b/hal-x86_64/src/time/pit.rs index 843d74d7..125f9037 100644 --- a/hal-x86_64/src/time/pit.rs +++ b/hal-x86_64/src/time/pit.rs @@ -9,7 +9,7 @@ use core::{ use mycelium_util::{ bits::{bitfield, enum_from_bits}, fmt, - sync::spin::Mutex, + sync::spin::{Mutex, Spinlock}, }; /// Intel 8253/8254 Programmable Interval Timer (PIT). @@ -210,7 +210,7 @@ enum_from_bits! { /// publicly and is represented as a singleton. It's stored in a [`Mutex`] in /// order to ensure that multiple CPU cores don't try to write conflicting /// configurations to the PIT's configuration ports. -pub static PIT: Mutex = Mutex::new(Pit::new()); +pub static PIT: Mutex = Mutex::with_raw_mutex(Pit::new(), Spinlock::new()); /// Are we currently sleeping on an interrupt? static SLEEPING: AtomicBool = AtomicBool::new(false); diff --git a/hal-x86_64/src/vga.rs b/hal-x86_64/src/vga.rs index b507d8d7..63eaa6da 100644 --- a/hal-x86_64/src/vga.rs +++ b/hal-x86_64/src/vga.rs @@ -5,12 +5,15 @@ use mycelium_util::{ }; use volatile::Volatile; static BUFFER: Lazy> = Lazy::new(|| { - spin::Mutex::new(Buffer { - col: 0, - row: 0, - color: ColorSpec::new(Color::LightGray, Color::Black), - buf: Volatile::new(unsafe { &mut *(0xb8000u64 as *mut Buf) }), - }) + spin::Mutex::with_raw_mutex( + Buffer { + col: 0, + row: 0, + color: ColorSpec::new(Color::LightGray, Color::Black), + buf: Volatile::new(unsafe { &mut *(0xb8000u64 as *mut Buf) }), + }, + spin::Spinlock::new(), + ) }); pub fn writer() -> Writer { diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index 67396dd5..e000020f 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -83,7 +83,8 @@ pub struct RwSpinlock { impl Spinlock { loom_const_fn! { - pub(crate) fn new() -> Self { + /// Returns a new `Spinlock`, in the unlocked state. + pub fn new() -> Self { Self { locked: AtomicBool::new(false) } } } diff --git a/maitake/src/time/timer.rs b/maitake/src/time/timer.rs index 224e2444..3877780d 100644 --- a/maitake/src/time/timer.rs +++ b/maitake/src/time/timer.rs @@ -8,13 +8,11 @@ //! [future]: core::future::Future use super::clock::{self, Clock, Instant, Ticks}; use crate::{ - loom::sync::{ - atomic::{AtomicUsize, Ordering::*}, - spin::{Mutex, MutexGuard}, - }, + loom::sync::atomic::{AtomicUsize, Ordering::*}, util::expect_display, }; use core::time::Duration; +use maitake_sync::blocking::Mutex; use mycelium_util::fmt; #[cfg(test)] @@ -393,12 +391,7 @@ impl Timer { // instead, if the timer wheel is busy (e.g. the timer ISR was called on // another core, or if a `Sleep` future is currently canceling itself), // we just add to a counter of pending ticks, and bail. - if let Some(mut core) = self.core.try_lock() { - Some(self.advance_locked(&mut core)) - } else { - trace!("could not lock timer wheel"); - None - } + self.core.try_with_lock(|core| self.advance_locked(core)) } /// Advance the timer to the current time, ensuring any [`Sleep`] futures that @@ -426,7 +419,7 @@ impl Timer { /// ensure that pending ticks are drained frequently. #[inline] pub fn turn(&self) -> Turn { - self.advance_locked(&mut self.core.lock()) + self.core.with_lock(|core| self.advance_locked(core)) } pub(in crate::time) fn ticks_to_dur(&self, ticks: Ticks) -> Duration { @@ -470,15 +463,12 @@ impl Timer { } } - fn core(&self) -> MutexGuard<'_, wheel::Core> { - self.core.lock() - } - #[cfg(all(test, not(loom)))] fn reset(&self) { - let mut core = self.core(); - *core = wheel::Core::new(); - self.pending_ticks.store(0, Release); + self.core.with_lock(|core| { + *core = wheel::Core::new(); + self.pending_ticks.store(0, Release); + }); } } @@ -489,12 +479,12 @@ impl fmt::Debug for Timer { pending_ticks, core, } = self; - f.debug_struct("Timer") - .field("clock", &clock) + let mut s = f.debug_struct("Timer"); + s.field("clock", &clock) .field("tick_duration", &clock.tick_duration()) - .field("pending_ticks", &pending_ticks.load(Acquire)) - .field("core", &fmt::opt(&core.try_lock()).or_else("")) - .finish() + .field("pending_ticks", &pending_ticks.load(Acquire)); + core.try_with_lock(|core| s.field("core", &core).finish()) + .unwrap_or_else(|| s.field("core", &"").finish()) } } diff --git a/maitake/src/time/timer/sleep.rs b/maitake/src/time/timer/sleep.rs index 92b291f1..a534b113 100644 --- a/maitake/src/time/timer/sleep.rs +++ b/maitake/src/time/timer/sleep.rs @@ -113,22 +113,26 @@ impl Future for Sleep<'_> { let ptr = unsafe { ptr::NonNull::from(Pin::into_inner_unchecked(this.entry.as_mut())) }; // Acquire the wheel lock to insert the sleep. - let mut core = this.timer.core(); - - // While we are holding the wheel lock, go ahead and advance the - // timer, too. This way, the timer wheel gets advanced more - // frequently than just when a scheduler tick completes or a - // timer IRQ fires, helping to increase timer accuracy. - this.timer.advance_locked(&mut core); - - match test_dbg!(core.register_sleep(ptr)) { - Poll::Ready(()) => { - *this.state = State::Completed; - return Poll::Ready(()); - } - Poll::Pending => { - *this.state = State::Registered; + let done = this.timer.core.with_lock(|core| { + // While we are holding the wheel lock, go ahead and advance the + // timer, too. This way, the timer wheel gets advanced more + // frequently than just when a scheduler tick completes or a + // timer IRQ fires, helping to increase timer accuracy. + this.timer.advance_locked(core); + + match test_dbg!(core.register_sleep(ptr)) { + Poll::Ready(()) => { + *this.state = State::Completed; + true + } + Poll::Pending => { + *this.state = State::Registered; + false + } } + }); + if done { + return Poll::Ready(()); } } State::Registered => {} @@ -154,7 +158,9 @@ impl PinnedDrop for Sleep<'_> { // yet, or it has already completed, we don't need to lock the timer to // remove it. if test_dbg!(this.entry.linked.load(Acquire)) { - this.timer.core().cancel_sleep(this.entry); + this.timer + .core + .with_lock(|core| core.cancel_sleep(this.entry)); } } } diff --git a/src/arch/x86_64/framebuf.rs b/src/arch/x86_64/framebuf.rs index fcb06477..226dafc4 100644 --- a/src/arch/x86_64/framebuf.rs +++ b/src/arch/x86_64/framebuf.rs @@ -54,7 +54,8 @@ pub(super) fn init(bootinfo: &mut BootInfo) -> bool { } // Okay, try to initialize the framebuffer - let Optional::Some(framebuffer) = mem::replace(&mut bootinfo.framebuffer, Optional::None) else { + let Optional::Some(framebuffer) = mem::replace(&mut bootinfo.framebuffer, Optional::None) + else { // The boot info does not contain a framebuffer configuration. Nothing // for us to do! return false; @@ -73,7 +74,10 @@ pub(super) fn init(bootinfo: &mut BootInfo) -> bool { x => unimplemented!("hahaha wtf, found a weird pixel format: {:?}", x), }, }; - FRAMEBUFFER.init((cfg, spin::Mutex::new(framebuffer))); + FRAMEBUFFER.init(( + cfg, + spin::Mutex::with_raw_mutex(framebuffer, spin::Spinlock::new()), + )); true } diff --git a/src/drivers/ps2_keyboard.rs b/src/drivers/ps2_keyboard.rs index 30e10cea..aa31a0ea 100644 --- a/src/drivers/ps2_keyboard.rs +++ b/src/drivers/ps2_keyboard.rs @@ -1,5 +1,8 @@ use maitake::sync::WaitQueue; -use mycelium_util::{fmt, sync::spin::Mutex}; +use mycelium_util::{ + fmt, + sync::spin::{self, Mutex}, +}; use pc_keyboard::{layouts, Keyboard}; pub use pc_keyboard::{DecodedKey, KeyCode}; @@ -19,10 +22,11 @@ pub struct Ps2Keyboard { static PS2_KEYBOARD: Ps2Keyboard = Ps2Keyboard { buf: thingbuf::StaticThingBuf::new(), - kbd: Mutex::new( + kbd: Mutex::with_raw_mutex( Keyboard::::new( pc_keyboard::HandleControl::MapLettersToUnicode, ), + spin::Spinlock::new(), ), waiters: WaitQueue::new(), }; diff --git a/util/src/sync.rs b/util/src/sync.rs index 08c17c68..3e8cacc8 100644 --- a/util/src/sync.rs +++ b/util/src/sync.rs @@ -9,14 +9,12 @@ pub use core::sync::atomic; pub mod cell; pub use self::once::{InitOnce, Lazy}; +pub use maitake_sync::blocking; pub use maitake_sync::{spin::once, util::CachePadded}; /// Spinlocks and related synchronization primitives. pub mod spin { - pub use maitake_sync::{ - spin::{Mutex, MutexGuard}, - util::Backoff, - }; + pub use maitake_sync::{spin::*, util::Backoff}; } /// A wrapper for the [`core::hint`] module that emits either [`loom`] spin loop /// hints (when `cfg(loom)` is enabled), or real spin loop hints when loom is From 76006e8e233bcd6179e128a856cc5fe2744a7141 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sun, 28 Jul 2024 13:33:31 -0700 Subject: [PATCH 25/41] Update semaphore.rs Co-authored-by: James Munns --- maitake-sync/src/semaphore.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/maitake-sync/src/semaphore.rs b/maitake-sync/src/semaphore.rs index 6797383c..73edc181 100644 --- a/maitake-sync/src/semaphore.rs +++ b/maitake-sync/src/semaphore.rs @@ -335,7 +335,7 @@ impl Semaphore { const CLOSED: usize = usize::MAX; loom_const_fn! { - /// Returns a new `Semaphore` with `permits` permits available, using the + /// Returns a new `Semaphore` with `permits` permits available, using the /// provided [`RawMutex`] implementation. /// /// This constructor allows a [`Semaphore`] to be constructed with any type that From 54984d62e6042ee00f4562a5963335bb21981519 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 29 Jul 2024 08:50:01 -0700 Subject: [PATCH 26/41] fix `loom` version of `with_raw_mutex` --- maitake-sync/src/loom.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index 4f7cf559..ce812fc6 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -105,7 +105,7 @@ mod inner { } #[track_caller] - pub(crate) fn with_raw_mutex(t: T) -> Self { + pub(crate) fn with_raw_mutex(t: T, _: Lock) -> Self { Self::new(t) } From aa7dc8f2ac2928d61e6f8e1526ec8b8f2b650bb6 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 29 Jul 2024 08:50:18 -0700 Subject: [PATCH 27/41] add missing `ConstInit` impls for `Spinlock`/`RwSpinlock` --- maitake-sync/src/spin.rs | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index e000020f..743450ab 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -140,6 +140,14 @@ unsafe impl RawMutex for Spinlock { } } +#[cfg(not(loom))] +impl blocking::ConstInit for Spinlock { + // As usual, clippy is totally wrong about this --- the whole point of this + // constant is to create a *new* spinlock every time. + #[allow(clippy::declare_interior_mutable_const)] + const INIT: Self = Spinlock::new(); +} + const UNLOCKED: usize = 0; const WRITER: usize = 1 << 0; const READER: usize = 1 << 1; @@ -249,6 +257,14 @@ unsafe impl RawRwLock for RwSpinlock { } } +#[cfg(not(loom))] +impl blocking::ConstInit for RwSpinlock { + // As usual, clippy is totally wrong about this --- the whole point of this + // constant is to create a *new* spinlock every time. + #[allow(clippy::declare_interior_mutable_const)] + const INIT: Self = RwSpinlock::new(); +} + impl fmt::Debug for RwSpinlock { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let state = &self.state.load(Relaxed); From 78610c3e3417c5970f6a5dbbf6ba0cd048693097 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 29 Jul 2024 08:51:02 -0700 Subject: [PATCH 28/41] rename `with_raw_mutex` to `new_with_raw_mutex` As mentioned in [this comment][1], `new_with_raw_mutex` is probably a better name for the constructor, because we also have `with_lock`, which isn't a constructor. Also, it has the side benefit of being autocomplete-friendly; it will be suggested if you type `new`, which seems good. [1]: https://github.com/hawkw/mycelium/pull/482#issuecomment-2256254561 --- hal-x86_64/src/interrupt.rs | 6 +++--- hal-x86_64/src/serial.rs | 2 +- hal-x86_64/src/time/pit.rs | 2 +- hal-x86_64/src/vga.rs | 2 +- maitake-sync/src/blocking.rs | 6 +++--- maitake-sync/src/blocking/mutex.rs | 10 +++++----- maitake-sync/src/blocking/rwlock.rs | 2 +- maitake-sync/src/loom.rs | 2 +- maitake-sync/src/mutex.rs | 8 ++++---- maitake-sync/src/rwlock.rs | 10 +++++----- maitake-sync/src/semaphore.rs | 8 ++++---- maitake-sync/src/wait_map.rs | 10 +++++----- maitake-sync/src/wait_queue.rs | 10 +++++----- maitake/src/scheduler/steal.rs | 4 +++- maitake/src/scheduler/tests.rs | 4 +++- maitake/src/time/timer/tests.rs | 2 +- src/arch/x86_64/framebuf.rs | 2 +- src/drivers/ps2_keyboard.rs | 2 +- 18 files changed, 48 insertions(+), 44 deletions(-) diff --git a/hal-x86_64/src/interrupt.rs b/hal-x86_64/src/interrupt.rs index 366abff1..4c224e66 100644 --- a/hal-x86_64/src/interrupt.rs +++ b/hal-x86_64/src/interrupt.rs @@ -128,7 +128,7 @@ pub struct Registers { } static IDT: spin::Mutex = - spin::Mutex::with_raw_mutex(idt::Idt::new(), spin::Spinlock::new()); + spin::Mutex::new_with_raw_mutex(idt::Idt::new(), spin::Spinlock::new()); static INTERRUPT_CONTROLLER: InitOnce = InitOnce::uninitialized(); impl Controller { @@ -208,7 +208,7 @@ impl Controller { InterruptModel::Apic { local, - io: spin::Mutex::with_raw_mutex(io, spin::Spinlock::new()), + io: spin::Mutex::new_with_raw_mutex(io, spin::Spinlock::new()), } } model => { @@ -226,7 +226,7 @@ impl Controller { // clear for you, the reader, that at this point they are definitely intentionally enabled. pics.enable(); } - InterruptModel::Pic(spin::Mutex::with_raw_mutex(pics, spin::Spinlock::new())) + InterruptModel::Pic(spin::Mutex::new_with_raw_mutex(pics, spin::Spinlock::new())) } }; tracing::trace!(interrupt_model = ?model); diff --git a/hal-x86_64/src/serial.rs b/hal-x86_64/src/serial.rs index 2e33bfd7..06311a4a 100644 --- a/hal-x86_64/src/serial.rs +++ b/hal-x86_64/src/serial.rs @@ -110,7 +110,7 @@ impl Port { })?; Ok(Self { - inner: spin::Mutex::with_raw_mutex(registers, spin::Spinlock::new()), + inner: spin::Mutex::new_with_raw_mutex(registers, spin::Spinlock::new()), }) } diff --git a/hal-x86_64/src/time/pit.rs b/hal-x86_64/src/time/pit.rs index 125f9037..32761ade 100644 --- a/hal-x86_64/src/time/pit.rs +++ b/hal-x86_64/src/time/pit.rs @@ -210,7 +210,7 @@ enum_from_bits! { /// publicly and is represented as a singleton. It's stored in a [`Mutex`] in /// order to ensure that multiple CPU cores don't try to write conflicting /// configurations to the PIT's configuration ports. -pub static PIT: Mutex = Mutex::with_raw_mutex(Pit::new(), Spinlock::new()); +pub static PIT: Mutex = Mutex::new_with_raw_mutex(Pit::new(), Spinlock::new()); /// Are we currently sleeping on an interrupt? static SLEEPING: AtomicBool = AtomicBool::new(false); diff --git a/hal-x86_64/src/vga.rs b/hal-x86_64/src/vga.rs index 63eaa6da..794c5742 100644 --- a/hal-x86_64/src/vga.rs +++ b/hal-x86_64/src/vga.rs @@ -5,7 +5,7 @@ use mycelium_util::{ }; use volatile::Volatile; static BUFFER: Lazy> = Lazy::new(|| { - spin::Mutex::with_raw_mutex( + spin::Mutex::new_with_raw_mutex( Buffer { col: 0, row: 0, diff --git a/maitake-sync/src/blocking.rs b/maitake-sync/src/blocking.rs index 49039bf4..e63cd80b 100644 --- a/maitake-sync/src/blocking.rs +++ b/maitake-sync/src/blocking.rs @@ -106,14 +106,14 @@ //! the interface provided by [`std::sync::Mutex`]. The [`Mutex::new`] function //! returns a [`Mutex`] using the default spinlock. To instead construct a //! [`Mutex`] with a custom [`RawMutex`] implementation, use the -//! [`Mutex::with_raw_mutex`] function. +//! [`Mutex::new_with_raw_mutex`] function. //! //! Furthermore, many *async* synchronization primitives provided by this crate, //! such as the [async `Mutex`](crate::Mutex), [async `RwLock`], [`WaitQueue`], //! [`WaitMap`], and [`Semaphore`], internally depend on the blocking `Mutex` //! for wait list synchronization. These types are *also* generic over a `Lock` -//! type parameter, and also provide `with_raw_mutex` constructors, such as -//! [`WaitQueue::with_raw_mutex`](crate::WaitQueue::with_raw_mutex). This allows +//! type parameter, and also provide `new_with_raw_mutex` constructors, such as +//! [`WaitQueue::new_with_raw_mutex`](crate::WaitQueue::new_with_raw_mutex). This allows //! the blocking mutex used by these types to be overridden. The majority //! `maitake-sync`'s async synchronization types only require the `Lock` type to //! implement [`ScopedRawMutex`]. However, the [`Semaphore`] and [async diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 03bfdb08..4a23ce74 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -23,7 +23,7 @@ pub use mutex_traits::{RawMutex, ScopedRawMutex}; /// This type is generic over a `Lock` type parameter which represents a raw /// mutex implementation. By default, this is the [`DefaultMutex`]. To construct /// a new `Mutex` with an alternative raw mutex implementation, use the -/// [`Mutex::with_raw_mutex`] cosntructor. See the [module-level documentation +/// [`Mutex::new_with_raw_mutex`] cosntructor. See the [module-level documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for /// more details. @@ -79,7 +79,7 @@ impl Mutex { /// /// This constructor returns a mutex that uses the [`DefaultMutex`] /// implementation. To use an alternative `RawMutex` type, use the - /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. + /// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor, instead. /// /// # Examples /// @@ -109,7 +109,7 @@ impl Mutex { /// /// The returned `Mutex` is in an unlocked state, ready for use. #[must_use] - pub fn with_raw_mutex(data: T, lock: Lock) -> Self { + pub fn new_with_raw_mutex(data: T, lock: Lock) -> Self { Self { lock, data: UnsafeCell::new(data), @@ -422,7 +422,7 @@ mod tests { #[test] fn multithreaded() { loom::model(|| { - let mutex = Arc::new(Mutex::with_raw_mutex(String::new(), Spinlock::new())); + let mutex = Arc::new(Mutex::new_with_raw_mutex(String::new(), Spinlock::new())); let mutex2 = mutex.clone(); let t1 = thread::spawn(move || { @@ -447,7 +447,7 @@ mod tests { #[test] fn try_lock() { loom::model(|| { - let mutex = Mutex::with_raw_mutex(42, Spinlock::new()); + let mutex = Mutex::new_with_raw_mutex(42, Spinlock::new()); // First lock succeeds let a = mutex.try_lock(); assert_eq!(a.as_ref().map(|r| **r), Some(42)); diff --git a/maitake-sync/src/blocking/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs index 45fa945f..6f4d6f94 100644 --- a/maitake-sync/src/blocking/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -213,7 +213,7 @@ impl RwLock { #[cfg(feature = "lock_api")] impl RwLock { /// Creates a new instance of an `RwLock` which is unlocked. - pub const fn with_raw_mutex(data: T) -> Self { + pub const fn new_with_raw_mutex(data: T) -> Self { RwLock { data: UnsafeCell::new(data), lock: Lock::INIT, diff --git a/maitake-sync/src/loom.rs b/maitake-sync/src/loom.rs index ce812fc6..d95558e7 100644 --- a/maitake-sync/src/loom.rs +++ b/maitake-sync/src/loom.rs @@ -105,7 +105,7 @@ mod inner { } #[track_caller] - pub(crate) fn with_raw_mutex(t: T, _: Lock) -> Self { + pub(crate) fn new_with_raw_mutex(t: T, _: Lock) -> Self { Self::new(t) } diff --git a/maitake-sync/src/mutex.rs b/maitake-sync/src/mutex.rs index 66cf987d..2edd86fe 100644 --- a/maitake-sync/src/mutex.rs +++ b/maitake-sync/src/mutex.rs @@ -62,7 +62,7 @@ mod tests; /// synchronize access to its wait list. By default, the [`DefaultMutex`] type /// is used as the underlying mutex implementation. To use an alternative /// [`ScopedRawMutex`] implementation, use the -/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor. See [the documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for more /// details. @@ -166,7 +166,7 @@ impl Mutex { /// /// This constructor returns a [`Mutex`] that uses a [`DefaultMutex`] as the /// underlying blocking mutex implementation. To use an alternative - /// [`ScopedRawMutex`] implementation, use the [`Mutex::with_raw_mutex`] + /// [`ScopedRawMutex`] implementation, use the [`Mutex::new_with_raw_mutex`] /// constructor instead. See [the documentation on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. @@ -187,7 +187,7 @@ impl Mutex { /// ``` #[must_use] pub fn new(data: T) -> Self { - Self::with_raw_mutex(data, DefaultMutex::new()) + Self::new_with_raw_mutex(data, DefaultMutex::new()) } } } @@ -205,7 +205,7 @@ impl Mutex { /// implementation. See [the documentation on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. - pub fn with_raw_mutex(data: T, lock: L) -> Self { + pub fn new_with_raw_mutex(data: T, lock: L) -> Self { Self { // The queue must start with a single stored wakeup, so that the // first task that tries to acquire the lock will succeed diff --git a/maitake-sync/src/rwlock.rs b/maitake-sync/src/rwlock.rs index 41b1e294..4bd13f51 100644 --- a/maitake-sync/src/rwlock.rs +++ b/maitake-sync/src/rwlock.rs @@ -64,7 +64,7 @@ mod tests; /// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to /// synchronize access to its wait list. By default, this is a [`Spinlock`]. To /// use an alternative [`RawMutex`] implementation, use the -/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor. See [the documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for more /// details. @@ -185,7 +185,7 @@ impl RwLock { /// /// This constructor returns a `RwLock` that uses a [`Spinlock`] as the /// underlying blocking mutex implementation. To use an alternative - /// [`RawMutex`] implementation, use the [`RwLock::with_raw_mutex`] + /// [`RawMutex`] implementation, use the [`RwLock::new_with_raw_mutex`] /// constructor instead. See [the documentation on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. @@ -209,7 +209,7 @@ impl RwLock { /// ``` #[must_use] pub fn new(data: T) -> Self { - Self::with_raw_mutex(data, Spinlock::new()) + Self::new_with_raw_mutex(data, Spinlock::new()) } } } @@ -225,9 +225,9 @@ impl RwLock { /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. #[must_use] - pub fn with_raw_mutex(data: T, lock: Lock) -> Self { + pub fn new_with_raw_mutex(data: T, lock: Lock) -> Self { Self { - sem: Semaphore::with_raw_mutex(Self::MAX_READERS, lock), + sem: Semaphore::new_with_raw_mutex(Self::MAX_READERS, lock), data: UnsafeCell::new(data), } } diff --git a/maitake-sync/src/semaphore.rs b/maitake-sync/src/semaphore.rs index 73edc181..716f0ebb 100644 --- a/maitake-sync/src/semaphore.rs +++ b/maitake-sync/src/semaphore.rs @@ -64,7 +64,7 @@ mod tests; /// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to /// synchronize access to its wait list. By default, this is a [`Spinlock`]. To /// use an alternative [`RawMutex`] implementation, use the -/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor. See [the documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for more /// details. @@ -318,7 +318,7 @@ impl Semaphore { /// [`MAX_PERMITS`]: Self::MAX_PERMITS #[must_use] pub fn new(permits: usize) -> Self { - Self::with_raw_mutex(permits, Spinlock::new()) + Self::new_with_raw_mutex(permits, Spinlock::new()) } } } @@ -349,14 +349,14 @@ impl Semaphore { /// If `permits` is less than [`MAX_PERMITS`] ([`usize::MAX`] - 1). /// /// [`MAX_PERMITS`]: Self::MAX_PERMITS - pub fn with_raw_mutex(permits: usize, lock: Lock) -> Self { + pub fn new_with_raw_mutex(permits: usize, lock: Lock) -> Self { assert!( permits <= Self::MAX_PERMITS, "a semaphore may not have more than Semaphore::MAX_PERMITS permits", ); Self { permits: CachePadded::new(AtomicUsize::new(permits)), - waiters: Mutex::with_raw_mutex(SemQueue::new(), lock) + waiters: Mutex::new_with_raw_mutex(SemQueue::new(), lock) } } } diff --git a/maitake-sync/src/wait_map.rs b/maitake-sync/src/wait_map.rs index e9e54507..07bddaf2 100644 --- a/maitake-sync/src/wait_map.rs +++ b/maitake-sync/src/wait_map.rs @@ -88,7 +88,7 @@ const fn notified(data: T) -> Poll> { /// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to /// synchronize access to its wait list. By default, this is a [`DefaultMutex`]. To /// use an alternative [`ScopedRawMutex`] implementation, use the -/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor. See [the documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for more /// details. @@ -454,13 +454,13 @@ impl WaitMap { /// This constructor returns a `WaitMap` that uses a [`DefaultMutex`] as /// the [`ScopedRawMutex`] implementation for wait list synchronization. /// To use a different [`ScopedRawMutex`] implementation, use the - /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See + /// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor, instead. See /// [the documentation on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. #[must_use] pub fn new() -> Self { - Self::with_raw_mutex(DefaultMutex::new()) + Self::new_with_raw_mutex(DefaultMutex::new()) } } } @@ -480,10 +480,10 @@ where /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. #[must_use] - pub fn with_raw_mutex(lock: Lock) -> Self { + pub fn new_with_raw_mutex(lock: Lock) -> Self { Self { state: CachePadded::new(AtomicUsize::new(State::Empty.into_usize())), - queue: Mutex::with_raw_mutex(List::new(), lock), + queue: Mutex::new_with_raw_mutex(List::new(), lock), } } } diff --git a/maitake-sync/src/wait_queue.rs b/maitake-sync/src/wait_queue.rs index 26a55f10..e7459f55 100644 --- a/maitake-sync/src/wait_queue.rs +++ b/maitake-sync/src/wait_queue.rs @@ -51,7 +51,7 @@ mod tests; /// This type uses a [blocking `Mutex`](crate::blocking::Mutex) internally to /// synchronize access to its wait list. By default, this is a [`DefaultMutex`]. To /// use an alternative [`ScopedRawMutex`] implementation, use the -/// [`with_raw_mutex`](Self::with_raw_mutex) constructor. See [the documentation +/// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor. See [the documentation /// on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) for more /// details. @@ -368,13 +368,13 @@ impl WaitQueue { /// This constructor returns a `WaitQueue` that uses a [`DefaultMutex`] as /// the [`ScopedRawMutex`] implementation for wait list synchronization. /// To use a different [`ScopedRawMutex`] implementation, use the - /// [`with_raw_mutex`](Self::with_raw_mutex) constructor, instead. See + /// [`new_with_raw_mutex`](Self::new_with_raw_mutex) constructor, instead. See /// [the documentation on overriding mutex /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. #[must_use] pub fn new() -> Self { - Self::with_raw_mutex(DefaultMutex::new()) + Self::new_with_raw_mutex(DefaultMutex::new()) } } } @@ -393,7 +393,7 @@ where /// implementations](crate::blocking#overriding-mutex-implementations) /// for more details. #[must_use] - pub fn with_raw_mutex(lock: Lock) -> Self { + pub fn new_with_raw_mutex(lock: Lock) -> Self { Self::make(State::Empty, lock) } } @@ -416,7 +416,7 @@ where fn make(state: State, lock: Lock) -> Self { Self { state: CachePadded::new(AtomicUsize::new(state.into_usize())), - queue: Mutex::with_raw_mutex(List::new(), lock), + queue: Mutex::new_with_raw_mutex(List::new(), lock), } } } diff --git a/maitake/src/scheduler/steal.rs b/maitake/src/scheduler/steal.rs index fc105d9e..9d9aca94 100644 --- a/maitake/src/scheduler/steal.rs +++ b/maitake/src/scheduler/steal.rs @@ -192,7 +192,9 @@ impl<'worker, S: Schedule> Stealer<'worker, S> { /// - `true` if a task was successfully stolen. /// - `false` if the targeted queue is empty. pub fn spawn_one(&self, scheduler: &S) -> bool { - let Some(task) = self.queue.dequeue() else { return false }; + let Some(task) = self.queue.dequeue() else { + return false; + }; test_trace!(?task, "stole"); // decrement the target queue's task count diff --git a/maitake/src/scheduler/tests.rs b/maitake/src/scheduler/tests.rs index 028163e7..9f694574 100644 --- a/maitake/src/scheduler/tests.rs +++ b/maitake/src/scheduler/tests.rs @@ -36,7 +36,9 @@ impl Chan { let this = Arc::downgrade(&self); drop(self); futures_util::future::poll_fn(move |cx| { - let Some(this) = this.upgrade() else {return Poll::Ready(()) }; + let Some(this) = this.upgrade() else { + return Poll::Ready(()); + }; let res = this.task.wait(); futures_util::pin_mut!(res); diff --git a/maitake/src/time/timer/tests.rs b/maitake/src/time/timer/tests.rs index e7a2baa2..5f2c1e00 100644 --- a/maitake/src/time/timer/tests.rs +++ b/maitake/src/time/timer/tests.rs @@ -5,7 +5,7 @@ use core::cell::RefCell; use core::sync::atomic::{AtomicU64, Ordering}; use std::sync::Arc; -use crate::time::{Clock, timer::Ticks}; +use crate::time::{timer::Ticks, Clock}; use std::time::Duration; crate::loom::thread_local! { diff --git a/src/arch/x86_64/framebuf.rs b/src/arch/x86_64/framebuf.rs index 226dafc4..fac49270 100644 --- a/src/arch/x86_64/framebuf.rs +++ b/src/arch/x86_64/framebuf.rs @@ -76,7 +76,7 @@ pub(super) fn init(bootinfo: &mut BootInfo) -> bool { }; FRAMEBUFFER.init(( cfg, - spin::Mutex::with_raw_mutex(framebuffer, spin::Spinlock::new()), + spin::Mutex::new_with_raw_mutex(framebuffer, spin::Spinlock::new()), )); true } diff --git a/src/drivers/ps2_keyboard.rs b/src/drivers/ps2_keyboard.rs index aa31a0ea..36a29817 100644 --- a/src/drivers/ps2_keyboard.rs +++ b/src/drivers/ps2_keyboard.rs @@ -22,7 +22,7 @@ pub struct Ps2Keyboard { static PS2_KEYBOARD: Ps2Keyboard = Ps2Keyboard { buf: thingbuf::StaticThingBuf::new(), - kbd: Mutex::with_raw_mutex( + kbd: Mutex::new_with_raw_mutex( Keyboard::::new( pc_keyboard::HandleControl::MapLettersToUnicode, ), From 02036f15149379bb6d6da41799f644329228afa6 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 29 Jul 2024 08:58:11 -0700 Subject: [PATCH 29/41] docs fixes as suggested by @jamesmunns --- maitake-sync/README.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/maitake-sync/README.md b/maitake-sync/README.md index 2cf49289..adb4e549 100644 --- a/maitake-sync/README.md +++ b/maitake-sync/README.md @@ -209,7 +209,7 @@ The following features are available (this list is incomplete; you can help by [ | Feature | Default | Explanation | | :--- | :--- | :--- | | `alloc` | `true` | Enables [`liballoc`] dependency | -| `std` | `false` | Enables the Rust standard library, disabling `#![no-std]`. This implies the `alloc` feature. | +| `std` | `false` | Enables the Rust standard library, disabling `#![no-std]`. When `std` is enabled, the [`DefaultMutex`] type will use [`std::sync::Mutex`]. This implies the `alloc` feature. | | `critical-section` | `false` | Enables a variant of the [`DefaultMutex`] type that uses the [`critical-section`] crate. | | `no-cache-pad` | `false` | Inhibits cache padding for the [`CachePadded`] struct. When this feature is NOT enabled, the size will be determined based on target platform. | | `tracing` | `false` | Enables support for [`tracing`] diagnostics. Requires `liballoc`.| @@ -219,3 +219,5 @@ The following features are available (this list is incomplete; you can help by [ [`CachePadded`]: https://docs.rs/maitake-sync/latest/maitake_sync/util/struct.CachePadded.html [`tracing`]: https://crates.io/crates/tracing [core-error]: https://doc.rust-lang.org/stable/core/error/index.html +[`std::sync::Mutex`]: + https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html From 0472bf40c6c1d611b46baec1d4ab58b1b33a7844 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 29 Jul 2024 09:17:31 -0700 Subject: [PATCH 30/41] `maitake` should also include new docs and feature flags --- maitake-sync/README.md | 4 +-- maitake/Cargo.toml | 4 ++- maitake/README.md | 56 ++++++++++++++++++++++++++++++++++++++++-- 3 files changed, 59 insertions(+), 5 deletions(-) diff --git a/maitake-sync/README.md b/maitake-sync/README.md index adb4e549..92be1133 100644 --- a/maitake-sync/README.md +++ b/maitake-sync/README.md @@ -191,9 +191,9 @@ details. https://docs.rs/maitake-sync/latest/maitake_sync/blocking/struct.DefaultMutex.html [spinlock]: https://en.wikipedia.org/wiki/Spinlock [`RawMutex`]: - https://docs.rs/mutex-traitsc/latest/mutex_traits/trait.RawMutex.html + https://docs.rs/mutex-traits/latest/mutex_traits/trait.RawMutex.html [`ScopedRawMutex`]: - https://docs.rs/mutex-traitsc/latest/mutex_traits/trait.ScopedRawMutex.html + https://docs.rs/mutex-traits/latest/mutex_traits/trait.ScopedRawMutex.html [`mutex-traits`]: https://crates.io/crates/mutex-traits [`lock_api`]: https://crates.io/crates/lock_api [`critical-section`]: https://crates.io/crates/critical-section diff --git a/maitake/Cargo.toml b/maitake/Cargo.toml index 097042c2..b71d0642 100644 --- a/maitake/Cargo.toml +++ b/maitake/Cargo.toml @@ -31,8 +31,10 @@ links = "maitake" [features] default = ["alloc"] alloc = ["cordyceps/alloc", "maitake-sync/alloc"] +std = ["alloc", "maitake-sync/std"] no-cache-pad = ["mycelium-util/no-cache-pad", "cordyceps/no-cache-pad", "maitake-sync/no-cache-pad"] core-error = ["maitake-sync/core-error"] +critical-section = ["maitake-sync/critical-section"] [dependencies] mycelium-bitfield = { path = "../bitfield" } @@ -76,4 +78,4 @@ rustdoc-args = ["--cfg", "docsrs"] [[example]] name = "tokio-console" -required-features = ["tracing-01"] \ No newline at end of file +required-features = ["tracing-01"] diff --git a/maitake/README.md b/maitake/README.md index 32da7423..5154777a 100644 --- a/maitake/README.md +++ b/maitake/README.md @@ -253,7 +253,55 @@ critical section, are described [here][interrupt-cfgs]. [`portable-atomic`]: https://crates.io/crates/portable-atomic [`RUSTFLAGS`]: https://doc.rust-lang.org/cargo/reference/config.html#buildrustflags [single-core]: https://docs.rs/portable-atomic/latest/portable_atomic/#optional-cfg -[interrupt-cfgs]: https://github.com/taiki-e/portable-atomic/blob/HEAD/src/imp/interrupt/README.md +[interrupt-cfgs]: + https://github.com/taiki-e/portable-atomic/blob/HEAD/src/imp/interrupt/README.md + +### overriding blocking mutex implementations + +In addition to async locks, [`maitake::sync`][sync] also provides a [`blocking`] +module, which contains blocking [`blocking::Mutex`] and [`blocking::RwLock`] +types. Many of `maitake::sync`'s async synchronization primitives, including +[`WaitQueue`], [`Mutex`], [`RwLock`], and [`Semaphore`], internally use the +[`blocking::Mutex`] type for wait-list synchronization. By default, this type +uses a [`blocking::DefaultMutex`][`DefaultMutex`] as the underlying mutex +implementation, which attempts to provide the best generic mutex implementation +based on the currently enabled feature flags. + +However, in some cases, it may be desirable to provide a custom mutex +implementation. Therefore, `maitake::sync`'s [`blocking::Mutex`] type, and the +async synchronization primitives that depend on it, are generic over a `Lock` +type parameter which may be overridden using the [`RawMutex`] and +[`ScopedRawMutex`] traits from the [`mutex-traits`] crate, allowing alternative +blocking mutex implementations to be used with `maitake::sync`. Using the +[`mutex-traits`] adapters in the [`mutex`] crate, `maitake::sync`'s types may +also be used with raw mutex implementations that implement traits from the +[`lock_api`] and [`critical-section`] crates. + +See [the documentation on overriding mutex implementations][overriding] for more +details. + +[`blocking`]: + https://mycelium.elizas.website/maitake/sync/blocking/index.html +[`blocking::Mutex`]: + https://mycelium.elizas.website/maitake/sync/blocking/struct.Mutex.html +[`blocking::RwLock`]: + https://mycelium.elizas.website/maitake/sync/blocking/struct.RwLock.html +[`DefaultMutex`]: + https://mycelium.elizas.website/maitake/sync/blocking/struct.DefaultMutex.html +[`WaitQueue`]: + https://mycelium.elizas.website/maitake/sync/struct.WaitQueue.html +[`WaitMap`]: + https://mycelium.elizas.website/maitake/sync/struct.WaitMap.html +[spinlock]: https://en.wikipedia.org/wiki/Spinlock +[`RawMutex`]: + https://docs.rs/mutex-traits/latest/mutex_traits/trait.RawMutex.html +[`ScopedRawMutex`]: + https://docs.rs/mutex-traits/latest/mutex_traits/trait.ScopedRawMutex.html +[`mutex-traits`]: https://crates.io/crates/mutex-traits +[`lock_api`]: https://crates.io/crates/lock_api +[`critical-section`]: https://crates.io/crates/critical-section +[overriding]: + https://mycelium.elizas.website/maitake/sync/blocking/index.html#overriding-mutex-implementations ## features @@ -264,6 +312,8 @@ The following features are available (this list is incomplete; you can help by [ | Feature | Default | Explanation | | :--- | :--- | :--- | | `alloc` | `true` | Enables [`liballoc`] dependency | +| `std` | `false` | Enables the Rust standard library, disabling `#![no-std]`. When `std` is enabled, the [`DefaultMutex`] type will use [`std::sync::Mutex`]. This implies the `alloc` feature. | +| `critical-section` | `false` | Enables a variant of the [`DefaultMutex`] type that uses the [`critical-section`] crate. | | `no-cache-pad` | `false` | Inhibits cache padding for the [`CachePadded`] struct. When this feature is NOT enabled, the size will be determined based on target platform. | | `tracing-01` | `false` | Enables support for v0.1.x of [`tracing`] (the current release version). Requires `liballoc`.| | `tracing-02` | `false` | Enables support for the upcoming v0.2 of [`tracing`] (via a Git dependency). | @@ -272,4 +322,6 @@ The following features are available (this list is incomplete; you can help by [ [`liballoc`]: https://doc.rust-lang.org/alloc/ [`CachePadded`]: https://mycelium.elizas.website/mycelium_util/sync/struct.cachepadded [`tracing`]: https://crates.io/crates/tracing -[core-error]: https://doc.rust-lang.org/stable/core/error/index.html \ No newline at end of file +[core-error]: https://doc.rust-lang.org/stable/core/error/index.html +[`std::sync::Mutex`]: + https://doc.rust-lang.org/stable/std/sync/struct.Mutex.html From 45e638064b14fcb13772804dc68810250623115d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 29 Jul 2024 09:20:53 -0700 Subject: [PATCH 31/41] `critical-section` features should also set `portable-atomic`'s feature --- Cargo.lock | 4 ++++ maitake-sync/Cargo.toml | 1 + maitake-sync/README.md | 2 +- maitake/Cargo.toml | 2 +- maitake/README.md | 3 ++- 5 files changed, 9 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 31960cd1..e7a986ba 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1056,6 +1056,7 @@ name = "hal-core" version = "0.1.0" dependencies = [ "embedded-graphics-core", + "maitake-sync", "mycelium-util", "tracing 0.2.0", ] @@ -1897,6 +1898,9 @@ name = "portable-atomic" version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7170ef9988bc169ba16dd36a7fa041e5c4cbeb6a35b76d4c03daded371eae7c0" +dependencies = [ + "critical-section", +] [[package]] name = "powerfmt" diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index 0a604265..66b03f19 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -26,6 +26,7 @@ alloc = ["cordyceps/alloc"] std = ["alloc"] no-cache-pad = ["cordyceps/no-cache-pad"] core-error = [] +critical-section = ["dep:critical-section", "portable-atomic/critical-section"] [dependencies] mycelium-bitfield = { version = "0.1.3", path = "../bitfield" } diff --git a/maitake-sync/README.md b/maitake-sync/README.md index 92be1133..e8897d20 100644 --- a/maitake-sync/README.md +++ b/maitake-sync/README.md @@ -210,7 +210,7 @@ The following features are available (this list is incomplete; you can help by [ | :--- | :--- | :--- | | `alloc` | `true` | Enables [`liballoc`] dependency | | `std` | `false` | Enables the Rust standard library, disabling `#![no-std]`. When `std` is enabled, the [`DefaultMutex`] type will use [`std::sync::Mutex`]. This implies the `alloc` feature. | -| `critical-section` | `false` | Enables a variant of the [`DefaultMutex`] type that uses the [`critical-section`] crate. | +| `critical-section` | `false` | Enables support for the [`critical-section`] crate. This includes a variant of the [`DefaultMutex`] type that uses a critical section, as well as the [`portable-atomic`] crate's `critical-section` feature (as [discussed above](#support-for-atomic-operations)) | | `no-cache-pad` | `false` | Inhibits cache padding for the [`CachePadded`] struct. When this feature is NOT enabled, the size will be determined based on target platform. | | `tracing` | `false` | Enables support for [`tracing`] diagnostics. Requires `liballoc`.| | `core-error` | `false` | Enables implementations of the [`core::error::Error` trait][core-error] for `maitake-sync`'s error types. *Requires a nightly Rust toolchain*. | diff --git a/maitake/Cargo.toml b/maitake/Cargo.toml index b71d0642..879f2883 100644 --- a/maitake/Cargo.toml +++ b/maitake/Cargo.toml @@ -34,7 +34,7 @@ alloc = ["cordyceps/alloc", "maitake-sync/alloc"] std = ["alloc", "maitake-sync/std"] no-cache-pad = ["mycelium-util/no-cache-pad", "cordyceps/no-cache-pad", "maitake-sync/no-cache-pad"] core-error = ["maitake-sync/core-error"] -critical-section = ["maitake-sync/critical-section"] +critical-section = ["maitake-sync/critical-section", "portable-atomic/critical-section"] [dependencies] mycelium-bitfield = { path = "../bitfield" } diff --git a/maitake/README.md b/maitake/README.md index 5154777a..3f38b869 100644 --- a/maitake/README.md +++ b/maitake/README.md @@ -313,7 +313,8 @@ The following features are available (this list is incomplete; you can help by [ | :--- | :--- | :--- | | `alloc` | `true` | Enables [`liballoc`] dependency | | `std` | `false` | Enables the Rust standard library, disabling `#![no-std]`. When `std` is enabled, the [`DefaultMutex`] type will use [`std::sync::Mutex`]. This implies the `alloc` feature. | -| `critical-section` | `false` | Enables a variant of the [`DefaultMutex`] type that uses the [`critical-section`] crate. | + +| `critical-section` | `false` | Enables support for the [`critical-section`] crate. This includes a variant of the [`DefaultMutex`] type that uses a critical section, as well as the [`portable-atomic`] crate's `critical-section` feature (as [discussed above](#support-for-atomic-operations)) | | `no-cache-pad` | `false` | Inhibits cache padding for the [`CachePadded`] struct. When this feature is NOT enabled, the size will be determined based on target platform. | | `tracing-01` | `false` | Enables support for v0.1.x of [`tracing`] (the current release version). Requires `liballoc`.| | `tracing-02` | `false` | Enables support for the upcoming v0.2 of [`tracing`] (via a Git dependency). | From dd6656ea1d2980a0e36827518f100085d673f39d Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Mon, 29 Jul 2024 09:30:36 -0700 Subject: [PATCH 32/41] remove `spin::Mutex` aliases --- hal-core/Cargo.toml | 3 +- hal-core/src/framebuffer.rs | 5 +++- hal-x86_64/src/interrupt.rs | 19 +++++++----- hal-x86_64/src/serial.rs | 12 +++++--- hal-x86_64/src/time/pit.rs | 4 +-- hal-x86_64/src/vga.rs | 8 ++--- maitake-sync/src/spin.rs | 41 +++++++++----------------- src/arch/x86_64/framebuf.rs | 15 +++++----- src/drivers/ps2_keyboard.rs | 58 ++++++++++++++++++------------------- trace/src/writer.rs | 21 ++++++++------ 10 files changed, 93 insertions(+), 93 deletions(-) diff --git a/hal-core/Cargo.toml b/hal-core/Cargo.toml index 1884cdfa..786b4223 100644 --- a/hal-core/Cargo.toml +++ b/hal-core/Cargo.toml @@ -8,6 +8,7 @@ license = "MIT" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] -tracing = { git = "https://github.com/tokio-rs/tracing", default_features = false } +tracing = { git = "https://github.com/tokio-rs/tracing", default_features = false } +maitake-sync = { path = "../maitake-sync", default-features = false } mycelium-util = { path = "../util" } embedded-graphics-core = { version = "0.3", optional = true } diff --git a/hal-core/src/framebuffer.rs b/hal-core/src/framebuffer.rs index 6db001d3..32465ac6 100644 --- a/hal-core/src/framebuffer.rs +++ b/hal-core/src/framebuffer.rs @@ -1,4 +1,5 @@ use core::ops::{Deref, DerefMut}; +use maitake_sync::blocking; #[cfg(feature = "embedded-graphics-core")] #[doc(cfg(feature = "embedded-graphics-core"))] @@ -6,6 +7,7 @@ mod embedded_graphics; #[cfg(feature = "embedded-graphics-core")] #[doc(cfg(feature = "embedded-graphics-core"))] pub use self::embedded_graphics::*; + pub trait Draw { /// Return the width of the framebuffer in pixels. fn width(&self) -> usize; @@ -185,9 +187,10 @@ macro_rules! deref_draw_body { }; } -impl<'lock, D> Draw for mycelium_util::sync::spin::MutexGuard<'lock, D> +impl<'lock, D, L> Draw for blocking::MutexGuard<'lock, D, L> where D: Draw, + L: blocking::RawMutex, { deref_draw_body! {} } diff --git a/hal-x86_64/src/interrupt.rs b/hal-x86_64/src/interrupt.rs index 4c224e66..c66f3f89 100644 --- a/hal-x86_64/src/interrupt.rs +++ b/hal-x86_64/src/interrupt.rs @@ -4,7 +4,11 @@ use hal_core::interrupt::Control; use hal_core::interrupt::{ctx, Handlers}; use mycelium_util::{ bits, fmt, - sync::{spin, InitOnce}, + sync::{ + blocking::{Mutex, MutexGuard}, + spin::Spinlock, + InitOnce, + }, }; pub mod apic; @@ -56,7 +60,7 @@ pub struct Interrupt { enum InterruptModel { /// Interrupts are handled by the [8259 Programmable Interrupt Controller /// (PIC)](pic). - Pic(spin::Mutex), + Pic(Mutex), /// Interrupts are handled by the [local] and [I/O] [Advanced Programmable /// Interrupt Controller (APIC)s][apics]. /// @@ -68,7 +72,7 @@ enum InterruptModel { // TODO(eliza): allow further configuration of the I/O APIC (e.g. // masking/unmasking stuff...) #[allow(dead_code)] - io: spin::Mutex, + io: Mutex, }, } @@ -127,14 +131,13 @@ pub struct Registers { _pad2: [u16; 3], } -static IDT: spin::Mutex = - spin::Mutex::new_with_raw_mutex(idt::Idt::new(), spin::Spinlock::new()); +static IDT: Mutex = Mutex::new_with_raw_mutex(idt::Idt::new(), Spinlock::new()); static INTERRUPT_CONTROLLER: InitOnce = InitOnce::uninitialized(); impl Controller { // const DEFAULT_IOAPIC_BASE_PADDR: u64 = 0xFEC00000; - pub fn idt() -> spin::MutexGuard<'static, idt::Idt> { + pub fn idt() -> MutexGuard<'static, idt::Idt, Spinlock> { IDT.lock() } @@ -208,7 +211,7 @@ impl Controller { InterruptModel::Apic { local, - io: spin::Mutex::new_with_raw_mutex(io, spin::Spinlock::new()), + io: Mutex::new_with_raw_mutex(io, Spinlock::new()), } } model => { @@ -226,7 +229,7 @@ impl Controller { // clear for you, the reader, that at this point they are definitely intentionally enabled. pics.enable(); } - InterruptModel::Pic(spin::Mutex::new_with_raw_mutex(pics, spin::Spinlock::new())) + InterruptModel::Pic(Mutex::new_with_raw_mutex(pics, Spinlock::new())) } }; tracing::trace!(interrupt_model = ?model); diff --git a/hal-x86_64/src/serial.rs b/hal-x86_64/src/serial.rs index 06311a4a..f1900f94 100644 --- a/hal-x86_64/src/serial.rs +++ b/hal-x86_64/src/serial.rs @@ -3,7 +3,11 @@ use crate::cpu; use core::{fmt, marker::PhantomData}; use mycelium_util::{ io, - sync::{spin, Lazy}, + sync::{ + blocking::{Mutex, MutexGuard}, + spin::Spinlock, + Lazy, + }, }; static COM1: Lazy> = Lazy::new(|| Port::new(0x3F8).ok()); @@ -29,7 +33,7 @@ pub fn com4() -> Option<&'static Port> { // #[derive(Debug)] pub struct Port { - inner: spin::Mutex, + inner: Mutex, } // #[derive(Debug)] @@ -40,7 +44,7 @@ pub struct Lock<'a, B = Blocking> { } struct LockInner<'a> { - inner: spin::MutexGuard<'a, Registers>, + inner: MutexGuard<'a, Registers, Spinlock>, prev_divisor: Option, } @@ -110,7 +114,7 @@ impl Port { })?; Ok(Self { - inner: spin::Mutex::new_with_raw_mutex(registers, spin::Spinlock::new()), + inner: Mutex::new_with_raw_mutex(registers, Spinlock::new()), }) } diff --git a/hal-x86_64/src/time/pit.rs b/hal-x86_64/src/time/pit.rs index 32761ade..df2e5731 100644 --- a/hal-x86_64/src/time/pit.rs +++ b/hal-x86_64/src/time/pit.rs @@ -9,7 +9,7 @@ use core::{ use mycelium_util::{ bits::{bitfield, enum_from_bits}, fmt, - sync::spin::{Mutex, Spinlock}, + sync::{blocking::Mutex, spin::Spinlock}, }; /// Intel 8253/8254 Programmable Interval Timer (PIT). @@ -210,7 +210,7 @@ enum_from_bits! { /// publicly and is represented as a singleton. It's stored in a [`Mutex`] in /// order to ensure that multiple CPU cores don't try to write conflicting /// configurations to the PIT's configuration ports. -pub static PIT: Mutex = Mutex::new_with_raw_mutex(Pit::new(), Spinlock::new()); +pub static PIT: Mutex = Mutex::new_with_raw_mutex(Pit::new(), Spinlock::new()); /// Are we currently sleeping on an interrupt? static SLEEPING: AtomicBool = AtomicBool::new(false); diff --git a/hal-x86_64/src/vga.rs b/hal-x86_64/src/vga.rs index 794c5742..92690cac 100644 --- a/hal-x86_64/src/vga.rs +++ b/hal-x86_64/src/vga.rs @@ -1,18 +1,18 @@ use core::fmt; use mycelium_util::{ io, - sync::{spin, Lazy}, + sync::{blocking::Mutex, spin::Spinlock, Lazy}, }; use volatile::Volatile; -static BUFFER: Lazy> = Lazy::new(|| { - spin::Mutex::new_with_raw_mutex( +static BUFFER: Lazy> = Lazy::new(|| { + Mutex::new_with_raw_mutex( Buffer { col: 0, row: 0, color: ColorSpec::new(Color::LightGray, Color::Black), buf: Volatile::new(unsafe { &mut *(0xb8000u64 as *mut Buf) }), }, - spin::Spinlock::new(), + Spinlock::new(), ) }); diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index 743450ab..20003aa2 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -20,8 +20,10 @@ //! //! This module provides the following APIs: //! -//! - [`Mutex`]: a synchronous [mutual exclusion] spinlock. -//! - [`RwLock`]: a synchronous [reader-writer] spinlock. +//! - [`Spinlock`]: a synchronous [mutual exclusion] spinlock, which implements +//! the [`blocking::RawMutex`] and [`blocking::RawScopedMutex`] traits. +//! - [`RwSpinlock`]: a synchronous [reader-writer] spinlock, which implements +//! the [`blocking::RawRwLock`] trait. //! - [`InitOnce`]: a cell storing a [`MaybeUninit`](core::mem::MaybeUninit) //! value which must be manually initialized prior to use. //! - [`Lazy`]: an [`InitOnce`] cell coupled with an initializer function. The @@ -40,41 +42,24 @@ use crate::{ util::{fmt, Backoff}, }; -/// A type alias for a [`blocking::Mutex`] which explicitly uses a [`Spinlock`] -/// for synchronization. See the [`blocking::Mutex`] type's documentation for -/// details. -pub type Mutex = blocking::Mutex; - -/// A type alias for a [`blocking::MutexGuard`] returned by a -/// [`spin::Mutex`](super::Mutex). See the [`blocking::MutexGuard`] type's -/// documentation for details. -pub type MutexGuard<'a, T> = blocking::MutexGuard<'a, T, Spinlock>; - -/// A type alias for a [`blocking::RwLock`] which explicitly uses a [`RwSpinlock`] -/// for synchronization. See the [`blocking::RwLock`] type's documentation for -/// details. -pub type RwLock = blocking::RwLock; - -/// A type alias for a [`blocking::RwLockReadGuard`] returned by a -/// [`spin::RwLock`](super::RwLock). See the [`blocking::RwLockReadGuard`] type's -/// documentation for details. -pub type RwLockReadGuard<'a, T> = blocking::RwLockReadGuard<'a, T, RwSpinlock>; - -/// A type alias for a [`blocking::RwLockWriteGuard`] returned by a -/// [`spin::RwLock`](super::RwLock). See the [`blocking::RwLockWriteGuard`] type's -/// documentation for details. -pub type RwLockWriteGuard<'a, T> = blocking::RwLockWriteGuard<'a, T, RwSpinlock>; - /// A spinlock-based [`RawMutex`] implementation. /// /// This mutex will spin with an exponential backoff while waiting for the lock /// to become available. +/// +/// This type implements the [`RawMutex`] and [`RawScopedMutex`] traits from the +/// [`mutex-traits`] crate. This allows it to be used with the +/// [`blocking::Mutex`] type when a spinlock-based mutex is needed. #[derive(Debug)] pub struct Spinlock { locked: AtomicBool, } /// A spinlock-based [`RawRwLock`] implementation. +/// +/// This type implements the [`blocking::RawRwLock`] trait. This allows it to be +/// used with the [`blocking::RwLock`] type when a spinlock-based reader-writer +/// lock is needed. pub struct RwSpinlock { state: AtomicUsize, } @@ -268,7 +253,7 @@ impl blocking::ConstInit for RwSpinlock { impl fmt::Debug for RwSpinlock { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { let state = &self.state.load(Relaxed); - f.debug_struct("RawSpinRwLock") + f.debug_struct("RwSpinlock") // N.B.: this does *not* use the `reader_count` and `has_writer` // methods *intentionally*, because those two methods perform // independent reads of the lock's state, and may race with other diff --git a/src/arch/x86_64/framebuf.rs b/src/arch/x86_64/framebuf.rs index fac49270..5fa161ce 100644 --- a/src/arch/x86_64/framebuf.rs +++ b/src/arch/x86_64/framebuf.rs @@ -4,10 +4,14 @@ use core::{ ops::{Deref, DerefMut}, }; use hal_x86_64::framebuffer::{self, Framebuffer}; -use mycelium_util::sync::{spin, InitOnce}; +use mycelium_util::sync::{ + blocking::{Mutex, MutexGuard}, + spin::Spinlock, + InitOnce, +}; #[derive(Debug)] -pub struct FramebufGuard(spin::MutexGuard<'static, info::FrameBuffer>); +pub struct FramebufGuard(MutexGuard<'static, info::FrameBuffer, Spinlock>); pub type FramebufWriter = Framebuffer<'static, FramebufGuard>; /// Locks the framebuffer and returns a [`FramebufWriter`]. @@ -74,14 +78,11 @@ pub(super) fn init(bootinfo: &mut BootInfo) -> bool { x => unimplemented!("hahaha wtf, found a weird pixel format: {:?}", x), }, }; - FRAMEBUFFER.init(( - cfg, - spin::Mutex::new_with_raw_mutex(framebuffer, spin::Spinlock::new()), - )); + FRAMEBUFFER.init((cfg, Mutex::new_with_raw_mutex(framebuffer, Spinlock::new()))); true } -static FRAMEBUFFER: InitOnce<(framebuffer::Config, spin::Mutex)> = +static FRAMEBUFFER: InitOnce<(framebuffer::Config, Mutex)> = InitOnce::uninitialized(); impl Deref for FramebufGuard { diff --git a/src/drivers/ps2_keyboard.rs b/src/drivers/ps2_keyboard.rs index 36a29817..8fe3e628 100644 --- a/src/drivers/ps2_keyboard.rs +++ b/src/drivers/ps2_keyboard.rs @@ -1,8 +1,5 @@ use maitake::sync::WaitQueue; -use mycelium_util::{ - fmt, - sync::spin::{self, Mutex}, -}; +use mycelium_util::{fmt, sync::blocking::Mutex}; use pc_keyboard::{layouts, Keyboard}; pub use pc_keyboard::{DecodedKey, KeyCode}; @@ -22,11 +19,10 @@ pub struct Ps2Keyboard { static PS2_KEYBOARD: Ps2Keyboard = Ps2Keyboard { buf: thingbuf::StaticThingBuf::new(), - kbd: Mutex::new_with_raw_mutex( + kbd: Mutex::new( Keyboard::::new( pc_keyboard::HandleControl::MapLettersToUnicode, ), - spin::Spinlock::new(), ), waiters: WaitQueue::new(), }; @@ -49,29 +45,33 @@ pub async fn next_key() -> DecodedKey { } pub(crate) fn handle_scancode(scancode: u8) { - let mut kbd = PS2_KEYBOARD + PS2_KEYBOARD .kbd - .try_lock() - .expect("handle_scancode should only be called in an ISR!"); - match kbd.add_byte(scancode) { - Err(error) => { - tracing::warn!( - ?error, - scancode = fmt::hex(&scancode), - "error decoding scancode, ignoring it!" - ); - } - // state advanced, no character decoded yet - Ok(None) => {} - // got a key event - Ok(Some(event)) => { - if let Some(decoded_key) = kbd.process_keyevent(event) { - // got something! - if let Err(decoded_key) = PS2_KEYBOARD.buf.push(Some(decoded_key)) { - tracing::warn!(?decoded_key, "keyboard buffer full, dropping key event!") + .try_with_lock(|kbd| { + match kbd.add_byte(scancode) { + Err(error) => { + tracing::warn!( + ?error, + scancode = fmt::hex(&scancode), + "error decoding scancode, ignoring it!" + ); } - PS2_KEYBOARD.waiters.wake_all(); - } - } - }; + // state advanced, no character decoded yet + Ok(None) => {} + // got a key event + Ok(Some(event)) => { + if let Some(decoded_key) = kbd.process_keyevent(event) { + // got something! + if let Err(decoded_key) = PS2_KEYBOARD.buf.push(Some(decoded_key)) { + tracing::warn!( + ?decoded_key, + "keyboard buffer full, dropping key event!" + ) + } + PS2_KEYBOARD.waiters.wake_all(); + } + } + }; + }) + .expect("handle_scancode should only be called in an ISR!"); } diff --git a/trace/src/writer.rs b/trace/src/writer.rs index cc5db35b..9e2b9cd0 100644 --- a/trace/src/writer.rs +++ b/trace/src/writer.rs @@ -2,10 +2,11 @@ //! //! [`fmt::Write`]: mycelium_util::fmt::Write use crate::color::{Color, SetColor}; -use mycelium_util::{ - fmt::{self, Debug}, - sync::spin::{Mutex, MutexGuard}, +use maitake::sync::{ + blocking::{Mutex, MutexGuard, RawMutex}, + spin::Spinlock, }; +use mycelium_util::fmt::{self, Debug}; use tracing_core::Metadata; /// A type that can create [`fmt::Write`] instances. @@ -202,11 +203,11 @@ pub trait MakeWriterExt<'a>: MakeWriter<'a> { /// a newtype that forwards the trait implementation. /// /// [`fmt::Write`]: mycelium_util::fmt::Write -/// [`MutexGuard`]: mycelium_util::sync::spin::MutexGuard -/// [`Mutex`]: mycelium_util::sync::spin::Mutex +/// [`MutexGuard`]: maitake_sync::blocking:::MutexGuard +/// [`Mutex`]: mycelium_util::blocking::Mutex /// [`MakeWriter`]: trait.MakeWriter.html #[derive(Debug)] -pub struct MutexGuardWriter<'a, W>(MutexGuard<'a, W>); +pub struct MutexGuardWriter<'a, W, Lock: RawMutex = Spinlock>(MutexGuard<'a, W, Lock>); // TODO(eliza): put this back if needed /* @@ -382,20 +383,22 @@ where // === impl Mutex/MutexGuardWriter === -impl<'a, W> MakeWriter<'a> for Mutex +impl<'a, W, Lock> MakeWriter<'a> for Mutex where W: fmt::Write + 'a, + Lock: RawMutex + 'a, { - type Writer = MutexGuardWriter<'a, W>; + type Writer = MutexGuardWriter<'a, W, Lock>; fn make_writer(&'a self) -> Self::Writer { MutexGuardWriter(self.lock()) } } -impl<'a, W> fmt::Write for MutexGuardWriter<'a, W> +impl<'a, W, Lock> fmt::Write for MutexGuardWriter<'a, W, Lock> where W: fmt::Write, + Lock: RawMutex, { #[inline] fn write_str(&mut self, s: &str) -> fmt::Result { From 99ce4e19fd9372f311e83da30c08002da35d8194 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 08:33:57 -0700 Subject: [PATCH 33/41] remove remaining `spin::Mutex` references --- maitake-sync/src/blocking/rwlock.rs | 4 ++-- maitake/src/task/task_list.rs | 6 ++++-- maitake/tests/scheduler/alloc.rs | 4 ++-- trace/src/writer.rs | 5 +++-- util/src/sync/spin/mutex.rs | 2 +- 5 files changed, 12 insertions(+), 9 deletions(-) diff --git a/maitake-sync/src/blocking/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs index 6f4d6f94..f6ad4351 100644 --- a/maitake-sync/src/blocking/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -17,7 +17,7 @@ use core::ops::{Deref, DerefMut}; /// of the underlying data (exclusive access) and the read portion of this lock /// typically allows for read-only access (shared access). /// -/// In comparison, a [`spin::Mutex`] does not distinguish between readers or writers +/// In comparison, a [`blocking::Mutex`] does not distinguish between readers or writers /// that acquire the lock, therefore blocking any threads waiting for the lock to /// become available. An `RwLock` will allow any number of readers to acquire the /// lock as long as a writer is not holding the lock. @@ -31,7 +31,7 @@ use core::ops::{Deref, DerefMut}; /// When `cfg(loom)` is enabled, this mutex will use Loom's simulated atomics, /// checked `UnsafeCell`, and simulated spin loop hints. /// -/// [`spin::Mutex`]: crate::spin::Mutex +/// [`blocking::Mutex`]: crate::blocking::Mutex /// [readers-writer lock]: https://en.wikipedia.org/wiki/Readers%E2%80%93writer_lock pub struct RwLock { lock: Lock, diff --git a/maitake/src/task/task_list.rs b/maitake/src/task/task_list.rs index 7aa6a15f..15a5e2cc 100644 --- a/maitake/src/task/task_list.rs +++ b/maitake/src/task/task_list.rs @@ -1,7 +1,9 @@ use super::Task; -use mycelium_util::{intrusive::list, sync::spin}; +use maitake_sync::blocking; +use mycelium_util::intrusive::list; + pub(crate) struct TaskList { - inner: spin::Mutex, + inner: blocking::Mutex, } struct Inner { diff --git a/maitake/tests/scheduler/alloc.rs b/maitake/tests/scheduler/alloc.rs index 00bdcde3..c2e94a3b 100644 --- a/maitake/tests/scheduler/alloc.rs +++ b/maitake/tests/scheduler/alloc.rs @@ -1,5 +1,5 @@ use super::*; -use mycelium_util::sync::{spin::Mutex, Lazy}; +use mycelium_util::sync::{blocking::Mutex, spin::Spinlock, Lazy}; #[test] fn basically_works() { @@ -71,7 +71,7 @@ fn many_yields() { fn steal_blocked() { static SCHEDULER_1: Lazy = Lazy::new(StaticScheduler::new); static SCHEDULER_2: Lazy = Lazy::new(StaticScheduler::new); - static MUTEX: Mutex<()> = Mutex::new(()); + static MUTEX: Mutex<(), Spinlock> = Mutex::new_with_raw_mutex((), Spinlock::new()); static READY: AtomicBool = AtomicBool::new(false); static IT_WORKED: AtomicBool = AtomicBool::new(false); diff --git a/trace/src/writer.rs b/trace/src/writer.rs index 9e2b9cd0..71b9151c 100644 --- a/trace/src/writer.rs +++ b/trace/src/writer.rs @@ -13,8 +13,9 @@ use tracing_core::Metadata; /// /// This trait is already implemented for function pointers and /// immutably-borrowing closures that return an instance of [`fmt::Write`], -/// Additionally, it is implemented for [`mycelium_util::sync::spin::Mutex`] -/// when the type inside the mutex implements [`fmt::Write`]. +/// Additionally, it is implemented for [`maitake::sync::blocking::Mutex`] +/// when the type inside the mutex implements [`fmt::Write`] and the `Lock` type +/// implements [`RawMutex`]. /// /// The [`MakeWriter::make_writer_for`] method takes [`Metadata`] describing a /// span or event and returns a writer. `MakeWriter`s can optionally provide diff --git a/util/src/sync/spin/mutex.rs b/util/src/sync/spin/mutex.rs index 355e1cf0..9c01ecb3 100644 --- a/util/src/sync/spin/mutex.rs +++ b/util/src/sync/spin/mutex.rs @@ -58,7 +58,7 @@ impl Mutex { /// # Examples /// /// ``` - /// use mycelium_util::sync::spin::Mutex; + /// use mycelium_util::sync::blocking::Mutex; /// /// let mutex = Mutex::new(0); /// ``` From f3707939df12fecacf534c467726d56f167b885b Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 08:43:09 -0700 Subject: [PATCH 34/41] sync: always enable `tracing` with `cfg(loom)` Loom depends on `tracing` anyway, so `tracing` is always in the dependency tree when `cfg(loom)` is set. So, there isn't really any sense in disabling `tracing` in `maitake-sync` when `cfg(loom)` is set, even if the tracing feature isn't explicitly enabled. --- maitake-sync/Cargo.toml | 1 + maitake-sync/src/blocking/default_mutex.rs | 32 ++++++++-------------- maitake-sync/src/spin.rs | 2 +- maitake-sync/src/util.rs | 4 +-- maitake-sync/src/util/backoff.rs | 2 ++ maitake-sync/src/util/fmt.rs | 2 +- 6 files changed, 18 insertions(+), 25 deletions(-) diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index 66b03f19..eadcdb39 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -42,6 +42,7 @@ tracing = { version = "0.1", default_features = false, optional = true } # the cfg is enabled). [target.'cfg(loom)'.dependencies] loom = { version = "0.7", default_features = false } +tracing = { version = "0.1", default_features = false } [dev-dependencies] futures-util = "0.3" diff --git a/maitake-sync/src/blocking/default_mutex.rs b/maitake-sync/src/blocking/default_mutex.rs index 30dbcbd3..deac18aa 100644 --- a/maitake-sync/src/blocking/default_mutex.rs +++ b/maitake-sync/src/blocking/default_mutex.rs @@ -158,8 +158,8 @@ use spin_impl::SpinDefaultMutex as Inner; #[cfg(loom)] mod loom_impl { use super::ScopedRawMutex; - #[cfg(any(feature = "tracing", test))] use core::panic::Location; + use tracing::{debug, debug_span}; #[derive(Debug)] pub(super) struct LoomDefaultMutex(loom::sync::Mutex<()>); @@ -178,24 +178,21 @@ mod loom_impl { #[track_caller] #[inline] fn with_lock(&self, f: impl FnOnce() -> R) -> R { - #[cfg(any(feature = "tracing", test))] let location = Location::caller(); - #[cfg(any(feature = "tracing", test))] - tracing::trace!( + trace!( target: "maitake_sync::blocking", %location, "DefaultMutex::with_lock()", ); let guard = self.0.lock(); - let _span = tracing::debug_span!( + let _span = debug_span!( target: "maitake_sync::blocking", "locked", %location, ) .entered(); - #[cfg(any(feature = "tracing", test))] - tracing::debug!( + debug!( target: "maitake_sync::blocking", "DefaultMutex::with_lock() -> locked", ); @@ -203,8 +200,7 @@ mod loom_impl { let result = f(); drop(guard); - #[cfg(any(feature = "tracing", test))] - tracing::debug!( + debug!( target: "maitake_sync::blocking", "DefaultMutex::with_lock() -> unlocked", ); @@ -215,10 +211,8 @@ mod loom_impl { #[track_caller] #[inline] fn try_with_lock(&self, f: impl FnOnce() -> R) -> Option { - #[cfg(any(feature = "tracing", test))] let location = Location::caller(); - #[cfg(any(feature = "tracing", test))] - tracing::trace!( + trace!( target: "maitake_sync::blocking", %location, "DefaultMutex::try_with_lock()", @@ -226,11 +220,9 @@ mod loom_impl { match self.0.try_lock() { Ok(guard) => { - let _span = - tracing::debug_span!(target: "maitake_sync::blocking", "locked", %location) - .entered(); - #[cfg(any(feature = "tracing", test))] - tracing::debug!( + let _span = debug_span!(target: "maitake_sync::blocking", "locked", %location) + .entered(); + debug!( target: "maitake_sync::blocking", "DefaultMutex::try_with_lock() -> locked", ); @@ -238,8 +230,7 @@ mod loom_impl { let result = f(); drop(guard); - #[cfg(any(feature = "tracing", test))] - tracing::debug!( + debug!( target: "maitake_sync::blocking", "DefaultMutex::try_with_lock() -> unlocked", ); @@ -247,8 +238,7 @@ mod loom_impl { Some(result) } Err(_) => { - #[cfg(any(feature = "tracing", test))] - tracing::debug!( + debug!( target: "maitake_sync::blocking", %location, "DefaultMutex::try_with_lock() -> already locked", diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index 20003aa2..6153a683 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -37,7 +37,7 @@ pub mod once; pub use self::once::{InitOnce, Lazy}; use crate::{ - blocking::{self, RawMutex, RawRwLock}, + blocking::{RawMutex, RawRwLock}, loom::sync::atomic::{AtomicBool, AtomicUsize, Ordering::*}, util::{fmt, Backoff}, }; diff --git a/maitake-sync/src/util.rs b/maitake-sync/src/util.rs index 8b442759..bb15dc3a 100644 --- a/maitake-sync/src/util.rs +++ b/maitake-sync/src/util.rs @@ -10,12 +10,12 @@ //! - [`Backoff`]: exponential backoff for spin loops //! - [`CachePadded`]: pads and aligns a value to the size of a cache line -#[cfg(any(test, feature = "tracing"))] +#[cfg(any(test, feature = "tracing", loom))] macro_rules! trace { ($($t:tt)*) => { tracing::trace!($($t)*) } } -#[cfg(not(any(test, feature = "tracing")))] +#[cfg(not(any(test, feature = "tracing", loom)))] macro_rules! trace { ($($t:tt)*) => {}; } diff --git a/maitake-sync/src/util/backoff.rs b/maitake-sync/src/util/backoff.rs index 4a512d92..0208099a 100644 --- a/maitake-sync/src/util/backoff.rs +++ b/maitake-sync/src/util/backoff.rs @@ -51,7 +51,9 @@ impl Backoff { #[inline(always)] pub fn spin(&mut self) { // Issue 2^exp pause instructions. + #[cfg_attr(loom, allow(unused_variables))] let spins = 1 << self.exp; + #[cfg(not(loom))] for _ in 0..spins { crate::loom::hint::spin_loop(); diff --git a/maitake-sync/src/util/fmt.rs b/maitake-sync/src/util/fmt.rs index 59a7bffb..1b0fcf2a 100644 --- a/maitake-sync/src/util/fmt.rs +++ b/maitake-sync/src/util/fmt.rs @@ -19,7 +19,7 @@ pub(crate) struct FmtOption<'a, T> { // === impl FormatWith === -#[cfg(any(test, feature = "tracing"))] +#[cfg(any(test, feature = "tracing", loom))] #[inline] #[must_use] pub(crate) fn ptr(value: T) -> FormatWith { From 0be56fde269db504f454fb54631f43dbf08c6a09 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 08:50:30 -0700 Subject: [PATCH 35/41] oh THAT'S what the import was for --- maitake-sync/src/spin.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index 6153a683..02a2d928 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -126,7 +126,7 @@ unsafe impl RawMutex for Spinlock { } #[cfg(not(loom))] -impl blocking::ConstInit for Spinlock { +impl crate::blocking::ConstInit for Spinlock { // As usual, clippy is totally wrong about this --- the whole point of this // constant is to create a *new* spinlock every time. #[allow(clippy::declare_interior_mutable_const)] @@ -243,7 +243,7 @@ unsafe impl RawRwLock for RwSpinlock { } #[cfg(not(loom))] -impl blocking::ConstInit for RwSpinlock { +impl crate::blocking::ConstInit for RwSpinlock { // As usual, clippy is totally wrong about this --- the whole point of this // constant is to create a *new* spinlock every time. #[allow(clippy::declare_interior_mutable_const)] From 48c84af9bdb6bb19b81c17fb5adf04ae047cbfd6 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 09:54:16 -0700 Subject: [PATCH 36/41] fix docs import error --- maitake-sync/src/spin.rs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index 02a2d928..5d475ceb 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -37,10 +37,17 @@ pub mod once; pub use self::once::{InitOnce, Lazy}; use crate::{ - blocking::{RawMutex, RawRwLock}, + blocking, loom::sync::atomic::{AtomicBool, AtomicUsize, Ordering::*}, util::{fmt, Backoff}, }; +// This import is pulled out because we want to reference it in docs, and +// importing `use crate::blocking; use blocking::{RawMutex, RawRwLock};` makes +// `blocking` appear used even if it's not referenced directly, while +// `use crate::blocking::{self, RawMutex, RawRwLock};` causes the `self` to +// appear "unused" to rustc. Yes, this is stupid, but this workaround felt +// better than `allow(unused_imports)`. +use blocking::{RawMutex, RawRwLock}; /// A spinlock-based [`RawMutex`] implementation. /// @@ -126,7 +133,7 @@ unsafe impl RawMutex for Spinlock { } #[cfg(not(loom))] -impl crate::blocking::ConstInit for Spinlock { +impl blocking::ConstInit for Spinlock { // As usual, clippy is totally wrong about this --- the whole point of this // constant is to create a *new* spinlock every time. #[allow(clippy::declare_interior_mutable_const)] @@ -243,7 +250,7 @@ unsafe impl RawRwLock for RwSpinlock { } #[cfg(not(loom))] -impl crate::blocking::ConstInit for RwSpinlock { +impl blocking::ConstInit for RwSpinlock { // As usual, clippy is totally wrong about this --- the whole point of this // constant is to create a *new* spinlock every time. #[allow(clippy::declare_interior_mutable_const)] From ddb23464ec1ff47165c5a2ba05d2d5399d10e311 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 09:55:19 -0700 Subject: [PATCH 37/41] s/RawScopedMutex/ScopedRawMutex in docs --- maitake-sync/src/spin.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/maitake-sync/src/spin.rs b/maitake-sync/src/spin.rs index 5d475ceb..0f34f43e 100644 --- a/maitake-sync/src/spin.rs +++ b/maitake-sync/src/spin.rs @@ -21,7 +21,7 @@ //! This module provides the following APIs: //! //! - [`Spinlock`]: a synchronous [mutual exclusion] spinlock, which implements -//! the [`blocking::RawMutex`] and [`blocking::RawScopedMutex`] traits. +//! the [`blocking::RawMutex`] and [`blocking::ScopedRawMutex`] traits. //! - [`RwSpinlock`]: a synchronous [reader-writer] spinlock, which implements //! the [`blocking::RawRwLock`] trait. //! - [`InitOnce`]: a cell storing a [`MaybeUninit`](core::mem::MaybeUninit) @@ -54,7 +54,8 @@ use blocking::{RawMutex, RawRwLock}; /// This mutex will spin with an exponential backoff while waiting for the lock /// to become available. /// -/// This type implements the [`RawMutex`] and [`RawScopedMutex`] traits from the +/// This type implements the [`RawMutex`] and +/// [`ScopedRawMutex`](mutex_traits::ScopedRawMutex) traits from the /// [`mutex-traits`] crate. This allows it to be used with the /// [`blocking::Mutex`] type when a spinlock-based mutex is needed. #[derive(Debug)] From 3643e5cb9a13670594cbe09d3e448001f14ef680 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 10:03:07 -0700 Subject: [PATCH 38/41] fixup and document `Send`/`Sync` impls --- maitake-sync/src/blocking/mutex.rs | 31 +++++++++++++++++---------- maitake-sync/src/blocking/rwlock.rs | 33 ++++++++++++++++++----------- 2 files changed, 41 insertions(+), 23 deletions(-) diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 4a23ce74..959efa08 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -3,7 +3,10 @@ use crate::{ loom::cell::{MutPtr, UnsafeCell}, util::fmt, }; -use core::ops::{Deref, DerefMut}; +use core::{ + marker::PhantomData, + ops::{Deref, DerefMut}, +}; pub use mutex_traits::{RawMutex, ScopedRawMutex}; @@ -69,6 +72,7 @@ pub struct Mutex { pub struct MutexGuard<'a, T, Lock: RawMutex> { ptr: MutPtr, lock: &'a Lock, + _marker: PhantomData, } impl Mutex { @@ -215,6 +219,7 @@ where MutexGuard { ptr: self.data.get_mut(), lock: &self.lock, + _marker: PhantomData, } } @@ -314,8 +319,8 @@ where } } -unsafe impl Send for Mutex {} -unsafe impl Sync for Mutex {} +unsafe impl Send for Mutex {} +unsafe impl Sync for Mutex {} // === impl MutexGuard === @@ -395,17 +400,21 @@ where } } +/// A [`MutexGuard`] is only [`Send`] if: +/// +/// 1. the protected data (`T`) is `Send`, because the guard may be used to +/// mutably access the protected data, and can therefore be used to move it +/// using [`core::mem::replace`] or similar. +/// 2. the `Lock` type parameter is [`Sync`], because the guard contains a +/// reference to the `Lock` type, and therefore, sending the guard is sharing +/// a reference to the `Lock`. +/// 3. the `Lock` type's [`RawMutex::GuardMarker`] associated type is [`Send`], +/// because this indicates that the `Lock` type agrees that guards may be +/// [`Send`]. unsafe impl Send for MutexGuard<'_, T, Lock> where T: Send, - Lock: RawMutex, - Lock::GuardMarker: Send, -{ -} -unsafe impl Sync for MutexGuard<'_, T, Lock> -where - T: Send, - Lock: RawMutex, + Lock: RawMutex + Sync, Lock::GuardMarker: Send, { } diff --git a/maitake-sync/src/blocking/rwlock.rs b/maitake-sync/src/blocking/rwlock.rs index f6ad4351..dc4bac24 100644 --- a/maitake-sync/src/blocking/rwlock.rs +++ b/maitake-sync/src/blocking/rwlock.rs @@ -8,7 +8,10 @@ use crate::{ spin::RwSpinlock, util::fmt, }; -use core::ops::{Deref, DerefMut}; +use core::{ + marker::PhantomData, + ops::{Deref, DerefMut}, +}; /// A spinlock-based [readers-writer lock]. /// @@ -53,6 +56,7 @@ pub struct RwLock { pub struct RwLockReadGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { ptr: ConstPtr, lock: &'lock Lock, + _marker: PhantomData, } /// An RAII implementation of a "scoped write lock" of a [`RwLock`]. When this @@ -70,6 +74,7 @@ pub struct RwLockReadGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { pub struct RwLockWriteGuard<'lock, T: ?Sized, Lock: RawRwLock = RwSpinlock> { ptr: MutPtr, lock: &'lock Lock, + _marker: PhantomData, } /// Trait abstracting over blocking [`RwLock`] implementations (`maitake-sync`'s @@ -226,6 +231,7 @@ impl RwLock { RwLockReadGuard { ptr: self.data.get(), lock: &self.lock, + _marker: PhantomData, } } @@ -233,6 +239,7 @@ impl RwLock { RwLockWriteGuard { ptr: self.data.get_mut(), lock: &self.lock, + _marker: PhantomData, } } @@ -433,23 +440,25 @@ where } } -/// A [`RwLockReadGuard`] only allows immutable (`&T`) access to a `T`. -/// Therefore, it is [`Send`] and [`Sync`] as long as `T` is [`Sync`], because -/// it can be used to *share* references to a `T` across multiple threads -/// (requiring `T: Sync`), but it *cannot* be used to move ownership of a `T` -/// across thread boundaries, as the `T` cannot be taken out of the lock through -/// a `RwLockReadGuard`. -unsafe impl Send for RwLockReadGuard<'_, T, Lock> +/// A [`RwLockReadGuard`] is [`Sync`] if both `T` and the `Lock` type parameter +/// are [`Sync`]. +unsafe impl Sync for RwLockReadGuard<'_, T, Lock> where T: ?Sized + Sync, - Lock: RawRwLock, - Lock::GuardMarker: Send, + Lock: RawRwLock + Sync, { } -unsafe impl Sync for RwLockReadGuard<'_, T, Lock> +/// A [`RwLockReadGuard`] is [`Send`] if both `T` and the `Lock` type parameter +/// are [`Sync`], because sending a `RwLockReadGuard` is equivalent to sending a +/// `&(T, Lock)`. +/// +/// Additionally, the `Lock` type's [`RawRwLock::GuardMarker`] must indicate +/// that the guard is [`Send`]. +unsafe impl Send for RwLockReadGuard<'_, T, Lock> where T: ?Sized + Sync, - Lock: RawRwLock, + Lock: RawRwLock + Sync, + Lock::GuardMarker: Send, { } From 9e5a3bf61a9d00aa4c0c957296a338b42671d333 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 10:21:38 -0700 Subject: [PATCH 39/41] mutex-traits is stable now --- Cargo.lock | 5 +++-- Cargo.toml | 1 - maitake-sync/Cargo.toml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index e7a986ba..88156a48 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1540,8 +1540,9 @@ dependencies = [ [[package]] name = "mutex-traits" -version = "0.999.0" -source = "git+https://github.com/tosc-rs/scoped-mutex#67dedfa152e553ccee1ae78342eae7f6bb67cb96" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fd54cb762feb1788c74f5d3387983864d2365ca1819043d2addb76db80169102" [[package]] name = "mycelium-alloc" diff --git a/Cargo.toml b/Cargo.toml index 704f7a9b..f1ca61c1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -98,7 +98,6 @@ dynamic-range-start = "0xFFFF_8000_0000_0000" # use `tracing` 0.2 from git tracing = { git = "https://github.com/tokio-rs/tracing" } tracing-core = { git = "https://github.com/tokio-rs/tracing" } -mutex-traits = { git = "https://github.com/tosc-rs/scoped-mutex" } # Custom profile for Loom tests: enable release optimizations so that the loom # tests are less slow, but don't disable debug assertions. diff --git a/maitake-sync/Cargo.toml b/maitake-sync/Cargo.toml index eadcdb39..8607bdd9 100644 --- a/maitake-sync/Cargo.toml +++ b/maitake-sync/Cargo.toml @@ -30,7 +30,7 @@ critical-section = ["dep:critical-section", "portable-atomic/critical-section"] [dependencies] mycelium-bitfield = { version = "0.1.3", path = "../bitfield" } -mutex-traits = "0.999.0" +mutex-traits = "1.0.0" cordyceps = { version = "0.3.0", path = "../cordyceps" } critical-section = { version = "1.1", optional = true } pin-project = "1" From 3251c85b4fe2d78dfc221fa849ff608a07e52360 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 10:27:05 -0700 Subject: [PATCH 40/41] fix wrong sync impl for mutex --- maitake-sync/src/blocking/mutex.rs | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/maitake-sync/src/blocking/mutex.rs b/maitake-sync/src/blocking/mutex.rs index 959efa08..3fddc328 100644 --- a/maitake-sync/src/blocking/mutex.rs +++ b/maitake-sync/src/blocking/mutex.rs @@ -320,7 +320,18 @@ where } unsafe impl Send for Mutex {} -unsafe impl Sync for Mutex {} +/// A `Mutex` is [`Sync`] if `T` is [`Send`] and `Lock` is [`Sync`]. +/// +/// `T` must be [`Send`] because shared references to the `Mutex` allow mutable +/// access to `T` (via a [`MutexGuard`] or [`Mutex::with_lock`]), which can be +/// used to move `T` between threads using [`core::mem::replace`] or similar. +/// `T` does **not** need to be [`Sync`], and, in fact, a `Mutex` is often used +/// to protect `!Sync` data. +/// +/// The `Lock` type must be `Sync` because sharing references to a mutex +/// implicitly share references to the `Lock` type as well --- locking the mutex +/// references it. +unsafe impl Sync for Mutex {} // === impl MutexGuard === From 93974d6ffe12ae75a29ce79dc30681d5bda7b083 Mon Sep 17 00:00:00 2001 From: Eliza Weisman Date: Sat, 10 Aug 2024 10:49:55 -0700 Subject: [PATCH 41/41] docs fixup --- trace/src/writer.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/trace/src/writer.rs b/trace/src/writer.rs index 71b9151c..b583d864 100644 --- a/trace/src/writer.rs +++ b/trace/src/writer.rs @@ -204,8 +204,8 @@ pub trait MakeWriterExt<'a>: MakeWriter<'a> { /// a newtype that forwards the trait implementation. /// /// [`fmt::Write`]: mycelium_util::fmt::Write -/// [`MutexGuard`]: maitake_sync::blocking:::MutexGuard -/// [`Mutex`]: mycelium_util::blocking::Mutex +/// [`MutexGuard`]: maitake::sync::blocking:::MutexGuard +/// [`Mutex`]: maitake::sync::blocking::Mutex /// [`MakeWriter`]: trait.MakeWriter.html #[derive(Debug)] pub struct MutexGuardWriter<'a, W, Lock: RawMutex = Spinlock>(MutexGuard<'a, W, Lock>);