summaryrefslogtreecommitdiff
path: root/rust/kernel/sync
diff options
context:
space:
mode:
Diffstat (limited to 'rust/kernel/sync')
-rw-r--r--rust/kernel/sync/arc.rs63
-rw-r--r--rust/kernel/sync/aref.rs17
-rw-r--r--rust/kernel/sync/atomic.rs551
-rw-r--r--rust/kernel/sync/atomic/internal.rs265
-rw-r--r--rust/kernel/sync/atomic/ordering.rs104
-rw-r--r--rust/kernel/sync/atomic/predefine.rs169
-rw-r--r--rust/kernel/sync/barrier.rs61
-rw-r--r--rust/kernel/sync/refcount.rs113
8 files changed, 1302 insertions, 41 deletions
diff --git a/rust/kernel/sync/arc.rs b/rust/kernel/sync/arc.rs
index 63a66761d0c7..289f77abf415 100644
--- a/rust/kernel/sync/arc.rs
+++ b/rust/kernel/sync/arc.rs
@@ -8,7 +8,7 @@
//! threads.
//!
//! It is different from the standard library's [`Arc`] in a few ways:
-//! 1. It is backed by the kernel's `refcount_t` type.
+//! 1. It is backed by the kernel's [`Refcount`] type.
//! 2. It does not support weak references, which allows it to be half the size.
//! 3. It saturates the reference count instead of aborting when it goes over a threshold.
//! 4. It does not provide a `get_mut` method, so the ref counted object is pinned.
@@ -18,16 +18,16 @@
use crate::{
alloc::{AllocError, Flags, KBox},
- bindings,
ffi::c_void,
+ fmt,
init::InPlaceInit,
+ sync::Refcount,
try_init,
- types::{ForeignOwnable, Opaque},
+ types::ForeignOwnable,
};
use core::{
alloc::Layout,
borrow::{Borrow, BorrowMut},
- fmt,
marker::PhantomData,
mem::{ManuallyDrop, MaybeUninit},
ops::{Deref, DerefMut},
@@ -145,7 +145,7 @@ pub struct Arc<T: ?Sized> {
#[pin_data]
#[repr(C)]
struct ArcInner<T: ?Sized> {
- refcount: Opaque<bindings::refcount_t>,
+ refcount: Refcount,
data: T,
}
@@ -157,7 +157,7 @@ impl<T: ?Sized> ArcInner<T> {
/// `ptr` must have been returned by a previous call to [`Arc::into_raw`], and the `Arc` must
/// not yet have been destroyed.
unsafe fn container_of(ptr: *const T) -> NonNull<ArcInner<T>> {
- let refcount_layout = Layout::new::<bindings::refcount_t>();
+ let refcount_layout = Layout::new::<Refcount>();
// SAFETY: The caller guarantees that the pointer is valid.
let val_layout = Layout::for_value(unsafe { &*ptr });
// SAFETY: We're computing the layout of a real struct that existed when compiling this
@@ -229,8 +229,7 @@ impl<T> Arc<T> {
pub fn new(contents: T, flags: Flags) -> Result<Self, AllocError> {
// INVARIANT: The refcount is initialised to a non-zero value.
let value = ArcInner {
- // SAFETY: There are no safety requirements for this FFI call.
- refcount: Opaque::new(unsafe { bindings::REFCOUNT_INIT(1) }),
+ refcount: Refcount::new(1),
data: contents,
};
@@ -321,7 +320,7 @@ impl<T: ?Sized> Arc<T> {
/// use kernel::sync::{Arc, UniqueArc};
///
/// let arc = Arc::new(42, GFP_KERNEL)?;
- /// let unique_arc = arc.into_unique_or_drop();
+ /// let unique_arc = Arc::into_unique_or_drop(arc);
///
/// // The above conversion should succeed since refcount of `arc` is 1.
/// assert!(unique_arc.is_some());
@@ -337,35 +336,30 @@ impl<T: ?Sized> Arc<T> {
/// let arc = Arc::new(42, GFP_KERNEL)?;
/// let another = arc.clone();
///
- /// let unique_arc = arc.into_unique_or_drop();
+ /// let unique_arc = Arc::into_unique_or_drop(arc);
///
/// // The above conversion should fail since refcount of `arc` is >1.
/// assert!(unique_arc.is_none());
///
/// # Ok::<(), Error>(())
/// ```
- pub fn into_unique_or_drop(self) -> Option<Pin<UniqueArc<T>>> {
+ pub fn into_unique_or_drop(this: Self) -> Option<Pin<UniqueArc<T>>> {
// We will manually manage the refcount in this method, so we disable the destructor.
- let me = ManuallyDrop::new(self);
+ let this = ManuallyDrop::new(this);
// SAFETY: We own a refcount, so the pointer is still valid.
- let refcount = unsafe { me.ptr.as_ref() }.refcount.get();
+ let refcount = unsafe { &this.ptr.as_ref().refcount };
// If the refcount reaches a non-zero value, then we have destroyed this `Arc` and will
// return without further touching the `Arc`. If the refcount reaches zero, then there are
// no other arcs, and we can create a `UniqueArc`.
- //
- // SAFETY: We own a refcount, so the pointer is not dangling.
- let is_zero = unsafe { bindings::refcount_dec_and_test(refcount) };
- if is_zero {
- // SAFETY: We have exclusive access to the arc, so we can perform unsynchronized
- // accesses to the refcount.
- unsafe { core::ptr::write(refcount, bindings::REFCOUNT_INIT(1)) };
+ if refcount.dec_and_test() {
+ refcount.set(1);
// INVARIANT: We own the only refcount to this arc, so we may create a `UniqueArc`. We
// must pin the `UniqueArc` because the values was previously in an `Arc`, and they pin
// their values.
Some(Pin::from(UniqueArc {
- inner: ManuallyDrop::into_inner(me),
+ inner: ManuallyDrop::into_inner(this),
}))
} else {
None
@@ -373,10 +367,10 @@ impl<T: ?Sized> Arc<T> {
}
}
-// SAFETY: The pointer returned by `into_foreign` comes from a well aligned
-// pointer to `ArcInner<T>`.
+// SAFETY: The pointer returned by `into_foreign` was originally allocated as an
+// `KBox<ArcInner<T>>`, so that type is what determines the alignment.
unsafe impl<T: 'static> ForeignOwnable for Arc<T> {
- const FOREIGN_ALIGN: usize = core::mem::align_of::<ArcInner<T>>();
+ const FOREIGN_ALIGN: usize = <KBox<ArcInner<T>> as ForeignOwnable>::FOREIGN_ALIGN;
type Borrowed<'a> = ArcBorrow<'a, T>;
type BorrowedMut<'a> = Self::Borrowed<'a>;
@@ -456,14 +450,10 @@ impl<T: ?Sized> Borrow<T> for Arc<T> {
impl<T: ?Sized> Clone for Arc<T> {
fn clone(&self) -> Self {
- // SAFETY: By the type invariant, there is necessarily a reference to the object, so it is
- // safe to dereference it.
- let refcount = unsafe { self.ptr.as_ref() }.refcount.get();
-
- // INVARIANT: C `refcount_inc` saturates the refcount, so it cannot overflow to zero.
+ // INVARIANT: `Refcount` saturates the refcount, so it cannot overflow to zero.
// SAFETY: By the type invariant, there is necessarily a reference to the object, so it is
// safe to increment the refcount.
- unsafe { bindings::refcount_inc(refcount) };
+ unsafe { self.ptr.as_ref() }.refcount.inc();
// SAFETY: We just incremented the refcount. This increment is now owned by the new `Arc`.
unsafe { Self::from_inner(self.ptr) }
@@ -472,16 +462,10 @@ impl<T: ?Sized> Clone for Arc<T> {
impl<T: ?Sized> Drop for Arc<T> {
fn drop(&mut self) {
- // SAFETY: By the type invariant, there is necessarily a reference to the object. We cannot
- // touch `refcount` after it's decremented to a non-zero value because another thread/CPU
- // may concurrently decrement it to zero and free it. It is ok to have a raw pointer to
- // freed/invalid memory as long as it is never dereferenced.
- let refcount = unsafe { self.ptr.as_ref() }.refcount.get();
-
// INVARIANT: If the refcount reaches zero, there are no other instances of `Arc`, and
// this instance is being dropped, so the broken invariant is not observable.
- // SAFETY: Also by the type invariant, we are allowed to decrement the refcount.
- let is_zero = unsafe { bindings::refcount_dec_and_test(refcount) };
+ // SAFETY: By the type invariant, there is necessarily a reference to the object.
+ let is_zero = unsafe { self.ptr.as_ref() }.refcount.dec_and_test();
if is_zero {
// The count reached zero, we must free the memory.
//
@@ -775,8 +759,7 @@ impl<T> UniqueArc<T> {
// INVARIANT: The refcount is initialised to a non-zero value.
let inner = KBox::try_init::<AllocError>(
try_init!(ArcInner {
- // SAFETY: There are no safety requirements for this FFI call.
- refcount: Opaque::new(unsafe { bindings::REFCOUNT_INIT(1) }),
+ refcount: Refcount::new(1),
data <- pin_init::uninit::<T, AllocError>(),
}? AllocError),
flags,
diff --git a/rust/kernel/sync/aref.rs b/rust/kernel/sync/aref.rs
index dbd77bb68617..0d24a0432015 100644
--- a/rust/kernel/sync/aref.rs
+++ b/rust/kernel/sync/aref.rs
@@ -1,6 +1,21 @@
// SPDX-License-Identifier: GPL-2.0
//! Internal reference counting support.
+//!
+//! Many C types already have their own reference counting mechanism (e.g. by storing a
+//! `refcount_t`). This module provides support for directly using their internal reference count
+//! from Rust; instead of making users have to use an additional Rust-reference count in the form of
+//! [`Arc`].
+//!
+//! The smart pointer [`ARef<T>`] acts similarly to [`Arc<T>`] in that it holds a refcount on the
+//! underlying object, but this refcount is internal to the object. It essentially is a Rust
+//! implementation of the `get_` and `put_` pattern used in C for reference counting.
+//!
+//! To make use of [`ARef<MyType>`], `MyType` needs to implement [`AlwaysRefCounted`]. It is a trait
+//! for accessing the internal reference count of an object of the `MyType` type.
+//!
+//! [`Arc`]: crate::sync::Arc
+//! [`Arc<T>`]: crate::sync::Arc
use core::{marker::PhantomData, mem::ManuallyDrop, ops::Deref, ptr::NonNull};
@@ -97,7 +112,7 @@ impl<T: AlwaysRefCounted> ARef<T> {
///
/// ```
/// use core::ptr::NonNull;
- /// use kernel::types::{ARef, AlwaysRefCounted};
+ /// use kernel::sync::aref::{ARef, AlwaysRefCounted};
///
/// struct Empty {}
///
diff --git a/rust/kernel/sync/atomic.rs b/rust/kernel/sync/atomic.rs
new file mode 100644
index 000000000000..016a6bcaf080
--- /dev/null
+++ b/rust/kernel/sync/atomic.rs
@@ -0,0 +1,551 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Atomic primitives.
+//!
+//! These primitives have the same semantics as their C counterparts: and the precise definitions of
+//! semantics can be found at [`LKMM`]. Note that Linux Kernel Memory (Consistency) Model is the
+//! only model for Rust code in kernel, and Rust's own atomics should be avoided.
+//!
+//! # Data races
+//!
+//! [`LKMM`] atomics have different rules regarding data races:
+//!
+//! - A normal write from C side is treated as an atomic write if
+//! CONFIG_KCSAN_ASSUME_PLAIN_WRITES_ATOMIC=y.
+//! - Mixed-size atomic accesses don't cause data races.
+//!
+//! [`LKMM`]: srctree/tools/memory-model/
+
+mod internal;
+pub mod ordering;
+mod predefine;
+
+pub use internal::AtomicImpl;
+pub use ordering::{Acquire, Full, Relaxed, Release};
+
+use crate::build_error;
+use internal::{AtomicArithmeticOps, AtomicBasicOps, AtomicExchangeOps, AtomicRepr};
+use ordering::OrderingType;
+
+/// A memory location which can be safely modified from multiple execution contexts.
+///
+/// This has the same size, alignment and bit validity as the underlying type `T`. And it disables
+/// niche optimization for the same reason as [`UnsafeCell`].
+///
+/// The atomic operations are implemented in a way that is fully compatible with the [Linux Kernel
+/// Memory (Consistency) Model][LKMM], hence they should be modeled as the corresponding
+/// [`LKMM`][LKMM] atomic primitives. With the help of [`Atomic::from_ptr()`] and
+/// [`Atomic::as_ptr()`], this provides a way to interact with [C-side atomic operations]
+/// (including those without the `atomic` prefix, e.g. `READ_ONCE()`, `WRITE_ONCE()`,
+/// `smp_load_acquire()` and `smp_store_release()`).
+///
+/// # Invariants
+///
+/// `self.0` is a valid `T`.
+///
+/// [`UnsafeCell`]: core::cell::UnsafeCell
+/// [LKMM]: srctree/tools/memory-model/
+/// [C-side atomic operations]: srctree/Documentation/atomic_t.txt
+#[repr(transparent)]
+pub struct Atomic<T: AtomicType>(AtomicRepr<T::Repr>);
+
+// SAFETY: `Atomic<T>` is safe to share among execution contexts because all accesses are atomic.
+unsafe impl<T: AtomicType> Sync for Atomic<T> {}
+
+/// Types that support basic atomic operations.
+///
+/// # Round-trip transmutability
+///
+/// `T` is round-trip transmutable to `U` if and only if both of these properties hold:
+///
+/// - Any valid bit pattern for `T` is also a valid bit pattern for `U`.
+/// - Transmuting (e.g. using [`transmute()`]) a value of type `T` to `U` and then to `T` again
+/// yields a value that is in all aspects equivalent to the original value.
+///
+/// # Safety
+///
+/// - [`Self`] must have the same size and alignment as [`Self::Repr`].
+/// - [`Self`] must be [round-trip transmutable] to [`Self::Repr`].
+///
+/// Note that this is more relaxed than requiring the bi-directional transmutability (i.e.
+/// [`transmute()`] is always sound between `U` and `T`) because of the support for atomic
+/// variables over unit-only enums, see [Examples].
+///
+/// # Limitations
+///
+/// Because C primitives are used to implement the atomic operations, and a C function requires a
+/// valid object of a type to operate on (i.e. no `MaybeUninit<_>`), hence at the Rust <-> C
+/// surface, only types with all the bits initialized can be passed. As a result, types like `(u8,
+/// u16)` (padding bytes are uninitialized) are currently not supported.
+///
+/// # Examples
+///
+/// A unit-only enum that implements [`AtomicType`]:
+///
+/// ```
+/// use kernel::sync::atomic::{AtomicType, Atomic, Relaxed};
+///
+/// #[derive(Clone, Copy, PartialEq, Eq)]
+/// #[repr(i32)]
+/// enum State {
+/// Uninit = 0,
+/// Working = 1,
+/// Done = 2,
+/// };
+///
+/// // SAFETY: `State` and `i32` has the same size and alignment, and it's round-trip
+/// // transmutable to `i32`.
+/// unsafe impl AtomicType for State {
+/// type Repr = i32;
+/// }
+///
+/// let s = Atomic::new(State::Uninit);
+///
+/// assert_eq!(State::Uninit, s.load(Relaxed));
+/// ```
+/// [`transmute()`]: core::mem::transmute
+/// [round-trip transmutable]: AtomicType#round-trip-transmutability
+/// [Examples]: AtomicType#examples
+pub unsafe trait AtomicType: Sized + Send + Copy {
+ /// The backing atomic implementation type.
+ type Repr: AtomicImpl;
+}
+
+/// Types that support atomic add operations.
+///
+/// # Safety
+///
+// TODO: Properly defines `wrapping_add` in the following comment.
+/// `wrapping_add` any value of type `Self::Repr::Delta` obtained by [`Self::rhs_into_delta()`] to
+/// any value of type `Self::Repr` obtained through transmuting a value of type `Self` to must
+/// yield a value with a bit pattern also valid for `Self`.
+pub unsafe trait AtomicAdd<Rhs = Self>: AtomicType {
+ /// Converts `Rhs` into the `Delta` type of the atomic implementation.
+ fn rhs_into_delta(rhs: Rhs) -> <Self::Repr as AtomicImpl>::Delta;
+}
+
+#[inline(always)]
+const fn into_repr<T: AtomicType>(v: T) -> T::Repr {
+ // SAFETY: Per the safety requirement of `AtomicType`, `T` is round-trip transmutable to
+ // `T::Repr`, therefore the transmute operation is sound.
+ unsafe { core::mem::transmute_copy(&v) }
+}
+
+/// # Safety
+///
+/// `r` must be a valid bit pattern of `T`.
+#[inline(always)]
+const unsafe fn from_repr<T: AtomicType>(r: T::Repr) -> T {
+ // SAFETY: Per the safety requirement of the function, the transmute operation is sound.
+ unsafe { core::mem::transmute_copy(&r) }
+}
+
+impl<T: AtomicType> Atomic<T> {
+ /// Creates a new atomic `T`.
+ pub const fn new(v: T) -> Self {
+ // INVARIANT: Per the safety requirement of `AtomicType`, `into_repr(v)` is a valid `T`.
+ Self(AtomicRepr::new(into_repr(v)))
+ }
+
+ /// Creates a reference to an atomic `T` from a pointer of `T`.
+ ///
+ /// This usually is used when communicating with C side or manipulating a C struct, see
+ /// examples below.
+ ///
+ /// # Safety
+ ///
+ /// - `ptr` is aligned to `align_of::<T>()`.
+ /// - `ptr` is valid for reads and writes for `'a`.
+ /// - For the duration of `'a`, other accesses to `*ptr` must not cause data races (defined
+ /// by [`LKMM`]) against atomic operations on the returned reference. Note that if all other
+ /// accesses are atomic, then this safety requirement is trivially fulfilled.
+ ///
+ /// [`LKMM`]: srctree/tools/memory-model
+ ///
+ /// # Examples
+ ///
+ /// Using [`Atomic::from_ptr()`] combined with [`Atomic::load()`] or [`Atomic::store()`] can
+ /// achieve the same functionality as `READ_ONCE()`/`smp_load_acquire()` or
+ /// `WRITE_ONCE()`/`smp_store_release()` in C side:
+ ///
+ /// ```
+ /// # use kernel::types::Opaque;
+ /// use kernel::sync::atomic::{Atomic, Relaxed, Release};
+ ///
+ /// // Assume there is a C struct `foo`.
+ /// mod cbindings {
+ /// #[repr(C)]
+ /// pub(crate) struct foo {
+ /// pub(crate) a: i32,
+ /// pub(crate) b: i32
+ /// }
+ /// }
+ ///
+ /// let tmp = Opaque::new(cbindings::foo { a: 1, b: 2 });
+ ///
+ /// // struct foo *foo_ptr = ..;
+ /// let foo_ptr = tmp.get();
+ ///
+ /// // SAFETY: `foo_ptr` is valid, and `.a` is in bounds.
+ /// let foo_a_ptr = unsafe { &raw mut (*foo_ptr).a };
+ ///
+ /// // a = READ_ONCE(foo_ptr->a);
+ /// //
+ /// // SAFETY: `foo_a_ptr` is valid for read, and all other accesses on it is atomic, so no
+ /// // data race.
+ /// let a = unsafe { Atomic::from_ptr(foo_a_ptr) }.load(Relaxed);
+ /// # assert_eq!(a, 1);
+ ///
+ /// // smp_store_release(&foo_ptr->a, 2);
+ /// //
+ /// // SAFETY: `foo_a_ptr` is valid for writes, and all other accesses on it is atomic, so
+ /// // no data race.
+ /// unsafe { Atomic::from_ptr(foo_a_ptr) }.store(2, Release);
+ /// ```
+ pub unsafe fn from_ptr<'a>(ptr: *mut T) -> &'a Self
+ where
+ T: Sync,
+ {
+ // CAST: `T` and `Atomic<T>` have the same size, alignment and bit validity.
+ // SAFETY: Per function safety requirement, `ptr` is a valid pointer and the object will
+ // live long enough. It's safe to return a `&Atomic<T>` because function safety requirement
+ // guarantees other accesses won't cause data races.
+ unsafe { &*ptr.cast::<Self>() }
+ }
+
+ /// Returns a pointer to the underlying atomic `T`.
+ ///
+ /// Note that use of the return pointer must not cause data races defined by [`LKMM`].
+ ///
+ /// # Guarantees
+ ///
+ /// The returned pointer is valid and properly aligned (i.e. aligned to [`align_of::<T>()`]).
+ ///
+ /// [`LKMM`]: srctree/tools/memory-model
+ /// [`align_of::<T>()`]: core::mem::align_of
+ pub const fn as_ptr(&self) -> *mut T {
+ // GUARANTEE: Per the function guarantee of `AtomicRepr::as_ptr()`, the `self.0.as_ptr()`
+ // must be a valid and properly aligned pointer for `T::Repr`, and per the safety guarantee
+ // of `AtomicType`, it's a valid and properly aligned pointer of `T`.
+ self.0.as_ptr().cast()
+ }
+
+ /// Returns a mutable reference to the underlying atomic `T`.
+ ///
+ /// This is safe because the mutable reference of the atomic `T` guarantees exclusive access.
+ pub fn get_mut(&mut self) -> &mut T {
+ // CAST: `T` and `T::Repr` has the same size and alignment per the safety requirement of
+ // `AtomicType`, and per the type invariants `self.0` is a valid `T`, therefore the casting
+ // result is a valid pointer of `T`.
+ // SAFETY: The pointer is valid per the CAST comment above, and the mutable reference
+ // guarantees exclusive access.
+ unsafe { &mut *self.0.as_ptr().cast() }
+ }
+}
+
+impl<T: AtomicType> Atomic<T>
+where
+ T::Repr: AtomicBasicOps,
+{
+ /// Loads the value from the atomic `T`.
+ ///
+ /// # Examples
+ ///
+ /// ```
+ /// use kernel::sync::atomic::{Atomic, Relaxed};
+ ///
+ /// let x = Atomic::new(42i32);
+ ///
+ /// assert_eq!(42, x.load(Relaxed));
+ ///
+ /// let x = Atomic::new(42i64);
+ ///
+ /// assert_eq!(42, x.load(Relaxed));
+ /// ```
+ #[doc(alias("atomic_read", "atomic64_read"))]
+ #[inline(always)]
+ pub fn load<Ordering: ordering::AcquireOrRelaxed>(&self, _: Ordering) -> T {
+ let v = {
+ match Ordering::TYPE {
+ OrderingType::Relaxed => T::Repr::atomic_read(&self.0),
+ OrderingType::Acquire => T::Repr::atomic_read_acquire(&self.0),
+ _ => build_error!("Wrong ordering"),
+ }
+ };
+
+ // SAFETY: `v` comes from reading `self.0`, which is a valid `T` per the type invariants.
+ unsafe { from_repr(v) }
+ }
+
+ /// Stores a value to the atomic `T`.
+ ///
+ /// # Examples
+ ///
+ /// ```
+ /// use kernel::sync::atomic::{Atomic, Relaxed};
+ ///
+ /// let x = Atomic::new(42i32);
+ ///
+ /// assert_eq!(42, x.load(Relaxed));
+ ///
+ /// x.store(43, Relaxed);
+ ///
+ /// assert_eq!(43, x.load(Relaxed));
+ /// ```
+ #[doc(alias("atomic_set", "atomic64_set"))]
+ #[inline(always)]
+ pub fn store<Ordering: ordering::ReleaseOrRelaxed>(&self, v: T, _: Ordering) {
+ let v = into_repr(v);
+
+ // INVARIANT: `v` is a valid `T`, and is stored to `self.0` by `atomic_set*()`.
+ match Ordering::TYPE {
+ OrderingType::Relaxed => T::Repr::atomic_set(&self.0, v),
+ OrderingType::Release => T::Repr::atomic_set_release(&self.0, v),
+ _ => build_error!("Wrong ordering"),
+ }
+ }
+}
+
+impl<T: AtomicType> Atomic<T>
+where
+ T::Repr: AtomicExchangeOps,
+{
+ /// Atomic exchange.
+ ///
+ /// Atomically updates `*self` to `v` and returns the old value of `*self`.
+ ///
+ /// # Examples
+ ///
+ /// ```
+ /// use kernel::sync::atomic::{Atomic, Acquire, Relaxed};
+ ///
+ /// let x = Atomic::new(42);
+ ///
+ /// assert_eq!(42, x.xchg(52, Acquire));
+ /// assert_eq!(52, x.load(Relaxed));
+ /// ```
+ #[doc(alias("atomic_xchg", "atomic64_xchg", "swap"))]
+ #[inline(always)]
+ pub fn xchg<Ordering: ordering::Ordering>(&self, v: T, _: Ordering) -> T {
+ let v = into_repr(v);
+
+ // INVARIANT: `self.0` is a valid `T` after `atomic_xchg*()` because `v` is transmutable to
+ // `T`.
+ let ret = {
+ match Ordering::TYPE {
+ OrderingType::Full => T::Repr::atomic_xchg(&self.0, v),
+ OrderingType::Acquire => T::Repr::atomic_xchg_acquire(&self.0, v),
+ OrderingType::Release => T::Repr::atomic_xchg_release(&self.0, v),
+ OrderingType::Relaxed => T::Repr::atomic_xchg_relaxed(&self.0, v),
+ }
+ };
+
+ // SAFETY: `ret` comes from reading `*self`, which is a valid `T` per type invariants.
+ unsafe { from_repr(ret) }
+ }
+
+ /// Atomic compare and exchange.
+ ///
+ /// If `*self` == `old`, atomically updates `*self` to `new`. Otherwise, `*self` is not
+ /// modified.
+ ///
+ /// Compare: The comparison is done via the byte level comparison between `*self` and `old`.
+ ///
+ /// Ordering: When succeeds, provides the corresponding ordering as the `Ordering` type
+ /// parameter indicates, and a failed one doesn't provide any ordering, the load part of a
+ /// failed cmpxchg is a [`Relaxed`] load.
+ ///
+ /// Returns `Ok(value)` if cmpxchg succeeds, and `value` is guaranteed to be equal to `old`,
+ /// otherwise returns `Err(value)`, and `value` is the current value of `*self`.
+ ///
+ /// # Examples
+ ///
+ /// ```
+ /// use kernel::sync::atomic::{Atomic, Full, Relaxed};
+ ///
+ /// let x = Atomic::new(42);
+ ///
+ /// // Checks whether cmpxchg succeeded.
+ /// let success = x.cmpxchg(52, 64, Relaxed).is_ok();
+ /// # assert!(!success);
+ ///
+ /// // Checks whether cmpxchg failed.
+ /// let failure = x.cmpxchg(52, 64, Relaxed).is_err();
+ /// # assert!(failure);
+ ///
+ /// // Uses the old value if failed, probably re-try cmpxchg.
+ /// match x.cmpxchg(52, 64, Relaxed) {
+ /// Ok(_) => { },
+ /// Err(old) => {
+ /// // do something with `old`.
+ /// # assert_eq!(old, 42);
+ /// }
+ /// }
+ ///
+ /// // Uses the latest value regardlessly, same as atomic_cmpxchg() in C.
+ /// let latest = x.cmpxchg(42, 64, Full).unwrap_or_else(|old| old);
+ /// # assert_eq!(42, latest);
+ /// assert_eq!(64, x.load(Relaxed));
+ /// ```
+ ///
+ /// [`Relaxed`]: ordering::Relaxed
+ #[doc(alias(
+ "atomic_cmpxchg",
+ "atomic64_cmpxchg",
+ "atomic_try_cmpxchg",
+ "atomic64_try_cmpxchg",
+ "compare_exchange"
+ ))]
+ #[inline(always)]
+ pub fn cmpxchg<Ordering: ordering::Ordering>(
+ &self,
+ mut old: T,
+ new: T,
+ o: Ordering,
+ ) -> Result<T, T> {
+ // Note on code generation:
+ //
+ // try_cmpxchg() is used to implement cmpxchg(), and if the helper functions are inlined,
+ // the compiler is able to figure out that branch is not needed if the users don't care
+ // about whether the operation succeeds or not. One exception is on x86, due to commit
+ // 44fe84459faf ("locking/atomic: Fix atomic_try_cmpxchg() semantics"), the
+ // atomic_try_cmpxchg() on x86 has a branch even if the caller doesn't care about the
+ // success of cmpxchg and only wants to use the old value. For example, for code like:
+ //
+ // let latest = x.cmpxchg(42, 64, Full).unwrap_or_else(|old| old);
+ //
+ // It will still generate code:
+ //
+ // movl $0x40, %ecx
+ // movl $0x34, %eax
+ // lock
+ // cmpxchgl %ecx, 0x4(%rsp)
+ // jne 1f
+ // 2:
+ // ...
+ // 1: movl %eax, %ecx
+ // jmp 2b
+ //
+ // This might be "fixed" by introducing a try_cmpxchg_exclusive() that knows the "*old"
+ // location in the C function is always safe to write.
+ if self.try_cmpxchg(&mut old, new, o) {
+ Ok(old)
+ } else {
+ Err(old)
+ }
+ }
+
+ /// Atomic compare and exchange and returns whether the operation succeeds.
+ ///
+ /// If `*self` == `old`, atomically updates `*self` to `new`. Otherwise, `*self` is not
+ /// modified, `*old` is updated to the current value of `*self`.
+ ///
+ /// "Compare" and "Ordering" part are the same as [`Atomic::cmpxchg()`].
+ ///
+ /// Returns `true` means the cmpxchg succeeds otherwise returns `false`.
+ #[inline(always)]
+ fn try_cmpxchg<Ordering: ordering::Ordering>(&self, old: &mut T, new: T, _: Ordering) -> bool {
+ let mut tmp = into_repr(*old);
+ let new = into_repr(new);
+
+ // INVARIANT: `self.0` is a valid `T` after `atomic_try_cmpxchg*()` because `new` is
+ // transmutable to `T`.
+ let ret = {
+ match Ordering::TYPE {
+ OrderingType::Full => T::Repr::atomic_try_cmpxchg(&self.0, &mut tmp, new),
+ OrderingType::Acquire => {
+ T::Repr::atomic_try_cmpxchg_acquire(&self.0, &mut tmp, new)
+ }
+ OrderingType::Release => {
+ T::Repr::atomic_try_cmpxchg_release(&self.0, &mut tmp, new)
+ }
+ OrderingType::Relaxed => {
+ T::Repr::atomic_try_cmpxchg_relaxed(&self.0, &mut tmp, new)
+ }
+ }
+ };
+
+ // SAFETY: `tmp` comes from reading `*self`, which is a valid `T` per type invariants.
+ *old = unsafe { from_repr(tmp) };
+
+ ret
+ }
+}
+
+impl<T: AtomicType> Atomic<T>
+where
+ T::Repr: AtomicArithmeticOps,
+{
+ /// Atomic add.
+ ///
+ /// Atomically updates `*self` to `(*self).wrapping_add(v)`.
+ ///
+ /// # Examples
+ ///
+ /// ```
+ /// use kernel::sync::atomic::{Atomic, Relaxed};
+ ///
+ /// let x = Atomic::new(42);
+ ///
+ /// assert_eq!(42, x.load(Relaxed));
+ ///
+ /// x.add(12, Relaxed);
+ ///
+ /// assert_eq!(54, x.load(Relaxed));
+ /// ```
+ #[inline(always)]
+ pub fn add<Rhs>(&self, v: Rhs, _: ordering::Relaxed)
+ where
+ T: AtomicAdd<Rhs>,
+ {
+ let v = T::rhs_into_delta(v);
+
+ // INVARIANT: `self.0` is a valid `T` after `atomic_add()` due to safety requirement of
+ // `AtomicAdd`.
+ T::Repr::atomic_add(&self.0, v);
+ }
+
+ /// Atomic fetch and add.
+ ///
+ /// Atomically updates `*self` to `(*self).wrapping_add(v)`, and returns the value of `*self`
+ /// before the update.
+ ///
+ /// # Examples
+ ///
+ /// ```
+ /// use kernel::sync::atomic::{Atomic, Acquire, Full, Relaxed};
+ ///
+ /// let x = Atomic::new(42);
+ ///
+ /// assert_eq!(42, x.load(Relaxed));
+ ///
+ /// assert_eq!(54, { x.fetch_add(12, Acquire); x.load(Relaxed) });
+ ///
+ /// let x = Atomic::new(42);
+ ///
+ /// assert_eq!(42, x.load(Relaxed));
+ ///
+ /// assert_eq!(54, { x.fetch_add(12, Full); x.load(Relaxed) } );
+ /// ```
+ #[inline(always)]
+ pub fn fetch_add<Rhs, Ordering: ordering::Ordering>(&self, v: Rhs, _: Ordering) -> T
+ where
+ T: AtomicAdd<Rhs>,
+ {
+ let v = T::rhs_into_delta(v);
+
+ // INVARIANT: `self.0` is a valid `T` after `atomic_fetch_add*()` due to safety requirement
+ // of `AtomicAdd`.
+ let ret = {
+ match Ordering::TYPE {
+ OrderingType::Full => T::Repr::atomic_fetch_add(&self.0, v),
+ OrderingType::Acquire => T::Repr::atomic_fetch_add_acquire(&self.0, v),
+ OrderingType::Release => T::Repr::atomic_fetch_add_release(&self.0, v),
+ OrderingType::Relaxed => T::Repr::atomic_fetch_add_relaxed(&self.0, v),
+ }
+ };
+
+ // SAFETY: `ret` comes from reading `self.0`, which is a valid `T` per type invariants.
+ unsafe { from_repr(ret) }
+ }
+}
diff --git a/rust/kernel/sync/atomic/internal.rs b/rust/kernel/sync/atomic/internal.rs
new file mode 100644
index 000000000000..6fdd8e59f45b
--- /dev/null
+++ b/rust/kernel/sync/atomic/internal.rs
@@ -0,0 +1,265 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Atomic internal implementations.
+//!
+//! Provides 1:1 mapping to the C atomic operations.
+
+use crate::bindings;
+use crate::macros::paste;
+use core::cell::UnsafeCell;
+
+mod private {
+ /// Sealed trait marker to disable customized impls on atomic implementation traits.
+ pub trait Sealed {}
+}
+
+// `i32` and `i64` are only supported atomic implementations.
+impl private::Sealed for i32 {}
+impl private::Sealed for i64 {}
+
+/// A marker trait for types that implement atomic operations with C side primitives.
+///
+/// This trait is sealed, and only types that have directly mapping to the C side atomics should
+/// impl this:
+///
+/// - `i32` maps to `atomic_t`.
+/// - `i64` maps to `atomic64_t`.
+pub trait AtomicImpl: Sized + Send + Copy + private::Sealed {
+ /// The type of the delta in arithmetic or logical operations.
+ ///
+ /// For example, in `atomic_add(ptr, v)`, it's the type of `v`. Usually it's the same type of
+ /// [`Self`], but it may be different for the atomic pointer type.
+ type Delta;
+}
+
+// `atomic_t` implements atomic operations on `i32`.
+impl AtomicImpl for i32 {
+ type Delta = Self;
+}
+
+// `atomic64_t` implements atomic operations on `i64`.
+impl AtomicImpl for i64 {
+ type Delta = Self;
+}
+
+/// Atomic representation.
+#[repr(transparent)]
+pub struct AtomicRepr<T: AtomicImpl>(UnsafeCell<T>);
+
+impl<T: AtomicImpl> AtomicRepr<T> {
+ /// Creates a new atomic representation `T`.
+ pub const fn new(v: T) -> Self {
+ Self(UnsafeCell::new(v))
+ }
+
+ /// Returns a pointer to the underlying `T`.
+ ///
+ /// # Guarantees
+ ///
+ /// The returned pointer is valid and properly aligned (i.e. aligned to [`align_of::<T>()`]).
+ pub const fn as_ptr(&self) -> *mut T {
+ // GUARANTEE: `self.0` is an `UnsafeCell<T>`, therefore the pointer returned by `.get()`
+ // must be valid and properly aligned.
+ self.0.get()
+ }
+}
+
+// This macro generates the function signature with given argument list and return type.
+macro_rules! declare_atomic_method {
+ (
+ $(#[doc=$doc:expr])*
+ $func:ident($($arg:ident : $arg_type:ty),*) $(-> $ret:ty)?
+ ) => {
+ paste!(
+ $(#[doc = $doc])*
+ fn [< atomic_ $func >]($($arg: $arg_type,)*) $(-> $ret)?;
+ );
+ };
+ (
+ $(#[doc=$doc:expr])*
+ $func:ident [$variant:ident $($rest:ident)*]($($arg_sig:tt)*) $(-> $ret:ty)?
+ ) => {
+ paste!(
+ declare_atomic_method!(
+ $(#[doc = $doc])*
+ [< $func _ $variant >]($($arg_sig)*) $(-> $ret)?
+ );
+ );
+
+ declare_atomic_method!(
+ $(#[doc = $doc])*
+ $func [$($rest)*]($($arg_sig)*) $(-> $ret)?
+ );
+ };
+ (
+ $(#[doc=$doc:expr])*
+ $func:ident []($($arg_sig:tt)*) $(-> $ret:ty)?
+ ) => {
+ declare_atomic_method!(
+ $(#[doc = $doc])*
+ $func($($arg_sig)*) $(-> $ret)?
+ );
+ }
+}
+
+// This macro generates the function implementation with given argument list and return type, and it
+// will replace "call(...)" expression with "$ctype _ $func" to call the real C function.
+macro_rules! impl_atomic_method {
+ (
+ ($ctype:ident) $func:ident($($arg:ident: $arg_type:ty),*) $(-> $ret:ty)? {
+ $unsafe:tt { call($($c_arg:expr),*) }
+ }
+ ) => {
+ paste!(
+ #[inline(always)]
+ fn [< atomic_ $func >]($($arg: $arg_type,)*) $(-> $ret)? {
+ // TODO: Ideally we want to use the SAFETY comments written at the macro invocation
+ // (e.g. in `declare_and_impl_atomic_methods!()`, however, since SAFETY comments
+ // are just comments, and they are not passed to macros as tokens, therefore we
+ // cannot use them here. One potential improvement is that if we support using
+ // attributes as an alternative for SAFETY comments, then we can use that for macro
+ // generating code.
+ //
+ // SAFETY: specified on macro invocation.
+ $unsafe { bindings::[< $ctype _ $func >]($($c_arg,)*) }
+ }
+ );
+ };
+ (
+ ($ctype:ident) $func:ident[$variant:ident $($rest:ident)*]($($arg_sig:tt)*) $(-> $ret:ty)? {
+ $unsafe:tt { call($($arg:tt)*) }
+ }
+ ) => {
+ paste!(
+ impl_atomic_method!(
+ ($ctype) [< $func _ $variant >]($($arg_sig)*) $( -> $ret)? {
+ $unsafe { call($($arg)*) }
+ }
+ );
+ );
+ impl_atomic_method!(
+ ($ctype) $func [$($rest)*]($($arg_sig)*) $( -> $ret)? {
+ $unsafe { call($($arg)*) }
+ }
+ );
+ };
+ (
+ ($ctype:ident) $func:ident[]($($arg_sig:tt)*) $( -> $ret:ty)? {
+ $unsafe:tt { call($($arg:tt)*) }
+ }
+ ) => {
+ impl_atomic_method!(
+ ($ctype) $func($($arg_sig)*) $(-> $ret)? {
+ $unsafe { call($($arg)*) }
+ }
+ );
+ }
+}
+
+// Delcares $ops trait with methods and implements the trait for `i32` and `i64`.
+macro_rules! declare_and_impl_atomic_methods {
+ ($(#[$attr:meta])* $pub:vis trait $ops:ident {
+ $(
+ $(#[doc=$doc:expr])*
+ fn $func:ident [$($variant:ident),*]($($arg_sig:tt)*) $( -> $ret:ty)? {
+ $unsafe:tt { bindings::#call($($arg:tt)*) }
+ }
+ )*
+ }) => {
+ $(#[$attr])*
+ $pub trait $ops: AtomicImpl {
+ $(
+ declare_atomic_method!(
+ $(#[doc=$doc])*
+ $func[$($variant)*]($($arg_sig)*) $(-> $ret)?
+ );
+ )*
+ }
+
+ impl $ops for i32 {
+ $(
+ impl_atomic_method!(
+ (atomic) $func[$($variant)*]($($arg_sig)*) $(-> $ret)? {
+ $unsafe { call($($arg)*) }
+ }
+ );
+ )*
+ }
+
+ impl $ops for i64 {
+ $(
+ impl_atomic_method!(
+ (atomic64) $func[$($variant)*]($($arg_sig)*) $(-> $ret)? {
+ $unsafe { call($($arg)*) }
+ }
+ );
+ )*
+ }
+ }
+}
+
+declare_and_impl_atomic_methods!(
+ /// Basic atomic operations
+ pub trait AtomicBasicOps {
+ /// Atomic read (load).
+ fn read[acquire](a: &AtomicRepr<Self>) -> Self {
+ // SAFETY: `a.as_ptr()` is valid and properly aligned.
+ unsafe { bindings::#call(a.as_ptr().cast()) }
+ }
+
+ /// Atomic set (store).
+ fn set[release](a: &AtomicRepr<Self>, v: Self) {
+ // SAFETY: `a.as_ptr()` is valid and properly aligned.
+ unsafe { bindings::#call(a.as_ptr().cast(), v) }
+ }
+ }
+);
+
+declare_and_impl_atomic_methods!(
+ /// Exchange and compare-and-exchange atomic operations
+ pub trait AtomicExchangeOps {
+ /// Atomic exchange.
+ ///
+ /// Atomically updates `*a` to `v` and returns the old value.
+ fn xchg[acquire, release, relaxed](a: &AtomicRepr<Self>, v: Self) -> Self {
+ // SAFETY: `a.as_ptr()` is valid and properly aligned.
+ unsafe { bindings::#call(a.as_ptr().cast(), v) }
+ }
+
+ /// Atomic compare and exchange.
+ ///
+ /// If `*a` == `*old`, atomically updates `*a` to `new`. Otherwise, `*a` is not
+ /// modified, `*old` is updated to the current value of `*a`.
+ ///
+ /// Return `true` if the update of `*a` occurred, `false` otherwise.
+ fn try_cmpxchg[acquire, release, relaxed](
+ a: &AtomicRepr<Self>, old: &mut Self, new: Self
+ ) -> bool {
+ // SAFETY: `a.as_ptr()` is valid and properly aligned. `core::ptr::from_mut(old)`
+ // is valid and properly aligned.
+ unsafe { bindings::#call(a.as_ptr().cast(), core::ptr::from_mut(old), new) }
+ }
+ }
+);
+
+declare_and_impl_atomic_methods!(
+ /// Atomic arithmetic operations
+ pub trait AtomicArithmeticOps {
+ /// Atomic add (wrapping).
+ ///
+ /// Atomically updates `*a` to `(*a).wrapping_add(v)`.
+ fn add[](a: &AtomicRepr<Self>, v: Self::Delta) {
+ // SAFETY: `a.as_ptr()` is valid and properly aligned.
+ unsafe { bindings::#call(v, a.as_ptr().cast()) }
+ }
+
+ /// Atomic fetch and add (wrapping).
+ ///
+ /// Atomically updates `*a` to `(*a).wrapping_add(v)`, and returns the value of `*a`
+ /// before the update.
+ fn fetch_add[acquire, release, relaxed](a: &AtomicRepr<Self>, v: Self::Delta) -> Self {
+ // SAFETY: `a.as_ptr()` is valid and properly aligned.
+ unsafe { bindings::#call(v, a.as_ptr().cast()) }
+ }
+ }
+);
diff --git a/rust/kernel/sync/atomic/ordering.rs b/rust/kernel/sync/atomic/ordering.rs
new file mode 100644
index 000000000000..3f103aa8db99
--- /dev/null
+++ b/rust/kernel/sync/atomic/ordering.rs
@@ -0,0 +1,104 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Memory orderings.
+//!
+//! The semantics of these orderings follows the [`LKMM`] definitions and rules.
+//!
+//! - [`Acquire`] provides ordering between the load part of the annotated operation and all the
+//! following memory accesses, and if there is a store part, the store part has the [`Relaxed`]
+//! ordering.
+//! - [`Release`] provides ordering between all the preceding memory accesses and the store part of
+//! the annotated operation, and if there is a load part, the load part has the [`Relaxed`]
+//! ordering.
+//! - [`Full`] means "fully-ordered", that is:
+//! - It provides ordering between all the preceding memory accesses and the annotated operation.
+//! - It provides ordering between the annotated operation and all the following memory accesses.
+//! - It provides ordering between all the preceding memory accesses and all the following memory
+//! accesses.
+//! - All the orderings are the same strength as a full memory barrier (i.e. `smp_mb()`).
+//! - [`Relaxed`] provides no ordering except the dependency orderings. Dependency orderings are
+//! described in "DEPENDENCY RELATIONS" in [`LKMM`]'s [`explanation`].
+//!
+//! [`LKMM`]: srctree/tools/memory-model/
+//! [`explanation`]: srctree/tools/memory-model/Documentation/explanation.txt
+
+/// The annotation type for relaxed memory ordering, for the description of relaxed memory
+/// ordering, see [module-level documentation].
+///
+/// [module-level documentation]: crate::sync::atomic::ordering
+pub struct Relaxed;
+
+/// The annotation type for acquire memory ordering, for the description of acquire memory
+/// ordering, see [module-level documentation].
+///
+/// [module-level documentation]: crate::sync::atomic::ordering
+pub struct Acquire;
+
+/// The annotation type for release memory ordering, for the description of release memory
+/// ordering, see [module-level documentation].
+///
+/// [module-level documentation]: crate::sync::atomic::ordering
+pub struct Release;
+
+/// The annotation type for fully-ordered memory ordering, for the description fully-ordered memory
+/// ordering, see [module-level documentation].
+///
+/// [module-level documentation]: crate::sync::atomic::ordering
+pub struct Full;
+
+/// Describes the exact memory ordering.
+#[doc(hidden)]
+pub enum OrderingType {
+ /// Relaxed ordering.
+ Relaxed,
+ /// Acquire ordering.
+ Acquire,
+ /// Release ordering.
+ Release,
+ /// Fully-ordered.
+ Full,
+}
+
+mod internal {
+ /// Sealed trait, can be only implemented inside atomic mod.
+ pub trait Sealed {}
+
+ impl Sealed for super::Relaxed {}
+ impl Sealed for super::Acquire {}
+ impl Sealed for super::Release {}
+ impl Sealed for super::Full {}
+}
+
+/// The trait bound for annotating operations that support any ordering.
+pub trait Ordering: internal::Sealed {
+ /// Describes the exact memory ordering.
+ const TYPE: OrderingType;
+}
+
+impl Ordering for Relaxed {
+ const TYPE: OrderingType = OrderingType::Relaxed;
+}
+
+impl Ordering for Acquire {
+ const TYPE: OrderingType = OrderingType::Acquire;
+}
+
+impl Ordering for Release {
+ const TYPE: OrderingType = OrderingType::Release;
+}
+
+impl Ordering for Full {
+ const TYPE: OrderingType = OrderingType::Full;
+}
+
+/// The trait bound for operations that only support acquire or relaxed ordering.
+pub trait AcquireOrRelaxed: Ordering {}
+
+impl AcquireOrRelaxed for Acquire {}
+impl AcquireOrRelaxed for Relaxed {}
+
+/// The trait bound for operations that only support release or relaxed ordering.
+pub trait ReleaseOrRelaxed: Ordering {}
+
+impl ReleaseOrRelaxed for Release {}
+impl ReleaseOrRelaxed for Relaxed {}
diff --git a/rust/kernel/sync/atomic/predefine.rs b/rust/kernel/sync/atomic/predefine.rs
new file mode 100644
index 000000000000..45a17985cda4
--- /dev/null
+++ b/rust/kernel/sync/atomic/predefine.rs
@@ -0,0 +1,169 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Pre-defined atomic types
+
+use crate::static_assert;
+use core::mem::{align_of, size_of};
+
+// SAFETY: `i32` has the same size and alignment with itself, and is round-trip transmutable to
+// itself.
+unsafe impl super::AtomicType for i32 {
+ type Repr = i32;
+}
+
+// SAFETY: The wrapping add result of two `i32`s is a valid `i32`.
+unsafe impl super::AtomicAdd<i32> for i32 {
+ fn rhs_into_delta(rhs: i32) -> i32 {
+ rhs
+ }
+}
+
+// SAFETY: `i64` has the same size and alignment with itself, and is round-trip transmutable to
+// itself.
+unsafe impl super::AtomicType for i64 {
+ type Repr = i64;
+}
+
+// SAFETY: The wrapping add result of two `i64`s is a valid `i64`.
+unsafe impl super::AtomicAdd<i64> for i64 {
+ fn rhs_into_delta(rhs: i64) -> i64 {
+ rhs
+ }
+}
+
+// Defines an internal type that always maps to the integer type which has the same size alignment
+// as `isize` and `usize`, and `isize` and `usize` are always bi-directional transmutable to
+// `isize_atomic_repr`, which also always implements `AtomicImpl`.
+#[allow(non_camel_case_types)]
+#[cfg(not(CONFIG_64BIT))]
+type isize_atomic_repr = i32;
+#[allow(non_camel_case_types)]
+#[cfg(CONFIG_64BIT)]
+type isize_atomic_repr = i64;
+
+// Ensure size and alignment requirements are checked.
+static_assert!(size_of::<isize>() == size_of::<isize_atomic_repr>());
+static_assert!(align_of::<isize>() == align_of::<isize_atomic_repr>());
+static_assert!(size_of::<usize>() == size_of::<isize_atomic_repr>());
+static_assert!(align_of::<usize>() == align_of::<isize_atomic_repr>());
+
+// SAFETY: `isize` has the same size and alignment with `isize_atomic_repr`, and is round-trip
+// transmutable to `isize_atomic_repr`.
+unsafe impl super::AtomicType for isize {
+ type Repr = isize_atomic_repr;
+}
+
+// SAFETY: The wrapping add result of two `isize_atomic_repr`s is a valid `usize`.
+unsafe impl super::AtomicAdd<isize> for isize {
+ fn rhs_into_delta(rhs: isize) -> isize_atomic_repr {
+ rhs as isize_atomic_repr
+ }
+}
+
+// SAFETY: `u32` and `i32` has the same size and alignment, and `u32` is round-trip transmutable to
+// `i32`.
+unsafe impl super::AtomicType for u32 {
+ type Repr = i32;
+}
+
+// SAFETY: The wrapping add result of two `i32`s is a valid `u32`.
+unsafe impl super::AtomicAdd<u32> for u32 {
+ fn rhs_into_delta(rhs: u32) -> i32 {
+ rhs as i32
+ }
+}
+
+// SAFETY: `u64` and `i64` has the same size and alignment, and `u64` is round-trip transmutable to
+// `i64`.
+unsafe impl super::AtomicType for u64 {
+ type Repr = i64;
+}
+
+// SAFETY: The wrapping add result of two `i64`s is a valid `u64`.
+unsafe impl super::AtomicAdd<u64> for u64 {
+ fn rhs_into_delta(rhs: u64) -> i64 {
+ rhs as i64
+ }
+}
+
+// SAFETY: `usize` has the same size and alignment with `isize_atomic_repr`, and is round-trip
+// transmutable to `isize_atomic_repr`.
+unsafe impl super::AtomicType for usize {
+ type Repr = isize_atomic_repr;
+}
+
+// SAFETY: The wrapping add result of two `isize_atomic_repr`s is a valid `usize`.
+unsafe impl super::AtomicAdd<usize> for usize {
+ fn rhs_into_delta(rhs: usize) -> isize_atomic_repr {
+ rhs as isize_atomic_repr
+ }
+}
+
+use crate::macros::kunit_tests;
+
+#[kunit_tests(rust_atomics)]
+mod tests {
+ use super::super::*;
+
+ // Call $fn($val) with each $type of $val.
+ macro_rules! for_each_type {
+ ($val:literal in [$($type:ty),*] $fn:expr) => {
+ $({
+ let v: $type = $val;
+
+ $fn(v);
+ })*
+ }
+ }
+
+ #[test]
+ fn atomic_basic_tests() {
+ for_each_type!(42 in [i32, i64, u32, u64, isize, usize] |v| {
+ let x = Atomic::new(v);
+
+ assert_eq!(v, x.load(Relaxed));
+ });
+ }
+
+ #[test]
+ fn atomic_xchg_tests() {
+ for_each_type!(42 in [i32, i64, u32, u64, isize, usize] |v| {
+ let x = Atomic::new(v);
+
+ let old = v;
+ let new = v + 1;
+
+ assert_eq!(old, x.xchg(new, Full));
+ assert_eq!(new, x.load(Relaxed));
+ });
+ }
+
+ #[test]
+ fn atomic_cmpxchg_tests() {
+ for_each_type!(42 in [i32, i64, u32, u64, isize, usize] |v| {
+ let x = Atomic::new(v);
+
+ let old = v;
+ let new = v + 1;
+
+ assert_eq!(Err(old), x.cmpxchg(new, new, Full));
+ assert_eq!(old, x.load(Relaxed));
+ assert_eq!(Ok(old), x.cmpxchg(old, new, Relaxed));
+ assert_eq!(new, x.load(Relaxed));
+ });
+ }
+
+ #[test]
+ fn atomic_arithmetic_tests() {
+ for_each_type!(42 in [i32, i64, u32, u64, isize, usize] |v| {
+ let x = Atomic::new(v);
+
+ assert_eq!(v, x.fetch_add(12, Full));
+ assert_eq!(v + 12, x.load(Relaxed));
+
+ x.add(13, Relaxed);
+
+ assert_eq!(v + 25, x.load(Relaxed));
+ });
+ }
+}
diff --git a/rust/kernel/sync/barrier.rs b/rust/kernel/sync/barrier.rs
new file mode 100644
index 000000000000..8f2d435fcd94
--- /dev/null
+++ b/rust/kernel/sync/barrier.rs
@@ -0,0 +1,61 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Memory barriers.
+//!
+//! These primitives have the same semantics as their C counterparts: and the precise definitions
+//! of semantics can be found at [`LKMM`].
+//!
+//! [`LKMM`]: srctree/tools/memory-model/
+
+/// A compiler barrier.
+///
+/// A barrier that prevents compiler from reordering memory accesses across the barrier.
+#[inline(always)]
+pub(crate) fn barrier() {
+ // By default, Rust inline asms are treated as being able to access any memory or flags, hence
+ // it suffices as a compiler barrier.
+ //
+ // SAFETY: An empty asm block.
+ unsafe { core::arch::asm!("") };
+}
+
+/// A full memory barrier.
+///
+/// A barrier that prevents compiler and CPU from reordering memory accesses across the barrier.
+#[inline(always)]
+pub fn smp_mb() {
+ if cfg!(CONFIG_SMP) {
+ // SAFETY: `smp_mb()` is safe to call.
+ unsafe { bindings::smp_mb() };
+ } else {
+ barrier();
+ }
+}
+
+/// A write-write memory barrier.
+///
+/// A barrier that prevents compiler and CPU from reordering memory write accesses across the
+/// barrier.
+#[inline(always)]
+pub fn smp_wmb() {
+ if cfg!(CONFIG_SMP) {
+ // SAFETY: `smp_wmb()` is safe to call.
+ unsafe { bindings::smp_wmb() };
+ } else {
+ barrier();
+ }
+}
+
+/// A read-read memory barrier.
+///
+/// A barrier that prevents compiler and CPU from reordering memory read accesses across the
+/// barrier.
+#[inline(always)]
+pub fn smp_rmb() {
+ if cfg!(CONFIG_SMP) {
+ // SAFETY: `smp_rmb()` is safe to call.
+ unsafe { bindings::smp_rmb() };
+ } else {
+ barrier();
+ }
+}
diff --git a/rust/kernel/sync/refcount.rs b/rust/kernel/sync/refcount.rs
new file mode 100644
index 000000000000..19236a5bccde
--- /dev/null
+++ b/rust/kernel/sync/refcount.rs
@@ -0,0 +1,113 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Atomic reference counting.
+//!
+//! C header: [`include/linux/refcount.h`](srctree/include/linux/refcount.h)
+
+use crate::build_assert;
+use crate::sync::atomic::Atomic;
+use crate::types::Opaque;
+
+/// Atomic reference counter.
+///
+/// This type is conceptually an atomic integer, but provides saturation semantics compared to
+/// normal atomic integers. Values in the negative range when viewed as a signed integer are
+/// saturation (bad) values. For details about the saturation semantics, please refer to top of
+/// [`include/linux/refcount.h`](srctree/include/linux/refcount.h).
+///
+/// Wraps the kernel's C `refcount_t`.
+#[repr(transparent)]
+pub struct Refcount(Opaque<bindings::refcount_t>);
+
+impl Refcount {
+ /// Construct a new [`Refcount`] from an initial value.
+ ///
+ /// The initial value should be non-saturated.
+ #[inline]
+ pub fn new(value: i32) -> Self {
+ build_assert!(value >= 0, "initial value saturated");
+ // SAFETY: There are no safety requirements for this FFI call.
+ Self(Opaque::new(unsafe { bindings::REFCOUNT_INIT(value) }))
+ }
+
+ #[inline]
+ fn as_ptr(&self) -> *mut bindings::refcount_t {
+ self.0.get()
+ }
+
+ /// Get the underlying atomic counter that backs the refcount.
+ ///
+ /// NOTE: Usage of this function is discouraged as it can circumvent the protections offered by
+ /// `refcount.h`. If there is no way to achieve the result using APIs in `refcount.h`, then
+ /// this function can be used. Otherwise consider adding a binding for the required API.
+ #[inline]
+ pub fn as_atomic(&self) -> &Atomic<i32> {
+ let ptr = self.0.get().cast();
+ // SAFETY: `refcount_t` is a transparent wrapper of `atomic_t`, which is an atomic 32-bit
+ // integer that is layout-wise compatible with `Atomic<i32>`. All values are valid for
+ // `refcount_t`, despite some of the values being considered saturated and "bad".
+ unsafe { &*ptr }
+ }
+
+ /// Set a refcount's value.
+ #[inline]
+ pub fn set(&self, value: i32) {
+ // SAFETY: `self.as_ptr()` is valid.
+ unsafe { bindings::refcount_set(self.as_ptr(), value) }
+ }
+
+ /// Increment a refcount.
+ ///
+ /// It will saturate if overflows and `WARN`. It will also `WARN` if the refcount is 0, as this
+ /// represents a possible use-after-free condition.
+ ///
+ /// Provides no memory ordering, it is assumed that caller already has a reference on the
+ /// object.
+ #[inline]
+ pub fn inc(&self) {
+ // SAFETY: self is valid.
+ unsafe { bindings::refcount_inc(self.as_ptr()) }
+ }
+
+ /// Decrement a refcount.
+ ///
+ /// It will `WARN` on underflow and fail to decrement when saturated.
+ ///
+ /// Provides release memory ordering, such that prior loads and stores are done
+ /// before.
+ #[inline]
+ pub fn dec(&self) {
+ // SAFETY: `self.as_ptr()` is valid.
+ unsafe { bindings::refcount_dec(self.as_ptr()) }
+ }
+
+ /// Decrement a refcount and test if it is 0.
+ ///
+ /// It will `WARN` on underflow and fail to decrement when saturated.
+ ///
+ /// Provides release memory ordering, such that prior loads and stores are done
+ /// before, and provides an acquire ordering on success such that memory deallocation
+ /// must come after.
+ ///
+ /// Returns true if the resulting refcount is 0, false otherwise.
+ ///
+ /// # Notes
+ ///
+ /// A common pattern of using `Refcount` is to free memory when the reference count reaches
+ /// zero. This means that the reference to `Refcount` could become invalid after calling this
+ /// function. This is fine as long as the reference to `Refcount` is no longer used when this
+ /// function returns `false`. It is not necessary to use raw pointers in this scenario, see
+ /// <https://github.com/rust-lang/rust/issues/55005>.
+ #[inline]
+ #[must_use = "use `dec` instead if you do not need to test if it is 0"]
+ pub fn dec_and_test(&self) -> bool {
+ // SAFETY: `self.as_ptr()` is valid.
+ unsafe { bindings::refcount_dec_and_test(self.as_ptr()) }
+ }
+}
+
+// SAFETY: `refcount_t` is thread-safe.
+unsafe impl Send for Refcount {}
+
+// SAFETY: `refcount_t` is thread-safe.
+unsafe impl Sync for Refcount {}