[PATCH] [RFC] rust: add PidNamespace wrapper

Christian Brauner posted 1 patch 2 months ago
rust/helpers/helpers.c       |  1 +
rust/helpers/pid_namespace.c | 26 ++++++++++++++
rust/kernel/lib.rs           |  1 +
rust/kernel/pid_namespace.rs | 68 ++++++++++++++++++++++++++++++++++++
rust/kernel/task.rs          | 56 +++++++++++++++++++++++++----
5 files changed, 146 insertions(+), 6 deletions(-)
create mode 100644 rust/helpers/pid_namespace.c
create mode 100644 rust/kernel/pid_namespace.rs
[PATCH] [RFC] rust: add PidNamespace wrapper
Posted by Christian Brauner 2 months ago
Ok, so here's my feeble attempt at getting something going for wrapping
struct pid_namespace as struct pid_namespace indirectly came up in the
file abstraction thread.

The lifetime of a pid namespace is intimately tied to the lifetime of
task. The pid namespace of a task doesn't ever change. A
unshare(CLONE_NEWPID) or setns(fd_pidns/pidfd, CLONE_NEWPID) will not
change the task's pid namespace only the pid namespace of children
spawned by the task. This invariant is important to keep in mind.

After a task is reaped it will be detached from its associated struct
pids via __unhash_process(). This will also set task->thread_pid to
NULL.

In order to retrieve the pid namespace of a task task_active_pid_ns()
can be used. The helper works on both current and non-current taks but
the requirements are slightly different in both cases and it depends on
where the helper is called.

The rules for this are simple but difficult for me to translate into
Rust. If task_active_pid_ns() is called on current then no RCU locking
is needed as current is obviously alive. On the other hand calling
task_active_pid_ns() after release_task() would work but it would mean
task_active_pid_ns() will return NULL.

Calling task_active_pid_ns() on a non-current task, while valid, must be
under RCU or other protection mechanism as the task might be
release_task() and thus in __unhash_process().

Handling that in a single impl seemed cumbersome but that may just be
my lack of kernel Rust experience.

It would of course be possible to add an always refcounted PidNamespace
impl to Task but that would be pointless refcount bumping for the usual
case where the caller retrieves the pid namespace of current.

Instead I added a macro that gets the active pid namespace of current
and a task_get_pid_ns() impl that returns an Option<ARef<PidNamespace>>.
Returning an Option<ARef<PidNamespace>> forces the caller to make a
conscious decision instead of just silently translating a NULL to
current pid namespace when passed to e.g., task_tgid_nr_ns().

Signed-off-by: Christian Brauner <brauner@kernel.org>
---
 rust/helpers/helpers.c       |  1 +
 rust/helpers/pid_namespace.c | 26 ++++++++++++++
 rust/kernel/lib.rs           |  1 +
 rust/kernel/pid_namespace.rs | 68 ++++++++++++++++++++++++++++++++++++
 rust/kernel/task.rs          | 56 +++++++++++++++++++++++++----
 5 files changed, 146 insertions(+), 6 deletions(-)
 create mode 100644 rust/helpers/pid_namespace.c
 create mode 100644 rust/kernel/pid_namespace.rs

diff --git a/rust/helpers/helpers.c b/rust/helpers/helpers.c
index 62022b18caf5..d553ad9361ce 100644
--- a/rust/helpers/helpers.c
+++ b/rust/helpers/helpers.c
@@ -17,6 +17,7 @@
 #include "kunit.c"
 #include "mutex.c"
 #include "page.c"
+#include "pid_namespace.c"
 #include "rbtree.c"
 #include "refcount.c"
 #include "security.c"
diff --git a/rust/helpers/pid_namespace.c b/rust/helpers/pid_namespace.c
new file mode 100644
index 000000000000..f41482bdec9a
--- /dev/null
+++ b/rust/helpers/pid_namespace.c
@@ -0,0 +1,26 @@
+// SPDX-License-Identifier: GPL-2.0
+
+#include <linux/pid_namespace.h>
+#include <linux/cleanup.h>
+
+struct pid_namespace *rust_helper_get_pid_ns(struct pid_namespace *ns)
+{
+	return get_pid_ns(ns);
+}
+
+void rust_helper_put_pid_ns(struct pid_namespace *ns)
+{
+	put_pid_ns(ns);
+}
+
+/* Get a reference on a task's pid namespace. */
+struct pid_namespace *rust_helper_task_get_pid_ns(struct task_struct *task)
+{
+	struct pid_namespace *pid_ns;
+
+	guard(rcu)();
+	pid_ns = task_active_pid_ns(task);
+	if (pid_ns)
+		get_pid_ns(pid_ns);
+	return pid_ns;
+}
diff --git a/rust/kernel/lib.rs b/rust/kernel/lib.rs
index ff7d88022c57..0e78ec9d06e0 100644
--- a/rust/kernel/lib.rs
+++ b/rust/kernel/lib.rs
@@ -44,6 +44,7 @@
 #[cfg(CONFIG_NET)]
 pub mod net;
 pub mod page;
+pub mod pid_namespace;
 pub mod prelude;
 pub mod print;
 pub mod sizes;
diff --git a/rust/kernel/pid_namespace.rs b/rust/kernel/pid_namespace.rs
new file mode 100644
index 000000000000..cd12c21a68cb
--- /dev/null
+++ b/rust/kernel/pid_namespace.rs
@@ -0,0 +1,68 @@
+// SPDX-License-Identifier: GPL-2.0
+
+//! Pid namespaces.
+//!
+//! C header: [`include/linux/pid_namespace.h`](srctree/include/linux/pid_namespace.h) and
+//! [`include/linux/pid.h`](srctree/include/linux/pid.h)
+
+use crate::{
+    bindings,
+    types::{AlwaysRefCounted, Opaque},
+};
+use core::{
+    ptr,
+};
+
+/// Wraps the kernel's `struct pid_namespace`. Thread safe.
+///
+/// This structure represents the Rust abstraction for a C `struct pid_namespace`. This
+/// implementation abstracts the usage of an already existing C `struct pid_namespace` within Rust
+/// code that we get passed from the C side.
+#[repr(transparent)]
+pub struct PidNamespace {
+    inner: Opaque<bindings::pid_namespace>,
+}
+
+impl PidNamespace {
+    /// Returns a raw pointer to the inner C struct.
+    #[inline]
+    pub fn as_ptr(&self) -> *mut bindings::pid_namespace {
+        self.inner.get()
+    }
+
+    /// Creates a reference to a [`PidNamespace`] from a valid pointer.
+    ///
+    /// # Safety
+    ///
+    /// The caller must ensure that `ptr` is valid and remains valid for the lifetime of the
+    /// returned [`PidNamespace`] reference.
+    pub unsafe fn from_ptr<'a>(ptr: *const bindings::pid_namespace) -> &'a Self {
+        // SAFETY: The safety requirements guarantee the validity of the dereference, while the
+        // `PidNamespace` type being transparent makes the cast ok.
+        unsafe { &*ptr.cast() }
+    }
+}
+
+// SAFETY: Instances of `PidNamespace` are always reference-counted.
+unsafe impl AlwaysRefCounted for PidNamespace {
+    #[inline]
+    fn inc_ref(&self) {
+        // SAFETY: The existence of a shared reference means that the refcount is nonzero.
+        unsafe { bindings::get_pid_ns(self.as_ptr()) };
+    }
+
+    #[inline]
+    unsafe fn dec_ref(obj: ptr::NonNull<PidNamespace>) {
+        // SAFETY: The safety requirements guarantee that the refcount is non-zero.
+        unsafe { bindings::put_pid_ns(obj.cast().as_ptr()) }
+    }
+}
+
+// SAFETY:
+// - `PidNamespace::dec_ref` can be called from any thread.
+// - It is okay to send ownership of `PidNamespace` across thread boundaries.
+unsafe impl Send for PidNamespace {}
+
+// SAFETY: It's OK to access `PidNamespace` through shared references from other threads because
+// we're either accessing properties that don't change or that are properly synchronised by C code.
+unsafe impl Sync for PidNamespace {}
diff --git a/rust/kernel/task.rs b/rust/kernel/task.rs
index 1a36a9f19368..89a431dfac5d 100644
--- a/rust/kernel/task.rs
+++ b/rust/kernel/task.rs
@@ -6,7 +6,8 @@
 
 use crate::{
     bindings,
-    types::{NotThreadSafe, Opaque},
+    pid_namespace::PidNamespace,
+    types::{ARef, NotThreadSafe, Opaque},
 };
 use core::{
     cmp::{Eq, PartialEq},
@@ -36,6 +37,37 @@ macro_rules! current {
     };
 }
 
+/// Returns the currently running task's pid namespace.
+///
+/// The lifetime of `PidNamespace` is intimately tied to the lifetime of `Task`. The pid namespace
+/// of a `Task` doesn't ever change. A `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd,
+/// CLONE_NEWPID)` will not change the task's pid namespace. This invariant is important to keep in
+/// mind.
+///
+/// After a task is reaped it will be detached from its associated `struct pid`s via
+/// __unhash_process(). This will specifically set `task->thread_pid` to `NULL`.
+///
+/// In order to retrieve the pid namespace of a task `task_active_pid_ns()` can be used. The rules
+/// for this are simple but difficult for me to translate into Rust. If `task_active_pid_ns()` is
+/// called from `current` then no RCU locking is needed as current is obviously alive. However,
+/// calling `task_active_pid_ns()` on a non-`current` task, while valid, must be under RCU or other
+/// protection as the task might be in __unhash_process().
+///
+/// We could add an always refcounted `PidNamespace` impl to `Task` but that would be pointless
+/// refcount bumping for the usual case where the caller retrieves the pid namespace of `current`.
+///
+/// So I added a macro that gets the active pid namespace of `current` and a `task_get_pid_ns()`
+/// impl that returns an `ARef<PidNamespace>` or `None` if the pid namespace is `NULL`. Returning
+/// an `Option<ARef<PidNamespace>>` forces the caller to make a conscious decision what instead of
+/// just silently translating a `NULL` to `current`'s pid namespace.
+#[macro_export]
+macro_rules! current_pid_ns {
+    () => {
+        let ptr = current()
+        unsafe { PidNamespace::from_ptr(bindings::task_active_pid_ns(ptr)) }
+    };
+}
+
 /// Wraps the kernel's `struct task_struct`.
 ///
 /// # Invariants
@@ -182,11 +214,23 @@ pub fn signal_pending(&self) -> bool {
         unsafe { bindings::signal_pending(self.0.get()) != 0 }
     }
 
-    /// Returns the given task's pid in the current pid namespace.
-    pub fn pid_in_current_ns(&self) -> Pid {
-        // SAFETY: We know that `self.0.get()` is valid by the type invariant, and passing a null
-        // pointer as the namespace is correct for using the current namespace.
-        unsafe { bindings::task_tgid_nr_ns(self.0.get(), ptr::null_mut()) }
+    /// Returns task's pid namespace with elevated reference count
+    pub fn task_get_pid_ns(&self) -> Option<ARef<PidNamespace>> {
+        let ptr = unsafe { bindings::task_get_pid_ns(self.0.get()) };
+        if ptr.is_null() {
+            None
+        } else {
+            // SAFETY: `ptr` is valid by the safety requirements of this function. And we own a
+            // reference count via `task_get_pid_ns()`.
+            // CAST: `Self` is a `repr(transparent)` wrapper around `bindings::pid_namespace`.
+            Some(unsafe { ARef::from_raw(ptr::NonNull::new_unchecked(ptr.cast::<PidNamespace>())) })
+        }
+    }
+
+    /// Returns the given task's pid in the provided pid namespace.
+    pub fn task_tgid_nr_ns(&self, pidns: &PidNamespace) -> Pid {
+        // SAFETY: We know that `self.0.get()` is valid by the type invariant.
+        unsafe { bindings::task_tgid_nr_ns(self.0.get(), pidns.as_ptr()) }
     }
 
     /// Wakes up the task.
-- 
2.45.2
Re: [PATCH] [RFC] rust: add PidNamespace wrapper
Posted by Alice Ryhl 2 months ago
On Thu, Sep 26, 2024 at 6:36 PM Christian Brauner <brauner@kernel.org> wrote:
>
> Ok, so here's my feeble attempt at getting something going for wrapping
> struct pid_namespace as struct pid_namespace indirectly came up in the
> file abstraction thread.

This looks great!

> The lifetime of a pid namespace is intimately tied to the lifetime of
> task. The pid namespace of a task doesn't ever change. A
> unshare(CLONE_NEWPID) or setns(fd_pidns/pidfd, CLONE_NEWPID) will not
> change the task's pid namespace only the pid namespace of children
> spawned by the task. This invariant is important to keep in mind.
>
> After a task is reaped it will be detached from its associated struct
> pids via __unhash_process(). This will also set task->thread_pid to
> NULL.
>
> In order to retrieve the pid namespace of a task task_active_pid_ns()
> can be used. The helper works on both current and non-current taks but
> the requirements are slightly different in both cases and it depends on
> where the helper is called.
>
> The rules for this are simple but difficult for me to translate into
> Rust. If task_active_pid_ns() is called on current then no RCU locking
> is needed as current is obviously alive. On the other hand calling
> task_active_pid_ns() after release_task() would work but it would mean
> task_active_pid_ns() will return NULL.
>
> Calling task_active_pid_ns() on a non-current task, while valid, must be
> under RCU or other protection mechanism as the task might be
> release_task() and thus in __unhash_process().

Just to confirm, calling task_active_pid_ns() on a non-current task
requires the rcu lock even if you own a refcont on the task?

Alice
Re: [PATCH] [RFC] rust: add PidNamespace wrapper
Posted by Christian Brauner 2 months ago
On Fri, Sep 27, 2024 at 02:04:13PM GMT, Alice Ryhl wrote:
> On Thu, Sep 26, 2024 at 6:36 PM Christian Brauner <brauner@kernel.org> wrote:
> >
> > Ok, so here's my feeble attempt at getting something going for wrapping
> > struct pid_namespace as struct pid_namespace indirectly came up in the
> > file abstraction thread.
> 
> This looks great!

Thanks!

> 
> > The lifetime of a pid namespace is intimately tied to the lifetime of
> > task. The pid namespace of a task doesn't ever change. A
> > unshare(CLONE_NEWPID) or setns(fd_pidns/pidfd, CLONE_NEWPID) will not
> > change the task's pid namespace only the pid namespace of children
> > spawned by the task. This invariant is important to keep in mind.
> >
> > After a task is reaped it will be detached from its associated struct
> > pids via __unhash_process(). This will also set task->thread_pid to
> > NULL.
> >
> > In order to retrieve the pid namespace of a task task_active_pid_ns()
> > can be used. The helper works on both current and non-current taks but
> > the requirements are slightly different in both cases and it depends on
> > where the helper is called.
> >
> > The rules for this are simple but difficult for me to translate into
> > Rust. If task_active_pid_ns() is called on current then no RCU locking
> > is needed as current is obviously alive. On the other hand calling
> > task_active_pid_ns() after release_task() would work but it would mean
> > task_active_pid_ns() will return NULL.
> >
> > Calling task_active_pid_ns() on a non-current task, while valid, must be
> > under RCU or other protection mechanism as the task might be
> > release_task() and thus in __unhash_process().
> 
> Just to confirm, calling task_active_pid_ns() on a non-current task
> requires the rcu lock even if you own a refcont on the task?

Interesting question. Afaik, yes. task_active_pid_ns() goes via
task->thread_pid which is a shorthand for task->pid_links[PIDTYPE_PID].

This will be NULLed when the task exits and is dead (so usually when
someone has waited on it - ignoring ptrace for sanity reasons and
autoreaping the latter amounts to the same thing just in-kernel):

T1                      T2                                                   T3
exit(0);
                        wait(T1)
                        -> wait_task_zombie()
                           -> release_task()
                              -> __exit_signals()
                                 -> __unash_process()
                                    // sets task->thread_pid == NULL         task_active_pid_ns(T1)
                                    // task->pid_links[PIDTYPE_PID] == NULL

So having a reference to struct task_struct doesn't prevent
task->thread_pid becoming NULL.

And you touch upon a very interesting point. The lifetime of struct
pid_namespace is actually tied to struct pid much tighter than it is to
struct task_struct. So when a task is released (transitions from zombie
to dead in the common case) the following happens:

release_task()
-> __exit_signals()
   -> thread_pid = get_pid(task->thread_pid)
      -> __unhash_process()
         -> detach_pid(PIDTYPE_PID)
            -> __change_pid()
               {
                       task->thread_pid = NULL;
                       task->pid_links[PIDTYPE_PID] = NULL;
                       free_pid(thread_pid)
               }
         put_pid(thread_pid)

And the free_pid() in __change_pid() does a delayed_put_pid() via
call_rcu().

So afaiu, taking the rcu_read_lock() synchronizes against that
delayed_put_pid() in __change_pid() so the call_rcu() will wait until
everyone who does

rcu_read_lock()
task_active_pid_ns(task)
rcu_read_unlock()

and sees task->thread_pid non-NULL, is done. This way no additional
reference count on struct task_struct or struct pid is needed before
plucking the pid namespace from there. Does that make sense or have I
gotten it all wrong?
Re: [PATCH] [RFC] rust: add PidNamespace wrapper
Posted by Alice Ryhl 2 months ago
On Fri, Sep 27, 2024 at 4:21 PM Christian Brauner <brauner@kernel.org> wrote:
>
> On Fri, Sep 27, 2024 at 02:04:13PM GMT, Alice Ryhl wrote:
> > On Thu, Sep 26, 2024 at 6:36 PM Christian Brauner <brauner@kernel.org> wrote:
> > >
> > > Ok, so here's my feeble attempt at getting something going for wrapping
> > > struct pid_namespace as struct pid_namespace indirectly came up in the
> > > file abstraction thread.
> >
> > This looks great!
>
> Thanks!
>
> >
> > > The lifetime of a pid namespace is intimately tied to the lifetime of
> > > task. The pid namespace of a task doesn't ever change. A
> > > unshare(CLONE_NEWPID) or setns(fd_pidns/pidfd, CLONE_NEWPID) will not
> > > change the task's pid namespace only the pid namespace of children
> > > spawned by the task. This invariant is important to keep in mind.
> > >
> > > After a task is reaped it will be detached from its associated struct
> > > pids via __unhash_process(). This will also set task->thread_pid to
> > > NULL.
> > >
> > > In order to retrieve the pid namespace of a task task_active_pid_ns()
> > > can be used. The helper works on both current and non-current taks but
> > > the requirements are slightly different in both cases and it depends on
> > > where the helper is called.
> > >
> > > The rules for this are simple but difficult for me to translate into
> > > Rust. If task_active_pid_ns() is called on current then no RCU locking
> > > is needed as current is obviously alive. On the other hand calling
> > > task_active_pid_ns() after release_task() would work but it would mean
> > > task_active_pid_ns() will return NULL.
> > >
> > > Calling task_active_pid_ns() on a non-current task, while valid, must be
> > > under RCU or other protection mechanism as the task might be
> > > release_task() and thus in __unhash_process().
> >
> > Just to confirm, calling task_active_pid_ns() on a non-current task
> > requires the rcu lock even if you own a refcont on the task?
>
> Interesting question. Afaik, yes. task_active_pid_ns() goes via
> task->thread_pid which is a shorthand for task->pid_links[PIDTYPE_PID].
>
> This will be NULLed when the task exits and is dead (so usually when
> someone has waited on it - ignoring ptrace for sanity reasons and
> autoreaping the latter amounts to the same thing just in-kernel):
>
> T1                      T2                                                   T3
> exit(0);
>                         wait(T1)
>                         -> wait_task_zombie()
>                            -> release_task()
>                               -> __exit_signals()
>                                  -> __unash_process()
>                                     // sets task->thread_pid == NULL         task_active_pid_ns(T1)
>                                     // task->pid_links[PIDTYPE_PID] == NULL
>
> So having a reference to struct task_struct doesn't prevent
> task->thread_pid becoming NULL.
>
> And you touch upon a very interesting point. The lifetime of struct
> pid_namespace is actually tied to struct pid much tighter than it is to
> struct task_struct. So when a task is released (transitions from zombie
> to dead in the common case) the following happens:
>
> release_task()
> -> __exit_signals()
>    -> thread_pid = get_pid(task->thread_pid)
>       -> __unhash_process()
>          -> detach_pid(PIDTYPE_PID)
>             -> __change_pid()
>                {
>                        task->thread_pid = NULL;
>                        task->pid_links[PIDTYPE_PID] = NULL;
>                        free_pid(thread_pid)
>                }
>          put_pid(thread_pid)
>
> And the free_pid() in __change_pid() does a delayed_put_pid() via
> call_rcu().
>
> So afaiu, taking the rcu_read_lock() synchronizes against that
> delayed_put_pid() in __change_pid() so the call_rcu() will wait until
> everyone who does
>
> rcu_read_lock()
> task_active_pid_ns(task)
> rcu_read_unlock()
>
> and sees task->thread_pid non-NULL, is done. This way no additional
> reference count on struct task_struct or struct pid is needed before
> plucking the pid namespace from there. Does that make sense or have I
> gotten it all wrong?

Okay. I agree that the code you have is the best we can do; at least
until we get an rcu guard in Rust.

The macro doesn't quite work. You need to do something to constrain
the lifetime used by `PidNamespace::from_ptr`. Right now, there is no
constraint on the lifetime, so the caller can just pick the lifetime
'static which is the lifetime that never ends. We want to constrain it
to a lifetime that ends before the task dies. The easiest is to create
a local variable and use the lifetime of that local variable. That
way, the reference can never escape the current function, and hence,
can't escape the current task.

More generally, I'm sure there are lots of fields in current where we
can access them without rcu only because we know the current task
isn't going to die on us. I don't think we should have a macro for
every single one. I think we can put together a single macro for
getting a lifetime that ends before returning to userspace, and then
reuse that lifetime for both `current` and `current_pid_ns`, and
possibly also the `DeferredFd` patch.

Alice
[PATCH v2] rust: add PidNamespace
Posted by Christian Brauner 1 month, 4 weeks ago
The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.

The `PidNamespace` of a `Task` doesn't ever change once the `Task` is
alive. A `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)`
will not have an effect on the calling `Task`'s pid namespace. It will
only effect the pid namespace of children created by the calling `Task`.
This invariant guarantees that after having acquired a reference to a
`Task`'s pid namespace it will remain unchanged.

When a task has exited and been reaped `release_task()` will be called.
This will set the `PidNamespace` of the task to `NULL`. So retrieving
the `PidNamespace` of a task that is dead will return `NULL`. Note, that
neither holding the RCU lock nor holding a referencing count to the
`Task` will prevent `release_task()` being called.

In order to retrieve the `PidNamespace` of a `Task` the
`task_active_pid_ns()` function can be used. There are two cases to
consider:

(1) retrieving the `PidNamespace` of the `current` task (2) retrieving
the `PidNamespace` of a non-`current` task

From system call context retrieving the `PidNamespace` for case (1) is
always safe and requires neither RCU locking nor a reference count to be
held. Retrieving the `PidNamespace` after `release_task()` for current
will return `NULL` but no codepath like that is exposed to Rust.

Retrieving the `PidNamespace` from system call context for (2) requires
RCU protection. Accessing `PidNamespace` outside of RCU protection
requires a reference count that must've been acquired while holding the
RCU lock. Note that accessing a non-`current` task means `NULL` can be
returned as the non-`current` task could have already passed through
`release_task()`.

To retrieve (1) the `current_pid_ns!()` macro should be used which
ensure that the returned `PidNamespace` cannot outlive the calling
scope. The associated `current_pid_ns()` function should not be called
directly as it could be abused to created an unbounded lifetime for
`PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the
common case of accessing `current`'s `PidNamespace` without RCU
protection and without having to acquire a reference count.

For (2) the `task_get_pid_ns()` method must be used. This will always
acquire a reference on `PidNamespace` and will return an `Option` to
force the caller to explicitly handle the case where `PidNamespace` is
`None`, something that tends to be forgotten when doing the equivalent
operation in `C`. Missing RCU primitives make it difficult to perform
operations that are otherwise safe without holding a reference count as
long as RCU protection is guaranteed. But it is not important currently.
But we do want it in the future.

Note for (2) the required RCU protection around calling
`task_active_pid_ns()` synchronizes against putting the last reference
of the associated `struct pid` of `task->thread_pid`. The `struct pid`
stored in that field is used to retrieve the `PidNamespace` of the
caller. When `release_task()` is called `task->thread_pid` will be
`NULL`ed and `put_pid()` on said `struct pid` will be delayed in
`free_pid()` via `call_rcu()` allowing everyone with an RCU protected
access to the `struct pid` acquired from `task->thread_pid` to finish.

Signed-off-by: Christian Brauner <brauner@kernel.org>
---
 rust/helpers/helpers.c       |   1 +
 rust/helpers/pid_namespace.c |  26 ++++++++++
 rust/kernel/lib.rs           |   1 +
 rust/kernel/pid_namespace.rs |  70 +++++++++++++++++++++++++
 rust/kernel/task.rs          | 119 ++++++++++++++++++++++++++++++++++++++++---
 5 files changed, 211 insertions(+), 6 deletions(-)

diff --git a/rust/helpers/helpers.c b/rust/helpers/helpers.c
index 62022b18caf5ec17231fd0e7be1234592d1146e3..d553ad9361ce17950d505c3b372a568730020e2f 100644
--- a/rust/helpers/helpers.c
+++ b/rust/helpers/helpers.c
@@ -17,6 +17,7 @@
 #include "kunit.c"
 #include "mutex.c"
 #include "page.c"
+#include "pid_namespace.c"
 #include "rbtree.c"
 #include "refcount.c"
 #include "security.c"
diff --git a/rust/helpers/pid_namespace.c b/rust/helpers/pid_namespace.c
new file mode 100644
index 0000000000000000000000000000000000000000..f41482bdec9a7c4e84b81ec141027fbd65251230
--- /dev/null
+++ b/rust/helpers/pid_namespace.c
@@ -0,0 +1,26 @@
+// SPDX-License-Identifier: GPL-2.0
+
+#include <linux/pid_namespace.h>
+#include <linux/cleanup.h>
+
+struct pid_namespace *rust_helper_get_pid_ns(struct pid_namespace *ns)
+{
+	return get_pid_ns(ns);
+}
+
+void rust_helper_put_pid_ns(struct pid_namespace *ns)
+{
+	put_pid_ns(ns);
+}
+
+/* Get a reference on a task's pid namespace. */
+struct pid_namespace *rust_helper_task_get_pid_ns(struct task_struct *task)
+{
+	struct pid_namespace *pid_ns;
+
+	guard(rcu)();
+	pid_ns = task_active_pid_ns(task);
+	if (pid_ns)
+		get_pid_ns(pid_ns);
+	return pid_ns;
+}
diff --git a/rust/kernel/lib.rs b/rust/kernel/lib.rs
index ff7d88022c57ca232dc028066dfa062f3fc84d1c..0e78ec9d06e0199dfafc40988a2ae86cd5df949c 100644
--- a/rust/kernel/lib.rs
+++ b/rust/kernel/lib.rs
@@ -44,6 +44,7 @@
 #[cfg(CONFIG_NET)]
 pub mod net;
 pub mod page;
+pub mod pid_namespace;
 pub mod prelude;
 pub mod print;
 pub mod sizes;
diff --git a/rust/kernel/pid_namespace.rs b/rust/kernel/pid_namespace.rs
new file mode 100644
index 0000000000000000000000000000000000000000..9a0509e802b4939ad853a802ee6d069a5f00c9df
--- /dev/null
+++ b/rust/kernel/pid_namespace.rs
@@ -0,0 +1,70 @@
+// SPDX-License-Identifier: GPL-2.0
+
+// Copyright (c) 2024 Christian Brauner <brauner@kernel.org>
+
+//! Pid namespaces.
+//!
+//! C header: [`include/linux/pid_namespace.h`](srctree/include/linux/pid_namespace.h) and
+//! [`include/linux/pid.h`](srctree/include/linux/pid.h)
+
+use crate::{
+    bindings,
+    types::{AlwaysRefCounted, Opaque},
+};
+use core::{
+    ptr,
+};
+
+/// Wraps the kernel's `struct pid_namespace`. Thread safe.
+///
+/// This structure represents the Rust abstraction for a C `struct pid_namespace`. This
+/// implementation abstracts the usage of an already existing C `struct pid_namespace` within Rust
+/// code that we get passed from the C side.
+#[repr(transparent)]
+pub struct PidNamespace {
+    inner: Opaque<bindings::pid_namespace>,
+}
+
+impl PidNamespace {
+    /// Returns a raw pointer to the inner C struct.
+    #[inline]
+    pub fn as_ptr(&self) -> *mut bindings::pid_namespace {
+        self.inner.get()
+    }
+
+    /// Creates a reference to a [`PidNamespace`] from a valid pointer.
+    ///
+    /// # Safety
+    ///
+    /// The caller must ensure that `ptr` is valid and remains valid for the lifetime of the
+    /// returned [`PidNamespace`] reference.
+    pub unsafe fn from_ptr<'a>(ptr: *const bindings::pid_namespace) -> &'a Self {
+        // SAFETY: The safety requirements guarantee the validity of the dereference, while the
+        // `PidNamespace` type being transparent makes the cast ok.
+        unsafe { &*ptr.cast() }
+    }
+}
+
+// SAFETY: Instances of `PidNamespace` are always reference-counted.
+unsafe impl AlwaysRefCounted for PidNamespace {
+    #[inline]
+    fn inc_ref(&self) {
+        // SAFETY: The existence of a shared reference means that the refcount is nonzero.
+        unsafe { bindings::get_pid_ns(self.as_ptr()) };
+    }
+
+    #[inline]
+    unsafe fn dec_ref(obj: ptr::NonNull<PidNamespace>) {
+        // SAFETY: The safety requirements guarantee that the refcount is non-zero.
+        unsafe { bindings::put_pid_ns(obj.cast().as_ptr()) }
+    }
+}
+
+// SAFETY:
+// - `PidNamespace::dec_ref` can be called from any thread.
+// - It is okay to send ownership of `PidNamespace` across thread boundaries.
+unsafe impl Send for PidNamespace {}
+
+// SAFETY: It's OK to access `PidNamespace` through shared references from other threads because
+// we're either accessing properties that don't change or that are properly synchronised by C code.
+unsafe impl Sync for PidNamespace {}
diff --git a/rust/kernel/task.rs b/rust/kernel/task.rs
index 1a36a9f193685393e7211793b6e6dd7576af8bfd..92603cdb543d9617f1f7d092edb87ccb66c9f0c1 100644
--- a/rust/kernel/task.rs
+++ b/rust/kernel/task.rs
@@ -6,7 +6,8 @@
 
 use crate::{
     bindings,
-    types::{NotThreadSafe, Opaque},
+    pid_namespace::PidNamespace,
+    types::{ARef, NotThreadSafe, Opaque},
 };
 use core::{
     cmp::{Eq, PartialEq},
@@ -36,6 +37,65 @@ macro_rules! current {
     };
 }
 
+/// Returns the currently running task's pid namespace.
+///
+/// The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.
+///
+/// The `PidNamespace` of a `Task` doesn't ever change once the `Task` is alive. A
+/// `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)` will not have an effect on the
+/// calling `Task`'s pid namespace. It will only effect the pid namespace of children created by
+/// the calling `Task`. This invariant guarantees that after having acquired a reference to a
+/// `Task`'s pid namespace it will remain unchanged.
+///
+/// When a task has exited and been reaped `release_task()` will be called. This will set the
+/// `PidNamespace` of the task to `NULL`. So retrieving the `PidNamespace` of a task that is dead
+/// will return `NULL`. Note, that neither holding the RCU lock nor holding a referencing count to
+/// the `Task` will prevent `release_task()` being called.
+///
+/// In order to retrieve the `PidNamespace` of a `Task` the `task_active_pid_ns()` function can be
+/// used. There are two cases to consider:
+///
+/// (1) retrieving the `PidNamespace` of the `current` task
+/// (2) retrieving the `PidNamespace` of a non-`current` task
+///
+/// From system call context retrieving the `PidNamespace` for case (1) is always safe and requires
+/// neither RCU locking nor a reference count to be held. Retrieving the `PidNamespace` after
+/// `release_task()` for current will return `NULL` but no codepath like that is exposed to Rust.
+///
+/// Retrieving the `PidNamespace` from system call context for (2) requires RCU protection.
+/// Accessing `PidNamespace` outside of RCU protection requires a reference count that must've been
+/// acquired while holding the RCU lock. Note that accessing a non-`current` task means `NULL` can
+/// be returned as the non-`current` task could have already passed through `release_task()`.
+///
+/// To retrieve (1) the `current_pid_ns!()` macro should be used which ensure that the returned
+/// `PidNamespace` cannot outlive the calling scope. The associated `current_pid_ns()` function
+/// should not be called directly as it could be abused to created an unbounded lifetime for
+/// `PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the common case of
+/// accessing `current`'s `PidNamespace` without RCU protection and without having to acquire a
+/// reference count.
+///
+/// For (2) the `task_get_pid_ns()` method must be used. This will always acquire a reference on
+/// `PidNamespace` and will return an `Option` to force the caller to explicitly handle the case
+/// where `PidNamespace` is `None`, something that tends to be forgotten when doing the equivalent
+/// operation in `C`. Missing RCU primitives make it difficult to perform operations that are
+/// otherwise safe without holding a reference count as long as RCU protection is guaranteed. But
+/// it is not important currently. But we do want it in the future.
+///
+/// Note for (2) the required RCU protection around calling `task_active_pid_ns()` synchronizes
+/// against putting the last reference of the associated `struct pid` of `task->thread_pid`.
+/// The `struct pid` stored in that field is used to retrieve the `PidNamespace` of the caller.
+/// When `release_task()` is called `task->thread_pid` will be `NULL`ed and `put_pid()` on said
+/// `struct pid` will be delayed in `free_pid()` via `call_rcu()` allowing everyone with an RCU
+/// protected access to the `struct pid` acquired from `task->thread_pid` to finish.
+#[macro_export]
+macro_rules! current_pid_ns {
+    () => {
+        // SAFETY: Deref + addr-of below create a temporary `PidNamespaceRef` that cannot outlive
+        // the caller.
+        unsafe { &*$crate::task::Task::current_pid_ns() }
+    };
+}
+
 /// Wraps the kernel's `struct task_struct`.
 ///
 /// # Invariants
@@ -145,6 +205,41 @@ fn deref(&self) -> &Self::Target {
         }
     }
 
+    /// Returns a PidNamespace reference for the currently executing task's/thread's pid namespace.
+    ///
+    /// This function can be used to create an unbounded lifetime by e.g., storing the returned
+    /// PidNamespace in a global variable which would be a bug. So the recommended way to get the
+    /// current task's/thread's pid namespace is to use the [`current_pid_ns`] macro because it is
+    /// safe.
+    ///
+    /// # Safety
+    ///
+    /// Callers must ensure that the returned object doesn't outlive the current task/thread.
+    pub unsafe fn current_pid_ns() -> impl Deref<Target = PidNamespace> {
+        struct PidNamespaceRef<'a> {
+            task: &'a PidNamespace,
+            _not_send: NotThreadSafe,
+        }
+
+        impl Deref for PidNamespaceRef<'_> {
+            type Target = PidNamespace;
+
+            fn deref(&self) -> &Self::Target {
+                self.task
+            }
+        }
+
+        let pidns = unsafe { bindings::task_active_pid_ns(Task::current_raw()) };
+        PidNamespaceRef {
+            // SAFETY: If the current thread is still running, the current task and its associated
+            // pid namespace are valid. Given that `PidNamespaceRef` is not `Send`, we know it
+            // cannot be transferred to another thread (where it could potentially outlive the
+            // current `Task`).
+            task: unsafe { &*pidns.cast() },
+            _not_send: NotThreadSafe,
+        }
+    }
+
     /// Returns the group leader of the given task.
     pub fn group_leader(&self) -> &Task {
         // SAFETY: By the type invariant, we know that `self.0` is a valid task. Valid tasks always
@@ -182,11 +277,23 @@ pub fn signal_pending(&self) -> bool {
         unsafe { bindings::signal_pending(self.0.get()) != 0 }
     }
 
-    /// Returns the given task's pid in the current pid namespace.
-    pub fn pid_in_current_ns(&self) -> Pid {
-        // SAFETY: We know that `self.0.get()` is valid by the type invariant, and passing a null
-        // pointer as the namespace is correct for using the current namespace.
-        unsafe { bindings::task_tgid_nr_ns(self.0.get(), ptr::null_mut()) }
+    /// Returns task's pid namespace with elevated reference count
+    pub fn task_get_pid_ns(&self) -> Option<ARef<PidNamespace>> {
+        let ptr = unsafe { bindings::task_get_pid_ns(self.0.get()) };
+        if ptr.is_null() {
+            None
+        } else {
+            // SAFETY: `ptr` is valid by the safety requirements of this function. And we own a
+            // reference count via `task_get_pid_ns()`.
+            // CAST: `Self` is a `repr(transparent)` wrapper around `bindings::pid_namespace`.
+            Some(unsafe { ARef::from_raw(ptr::NonNull::new_unchecked(ptr.cast::<PidNamespace>())) })
+        }
+    }
+
+    /// Returns the given task's pid in the provided pid namespace.
+    pub fn task_tgid_nr_ns(&self, pidns: &PidNamespace) -> Pid {
+        // SAFETY: We know that `self.0.get()` is valid by the type invariant.
+        unsafe { bindings::task_tgid_nr_ns(self.0.get(), pidns.as_ptr()) }
     }
 
     /// Wakes up the task.

---
base-commit: e9980e40804730de33c1563d9ac74d5b51591ec0
change-id: 20241001-brauner-rust-pid_namespace-52b0c92c8359
Re: [PATCH v2] rust: add PidNamespace
Posted by Gary Guo 1 month, 4 weeks ago
On Tue, 01 Oct 2024 11:43:42 +0200
Christian Brauner <brauner@kernel.org> wrote:

> The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.
> 
> The `PidNamespace` of a `Task` doesn't ever change once the `Task` is
> alive. A `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)`
> will not have an effect on the calling `Task`'s pid namespace. It will
> only effect the pid namespace of children created by the calling `Task`.
> This invariant guarantees that after having acquired a reference to a
> `Task`'s pid namespace it will remain unchanged.
> 
> When a task has exited and been reaped `release_task()` will be called.
> This will set the `PidNamespace` of the task to `NULL`. So retrieving
> the `PidNamespace` of a task that is dead will return `NULL`. Note, that
> neither holding the RCU lock nor holding a referencing count to the
> `Task` will prevent `release_task()` being called.
> 
> In order to retrieve the `PidNamespace` of a `Task` the
> `task_active_pid_ns()` function can be used. There are two cases to
> consider:
> 
> (1) retrieving the `PidNamespace` of the `current` task (2) retrieving
> the `PidNamespace` of a non-`current` task
> 
> From system call context retrieving the `PidNamespace` for case (1) is
> always safe and requires neither RCU locking nor a reference count to be
> held. Retrieving the `PidNamespace` after `release_task()` for current
> will return `NULL` but no codepath like that is exposed to Rust.
> 
> Retrieving the `PidNamespace` from system call context for (2) requires
> RCU protection. Accessing `PidNamespace` outside of RCU protection
> requires a reference count that must've been acquired while holding the
> RCU lock. Note that accessing a non-`current` task means `NULL` can be
> returned as the non-`current` task could have already passed through
> `release_task()`.
> 
> To retrieve (1) the `current_pid_ns!()` macro should be used which
> ensure that the returned `PidNamespace` cannot outlive the calling
> scope. The associated `current_pid_ns()` function should not be called
> directly as it could be abused to created an unbounded lifetime for
> `PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the
> common case of accessing `current`'s `PidNamespace` without RCU
> protection and without having to acquire a reference count.
> 
> For (2) the `task_get_pid_ns()` method must be used. This will always
> acquire a reference on `PidNamespace` and will return an `Option` to
> force the caller to explicitly handle the case where `PidNamespace` is
> `None`, something that tends to be forgotten when doing the equivalent
> operation in `C`. Missing RCU primitives make it difficult to perform
> operations that are otherwise safe without holding a reference count as
> long as RCU protection is guaranteed. But it is not important currently.
> But we do want it in the future.
> 
> Note for (2) the required RCU protection around calling
> `task_active_pid_ns()` synchronizes against putting the last reference
> of the associated `struct pid` of `task->thread_pid`. The `struct pid`
> stored in that field is used to retrieve the `PidNamespace` of the
> caller. When `release_task()` is called `task->thread_pid` will be
> `NULL`ed and `put_pid()` on said `struct pid` will be delayed in
> `free_pid()` via `call_rcu()` allowing everyone with an RCU protected
> access to the `struct pid` acquired from `task->thread_pid` to finish.
> 
> Signed-off-by: Christian Brauner <brauner@kernel.org>
> ---
>  rust/helpers/helpers.c       |   1 +
>  rust/helpers/pid_namespace.c |  26 ++++++++++
>  rust/kernel/lib.rs           |   1 +
>  rust/kernel/pid_namespace.rs |  70 +++++++++++++++++++++++++
>  rust/kernel/task.rs          | 119 ++++++++++++++++++++++++++++++++++++++++---
>  5 files changed, 211 insertions(+), 6 deletions(-)
> 
> diff --git a/rust/helpers/helpers.c b/rust/helpers/helpers.c
> index 62022b18caf5ec17231fd0e7be1234592d1146e3..d553ad9361ce17950d505c3b372a568730020e2f 100644
> --- a/rust/helpers/helpers.c
> +++ b/rust/helpers/helpers.c
> @@ -17,6 +17,7 @@
>  #include "kunit.c"
>  #include "mutex.c"
>  #include "page.c"
> +#include "pid_namespace.c"
>  #include "rbtree.c"
>  #include "refcount.c"
>  #include "security.c"
> diff --git a/rust/helpers/pid_namespace.c b/rust/helpers/pid_namespace.c
> new file mode 100644
> index 0000000000000000000000000000000000000000..f41482bdec9a7c4e84b81ec141027fbd65251230
> --- /dev/null
> +++ b/rust/helpers/pid_namespace.c
> @@ -0,0 +1,26 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +#include <linux/pid_namespace.h>
> +#include <linux/cleanup.h>
> +
> +struct pid_namespace *rust_helper_get_pid_ns(struct pid_namespace *ns)
> +{
> +	return get_pid_ns(ns);
> +}
> +
> +void rust_helper_put_pid_ns(struct pid_namespace *ns)
> +{
> +	put_pid_ns(ns);
> +}
> +
> +/* Get a reference on a task's pid namespace. */
> +struct pid_namespace *rust_helper_task_get_pid_ns(struct task_struct *task)
> +{
> +	struct pid_namespace *pid_ns;
> +
> +	guard(rcu)();
> +	pid_ns = task_active_pid_ns(task);
> +	if (pid_ns)
> +		get_pid_ns(pid_ns);
> +	return pid_ns;
> +}
> diff --git a/rust/kernel/lib.rs b/rust/kernel/lib.rs
> index ff7d88022c57ca232dc028066dfa062f3fc84d1c..0e78ec9d06e0199dfafc40988a2ae86cd5df949c 100644
> --- a/rust/kernel/lib.rs
> +++ b/rust/kernel/lib.rs
> @@ -44,6 +44,7 @@
>  #[cfg(CONFIG_NET)]
>  pub mod net;
>  pub mod page;
> +pub mod pid_namespace;
>  pub mod prelude;
>  pub mod print;
>  pub mod sizes;
> diff --git a/rust/kernel/pid_namespace.rs b/rust/kernel/pid_namespace.rs
> new file mode 100644
> index 0000000000000000000000000000000000000000..9a0509e802b4939ad853a802ee6d069a5f00c9df
> --- /dev/null
> +++ b/rust/kernel/pid_namespace.rs
> @@ -0,0 +1,70 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +// Copyright (c) 2024 Christian Brauner <brauner@kernel.org>
> +
> +//! Pid namespaces.
> +//!
> +//! C header: [`include/linux/pid_namespace.h`](srctree/include/linux/pid_namespace.h) and
> +//! [`include/linux/pid.h`](srctree/include/linux/pid.h)
> +
> +use crate::{
> +    bindings,
> +    types::{AlwaysRefCounted, Opaque},
> +};
> +use core::{
> +    ptr,
> +};
> +
> +/// Wraps the kernel's `struct pid_namespace`. Thread safe.
> +///
> +/// This structure represents the Rust abstraction for a C `struct pid_namespace`. This
> +/// implementation abstracts the usage of an already existing C `struct pid_namespace` within Rust
> +/// code that we get passed from the C side.
> +#[repr(transparent)]
> +pub struct PidNamespace {
> +    inner: Opaque<bindings::pid_namespace>,
> +}
> +
> +impl PidNamespace {
> +    /// Returns a raw pointer to the inner C struct.
> +    #[inline]
> +    pub fn as_ptr(&self) -> *mut bindings::pid_namespace {
> +        self.inner.get()
> +    }
> +
> +    /// Creates a reference to a [`PidNamespace`] from a valid pointer.
> +    ///
> +    /// # Safety
> +    ///
> +    /// The caller must ensure that `ptr` is valid and remains valid for the lifetime of the
> +    /// returned [`PidNamespace`] reference.
> +    pub unsafe fn from_ptr<'a>(ptr: *const bindings::pid_namespace) -> &'a Self {
> +        // SAFETY: The safety requirements guarantee the validity of the dereference, while the
> +        // `PidNamespace` type being transparent makes the cast ok.
> +        unsafe { &*ptr.cast() }
> +    }
> +}
> +
> +// SAFETY: Instances of `PidNamespace` are always reference-counted.
> +unsafe impl AlwaysRefCounted for PidNamespace {
> +    #[inline]
> +    fn inc_ref(&self) {
> +        // SAFETY: The existence of a shared reference means that the refcount is nonzero.
> +        unsafe { bindings::get_pid_ns(self.as_ptr()) };
> +    }
> +
> +    #[inline]
> +    unsafe fn dec_ref(obj: ptr::NonNull<PidNamespace>) {
> +        // SAFETY: The safety requirements guarantee that the refcount is non-zero.
> +        unsafe { bindings::put_pid_ns(obj.cast().as_ptr()) }
> +    }
> +}
> +
> +// SAFETY:
> +// - `PidNamespace::dec_ref` can be called from any thread.
> +// - It is okay to send ownership of `PidNamespace` across thread boundaries.
> +unsafe impl Send for PidNamespace {}
> +
> +// SAFETY: It's OK to access `PidNamespace` through shared references from other threads because
> +// we're either accessing properties that don't change or that are properly synchronised by C code.
> +unsafe impl Sync for PidNamespace {}
> diff --git a/rust/kernel/task.rs b/rust/kernel/task.rs
> index 1a36a9f193685393e7211793b6e6dd7576af8bfd..92603cdb543d9617f1f7d092edb87ccb66c9f0c1 100644
> --- a/rust/kernel/task.rs
> +++ b/rust/kernel/task.rs
> @@ -6,7 +6,8 @@
>  
>  use crate::{
>      bindings,
> -    types::{NotThreadSafe, Opaque},
> +    pid_namespace::PidNamespace,
> +    types::{ARef, NotThreadSafe, Opaque},
>  };
>  use core::{
>      cmp::{Eq, PartialEq},
> @@ -36,6 +37,65 @@ macro_rules! current {
>      };
>  }
>  
> +/// Returns the currently running task's pid namespace.
> +///
> +/// The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.
> +///
> +/// The `PidNamespace` of a `Task` doesn't ever change once the `Task` is alive. A
> +/// `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)` will not have an effect on the
> +/// calling `Task`'s pid namespace. It will only effect the pid namespace of children created by
> +/// the calling `Task`. This invariant guarantees that after having acquired a reference to a
> +/// `Task`'s pid namespace it will remain unchanged.
> +///
> +/// When a task has exited and been reaped `release_task()` will be called. This will set the
> +/// `PidNamespace` of the task to `NULL`. So retrieving the `PidNamespace` of a task that is dead
> +/// will return `NULL`. Note, that neither holding the RCU lock nor holding a referencing count to
> +/// the `Task` will prevent `release_task()` being called.
> +///
> +/// In order to retrieve the `PidNamespace` of a `Task` the `task_active_pid_ns()` function can be
> +/// used. There are two cases to consider:
> +///
> +/// (1) retrieving the `PidNamespace` of the `current` task
> +/// (2) retrieving the `PidNamespace` of a non-`current` task
> +///
> +/// From system call context retrieving the `PidNamespace` for case (1) is always safe and requires
> +/// neither RCU locking nor a reference count to be held. Retrieving the `PidNamespace` after
> +/// `release_task()` for current will return `NULL` but no codepath like that is exposed to Rust.
> +///
> +/// Retrieving the `PidNamespace` from system call context for (2) requires RCU protection.
> +/// Accessing `PidNamespace` outside of RCU protection requires a reference count that must've been
> +/// acquired while holding the RCU lock. Note that accessing a non-`current` task means `NULL` can
> +/// be returned as the non-`current` task could have already passed through `release_task()`.
> +///
> +/// To retrieve (1) the `current_pid_ns!()` macro should be used which ensure that the returned
> +/// `PidNamespace` cannot outlive the calling scope. The associated `current_pid_ns()` function
> +/// should not be called directly as it could be abused to created an unbounded lifetime for
> +/// `PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the common case of
> +/// accessing `current`'s `PidNamespace` without RCU protection and without having to acquire a
> +/// reference count.
> +///
> +/// For (2) the `task_get_pid_ns()` method must be used. This will always acquire a reference on
> +/// `PidNamespace` and will return an `Option` to force the caller to explicitly handle the case
> +/// where `PidNamespace` is `None`, something that tends to be forgotten when doing the equivalent
> +/// operation in `C`. Missing RCU primitives make it difficult to perform operations that are
> +/// otherwise safe without holding a reference count as long as RCU protection is guaranteed. But
> +/// it is not important currently. But we do want it in the future.
> +///
> +/// Note for (2) the required RCU protection around calling `task_active_pid_ns()` synchronizes
> +/// against putting the last reference of the associated `struct pid` of `task->thread_pid`.
> +/// The `struct pid` stored in that field is used to retrieve the `PidNamespace` of the caller.
> +/// When `release_task()` is called `task->thread_pid` will be `NULL`ed and `put_pid()` on said
> +/// `struct pid` will be delayed in `free_pid()` via `call_rcu()` allowing everyone with an RCU
> +/// protected access to the `struct pid` acquired from `task->thread_pid` to finish.

Is the comment here in the wrong place? The macro here is just getting
`current` one. Perhaps move it to the `task_get_pid_ns`, and as a
normal comment, since this is impl detail and not something for user to
worry about (yet)?

> +#[macro_export]
> +macro_rules! current_pid_ns {
> +    () => {
> +        // SAFETY: Deref + addr-of below create a temporary `PidNamespaceRef` that cannot outlive
> +        // the caller.
> +        unsafe { &*$crate::task::Task::current_pid_ns() }
> +    };
> +}
> +
>  /// Wraps the kernel's `struct task_struct`.
>  ///
>  /// # Invariants
> @@ -145,6 +205,41 @@ fn deref(&self) -> &Self::Target {
>          }
>      }
>  
> +    /// Returns a PidNamespace reference for the currently executing task's/thread's pid namespace.
> +    ///
> +    /// This function can be used to create an unbounded lifetime by e.g., storing the returned
> +    /// PidNamespace in a global variable which would be a bug. So the recommended way to get the
> +    /// current task's/thread's pid namespace is to use the [`current_pid_ns`] macro because it is
> +    /// safe.
> +    ///
> +    /// # Safety
> +    ///
> +    /// Callers must ensure that the returned object doesn't outlive the current task/thread.
> +    pub unsafe fn current_pid_ns() -> impl Deref<Target = PidNamespace> {
> +        struct PidNamespaceRef<'a> {
> +            task: &'a PidNamespace,
> +            _not_send: NotThreadSafe,
> +        }
> +
> +        impl Deref for PidNamespaceRef<'_> {
> +            type Target = PidNamespace;
> +
> +            fn deref(&self) -> &Self::Target {
> +                self.task
> +            }
> +        }
> +
> +        let pidns = unsafe { bindings::task_active_pid_ns(Task::current_raw()) };
> +        PidNamespaceRef {
> +            // SAFETY: If the current thread is still running, the current task and its associated
> +            // pid namespace are valid. Given that `PidNamespaceRef` is not `Send`, we know it
> +            // cannot be transferred to another thread (where it could potentially outlive the
> +            // current `Task`).
> +            task: unsafe { &*pidns.cast() },
> +            _not_send: NotThreadSafe,
> +        }
> +    }
> +
>      /// Returns the group leader of the given task.
>      pub fn group_leader(&self) -> &Task {
>          // SAFETY: By the type invariant, we know that `self.0` is a valid task. Valid tasks always
> @@ -182,11 +277,23 @@ pub fn signal_pending(&self) -> bool {
>          unsafe { bindings::signal_pending(self.0.get()) != 0 }
>      }
>  
> -    /// Returns the given task's pid in the current pid namespace.
> -    pub fn pid_in_current_ns(&self) -> Pid {
> -        // SAFETY: We know that `self.0.get()` is valid by the type invariant, and passing a null
> -        // pointer as the namespace is correct for using the current namespace.
> -        unsafe { bindings::task_tgid_nr_ns(self.0.get(), ptr::null_mut()) }
> +    /// Returns task's pid namespace with elevated reference count
> +    pub fn task_get_pid_ns(&self) -> Option<ARef<PidNamespace>> {

Given that this is within `Task`, the full name of the function became
`Task::task_get_pid_ns`. So this can just be `get_pid_ns`?

> +        let ptr = unsafe { bindings::task_get_pid_ns(self.0.get()) };
> +        if ptr.is_null() {
> +            None
> +        } else {
> +            // SAFETY: `ptr` is valid by the safety requirements of this function. And we own a
> +            // reference count via `task_get_pid_ns()`.
> +            // CAST: `Self` is a `repr(transparent)` wrapper around `bindings::pid_namespace`.
> +            Some(unsafe { ARef::from_raw(ptr::NonNull::new_unchecked(ptr.cast::<PidNamespace>())) })
> +        }
> +    }
> +
> +    /// Returns the given task's pid in the provided pid namespace.
> +    pub fn task_tgid_nr_ns(&self, pidns: &PidNamespace) -> Pid {

Similarly, this can drop `task_` prefix as it's already scoped to
`Task`.

PS. I think I quite like the more descriptive name in Alice's patch,
maybe `Task::tgid_in_ns` could be a good name for this?

If there's concern about documentation searchability, there is a
feature in rustdoc where you can put

	#[doc(alias = "task_tgid_nr_ns")]

and then the function will be searchable with the C name.

> +        // SAFETY: We know that `self.0.get()` is valid by the type invariant.
> +        unsafe { bindings::task_tgid_nr_ns(self.0.get(), pidns.as_ptr()) }
>      }
>  
>      /// Wakes up the task.
> 
> ---
> base-commit: e9980e40804730de33c1563d9ac74d5b51591ec0
> change-id: 20241001-brauner-rust-pid_namespace-52b0c92c8359
> 
> 

Best,
Gary
Re: [PATCH v2] rust: add PidNamespace
Posted by Christian Brauner 1 month, 3 weeks ago
On Tue, Oct 01, 2024 at 08:10:54PM GMT, Gary Guo wrote:
> On Tue, 01 Oct 2024 11:43:42 +0200
> Christian Brauner <brauner@kernel.org> wrote:
> 
> > The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.
> > 
> > The `PidNamespace` of a `Task` doesn't ever change once the `Task` is
> > alive. A `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)`
> > will not have an effect on the calling `Task`'s pid namespace. It will
> > only effect the pid namespace of children created by the calling `Task`.
> > This invariant guarantees that after having acquired a reference to a
> > `Task`'s pid namespace it will remain unchanged.
> > 
> > When a task has exited and been reaped `release_task()` will be called.
> > This will set the `PidNamespace` of the task to `NULL`. So retrieving
> > the `PidNamespace` of a task that is dead will return `NULL`. Note, that
> > neither holding the RCU lock nor holding a referencing count to the
> > `Task` will prevent `release_task()` being called.
> > 
> > In order to retrieve the `PidNamespace` of a `Task` the
> > `task_active_pid_ns()` function can be used. There are two cases to
> > consider:
> > 
> > (1) retrieving the `PidNamespace` of the `current` task (2) retrieving
> > the `PidNamespace` of a non-`current` task
> > 
> > From system call context retrieving the `PidNamespace` for case (1) is
> > always safe and requires neither RCU locking nor a reference count to be
> > held. Retrieving the `PidNamespace` after `release_task()` for current
> > will return `NULL` but no codepath like that is exposed to Rust.
> > 
> > Retrieving the `PidNamespace` from system call context for (2) requires
> > RCU protection. Accessing `PidNamespace` outside of RCU protection
> > requires a reference count that must've been acquired while holding the
> > RCU lock. Note that accessing a non-`current` task means `NULL` can be
> > returned as the non-`current` task could have already passed through
> > `release_task()`.
> > 
> > To retrieve (1) the `current_pid_ns!()` macro should be used which
> > ensure that the returned `PidNamespace` cannot outlive the calling
> > scope. The associated `current_pid_ns()` function should not be called
> > directly as it could be abused to created an unbounded lifetime for
> > `PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the
> > common case of accessing `current`'s `PidNamespace` without RCU
> > protection and without having to acquire a reference count.
> > 
> > For (2) the `task_get_pid_ns()` method must be used. This will always
> > acquire a reference on `PidNamespace` and will return an `Option` to
> > force the caller to explicitly handle the case where `PidNamespace` is
> > `None`, something that tends to be forgotten when doing the equivalent
> > operation in `C`. Missing RCU primitives make it difficult to perform
> > operations that are otherwise safe without holding a reference count as
> > long as RCU protection is guaranteed. But it is not important currently.
> > But we do want it in the future.
> > 
> > Note for (2) the required RCU protection around calling
> > `task_active_pid_ns()` synchronizes against putting the last reference
> > of the associated `struct pid` of `task->thread_pid`. The `struct pid`
> > stored in that field is used to retrieve the `PidNamespace` of the
> > caller. When `release_task()` is called `task->thread_pid` will be
> > `NULL`ed and `put_pid()` on said `struct pid` will be delayed in
> > `free_pid()` via `call_rcu()` allowing everyone with an RCU protected
> > access to the `struct pid` acquired from `task->thread_pid` to finish.
> > 
> > Signed-off-by: Christian Brauner <brauner@kernel.org>
> > ---
> >  rust/helpers/helpers.c       |   1 +
> >  rust/helpers/pid_namespace.c |  26 ++++++++++
> >  rust/kernel/lib.rs           |   1 +
> >  rust/kernel/pid_namespace.rs |  70 +++++++++++++++++++++++++
> >  rust/kernel/task.rs          | 119 ++++++++++++++++++++++++++++++++++++++++---
> >  5 files changed, 211 insertions(+), 6 deletions(-)
> > 
> > diff --git a/rust/helpers/helpers.c b/rust/helpers/helpers.c
> > index 62022b18caf5ec17231fd0e7be1234592d1146e3..d553ad9361ce17950d505c3b372a568730020e2f 100644
> > --- a/rust/helpers/helpers.c
> > +++ b/rust/helpers/helpers.c
> > @@ -17,6 +17,7 @@
> >  #include "kunit.c"
> >  #include "mutex.c"
> >  #include "page.c"
> > +#include "pid_namespace.c"
> >  #include "rbtree.c"
> >  #include "refcount.c"
> >  #include "security.c"
> > diff --git a/rust/helpers/pid_namespace.c b/rust/helpers/pid_namespace.c
> > new file mode 100644
> > index 0000000000000000000000000000000000000000..f41482bdec9a7c4e84b81ec141027fbd65251230
> > --- /dev/null
> > +++ b/rust/helpers/pid_namespace.c
> > @@ -0,0 +1,26 @@
> > +// SPDX-License-Identifier: GPL-2.0
> > +
> > +#include <linux/pid_namespace.h>
> > +#include <linux/cleanup.h>
> > +
> > +struct pid_namespace *rust_helper_get_pid_ns(struct pid_namespace *ns)
> > +{
> > +	return get_pid_ns(ns);
> > +}
> > +
> > +void rust_helper_put_pid_ns(struct pid_namespace *ns)
> > +{
> > +	put_pid_ns(ns);
> > +}
> > +
> > +/* Get a reference on a task's pid namespace. */
> > +struct pid_namespace *rust_helper_task_get_pid_ns(struct task_struct *task)
> > +{
> > +	struct pid_namespace *pid_ns;
> > +
> > +	guard(rcu)();
> > +	pid_ns = task_active_pid_ns(task);
> > +	if (pid_ns)
> > +		get_pid_ns(pid_ns);
> > +	return pid_ns;
> > +}
> > diff --git a/rust/kernel/lib.rs b/rust/kernel/lib.rs
> > index ff7d88022c57ca232dc028066dfa062f3fc84d1c..0e78ec9d06e0199dfafc40988a2ae86cd5df949c 100644
> > --- a/rust/kernel/lib.rs
> > +++ b/rust/kernel/lib.rs
> > @@ -44,6 +44,7 @@
> >  #[cfg(CONFIG_NET)]
> >  pub mod net;
> >  pub mod page;
> > +pub mod pid_namespace;
> >  pub mod prelude;
> >  pub mod print;
> >  pub mod sizes;
> > diff --git a/rust/kernel/pid_namespace.rs b/rust/kernel/pid_namespace.rs
> > new file mode 100644
> > index 0000000000000000000000000000000000000000..9a0509e802b4939ad853a802ee6d069a5f00c9df
> > --- /dev/null
> > +++ b/rust/kernel/pid_namespace.rs
> > @@ -0,0 +1,70 @@
> > +// SPDX-License-Identifier: GPL-2.0
> > +
> > +// Copyright (c) 2024 Christian Brauner <brauner@kernel.org>
> > +
> > +//! Pid namespaces.
> > +//!
> > +//! C header: [`include/linux/pid_namespace.h`](srctree/include/linux/pid_namespace.h) and
> > +//! [`include/linux/pid.h`](srctree/include/linux/pid.h)
> > +
> > +use crate::{
> > +    bindings,
> > +    types::{AlwaysRefCounted, Opaque},
> > +};
> > +use core::{
> > +    ptr,
> > +};
> > +
> > +/// Wraps the kernel's `struct pid_namespace`. Thread safe.
> > +///
> > +/// This structure represents the Rust abstraction for a C `struct pid_namespace`. This
> > +/// implementation abstracts the usage of an already existing C `struct pid_namespace` within Rust
> > +/// code that we get passed from the C side.
> > +#[repr(transparent)]
> > +pub struct PidNamespace {
> > +    inner: Opaque<bindings::pid_namespace>,
> > +}
> > +
> > +impl PidNamespace {
> > +    /// Returns a raw pointer to the inner C struct.
> > +    #[inline]
> > +    pub fn as_ptr(&self) -> *mut bindings::pid_namespace {
> > +        self.inner.get()
> > +    }
> > +
> > +    /// Creates a reference to a [`PidNamespace`] from a valid pointer.
> > +    ///
> > +    /// # Safety
> > +    ///
> > +    /// The caller must ensure that `ptr` is valid and remains valid for the lifetime of the
> > +    /// returned [`PidNamespace`] reference.
> > +    pub unsafe fn from_ptr<'a>(ptr: *const bindings::pid_namespace) -> &'a Self {
> > +        // SAFETY: The safety requirements guarantee the validity of the dereference, while the
> > +        // `PidNamespace` type being transparent makes the cast ok.
> > +        unsafe { &*ptr.cast() }
> > +    }
> > +}
> > +
> > +// SAFETY: Instances of `PidNamespace` are always reference-counted.
> > +unsafe impl AlwaysRefCounted for PidNamespace {
> > +    #[inline]
> > +    fn inc_ref(&self) {
> > +        // SAFETY: The existence of a shared reference means that the refcount is nonzero.
> > +        unsafe { bindings::get_pid_ns(self.as_ptr()) };
> > +    }
> > +
> > +    #[inline]
> > +    unsafe fn dec_ref(obj: ptr::NonNull<PidNamespace>) {
> > +        // SAFETY: The safety requirements guarantee that the refcount is non-zero.
> > +        unsafe { bindings::put_pid_ns(obj.cast().as_ptr()) }
> > +    }
> > +}
> > +
> > +// SAFETY:
> > +// - `PidNamespace::dec_ref` can be called from any thread.
> > +// - It is okay to send ownership of `PidNamespace` across thread boundaries.
> > +unsafe impl Send for PidNamespace {}
> > +
> > +// SAFETY: It's OK to access `PidNamespace` through shared references from other threads because
> > +// we're either accessing properties that don't change or that are properly synchronised by C code.
> > +unsafe impl Sync for PidNamespace {}
> > diff --git a/rust/kernel/task.rs b/rust/kernel/task.rs
> > index 1a36a9f193685393e7211793b6e6dd7576af8bfd..92603cdb543d9617f1f7d092edb87ccb66c9f0c1 100644
> > --- a/rust/kernel/task.rs
> > +++ b/rust/kernel/task.rs
> > @@ -6,7 +6,8 @@
> >  
> >  use crate::{
> >      bindings,
> > -    types::{NotThreadSafe, Opaque},
> > +    pid_namespace::PidNamespace,
> > +    types::{ARef, NotThreadSafe, Opaque},
> >  };
> >  use core::{
> >      cmp::{Eq, PartialEq},
> > @@ -36,6 +37,65 @@ macro_rules! current {
> >      };
> >  }
> >  
> > +/// Returns the currently running task's pid namespace.
> > +///
> > +/// The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.
> > +///
> > +/// The `PidNamespace` of a `Task` doesn't ever change once the `Task` is alive. A
> > +/// `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)` will not have an effect on the
> > +/// calling `Task`'s pid namespace. It will only effect the pid namespace of children created by
> > +/// the calling `Task`. This invariant guarantees that after having acquired a reference to a
> > +/// `Task`'s pid namespace it will remain unchanged.
> > +///
> > +/// When a task has exited and been reaped `release_task()` will be called. This will set the
> > +/// `PidNamespace` of the task to `NULL`. So retrieving the `PidNamespace` of a task that is dead
> > +/// will return `NULL`. Note, that neither holding the RCU lock nor holding a referencing count to
> > +/// the `Task` will prevent `release_task()` being called.
> > +///
> > +/// In order to retrieve the `PidNamespace` of a `Task` the `task_active_pid_ns()` function can be
> > +/// used. There are two cases to consider:
> > +///
> > +/// (1) retrieving the `PidNamespace` of the `current` task
> > +/// (2) retrieving the `PidNamespace` of a non-`current` task
> > +///
> > +/// From system call context retrieving the `PidNamespace` for case (1) is always safe and requires
> > +/// neither RCU locking nor a reference count to be held. Retrieving the `PidNamespace` after
> > +/// `release_task()` for current will return `NULL` but no codepath like that is exposed to Rust.
> > +///
> > +/// Retrieving the `PidNamespace` from system call context for (2) requires RCU protection.
> > +/// Accessing `PidNamespace` outside of RCU protection requires a reference count that must've been
> > +/// acquired while holding the RCU lock. Note that accessing a non-`current` task means `NULL` can
> > +/// be returned as the non-`current` task could have already passed through `release_task()`.
> > +///
> > +/// To retrieve (1) the `current_pid_ns!()` macro should be used which ensure that the returned
> > +/// `PidNamespace` cannot outlive the calling scope. The associated `current_pid_ns()` function
> > +/// should not be called directly as it could be abused to created an unbounded lifetime for
> > +/// `PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the common case of
> > +/// accessing `current`'s `PidNamespace` without RCU protection and without having to acquire a
> > +/// reference count.
> > +///
> > +/// For (2) the `task_get_pid_ns()` method must be used. This will always acquire a reference on
> > +/// `PidNamespace` and will return an `Option` to force the caller to explicitly handle the case
> > +/// where `PidNamespace` is `None`, something that tends to be forgotten when doing the equivalent
> > +/// operation in `C`. Missing RCU primitives make it difficult to perform operations that are
> > +/// otherwise safe without holding a reference count as long as RCU protection is guaranteed. But
> > +/// it is not important currently. But we do want it in the future.
> > +///
> > +/// Note for (2) the required RCU protection around calling `task_active_pid_ns()` synchronizes
> > +/// against putting the last reference of the associated `struct pid` of `task->thread_pid`.
> > +/// The `struct pid` stored in that field is used to retrieve the `PidNamespace` of the caller.
> > +/// When `release_task()` is called `task->thread_pid` will be `NULL`ed and `put_pid()` on said
> > +/// `struct pid` will be delayed in `free_pid()` via `call_rcu()` allowing everyone with an RCU
> > +/// protected access to the `struct pid` acquired from `task->thread_pid` to finish.
> 
> Is the comment here in the wrong place? The macro here is just getting
> `current` one. Perhaps move it to the `task_get_pid_ns`, and as a
> normal comment, since this is impl detail and not something for user to
> worry about (yet)?

Sure.

> 
> > +#[macro_export]
> > +macro_rules! current_pid_ns {
> > +    () => {
> > +        // SAFETY: Deref + addr-of below create a temporary `PidNamespaceRef` that cannot outlive
> > +        // the caller.
> > +        unsafe { &*$crate::task::Task::current_pid_ns() }
> > +    };
> > +}
> > +
> >  /// Wraps the kernel's `struct task_struct`.
> >  ///
> >  /// # Invariants
> > @@ -145,6 +205,41 @@ fn deref(&self) -> &Self::Target {
> >          }
> >      }
> >  
> > +    /// Returns a PidNamespace reference for the currently executing task's/thread's pid namespace.
> > +    ///
> > +    /// This function can be used to create an unbounded lifetime by e.g., storing the returned
> > +    /// PidNamespace in a global variable which would be a bug. So the recommended way to get the
> > +    /// current task's/thread's pid namespace is to use the [`current_pid_ns`] macro because it is
> > +    /// safe.
> > +    ///
> > +    /// # Safety
> > +    ///
> > +    /// Callers must ensure that the returned object doesn't outlive the current task/thread.
> > +    pub unsafe fn current_pid_ns() -> impl Deref<Target = PidNamespace> {
> > +        struct PidNamespaceRef<'a> {
> > +            task: &'a PidNamespace,
> > +            _not_send: NotThreadSafe,
> > +        }
> > +
> > +        impl Deref for PidNamespaceRef<'_> {
> > +            type Target = PidNamespace;
> > +
> > +            fn deref(&self) -> &Self::Target {
> > +                self.task
> > +            }
> > +        }
> > +
> > +        let pidns = unsafe { bindings::task_active_pid_ns(Task::current_raw()) };
> > +        PidNamespaceRef {
> > +            // SAFETY: If the current thread is still running, the current task and its associated
> > +            // pid namespace are valid. Given that `PidNamespaceRef` is not `Send`, we know it
> > +            // cannot be transferred to another thread (where it could potentially outlive the
> > +            // current `Task`).
> > +            task: unsafe { &*pidns.cast() },
> > +            _not_send: NotThreadSafe,
> > +        }
> > +    }
> > +
> >      /// Returns the group leader of the given task.
> >      pub fn group_leader(&self) -> &Task {
> >          // SAFETY: By the type invariant, we know that `self.0` is a valid task. Valid tasks always
> > @@ -182,11 +277,23 @@ pub fn signal_pending(&self) -> bool {
> >          unsafe { bindings::signal_pending(self.0.get()) != 0 }
> >      }
> >  
> > -    /// Returns the given task's pid in the current pid namespace.
> > -    pub fn pid_in_current_ns(&self) -> Pid {
> > -        // SAFETY: We know that `self.0.get()` is valid by the type invariant, and passing a null
> > -        // pointer as the namespace is correct for using the current namespace.
> > -        unsafe { bindings::task_tgid_nr_ns(self.0.get(), ptr::null_mut()) }
> > +    /// Returns task's pid namespace with elevated reference count
> > +    pub fn task_get_pid_ns(&self) -> Option<ARef<PidNamespace>> {
> 
> Given that this is within `Task`, the full name of the function became
> `Task::task_get_pid_ns`. So this can just be `get_pid_ns`?

Fair.

> 
> > +        let ptr = unsafe { bindings::task_get_pid_ns(self.0.get()) };
> > +        if ptr.is_null() {
> > +            None
> > +        } else {
> > +            // SAFETY: `ptr` is valid by the safety requirements of this function. And we own a
> > +            // reference count via `task_get_pid_ns()`.
> > +            // CAST: `Self` is a `repr(transparent)` wrapper around `bindings::pid_namespace`.
> > +            Some(unsafe { ARef::from_raw(ptr::NonNull::new_unchecked(ptr.cast::<PidNamespace>())) })
> > +        }
> > +    }
> > +
> > +    /// Returns the given task's pid in the provided pid namespace.
> > +    pub fn task_tgid_nr_ns(&self, pidns: &PidNamespace) -> Pid {
> 
> Similarly, this can drop `task_` prefix as it's already scoped to
> `Task`.
> 
> PS. I think I quite like the more descriptive name in Alice's patch,
> maybe `Task::tgid_in_ns` could be a good name for this?

I'm not found of the "in" part. tgid_nr_ns() is fine with me.

> 
> If there's concern about documentation searchability, there is a
> feature in rustdoc where you can put
> 
> 	#[doc(alias = "task_tgid_nr_ns")]
> 
> and then the function will be searchable with the C name.

Sounds good.
Re: [PATCH v2] rust: add PidNamespace
Posted by Alice Ryhl 1 month, 4 weeks ago
On Tue, Oct 1, 2024 at 11:44 AM Christian Brauner <brauner@kernel.org> wrote:
>
> The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.
>
> The `PidNamespace` of a `Task` doesn't ever change once the `Task` is
> alive. A `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)`
> will not have an effect on the calling `Task`'s pid namespace. It will
> only effect the pid namespace of children created by the calling `Task`.
> This invariant guarantees that after having acquired a reference to a
> `Task`'s pid namespace it will remain unchanged.
>
> When a task has exited and been reaped `release_task()` will be called.
> This will set the `PidNamespace` of the task to `NULL`. So retrieving
> the `PidNamespace` of a task that is dead will return `NULL`. Note, that
> neither holding the RCU lock nor holding a referencing count to the
> `Task` will prevent `release_task()` being called.
>
> In order to retrieve the `PidNamespace` of a `Task` the
> `task_active_pid_ns()` function can be used. There are two cases to
> consider:
>
> (1) retrieving the `PidNamespace` of the `current` task (2) retrieving
> the `PidNamespace` of a non-`current` task
>
> From system call context retrieving the `PidNamespace` for case (1) is
> always safe and requires neither RCU locking nor a reference count to be
> held. Retrieving the `PidNamespace` after `release_task()` for current
> will return `NULL` but no codepath like that is exposed to Rust.
>
> Retrieving the `PidNamespace` from system call context for (2) requires
> RCU protection. Accessing `PidNamespace` outside of RCU protection
> requires a reference count that must've been acquired while holding the
> RCU lock. Note that accessing a non-`current` task means `NULL` can be
> returned as the non-`current` task could have already passed through
> `release_task()`.
>
> To retrieve (1) the `current_pid_ns!()` macro should be used which
> ensure that the returned `PidNamespace` cannot outlive the calling
> scope. The associated `current_pid_ns()` function should not be called
> directly as it could be abused to created an unbounded lifetime for
> `PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the
> common case of accessing `current`'s `PidNamespace` without RCU
> protection and without having to acquire a reference count.
>
> For (2) the `task_get_pid_ns()` method must be used. This will always
> acquire a reference on `PidNamespace` and will return an `Option` to
> force the caller to explicitly handle the case where `PidNamespace` is
> `None`, something that tends to be forgotten when doing the equivalent
> operation in `C`. Missing RCU primitives make it difficult to perform
> operations that are otherwise safe without holding a reference count as
> long as RCU protection is guaranteed. But it is not important currently.
> But we do want it in the future.
>
> Note for (2) the required RCU protection around calling
> `task_active_pid_ns()` synchronizes against putting the last reference
> of the associated `struct pid` of `task->thread_pid`. The `struct pid`
> stored in that field is used to retrieve the `PidNamespace` of the
> caller. When `release_task()` is called `task->thread_pid` will be
> `NULL`ed and `put_pid()` on said `struct pid` will be delayed in
> `free_pid()` via `call_rcu()` allowing everyone with an RCU protected
> access to the `struct pid` acquired from `task->thread_pid` to finish.
>
> Signed-off-by: Christian Brauner <brauner@kernel.org>

Overall looks good to me, but a few comments below.

Also, I think it would be fine to send the next version without it
being a reply to the file bindings thread.

>  rust/helpers/helpers.c       |   1 +
>  rust/helpers/pid_namespace.c |  26 ++++++++++
>  rust/kernel/lib.rs           |   1 +
>  rust/kernel/pid_namespace.rs |  70 +++++++++++++++++++++++++
>  rust/kernel/task.rs          | 119 ++++++++++++++++++++++++++++++++++++++++---
>  5 files changed, 211 insertions(+), 6 deletions(-)
>
> diff --git a/rust/helpers/helpers.c b/rust/helpers/helpers.c
> index 62022b18caf5ec17231fd0e7be1234592d1146e3..d553ad9361ce17950d505c3b372a568730020e2f 100644
> --- a/rust/helpers/helpers.c
> +++ b/rust/helpers/helpers.c
> @@ -17,6 +17,7 @@
>  #include "kunit.c"
>  #include "mutex.c"
>  #include "page.c"
> +#include "pid_namespace.c"
>  #include "rbtree.c"
>  #include "refcount.c"
>  #include "security.c"
> diff --git a/rust/helpers/pid_namespace.c b/rust/helpers/pid_namespace.c
> new file mode 100644
> index 0000000000000000000000000000000000000000..f41482bdec9a7c4e84b81ec141027fbd65251230
> --- /dev/null
> +++ b/rust/helpers/pid_namespace.c
> @@ -0,0 +1,26 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +#include <linux/pid_namespace.h>
> +#include <linux/cleanup.h>
> +
> +struct pid_namespace *rust_helper_get_pid_ns(struct pid_namespace *ns)
> +{
> +       return get_pid_ns(ns);
> +}
> +
> +void rust_helper_put_pid_ns(struct pid_namespace *ns)
> +{
> +       put_pid_ns(ns);
> +}
> +
> +/* Get a reference on a task's pid namespace. */
> +struct pid_namespace *rust_helper_task_get_pid_ns(struct task_struct *task)
> +{
> +       struct pid_namespace *pid_ns;
> +
> +       guard(rcu)();
> +       pid_ns = task_active_pid_ns(task);
> +       if (pid_ns)
> +               get_pid_ns(pid_ns);
> +       return pid_ns;
> +}
> diff --git a/rust/kernel/lib.rs b/rust/kernel/lib.rs
> index ff7d88022c57ca232dc028066dfa062f3fc84d1c..0e78ec9d06e0199dfafc40988a2ae86cd5df949c 100644
> --- a/rust/kernel/lib.rs
> +++ b/rust/kernel/lib.rs
> @@ -44,6 +44,7 @@
>  #[cfg(CONFIG_NET)]
>  pub mod net;
>  pub mod page;
> +pub mod pid_namespace;
>  pub mod prelude;
>  pub mod print;
>  pub mod sizes;
> diff --git a/rust/kernel/pid_namespace.rs b/rust/kernel/pid_namespace.rs
> new file mode 100644
> index 0000000000000000000000000000000000000000..9a0509e802b4939ad853a802ee6d069a5f00c9df
> --- /dev/null
> +++ b/rust/kernel/pid_namespace.rs
> @@ -0,0 +1,70 @@
> +// SPDX-License-Identifier: GPL-2.0
> +
> +// Copyright (c) 2024 Christian Brauner <brauner@kernel.org>
> +
> +//! Pid namespaces.
> +//!
> +//! C header: [`include/linux/pid_namespace.h`](srctree/include/linux/pid_namespace.h) and
> +//! [`include/linux/pid.h`](srctree/include/linux/pid.h)
> +
> +use crate::{
> +    bindings,
> +    types::{AlwaysRefCounted, Opaque},
> +};
> +use core::{
> +    ptr,
> +};

This doesn't pass the rustfmt check.

$ rustfmt --check rust/kernel/pid_namespace.rs
Diff in /home/aliceryhl/rust-for-linux/rust/kernel/pid_namespace.rs:11:
     bindings,
     types::{AlwaysRefCounted, Opaque},
 };
-use core::{
-    ptr,
-};
+use core::ptr;

 /// Wraps the kernel's `struct pid_namespace`. Thread safe.
 ///

> +    /// Returns a PidNamespace reference for the currently executing task's/thread's pid namespace.
> +    ///
> +    /// This function can be used to create an unbounded lifetime by e.g., storing the returned
> +    /// PidNamespace in a global variable which would be a bug. So the recommended way to get the
> +    /// current task's/thread's pid namespace is to use the [`current_pid_ns`] macro because it is
> +    /// safe.
> +    ///
> +    /// # Safety
> +    ///
> +    /// Callers must ensure that the returned object doesn't outlive the current task/thread.
> +    pub unsafe fn current_pid_ns() -> impl Deref<Target = PidNamespace> {
> +        struct PidNamespaceRef<'a> {
> +            task: &'a PidNamespace,
> +            _not_send: NotThreadSafe,
> +        }
> +
> +        impl Deref for PidNamespaceRef<'_> {
> +            type Target = PidNamespace;
> +
> +            fn deref(&self) -> &Self::Target {
> +                self.task
> +            }
> +        }
> +
> +        let pidns = unsafe { bindings::task_active_pid_ns(Task::current_raw()) };
> +        PidNamespaceRef {
> +            // SAFETY: If the current thread is still running, the current task and its associated
> +            // pid namespace are valid. Given that `PidNamespaceRef` is not `Send`, we know it
> +            // cannot be transferred to another thread (where it could potentially outlive the
> +            // current `Task`).
> +            task: unsafe { &*pidns.cast() },

This could use `PidNamespace::from_ptr` instead of the cast.

Also, the safety comment about it not being Send seems incomplete. The
real reason it's okay is that the caller must ensure that the
PidNamespaceRef doesn't outlive the current task/thread.

> +    /// Returns the given task's pid in the provided pid namespace.
> +    pub fn task_tgid_nr_ns(&self, pidns: &PidNamespace) -> Pid {
> +        // SAFETY: We know that `self.0.get()` is valid by the type invariant.
> +        unsafe { bindings::task_tgid_nr_ns(self.0.get(), pidns.as_ptr()) }
>      }

The underlying C function accepts null pointers for the namespace. We
could do the same by accepting `pidns: Option<&PidNamespace>`.

Alice
Re: [PATCH v2] rust: add PidNamespace
Posted by Christian Brauner 1 month, 4 weeks ago
On Tue, Oct 01, 2024 at 12:26:27PM GMT, Alice Ryhl wrote:
> On Tue, Oct 1, 2024 at 11:44 AM Christian Brauner <brauner@kernel.org> wrote:
> >
> > The lifetime of `PidNamespace` is bound to `Task` and `struct pid`.
> >
> > The `PidNamespace` of a `Task` doesn't ever change once the `Task` is
> > alive. A `unshare(CLONE_NEWPID)` or `setns(fd_pidns/pidfd, CLONE_NEWPID)`
> > will not have an effect on the calling `Task`'s pid namespace. It will
> > only effect the pid namespace of children created by the calling `Task`.
> > This invariant guarantees that after having acquired a reference to a
> > `Task`'s pid namespace it will remain unchanged.
> >
> > When a task has exited and been reaped `release_task()` will be called.
> > This will set the `PidNamespace` of the task to `NULL`. So retrieving
> > the `PidNamespace` of a task that is dead will return `NULL`. Note, that
> > neither holding the RCU lock nor holding a referencing count to the
> > `Task` will prevent `release_task()` being called.
> >
> > In order to retrieve the `PidNamespace` of a `Task` the
> > `task_active_pid_ns()` function can be used. There are two cases to
> > consider:
> >
> > (1) retrieving the `PidNamespace` of the `current` task (2) retrieving
> > the `PidNamespace` of a non-`current` task
> >
> > From system call context retrieving the `PidNamespace` for case (1) is
> > always safe and requires neither RCU locking nor a reference count to be
> > held. Retrieving the `PidNamespace` after `release_task()` for current
> > will return `NULL` but no codepath like that is exposed to Rust.
> >
> > Retrieving the `PidNamespace` from system call context for (2) requires
> > RCU protection. Accessing `PidNamespace` outside of RCU protection
> > requires a reference count that must've been acquired while holding the
> > RCU lock. Note that accessing a non-`current` task means `NULL` can be
> > returned as the non-`current` task could have already passed through
> > `release_task()`.
> >
> > To retrieve (1) the `current_pid_ns!()` macro should be used which
> > ensure that the returned `PidNamespace` cannot outlive the calling
> > scope. The associated `current_pid_ns()` function should not be called
> > directly as it could be abused to created an unbounded lifetime for
> > `PidNamespace`. The `current_pid_ns!()` macro allows Rust to handle the
> > common case of accessing `current`'s `PidNamespace` without RCU
> > protection and without having to acquire a reference count.
> >
> > For (2) the `task_get_pid_ns()` method must be used. This will always
> > acquire a reference on `PidNamespace` and will return an `Option` to
> > force the caller to explicitly handle the case where `PidNamespace` is
> > `None`, something that tends to be forgotten when doing the equivalent
> > operation in `C`. Missing RCU primitives make it difficult to perform
> > operations that are otherwise safe without holding a reference count as
> > long as RCU protection is guaranteed. But it is not important currently.
> > But we do want it in the future.
> >
> > Note for (2) the required RCU protection around calling
> > `task_active_pid_ns()` synchronizes against putting the last reference
> > of the associated `struct pid` of `task->thread_pid`. The `struct pid`
> > stored in that field is used to retrieve the `PidNamespace` of the
> > caller. When `release_task()` is called `task->thread_pid` will be
> > `NULL`ed and `put_pid()` on said `struct pid` will be delayed in
> > `free_pid()` via `call_rcu()` allowing everyone with an RCU protected
> > access to the `struct pid` acquired from `task->thread_pid` to finish.
> >
> > Signed-off-by: Christian Brauner <brauner@kernel.org>
> 
> Overall looks good to me, but a few comments below.
> 
> Also, I think it would be fine to send the next version without it
> being a reply to the file bindings thread.
> 
> >  rust/helpers/helpers.c       |   1 +
> >  rust/helpers/pid_namespace.c |  26 ++++++++++
> >  rust/kernel/lib.rs           |   1 +
> >  rust/kernel/pid_namespace.rs |  70 +++++++++++++++++++++++++
> >  rust/kernel/task.rs          | 119 ++++++++++++++++++++++++++++++++++++++++---
> >  5 files changed, 211 insertions(+), 6 deletions(-)
> >
> > diff --git a/rust/helpers/helpers.c b/rust/helpers/helpers.c
> > index 62022b18caf5ec17231fd0e7be1234592d1146e3..d553ad9361ce17950d505c3b372a568730020e2f 100644
> > --- a/rust/helpers/helpers.c
> > +++ b/rust/helpers/helpers.c
> > @@ -17,6 +17,7 @@
> >  #include "kunit.c"
> >  #include "mutex.c"
> >  #include "page.c"
> > +#include "pid_namespace.c"
> >  #include "rbtree.c"
> >  #include "refcount.c"
> >  #include "security.c"
> > diff --git a/rust/helpers/pid_namespace.c b/rust/helpers/pid_namespace.c
> > new file mode 100644
> > index 0000000000000000000000000000000000000000..f41482bdec9a7c4e84b81ec141027fbd65251230
> > --- /dev/null
> > +++ b/rust/helpers/pid_namespace.c
> > @@ -0,0 +1,26 @@
> > +// SPDX-License-Identifier: GPL-2.0
> > +
> > +#include <linux/pid_namespace.h>
> > +#include <linux/cleanup.h>
> > +
> > +struct pid_namespace *rust_helper_get_pid_ns(struct pid_namespace *ns)
> > +{
> > +       return get_pid_ns(ns);
> > +}
> > +
> > +void rust_helper_put_pid_ns(struct pid_namespace *ns)
> > +{
> > +       put_pid_ns(ns);
> > +}
> > +
> > +/* Get a reference on a task's pid namespace. */
> > +struct pid_namespace *rust_helper_task_get_pid_ns(struct task_struct *task)
> > +{
> > +       struct pid_namespace *pid_ns;
> > +
> > +       guard(rcu)();
> > +       pid_ns = task_active_pid_ns(task);
> > +       if (pid_ns)
> > +               get_pid_ns(pid_ns);
> > +       return pid_ns;
> > +}
> > diff --git a/rust/kernel/lib.rs b/rust/kernel/lib.rs
> > index ff7d88022c57ca232dc028066dfa062f3fc84d1c..0e78ec9d06e0199dfafc40988a2ae86cd5df949c 100644
> > --- a/rust/kernel/lib.rs
> > +++ b/rust/kernel/lib.rs
> > @@ -44,6 +44,7 @@
> >  #[cfg(CONFIG_NET)]
> >  pub mod net;
> >  pub mod page;
> > +pub mod pid_namespace;
> >  pub mod prelude;
> >  pub mod print;
> >  pub mod sizes;
> > diff --git a/rust/kernel/pid_namespace.rs b/rust/kernel/pid_namespace.rs
> > new file mode 100644
> > index 0000000000000000000000000000000000000000..9a0509e802b4939ad853a802ee6d069a5f00c9df
> > --- /dev/null
> > +++ b/rust/kernel/pid_namespace.rs
> > @@ -0,0 +1,70 @@
> > +// SPDX-License-Identifier: GPL-2.0
> > +
> > +// Copyright (c) 2024 Christian Brauner <brauner@kernel.org>
> > +
> > +//! Pid namespaces.
> > +//!
> > +//! C header: [`include/linux/pid_namespace.h`](srctree/include/linux/pid_namespace.h) and
> > +//! [`include/linux/pid.h`](srctree/include/linux/pid.h)
> > +
> > +use crate::{
> > +    bindings,
> > +    types::{AlwaysRefCounted, Opaque},
> > +};
> > +use core::{
> > +    ptr,
> > +};
> 
> This doesn't pass the rustfmt check.

Ok. Why does it pass the build then? Seems like it should just fail the build.

> 
> $ rustfmt --check rust/kernel/pid_namespace.rs
> Diff in /home/aliceryhl/rust-for-linux/rust/kernel/pid_namespace.rs:11:
>      bindings,
>      types::{AlwaysRefCounted, Opaque},
>  };
> -use core::{
> -    ptr,
> -};
> +use core::ptr;
> 
>  /// Wraps the kernel's `struct pid_namespace`. Thread safe.
>  ///
> 
> > +    /// Returns a PidNamespace reference for the currently executing task's/thread's pid namespace.
> > +    ///
> > +    /// This function can be used to create an unbounded lifetime by e.g., storing the returned
> > +    /// PidNamespace in a global variable which would be a bug. So the recommended way to get the
> > +    /// current task's/thread's pid namespace is to use the [`current_pid_ns`] macro because it is
> > +    /// safe.
> > +    ///
> > +    /// # Safety
> > +    ///
> > +    /// Callers must ensure that the returned object doesn't outlive the current task/thread.
> > +    pub unsafe fn current_pid_ns() -> impl Deref<Target = PidNamespace> {
> > +        struct PidNamespaceRef<'a> {
> > +            task: &'a PidNamespace,
> > +            _not_send: NotThreadSafe,
> > +        }
> > +
> > +        impl Deref for PidNamespaceRef<'_> {
> > +            type Target = PidNamespace;
> > +
> > +            fn deref(&self) -> &Self::Target {
> > +                self.task
> > +            }
> > +        }
> > +
> > +        let pidns = unsafe { bindings::task_active_pid_ns(Task::current_raw()) };
> > +        PidNamespaceRef {
> > +            // SAFETY: If the current thread is still running, the current task and its associated
> > +            // pid namespace are valid. Given that `PidNamespaceRef` is not `Send`, we know it
> > +            // cannot be transferred to another thread (where it could potentially outlive the
> > +            // current `Task`).
> > +            task: unsafe { &*pidns.cast() },
> 
> This could use `PidNamespace::from_ptr` instead of the cast.

Ok.

> Also, the safety comment about it not being Send seems incomplete. The
> real reason it's okay is that the caller must ensure that the
> PidNamespaceRef doesn't outlive the current task/thread.

Right, but that already documented at the top of the function.

> 
> > +    /// Returns the given task's pid in the provided pid namespace.
> > +    pub fn task_tgid_nr_ns(&self, pidns: &PidNamespace) -> Pid {
> > +        // SAFETY: We know that `self.0.get()` is valid by the type invariant.
> > +        unsafe { bindings::task_tgid_nr_ns(self.0.get(), pidns.as_ptr()) }
> >      }
> 
> The underlying C function accepts null pointers for the namespace. We
> could do the same by accepting `pidns: Option<&PidNamespace>`.

Seems fine.
Re: [PATCH v2] rust: add PidNamespace
Posted by Miguel Ojeda 1 month, 4 weeks ago
On Tue, Oct 1, 2024 at 4:17 PM Christian Brauner <brauner@kernel.org> wrote:
>
> Ok. Why does it pass the build then? Seems like it should just fail the build.

It is part of `make rustfmt` / `make rustfmtcheck`.

I would be happy to make it part of the normal build if people agree
-- though it could be annoying in some cases, e.g. iterating small
changes while developing.

If we do that, it would be nice if -next does it too, but I think
Stephen is already building Rust for x86_64 allmodconfig (Cc'd).

Cheers,
Miguel
Re: [PATCH v2] rust: add PidNamespace
Posted by Christian Brauner 1 month, 3 weeks ago
On Tue, Oct 01, 2024 at 05:45:15PM GMT, Miguel Ojeda wrote:
> On Tue, Oct 1, 2024 at 4:17 PM Christian Brauner <brauner@kernel.org> wrote:
> >
> > Ok. Why does it pass the build then? Seems like it should just fail the build.
> 
> It is part of `make rustfmt` / `make rustfmtcheck`.
> 
> I would be happy to make it part of the normal build if people agree
> -- though it could be annoying in some cases, e.g. iterating small
> changes while developing.

You could consider adding a way to turn it off then instead of turning
it on.

> 
> If we do that, it would be nice if -next does it too, but I think
> Stephen is already building Rust for x86_64 allmodconfig (Cc'd).

Imho, since Rust enforces code formatting style I see no point in not
immediately failing the build because of formatting issues.
Re: [PATCH v2] rust: add PidNamespace
Posted by Miguel Ojeda 1 month, 3 weeks ago
On Wed, Oct 2, 2024 at 12:14 PM Christian Brauner <brauner@kernel.org> wrote:
>
> You could consider adding a way to turn it off then instead of turning
> it on.
>
> Imho, since Rust enforces code formatting style I see no point in not
> immediately failing the build because of formatting issues.

For maintainers, it would be better if we could unconditionally do it,
but like with other diagnostics, it is a balance.

If there is a way out (like something like `WERROR` or perhaps a "dev
mode" like `make D=1` that could encompass other bits), then I think
it should be OK. Any preference?

(We also need to be careful about `rustfmt` having e.g. bugs in future
versions that change the output, but since we are in the Rust CI and
we can test the nightly compiler, the risk should be low.)

Cheers,
Miguel