Skip to content

Commit cdc5556

Browse files
author
Sven Van Asbroeck
committed
rust/kernel/sync: Introduce BoxedMutex and BoxedCondVar
Introduce `Boxed` flavours of `Mutex` and `CondVar`. These flavours do not force users to keep them `Pin`ned. Which means users no longer need to make complex `Pin` inferences, or use `unsafe` blocks simply to use `Mutex` or `CondVar`. Signed-off-by: Sven Van Asbroeck <[email protected]>
1 parent 5dd07d5 commit cdc5556

File tree

4 files changed

+245
-22
lines changed

4 files changed

+245
-22
lines changed

rust/kernel/sync/condvar.rs

+117
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,8 @@
88
use super::{Guard, Lock, NeedsLockClass};
99
use crate::bindings;
1010
use crate::str::CStr;
11+
use crate::Result;
12+
use alloc::boxed::Box;
1113
use core::{cell::UnsafeCell, marker::PhantomPinned, mem::MaybeUninit, pin::Pin};
1214

1315
extern "C" {
@@ -135,3 +137,118 @@ impl NeedsLockClass for CondVar {
135137
unsafe { bindings::__init_waitqueue_head(self.wait_list.get(), name.as_char_ptr(), key) };
136138
}
137139
}
140+
141+
/// Exposes the kernel's [`struct wait_queue_head`] as a condition variable. It allows the caller to
142+
/// atomically release the given lock and go to sleep. It reacquires the lock when it wakes up. And
143+
/// it wakes up when notified by another thread (via [`CondVar::notify_one`] or
144+
/// [`CondVar::notify_all`]) or because the thread received a signal.
145+
///
146+
/// [`struct wait_queue_head`]: ../../../include/linux/wait.h
147+
///
148+
/// # Invariants
149+
///
150+
/// `wait_list` never moves out of its [`Box`].
151+
pub struct BoxedCondVar {
152+
/// A `bindings::wait_queue_head` kernel object.
153+
/// It contains a [`struct list_head`] that is self-referential, so
154+
/// it cannot be safely moved once it is initialised.
155+
/// We guarantee that it will never move, as per the invariant above.
156+
wait_list: Box<UnsafeCell<bindings::wait_queue_head>>,
157+
}
158+
159+
// SAFETY: `CondVar` only uses a `struct wait_queue_head`, which is safe to use on any thread.
160+
unsafe impl Send for BoxedCondVar {}
161+
162+
// SAFETY: `CondVar` only uses a `struct wait_queue_head`, which is safe to use on multiple threads
163+
// concurrently.
164+
unsafe impl Sync for BoxedCondVar {}
165+
166+
impl BoxedCondVar {
167+
/// Constructs a new condition variable.
168+
///
169+
/// # Safety
170+
///
171+
/// `key` must point to a valid memory location as it will be used by the kernel.
172+
pub unsafe fn new_with_key(
173+
name: &'static CStr,
174+
key: *mut bindings::lock_class_key,
175+
) -> Result<Self> {
176+
let cv = Self {
177+
wait_list: Box::try_new(UnsafeCell::new(bindings::wait_queue_head::default()))?,
178+
};
179+
unsafe {
180+
bindings::__init_waitqueue_head(cv.wait_list.get(), name.as_char_ptr(), key);
181+
}
182+
Ok(cv)
183+
}
184+
185+
/// Atomically releases the given lock (whose ownership is proven by the guard) and puts the
186+
/// thread to sleep. It wakes up when notified by [`CondVar::notify_one`] or
187+
/// [`CondVar::notify_all`], or when the thread receives a signal.
188+
///
189+
/// Returns whether there is a signal pending.
190+
#[must_use = "wait returns if a signal is pending, so the caller must check the return value"]
191+
pub fn wait<L: Lock>(&self, guard: &mut Guard<'_, L>) -> bool {
192+
let lock = guard.lock;
193+
let mut wait = MaybeUninit::<bindings::wait_queue_entry>::uninit();
194+
195+
// SAFETY: `wait` points to valid memory.
196+
unsafe { rust_helper_init_wait(wait.as_mut_ptr()) };
197+
198+
// SAFETY: Both `wait` and `wait_list` point to valid memory.
199+
unsafe {
200+
bindings::prepare_to_wait_exclusive(
201+
self.wait_list.get(),
202+
wait.as_mut_ptr(),
203+
bindings::TASK_INTERRUPTIBLE as _,
204+
);
205+
}
206+
207+
// SAFETY: The guard is evidence that the caller owns the lock.
208+
unsafe { lock.unlock() };
209+
210+
// SAFETY: No arguments, switches to another thread.
211+
unsafe { bindings::schedule() };
212+
213+
lock.lock_noguard();
214+
215+
// SAFETY: Both `wait` and `wait_list` point to valid memory.
216+
unsafe { bindings::finish_wait(self.wait_list.get(), wait.as_mut_ptr()) };
217+
218+
super::signal_pending()
219+
}
220+
221+
/// Calls the kernel function to notify the appropriate number of threads with the given flags.
222+
fn notify(&self, count: i32, flags: u32) {
223+
// SAFETY: `wait_list` points to valid memory.
224+
unsafe {
225+
bindings::__wake_up(
226+
self.wait_list.get(),
227+
bindings::TASK_NORMAL,
228+
count,
229+
flags as _,
230+
)
231+
};
232+
}
233+
234+
/// Wakes a single waiter up, if any. This is not 'sticky' in the sense that if no thread is
235+
/// waiting, the notification is lost completely (as opposed to automatically waking up the
236+
/// next waiter).
237+
pub fn notify_one(&self) {
238+
self.notify(1, 0);
239+
}
240+
241+
/// Wakes all waiters up, if any. This is not 'sticky' in the sense that if no thread is
242+
/// waiting, the notification is lost completely (as opposed to automatically waking up the
243+
/// next waiter).
244+
pub fn notify_all(&self) {
245+
self.notify(0, 0);
246+
}
247+
248+
/// Wakes all waiters up. If they were added by `epoll`, they are also removed from the list of
249+
/// waiters. This is useful when cleaning up a condition variable that may be waited on by
250+
/// threads that use `epoll`.
251+
pub fn free_waiters(&self) {
252+
self.notify(1, bindings::POLLHUP | POLLFREE);
253+
}
254+
}

rust/kernel/sync/mod.rs

+30
Original file line numberDiff line numberDiff line change
@@ -30,9 +30,11 @@ mod mutex;
3030
mod spinlock;
3131

3232
pub use arc::{Ref, RefCount, RefCounted};
33+
pub use condvar::BoxedCondVar;
3334
pub use condvar::CondVar;
3435
pub use guard::{Guard, Lock};
3536
pub use locked_by::LockedBy;
37+
pub use mutex::BoxedMutex;
3638
pub use mutex::Mutex;
3739
pub use spinlock::SpinLock;
3840

@@ -58,6 +60,34 @@ macro_rules! init_with_lockdep {
5860
}};
5961
}
6062

63+
#[doc(hidden)]
64+
#[macro_export]
65+
macro_rules! boxed_mutex {
66+
($obj:expr, $name:literal) => {{
67+
static mut CLASS: core::mem::MaybeUninit<$crate::bindings::lock_class_key> =
68+
core::mem::MaybeUninit::uninit();
69+
// SAFETY: `CLASS` is never used by Rust code directly; the kernel may change it though.
70+
#[allow(unused_unsafe)]
71+
unsafe {
72+
$crate::sync::BoxedMutex::new_with_key($obj, $crate::c_str!($name), CLASS.as_mut_ptr())
73+
}
74+
}};
75+
}
76+
77+
#[doc(hidden)]
78+
#[macro_export]
79+
macro_rules! boxed_condvar {
80+
($name:literal) => {{
81+
static mut CLASS: core::mem::MaybeUninit<$crate::bindings::lock_class_key> =
82+
core::mem::MaybeUninit::uninit();
83+
// SAFETY: `CLASS` is never used by Rust code directly; the kernel may change it though.
84+
#[allow(unused_unsafe)]
85+
unsafe {
86+
$crate::sync::BoxedCondVar::new_with_key($crate::c_str!($name), CLASS.as_mut_ptr())
87+
}
88+
}};
89+
}
90+
6191
/// A trait for types that need a lock class during initialisation.
6292
///
6393
/// Implementers of this trait benefit from the [`init_with_lockdep`] macro that generates a new

rust/kernel/sync/mutex.rs

+84
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,8 @@
77
use super::{Guard, Lock, NeedsLockClass};
88
use crate::bindings;
99
use crate::str::CStr;
10+
use crate::Result;
11+
use alloc::boxed::Box;
1012
use core::{cell::UnsafeCell, marker::PhantomPinned, pin::Pin};
1113

1214
/// Safely initialises a [`Mutex`] with the given name, generating a new lock class.
@@ -99,3 +101,85 @@ impl<T: ?Sized> Lock for Mutex<T> {
99101
&self.data
100102
}
101103
}
104+
105+
/// Exposes the kernel's [`struct mutex`]. When multiple threads attempt to lock the same mutex,
106+
/// only one at a time is allowed to progress, the others will block (sleep) until the mutex is
107+
/// unlocked, at which point another thread will be allowed to wake up and make progress.
108+
///
109+
/// Since it may block, [`Mutex`] needs to be used with care in atomic contexts.
110+
///
111+
/// [`struct mutex`]: ../../../include/linux/mutex.h
112+
///
113+
/// # Invariants
114+
///
115+
/// `mutex` never moves out of its [`Box`].
116+
pub struct BoxedMutex<T: ?Sized> {
117+
/// A `struct mutex` kernel object.
118+
/// It contains a [`struct mutex`] that is self-referential, so it
119+
/// cannot be safely moved once it is initialised. We guarantee that
120+
/// it will never move, as per the invariant above.
121+
mutex: Box<UnsafeCell<bindings::mutex>>,
122+
123+
/// The data protected by the mutex.
124+
data: UnsafeCell<T>,
125+
}
126+
127+
// SAFETY: `Mutex` can be transferred across thread boundaries iff the data it protects can.
128+
unsafe impl<T: ?Sized + Send> Send for BoxedMutex<T> {}
129+
130+
// SAFETY: `Mutex` serialises the interior mutability it provides, so it is `Sync` as long as the
131+
// data it protects is `Send`.
132+
unsafe impl<T: ?Sized + Send> Sync for BoxedMutex<T> {}
133+
134+
impl<T> BoxedMutex<T> {
135+
/// Constructs a new mutex.
136+
///
137+
/// # Safety
138+
///
139+
/// `key` must point to a valid memory location as it will be used by the kernel.
140+
pub unsafe fn new_with_key(
141+
t: T,
142+
name: &'static CStr,
143+
key: *mut bindings::lock_class_key,
144+
) -> Result<Self> {
145+
let m = Self {
146+
mutex: Box::try_new(UnsafeCell::new(bindings::mutex::default()))?,
147+
data: UnsafeCell::new(t),
148+
};
149+
unsafe {
150+
bindings::__mutex_init(m.mutex.get(), name.as_char_ptr(), key);
151+
}
152+
Ok(m)
153+
}
154+
}
155+
156+
impl<T: ?Sized> Lock for BoxedMutex<T> {
157+
type Inner = T;
158+
159+
fn lock_noguard(&self) {
160+
// SAFETY: `mutex` points to valid memory.
161+
unsafe {
162+
rust_helper_mutex_lock(self.mutex.get());
163+
}
164+
}
165+
166+
unsafe fn unlock(&self) {
167+
unsafe {
168+
bindings::mutex_unlock(self.mutex.get());
169+
}
170+
}
171+
172+
fn locked_data(&self) -> &UnsafeCell<T> {
173+
&self.data
174+
}
175+
}
176+
177+
impl<T: ?Sized> BoxedMutex<T> {
178+
/// Locks the mutex and gives the caller access to the data protected by it. Only one thread at
179+
/// a time is allowed to access the protected data.
180+
pub fn lock(&self) -> Guard<'_, Self> {
181+
self.lock_noguard();
182+
// SAFETY: The mutex was just acquired.
183+
unsafe { Guard::new(self) }
184+
}
185+
}

samples/rust/rust_miscdev.rs

+14-22
Original file line numberDiff line numberDiff line change
@@ -9,12 +9,12 @@ use alloc::{boxed::Box, sync::Arc};
99
use core::pin::Pin;
1010
use kernel::prelude::*;
1111
use kernel::{
12-
c_str,
12+
boxed_condvar, boxed_mutex, c_str,
1313
file::File,
1414
file_operations::{FileOpener, FileOperations},
1515
io_buffer::{IoBufferReader, IoBufferWriter},
1616
miscdev,
17-
sync::{CondVar, Mutex},
17+
sync::{BoxedCondVar, BoxedMutex},
1818
Error,
1919
};
2020

@@ -33,34 +33,26 @@ struct SharedStateInner {
3333
}
3434

3535
struct SharedState {
36-
state_changed: CondVar,
37-
inner: Mutex<SharedStateInner>,
36+
state_changed: BoxedCondVar,
37+
inner: BoxedMutex<SharedStateInner>,
3838
}
3939

4040
impl SharedState {
41-
fn try_new() -> Result<Pin<Arc<Self>>> {
42-
let state = Arc::try_pin(Self {
43-
// SAFETY: `condvar_init!` is called below.
44-
state_changed: unsafe { CondVar::new() },
45-
// SAFETY: `mutex_init!` is called below.
46-
inner: unsafe { Mutex::new(SharedStateInner { token_count: 0 }) },
47-
})?;
48-
// SAFETY: `state_changed` is pinned behind `Pin<Arc>`.
49-
let state_changed = unsafe { Pin::new_unchecked(&state.state_changed) };
50-
kernel::condvar_init!(state_changed, "SharedState::state_changed");
51-
// SAFETY: `inner` is pinned behind `Pin<Arc>`.
52-
let inner = unsafe { Pin::new_unchecked(&state.inner) };
53-
kernel::mutex_init!(inner, "SharedState::inner");
54-
Ok(state)
41+
fn try_new() -> Result<Arc<Self>> {
42+
let state = Self {
43+
state_changed: boxed_condvar!("SharedState::state_changed")?,
44+
inner: boxed_mutex!(SharedStateInner { token_count: 0 }, "SharedState::inner")?,
45+
};
46+
Ok(Arc::try_new(state)?)
5547
}
5648
}
5749

5850
struct Token {
59-
shared: Pin<Arc<SharedState>>,
51+
shared: Arc<SharedState>,
6052
}
6153

62-
impl FileOpener<Pin<Arc<SharedState>>> for Token {
63-
fn open(shared: &Pin<Arc<SharedState>>) -> Result<Self::Wrapper> {
54+
impl FileOpener<Arc<SharedState>> for Token {
55+
fn open(shared: &Arc<SharedState>) -> Result<Self::Wrapper> {
6456
Ok(Box::try_new(Self {
6557
shared: shared.clone(),
6658
})?)
@@ -122,7 +114,7 @@ impl FileOperations for Token {
122114
}
123115

124116
struct RustMiscdev {
125-
_dev: Pin<Box<miscdev::Registration<Pin<Arc<SharedState>>>>>,
117+
_dev: Pin<Box<miscdev::Registration<Arc<SharedState>>>>,
126118
}
127119

128120
impl KernelModule for RustMiscdev {

0 commit comments

Comments
 (0)