about summary refs log tree commit diff
path: root/library/std/src/sys/sync/mutex
diff options
context:
space:
mode:
authorjoboet <jonasboettiger@icloud.com>2024-03-12 14:55:06 +0100
committerjoboet <jonasboettiger@icloud.com>2024-03-12 15:41:06 +0100
commit22a5267c83a3e17f2b763279eb24bb632c45dc6b (patch)
tree81eca4925b77ec92b4f2fd66962af948fe59d094 /library/std/src/sys/sync/mutex
parent3b85d2c7fc6d1698e68b94f7bc1a5c9633f2554d (diff)
downloadrust-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.rs164
-rw-r--r--library/std/src/sys/sync/mutex/futex.rs108
-rw-r--r--library/std/src/sys/sync/mutex/itron.rs68
-rw-r--r--library/std/src/sys/sync/mutex/mod.rs39
-rw-r--r--library/std/src/sys/sync/mutex/no_threads.rs32
-rw-r--r--library/std/src/sys/sync/mutex/pthread.rs148
-rw-r--r--library/std/src/sys/sync/mutex/sgx.rs59
-rw-r--r--library/std/src/sys/sync/mutex/windows7.rs54
-rw-r--r--library/std/src/sys/sync/mutex/xous.rs110
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();
+        }
+    }
+}