Refactor Rwlock to take type parameter

This commit is contained in:
jiangjianfeng
2024-11-19 10:35:20 +00:00
committed by Tate, Hongliang Tian
parent ac1a6be05d
commit 495c93c2ad
20 changed files with 205 additions and 363 deletions

View File

@ -13,7 +13,7 @@ use x86_64::registers::rflags::{self, RFlags};
use super::iommu::{alloc_irt_entry, has_interrupt_remapping, IrtEntryHandle};
use crate::{
cpu::CpuId,
sync::{LocalIrqDisabled, Mutex, RwLock, RwLockReadGuard, SpinLock},
sync::{LocalIrqDisabled, Mutex, PreemptDisabled, RwLock, RwLockReadGuard, SpinLock},
trap::TrapFrame,
};
@ -119,7 +119,9 @@ impl IrqLine {
self.irq_num
}
pub fn callback_list(&self) -> RwLockReadGuard<alloc::vec::Vec<CallbackElement>> {
pub fn callback_list(
&self,
) -> RwLockReadGuard<alloc::vec::Vec<CallbackElement>, PreemptDisabled> {
self.callback_list.read()
}

View File

@ -25,7 +25,7 @@ use crate::{
Frame, PageProperty, VmReader, VmWriter, MAX_USERSPACE_VADDR,
},
prelude::*,
sync::{RwLock, RwLockReadGuard},
sync::{PreemptDisabled, RwLock, RwLockReadGuard},
task::{disable_preempt, DisabledPreemptGuard},
Error,
};
@ -283,7 +283,7 @@ impl Cursor<'_> {
pub struct CursorMut<'a, 'b> {
pt_cursor: page_table::CursorMut<'a, UserMode, PageTableEntry, PagingConsts>,
#[allow(dead_code)]
activation_lock: RwLockReadGuard<'b, ()>,
activation_lock: RwLockReadGuard<'b, (), PreemptDisabled>,
// We have a read lock so the CPU set in the flusher is always a superset
// of actual activated CPUs.
flusher: TlbFlusher<DisabledPreemptGuard>,

View File

@ -22,6 +22,8 @@ pub use self::{
ArcRwMutexReadGuard, ArcRwMutexUpgradeableGuard, ArcRwMutexWriteGuard, RwMutex,
RwMutexReadGuard, RwMutexUpgradeableGuard, RwMutexWriteGuard,
},
spin::{ArcSpinLockGuard, LocalIrqDisabled, PreemptDisabled, SpinLock, SpinLockGuard},
spin::{
ArcSpinLockGuard, GuardTransfer, LocalIrqDisabled, PreemptDisabled, SpinLock, SpinLockGuard,
},
wait::{WaitQueue, Waiter, Waker},
};

View File

@ -6,6 +6,7 @@ use alloc::sync::Arc;
use core::{
cell::UnsafeCell,
fmt,
marker::PhantomData,
ops::{Deref, DerefMut},
sync::atomic::{
AtomicUsize,
@ -13,10 +14,7 @@ use core::{
},
};
use crate::{
task::{disable_preempt, DisabledPreemptGuard},
trap::{disable_local, DisabledLocalIrqGuard},
};
use super::{spin::Guardian, GuardTransfer, PreemptDisabled};
/// Spin-based Read-write Lock
///
@ -33,10 +31,6 @@ use crate::{
/// periods of time, and the overhead of context switching is higher than
/// the cost of spinning.
///
/// The lock provides methods to safely acquire locks with interrupts
/// disabled, preventing deadlocks in scenarios where locks are used within
/// interrupt handlers.
///
/// In addition to traditional read and write locks, this implementation
/// provides the upgradeable read lock (`upread lock`). The `upread lock`
/// can be upgraded to write locks atomically, useful in scenarios
@ -59,10 +53,9 @@ use crate::{
/// This lock should not be used in scenarios where lock-holding times are
/// long as it can lead to CPU resource wastage due to spinning.
///
/// # Safety
/// # About Guard
///
/// Use interrupt-disabled version methods when dealing with interrupt-related read-write locks,
/// as nested interrupts may lead to a deadlock if not properly handled.
/// See the comments of [`SpinLock`].
///
/// # Examples
///
@ -99,7 +92,10 @@ use crate::{
/// assert_eq!(*w2, 7);
/// } // write lock is dropped at this point
/// ```
pub struct RwLock<T: ?Sized> {
///
/// [`SpinLock`]: super::SpinLock
pub struct RwLock<T: ?Sized, Guard = PreemptDisabled> {
guard: PhantomData<Guard>,
/// The internal representation of the lock state is as follows:
/// - **Bit 63:** Writer lock.
/// - **Bit 62:** Upgradeable reader lock.
@ -115,152 +111,25 @@ const UPGRADEABLE_READER: usize = 1 << (usize::BITS - 2);
const BEING_UPGRADED: usize = 1 << (usize::BITS - 3);
const MAX_READER: usize = 1 << (usize::BITS - 4);
impl<T> RwLock<T> {
impl<T, G> RwLock<T, G> {
/// Creates a new spin-based read-write lock with an initial value.
pub const fn new(val: T) -> Self {
Self {
val: UnsafeCell::new(val),
guard: PhantomData,
lock: AtomicUsize::new(0),
val: UnsafeCell::new(val),
}
}
}
impl<T: ?Sized> RwLock<T> {
/// Acquires a read lock while disabling the local IRQs and spin-wait
/// until it can be acquired.
///
/// The calling thread will spin-wait until there are no writers or
/// upgrading upreaders present. There is no guarantee for the order
/// in which other readers or writers waiting simultaneously will
/// obtain the lock. Once this lock is acquired, the calling thread
/// will not be interrupted.
pub fn read_irq_disabled(&self) -> RwLockReadGuard<T> {
loop {
if let Some(readguard) = self.try_read_irq_disabled() {
return readguard;
} else {
core::hint::spin_loop();
}
}
}
/// Acquires a write lock while disabling the local IRQs and spin-wait
/// until it can be acquired.
///
/// The calling thread will spin-wait until there are no other writers,
/// upreaders or readers present. There is no guarantee for the order
/// in which other readers or writers waiting simultaneously will
/// obtain the lock. Once this lock is acquired, the calling thread
/// will not be interrupted.
pub fn write_irq_disabled(&self) -> RwLockWriteGuard<T> {
loop {
if let Some(writeguard) = self.try_write_irq_disabled() {
return writeguard;
} else {
core::hint::spin_loop();
}
}
}
/// Acquires an upgradeable reader (upreader) while disabling local IRQs
/// and spin-wait until it can be acquired.
///
/// The calling thread will spin-wait until there are no other writers,
/// or upreaders. There is no guarantee for the order in which other
/// readers or writers waiting simultaneously will obtain the lock. Once
/// this lock is acquired, the calling thread will not be interrupted.
///
/// Upreader will not block new readers until it tries to upgrade. Upreader
/// and reader do not differ before invoking the upgread method. However,
/// only one upreader can exist at any time to avoid deadlock in the
/// upgread method.
pub fn upread_irq_disabled(&self) -> RwLockUpgradeableGuard<T> {
loop {
if let Some(guard) = self.try_upread_irq_disabled() {
return guard;
} else {
core::hint::spin_loop();
}
}
}
/// Attempts to acquire a read lock while disabling local IRQs.
///
/// This function will never spin-wait and will return immediately. When
/// multiple readers or writers attempt to acquire the lock, this method
/// does not guarantee any order. Interrupts will automatically be restored
/// when acquiring fails.
pub fn try_read_irq_disabled(&self) -> Option<RwLockReadGuard<T>> {
let irq_guard = disable_local();
let lock = self.lock.fetch_add(READER, Acquire);
if lock & (WRITER | MAX_READER | BEING_UPGRADED) == 0 {
Some(RwLockReadGuard {
inner: self,
inner_guard: InnerGuard::IrqGuard(irq_guard),
})
} else {
self.lock.fetch_sub(READER, Release);
None
}
}
/// Attempts to acquire a write lock while disabling local IRQs.
///
/// This function will never spin-wait and will return immediately. When
/// multiple readers or writers attempt to acquire the lock, this method
/// does not guarantee any order. Interrupts will automatically be restored
/// when acquiring fails.
pub fn try_write_irq_disabled(&self) -> Option<RwLockWriteGuard<T>> {
let irq_guard = disable_local();
if self
.lock
.compare_exchange(0, WRITER, Acquire, Relaxed)
.is_ok()
{
Some(RwLockWriteGuard {
inner: self,
inner_guard: InnerGuard::IrqGuard(irq_guard),
})
} else {
None
}
}
/// Attempts to acquire a upread lock while disabling local IRQs.
///
/// This function will never spin-wait and will return immediately. When
/// multiple readers or writers attempt to acquire the lock, this method
/// does not guarantee any order. Interrupts will automatically be restored
/// when acquiring fails.
pub fn try_upread_irq_disabled(&self) -> Option<RwLockUpgradeableGuard<T>> {
let irq_guard = disable_local();
let lock = self.lock.fetch_or(UPGRADEABLE_READER, Acquire) & (WRITER | UPGRADEABLE_READER);
if lock == 0 {
return Some(RwLockUpgradeableGuard {
inner: self,
inner_guard: InnerGuard::IrqGuard(irq_guard),
});
} else if lock == WRITER {
self.lock.fetch_sub(UPGRADEABLE_READER, Release);
}
None
}
impl<T: ?Sized, G: Guardian> RwLock<T, G> {
/// Acquires a read lock and spin-wait until it can be acquired.
///
/// The calling thread will spin-wait until there are no writers or
/// upgrading upreaders present. There is no guarantee for the order
/// in which other readers or writers waiting simultaneously will
/// obtain the lock.
///
/// This method does not disable interrupts, so any locks related to
/// interrupt context should avoid using this method, and use [`read_irq_disabled`]
/// instead. When IRQ handlers are allowed to be executed while holding
/// this lock, it is preferable to use this method over the [`read_irq_disabled`]
/// method as it has a higher efficiency.
///
/// [`read_irq_disabled`]: Self::read_irq_disabled
pub fn read(&self) -> RwLockReadGuard<T> {
pub fn read(&self) -> RwLockReadGuard<T, G> {
loop {
if let Some(readguard) = self.try_read() {
return readguard;
@ -276,7 +145,7 @@ impl<T: ?Sized> RwLock<T> {
/// for compile-time checked lifetimes of the read guard.
///
/// [`read`]: Self::read
pub fn read_arc(self: &Arc<Self>) -> ArcRwLockReadGuard<T> {
pub fn read_arc(self: &Arc<Self>) -> ArcRwLockReadGuard<T, G> {
loop {
if let Some(readguard) = self.try_read_arc() {
return readguard;
@ -292,15 +161,7 @@ impl<T: ?Sized> RwLock<T> {
/// upreaders or readers present. There is no guarantee for the order
/// in which other readers or writers waiting simultaneously will
/// obtain the lock.
///
/// This method does not disable interrupts, so any locks related to
/// interrupt context should avoid using this method, and use [`write_irq_disabled`]
/// instead. When IRQ handlers are allowed to be executed while holding
/// this lock, it is preferable to use this method over the [`write_irq_disabled`]
/// method as it has a higher efficiency.
///
/// [`write_irq_disabled`]: Self::write_irq_disabled
pub fn write(&self) -> RwLockWriteGuard<T> {
pub fn write(&self) -> RwLockWriteGuard<T, G> {
loop {
if let Some(writeguard) = self.try_write() {
return writeguard;
@ -316,7 +177,7 @@ impl<T: ?Sized> RwLock<T> {
/// for compile-time checked lifetimes of the lock guard.
///
/// [`write`]: Self::write
pub fn write_arc(self: &Arc<Self>) -> ArcRwLockWriteGuard<T> {
pub fn write_arc(self: &Arc<Self>) -> ArcRwLockWriteGuard<T, G> {
loop {
if let Some(writeguard) = self.try_write_arc() {
return writeguard;
@ -336,15 +197,7 @@ impl<T: ?Sized> RwLock<T> {
/// and reader do not differ before invoking the upgread method. However,
/// only one upreader can exist at any time to avoid deadlock in the
/// upgread method.
///
/// This method does not disable interrupts, so any locks related to
/// interrupt context should avoid using this method, and use [`upread_irq_disabled`]
/// instead. When IRQ handlers are allowed to be executed while holding
/// this lock, it is preferable to use this method over the [`upread_irq_disabled`]
/// method as it has a higher efficiency.
///
/// [`upread_irq_disabled`]: Self::upread_irq_disabled
pub fn upread(&self) -> RwLockUpgradeableGuard<T> {
pub fn upread(&self) -> RwLockUpgradeableGuard<T, G> {
loop {
if let Some(guard) = self.try_upread() {
return guard;
@ -360,7 +213,7 @@ impl<T: ?Sized> RwLock<T> {
/// for compile-time checked lifetimes of the lock guard.
///
/// [`upread`]: Self::upread
pub fn upread_arc(self: &Arc<Self>) -> ArcRwLockUpgradeableGuard<T> {
pub fn upread_arc(self: &Arc<Self>) -> ArcRwLockUpgradeableGuard<T, G> {
loop {
if let Some(guard) = self.try_upread_arc() {
return guard;
@ -373,23 +226,11 @@ impl<T: ?Sized> RwLock<T> {
/// Attempts to acquire a read lock.
///
/// This function will never spin-wait and will return immediately.
///
/// This method does not disable interrupts, so any locks related to
/// interrupt context should avoid using this method, and use
/// [`try_read_irq_disabled`] instead. When IRQ handlers are allowed to
/// be executed while holding this lock, it is preferable to use this
/// method over the [`try_read_irq_disabled`] method as it has a higher
/// efficiency.
///
/// [`try_read_irq_disabled`]: Self::try_read_irq_disabled
pub fn try_read(&self) -> Option<RwLockReadGuard<T>> {
let guard = disable_preempt();
pub fn try_read(&self) -> Option<RwLockReadGuard<T, G>> {
let guard = G::guard();
let lock = self.lock.fetch_add(READER, Acquire);
if lock & (WRITER | MAX_READER | BEING_UPGRADED) == 0 {
Some(RwLockReadGuard {
inner: self,
inner_guard: InnerGuard::PreemptGuard(guard),
})
Some(RwLockReadGuard { inner: self, guard })
} else {
self.lock.fetch_sub(READER, Release);
None
@ -402,13 +243,13 @@ impl<T: ?Sized> RwLock<T> {
/// for compile-time checked lifetimes of the lock guard.
///
/// [`try_read`]: Self::try_read
pub fn try_read_arc(self: &Arc<Self>) -> Option<ArcRwLockReadGuard<T>> {
let guard = disable_preempt();
pub fn try_read_arc(self: &Arc<Self>) -> Option<ArcRwLockReadGuard<T, G>> {
let guard = G::guard();
let lock = self.lock.fetch_add(READER, Acquire);
if lock & (WRITER | MAX_READER | BEING_UPGRADED) == 0 {
Some(ArcRwLockReadGuard {
inner: self.clone(),
inner_guard: InnerGuard::PreemptGuard(guard),
guard,
})
} else {
self.lock.fetch_sub(READER, Release);
@ -419,26 +260,14 @@ impl<T: ?Sized> RwLock<T> {
/// Attempts to acquire a write lock.
///
/// This function will never spin-wait and will return immediately.
///
/// This method does not disable interrupts, so any locks related to
/// interrupt context should avoid using this method, and use
/// [`try_write_irq_disabled`] instead. When IRQ handlers are allowed to
/// be executed while holding this lock, it is preferable to use this
/// method over the [`try_write_irq_disabled`] method as it has a higher
/// efficiency.
///
/// [`try_write_irq_disabled`]: Self::try_write_irq_disabled
pub fn try_write(&self) -> Option<RwLockWriteGuard<T>> {
let guard = disable_preempt();
pub fn try_write(&self) -> Option<RwLockWriteGuard<T, G>> {
let guard = G::guard();
if self
.lock
.compare_exchange(0, WRITER, Acquire, Relaxed)
.is_ok()
{
Some(RwLockWriteGuard {
inner: self,
inner_guard: InnerGuard::PreemptGuard(guard),
})
Some(RwLockWriteGuard { inner: self, guard })
} else {
None
}
@ -450,8 +279,8 @@ impl<T: ?Sized> RwLock<T> {
/// for compile-time checked lifetimes of the lock guard.
///
/// [`try_write`]: Self::try_write
fn try_write_arc(self: &Arc<Self>) -> Option<ArcRwLockWriteGuard<T>> {
let guard = disable_preempt();
fn try_write_arc(self: &Arc<Self>) -> Option<ArcRwLockWriteGuard<T, G>> {
let guard = G::guard();
if self
.lock
.compare_exchange(0, WRITER, Acquire, Relaxed)
@ -459,7 +288,7 @@ impl<T: ?Sized> RwLock<T> {
{
Some(ArcRwLockWriteGuard {
inner: self.clone(),
inner_guard: InnerGuard::PreemptGuard(guard),
guard,
})
} else {
None
@ -469,23 +298,11 @@ impl<T: ?Sized> RwLock<T> {
/// Attempts to acquire an upread lock.
///
/// This function will never spin-wait and will return immediately.
///
/// This method does not disable interrupts, so any locks related to
/// interrupt context should avoid using this method, and use
/// [`try_upread_irq_disabled`] instead. When IRQ handlers are allowed to
/// be executed while holding this lock, it is preferable to use this
/// method over the [`try_upread_irq_disabled`] method as it has a higher
/// efficiency.
///
/// [`try_upread_irq_disabled`]: Self::try_upread_irq_disabled
pub fn try_upread(&self) -> Option<RwLockUpgradeableGuard<T>> {
let guard = disable_preempt();
pub fn try_upread(&self) -> Option<RwLockUpgradeableGuard<T, G>> {
let guard = G::guard();
let lock = self.lock.fetch_or(UPGRADEABLE_READER, Acquire) & (WRITER | UPGRADEABLE_READER);
if lock == 0 {
return Some(RwLockUpgradeableGuard {
inner: self,
inner_guard: InnerGuard::PreemptGuard(guard),
});
return Some(RwLockUpgradeableGuard { inner: self, guard });
} else if lock == WRITER {
self.lock.fetch_sub(UPGRADEABLE_READER, Release);
}
@ -498,13 +315,13 @@ impl<T: ?Sized> RwLock<T> {
/// for compile-time checked lifetimes of the lock guard.
///
/// [`try_upread`]: Self::try_upread
pub fn try_upread_arc(self: &Arc<Self>) -> Option<ArcRwLockUpgradeableGuard<T>> {
let guard = disable_preempt();
pub fn try_upread_arc(self: &Arc<Self>) -> Option<ArcRwLockUpgradeableGuard<T, G>> {
let guard = G::guard();
let lock = self.lock.fetch_or(UPGRADEABLE_READER, Acquire) & (WRITER | UPGRADEABLE_READER);
if lock == 0 {
return Some(ArcRwLockUpgradeableGuard {
inner: self.clone(),
inner_guard: InnerGuard::PreemptGuard(guard),
guard,
});
} else if lock == WRITER {
self.lock.fetch_sub(UPGRADEABLE_READER, Release);
@ -513,7 +330,7 @@ impl<T: ?Sized> RwLock<T> {
}
}
impl<T: ?Sized + fmt::Debug> fmt::Debug for RwLock<T> {
impl<T: ?Sized + fmt::Debug, G> fmt::Debug for RwLock<T, G> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
fmt::Debug::fmt(&self.val, f)
}
@ -521,62 +338,53 @@ impl<T: ?Sized + fmt::Debug> fmt::Debug for RwLock<T> {
/// Because there can be more than one readers to get the T's immutable ref,
/// so T must be Sync to guarantee the sharing safety.
unsafe impl<T: ?Sized + Send> Send for RwLock<T> {}
unsafe impl<T: ?Sized + Send + Sync> Sync for RwLock<T> {}
unsafe impl<T: ?Sized + Send, G> Send for RwLock<T, G> {}
unsafe impl<T: ?Sized + Send + Sync, G> Sync for RwLock<T, G> {}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> !Send for RwLockWriteGuard_<T, R> {}
unsafe impl<T: ?Sized + Sync, R: Deref<Target = RwLock<T>> + Clone + Sync> Sync
for RwLockWriteGuard_<T, R>
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> !Send
for RwLockWriteGuard_<T, R, G>
{
}
unsafe impl<T: ?Sized + Sync, R: Deref<Target = RwLock<T, G>> + Clone + Sync, G: Guardian> Sync
for RwLockWriteGuard_<T, R, G>
{
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> !Send for RwLockReadGuard_<T, R> {}
unsafe impl<T: ?Sized + Sync, R: Deref<Target = RwLock<T>> + Clone + Sync> Sync
for RwLockReadGuard_<T, R>
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> !Send
for RwLockReadGuard_<T, R, G>
{
}
unsafe impl<T: ?Sized + Sync, R: Deref<Target = RwLock<T, G>> + Clone + Sync, G: Guardian> Sync
for RwLockReadGuard_<T, R, G>
{
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> !Send for RwLockUpgradeableGuard_<T, R> {}
unsafe impl<T: ?Sized + Sync, R: Deref<Target = RwLock<T>> + Clone + Sync> Sync
for RwLockUpgradeableGuard_<T, R>
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> !Send
for RwLockUpgradeableGuard_<T, R, G>
{
}
enum InnerGuard {
IrqGuard(DisabledLocalIrqGuard),
PreemptGuard(DisabledPreemptGuard),
}
impl InnerGuard {
/// Transfers the current guard to a new `InnerGuard` instance ensuring atomicity during lock upgrades or downgrades.
///
/// This function guarantees that there will be no 'gaps' between the destruction of the old guard and
/// the creation of the new guard, maintaining the atomicity of lock transitions.
fn transfer_to(&mut self) -> Self {
match self {
InnerGuard::IrqGuard(irq_guard) => InnerGuard::IrqGuard(irq_guard.transfer_to()),
InnerGuard::PreemptGuard(preempt_guard) => {
InnerGuard::PreemptGuard(preempt_guard.transfer_to())
}
}
}
unsafe impl<T: ?Sized + Sync, R: Deref<Target = RwLock<T, G>> + Clone + Sync, G: Guardian> Sync
for RwLockUpgradeableGuard_<T, R, G>
{
}
/// A guard that provides immutable data access.
#[clippy::has_significant_drop]
#[must_use]
pub struct RwLockReadGuard_<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> {
inner_guard: InnerGuard,
pub struct RwLockReadGuard_<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> {
guard: G::Guard,
inner: R,
}
/// A guard that provides shared read access to the data protected by a [`RwLock`].
pub type RwLockReadGuard<'a, T> = RwLockReadGuard_<T, &'a RwLock<T>>;
pub type RwLockReadGuard<'a, T, G> = RwLockReadGuard_<T, &'a RwLock<T, G>, G>;
/// A guard that provides shared read access to the data protected by a `Arc<RwLock>`.
pub type ArcRwLockReadGuard<T> = RwLockReadGuard_<T, Arc<RwLock<T>>>;
pub type ArcRwLockReadGuard<T, G> = RwLockReadGuard_<T, Arc<RwLock<T, G>>, G>;
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Deref for RwLockReadGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> Deref
for RwLockReadGuard_<T, R, G>
{
type Target = T;
fn deref(&self) -> &T {
@ -584,14 +392,16 @@ impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Deref for RwLockReadGuard_
}
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Drop for RwLockReadGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> Drop
for RwLockReadGuard_<T, R, G>
{
fn drop(&mut self) {
self.inner.lock.fetch_sub(READER, Release);
}
}
impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T>> + Clone> fmt::Debug
for RwLockReadGuard_<T, R>
impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> fmt::Debug
for RwLockReadGuard_<T, R, G>
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
fmt::Debug::fmt(&**self, f)
@ -599,17 +409,19 @@ impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T>> + Clone> fmt::Debug
}
/// A guard that provides mutable data access.
pub struct RwLockWriteGuard_<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> {
inner_guard: InnerGuard,
pub struct RwLockWriteGuard_<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> {
guard: G::Guard,
inner: R,
}
/// A guard that provides exclusive write access to the data protected by a [`RwLock`].
pub type RwLockWriteGuard<'a, T> = RwLockWriteGuard_<T, &'a RwLock<T>>;
pub type RwLockWriteGuard<'a, T, G> = RwLockWriteGuard_<T, &'a RwLock<T, G>, G>;
/// A guard that provides exclusive write access to the data protected by a `Arc<RwLock>`.
pub type ArcRwLockWriteGuard<T> = RwLockWriteGuard_<T, Arc<RwLock<T>>>;
pub type ArcRwLockWriteGuard<T, G> = RwLockWriteGuard_<T, Arc<RwLock<T, G>>, G>;
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Deref for RwLockWriteGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> Deref
for RwLockWriteGuard_<T, R, G>
{
type Target = T;
fn deref(&self) -> &T {
@ -617,11 +429,11 @@ impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Deref for RwLockWriteGuard
}
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> RwLockWriteGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> RwLockWriteGuard_<T, R, G> {
/// Atomically downgrades a write guard to an upgradeable reader guard.
///
/// This method always succeeds because the lock is exclusively held by the writer.
pub fn downgrade(mut self) -> RwLockUpgradeableGuard_<T, R> {
pub fn downgrade(mut self) -> RwLockUpgradeableGuard_<T, R, G> {
loop {
self = match self.try_downgrade() {
Ok(guard) => return guard,
@ -632,36 +444,40 @@ impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> RwLockWriteGuard_<T, R> {
/// This is not exposed as a public method to prevent intermediate lock states from affecting the
/// downgrade process.
fn try_downgrade(mut self) -> Result<RwLockUpgradeableGuard_<T, R>, Self> {
fn try_downgrade(mut self) -> Result<RwLockUpgradeableGuard_<T, R, G>, Self> {
let inner = self.inner.clone();
let res = self
.inner
.lock
.compare_exchange(WRITER, UPGRADEABLE_READER, AcqRel, Relaxed);
if res.is_ok() {
let inner_guard = self.inner_guard.transfer_to();
let guard = self.guard.transfer_to();
drop(self);
Ok(RwLockUpgradeableGuard_ { inner, inner_guard })
Ok(RwLockUpgradeableGuard_ { inner, guard })
} else {
Err(self)
}
}
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> DerefMut for RwLockWriteGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> DerefMut
for RwLockWriteGuard_<T, R, G>
{
fn deref_mut(&mut self) -> &mut Self::Target {
unsafe { &mut *self.inner.val.get() }
}
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Drop for RwLockWriteGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> Drop
for RwLockWriteGuard_<T, R, G>
{
fn drop(&mut self) {
self.inner.lock.fetch_and(!WRITER, Release);
}
}
impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T>> + Clone> fmt::Debug
for RwLockWriteGuard_<T, R>
impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> fmt::Debug
for RwLockWriteGuard_<T, R, G>
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
fmt::Debug::fmt(&**self, f)
@ -670,23 +486,26 @@ impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T>> + Clone> fmt::Debug
/// A guard that provides immutable data access but can be atomically
/// upgraded to `RwLockWriteGuard`.
pub struct RwLockUpgradeableGuard_<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> {
inner_guard: InnerGuard,
pub struct RwLockUpgradeableGuard_<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian>
{
guard: G::Guard,
inner: R,
}
/// A upgradable guard that provides read access to the data protected by a [`RwLock`].
pub type RwLockUpgradeableGuard<'a, T> = RwLockUpgradeableGuard_<T, &'a RwLock<T>>;
pub type RwLockUpgradeableGuard<'a, T, G> = RwLockUpgradeableGuard_<T, &'a RwLock<T, G>, G>;
/// A upgradable guard that provides read access to the data protected by a `Arc<RwLock>`.
pub type ArcRwLockUpgradeableGuard<T> = RwLockUpgradeableGuard_<T, Arc<RwLock<T>>>;
pub type ArcRwLockUpgradeableGuard<T, G> = RwLockUpgradeableGuard_<T, Arc<RwLock<T, G>>, G>;
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> RwLockUpgradeableGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian>
RwLockUpgradeableGuard_<T, R, G>
{
/// Upgrades this upread guard to a write guard atomically.
///
/// After calling this method, subsequent readers will be blocked
/// while previous readers remain unaffected. The calling thread
/// will spin-wait until previous readers finish.
pub fn upgrade(mut self) -> RwLockWriteGuard_<T, R> {
pub fn upgrade(mut self) -> RwLockWriteGuard_<T, R, G> {
self.inner.lock.fetch_or(BEING_UPGRADED, Acquire);
loop {
self = match self.try_upgrade() {
@ -698,7 +517,7 @@ impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> RwLockUpgradeableGuard_<T,
/// Attempts to upgrade this upread guard to a write guard atomically.
///
/// This function will never spin-wait and will return immediately.
pub fn try_upgrade(mut self) -> Result<RwLockWriteGuard_<T, R>, Self> {
pub fn try_upgrade(mut self) -> Result<RwLockWriteGuard_<T, R, G>, Self> {
let res = self.inner.lock.compare_exchange(
UPGRADEABLE_READER | BEING_UPGRADED,
WRITER | UPGRADEABLE_READER,
@ -707,16 +526,18 @@ impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> RwLockUpgradeableGuard_<T,
);
if res.is_ok() {
let inner = self.inner.clone();
let inner_guard = self.inner_guard.transfer_to();
let guard = self.guard.transfer_to();
drop(self);
Ok(RwLockWriteGuard_ { inner, inner_guard })
Ok(RwLockWriteGuard_ { inner, guard })
} else {
Err(self)
}
}
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Deref for RwLockUpgradeableGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> Deref
for RwLockUpgradeableGuard_<T, R, G>
{
type Target = T;
fn deref(&self) -> &T {
@ -724,14 +545,16 @@ impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Deref for RwLockUpgradeabl
}
}
impl<T: ?Sized, R: Deref<Target = RwLock<T>> + Clone> Drop for RwLockUpgradeableGuard_<T, R> {
impl<T: ?Sized, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> Drop
for RwLockUpgradeableGuard_<T, R, G>
{
fn drop(&mut self) {
self.inner.lock.fetch_sub(UPGRADEABLE_READER, Release);
}
}
impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T>> + Clone> fmt::Debug
for RwLockUpgradeableGuard_<T, R>
impl<T: ?Sized + fmt::Debug, R: Deref<Target = RwLock<T, G>> + Clone, G: Guardian> fmt::Debug
for RwLockUpgradeableGuard_<T, R, G>
{
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
fmt::Debug::fmt(&**self, f)

View File

@ -24,12 +24,15 @@ use crate::{
/// - if `G` is [`PreemptDisabled`], preemption is disabled;
/// - if `G` is [`LocalIrqDisabled`], local IRQs are disabled.
///
/// The `G` can also be provided by other crates other than ostd,
/// if it behaves similar like [`PreemptDisabled`] or [`LocalIrqDisabled`].
///
/// The guard behavior can be temporarily upgraded from [`PreemptDisabled`] to
/// [`LocalIrqDisabled`] using the [`disable_irq`] method.
///
/// [`disable_irq`]: Self::disable_irq
#[repr(transparent)]
pub struct SpinLock<T: ?Sized, G = PreemptDisabled> {
pub struct SpinLock<T: ?Sized, G: Guardian = PreemptDisabled> {
phantom: PhantomData<G>,
/// Only the last field of a struct may have a dynamically sized type.
/// That's why SpinLockInner is put in the last field.
@ -44,12 +47,23 @@ struct SpinLockInner<T: ?Sized> {
/// A guardian that denotes the guard behavior for holding the spin lock.
pub trait Guardian {
/// The guard type.
type Guard;
type Guard: GuardTransfer;
/// Creates a new guard.
fn guard() -> Self::Guard;
}
/// The Guard can be transferred atomically.
pub trait GuardTransfer {
/// Atomically transfers the current guard to a new instance.
///
/// This function ensures that there are no 'gaps' between the destruction of the old guard and
/// the creation of the new guard, thereby maintaining the atomicity of guard transitions.
///
/// The original guard must be dropped immediately after calling this method.
fn transfer_to(&mut self) -> Self;
}
/// A guardian that disables preemption while holding the spin lock.
pub struct PreemptDisabled;
@ -165,15 +179,15 @@ impl<T: ?Sized, G: Guardian> SpinLock<T, G> {
}
}
impl<T: ?Sized + fmt::Debug, G> fmt::Debug for SpinLock<T, G> {
impl<T: ?Sized + fmt::Debug, G: Guardian> fmt::Debug for SpinLock<T, G> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
fmt::Debug::fmt(&self.inner.val, f)
}
}
// SAFETY: Only a single lock holder is permitted to access the inner data of Spinlock.
unsafe impl<T: ?Sized + Send, G> Send for SpinLock<T, G> {}
unsafe impl<T: ?Sized + Send, G> Sync for SpinLock<T, G> {}
unsafe impl<T: ?Sized + Send, G: Guardian> Send for SpinLock<T, G> {}
unsafe impl<T: ?Sized + Send, G: Guardian> Sync for SpinLock<T, G> {}
/// A guard that provides exclusive access to the data protected by a [`SpinLock`].
pub type SpinLockGuard<'a, T, G> = SpinLockGuard_<T, &'a SpinLock<T, G>, G>;

View File

@ -1,5 +1,7 @@
// SPDX-License-Identifier: MPL-2.0
use crate::sync::GuardTransfer;
/// A guard for disable preempt.
#[clippy::has_significant_drop]
#[must_use]
@ -16,10 +18,10 @@ impl DisabledPreemptGuard {
super::cpu_local::inc_guard_count();
Self { _private: () }
}
}
/// Transfer this guard to a new guard.
/// This guard must be dropped after this function.
pub fn transfer_to(&self) -> Self {
impl GuardTransfer for DisabledPreemptGuard {
fn transfer_to(&mut self) -> Self {
disable_preempt()
}
}

View File

@ -7,6 +7,7 @@ use core::fmt::Debug;
use crate::{
arch::irq::{self, IrqCallbackHandle, IRQ_ALLOCATOR},
prelude::*,
sync::GuardTransfer,
trap::TrapFrame,
Error,
};
@ -149,10 +150,10 @@ impl DisabledLocalIrqGuard {
}
Self { was_enabled }
}
}
/// Transfers the saved IRQ status of this guard to a new guard.
/// The saved IRQ status of this guard is cleared.
pub fn transfer_to(&mut self) -> Self {
impl GuardTransfer for DisabledLocalIrqGuard {
fn transfer_to(&mut self) -> Self {
let was_enabled = self.was_enabled;
self.was_enabled = false;
Self { was_enabled }