diff --git a/crossbeam-epoch/src/collector.rs b/crossbeam-epoch/src/collector.rs index 129009995..e082f9df3 100644 --- a/crossbeam-epoch/src/collector.rs +++ b/crossbeam-epoch/src/collector.rs @@ -14,11 +14,14 @@ /// ``` use core::fmt; +use alloc::rc::Rc; + use crate::guard::Guard; use crate::internal::{Global, Local}; use crate::primitive::sync::Arc; /// An epoch-based garbage collector. +#[derive(Clone, Default, Debug)] pub struct Collector { pub(crate) global: Arc, } @@ -26,14 +29,6 @@ pub struct Collector { unsafe impl Send for Collector {} unsafe impl Sync for Collector {} -impl Default for Collector { - fn default() -> Self { - Self { - global: Arc::new(Global::new()), - } - } -} - impl Collector { /// Creates a new collector. pub fn new() -> Self { @@ -42,22 +37,7 @@ impl Collector { /// Registers a new handle for the collector. pub fn register(&self) -> LocalHandle { - Local::register(self) - } -} - -impl Clone for Collector { - /// Creates another reference to the same garbage collector. - fn clone(&self) -> Self { - Collector { - global: self.global.clone(), - } - } -} - -impl fmt::Debug for Collector { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.pad("Collector { .. }") + Local::new(self) } } @@ -70,36 +50,31 @@ impl PartialEq for Collector { impl Eq for Collector {} /// A handle to a garbage collector. + pub struct LocalHandle { - pub(crate) local: *const Local, + pub(crate) local: Rc, } impl LocalHandle { /// Pins the handle. #[inline] pub fn pin(&self) -> Guard { - unsafe { (*self.local).pin() } + self.local.pin(); + Guard { + local: Some(self.local.clone()), + } } /// Returns `true` if the handle is pinned. #[inline] pub fn is_pinned(&self) -> bool { - unsafe { (*self.local).is_pinned() } + self.local.is_pinned() } /// Returns the `Collector` associated with this handle. #[inline] pub fn collector(&self) -> &Collector { - unsafe { (*self.local).collector() } - } -} - -impl Drop for LocalHandle { - #[inline] - fn drop(&mut self) { - unsafe { - Local::release_handle(&*self.local); - } + self.local.collector() } } @@ -139,43 +114,6 @@ mod tests { assert!(!handle.is_pinned()); } - #[test] - fn flush_local_bag() { - let collector = Collector::new(); - let handle = collector.register(); - drop(collector); - - for _ in 0..100 { - let guard = &handle.pin(); - unsafe { - let a = Owned::new(7).into_shared(guard); - guard.defer_destroy(a); - - assert!(!(*guard.local).bag.with(|b| (*b).is_empty())); - - while !(*guard.local).bag.with(|b| (*b).is_empty()) { - guard.flush(); - } - } - } - } - - #[test] - fn garbage_buffering() { - let collector = Collector::new(); - let handle = collector.register(); - drop(collector); - - let guard = &handle.pin(); - unsafe { - for _ in 0..10 { - let a = Owned::new(7).into_shared(guard); - guard.defer_destroy(a); - } - assert!(!(*guard.local).bag.with(|b| (*b).is_empty())); - } - } - #[test] fn pin_holds_advance() { #[cfg(miri)] @@ -193,9 +131,12 @@ mod tests { let guard = &handle.pin(); let before = collector.global.epoch.load(Ordering::Relaxed); - collector.global.collect(guard); - let after = collector.global.epoch.load(Ordering::Relaxed); + guard.flush(); + let mut after = collector.global.epoch.load(Ordering::Relaxed); + if after < before { + after += 3; + } assert!(after.wrapping_sub(before) <= 2); } }); @@ -204,80 +145,6 @@ mod tests { .unwrap(); } - #[cfg(not(crossbeam_sanitize))] // TODO: assertions failed due to `cfg(crossbeam_sanitize)` reduce `internal::MAX_OBJECTS` - #[test] - fn incremental() { - #[cfg(miri)] - const COUNT: usize = 500; - #[cfg(not(miri))] - const COUNT: usize = 100_000; - static DESTROYS: AtomicUsize = AtomicUsize::new(0); - - let collector = Collector::new(); - let handle = collector.register(); - - unsafe { - let guard = &handle.pin(); - for _ in 0..COUNT { - let a = Owned::new(7i32).into_shared(guard); - guard.defer_unchecked(move || { - drop(a.into_owned()); - DESTROYS.fetch_add(1, Ordering::Relaxed); - }); - } - guard.flush(); - } - - let mut last = 0; - - while last < COUNT { - let curr = DESTROYS.load(Ordering::Relaxed); - assert!(curr - last <= 1024); - last = curr; - - let guard = &handle.pin(); - collector.global.collect(guard); - } - assert!(DESTROYS.load(Ordering::Relaxed) == COUNT); - } - - #[test] - fn buffering() { - const COUNT: usize = 10; - #[cfg(miri)] - const N: usize = 500; - #[cfg(not(miri))] - const N: usize = 100_000; - static DESTROYS: AtomicUsize = AtomicUsize::new(0); - - let collector = Collector::new(); - let handle = collector.register(); - - unsafe { - let guard = &handle.pin(); - for _ in 0..COUNT { - let a = Owned::new(7i32).into_shared(guard); - guard.defer_unchecked(move || { - drop(a.into_owned()); - DESTROYS.fetch_add(1, Ordering::Relaxed); - }); - } - } - - for _ in 0..N { - collector.global.collect(&handle.pin()); - } - assert!(DESTROYS.load(Ordering::Relaxed) < COUNT); - - handle.pin().flush(); - - while DESTROYS.load(Ordering::Relaxed) < COUNT { - let guard = &handle.pin(); - collector.global.collect(guard); - } - assert_eq!(DESTROYS.load(Ordering::Relaxed), COUNT); - } - #[test] fn count_drops() { #[cfg(miri)] @@ -307,9 +174,8 @@ mod tests { guard.flush(); } - while DROPS.load(Ordering::Relaxed) < COUNT { - let guard = &handle.pin(); - collector.global.collect(guard); + for _ in 0..6 { + handle.pin().flush(); } assert_eq!(DROPS.load(Ordering::Relaxed), COUNT); } @@ -338,9 +204,8 @@ mod tests { guard.flush(); } - while DESTROYS.load(Ordering::Relaxed) < COUNT { - let guard = &handle.pin(); - collector.global.collect(guard); + for _ in 0..6 { + handle.pin().flush(); } assert_eq!(DESTROYS.load(Ordering::Relaxed), COUNT); } @@ -376,9 +241,9 @@ mod tests { guard.flush(); } - while DROPS.load(Ordering::Relaxed) < COUNT { + for _ in 0..6 { guard.repin(); - collector.global.collect(&guard); + guard.flush(); } assert_eq!(DROPS.load(Ordering::Relaxed), COUNT); } @@ -411,16 +276,15 @@ mod tests { guard.flush(); } - while DESTROYS.load(Ordering::Relaxed) < COUNT { - let guard = &handle.pin(); - collector.global.collect(guard); + for _ in 0..6 { + handle.pin().flush(); } assert_eq!(DESTROYS.load(Ordering::Relaxed), COUNT); } #[test] fn stress() { - const THREADS: usize = 8; + const THREADS: usize = 3; #[cfg(miri)] const COUNT: usize = 500; #[cfg(not(miri))] @@ -435,7 +299,7 @@ mod tests { } } - let collector = Collector::new(); + let collector = &Collector::new(); thread::scope(|scope| { for _ in 0..THREADS { @@ -454,9 +318,8 @@ mod tests { .unwrap(); let handle = collector.register(); - while DROPS.load(Ordering::Relaxed) < COUNT * THREADS { - let guard = &handle.pin(); - collector.global.collect(guard); + for _ in 0..6 { + handle.pin().flush(); } assert_eq!(DROPS.load(Ordering::Relaxed), COUNT * THREADS); } diff --git a/crossbeam-epoch/src/deferred.rs b/crossbeam-epoch/src/deferred.rs index 2f3d79fdf..290001e6c 100644 --- a/crossbeam-epoch/src/deferred.rs +++ b/crossbeam-epoch/src/deferred.rs @@ -29,6 +29,7 @@ impl fmt::Debug for Deferred { } impl Deferred { + #[cfg(crossbeam_loom)] pub(crate) const NO_OP: Self = { fn no_op_call(_raw: *mut u8) {} Self { diff --git a/crossbeam-epoch/src/epoch.rs b/crossbeam-epoch/src/epoch.rs deleted file mode 100644 index 663508bd7..000000000 --- a/crossbeam-epoch/src/epoch.rs +++ /dev/null @@ -1,133 +0,0 @@ -//! The global epoch -//! -//! The last bit in this number is unused and is always zero. Every so often the global epoch is -//! incremented, i.e. we say it "advances". A pinned participant may advance the global epoch only -//! if all currently pinned participants have been pinned in the current epoch. -//! -//! If an object became garbage in some epoch, then we can be sure that after two advancements no -//! participant will hold a reference to it. That is the crux of safe memory reclamation. - -use crate::primitive::sync::atomic::AtomicUsize; -use core::sync::atomic::Ordering; - -/// An epoch that can be marked as pinned or unpinned. -/// -/// Internally, the epoch is represented as an integer that wraps around at some unspecified point -/// and a flag that represents whether it is pinned or unpinned. -#[derive(Copy, Clone, Default, Debug, Eq, PartialEq)] -pub(crate) struct Epoch { - /// The least significant bit is set if pinned. The rest of the bits hold the epoch. - data: usize, -} - -impl Epoch { - /// Returns the starting epoch in unpinned state. - #[inline] - pub(crate) fn starting() -> Self { - Self::default() - } - - /// Returns the number of epochs `self` is ahead of `rhs`. - /// - /// Internally, epochs are represented as numbers in the range `(isize::MIN / 2) .. (isize::MAX - /// / 2)`, so the returned distance will be in the same interval. - pub(crate) fn wrapping_sub(self, rhs: Self) -> isize { - // The result is the same with `(self.data & !1).wrapping_sub(rhs.data & !1) as isize >> 1`, - // because the possible difference of LSB in `(self.data & !1).wrapping_sub(rhs.data & !1)` - // will be ignored in the shift operation. - self.data.wrapping_sub(rhs.data & !1) as isize >> 1 - } - - /// Returns `true` if the epoch is marked as pinned. - #[inline] - pub(crate) fn is_pinned(self) -> bool { - (self.data & 1) == 1 - } - - /// Returns the same epoch, but marked as pinned. - #[inline] - pub(crate) fn pinned(self) -> Epoch { - Epoch { - data: self.data | 1, - } - } - - /// Returns the same epoch, but marked as unpinned. - #[inline] - pub(crate) fn unpinned(self) -> Epoch { - Epoch { - data: self.data & !1, - } - } - - /// Returns the successor epoch. - /// - /// The returned epoch will be marked as pinned only if the previous one was as well. - #[inline] - pub(crate) fn successor(self) -> Epoch { - Epoch { - data: self.data.wrapping_add(2), - } - } -} - -/// An atomic value that holds an `Epoch`. -#[derive(Default, Debug)] -pub(crate) struct AtomicEpoch { - /// Since `Epoch` is just a wrapper around `usize`, an `AtomicEpoch` is similarly represented - /// using an `AtomicUsize`. - data: AtomicUsize, -} - -impl AtomicEpoch { - /// Creates a new atomic epoch. - #[inline] - pub(crate) fn new(epoch: Epoch) -> Self { - let data = AtomicUsize::new(epoch.data); - AtomicEpoch { data } - } - - /// Loads a value from the atomic epoch. - #[inline] - pub(crate) fn load(&self, ord: Ordering) -> Epoch { - Epoch { - data: self.data.load(ord), - } - } - - /// Stores a value into the atomic epoch. - #[inline] - pub(crate) fn store(&self, epoch: Epoch, ord: Ordering) { - self.data.store(epoch.data, ord); - } - - /// Stores a value into the atomic epoch if the current value is the same as `current`. - /// - /// The return value is a result indicating whether the new value was written and containing - /// the previous value. On success this value is guaranteed to be equal to `current`. - /// - /// This method takes two `Ordering` arguments to describe the memory - /// ordering of this operation. `success` describes the required ordering for the - /// read-modify-write operation that takes place if the comparison with `current` succeeds. - /// `failure` describes the required ordering for the load operation that takes place when - /// the comparison fails. Using `Acquire` as success ordering makes the store part - /// of this operation `Relaxed`, and using `Release` makes the successful load - /// `Relaxed`. The failure ordering can only be `SeqCst`, `Acquire` or `Relaxed` - /// and must be equivalent to or weaker than the success ordering. - #[inline] - pub(crate) fn compare_exchange( - &self, - current: Epoch, - new: Epoch, - success: Ordering, - failure: Ordering, - ) -> Result { - match self - .data - .compare_exchange(current.data, new.data, success, failure) - { - Ok(data) => Ok(Epoch { data }), - Err(data) => Err(Epoch { data }), - } - } -} diff --git a/crossbeam-epoch/src/guard.rs b/crossbeam-epoch/src/guard.rs index ba7fe1b11..d20a62764 100644 --- a/crossbeam-epoch/src/guard.rs +++ b/crossbeam-epoch/src/guard.rs @@ -1,6 +1,6 @@ use core::fmt; -use core::mem; +use alloc::rc::Rc; use scopeguard::defer; use crate::atomic::Shared; @@ -69,7 +69,7 @@ use crate::internal::Local; /// /// [`pin`]: super::pin pub struct Guard { - pub(crate) local: *const Local, + pub(crate) local: Option>, } impl Guard { @@ -192,7 +192,7 @@ impl Guard { F: FnOnce() -> R, { if let Some(local) = self.local.as_ref() { - local.defer(Deferred::new(move || drop(f())), self); + local.defer(Deferred::new(move || drop(f()))); } else { drop(f()); } @@ -292,8 +292,8 @@ impl Guard { /// guard.flush(); /// ``` pub fn flush(&self) { - if let Some(local) = unsafe { self.local.as_ref() } { - local.flush(self); + if let Some(local) = self.local.as_ref() { + local.flush(); } } @@ -326,7 +326,7 @@ impl Guard { /// # unsafe { drop(a.into_owned()); } // avoid leak /// ``` pub fn repin(&mut self) { - if let Some(local) = unsafe { self.local.as_ref() } { + if let Some(local) = self.local.as_ref() { local.repin(); } } @@ -366,18 +366,14 @@ impl Guard { where F: FnOnce() -> R, { - if let Some(local) = unsafe { self.local.as_ref() } { - // We need to acquire a handle here to ensure the Local doesn't - // disappear from under us. - local.acquire_handle(); + if let Some(local) = self.local.as_ref() { local.unpin(); } // Ensure the Guard is re-pinned even if the function panics defer! { - if let Some(local) = unsafe { self.local.as_ref() } { - mem::forget(local.pin()); - local.release_handle(); + if let Some(local) = self.local.as_ref() { + local.pin(); } } @@ -401,14 +397,14 @@ impl Guard { /// assert!(guard1.collector() == guard2.collector()); /// ``` pub fn collector(&self) -> Option<&Collector> { - unsafe { self.local.as_ref().map(|local| local.collector()) } + self.local.as_ref().map(|local| local.collector()) } } impl Drop for Guard { #[inline] fn drop(&mut self) { - if let Some(local) = unsafe { self.local.as_ref() } { + if let Some(local) = self.local.as_ref() { local.unpin(); } } @@ -513,8 +509,6 @@ pub unsafe fn unprotected() -> &'static Guard { // a `static` struct GuardWrapper(Guard); unsafe impl Sync for GuardWrapper {} - static UNPROTECTED: GuardWrapper = GuardWrapper(Guard { - local: core::ptr::null(), - }); + static UNPROTECTED: GuardWrapper = GuardWrapper(Guard { local: None }); &UNPROTECTED.0 } diff --git a/crossbeam-epoch/src/internal.rs b/crossbeam-epoch/src/internal.rs index 486993c63..09cd4cd63 100644 --- a/crossbeam-epoch/src/internal.rs +++ b/crossbeam-epoch/src/internal.rs @@ -1,17 +1,9 @@ //! The global data and participant for garbage collection. //! -//! # Registration -//! -//! In order to track all participants in one place, we need some form of participant -//! registration. When a participant is created, it is registered to a global lock-free -//! singly-linked list of registries; and when a participant is leaving, it is unregistered from the -//! list. -//! //! # Pinning //! -//! Every participant contains an integer that tells whether the participant is pinned and if so, -//! what was the global epoch at the time it was pinned. Participants also hold a pin counter that -//! aids in periodic global epoch advancement. +//! When a participant is pinned, it increments a global counter for the current epoch. When it is +//! unpinned, it decrements that counter. //! //! When a participant is pinned, a `Guard` is returned as a witness that the participant is pinned. //! Guards are necessary for performing atomic operations, and for freeing/dropping locations. @@ -20,319 +12,114 @@ //! //! Objects that get unlinked from concurrent data structures must be stashed away until the global //! epoch sufficiently advances so that they become safe for destruction. Pointers to such objects -//! are pushed into a thread-local bag, and when it becomes full, the bag is marked with the current -//! global epoch and pushed into the global queue of bags. We store objects in thread-local storages -//! for amortizing the synchronization cost of pushing the garbages to a global queue. +//! are pushed into a thread-local array, and when it becomes larger than a certain size, the items +//! are moved to the global garbage pile for the current epock. We store objects in thread-local +//! storage to reduce contention on the global data structure. Whenever objects are put on a global +//! pile, the current thread also tries to advance the epoch. +//! +//! # Ordering //! -//! # Global queue +//! There may be threads pinned in up to two epochs at once. This is required for wait-freedom. +//! A thread can advance the epoch once it sees that no threads are pinned in the epoch before it. +//! This ensures that all pins in epoch n happen-before all pins in epoch n+2, and the thread that +//! advances from n+1 to n+2 additionally happens-after all pins in n+1. The first condition means +//! that if a thread pinned in n unlinks a pointer from the concurrent data structure, only threads +//! pinned in n+1 and earler could have found it, and the second additionally means that the thread +//! that advanced to n+2 may safely delete that pointer. //! -//! Whenever a bag is pushed into a queue, the objects in some bags in the queue are collected and -//! destroyed along the way. This design reduces contention on data structures. The global queue -//! cannot be explicitly accessed: the only way to interact with it is by calling functions -//! `defer()` that adds an object to the thread-local bag, or `collect()` that manually triggers -//! garbage collection. +//! # Global piles //! -//! Ideally each instance of concurrent data structure may have its own queue that gets fully +//! Each epoch has its own global pile of garbage. The piles may be concurrently inserted into, +//! but clearing them is not thread safe. While it is being cleared, we lock the counter for its +//! epoch, so no thread can pin itself there. +//! +//! Ideally each instance of concurrent data structure may have its own collector that gets fully //! destroyed as soon as the data structure gets dropped. -use crate::primitive::cell::UnsafeCell; -use crate::primitive::sync::atomic; +use crate::sync::{pile::Pile, refcount::RefCount}; use core::cell::Cell; -use core::mem::{self, ManuallyDrop}; -use core::num::Wrapping; use core::sync::atomic::Ordering; -use core::{fmt, ptr}; +use alloc::rc::Rc; use crossbeam_utils::CachePadded; -use memoffset::offset_of; -use crate::atomic::{Owned, Shared}; use crate::collector::{Collector, LocalHandle}; use crate::deferred::Deferred; -use crate::epoch::{AtomicEpoch, Epoch}; -use crate::guard::{unprotected, Guard}; -use crate::sync::list::{Entry, IsElement, IterError, List}; -use crate::sync::queue::Queue; - -/// Maximum number of objects a bag can contain. -#[cfg(not(any(crossbeam_sanitize, miri)))] -const MAX_OBJECTS: usize = 64; -// Makes it more likely to trigger any potential data races. -#[cfg(any(crossbeam_sanitize, miri))] -const MAX_OBJECTS: usize = 4; - -/// A bag of deferred functions. -pub(crate) struct Bag { - /// Stashed objects. - deferreds: [Deferred; MAX_OBJECTS], - len: usize, -} - -/// `Bag::try_push()` requires that it is safe for another thread to execute the given functions. -unsafe impl Send for Bag {} - -impl Bag { - /// Returns a new, empty bag. - pub(crate) fn new() -> Self { - Self::default() - } - - /// Returns `true` if the bag is empty. - pub(crate) fn is_empty(&self) -> bool { - self.len == 0 - } - - /// Attempts to insert a deferred function into the bag. - /// - /// Returns `Ok(())` if successful, and `Err(deferred)` for the given `deferred` if the bag is - /// full. - /// - /// # Safety - /// - /// It should be safe for another thread to execute the given function. - pub(crate) unsafe fn try_push(&mut self, deferred: Deferred) -> Result<(), Deferred> { - if self.len < MAX_OBJECTS { - self.deferreds[self.len] = deferred; - self.len += 1; - Ok(()) - } else { - Err(deferred) - } - } - - /// Seals the bag with the given epoch. - fn seal(self, epoch: Epoch) -> SealedBag { - SealedBag { epoch, _bag: self } - } -} - -impl Default for Bag { - fn default() -> Self { - Bag { - len: 0, - deferreds: [Deferred::NO_OP; MAX_OBJECTS], - } - } -} - -impl Drop for Bag { - fn drop(&mut self) { - // Call all deferred functions. - for deferred in &mut self.deferreds[..self.len] { - let no_op = Deferred::NO_OP; - let owned_deferred = mem::replace(deferred, no_op); - owned_deferred.call(); - } - } -} - -// can't #[derive(Debug)] because Debug is not implemented for arrays 64 items long -impl fmt::Debug for Bag { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Bag") - .field("deferreds", &&self.deferreds[..self.len]) - .finish() - } -} - -/// A pair of an epoch and a bag. -#[derive(Default, Debug)] -struct SealedBag { - epoch: Epoch, - _bag: Bag, -} - -/// It is safe to share `SealedBag` because `is_expired` only inspects the epoch. -unsafe impl Sync for SealedBag {} - -impl SealedBag { - /// Checks if it is safe to drop the bag w.r.t. the given global epoch. - fn is_expired(&self, global_epoch: Epoch) -> bool { - // A pinned participant can witness at most one epoch advancement. Therefore, any bag that - // is within one epoch of the current one cannot be destroyed yet. - global_epoch.wrapping_sub(self.epoch) >= 2 - } -} +use crate::primitive::sync::atomic::AtomicUsize; /// The global data for a garbage collector. +#[derive(Debug)] pub(crate) struct Global { - /// The intrusive linked list of `Local`s. - locals: List, - - /// The global queue of bags of deferred functions. - queue: Queue, - /// The global epoch. - pub(crate) epoch: CachePadded, + pub(crate) epoch: CachePadded, + pins: [RefCount; 3], + garbage: [Pile; 3], } -impl Global { - /// Number of bags to destroy. - const COLLECT_STEPS: usize = 8; - - /// Creates a new global data for garbage collection. - #[inline] - pub(crate) fn new() -> Self { - Self { - locals: List::new(), - queue: Queue::new(), - epoch: CachePadded::new(AtomicEpoch::new(Epoch::starting())), - } - } - - /// Pushes the bag into the global queue and replaces the bag with a new empty bag. - pub(crate) fn push_bag(&self, bag: &mut Bag, guard: &Guard) { - let bag = mem::replace(bag, Bag::new()); - - atomic::fence(Ordering::SeqCst); - - let epoch = self.epoch.load(Ordering::Relaxed); - self.queue.push(bag.seal(epoch), guard); - } - - /// Collects several bags from the global queue and executes deferred functions in them. - /// - /// Note: This may itself produce garbage and in turn allocate new bags. - /// - /// `pin()` rarely calls `collect()`, so we want the compiler to place that call on a cold - /// path. In other words, we want the compiler to optimize branching for the case when - /// `collect()` is not called. - #[cold] - pub(crate) fn collect(&self, guard: &Guard) { - let global_epoch = self.try_advance(guard); - - let steps = if cfg!(crossbeam_sanitize) { - usize::max_value() - } else { - Self::COLLECT_STEPS +impl Default for Global { + fn default() -> Self { + let result = Global { + epoch: Default::default(), + pins: Default::default(), + garbage: Default::default(), }; - - for _ in 0..steps { - match self.queue.try_pop_if( - |sealed_bag: &SealedBag| sealed_bag.is_expired(global_epoch), - guard, - ) { - None => break, - Some(sealed_bag) => drop(sealed_bag), - } - } - } - - /// Attempts to advance the global epoch. - /// - /// The global epoch can advance only if all currently pinned participants have been pinned in - /// the current epoch. - /// - /// Returns the current global epoch. - /// - /// `try_advance()` is annotated `#[cold]` because it is rarely called. - #[cold] - pub(crate) fn try_advance(&self, guard: &Guard) -> Epoch { - let global_epoch = self.epoch.load(Ordering::Relaxed); - atomic::fence(Ordering::SeqCst); - - // TODO(stjepang): `Local`s are stored in a linked list because linked lists are fairly - // easy to implement in a lock-free manner. However, traversal can be slow due to cache - // misses and data dependencies. We should experiment with other data structures as well. - for local in self.locals.iter(guard) { - match local { - Err(IterError::Stalled) => { - // A concurrent thread stalled this iteration. That thread might also try to - // advance the epoch, in which case we leave the job to it. Otherwise, the - // epoch will not be advanced. - return global_epoch; - } - Ok(local) => { - let local_epoch = local.epoch.load(Ordering::Relaxed); - - // If the participant was pinned in a different epoch, we cannot advance the - // global epoch just yet. - if local_epoch.is_pinned() && local_epoch.unpinned() != global_epoch { - return global_epoch; - } - } - } - } - atomic::fence(Ordering::Acquire); - - // All pinned participants were pinned in the current global epoch. - // Now let's advance the global epoch... - // - // Note that if another thread already advanced it before us, this store will simply - // overwrite the global epoch with the same value. This is true because `try_advance` was - // called from a thread that was pinned in `global_epoch`, and the global epoch cannot be - // advanced two steps ahead of it. - let new_epoch = global_epoch.successor(); - self.epoch.store(new_epoch, Ordering::Release); - new_epoch + result.pins[1].try_write(); + result } } /// Participant for garbage collection. pub(crate) struct Local { - /// A node in the intrusive linked list of `Local`s. - entry: Entry, - - /// The local epoch. - epoch: AtomicEpoch, - /// A reference to the global data. /// /// When all guards and handles get dropped, this reference is destroyed. - collector: UnsafeCell>, + collector: Collector, - /// The local bag of deferred functions. - pub(crate) bag: UnsafeCell, + /// The local epoch. + epoch: Cell, /// The number of guards keeping this participant pinned. guard_count: Cell, - /// The number of active handles. - handle_count: Cell, + /// Locally stored Deferred functions. Pushed in bulk to reduce contention. + buffer: Cell>, - /// Total number of pinnings performed. - /// - /// This is just an auxiliary counter that sometimes kicks off collection. - pin_count: Cell>, + /// A different number for each Local. Used as a hint for the [RefCount]. + id: usize, } // Make sure `Local` is less than or equal to 2048 bytes. -// https://github.com/crossbeam-rs/crossbeam/issues/551 -#[cfg(not(any(crossbeam_sanitize, miri)))] // `crossbeam_sanitize` and `miri` reduce the size of `Local` #[test] fn local_size() { - // TODO: https://github.com/crossbeam-rs/crossbeam/issues/869 - // assert!( - // core::mem::size_of::() <= 2048, - // "An allocation of `Local` should be <= 2048 bytes." - // ); + assert!( + core::mem::size_of::() <= 2048, + "An allocation of `Local` should be <= 2048 bytes." + ); +} + +#[cfg(not(crossbeam_loom))] +static LOCAL_ID: AtomicUsize = AtomicUsize::new(0); +#[cfg(crossbeam_loom)] +loom::lazy_static! { + // AtomicUsize::new is not const in Loom. + static ref LOCAL_ID: AtomicUsize = AtomicUsize::new(0); } impl Local { - /// Number of pinnings after which a participant will execute some deferred functions from the + /// Number of defers after which a participant will execute some deferred functions from the /// global queue. - const PINNINGS_BETWEEN_COLLECT: usize = 128; - - /// Registers a new `Local` in the provided `Global`. - pub(crate) fn register(collector: &Collector) -> LocalHandle { - unsafe { - // Since we dereference no pointers in this block, it is safe to use `unprotected`. - - let local = Owned::new(Local { - entry: Entry::default(), - epoch: AtomicEpoch::new(Epoch::starting()), - collector: UnsafeCell::new(ManuallyDrop::new(collector.clone())), - bag: UnsafeCell::new(Bag::new()), - guard_count: Cell::new(0), - handle_count: Cell::new(1), - pin_count: Cell::new(Wrapping(0)), - }) - .into_shared(unprotected()); - collector.global.locals.insert(local, unprotected()); - LocalHandle { - local: local.as_raw(), - } - } + const DEFERS_BETWEEN_COLLECT: usize = 16; + + /// Create a new [Local] + pub(crate) fn new(collector: &Collector) -> LocalHandle { + let local = Rc::new(Local { + id: LOCAL_ID.fetch_add(1, Ordering::Relaxed), + epoch: Cell::new(0), + collector: collector.clone(), + guard_count: Cell::new(0), + buffer: Cell::new(vec![]), + }); + LocalHandle { local } } /// Returns a reference to the `Global` in which this `Local` resides. @@ -344,7 +131,7 @@ impl Local { /// Returns a reference to the `Collector` in which this `Local` resides. #[inline] pub(crate) fn collector(&self) -> &Collector { - self.collector.with(|c| unsafe { &**c }) + &self.collector } /// Returns `true` if the current participant is pinned. @@ -353,91 +140,45 @@ impl Local { self.guard_count.get() > 0 } + /// Returns the most recent epoch the participant was pinned in. + #[inline] + pub(crate) fn epoch(&self) -> usize { + self.epoch.get() + } + /// Adds `deferred` to the thread-local bag. /// /// # Safety /// /// It should be safe for another thread to execute the given function. - pub(crate) unsafe fn defer(&self, mut deferred: Deferred, guard: &Guard) { - let bag = self.bag.with_mut(|b| &mut *b); - - while let Err(d) = bag.try_push(deferred) { - self.global().push_bag(bag, guard); - deferred = d; - } - } - - pub(crate) fn flush(&self, guard: &Guard) { - let bag = self.bag.with_mut(|b| unsafe { &mut *b }); - - if !bag.is_empty() { - self.global().push_bag(bag, guard); + pub(crate) unsafe fn defer(&self, deferred: Deferred) { + let mut buffer = self.buffer.replace(vec![]); + buffer.push(deferred); + let buffered = buffer.len(); + self.buffer.set(buffer); + + // After every `DEFERS_BETWEEN_COLLECT` try advancing the epoch and collecting + // some garbage. + if buffered > Self::DEFERS_BETWEEN_COLLECT { + self.flush(); } - - self.global().collect(guard); } +} +impl Local { /// Pins the `Local`. #[inline] - pub(crate) fn pin(&self) -> Guard { - let guard = Guard { local: self }; - + pub(crate) fn pin(&self) { let guard_count = self.guard_count.get(); self.guard_count.set(guard_count.checked_add(1).unwrap()); if guard_count == 0 { - let global_epoch = self.global().epoch.load(Ordering::Relaxed); - let new_epoch = global_epoch.pinned(); - - // Now we must store `new_epoch` into `self.epoch` and execute a `SeqCst` fence. - // The fence makes sure that any future loads from `Atomic`s will not happen before - // this store. - if cfg!(all( - any(target_arch = "x86", target_arch = "x86_64"), - not(miri) - )) { - // HACK(stjepang): On x86 architectures there are two different ways of executing - // a `SeqCst` fence. - // - // 1. `atomic::fence(SeqCst)`, which compiles into a `mfence` instruction. - // 2. `_.compare_exchange(_, _, SeqCst, SeqCst)`, which compiles into a `lock cmpxchg` - // instruction. - // - // Both instructions have the effect of a full barrier, but benchmarks have shown - // that the second one makes pinning faster in this particular case. It is not - // clear that this is permitted by the C++ memory model (SC fences work very - // differently from SC accesses), but experimental evidence suggests that this - // works fine. Using inline assembly would be a viable (and correct) alternative, - // but alas, that is not possible on stable Rust. - let current = Epoch::starting(); - let res = self.epoch.compare_exchange( - current, - new_epoch, - Ordering::SeqCst, - Ordering::SeqCst, - ); - debug_assert!(res.is_ok(), "participant was expected to be unpinned"); - // We add a compiler fence to make it less likely for LLVM to do something wrong - // here. Formally, this is not enough to get rid of data races; practically, - // it should go a long way. - atomic::compiler_fence(Ordering::SeqCst); - } else { - self.epoch.store(new_epoch, Ordering::Relaxed); - atomic::fence(Ordering::SeqCst); - } - - // Increment the pin counter. - let count = self.pin_count.get(); - self.pin_count.set(count + Wrapping(1)); - - // After every `PINNINGS_BETWEEN_COLLECT` try advancing the epoch and collecting - // some garbage. - if count.0 % Self::PINNINGS_BETWEEN_COLLECT == 0 { - self.global().collect(&guard); + let mut epoch = self.global().epoch.load(Ordering::Relaxed); + while !self.global().pins[epoch].try_read(self.id) { + epoch = (epoch + 1) % 3; } + self.epoch.set(epoch); } - - guard } /// Unpins the `Local`. @@ -447,11 +188,7 @@ impl Local { self.guard_count.set(guard_count - 1); if guard_count == 1 { - self.epoch.store(Epoch::starting(), Ordering::Release); - - if self.handle_count.get() == 0 { - self.finalize(); - } + self.global().pins[self.epoch()].done_read(self.id); } } @@ -462,98 +199,61 @@ impl Local { // Update the local epoch only if there's only one guard. if guard_count == 1 { - let epoch = self.epoch.load(Ordering::Relaxed); - let global_epoch = self.global().epoch.load(Ordering::Relaxed).pinned(); - - // Update the local epoch only if the global epoch is greater than the local epoch. - if epoch != global_epoch { - // We store the new epoch with `Release` because we need to ensure any memory - // accesses from the previous epoch do not leak into the new one. - self.epoch.store(global_epoch, Ordering::Release); - - // However, we don't need a following `SeqCst` fence, because it is safe for memory - // accesses from the new epoch to be executed before updating the local epoch. At - // worse, other threads will see the new epoch late and delay GC slightly. + let mut new_epoch = self.global().epoch.load(Ordering::Relaxed); + if self.epoch() != new_epoch { + self.global().pins[self.epoch()].done_read(self.id); + while !self.global().pins[new_epoch].try_read(self.id) { + new_epoch = (new_epoch + 1) % 3; + } + self.epoch.set(new_epoch); } } } - /// Increments the handle count. - #[inline] - pub(crate) fn acquire_handle(&self) { - let handle_count = self.handle_count.get(); - debug_assert!(handle_count >= 1); - self.handle_count.set(handle_count + 1); - } - - /// Decrements the handle count. - #[inline] - pub(crate) fn release_handle(&self) { - let guard_count = self.guard_count.get(); - let handle_count = self.handle_count.get(); - debug_assert!(handle_count >= 1); - self.handle_count.set(handle_count - 1); - - if guard_count == 0 && handle_count == 1 { - self.finalize(); - } - } - - /// Removes the `Local` from the global linked list. + /// `defer()` rarely calls `flush()`, so we want the compiler to place that call on a + /// cold path. In other words, we want the compiler to optimize branching for the case + /// when `flush()` is not called. #[cold] - fn finalize(&self) { - debug_assert_eq!(self.guard_count.get(), 0); - debug_assert_eq!(self.handle_count.get(), 0); - - // Temporarily increment handle count. This is required so that the following call to `pin` - // doesn't call `finalize` again. - self.handle_count.set(1); - unsafe { - // Pin and move the local bag into the global queue. It's important that `push_bag` - // doesn't defer destruction on any new garbage. - let guard = &self.pin(); - self.global() - .push_bag(self.bag.with_mut(|b| &mut *b), guard); - } - // Revert the handle count back to zero. - self.handle_count.set(0); - - unsafe { - // Take the reference to the `Global` out of this `Local`. Since we're not protected - // by a guard at this time, it's crucial that the reference is read before marking the - // `Local` as deleted. - let collector: Collector = ptr::read(self.collector.with(|c| &*(*c))); - - // Mark this node in the linked list as deleted. - self.entry.delete(unprotected()); - - // Finally, drop the reference to the global. Note that this might be the last reference - // to the `Global`. If so, the global data will be destroyed and all deferred functions - // in its queue will be executed. - drop(collector); - } + pub(crate) fn flush(&self) { + let bag = &self.global().garbage[self.epoch()]; + let mut buffer = self.buffer.replace(vec![]); + // Safety: We are pinned to self.epoch at this point + unsafe { bag.try_push(&mut buffer) }; + self.buffer.set(buffer); + self.global().collect(self); } } -impl IsElement for Local { - fn entry_of(local: &Local) -> &Entry { - unsafe { - let entry_ptr = (local as *const Local as *const u8) - .add(offset_of!(Local, entry)) - .cast::(); - &*entry_ptr +impl Global { + /// Attempt to collect global garbage and advance the epoch + /// + /// Note: This may itself produce garbage. + pub(crate) fn collect(&self, local: &Local) { + let next = (local.epoch() + 1) % 3; + let previous = (local.epoch() + 2) % 3; + if self.pins[previous].try_write() { + scopeguard::defer! { + // try_read() calls in next now happen-after done_read() calls in previous + self.pins[next].done_write(); + self.epoch.store(next, Ordering::Relaxed); + } + unsafe { self.garbage[next].call() } } } +} - unsafe fn element_of(entry: &Entry) -> &Local { - let local_ptr = (entry as *const Entry as *const u8) - .sub(offset_of!(Local, entry)) - .cast::(); - &*local_ptr - } - - unsafe fn finalize(entry: &Entry, guard: &Guard) { - guard.defer_destroy(Shared::from(Self::element_of(entry) as *const _)); +impl Drop for Local { + fn drop(&mut self) { + let mut buffer = self.buffer.replace(vec![]); + if !buffer.is_empty() { + self.pin(); + let bag = &self.global().garbage[self.epoch()]; + while !buffer.is_empty() { + // Safety: We are pinned to self.epoch at this point + unsafe { bag.try_push(&mut buffer) } + } + self.unpin(); + } } } @@ -575,29 +275,4 @@ mod tests { d.call(); assert_eq!(FLAG.load(Ordering::Relaxed), 42); } - - #[test] - fn check_bag() { - static FLAG: AtomicUsize = AtomicUsize::new(0); - fn incr() { - FLAG.fetch_add(1, Ordering::Relaxed); - } - - let mut bag = Bag::new(); - assert!(bag.is_empty()); - - for _ in 0..MAX_OBJECTS { - assert!(unsafe { bag.try_push(Deferred::new(incr)).is_ok() }); - assert!(!bag.is_empty()); - assert_eq!(FLAG.load(Ordering::Relaxed), 0); - } - - let result = unsafe { bag.try_push(Deferred::new(incr)) }; - assert!(result.is_err()); - assert!(!bag.is_empty()); - assert_eq!(FLAG.load(Ordering::Relaxed), 0); - - drop(bag); - assert_eq!(FLAG.load(Ordering::Relaxed), MAX_OBJECTS); - } } diff --git a/crossbeam-epoch/src/lib.rs b/crossbeam-epoch/src/lib.rs index d00c97588..9d69d43da 100644 --- a/crossbeam-epoch/src/lib.rs +++ b/crossbeam-epoch/src/lib.rs @@ -145,7 +145,6 @@ cfg_if! { mod atomic; mod collector; mod deferred; - mod epoch; mod guard; mod internal; mod sync; diff --git a/crossbeam-epoch/src/sync/list.rs b/crossbeam-epoch/src/sync/list.rs deleted file mode 100644 index 52ffd6fca..000000000 --- a/crossbeam-epoch/src/sync/list.rs +++ /dev/null @@ -1,487 +0,0 @@ -//! Lock-free intrusive linked list. -//! -//! Ideas from Michael. High Performance Dynamic Lock-Free Hash Tables and List-Based Sets. SPAA -//! 2002. - -use core::marker::PhantomData; -use core::sync::atomic::Ordering::{Acquire, Relaxed, Release}; - -use crate::{unprotected, Atomic, Guard, Shared}; - -/// An entry in a linked list. -/// -/// An Entry is accessed from multiple threads, so it would be beneficial to put it in a different -/// cache-line than thread-local data in terms of performance. -#[derive(Debug)] -pub(crate) struct Entry { - /// The next entry in the linked list. - /// If the tag is 1, this entry is marked as deleted. - next: Atomic, -} - -/// Implementing this trait asserts that the type `T` can be used as an element in the intrusive -/// linked list defined in this module. `T` has to contain (or otherwise be linked to) an instance -/// of `Entry`. -/// -/// # Example -/// -/// ```ignore -/// struct A { -/// entry: Entry, -/// data: usize, -/// } -/// -/// impl IsElement for A { -/// fn entry_of(a: &A) -> &Entry { -/// let entry_ptr = ((a as usize) + offset_of!(A, entry)) as *const Entry; -/// unsafe { &*entry_ptr } -/// } -/// -/// unsafe fn element_of(entry: &Entry) -> &T { -/// let elem_ptr = ((entry as usize) - offset_of!(A, entry)) as *const T; -/// &*elem_ptr -/// } -/// -/// unsafe fn finalize(entry: &Entry, guard: &Guard) { -/// guard.defer_destroy(Shared::from(Self::element_of(entry) as *const _)); -/// } -/// } -/// ``` -/// -/// This trait is implemented on a type separate from `T` (although it can be just `T`), because -/// one type might be placeable into multiple lists, in which case it would require multiple -/// implementations of `IsElement`. In such cases, each struct implementing `IsElement` -/// represents a distinct `Entry` in `T`. -/// -/// For example, we can insert the following struct into two lists using `entry1` for one -/// and `entry2` for the other: -/// -/// ```ignore -/// struct B { -/// entry1: Entry, -/// entry2: Entry, -/// data: usize, -/// } -/// ``` -/// -pub(crate) trait IsElement { - /// Returns a reference to this element's `Entry`. - fn entry_of(_: &T) -> &Entry; - - /// Given a reference to an element's entry, returns that element. - /// - /// ```ignore - /// let elem = ListElement::new(); - /// assert_eq!(elem.entry_of(), - /// unsafe { ListElement::element_of(elem.entry_of()) } ); - /// ``` - /// - /// # Safety - /// - /// The caller has to guarantee that the `Entry` is called with was retrieved from an instance - /// of the element type (`T`). - unsafe fn element_of(_: &Entry) -> &T; - - /// The function that is called when an entry is unlinked from list. - /// - /// # Safety - /// - /// The caller has to guarantee that the `Entry` is called with was retrieved from an instance - /// of the element type (`T`). - unsafe fn finalize(_: &Entry, _: &Guard); -} - -/// A lock-free, intrusive linked list of type `T`. -#[derive(Debug)] -pub(crate) struct List = T> { - /// The head of the linked list. - head: Atomic, - - /// The phantom data for using `T` and `C`. - _marker: PhantomData<(T, C)>, -} - -/// An iterator used for retrieving values from the list. -pub(crate) struct Iter<'g, T, C: IsElement> { - /// The guard that protects the iteration. - guard: &'g Guard, - - /// Pointer from the predecessor to the current entry. - pred: &'g Atomic, - - /// The current entry. - curr: Shared<'g, Entry>, - - /// The list head, needed for restarting iteration. - head: &'g Atomic, - - /// Logically, we store a borrow of an instance of `T` and - /// use the type information from `C`. - _marker: PhantomData<(&'g T, C)>, -} - -/// An error that occurs during iteration over the list. -#[derive(PartialEq, Debug)] -pub(crate) enum IterError { - /// A concurrent thread modified the state of the list at the same place that this iterator - /// was inspecting. Subsequent iteration will restart from the beginning of the list. - Stalled, -} - -impl Default for Entry { - /// Returns the empty entry. - fn default() -> Self { - Self { - next: Atomic::null(), - } - } -} - -impl Entry { - /// Marks this entry as deleted, deferring the actual deallocation to a later iteration. - /// - /// # Safety - /// - /// The entry should be a member of a linked list, and it should not have been deleted. - /// It should be safe to call `C::finalize` on the entry after the `guard` is dropped, where `C` - /// is the associated helper for the linked list. - pub(crate) unsafe fn delete(&self, guard: &Guard) { - self.next.fetch_or(1, Release, guard); - } -} - -impl> List { - /// Returns a new, empty linked list. - pub(crate) fn new() -> Self { - Self { - head: Atomic::null(), - _marker: PhantomData, - } - } - - /// Inserts `entry` into the head of the list. - /// - /// # Safety - /// - /// You should guarantee that: - /// - /// - `container` is not null - /// - `container` is immovable, e.g. inside an `Owned` - /// - the same `Entry` is not inserted more than once - /// - the inserted object will be removed before the list is dropped - pub(crate) unsafe fn insert<'g>(&'g self, container: Shared<'g, T>, guard: &'g Guard) { - // Insert right after head, i.e. at the beginning of the list. - let to = &self.head; - // Get the intrusively stored Entry of the new element to insert. - let entry: &Entry = C::entry_of(container.deref()); - // Make a Shared ptr to that Entry. - let entry_ptr = Shared::from(entry as *const _); - // Read the current successor of where we want to insert. - let mut next = to.load(Relaxed, guard); - - loop { - // Set the Entry of the to-be-inserted element to point to the previous successor of - // `to`. - entry.next.store(next, Relaxed); - match to.compare_exchange_weak(next, entry_ptr, Release, Relaxed, guard) { - Ok(_) => break, - // We lost the race or weak CAS failed spuriously. Update the successor and try - // again. - Err(err) => next = err.current, - } - } - } - - /// Returns an iterator over all objects. - /// - /// # Caveat - /// - /// Every object that is inserted at the moment this function is called and persists at least - /// until the end of iteration will be returned. Since this iterator traverses a lock-free - /// linked list that may be concurrently modified, some additional caveats apply: - /// - /// 1. If a new object is inserted during iteration, it may or may not be returned. - /// 2. If an object is deleted during iteration, it may or may not be returned. - /// 3. The iteration may be aborted when it lost in a race condition. In this case, the winning - /// thread will continue to iterate over the same list. - pub(crate) fn iter<'g>(&'g self, guard: &'g Guard) -> Iter<'g, T, C> { - Iter { - guard, - pred: &self.head, - curr: self.head.load(Acquire, guard), - head: &self.head, - _marker: PhantomData, - } - } -} - -impl> Drop for List { - fn drop(&mut self) { - unsafe { - let guard = unprotected(); - let mut curr = self.head.load(Relaxed, guard); - while let Some(c) = curr.as_ref() { - let succ = c.next.load(Relaxed, guard); - // Verify that all elements have been removed from the list. - assert_eq!(succ.tag(), 1); - - C::finalize(curr.deref(), guard); - curr = succ; - } - } - } -} - -impl<'g, T: 'g, C: IsElement> Iterator for Iter<'g, T, C> { - type Item = Result<&'g T, IterError>; - - fn next(&mut self) -> Option { - while let Some(c) = unsafe { self.curr.as_ref() } { - let succ = c.next.load(Acquire, self.guard); - - if succ.tag() == 1 { - // This entry was removed. Try unlinking it from the list. - let succ = succ.with_tag(0); - - // The tag should always be zero, because removing a node after a logically deleted - // node leaves the list in an invalid state. - debug_assert!(self.curr.tag() == 0); - - // Try to unlink `curr` from the list, and get the new value of `self.pred`. - let succ = match self - .pred - .compare_exchange(self.curr, succ, Acquire, Acquire, self.guard) - { - Ok(_) => { - // We succeeded in unlinking `curr`, so we have to schedule - // deallocation. Deferred drop is okay, because `list.delete()` can only be - // called if `T: 'static`. - unsafe { - C::finalize(self.curr.deref(), self.guard); - } - - // `succ` is the new value of `self.pred`. - succ - } - Err(e) => { - // `e.current` is the current value of `self.pred`. - e.current - } - }; - - // If the predecessor node is already marked as deleted, we need to restart from - // `head`. - if succ.tag() != 0 { - self.pred = self.head; - self.curr = self.head.load(Acquire, self.guard); - - return Some(Err(IterError::Stalled)); - } - - // Move over the removed by only advancing `curr`, not `pred`. - self.curr = succ; - continue; - } - - // Move one step forward. - self.pred = &c.next; - self.curr = succ; - - return Some(Ok(unsafe { C::element_of(c) })); - } - - // We reached the end of the list. - None - } -} - -#[cfg(all(test, not(crossbeam_loom)))] -mod tests { - use super::*; - use crate::{Collector, Owned}; - use crossbeam_utils::thread; - use std::sync::Barrier; - - impl IsElement for Entry { - fn entry_of(entry: &Entry) -> &Entry { - entry - } - - unsafe fn element_of(entry: &Entry) -> &Entry { - entry - } - - unsafe fn finalize(entry: &Entry, guard: &Guard) { - guard.defer_destroy(Shared::from(Self::element_of(entry) as *const _)); - } - } - - /// Checks whether the list retains inserted elements - /// and returns them in the correct order. - #[test] - fn insert() { - let collector = Collector::new(); - let handle = collector.register(); - let guard = handle.pin(); - - let l: List = List::new(); - - let e1 = Owned::new(Entry::default()).into_shared(&guard); - let e2 = Owned::new(Entry::default()).into_shared(&guard); - let e3 = Owned::new(Entry::default()).into_shared(&guard); - - unsafe { - l.insert(e1, &guard); - l.insert(e2, &guard); - l.insert(e3, &guard); - } - - let mut iter = l.iter(&guard); - let maybe_e3 = iter.next(); - assert!(maybe_e3.is_some()); - assert!(maybe_e3.unwrap().unwrap() as *const Entry == e3.as_raw()); - let maybe_e2 = iter.next(); - assert!(maybe_e2.is_some()); - assert!(maybe_e2.unwrap().unwrap() as *const Entry == e2.as_raw()); - let maybe_e1 = iter.next(); - assert!(maybe_e1.is_some()); - assert!(maybe_e1.unwrap().unwrap() as *const Entry == e1.as_raw()); - assert!(iter.next().is_none()); - - unsafe { - e1.as_ref().unwrap().delete(&guard); - e2.as_ref().unwrap().delete(&guard); - e3.as_ref().unwrap().delete(&guard); - } - } - - /// Checks whether elements can be removed from the list and whether - /// the correct elements are removed. - #[test] - fn delete() { - let collector = Collector::new(); - let handle = collector.register(); - let guard = handle.pin(); - - let l: List = List::new(); - - let e1 = Owned::new(Entry::default()).into_shared(&guard); - let e2 = Owned::new(Entry::default()).into_shared(&guard); - let e3 = Owned::new(Entry::default()).into_shared(&guard); - unsafe { - l.insert(e1, &guard); - l.insert(e2, &guard); - l.insert(e3, &guard); - e2.as_ref().unwrap().delete(&guard); - } - - let mut iter = l.iter(&guard); - let maybe_e3 = iter.next(); - assert!(maybe_e3.is_some()); - assert!(maybe_e3.unwrap().unwrap() as *const Entry == e3.as_raw()); - let maybe_e1 = iter.next(); - assert!(maybe_e1.is_some()); - assert!(maybe_e1.unwrap().unwrap() as *const Entry == e1.as_raw()); - assert!(iter.next().is_none()); - - unsafe { - e1.as_ref().unwrap().delete(&guard); - e3.as_ref().unwrap().delete(&guard); - } - - let mut iter = l.iter(&guard); - assert!(iter.next().is_none()); - } - - const THREADS: usize = 8; - const ITERS: usize = 512; - - /// Contends the list on insert and delete operations to make sure they can run concurrently. - #[test] - fn insert_delete_multi() { - let collector = Collector::new(); - - let l: List = List::new(); - let b = Barrier::new(THREADS); - - thread::scope(|s| { - for _ in 0..THREADS { - s.spawn(|_| { - b.wait(); - - let handle = collector.register(); - let guard: Guard = handle.pin(); - let mut v = Vec::with_capacity(ITERS); - - for _ in 0..ITERS { - let e = Owned::new(Entry::default()).into_shared(&guard); - v.push(e); - unsafe { - l.insert(e, &guard); - } - } - - for e in v { - unsafe { - e.as_ref().unwrap().delete(&guard); - } - } - }); - } - }) - .unwrap(); - - let handle = collector.register(); - let guard = handle.pin(); - - let mut iter = l.iter(&guard); - assert!(iter.next().is_none()); - } - - /// Contends the list on iteration to make sure that it can be iterated over concurrently. - #[test] - fn iter_multi() { - let collector = Collector::new(); - - let l: List = List::new(); - let b = Barrier::new(THREADS); - - thread::scope(|s| { - for _ in 0..THREADS { - s.spawn(|_| { - b.wait(); - - let handle = collector.register(); - let guard: Guard = handle.pin(); - let mut v = Vec::with_capacity(ITERS); - - for _ in 0..ITERS { - let e = Owned::new(Entry::default()).into_shared(&guard); - v.push(e); - unsafe { - l.insert(e, &guard); - } - } - - let mut iter = l.iter(&guard); - for _ in 0..ITERS { - assert!(iter.next().is_some()); - } - - for e in v { - unsafe { - e.as_ref().unwrap().delete(&guard); - } - } - }); - } - }) - .unwrap(); - - let handle = collector.register(); - let guard = handle.pin(); - - let mut iter = l.iter(&guard); - assert!(iter.next().is_none()); - } -} diff --git a/crossbeam-epoch/src/sync/mod.rs b/crossbeam-epoch/src/sync/mod.rs index 08981be25..75398e42a 100644 --- a/crossbeam-epoch/src/sync/mod.rs +++ b/crossbeam-epoch/src/sync/mod.rs @@ -1,7 +1,8 @@ //! Synchronization primitives. -pub(crate) mod list; #[cfg(feature = "std")] #[cfg(not(crossbeam_loom))] pub(crate) mod once_lock; -pub(crate) mod queue; + +pub(crate) mod pile; +pub(crate) mod refcount; diff --git a/crossbeam-epoch/src/sync/pile.rs b/crossbeam-epoch/src/sync/pile.rs new file mode 100644 index 000000000..f1aeb5fd1 --- /dev/null +++ b/crossbeam-epoch/src/sync/pile.rs @@ -0,0 +1,185 @@ +use crate::primitive::cell::UnsafeCell; +use core::fmt; +use core::mem::MaybeUninit; +use core::sync::atomic::Ordering; + +use crossbeam_utils::CachePadded; + +use crate::deferred::Deferred; +use crate::primitive::sync::atomic::{AtomicPtr, AtomicUsize}; + +enum PushResult { + Done, + ShouldRealloc, +} + +/// An array of [Deferred] +struct Segment { + len: CachePadded, + // Work around lack of UnsafeCell::raw_get (1.56) + #[cfg(crossbeam_loom)] + deferreds: Box<[UnsafeCell]>, + #[cfg(not(crossbeam_loom))] + deferreds: Box<[MaybeUninit>]>, +} + +// For some reason using Owned<> for this gives stacked borrows errors +#[cfg_attr(crossbeam_loom, allow(dead_code))] +fn uninit_boxed_slice(len: usize) -> Box<[MaybeUninit]> { + unsafe { + let ptr = if len == 0 { + core::mem::align_of::() as *mut MaybeUninit + } else { + alloc::alloc::alloc(alloc::alloc::Layout::array::(len).unwrap()) as _ + }; + Box::from_raw(core::slice::from_raw_parts_mut(ptr, len)) + } +} + +impl Segment { + #[cfg(crossbeam_loom)] + fn new(capacity: usize) -> Self { + let deferreds = (0..capacity) + .map(|_| UnsafeCell::new(Deferred::NO_OP)) + .collect(); + Self { + len: CachePadded::new(AtomicUsize::new(0)), + deferreds, + } + } + #[cfg(not(crossbeam_loom))] + fn new(capacity: usize) -> Self { + Self { + len: CachePadded::new(AtomicUsize::new(0)), + deferreds: uninit_boxed_slice(capacity), + } + } + + fn capacity(&self) -> usize { + self.deferreds.len() + } + + /// # Safety: + /// This must not be called concurrently with [call()]. + unsafe fn try_push(&self, deferred: &mut Vec) -> PushResult { + let slot = self.len.fetch_add(deferred.len(), Ordering::Relaxed); + if slot >= self.capacity() { + return PushResult::Done; + } + let end = slot + deferred.len(); + let not_pushed = end.saturating_sub(self.capacity()); + for (slot, deferred) in self.deferreds[slot..] + .iter() + .zip(deferred.drain(not_pushed..)) + { + // Work around lack of UnsafeCell::raw_get (1.56) + #[cfg(crossbeam_loom)] + slot.with_mut(|slot| slot.write(deferred)); + #[cfg(not(crossbeam_loom))] + core::ptr::write(slot.as_ptr() as *mut _, deferred); + } + if end >= self.capacity() { + PushResult::ShouldRealloc + } else { + PushResult::Done + } + } + + /// # Safety: + /// This must not be called concurrently with [try_push()] or [call()]. + unsafe fn call(&self) { + let end = self.capacity().min(self.len.load(Ordering::Relaxed)); + for deferred in &self.deferreds[..end] { + #[cfg(crossbeam_loom)] + deferred.with(|ptr| ptr.read().call()); + #[cfg(not(crossbeam_loom))] + deferred.assume_init_read().with(|ptr| ptr.read().call()) + } + self.len.store(0, Ordering::Relaxed); + } +} + +impl Drop for Segment { + fn drop(&mut self) { + unsafe { self.call() } + } +} + +/// A stack of garbage. +pub(crate) struct Pile { + /// Stashed objects. + current: AtomicPtr, +} + +impl Pile { + /// # Safety: + /// This must not be called concurrently with [call()]. + pub(crate) unsafe fn try_push(&self, deferred: &mut Vec) { + let segment = self.current.load(Ordering::Acquire); + if let PushResult::ShouldRealloc = (*segment).try_push(deferred) { + let next = Box::into_raw(Box::new(Segment::new((*segment).capacity() * 2))); + // Synchronize initialization of the Segment + self.current.store(next, Ordering::Release); + deferred.push(Deferred::new(move || drop(Box::from_raw(segment)))); + // We're potentially holding a lot of garbage now, make an attempt to get rid of it sooner + self.try_push(deferred) + } + } + + /// # Safety: + /// This must not be called concurrently with [try_push()] or [call()]. + pub(crate) unsafe fn call(&self) { + (*self.current.load(Ordering::Relaxed)).call() + } +} + +impl Default for Pile { + fn default() -> Self { + Pile { + current: AtomicPtr::new(Box::into_raw(Box::new(Segment::new(64)))), + } + } +} + +impl Drop for Pile { + fn drop(&mut self) { + unsafe { + // Ordering is taken care of because `Global` is behind an `Arc` + drop(Box::from_raw(self.current.load(Ordering::Relaxed))); + } + } +} + +impl fmt::Debug for Pile { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Pile").finish_non_exhaustive() + } +} + +#[cfg(all(test, not(crossbeam_loom)))] +mod tests { + use super::*; + + #[test] + fn check_bag() { + static FLAG: AtomicUsize = AtomicUsize::new(0); + fn incr() { + FLAG.fetch_add(1, Ordering::Relaxed); + } + + let bag = Pile::default(); + let mut buffer1 = (0..3).map(|_| Deferred::new(incr)).collect(); + let mut buffer2 = (0..18).map(|_| Deferred::new(incr)).collect(); + + unsafe { bag.try_push(&mut buffer1) }; + assert_eq!(buffer1.len(), 0); + assert_eq!(FLAG.load(Ordering::Relaxed), 0); + unsafe { bag.call() }; + assert_eq!(FLAG.load(Ordering::Relaxed), 3); + unsafe { bag.try_push(&mut buffer2) }; + assert_eq!(buffer2.len(), 0); + assert_eq!(FLAG.load(Ordering::Relaxed), 3); + drop(bag); + assert_eq!(FLAG.load(Ordering::Relaxed), 21); + } +} diff --git a/crossbeam-epoch/src/sync/queue.rs b/crossbeam-epoch/src/sync/queue.rs deleted file mode 100644 index 950043881..000000000 --- a/crossbeam-epoch/src/sync/queue.rs +++ /dev/null @@ -1,469 +0,0 @@ -//! Michael-Scott lock-free queue. -//! -//! Usable with any number of producers and consumers. -//! -//! Michael and Scott. Simple, Fast, and Practical Non-Blocking and Blocking Concurrent Queue -//! Algorithms. PODC 1996. -//! -//! Simon Doherty, Lindsay Groves, Victor Luchangco, and Mark Moir. 2004b. Formal Verification of a -//! Practical Lock-Free Queue Algorithm. - -use core::mem::MaybeUninit; -use core::sync::atomic::Ordering::{Acquire, Relaxed, Release}; - -use crossbeam_utils::CachePadded; - -use crate::{unprotected, Atomic, Guard, Owned, Shared}; - -// The representation here is a singly-linked list, with a sentinel node at the front. In general -// the `tail` pointer may lag behind the actual tail. Non-sentinel nodes are either all `Data` or -// all `Blocked` (requests for data from blocked threads). -#[derive(Debug)] -pub(crate) struct Queue { - head: CachePadded>>, - tail: CachePadded>>, -} - -struct Node { - /// The slot in which a value of type `T` can be stored. - /// - /// The type of `data` is `MaybeUninit` because a `Node` doesn't always contain a `T`. - /// For example, the sentinel node in a queue never contains a value: its slot is always empty. - /// Other nodes start their life with a push operation and contain a value until it gets popped - /// out. After that such empty nodes get added to the collector for destruction. - data: MaybeUninit, - - next: Atomic>, -} - -// Any particular `T` should never be accessed concurrently, so no need for `Sync`. -unsafe impl Sync for Queue {} -unsafe impl Send for Queue {} - -impl Queue { - /// Create a new, empty queue. - pub(crate) fn new() -> Queue { - let q = Queue { - head: CachePadded::new(Atomic::null()), - tail: CachePadded::new(Atomic::null()), - }; - let sentinel = Owned::new(Node { - data: MaybeUninit::uninit(), - next: Atomic::null(), - }); - unsafe { - let guard = unprotected(); - let sentinel = sentinel.into_shared(guard); - q.head.store(sentinel, Relaxed); - q.tail.store(sentinel, Relaxed); - q - } - } - - /// Attempts to atomically place `n` into the `next` pointer of `onto`, and returns `true` on - /// success. The queue's `tail` pointer may be updated. - #[inline(always)] - fn push_internal( - &self, - onto: Shared<'_, Node>, - new: Shared<'_, Node>, - guard: &Guard, - ) -> bool { - // is `onto` the actual tail? - let o = unsafe { onto.deref() }; - let next = o.next.load(Acquire, guard); - if unsafe { next.as_ref().is_some() } { - // if not, try to "help" by moving the tail pointer forward - let _ = self - .tail - .compare_exchange(onto, next, Release, Relaxed, guard); - false - } else { - // looks like the actual tail; attempt to link in `n` - let result = o - .next - .compare_exchange(Shared::null(), new, Release, Relaxed, guard) - .is_ok(); - if result { - // try to move the tail pointer forward - let _ = self - .tail - .compare_exchange(onto, new, Release, Relaxed, guard); - } - result - } - } - - /// Adds `t` to the back of the queue, possibly waking up threads blocked on `pop`. - pub(crate) fn push(&self, t: T, guard: &Guard) { - let new = Owned::new(Node { - data: MaybeUninit::new(t), - next: Atomic::null(), - }); - let new = Owned::into_shared(new, guard); - - loop { - // We push onto the tail, so we'll start optimistically by looking there first. - let tail = self.tail.load(Acquire, guard); - - // Attempt to push onto the `tail` snapshot; fails if `tail.next` has changed. - if self.push_internal(tail, new, guard) { - break; - } - } - } - - /// Attempts to pop a data node. `Ok(None)` if queue is empty; `Err(())` if lost race to pop. - #[inline(always)] - fn pop_internal(&self, guard: &Guard) -> Result, ()> { - let head = self.head.load(Acquire, guard); - let h = unsafe { head.deref() }; - let next = h.next.load(Acquire, guard); - match unsafe { next.as_ref() } { - Some(n) => unsafe { - self.head - .compare_exchange(head, next, Release, Relaxed, guard) - .map(|_| { - let tail = self.tail.load(Relaxed, guard); - // Advance the tail so that we don't retire a pointer to a reachable node. - if head == tail { - let _ = self - .tail - .compare_exchange(tail, next, Release, Relaxed, guard); - } - guard.defer_destroy(head); - // TODO: Replace with MaybeUninit::read when api is stable - Some(n.data.as_ptr().read()) - }) - .map_err(|_| ()) - }, - None => Ok(None), - } - } - - /// Attempts to pop a data node, if the data satisfies the given condition. `Ok(None)` if queue - /// is empty or the data does not satisfy the condition; `Err(())` if lost race to pop. - #[inline(always)] - fn pop_if_internal(&self, condition: F, guard: &Guard) -> Result, ()> - where - T: Sync, - F: Fn(&T) -> bool, - { - let head = self.head.load(Acquire, guard); - let h = unsafe { head.deref() }; - let next = h.next.load(Acquire, guard); - match unsafe { next.as_ref() } { - Some(n) if condition(unsafe { &*n.data.as_ptr() }) => unsafe { - self.head - .compare_exchange(head, next, Release, Relaxed, guard) - .map(|_| { - let tail = self.tail.load(Relaxed, guard); - // Advance the tail so that we don't retire a pointer to a reachable node. - if head == tail { - let _ = self - .tail - .compare_exchange(tail, next, Release, Relaxed, guard); - } - guard.defer_destroy(head); - Some(n.data.as_ptr().read()) - }) - .map_err(|_| ()) - }, - None | Some(_) => Ok(None), - } - } - - /// Attempts to dequeue from the front. - /// - /// Returns `None` if the queue is observed to be empty. - pub(crate) fn try_pop(&self, guard: &Guard) -> Option { - loop { - if let Ok(head) = self.pop_internal(guard) { - return head; - } - } - } - - /// Attempts to dequeue from the front, if the item satisfies the given condition. - /// - /// Returns `None` if the queue is observed to be empty, or the head does not satisfy the given - /// condition. - pub(crate) fn try_pop_if(&self, condition: F, guard: &Guard) -> Option - where - T: Sync, - F: Fn(&T) -> bool, - { - loop { - if let Ok(head) = self.pop_if_internal(&condition, guard) { - return head; - } - } - } -} - -impl Drop for Queue { - fn drop(&mut self) { - unsafe { - let guard = unprotected(); - - while self.try_pop(guard).is_some() {} - - // Destroy the remaining sentinel node. - let sentinel = self.head.load(Relaxed, guard); - drop(sentinel.into_owned()); - } - } -} - -#[cfg(all(test, not(crossbeam_loom)))] -mod test { - use super::*; - use crate::pin; - use crossbeam_utils::thread; - - struct Queue { - queue: super::Queue, - } - - impl Queue { - pub(crate) fn new() -> Queue { - Queue { - queue: super::Queue::new(), - } - } - - pub(crate) fn push(&self, t: T) { - let guard = &pin(); - self.queue.push(t, guard); - } - - pub(crate) fn is_empty(&self) -> bool { - let guard = &pin(); - let head = self.queue.head.load(Acquire, guard); - let h = unsafe { head.deref() }; - h.next.load(Acquire, guard).is_null() - } - - pub(crate) fn try_pop(&self) -> Option { - let guard = &pin(); - self.queue.try_pop(guard) - } - - pub(crate) fn pop(&self) -> T { - loop { - match self.try_pop() { - None => continue, - Some(t) => return t, - } - } - } - } - - #[cfg(miri)] - const CONC_COUNT: i64 = 1000; - #[cfg(not(miri))] - const CONC_COUNT: i64 = 1000000; - - #[test] - fn push_try_pop_1() { - let q: Queue = Queue::new(); - assert!(q.is_empty()); - q.push(37); - assert!(!q.is_empty()); - assert_eq!(q.try_pop(), Some(37)); - assert!(q.is_empty()); - } - - #[test] - fn push_try_pop_2() { - let q: Queue = Queue::new(); - assert!(q.is_empty()); - q.push(37); - q.push(48); - assert_eq!(q.try_pop(), Some(37)); - assert!(!q.is_empty()); - assert_eq!(q.try_pop(), Some(48)); - assert!(q.is_empty()); - } - - #[test] - fn push_try_pop_many_seq() { - let q: Queue = Queue::new(); - assert!(q.is_empty()); - for i in 0..200 { - q.push(i) - } - assert!(!q.is_empty()); - for i in 0..200 { - assert_eq!(q.try_pop(), Some(i)); - } - assert!(q.is_empty()); - } - - #[test] - fn push_pop_1() { - let q: Queue = Queue::new(); - assert!(q.is_empty()); - q.push(37); - assert!(!q.is_empty()); - assert_eq!(q.pop(), 37); - assert!(q.is_empty()); - } - - #[test] - fn push_pop_2() { - let q: Queue = Queue::new(); - q.push(37); - q.push(48); - assert_eq!(q.pop(), 37); - assert_eq!(q.pop(), 48); - } - - #[test] - fn push_pop_many_seq() { - let q: Queue = Queue::new(); - assert!(q.is_empty()); - for i in 0..200 { - q.push(i) - } - assert!(!q.is_empty()); - for i in 0..200 { - assert_eq!(q.pop(), i); - } - assert!(q.is_empty()); - } - - #[test] - fn push_try_pop_many_spsc() { - let q: Queue = Queue::new(); - assert!(q.is_empty()); - - thread::scope(|scope| { - scope.spawn(|_| { - let mut next = 0; - - while next < CONC_COUNT { - if let Some(elem) = q.try_pop() { - assert_eq!(elem, next); - next += 1; - } - } - }); - - for i in 0..CONC_COUNT { - q.push(i) - } - }) - .unwrap(); - } - - #[test] - fn push_try_pop_many_spmc() { - fn recv(_t: i32, q: &Queue) { - let mut cur = -1; - for _i in 0..CONC_COUNT { - if let Some(elem) = q.try_pop() { - assert!(elem > cur); - cur = elem; - - if cur == CONC_COUNT - 1 { - break; - } - } - } - } - - let q: Queue = Queue::new(); - assert!(q.is_empty()); - thread::scope(|scope| { - for i in 0..3 { - let q = &q; - scope.spawn(move |_| recv(i, q)); - } - - scope.spawn(|_| { - for i in 0..CONC_COUNT { - q.push(i); - } - }); - }) - .unwrap(); - } - - #[test] - fn push_try_pop_many_mpmc() { - enum LR { - Left(i64), - Right(i64), - } - - let q: Queue = Queue::new(); - assert!(q.is_empty()); - - thread::scope(|scope| { - for _t in 0..2 { - scope.spawn(|_| { - for i in CONC_COUNT - 1..CONC_COUNT { - q.push(LR::Left(i)) - } - }); - scope.spawn(|_| { - for i in CONC_COUNT - 1..CONC_COUNT { - q.push(LR::Right(i)) - } - }); - scope.spawn(|_| { - let mut vl = vec![]; - let mut vr = vec![]; - for _i in 0..CONC_COUNT { - match q.try_pop() { - Some(LR::Left(x)) => vl.push(x), - Some(LR::Right(x)) => vr.push(x), - _ => {} - } - } - - let mut vl2 = vl.clone(); - let mut vr2 = vr.clone(); - vl2.sort_unstable(); - vr2.sort_unstable(); - - assert_eq!(vl, vl2); - assert_eq!(vr, vr2); - }); - } - }) - .unwrap(); - } - - #[test] - fn push_pop_many_spsc() { - let q: Queue = Queue::new(); - - thread::scope(|scope| { - scope.spawn(|_| { - let mut next = 0; - while next < CONC_COUNT { - assert_eq!(q.pop(), next); - next += 1; - } - }); - - for i in 0..CONC_COUNT { - q.push(i) - } - }) - .unwrap(); - assert!(q.is_empty()); - } - - #[test] - fn is_empty_dont_pop() { - let q: Queue = Queue::new(); - q.push(20); - q.push(20); - assert!(!q.is_empty()); - assert!(!q.is_empty()); - assert!(q.try_pop().is_some()); - } -} diff --git a/crossbeam-epoch/src/sync/refcount.rs b/crossbeam-epoch/src/sync/refcount.rs new file mode 100644 index 000000000..fe39cc077 --- /dev/null +++ b/crossbeam-epoch/src/sync/refcount.rs @@ -0,0 +1,54 @@ +use core::sync::atomic::Ordering; + +use crate::primitive::sync::atomic::AtomicUsize; +use crossbeam_utils::CachePadded; + +const STRIPES: usize = 16; +const HIGH_BIT: usize = !(usize::MAX >> 1); +const MAX_FAILED_RLOCKS: usize = HIGH_BIT + (HIGH_BIT >> 1); + +#[derive(Default, Debug)] +/// Divided into a number of separate atomics to reduce read contention. +/// Uses the almost the same algorithm as atomic_refcell. +pub(crate) struct RefCount { + counts: [CachePadded; STRIPES], +} + +impl RefCount { + pub(crate) fn try_read(&self, hint: usize) -> bool { + let state = self.counts[hint % STRIPES].fetch_add(1, Ordering::Acquire); + if state < HIGH_BIT { + return true; + } + if state > MAX_FAILED_RLOCKS { + panic!("Too many failed pins"); + } + return false; + } + pub(crate) fn done_read(&self, hint: usize) { + self.counts[hint % STRIPES].fetch_sub(1, Ordering::Release); + } + + pub(crate) fn try_write(&self) -> bool { + if self.counts[0].load(Ordering::Relaxed) != 0 { + return false; + } + for count in &self.counts[1..] { + match count.load(Ordering::Acquire) { + HIGH_BIT => {} + 0 if count + .compare_exchange(0, HIGH_BIT, Ordering::Acquire, Ordering::Relaxed) + .is_ok() => {} + _ => return false, + } + } + return self.counts[0] + .compare_exchange(0, HIGH_BIT, Ordering::Acquire, Ordering::Relaxed) + .is_ok(); + } + pub(crate) fn done_write(&self) { + for count in &self.counts { + count.store(0, Ordering::Release) + } + } +} diff --git a/crossbeam-epoch/tests/loom.rs b/crossbeam-epoch/tests/loom.rs index 4e56acdbc..2aa4469fe 100644 --- a/crossbeam-epoch/tests/loom.rs +++ b/crossbeam-epoch/tests/loom.rs @@ -5,6 +5,7 @@ use loom_crate as loom; use epoch::*; use epoch::{Atomic, Owned}; +use loom::cell::Cell; use loom::sync::atomic::Ordering::{self, Acquire, Relaxed, Release}; use loom::sync::Arc; use loom::thread::spawn; @@ -45,6 +46,81 @@ fn it_works() { }) } +#[test] +fn simple() { + loom::model(|| { + println!("----"); + let collector = Collector::new(); + let item: Atomic> = Atomic::from(Owned::new(Cell::new(7))); + let item = Arc::new(item); + let item2 = item.clone(); + let collector2 = collector.clone(); + + let jh = loom::thread::spawn(move || { + // Fetch and modify the item + let guard = collector2.register().pin(); + let item = item2.load(Ordering::Relaxed, &guard); + if let Some(cell) = unsafe { item.as_ref() } { + println!("1: Item still there"); + cell.set(8); + } + }); + + // Unlink and "destroy" the item + let mut guard = collector.register().pin(); + let cell = item + .swap(Shared::null(), Ordering::Relaxed, &guard) + .as_raw(); + println!("0: Item removed"); + unsafe { guard.defer_unchecked(move || (*cell).set(9)) }; + // Advance the epoch three times so the thread can happen after the deferred function runs + guard.flush(); + guard.repin(); + guard.flush(); + guard.repin(); + guard.flush(); + + jh.join().unwrap(); + }) +} + +#[test] +fn simple2() { + loom::model(|| { + println!("----"); + let collector = Collector::new(); + let item: Atomic> = Atomic::from(Owned::new(Cell::new(7))); + let item = Arc::new(item); + let item2 = item.clone(); + let collector2 = collector.clone(); + + let jh = loom::thread::spawn(move || { + // Unlink and "destroy" the item + let guard = collector2.register().pin(); + let cell = item + .swap(Shared::null(), Ordering::Relaxed, &guard) + .as_raw(); + println!("1: Item removed"); + unsafe { guard.defer_unchecked(move || (*cell).set(9)) }; + }); + + // Fetch and modify the item + let mut guard = collector.register().pin(); + let item = item2.load(Ordering::Relaxed, &guard); + if let Some(cell) = unsafe { item.as_ref() } { + println!("0: Item still there"); + cell.set(8); + } + guard.flush(); + guard.repin(); + guard.flush(); + guard.repin(); + guard.flush(); + + jh.join().unwrap(); + }) +} + #[test] fn treiber_stack() { /// Treiber's lock-free stack.