about summary refs log tree commit diff
path: root/library/std
diff options
context:
space:
mode:
Diffstat (limited to 'library/std')
-rw-r--r--library/std/src/sys/sync/condvar/mod.rs8
-rw-r--r--library/std/src/sys/sync/condvar/pthread.rs38
-rw-r--r--library/std/src/sys/sync/condvar/sgx.rs29
-rw-r--r--library/std/src/sys/sync/condvar/teeos.rs101
-rw-r--r--library/std/src/sys/sync/mod.rs3
-rw-r--r--library/std/src/sys/sync/mutex/mod.rs2
-rw-r--r--library/std/src/sys/sync/mutex/pthread.rs81
-rw-r--r--library/std/src/sys/sync/mutex/sgx.rs28
-rw-r--r--library/std/src/sys/sync/once_box.rs82
-rw-r--r--library/std/src/sys/sync/rwlock/teeos.rs8
-rw-r--r--library/std/src/sys_common/lazy_box.rs88
-rw-r--r--library/std/src/sys_common/mod.rs1
12 files changed, 185 insertions, 284 deletions
diff --git a/library/std/src/sys/sync/condvar/mod.rs b/library/std/src/sys/sync/condvar/mod.rs
index 6849cacf88e..d0c998a5597 100644
--- a/library/std/src/sys/sync/condvar/mod.rs
+++ b/library/std/src/sys/sync/condvar/mod.rs
@@ -12,7 +12,10 @@ cfg_if::cfg_if! {
     ))] {
         mod futex;
         pub use futex::Condvar;
-    } else if #[cfg(target_family = "unix")] {
+    } else if #[cfg(any(
+        target_family = "unix",
+        target_os = "teeos",
+    ))] {
         mod pthread;
         pub use pthread::Condvar;
     } else if #[cfg(all(target_os = "windows", target_vendor = "win7"))] {
@@ -24,9 +27,6 @@ cfg_if::cfg_if! {
     } else if #[cfg(target_os = "solid_asp3")] {
         mod itron;
         pub use itron::Condvar;
-    } else if #[cfg(target_os = "teeos")] {
-        mod teeos;
-        pub use teeos::Condvar;
     } else if #[cfg(target_os = "xous")] {
         mod xous;
         pub use xous::Condvar;
diff --git a/library/std/src/sys/sync/condvar/pthread.rs b/library/std/src/sys/sync/condvar/pthread.rs
index 5b5e7770b06..986cd0cb7d1 100644
--- a/library/std/src/sys/sync/condvar/pthread.rs
+++ b/library/std/src/sys/sync/condvar/pthread.rs
@@ -2,31 +2,25 @@ use crate::cell::UnsafeCell;
 use crate::ptr;
 use crate::sync::atomic::AtomicPtr;
 use crate::sync::atomic::Ordering::Relaxed;
-use crate::sys::sync::{Mutex, mutex};
+use crate::sys::sync::{Mutex, OnceBox};
 #[cfg(not(target_os = "nto"))]
 use crate::sys::time::TIMESPEC_MAX;
 #[cfg(target_os = "nto")]
 use crate::sys::time::TIMESPEC_MAX_CAPPED;
-use crate::sys_common::lazy_box::{LazyBox, LazyInit};
 use crate::time::Duration;
 
 struct AllocatedCondvar(UnsafeCell<libc::pthread_cond_t>);
 
 pub struct Condvar {
-    inner: LazyBox<AllocatedCondvar>,
+    inner: OnceBox<AllocatedCondvar>,
     mutex: AtomicPtr<libc::pthread_mutex_t>,
 }
 
-#[inline]
-fn raw(c: &Condvar) -> *mut libc::pthread_cond_t {
-    c.inner.0.get()
-}
-
 unsafe impl Send for AllocatedCondvar {}
 unsafe impl Sync for AllocatedCondvar {}
 
-impl LazyInit for AllocatedCondvar {
-    fn init() -> Box<Self> {
+impl AllocatedCondvar {
+    fn new() -> Box<Self> {
         let condvar = Box::new(AllocatedCondvar(UnsafeCell::new(libc::PTHREAD_COND_INITIALIZER)));
 
         cfg_if::cfg_if! {
@@ -37,7 +31,7 @@ impl LazyInit for AllocatedCondvar {
                 target_vendor = "apple",
             ))] {
                 // `pthread_condattr_setclock` is unfortunately not supported on these platforms.
-            } else if #[cfg(any(target_os = "espidf", target_os = "horizon"))] {
+            } else if #[cfg(any(target_os = "espidf", target_os = "horizon", target_os = "teeos"))] {
                 // NOTE: ESP-IDF's PTHREAD_COND_INITIALIZER support is not released yet
                 // So on that platform, init() should always be called
                 // Moreover, that platform does not have pthread_condattr_setclock support,
@@ -82,7 +76,11 @@ impl Drop for AllocatedCondvar {
 
 impl Condvar {
     pub const fn new() -> Condvar {
-        Condvar { inner: LazyBox::new(), mutex: AtomicPtr::new(ptr::null_mut()) }
+        Condvar { inner: OnceBox::new(), mutex: AtomicPtr::new(ptr::null_mut()) }
+    }
+
+    fn get(&self) -> *mut libc::pthread_cond_t {
+        self.inner.get_or_init(AllocatedCondvar::new).0.get()
     }
 
     #[inline]
@@ -98,21 +96,21 @@ impl Condvar {
 
     #[inline]
     pub fn notify_one(&self) {
-        let r = unsafe { libc::pthread_cond_signal(raw(self)) };
+        let r = unsafe { libc::pthread_cond_signal(self.get()) };
         debug_assert_eq!(r, 0);
     }
 
     #[inline]
     pub fn notify_all(&self) {
-        let r = unsafe { libc::pthread_cond_broadcast(raw(self)) };
+        let r = unsafe { libc::pthread_cond_broadcast(self.get()) };
         debug_assert_eq!(r, 0);
     }
 
     #[inline]
     pub unsafe fn wait(&self, mutex: &Mutex) {
-        let mutex = mutex::raw(mutex);
+        let mutex = mutex.get_assert_locked();
         self.verify(mutex);
-        let r = libc::pthread_cond_wait(raw(self), mutex);
+        let r = libc::pthread_cond_wait(self.get(), mutex);
         debug_assert_eq!(r, 0);
     }
 
@@ -129,7 +127,7 @@ impl Condvar {
     pub unsafe fn wait_timeout(&self, mutex: &Mutex, dur: Duration) -> bool {
         use crate::sys::time::Timespec;
 
-        let mutex = mutex::raw(mutex);
+        let mutex = mutex.get_assert_locked();
         self.verify(mutex);
 
         #[cfg(not(target_os = "nto"))]
@@ -144,7 +142,7 @@ impl Condvar {
             .and_then(|t| t.to_timespec_capped())
             .unwrap_or(TIMESPEC_MAX_CAPPED);
 
-        let r = libc::pthread_cond_timedwait(raw(self), mutex, &timeout);
+        let r = libc::pthread_cond_timedwait(self.get(), mutex, &timeout);
         assert!(r == libc::ETIMEDOUT || r == 0);
         r == 0
     }
@@ -162,7 +160,7 @@ impl Condvar {
         use crate::sys::time::SystemTime;
         use crate::time::Instant;
 
-        let mutex = mutex::raw(mutex);
+        let mutex = mutex.get_assert_locked();
         self.verify(mutex);
 
         // OSX implementation of `pthread_cond_timedwait` is buggy
@@ -188,7 +186,7 @@ impl Condvar {
             .and_then(|t| t.to_timespec())
             .unwrap_or(TIMESPEC_MAX);
 
-        let r = libc::pthread_cond_timedwait(raw(self), mutex, &timeout);
+        let r = libc::pthread_cond_timedwait(self.get(), mutex, &timeout);
         debug_assert!(r == libc::ETIMEDOUT || r == 0);
 
         // ETIMEDOUT is not a totally reliable method of determining timeout due
diff --git a/library/std/src/sys/sync/condvar/sgx.rs b/library/std/src/sys/sync/condvar/sgx.rs
index ecb5872f60d..e60715e4b59 100644
--- a/library/std/src/sys/sync/condvar/sgx.rs
+++ b/library/std/src/sys/sync/condvar/sgx.rs
@@ -1,44 +1,39 @@
 use crate::sys::pal::waitqueue::{SpinMutex, WaitQueue, WaitVariable};
-use crate::sys::sync::Mutex;
-use crate::sys_common::lazy_box::{LazyBox, LazyInit};
+use crate::sys::sync::{Mutex, OnceBox};
 use crate::time::Duration;
 
-/// FIXME: `UnsafeList` is not movable.
-struct AllocatedCondvar(SpinMutex<WaitVariable<()>>);
-
 pub struct Condvar {
-    inner: LazyBox<AllocatedCondvar>,
-}
-
-impl LazyInit for AllocatedCondvar {
-    fn init() -> Box<Self> {
-        Box::new(AllocatedCondvar(SpinMutex::new(WaitVariable::new(()))))
-    }
+    // FIXME: `UnsafeList` is not movable.
+    inner: OnceBox<SpinMutex<WaitVariable<()>>>,
 }
 
 impl Condvar {
     pub const fn new() -> Condvar {
-        Condvar { inner: LazyBox::new() }
+        Condvar { inner: OnceBox::new() }
+    }
+
+    fn get(&self) -> &SpinMutex<WaitVariable<()>> {
+        self.inner.get_or_init(|| Box::new(SpinMutex::new(WaitVariable::new(()))))
     }
 
     #[inline]
     pub fn notify_one(&self) {
-        let _ = WaitQueue::notify_one(self.inner.0.lock());
+        let _ = WaitQueue::notify_one(self.get().lock());
     }
 
     #[inline]
     pub fn notify_all(&self) {
-        let _ = WaitQueue::notify_all(self.inner.0.lock());
+        let _ = WaitQueue::notify_all(self.get().lock());
     }
 
     pub unsafe fn wait(&self, mutex: &Mutex) {
-        let guard = self.inner.0.lock();
+        let guard = self.get().lock();
         WaitQueue::wait(guard, || unsafe { mutex.unlock() });
         mutex.lock()
     }
 
     pub unsafe fn wait_timeout(&self, mutex: &Mutex, dur: Duration) -> bool {
-        let success = WaitQueue::wait_timeout(&self.inner.0, dur, || unsafe { mutex.unlock() });
+        let success = WaitQueue::wait_timeout(self.get(), dur, || unsafe { mutex.unlock() });
         mutex.lock();
         success
     }
diff --git a/library/std/src/sys/sync/condvar/teeos.rs b/library/std/src/sys/sync/condvar/teeos.rs
deleted file mode 100644
index 943867cd761..00000000000
--- a/library/std/src/sys/sync/condvar/teeos.rs
+++ /dev/null
@@ -1,101 +0,0 @@
-use crate::cell::UnsafeCell;
-use crate::ptr;
-use crate::sync::atomic::AtomicPtr;
-use crate::sync::atomic::Ordering::Relaxed;
-use crate::sys::sync::mutex::{self, Mutex};
-use crate::sys::time::TIMESPEC_MAX;
-use crate::sys_common::lazy_box::{LazyBox, LazyInit};
-use crate::time::Duration;
-
-extern "C" {
-    pub fn pthread_cond_timedwait(
-        cond: *mut libc::pthread_cond_t,
-        lock: *mut libc::pthread_mutex_t,
-        adstime: *const libc::timespec,
-    ) -> libc::c_int;
-}
-
-struct AllocatedCondvar(UnsafeCell<libc::pthread_cond_t>);
-
-pub struct Condvar {
-    inner: LazyBox<AllocatedCondvar>,
-    mutex: AtomicPtr<libc::pthread_mutex_t>,
-}
-
-#[inline]
-fn raw(c: &Condvar) -> *mut libc::pthread_cond_t {
-    c.inner.0.get()
-}
-
-unsafe impl Send for AllocatedCondvar {}
-unsafe impl Sync for AllocatedCondvar {}
-
-impl LazyInit for AllocatedCondvar {
-    fn init() -> Box<Self> {
-        let condvar = Box::new(AllocatedCondvar(UnsafeCell::new(libc::PTHREAD_COND_INITIALIZER)));
-
-        let r = unsafe { libc::pthread_cond_init(condvar.0.get(), crate::ptr::null()) };
-        assert_eq!(r, 0);
-
-        condvar
-    }
-}
-
-impl Drop for AllocatedCondvar {
-    #[inline]
-    fn drop(&mut self) {
-        let r = unsafe { libc::pthread_cond_destroy(self.0.get()) };
-        debug_assert_eq!(r, 0);
-    }
-}
-
-impl Condvar {
-    pub const fn new() -> Condvar {
-        Condvar { inner: LazyBox::new(), mutex: AtomicPtr::new(ptr::null_mut()) }
-    }
-
-    #[inline]
-    fn verify(&self, mutex: *mut libc::pthread_mutex_t) {
-        match self.mutex.compare_exchange(ptr::null_mut(), mutex, Relaxed, Relaxed) {
-            Ok(_) => {}                // Stored the address
-            Err(n) if n == mutex => {} // Lost a race to store the same address
-            _ => panic!("attempted to use a condition variable with two mutexes"),
-        }
-    }
-
-    #[inline]
-    pub fn notify_one(&self) {
-        let r = unsafe { libc::pthread_cond_signal(raw(self)) };
-        debug_assert_eq!(r, 0);
-    }
-
-    #[inline]
-    pub fn notify_all(&self) {
-        let r = unsafe { libc::pthread_cond_broadcast(raw(self)) };
-        debug_assert_eq!(r, 0);
-    }
-
-    #[inline]
-    pub unsafe fn wait(&self, mutex: &Mutex) {
-        let mutex = unsafe { mutex::raw(mutex) };
-        self.verify(mutex);
-        let r = unsafe { libc::pthread_cond_wait(raw(self), mutex) };
-        debug_assert_eq!(r, 0);
-    }
-
-    pub unsafe fn wait_timeout(&self, mutex: &Mutex, dur: Duration) -> bool {
-        use crate::sys::time::Timespec;
-
-        let mutex = unsafe { mutex::raw(mutex) };
-        self.verify(mutex);
-
-        let timeout = Timespec::now(libc::CLOCK_MONOTONIC)
-            .checked_add_duration(&dur)
-            .and_then(|t| t.to_timespec())
-            .unwrap_or(TIMESPEC_MAX);
-
-        let r = unsafe { pthread_cond_timedwait(raw(self), mutex, &timeout) };
-        assert!(r == libc::ETIMEDOUT || r == 0);
-        r == 0
-    }
-}
diff --git a/library/std/src/sys/sync/mod.rs b/library/std/src/sys/sync/mod.rs
index 52fac5902a2..0691e967851 100644
--- a/library/std/src/sys/sync/mod.rs
+++ b/library/std/src/sys/sync/mod.rs
@@ -1,11 +1,14 @@
 mod condvar;
 mod mutex;
 mod once;
+mod once_box;
 mod rwlock;
 mod thread_parking;
 
 pub use condvar::Condvar;
 pub use mutex::Mutex;
 pub use once::{Once, OnceState};
+#[allow(unused)] // Only used on some platforms.
+use once_box::OnceBox;
 pub use rwlock::RwLock;
 pub use thread_parking::Parker;
diff --git a/library/std/src/sys/sync/mutex/mod.rs b/library/std/src/sys/sync/mutex/mod.rs
index 73d9bd273de..360df3fc4b5 100644
--- a/library/std/src/sys/sync/mutex/mod.rs
+++ b/library/std/src/sys/sync/mutex/mod.rs
@@ -19,7 +19,7 @@ cfg_if::cfg_if! {
         target_os = "teeos",
     ))] {
         mod pthread;
-        pub use pthread::{Mutex, raw};
+        pub use pthread::Mutex;
     } else if #[cfg(all(target_os = "windows", target_vendor = "win7"))] {
         mod windows7;
         pub use windows7::{Mutex, raw};
diff --git a/library/std/src/sys/sync/mutex/pthread.rs b/library/std/src/sys/sync/mutex/pthread.rs
index 1c407bc2537..87c95f45f96 100644
--- a/library/std/src/sys/sync/mutex/pthread.rs
+++ b/library/std/src/sys/sync/mutex/pthread.rs
@@ -2,24 +2,19 @@ use crate::cell::UnsafeCell;
 use crate::io::Error;
 use crate::mem::{MaybeUninit, forget};
 use crate::sys::cvt_nz;
-use crate::sys_common::lazy_box::{LazyBox, LazyInit};
+use crate::sys::sync::OnceBox;
 
 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()
+    inner: OnceBox<AllocatedMutex>,
 }
 
 unsafe impl Send for AllocatedMutex {}
 unsafe impl Sync for AllocatedMutex {}
 
-impl LazyInit for AllocatedMutex {
-    fn init() -> Box<Self> {
+impl AllocatedMutex {
+    fn new() -> Box<Self> {
         let mutex = Box::new(AllocatedMutex(UnsafeCell::new(libc::PTHREAD_MUTEX_INITIALIZER)));
 
         // Issue #33770
@@ -60,24 +55,6 @@ impl LazyInit for AllocatedMutex {
 
         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 {
@@ -99,11 +76,33 @@ impl Drop for AllocatedMutex {
 impl Mutex {
     #[inline]
     pub const fn new() -> Mutex {
-        Mutex { inner: LazyBox::new() }
+        Mutex { inner: OnceBox::new() }
+    }
+
+    /// Gets access to the pthread mutex under the assumption that the mutex is
+    /// locked.
+    ///
+    /// This allows skipping the initialization check, as the mutex can only be
+    /// locked if it is already initialized, and allows relaxing the ordering
+    /// on the pointer load, since the allocation cannot have been modified
+    /// since the `lock` and the lock must have occurred on the current thread.
+    ///
+    /// # Safety
+    /// Causes undefined behaviour if the mutex is not locked.
+    #[inline]
+    pub(crate) unsafe fn get_assert_locked(&self) -> *mut libc::pthread_mutex_t {
+        unsafe { self.inner.get_unchecked().0.get() }
     }
 
     #[inline]
-    pub unsafe fn lock(&self) {
+    fn get(&self) -> *mut libc::pthread_mutex_t {
+        // If initialization fails, the mutex is destroyed. This is always sound,
+        // however, as the mutex cannot have been locked yet.
+        self.inner.get_or_init(AllocatedMutex::new).0.get()
+    }
+
+    #[inline]
+    pub fn lock(&self) {
         #[cold]
         #[inline(never)]
         fn fail(r: i32) -> ! {
@@ -111,7 +110,7 @@ impl Mutex {
             panic!("failed to lock mutex: {error}");
         }
 
-        let r = libc::pthread_mutex_lock(raw(self));
+        let r = unsafe { libc::pthread_mutex_lock(self.get()) };
         // 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
@@ -126,13 +125,29 @@ impl Mutex {
 
     #[inline]
     pub unsafe fn unlock(&self) {
-        let r = libc::pthread_mutex_unlock(raw(self));
+        let r = libc::pthread_mutex_unlock(self.get_assert_locked());
         debug_assert_eq!(r, 0);
     }
 
     #[inline]
-    pub unsafe fn try_lock(&self) -> bool {
-        libc::pthread_mutex_trylock(raw(self)) == 0
+    pub fn try_lock(&self) -> bool {
+        unsafe { libc::pthread_mutex_trylock(self.get()) == 0 }
+    }
+}
+
+impl Drop for Mutex {
+    fn drop(&mut self) {
+        let Some(mutex) = self.inner.take() else { return };
+        // 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);
+        }
     }
 }
 
diff --git a/library/std/src/sys/sync/mutex/sgx.rs b/library/std/src/sys/sync/mutex/sgx.rs
index 65d1e880f7b..8529e857970 100644
--- a/library/std/src/sys/sync/mutex/sgx.rs
+++ b/library/std/src/sys/sync/mutex/sgx.rs
@@ -1,28 +1,24 @@
 use crate::sys::pal::waitqueue::{SpinMutex, WaitQueue, WaitVariable, try_lock_or_false};
-use crate::sys_common::lazy_box::{LazyBox, LazyInit};
-
-/// FIXME: `UnsafeList` is not movable.
-struct AllocatedMutex(SpinMutex<WaitVariable<bool>>);
+use crate::sys::sync::OnceBox;
 
 pub struct Mutex {
-    inner: LazyBox<AllocatedMutex>,
-}
-
-impl LazyInit for AllocatedMutex {
-    fn init() -> Box<Self> {
-        Box::new(AllocatedMutex(SpinMutex::new(WaitVariable::new(false))))
-    }
+    // FIXME: `UnsafeList` is not movable.
+    inner: OnceBox<SpinMutex<WaitVariable<bool>>>,
 }
 
 // Implementation according to “Operating Systems: Three Easy Pieces”, chapter 28
 impl Mutex {
     pub const fn new() -> Mutex {
-        Mutex { inner: LazyBox::new() }
+        Mutex { inner: OnceBox::new() }
+    }
+
+    fn get(&self) -> &SpinMutex<WaitVariable<bool>> {
+        self.inner.get_or_init(|| Box::new(SpinMutex::new(WaitVariable::new(false))))
     }
 
     #[inline]
     pub fn lock(&self) {
-        let mut guard = self.inner.0.lock();
+        let mut guard = self.get().lock();
         if *guard.lock_var() {
             // Another thread has the lock, wait
             WaitQueue::wait(guard, || {})
@@ -35,7 +31,9 @@ impl Mutex {
 
     #[inline]
     pub unsafe fn unlock(&self) {
-        let guard = self.inner.0.lock();
+        // SAFETY: the mutex was locked by the current thread, so it has been
+        // initialized already.
+        let guard = unsafe { self.inner.get_unchecked().lock() };
         if let Err(mut guard) = WaitQueue::notify_one(guard) {
             // No other waiters, unlock
             *guard.lock_var_mut() = false;
@@ -46,7 +44,7 @@ impl Mutex {
 
     #[inline]
     pub fn try_lock(&self) -> bool {
-        let mut guard = try_lock_or_false!(self.inner.0);
+        let mut guard = try_lock_or_false!(self.get());
         if *guard.lock_var() {
             // Another thread has the lock
             false
diff --git a/library/std/src/sys/sync/once_box.rs b/library/std/src/sys/sync/once_box.rs
new file mode 100644
index 00000000000..1422b5a1721
--- /dev/null
+++ b/library/std/src/sys/sync/once_box.rs
@@ -0,0 +1,82 @@
+//! A racily-initialized alternative to `OnceLock<Box<T>>`.
+//!
+//! This is used to implement synchronization primitives that need allocation,
+//! like the pthread versions.
+
+#![allow(dead_code)] // Only used on some platforms.
+
+use crate::mem::replace;
+use crate::ptr::null_mut;
+use crate::sync::atomic::AtomicPtr;
+use crate::sync::atomic::Ordering::{AcqRel, Acquire, Relaxed};
+
+pub(crate) struct OnceBox<T> {
+    ptr: AtomicPtr<T>,
+}
+
+impl<T> OnceBox<T> {
+    #[inline]
+    pub const fn new() -> Self {
+        Self { ptr: AtomicPtr::new(null_mut()) }
+    }
+
+    /// Gets access to the value, assuming it is already initialized and this
+    /// initialization has been observed by the current thread.
+    ///
+    /// Since all modifications to the pointer have already been observed, the
+    /// pointer load in this function can be performed with relaxed ordering,
+    /// potentially allowing the optimizer to turn code like this:
+    /// ```rust, ignore
+    /// once_box.get_or_init(|| Box::new(42));
+    /// unsafe { once_box.get_unchecked() }
+    /// ```
+    /// into
+    /// ```rust, ignore
+    /// once_box.get_or_init(|| Box::new(42))
+    /// ```
+    ///
+    /// # Safety
+    /// This causes undefined behaviour if the assumption above is violated.
+    #[inline]
+    pub unsafe fn get_unchecked(&self) -> &T {
+        unsafe { &*self.ptr.load(Relaxed) }
+    }
+
+    #[inline]
+    pub fn get_or_init(&self, f: impl FnOnce() -> Box<T>) -> &T {
+        let ptr = self.ptr.load(Acquire);
+        match unsafe { ptr.as_ref() } {
+            Some(val) => val,
+            None => self.initialize(f),
+        }
+    }
+
+    #[inline]
+    pub fn take(&mut self) -> Option<Box<T>> {
+        let ptr = replace(self.ptr.get_mut(), null_mut());
+        if !ptr.is_null() { Some(unsafe { Box::from_raw(ptr) }) } else { None }
+    }
+
+    #[cold]
+    fn initialize(&self, f: impl FnOnce() -> Box<T>) -> &T {
+        let new_ptr = Box::into_raw(f());
+        match self.ptr.compare_exchange(null_mut(), new_ptr, AcqRel, Acquire) {
+            Ok(_) => unsafe { &*new_ptr },
+            Err(ptr) => {
+                // Lost the race to another thread.
+                // Drop the value we created, and use the one from the other thread instead.
+                drop(unsafe { Box::from_raw(new_ptr) });
+                unsafe { &*ptr }
+            }
+        }
+    }
+}
+
+unsafe impl<T: Send> Send for OnceBox<T> {}
+unsafe impl<T: Send + Sync> Sync for OnceBox<T> {}
+
+impl<T> Drop for OnceBox<T> {
+    fn drop(&mut self) {
+        self.take();
+    }
+}
diff --git a/library/std/src/sys/sync/rwlock/teeos.rs b/library/std/src/sys/sync/rwlock/teeos.rs
index ef9b1ab5154..76343022383 100644
--- a/library/std/src/sys/sync/rwlock/teeos.rs
+++ b/library/std/src/sys/sync/rwlock/teeos.rs
@@ -14,22 +14,22 @@ impl RwLock {
 
     #[inline]
     pub fn read(&self) {
-        unsafe { self.inner.lock() };
+        self.inner.lock()
     }
 
     #[inline]
     pub fn try_read(&self) -> bool {
-        unsafe { self.inner.try_lock() }
+        self.inner.try_lock()
     }
 
     #[inline]
     pub fn write(&self) {
-        unsafe { self.inner.lock() };
+        self.inner.lock()
     }
 
     #[inline]
     pub unsafe fn try_write(&self) -> bool {
-        unsafe { self.inner.try_lock() }
+        self.inner.try_lock()
     }
 
     #[inline]
diff --git a/library/std/src/sys_common/lazy_box.rs b/library/std/src/sys_common/lazy_box.rs
deleted file mode 100644
index b45b05f63ba..00000000000
--- a/library/std/src/sys_common/lazy_box.rs
+++ /dev/null
@@ -1,88 +0,0 @@
-#![allow(dead_code)] // Only used on some platforms.
-
-// This is used to wrap pthread {Mutex, Condvar, RwLock} in.
-
-use crate::marker::PhantomData;
-use crate::ops::{Deref, DerefMut};
-use crate::ptr::null_mut;
-use crate::sync::atomic::AtomicPtr;
-use crate::sync::atomic::Ordering::{AcqRel, Acquire};
-
-pub(crate) struct LazyBox<T: LazyInit> {
-    ptr: AtomicPtr<T>,
-    _phantom: PhantomData<T>,
-}
-
-pub(crate) trait LazyInit {
-    /// This is called before the box is allocated, to provide the value to
-    /// move into the new box.
-    ///
-    /// It might be called more than once per LazyBox, as multiple threads
-    /// might race to initialize it concurrently, each constructing and initializing
-    /// their own box. All but one of them will be passed to `cancel_init` right after.
-    fn init() -> Box<Self>;
-
-    /// Any surplus boxes from `init()` that lost the initialization race
-    /// are passed to this function for disposal.
-    ///
-    /// The default implementation calls destroy().
-    fn cancel_init(x: Box<Self>) {
-        Self::destroy(x);
-    }
-
-    /// This is called to destroy a used box.
-    ///
-    /// The default implementation just drops it.
-    fn destroy(_: Box<Self>) {}
-}
-
-impl<T: LazyInit> LazyBox<T> {
-    #[inline]
-    pub const fn new() -> Self {
-        Self { ptr: AtomicPtr::new(null_mut()), _phantom: PhantomData }
-    }
-
-    #[inline]
-    fn get_pointer(&self) -> *mut T {
-        let ptr = self.ptr.load(Acquire);
-        if ptr.is_null() { self.initialize() } else { ptr }
-    }
-
-    #[cold]
-    fn initialize(&self) -> *mut T {
-        let new_ptr = Box::into_raw(T::init());
-        match self.ptr.compare_exchange(null_mut(), new_ptr, AcqRel, Acquire) {
-            Ok(_) => new_ptr,
-            Err(ptr) => {
-                // Lost the race to another thread.
-                // Drop the box we created, and use the one from the other thread instead.
-                T::cancel_init(unsafe { Box::from_raw(new_ptr) });
-                ptr
-            }
-        }
-    }
-}
-
-impl<T: LazyInit> Deref for LazyBox<T> {
-    type Target = T;
-    #[inline]
-    fn deref(&self) -> &T {
-        unsafe { &*self.get_pointer() }
-    }
-}
-
-impl<T: LazyInit> DerefMut for LazyBox<T> {
-    #[inline]
-    fn deref_mut(&mut self) -> &mut T {
-        unsafe { &mut *self.get_pointer() }
-    }
-}
-
-impl<T: LazyInit> Drop for LazyBox<T> {
-    fn drop(&mut self) {
-        let ptr = *self.ptr.get_mut();
-        if !ptr.is_null() {
-            T::destroy(unsafe { Box::from_raw(ptr) });
-        }
-    }
-}
diff --git a/library/std/src/sys_common/mod.rs b/library/std/src/sys_common/mod.rs
index aa27886ff6f..4f7a131f6bb 100644
--- a/library/std/src/sys_common/mod.rs
+++ b/library/std/src/sys_common/mod.rs
@@ -22,7 +22,6 @@ mod tests;
 
 pub mod fs;
 pub mod io;
-pub mod lazy_box;
 pub mod process;
 pub mod wstr;
 pub mod wtf8;