diff options
| author | joboet <jonasboettiger@icloud.com> | 2024-03-12 14:55:06 +0100 |
|---|---|---|
| committer | joboet <jonasboettiger@icloud.com> | 2024-03-12 15:41:06 +0100 |
| commit | 22a5267c83a3e17f2b763279eb24bb632c45dc6b (patch) | |
| tree | 81eca4925b77ec92b4f2fd66962af948fe59d094 /library/std/src/sys/sync/mutex | |
| parent | 3b85d2c7fc6d1698e68b94f7bc1a5c9633f2554d (diff) | |
| download | rust-22a5267c83a3e17f2b763279eb24bb632c45dc6b.tar.gz rust-22a5267c83a3e17f2b763279eb24bb632c45dc6b.zip | |
std: move `Once` implementations to `sys`
Diffstat (limited to 'library/std/src/sys/sync/mutex')
| -rw-r--r-- | library/std/src/sys/sync/mutex/fuchsia.rs | 164 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/futex.rs | 108 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/itron.rs | 68 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/mod.rs | 39 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/no_threads.rs | 32 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/pthread.rs | 148 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/sgx.rs | 59 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/windows7.rs | 54 | ||||
| -rw-r--r-- | library/std/src/sys/sync/mutex/xous.rs | 110 |
9 files changed, 782 insertions, 0 deletions
diff --git a/library/std/src/sys/sync/mutex/fuchsia.rs b/library/std/src/sys/sync/mutex/fuchsia.rs new file mode 100644 index 00000000000..5d89e5a13fd --- /dev/null +++ b/library/std/src/sys/sync/mutex/fuchsia.rs @@ -0,0 +1,164 @@ +//! A priority inheriting mutex for Fuchsia. +//! +//! This is a port of the [mutex in Fuchsia's libsync]. Contrary to the original, +//! it does not abort the process when reentrant locking is detected, but deadlocks. +//! +//! Priority inheritance is achieved by storing the owning thread's handle in an +//! atomic variable. Fuchsia's futex operations support setting an owner thread +//! for a futex, which can boost that thread's priority while the futex is waited +//! upon. +//! +//! libsync is licenced under the following BSD-style licence: +//! +//! Copyright 2016 The Fuchsia Authors. +//! +//! Redistribution and use in source and binary forms, with or without +//! modification, are permitted provided that the following conditions are +//! met: +//! +//! * Redistributions of source code must retain the above copyright +//! notice, this list of conditions and the following disclaimer. +//! * Redistributions in binary form must reproduce the above +//! copyright notice, this list of conditions and the following +//! disclaimer in the documentation and/or other materials provided +//! with the distribution. +//! +//! THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +//! "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +//! LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +//! A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +//! OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +//! SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +//! LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +//! DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +//! THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +//! (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +//! OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +//! +//! [mutex in Fuchsia's libsync]: https://cs.opensource.google/fuchsia/fuchsia/+/main:zircon/system/ulib/sync/mutex.c + +use crate::sync::atomic::{ + AtomicU32, + Ordering::{Acquire, Relaxed, Release}, +}; +use crate::sys::futex::zircon::{ + zx_futex_wait, zx_futex_wake_single_owner, zx_handle_t, zx_thread_self, ZX_ERR_BAD_HANDLE, + ZX_ERR_BAD_STATE, ZX_ERR_INVALID_ARGS, ZX_ERR_TIMED_OUT, ZX_ERR_WRONG_TYPE, ZX_OK, + ZX_TIME_INFINITE, +}; + +// The lowest two bits of a `zx_handle_t` are always set, so the lowest bit is used to mark the +// mutex as contested by clearing it. +const CONTESTED_BIT: u32 = 1; +// This can never be a valid `zx_handle_t`. +const UNLOCKED: u32 = 0; + +pub struct Mutex { + futex: AtomicU32, +} + +#[inline] +fn to_state(owner: zx_handle_t) -> u32 { + owner +} + +#[inline] +fn to_owner(state: u32) -> zx_handle_t { + state | CONTESTED_BIT +} + +#[inline] +fn is_contested(state: u32) -> bool { + state & CONTESTED_BIT == 0 +} + +#[inline] +fn mark_contested(state: u32) -> u32 { + state & !CONTESTED_BIT +} + +impl Mutex { + #[inline] + pub const fn new() -> Mutex { + Mutex { futex: AtomicU32::new(UNLOCKED) } + } + + #[inline] + pub fn try_lock(&self) -> bool { + let thread_self = unsafe { zx_thread_self() }; + self.futex.compare_exchange(UNLOCKED, to_state(thread_self), Acquire, Relaxed).is_ok() + } + + #[inline] + pub fn lock(&self) { + let thread_self = unsafe { zx_thread_self() }; + if let Err(state) = + self.futex.compare_exchange(UNLOCKED, to_state(thread_self), Acquire, Relaxed) + { + unsafe { + self.lock_contested(state, thread_self); + } + } + } + + /// # Safety + /// `thread_self` must be the handle for the current thread. + #[cold] + unsafe fn lock_contested(&self, mut state: u32, thread_self: zx_handle_t) { + let owned_state = mark_contested(to_state(thread_self)); + loop { + // Mark the mutex as contested if it is not already. + let contested = mark_contested(state); + if is_contested(state) + || self.futex.compare_exchange(state, contested, Relaxed, Relaxed).is_ok() + { + // The mutex has been marked as contested, wait for the state to change. + unsafe { + match zx_futex_wait( + &self.futex, + AtomicU32::new(contested), + to_owner(state), + ZX_TIME_INFINITE, + ) { + ZX_OK | ZX_ERR_BAD_STATE | ZX_ERR_TIMED_OUT => (), + // Note that if a thread handle is reused after its associated thread + // exits without unlocking the mutex, an arbitrary thread's priority + // could be boosted by the wait, but there is currently no way to + // prevent that. + ZX_ERR_INVALID_ARGS | ZX_ERR_BAD_HANDLE | ZX_ERR_WRONG_TYPE => { + panic!( + "either the current thread is trying to lock a mutex it has + already locked, or the previous owner did not unlock the mutex + before exiting" + ) + } + error => panic!("unexpected error in zx_futex_wait: {error}"), + } + } + } + + // The state has changed or a wakeup occurred, try to lock the mutex. + match self.futex.compare_exchange(UNLOCKED, owned_state, Acquire, Relaxed) { + Ok(_) => return, + Err(updated) => state = updated, + } + } + } + + #[inline] + pub unsafe fn unlock(&self) { + if is_contested(self.futex.swap(UNLOCKED, Release)) { + // The woken thread will mark the mutex as contested again, + // and return here, waking until there are no waiters left, + // in which case this is a noop. + self.wake(); + } + } + + #[cold] + fn wake(&self) { + unsafe { + zx_futex_wake_single_owner(&self.futex); + } + } +} diff --git a/library/std/src/sys/sync/mutex/futex.rs b/library/std/src/sys/sync/mutex/futex.rs new file mode 100644 index 00000000000..7427cae94d6 --- /dev/null +++ b/library/std/src/sys/sync/mutex/futex.rs @@ -0,0 +1,108 @@ +use crate::sync::atomic::{ + self, + Ordering::{Acquire, Relaxed, Release}, +}; +use crate::sys::futex::{futex_wait, futex_wake}; + +cfg_if::cfg_if! { +if #[cfg(windows)] { + // On Windows we can have a smol futex + type Atomic = atomic::AtomicU8; + type State = u8; +} else { + type Atomic = atomic::AtomicU32; + type State = u32; +} +} + +pub struct Mutex { + futex: Atomic, +} + +const UNLOCKED: State = 0; +const LOCKED: State = 1; // locked, no other threads waiting +const CONTENDED: State = 2; // locked, and other threads waiting (contended) + +impl Mutex { + #[inline] + pub const fn new() -> Self { + Self { futex: Atomic::new(UNLOCKED) } + } + + #[inline] + pub fn try_lock(&self) -> bool { + self.futex.compare_exchange(UNLOCKED, LOCKED, Acquire, Relaxed).is_ok() + } + + #[inline] + pub fn lock(&self) { + if self.futex.compare_exchange(UNLOCKED, LOCKED, Acquire, Relaxed).is_err() { + self.lock_contended(); + } + } + + #[cold] + fn lock_contended(&self) { + // Spin first to speed things up if the lock is released quickly. + let mut state = self.spin(); + + // If it's unlocked now, attempt to take the lock + // without marking it as contended. + if state == UNLOCKED { + match self.futex.compare_exchange(UNLOCKED, LOCKED, Acquire, Relaxed) { + Ok(_) => return, // Locked! + Err(s) => state = s, + } + } + + loop { + // Put the lock in contended state. + // We avoid an unnecessary write if it as already set to CONTENDED, + // to be friendlier for the caches. + if state != CONTENDED && self.futex.swap(CONTENDED, Acquire) == UNLOCKED { + // We changed it from UNLOCKED to CONTENDED, so we just successfully locked it. + return; + } + + // Wait for the futex to change state, assuming it is still CONTENDED. + futex_wait(&self.futex, CONTENDED, None); + + // Spin again after waking up. + state = self.spin(); + } + } + + fn spin(&self) -> State { + let mut spin = 100; + loop { + // We only use `load` (and not `swap` or `compare_exchange`) + // while spinning, to be easier on the caches. + let state = self.futex.load(Relaxed); + + // We stop spinning when the mutex is UNLOCKED, + // but also when it's CONTENDED. + if state != LOCKED || spin == 0 { + return state; + } + + crate::hint::spin_loop(); + spin -= 1; + } + } + + #[inline] + pub unsafe fn unlock(&self) { + if self.futex.swap(UNLOCKED, Release) == CONTENDED { + // We only wake up one thread. When that thread locks the mutex, it + // will mark the mutex as CONTENDED (see lock_contended above), + // which makes sure that any other waiting threads will also be + // woken up eventually. + self.wake(); + } + } + + #[cold] + fn wake(&self) { + futex_wake(&self.futex); + } +} diff --git a/library/std/src/sys/sync/mutex/itron.rs b/library/std/src/sys/sync/mutex/itron.rs new file mode 100644 index 00000000000..a134eb2d1be --- /dev/null +++ b/library/std/src/sys/sync/mutex/itron.rs @@ -0,0 +1,68 @@ +//! Mutex implementation backed by μITRON mutexes. Assumes `acre_mtx` and +//! `TA_INHERIT` are available. +use crate::sys::pal::itron::{ + abi, + error::{expect_success, expect_success_aborting, fail, ItronError}, + spin::SpinIdOnceCell, +}; + +pub struct Mutex { + /// The ID of the underlying mutex object + mtx: SpinIdOnceCell<()>, +} + +/// Create a mutex object. This function never panics. +fn new_mtx() -> Result<abi::ID, ItronError> { + ItronError::err_if_negative(unsafe { + abi::acre_mtx(&abi::T_CMTX { + // Priority inheritance mutex + mtxatr: abi::TA_INHERIT, + // Unused + ceilpri: 0, + }) + }) +} + +impl Mutex { + #[inline] + pub const fn new() -> Mutex { + Mutex { mtx: SpinIdOnceCell::new() } + } + + /// Get the inner mutex's ID, which is lazily created. + fn raw(&self) -> abi::ID { + match self.mtx.get_or_try_init(|| new_mtx().map(|id| (id, ()))) { + Ok((id, ())) => id, + Err(e) => fail(e, &"acre_mtx"), + } + } + + pub fn lock(&self) { + let mtx = self.raw(); + expect_success(unsafe { abi::loc_mtx(mtx) }, &"loc_mtx"); + } + + pub unsafe fn unlock(&self) { + let mtx = unsafe { self.mtx.get_unchecked().0 }; + expect_success_aborting(unsafe { abi::unl_mtx(mtx) }, &"unl_mtx"); + } + + pub fn try_lock(&self) -> bool { + let mtx = self.raw(); + match unsafe { abi::ploc_mtx(mtx) } { + abi::E_TMOUT => false, + er => { + expect_success(er, &"ploc_mtx"); + true + } + } + } +} + +impl Drop for Mutex { + fn drop(&mut self) { + if let Some(mtx) = self.mtx.get().map(|x| x.0) { + expect_success_aborting(unsafe { abi::del_mtx(mtx) }, &"del_mtx"); + } + } +} diff --git a/library/std/src/sys/sync/mutex/mod.rs b/library/std/src/sys/sync/mutex/mod.rs new file mode 100644 index 00000000000..73d9bd273de --- /dev/null +++ b/library/std/src/sys/sync/mutex/mod.rs @@ -0,0 +1,39 @@ +cfg_if::cfg_if! { + if #[cfg(any( + all(target_os = "windows", not(target_vendor = "win7")), + target_os = "linux", + target_os = "android", + target_os = "freebsd", + target_os = "openbsd", + target_os = "dragonfly", + all(target_family = "wasm", target_feature = "atomics"), + target_os = "hermit", + ))] { + mod futex; + pub use futex::Mutex; + } else if #[cfg(target_os = "fuchsia")] { + mod fuchsia; + pub use fuchsia::Mutex; + } else if #[cfg(any( + target_family = "unix", + target_os = "teeos", + ))] { + mod pthread; + pub use pthread::{Mutex, raw}; + } else if #[cfg(all(target_os = "windows", target_vendor = "win7"))] { + mod windows7; + pub use windows7::{Mutex, raw}; + } else if #[cfg(all(target_vendor = "fortanix", target_env = "sgx"))] { + mod sgx; + pub use sgx::Mutex; + } else if #[cfg(target_os = "solid_asp3")] { + mod itron; + pub use itron::Mutex; + } else if #[cfg(target_os = "xous")] { + mod xous; + pub use xous::Mutex; + } else { + mod no_threads; + pub use no_threads::Mutex; + } +} diff --git a/library/std/src/sys/sync/mutex/no_threads.rs b/library/std/src/sys/sync/mutex/no_threads.rs new file mode 100644 index 00000000000..4a13c55fb8b --- /dev/null +++ b/library/std/src/sys/sync/mutex/no_threads.rs @@ -0,0 +1,32 @@ +use crate::cell::Cell; + +pub struct Mutex { + // This platform has no threads, so we can use a Cell here. + locked: Cell<bool>, +} + +unsafe impl Send for Mutex {} +unsafe impl Sync for Mutex {} // no threads on this platform + +impl Mutex { + #[inline] + #[rustc_const_stable(feature = "const_locks", since = "1.63.0")] + pub const fn new() -> Mutex { + Mutex { locked: Cell::new(false) } + } + + #[inline] + pub fn lock(&self) { + assert_eq!(self.locked.replace(true), false, "cannot recursively acquire mutex"); + } + + #[inline] + pub unsafe fn unlock(&self) { + self.locked.set(false); + } + + #[inline] + pub fn try_lock(&self) -> bool { + self.locked.replace(true) == false + } +} diff --git a/library/std/src/sys/sync/mutex/pthread.rs b/library/std/src/sys/sync/mutex/pthread.rs new file mode 100644 index 00000000000..ee0794334fb --- /dev/null +++ b/library/std/src/sys/sync/mutex/pthread.rs @@ -0,0 +1,148 @@ +use crate::cell::UnsafeCell; +use crate::io::Error; +use crate::mem::{forget, MaybeUninit}; +use crate::sys::cvt_nz; +use crate::sys_common::lazy_box::{LazyBox, LazyInit}; + +struct AllocatedMutex(UnsafeCell<libc::pthread_mutex_t>); + +pub struct Mutex { + inner: LazyBox<AllocatedMutex>, +} + +#[inline] +pub unsafe fn raw(m: &Mutex) -> *mut libc::pthread_mutex_t { + m.inner.0.get() +} + +unsafe impl Send for AllocatedMutex {} +unsafe impl Sync for AllocatedMutex {} + +impl LazyInit for AllocatedMutex { + fn init() -> Box<Self> { + let mutex = Box::new(AllocatedMutex(UnsafeCell::new(libc::PTHREAD_MUTEX_INITIALIZER))); + + // Issue #33770 + // + // A pthread mutex initialized with PTHREAD_MUTEX_INITIALIZER will have + // a type of PTHREAD_MUTEX_DEFAULT, which has undefined behavior if you + // try to re-lock it from the same thread when you already hold a lock + // (https://pubs.opengroup.org/onlinepubs/9699919799/functions/pthread_mutex_init.html). + // This is the case even if PTHREAD_MUTEX_DEFAULT == PTHREAD_MUTEX_NORMAL + // (https://github.com/rust-lang/rust/issues/33770#issuecomment-220847521) -- in that + // case, `pthread_mutexattr_settype(PTHREAD_MUTEX_DEFAULT)` will of course be the same + // as setting it to `PTHREAD_MUTEX_NORMAL`, but not setting any mode will result in + // a Mutex where re-locking is UB. + // + // In practice, glibc takes advantage of this undefined behavior to + // implement hardware lock elision, which uses hardware transactional + // memory to avoid acquiring the lock. While a transaction is in + // progress, the lock appears to be unlocked. This isn't a problem for + // other threads since the transactional memory will abort if a conflict + // is detected, however no abort is generated when re-locking from the + // same thread. + // + // Since locking the same mutex twice will result in two aliasing &mut + // references, we instead create the mutex with type + // PTHREAD_MUTEX_NORMAL which is guaranteed to deadlock if we try to + // re-lock it from the same thread, thus avoiding undefined behavior. + unsafe { + let mut attr = MaybeUninit::<libc::pthread_mutexattr_t>::uninit(); + cvt_nz(libc::pthread_mutexattr_init(attr.as_mut_ptr())).unwrap(); + let attr = PthreadMutexAttr(&mut attr); + cvt_nz(libc::pthread_mutexattr_settype( + attr.0.as_mut_ptr(), + libc::PTHREAD_MUTEX_NORMAL, + )) + .unwrap(); + cvt_nz(libc::pthread_mutex_init(mutex.0.get(), attr.0.as_ptr())).unwrap(); + } + + mutex + } + + fn destroy(mutex: Box<Self>) { + // We're not allowed to pthread_mutex_destroy a locked mutex, + // so check first if it's unlocked. + if unsafe { libc::pthread_mutex_trylock(mutex.0.get()) == 0 } { + unsafe { libc::pthread_mutex_unlock(mutex.0.get()) }; + drop(mutex); + } else { + // The mutex is locked. This happens if a MutexGuard is leaked. + // In this case, we just leak the Mutex too. + forget(mutex); + } + } + + fn cancel_init(_: Box<Self>) { + // In this case, we can just drop it without any checks, + // since it cannot have been locked yet. + } +} + +impl Drop for AllocatedMutex { + #[inline] + fn drop(&mut self) { + let r = unsafe { libc::pthread_mutex_destroy(self.0.get()) }; + if cfg!(target_os = "dragonfly") { + // On DragonFly pthread_mutex_destroy() returns EINVAL if called on a + // mutex that was just initialized with libc::PTHREAD_MUTEX_INITIALIZER. + // Once it is used (locked/unlocked) or pthread_mutex_init() is called, + // this behaviour no longer occurs. + debug_assert!(r == 0 || r == libc::EINVAL); + } else { + debug_assert_eq!(r, 0); + } + } +} + +impl Mutex { + #[inline] + pub const fn new() -> Mutex { + Mutex { inner: LazyBox::new() } + } + + #[inline] + pub unsafe fn lock(&self) { + #[cold] + #[inline(never)] + fn fail(r: i32) -> ! { + let error = Error::from_raw_os_error(r); + panic!("failed to lock mutex: {error}"); + } + + let r = libc::pthread_mutex_lock(raw(self)); + // As we set the mutex type to `PTHREAD_MUTEX_NORMAL` above, we expect + // the lock call to never fail. Unfortunately however, some platforms + // (Solaris) do not conform to the standard, and instead always provide + // deadlock detection. How kind of them! Unfortunately that means that + // we need to check the error code here. To save us from UB on other + // less well-behaved platforms in the future, we do it even on "good" + // platforms like macOS. See #120147 for more context. + if r != 0 { + fail(r) + } + } + + #[inline] + pub unsafe fn unlock(&self) { + let r = libc::pthread_mutex_unlock(raw(self)); + debug_assert_eq!(r, 0); + } + + #[inline] + pub unsafe fn try_lock(&self) -> bool { + libc::pthread_mutex_trylock(raw(self)) == 0 + } +} + +pub(super) struct PthreadMutexAttr<'a>(pub &'a mut MaybeUninit<libc::pthread_mutexattr_t>); + +impl Drop for PthreadMutexAttr<'_> { + fn drop(&mut self) { + unsafe { + let result = libc::pthread_mutexattr_destroy(self.0.as_mut_ptr()); + debug_assert_eq!(result, 0); + } + } +} diff --git a/library/std/src/sys/sync/mutex/sgx.rs b/library/std/src/sys/sync/mutex/sgx.rs new file mode 100644 index 00000000000..d37bd02adf8 --- /dev/null +++ b/library/std/src/sys/sync/mutex/sgx.rs @@ -0,0 +1,59 @@ +use crate::sys::pal::waitqueue::{try_lock_or_false, SpinMutex, WaitQueue, WaitVariable}; +use crate::sys_common::lazy_box::{LazyBox, LazyInit}; + +/// FIXME: `UnsafeList` is not movable. +struct AllocatedMutex(SpinMutex<WaitVariable<bool>>); + +pub struct Mutex { + inner: LazyBox<AllocatedMutex>, +} + +impl LazyInit for AllocatedMutex { + fn init() -> Box<Self> { + Box::new(AllocatedMutex(SpinMutex::new(WaitVariable::new(false)))) + } +} + +// Implementation according to “Operating Systems: Three Easy Pieces”, chapter 28 +impl Mutex { + pub const fn new() -> Mutex { + Mutex { inner: LazyBox::new() } + } + + #[inline] + pub fn lock(&self) { + let mut guard = self.inner.0.lock(); + if *guard.lock_var() { + // Another thread has the lock, wait + WaitQueue::wait(guard, || {}) + // Another thread has passed the lock to us + } else { + // We are just now obtaining the lock + *guard.lock_var_mut() = true; + } + } + + #[inline] + pub unsafe fn unlock(&self) { + let guard = self.inner.0.lock(); + if let Err(mut guard) = WaitQueue::notify_one(guard) { + // No other waiters, unlock + *guard.lock_var_mut() = false; + } else { + // There was a thread waiting, just pass the lock + } + } + + #[inline] + pub fn try_lock(&self) -> bool { + let mut guard = try_lock_or_false!(self.inner.0); + if *guard.lock_var() { + // Another thread has the lock + false + } else { + // We are just now obtaining the lock + *guard.lock_var_mut() = true; + true + } + } +} diff --git a/library/std/src/sys/sync/mutex/windows7.rs b/library/std/src/sys/sync/mutex/windows7.rs new file mode 100644 index 00000000000..ef2f84082cd --- /dev/null +++ b/library/std/src/sys/sync/mutex/windows7.rs @@ -0,0 +1,54 @@ +//! System Mutexes +//! +//! The Windows implementation of mutexes is a little odd and it might not be +//! immediately obvious what's going on. The primary oddness is that SRWLock is +//! used instead of CriticalSection, and this is done because: +//! +//! 1. SRWLock is several times faster than CriticalSection according to +//! benchmarks performed on both Windows 8 and Windows 7. +//! +//! 2. CriticalSection allows recursive locking while SRWLock deadlocks. The +//! Unix implementation deadlocks so consistency is preferred. See #19962 for +//! more details. +//! +//! 3. While CriticalSection is fair and SRWLock is not, the current Rust policy +//! is that there are no guarantees of fairness. + +use crate::cell::UnsafeCell; +use crate::sys::c; + +pub struct Mutex { + srwlock: UnsafeCell<c::SRWLOCK>, +} + +unsafe impl Send for Mutex {} +unsafe impl Sync for Mutex {} + +#[inline] +pub unsafe fn raw(m: &Mutex) -> c::PSRWLOCK { + m.srwlock.get() +} + +impl Mutex { + #[inline] + pub const fn new() -> Mutex { + Mutex { srwlock: UnsafeCell::new(c::SRWLOCK_INIT) } + } + + #[inline] + pub fn lock(&self) { + unsafe { + c::AcquireSRWLockExclusive(raw(self)); + } + } + + #[inline] + pub fn try_lock(&self) -> bool { + unsafe { c::TryAcquireSRWLockExclusive(raw(self)) != 0 } + } + + #[inline] + pub unsafe fn unlock(&self) { + c::ReleaseSRWLockExclusive(raw(self)); + } +} diff --git a/library/std/src/sys/sync/mutex/xous.rs b/library/std/src/sys/sync/mutex/xous.rs new file mode 100644 index 00000000000..a8c9518ff0b --- /dev/null +++ b/library/std/src/sys/sync/mutex/xous.rs @@ -0,0 +1,110 @@ +use crate::os::xous::ffi::{blocking_scalar, do_yield}; +use crate::os::xous::services::{ticktimer_server, TicktimerScalar}; +use crate::sync::atomic::{AtomicBool, AtomicUsize, Ordering::Relaxed, Ordering::SeqCst}; + +pub struct Mutex { + /// The "locked" value indicates how many threads are waiting on this + /// Mutex. Possible values are: + /// 0: The lock is unlocked + /// 1: The lock is locked and uncontended + /// >=2: The lock is locked and contended + /// + /// A lock is "contended" when there is more than one thread waiting + /// for a lock, or it is locked for long periods of time. Rather than + /// spinning, these locks send a Message to the ticktimer server + /// requesting that they be woken up when a lock is unlocked. + locked: AtomicUsize, + + /// Whether this Mutex ever was contended, and therefore made a trip + /// to the ticktimer server. If this was never set, then we were never + /// on the slow path and can skip deregistering the mutex. + contended: AtomicBool, +} + +impl Mutex { + #[inline] + #[rustc_const_stable(feature = "const_locks", since = "1.63.0")] + pub const fn new() -> Mutex { + Mutex { locked: AtomicUsize::new(0), contended: AtomicBool::new(false) } + } + + fn index(&self) -> usize { + core::ptr::from_ref(self).addr() + } + + #[inline] + pub unsafe fn lock(&self) { + // Try multiple times to acquire the lock without resorting to the ticktimer + // server. For locks that are held for a short amount of time, this will + // result in the ticktimer server never getting invoked. The `locked` value + // will be either 0 or 1. + for _attempts in 0..3 { + if unsafe { self.try_lock() } { + return; + } + do_yield(); + } + + // Try one more time to lock. If the lock is released between the previous code and + // here, then the inner `locked` value will be 1 at the end of this. If it was not + // locked, then the value will be more than 1, for example if there are multiple other + // threads waiting on this lock. + if unsafe { self.try_lock_or_poison() } { + return; + } + + // When this mutex is dropped, we will need to deregister it with the server. + self.contended.store(true, Relaxed); + + // The lock is now "contended". When the lock is released, a Message will get sent to the + // ticktimer server to wake it up. Note that this may already have happened, so the actual + // value of `lock` may be anything (0, 1, 2, ...). + blocking_scalar( + ticktimer_server(), + crate::os::xous::services::TicktimerScalar::LockMutex(self.index()).into(), + ) + .expect("failure to send LockMutex command"); + } + + #[inline] + pub unsafe fn unlock(&self) { + let prev = self.locked.fetch_sub(1, SeqCst); + + // If the previous value was 1, then this was a "fast path" unlock, so no + // need to involve the Ticktimer server + if prev == 1 { + return; + } + + // If it was 0, then something has gone seriously wrong and the counter + // has just wrapped around. + if prev == 0 { + panic!("mutex lock count underflowed"); + } + + // Unblock one thread that is waiting on this message. + blocking_scalar(ticktimer_server(), TicktimerScalar::UnlockMutex(self.index()).into()) + .expect("failure to send UnlockMutex command"); + } + + #[inline] + pub unsafe fn try_lock(&self) -> bool { + self.locked.compare_exchange(0, 1, SeqCst, SeqCst).is_ok() + } + + #[inline] + pub unsafe fn try_lock_or_poison(&self) -> bool { + self.locked.fetch_add(1, SeqCst) == 0 + } +} + +impl Drop for Mutex { + fn drop(&mut self) { + // If there was Mutex contention, then we involved the ticktimer. Free + // the resources associated with this Mutex as it is deallocated. + if self.contended.load(Relaxed) { + blocking_scalar(ticktimer_server(), TicktimerScalar::FreeMutex(self.index()).into()) + .ok(); + } + } +} |
