linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH 1/2] seccomp: notify user trap about unused filter
@ 2020-05-27 11:19 Christian Brauner
  2020-05-27 11:19 ` [PATCH 2/2] tests: test seccomp filter notifications Christian Brauner
                   ` (4 more replies)
  0 siblings, 5 replies; 22+ messages in thread
From: Christian Brauner @ 2020-05-27 11:19 UTC (permalink / raw)
  To: linux-kernel, Kees Cook
  Cc: Andy Lutomirski, Tycho Andersen, Matt Denton, Sargun Dhillon,
	Jann Horn, Chris Palmer, Aleksa Sarai, Robert Sesek,
	Jeffrey Vander Stoep, Linux Containers, Christian Brauner

We've been making heavy use of the seccomp notifier to intercept and
handle certain syscalls for containers. This patch allows a syscall
supervisor listening on a given notifier to be notified when a seccomp
filter has become unused.

A container is often managed by a singleton supervisor process the
so-called "monitor". This monitor process has an event loop which has
various event handlers registered. If the user specified a seccomp
profile that included a notifier for various syscalls then we also
register a seccomp notify even handler. For any container using a
separate pid namespace the lifecycle of the seccomp notifier is bound to
the init process of the pid namespace, i.e. when the init process exits
the filter must be unused.
If a new process attaches to a container we force it to assume a seccomp
profile. This can either be the same seccomp profile as the container
was started with or a modified one. If the attaching process makes use
of the seccomp notifier we will register a new seccomp notifier handler
in the monitor's event loop. However, when the attaching process exits
we can't simply delete the handler since other child processes could've
been created (daemons spawned etc.) that have inherited the seccomp
filter and so we need to keep the seccomp notifier fd alive in the event
loop. But this is problematic since we don't get a notification when the
seccomp filter has become unused and so we currently never remove the
seccomp notifier fd from the event loop and just keep accumulating fds
in the event loop. We've had this issue for a while but it has recently
become more pressing as more and larger users make use of this.

To fix this, we introduce a new "live" reference counter that tracks the
live tasks making use of a given filter and when a notifier is
registered waiting tasks will be notified that the filter is now empty
by receiving a (E)POLLHUP event.
The concept in this patch introduces is the same as for signal_struct,
i.e. reference counting for life-cycle management is decoupled from
reference counting live taks using the object.

There's probably some trickery possible but the second counter is just
the correct way of doing this imho and has precedence. The patch also
lifts the waitqeue from struct notification into into sruct
seccomp_filter. This is cleaner overall and let's us avoid having to
take the notifier mutex since we neither need to read nor modify the
notifier specific aspects of the seccomp filter. In the exit path I'd
very much like to avoid having to take the notifier mutex for each
filter in the task's filter hierarchy.

Cc: Tycho Andersen <tycho@tycho.ws>
Cc: Kees Cook <keescook@chromium.org>
Cc: Matt Denton <mpdenton@google.com>
Cc: Sargun Dhillon <sargun@sargun.me>
Cc: Jann Horn <jannh@google.com>
Cc: Chris Palmer <palmer@google.com>
Cc: Aleksa Sarai <cyphar@cyphar.com>
Cc: Robert Sesek <rsesek@google.com>
Cc: Jeffrey Vander Stoep <jeffv@google.com>
Cc: Linux Containers <containers@lists.linux-foundation.org>
Signed-off-by: Christian Brauner <christian.brauner@ubuntu.com>
---
 include/linux/seccomp.h |  5 +++++
 kernel/exit.c           |  2 ++
 kernel/seccomp.c        | 32 ++++++++++++++++++++++++++------
 3 files changed, 33 insertions(+), 6 deletions(-)

diff --git a/include/linux/seccomp.h b/include/linux/seccomp.h
index 4192369b8418..5935746de2b9 100644
--- a/include/linux/seccomp.h
+++ b/include/linux/seccomp.h
@@ -84,6 +84,7 @@ static inline int seccomp_mode(struct seccomp *s)
 #ifdef CONFIG_SECCOMP_FILTER
 extern void put_seccomp_filter(struct task_struct *tsk);
 extern void get_seccomp_filter(struct task_struct *tsk);
+extern void seccomp_filter_notify(const struct task_struct *tsk);
 #else  /* CONFIG_SECCOMP_FILTER */
 static inline void put_seccomp_filter(struct task_struct *tsk)
 {
@@ -93,6 +94,10 @@ static inline void get_seccomp_filter(struct task_struct *tsk)
 {
 	return;
 }
+static inline void seccomp_filter_notify(const struct task_struct *tsk)
+{
+	return;
+}
 #endif /* CONFIG_SECCOMP_FILTER */
 
 #if defined(CONFIG_SECCOMP_FILTER) && defined(CONFIG_CHECKPOINT_RESTORE)
diff --git a/kernel/exit.c b/kernel/exit.c
index ce2a75bc0ade..90fe507e1459 100644
--- a/kernel/exit.c
+++ b/kernel/exit.c
@@ -193,6 +193,8 @@ void release_task(struct task_struct *p)
 
 	cgroup_release(p);
 
+	seccomp_filter_notify(p);
+
 	write_lock_irq(&tasklist_lock);
 	ptrace_release_task(p);
 	thread_pid = get_pid(p->thread_pid);
diff --git a/kernel/seccomp.c b/kernel/seccomp.c
index 55a6184f5990..6c5760acff29 100644
--- a/kernel/seccomp.c
+++ b/kernel/seccomp.c
@@ -94,13 +94,11 @@ struct seccomp_knotif {
  *           filter->notify_lock.
  * @next_id: The id of the next request.
  * @notifications: A list of struct seccomp_knotif elements.
- * @wqh: A wait queue for poll.
  */
 struct notification {
 	struct semaphore request;
 	u64 next_id;
 	struct list_head notifications;
-	wait_queue_head_t wqh;
 };
 
 /**
@@ -115,6 +113,9 @@ struct notification {
  * @prog: the BPF program to evaluate
  * @notif: the struct that holds all notification related information
  * @notify_lock: A lock for all notification-related accesses.
+ * @wqh: A wait queue for poll if a notifier is in use.
+ * @live: tasks that actually use this filter, only to be altered
+ *          during fork(), exit()/free_task(), and filter installation
  *
  * seccomp_filter objects are organized in a tree linked via the @prev
  * pointer.  For any task, it appears to be a singly-linked list starting
@@ -133,6 +134,8 @@ struct seccomp_filter {
 	struct bpf_prog *prog;
 	struct notification *notif;
 	struct mutex notify_lock;
+	refcount_t live;
+	wait_queue_head_t wqh;
 };
 
 /* Limit any path through the tree to 256KB worth of instructions. */
@@ -396,6 +399,7 @@ static inline void seccomp_sync_threads(unsigned long flags)
 		 * allows a put before the assignment.)
 		 */
 		put_seccomp_filter(thread);
+		seccomp_filter_notify(thread);
 		smp_store_release(&thread->seccomp.filter,
 				  caller->seccomp.filter);
 
@@ -462,6 +466,8 @@ static struct seccomp_filter *seccomp_prepare_filter(struct sock_fprog *fprog)
 	}
 
 	refcount_set(&sfilter->usage, 1);
+	refcount_set(&sfilter->live, 1);
+	init_waitqueue_head(&sfilter->wqh);
 
 	return sfilter;
 }
@@ -564,6 +570,7 @@ void get_seccomp_filter(struct task_struct *tsk)
 	if (!orig)
 		return;
 	__get_seccomp_filter(orig);
+	refcount_inc(&orig->live);
 }
 
 static inline void seccomp_filter_free(struct seccomp_filter *filter)
@@ -590,6 +597,17 @@ void put_seccomp_filter(struct task_struct *tsk)
 	__put_seccomp_filter(tsk->seccomp.filter);
 }
 
+void seccomp_filter_notify(const struct task_struct *tsk)
+{
+	struct seccomp_filter *orig = tsk->seccomp.filter;
+
+	while (orig && refcount_dec_and_test(&orig->live)) {
+		if (waitqueue_active(&orig->wqh))
+			wake_up_poll(&orig->wqh, EPOLLHUP);
+		orig = orig->prev;
+	}
+}
+
 static void seccomp_init_siginfo(kernel_siginfo_t *info, int syscall, int reason)
 {
 	clear_siginfo(info);
@@ -757,7 +775,7 @@ static int seccomp_do_user_notification(int this_syscall,
 	list_add(&n.list, &match->notif->notifications);
 
 	up(&match->notif->request);
-	wake_up_poll(&match->notif->wqh, EPOLLIN | EPOLLRDNORM);
+	wake_up_poll(&match->wqh, EPOLLIN | EPOLLRDNORM);
 	mutex_unlock(&match->notify_lock);
 
 	/*
@@ -1064,7 +1082,7 @@ static long seccomp_notify_recv(struct seccomp_filter *filter,
 	unotif.data = *(knotif->data);
 
 	knotif->state = SECCOMP_NOTIFY_SENT;
-	wake_up_poll(&filter->notif->wqh, EPOLLOUT | EPOLLWRNORM);
+	wake_up_poll(&filter->wqh, EPOLLOUT | EPOLLWRNORM);
 	ret = 0;
 out:
 	mutex_unlock(&filter->notify_lock);
@@ -1200,7 +1218,7 @@ static __poll_t seccomp_notify_poll(struct file *file,
 	__poll_t ret = 0;
 	struct seccomp_knotif *cur;
 
-	poll_wait(file, &filter->notif->wqh, poll_tab);
+	poll_wait(file, &filter->wqh, poll_tab);
 
 	if (mutex_lock_interruptible(&filter->notify_lock) < 0)
 		return EPOLLERR;
@@ -1216,6 +1234,9 @@ static __poll_t seccomp_notify_poll(struct file *file,
 
 	mutex_unlock(&filter->notify_lock);
 
+	if (refcount_read(&filter->live) == 0)
+		ret |= EPOLLHUP;
+
 	return ret;
 }
 
@@ -1244,7 +1265,6 @@ static struct file *init_listener(struct seccomp_filter *filter)
 	sema_init(&filter->notif->request, 0);
 	filter->notif->next_id = get_random_u64();
 	INIT_LIST_HEAD(&filter->notif->notifications);
-	init_waitqueue_head(&filter->notif->wqh);
 
 	ret = anon_inode_getfile("seccomp notify", &seccomp_notify_ops,
 				 filter, O_RDWR);

base-commit: b9bbe6ed63b2b9f2c9ee5cbd0f2c946a2723f4ce
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 22+ messages in thread

* [PATCH 2/2] tests: test seccomp filter notifications
  2020-05-27 11:19 [PATCH 1/2] seccomp: notify user trap about unused filter Christian Brauner
@ 2020-05-27 11:19 ` Christian Brauner
  2020-05-27 15:25 ` [PATCH 1/2] seccomp: notify user trap about unused filter Tycho Andersen
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 22+ messages in thread
From: Christian Brauner @ 2020-05-27 11:19 UTC (permalink / raw)
  To: linux-kernel, Kees Cook
  Cc: Andy Lutomirski, Tycho Andersen, Matt Denton, Sargun Dhillon,
	Jann Horn, Chris Palmer, Aleksa Sarai, Robert Sesek,
	Jeffrey Vander Stoep, Linux Containers, Christian Brauner

This verifies we're correctly notified when a seccomp filter becomes
unused when a notifier is in use.

Signed-off-by: Christian Brauner <christian.brauner@ubuntu.com>
---
 tools/testing/selftests/seccomp/seccomp_bpf.c | 137 ++++++++++++++++++
 1 file changed, 137 insertions(+)

diff --git a/tools/testing/selftests/seccomp/seccomp_bpf.c b/tools/testing/selftests/seccomp/seccomp_bpf.c
index c0aa46ce14f6..472b93b1c3d5 100644
--- a/tools/testing/selftests/seccomp/seccomp_bpf.c
+++ b/tools/testing/selftests/seccomp/seccomp_bpf.c
@@ -45,12 +45,14 @@
 #include <sys/socket.h>
 #include <sys/ioctl.h>
 #include <linux/kcmp.h>
+#include <sys/epoll.h>
 
 #include <unistd.h>
 #include <sys/syscall.h>
 #include <poll.h>
 
 #include "../kselftest_harness.h"
+#include "../clone3/clone3_selftests.h"
 
 #ifndef PR_SET_PTRACER
 # define PR_SET_PTRACER 0x59616d61
@@ -3686,6 +3688,141 @@ TEST(user_notification_continue)
 	}
 }
 
+TEST(user_notification_filter_empty)
+{
+	pid_t pid;
+	long ret;
+	int status;
+	struct pollfd pollfd;
+	struct clone_args args = {
+		.flags = CLONE_FILES,
+		.exit_signal = SIGCHLD,
+	};
+
+	ret = prctl(PR_SET_NO_NEW_PRIVS, 1, 0, 0, 0);
+	ASSERT_EQ(0, ret) {
+		TH_LOG("Kernel does not support PR_SET_NO_NEW_PRIVS!");
+	}
+
+	pid = sys_clone3(&args, sizeof(args));
+	ASSERT_GE(pid, 0);
+
+	if (pid == 0) {
+		int listener;
+
+		listener = user_trap_syscall(__NR_mknod, SECCOMP_FILTER_FLAG_NEW_LISTENER);
+		if (listener < 0)
+			_exit(EXIT_FAILURE);
+
+		if (dup2(listener, 200) != 200)
+			_exit(EXIT_FAILURE);
+
+		close(listener);
+
+		_exit(EXIT_SUCCESS);
+	}
+
+	EXPECT_EQ(waitpid(pid, &status, 0), pid);
+	EXPECT_EQ(true, WIFEXITED(status));
+	EXPECT_EQ(0, WEXITSTATUS(status));
+
+	/*
+	 * The seccomp filter has become unused so we should be notified once
+	 * the kernel gets around to cleaning up task struct.
+	 */
+	pollfd.fd = 200;
+	pollfd.events = POLLHUP;
+
+	EXPECT_GT(poll(&pollfd, 1, -1), 0);
+	EXPECT_GT((pollfd.revents & POLLHUP) ?: 0, 0);
+}
+
+static void *do_thread(void *data)
+{
+	return NULL;
+}
+
+TEST(user_notification_filter_empty_threaded)
+{
+	pid_t pid;
+	long ret;
+	int status;
+	struct pollfd pollfd;
+	struct clone_args args = {
+		.flags = CLONE_FILES,
+		.exit_signal = SIGCHLD,
+	};
+
+	ret = prctl(PR_SET_NO_NEW_PRIVS, 1, 0, 0, 0);
+	ASSERT_EQ(0, ret) {
+		TH_LOG("Kernel does not support PR_SET_NO_NEW_PRIVS!");
+	}
+
+	pid = sys_clone3(&args, sizeof(args));
+	ASSERT_GE(pid, 0);
+
+	if (pid == 0) {
+		pid_t pid1, pid2;
+		int listener, status;
+		pthread_t thread;
+
+		listener = user_trap_syscall(__NR_dup, SECCOMP_FILTER_FLAG_NEW_LISTENER);
+		if (listener < 0)
+			_exit(EXIT_FAILURE);
+
+		if (dup2(listener, 200) != 200)
+			_exit(EXIT_FAILURE);
+
+		close(listener);
+
+		pid1 = fork();
+		if (pid1 < 0)
+			_exit(EXIT_FAILURE);
+
+		if (pid1 == 0)
+			_exit(EXIT_SUCCESS);
+
+		pid2 = fork();
+		if (pid2 < 0)
+			_exit(EXIT_FAILURE);
+
+		if (pid2 == 0)
+			_exit(EXIT_SUCCESS);
+
+		if (pthread_create(&thread, NULL, do_thread, NULL) ||
+		    pthread_join(thread, NULL))
+			_exit(EXIT_FAILURE);
+
+		if (pthread_create(&thread, NULL, do_thread, NULL) ||
+		    pthread_join(thread, NULL))
+			_exit(EXIT_FAILURE);
+
+		if (waitpid(pid1, &status, 0) != pid1 || !WIFEXITED(status) ||
+		    WEXITSTATUS(status))
+			_exit(EXIT_FAILURE);
+
+		if (waitpid(pid2, &status, 0) != pid2 || !WIFEXITED(status) ||
+		    WEXITSTATUS(status))
+			_exit(EXIT_FAILURE);
+
+		exit(EXIT_SUCCESS);
+	}
+
+	EXPECT_EQ(waitpid(pid, &status, 0), pid);
+	EXPECT_EQ(true, WIFEXITED(status));
+	EXPECT_EQ(0, WEXITSTATUS(status));
+
+	/*
+	 * The seccomp filter has become unused so we should be notified once
+	 * the kernel gets around to cleaning up task struct.
+	 */
+	pollfd.fd = 200;
+	pollfd.events = POLLHUP;
+
+	EXPECT_GT(poll(&pollfd, 1, -1), 0);
+	EXPECT_GT((pollfd.revents & POLLHUP) ?: 0, 0);
+}
+
 /*
  * TODO:
  * - add microbenchmarks
-- 
2.26.2


^ permalink raw reply related	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 11:19 [PATCH 1/2] seccomp: notify user trap about unused filter Christian Brauner
  2020-05-27 11:19 ` [PATCH 2/2] tests: test seccomp filter notifications Christian Brauner
@ 2020-05-27 15:25 ` Tycho Andersen
  2020-05-27 15:35   ` Christian Brauner
  2020-05-27 17:37 ` Sargun Dhillon
                   ` (2 subsequent siblings)
  4 siblings, 1 reply; 22+ messages in thread
From: Tycho Andersen @ 2020-05-27 15:25 UTC (permalink / raw)
  To: Christian Brauner
  Cc: linux-kernel, Kees Cook, Andy Lutomirski, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 01:19:01PM +0200, Christian Brauner wrote:
> +void seccomp_filter_notify(const struct task_struct *tsk)
> +{
> +	struct seccomp_filter *orig = tsk->seccomp.filter;
> +
> +	while (orig && refcount_dec_and_test(&orig->live)) {
> +		if (waitqueue_active(&orig->wqh))
> +			wake_up_poll(&orig->wqh, EPOLLHUP);
> +		orig = orig->prev;
> +	}
> +}

Is there a reason this can't live in put_seccomp_filter()?

Tycho

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 15:25 ` [PATCH 1/2] seccomp: notify user trap about unused filter Tycho Andersen
@ 2020-05-27 15:35   ` Christian Brauner
  0 siblings, 0 replies; 22+ messages in thread
From: Christian Brauner @ 2020-05-27 15:35 UTC (permalink / raw)
  To: Tycho Andersen
  Cc: linux-kernel, Kees Cook, Andy Lutomirski, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 09:25:59AM -0600, Tycho Andersen wrote:
> On Wed, May 27, 2020 at 01:19:01PM +0200, Christian Brauner wrote:
> > +void seccomp_filter_notify(const struct task_struct *tsk)
> > +{
> > +	struct seccomp_filter *orig = tsk->seccomp.filter;
> > +
> > +	while (orig && refcount_dec_and_test(&orig->live)) {
> > +		if (waitqueue_active(&orig->wqh))
> > +			wake_up_poll(&orig->wqh, EPOLLHUP);
> > +		orig = orig->prev;
> > +	}
> > +}
> 
> Is there a reason this can't live in put_seccomp_filter()?

put_seccomp_filter() is called from free_task() which in turn gets
called via delayed_put_task_struct through call_rcu() so the
notification will happen at some point whereas you'd often want the
notification to happen at reliable point. This is why most of that stuff
happens in release_task() which is called in the exit path when the task
is finally marked as dead. This is similar to how cgroup_release() is
called from release_task() whereas cgroup_free() is called from
free_task() or other stuff.

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 11:19 [PATCH 1/2] seccomp: notify user trap about unused filter Christian Brauner
  2020-05-27 11:19 ` [PATCH 2/2] tests: test seccomp filter notifications Christian Brauner
  2020-05-27 15:25 ` [PATCH 1/2] seccomp: notify user trap about unused filter Tycho Andersen
@ 2020-05-27 17:37 ` Sargun Dhillon
  2020-05-27 19:13   ` Christian Brauner
  2020-05-27 21:43 ` Kees Cook
  2020-05-28  4:04 ` Jann Horn
  4 siblings, 1 reply; 22+ messages in thread
From: Sargun Dhillon @ 2020-05-27 17:37 UTC (permalink / raw)
  To: Christian Brauner
  Cc: linux-kernel, Kees Cook, Andy Lutomirski, Tycho Andersen,
	Matt Denton, Jann Horn, Chris Palmer, Aleksa Sarai, Robert Sesek,
	Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 01:19:01PM +0200, Christian Brauner wrote:
> +void seccomp_filter_notify(const struct task_struct *tsk)
> +{
> +	struct seccomp_filter *orig = tsk->seccomp.filter;
> +
> +	while (orig && refcount_dec_and_test(&orig->live)) {
> +		if (waitqueue_active(&orig->wqh))
> +			wake_up_poll(&orig->wqh, EPOLLHUP);
> +		orig = orig->prev;
> +	}
> +}
> +
Any reason not to write this as:
for (orig = tsk->seccomp.filter; refcount_dec_and_test(&orig->live); orig = orig->prev)?

Also, for those of us who are plumbing in the likes of Go code into the
listener, where we don't have direct access to the epoll interface (at
least not out of the box), what do you think about exposing this on the RECV
ioctl? Or, do you think we should lump that into the "v2" receive API?

Either way, this seems useful, as right now, we're intertwining process
tree lifetime with manager lifetime. This seems cleaner.


^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 17:37 ` Sargun Dhillon
@ 2020-05-27 19:13   ` Christian Brauner
  0 siblings, 0 replies; 22+ messages in thread
From: Christian Brauner @ 2020-05-27 19:13 UTC (permalink / raw)
  To: Sargun Dhillon
  Cc: Kees Cook, Chris Palmer, Jann Horn, Robert Sesek,
	Jeffrey Vander Stoep, Linux Containers, linux-kernel,
	Matt Denton, Andy Lutomirski

On Wed, May 27, 2020 at 05:37:07PM +0000, Sargun Dhillon wrote:
> On Wed, May 27, 2020 at 01:19:01PM +0200, Christian Brauner wrote:
> > +void seccomp_filter_notify(const struct task_struct *tsk)
> > +{
> > +	struct seccomp_filter *orig = tsk->seccomp.filter;
> > +
> > +	while (orig && refcount_dec_and_test(&orig->live)) {
> > +		if (waitqueue_active(&orig->wqh))
> > +			wake_up_poll(&orig->wqh, EPOLLHUP);
> > +		orig = orig->prev;
> > +	}
> > +}
> > +
> Any reason not to write this as:
> for (orig = tsk->seccomp.filter; refcount_dec_and_test(&orig->live); orig = orig->prev)?

Mainly to follow coding style if you look at:

static void __put_seccomp_filter(struct seccomp_filter *orig)
{
	/* Clean up single-reference branches iteratively. */
	while (orig && refcount_dec_and_test(&orig->usage)) {
		struct seccomp_filter *freeme = orig;
		orig = orig->prev;
		seccomp_filter_free(freeme);
	}
}

seemed easier to just have a visual correspondence between those two
codepaths.

> 
> Also, for those of us who are plumbing in the likes of Go code into the
> listener, where we don't have direct access to the epoll interface (at
> least not out of the box), what do you think about exposing this on the RECV

I think requiring users to import
golang.org/x/sys/unix
is reasonable. You'll need to special case this to linux builds anyway
even if you have a client or some such that build on on-unixes. And even
if you don't want to import there's always the possibility to use cgo. :)

> ioctl? Or, do you think we should lump that into the "v2" receive API?

I'm confused how you want to plumb this into the ioctl. That seems
unpleasant and against usual poll/wait semantics. I'm now also wondering
how you're using this whole interface without poll. The idea is to wait
until you're notified you can receive.

> 
> Either way, this seems useful, as right now, we're intertwining process
> tree lifetime with manager lifetime. This seems cleaner.

Cool.
Christian

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 11:19 [PATCH 1/2] seccomp: notify user trap about unused filter Christian Brauner
                   ` (2 preceding siblings ...)
  2020-05-27 17:37 ` Sargun Dhillon
@ 2020-05-27 21:43 ` Kees Cook
  2020-05-27 21:52   ` Tycho Andersen
  2020-05-27 22:05   ` Christian Brauner
  2020-05-28  4:04 ` Jann Horn
  4 siblings, 2 replies; 22+ messages in thread
From: Kees Cook @ 2020-05-27 21:43 UTC (permalink / raw)
  To: Christian Brauner
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 01:19:01PM +0200, Christian Brauner wrote:
> loop. But this is problematic since we don't get a notification when the
> seccomp filter has become unused and so we currently never remove the
> seccomp notifier fd from the event loop and just keep accumulating fds
> in the event loop. We've had this issue for a while but it has recently
> become more pressing as more and larger users make use of this.

I had been under the (seemingly very wrong) understanding that when
all the tasks associated with a filter cease to exist, the notif fd is
effectively closed. But I see now where I got confused: this is only
half implemented: if the userspace end of the fd is closed, it'll get
cleaned up in the kernel, but we have nothing going the other direction
except the general object lifetime management on the filter itself.

So, yes, I accept the basic problem statement, "we have fds hanging
around that will never be used again, we need to notice that". :)

Why is EPOLLHUP needed? Can't the fd just get closed on the kernel end?
I would expect that to be visible as EPOLLHUP internally (though I
haven't looked through the VFS yet). And I haven't found how to
close/detach a anon file from the listener task. It strikes me that this
would actually be much cleaner: then we actually don't need the
additional __get_seccomp_filter() in init_listener() -- we simply
invalidate the file during __put_seccomp_filter().

(While I'm here -- why can there be only one listener per task? The
notifications are filter-specific, not task-specific?)

> To fix this, we introduce a new "live" reference counter that tracks the
> live tasks making use of a given filter and when a notifier is
> registered waiting tasks will be notified that the filter is now empty
> by receiving a (E)POLLHUP event.
> The concept in this patch introduces is the same as for signal_struct,
> i.e. reference counting for life-cycle management is decoupled from
> reference counting live taks using the object.

I will need convincing that life-cycle ref-counting needs to be decoupled
from usage ref-counting. I see what you're saying here and in the other
reply about where the notification is coming from (release vs put, etc),
but I think it'd be better if the EPOLLHUP was handled internally to the
VFS due to the kernel end of the file being closed.

> There's probably some trickery possible but the second counter is just
> the correct way of doing this imho and has precedence. The patch also
> lifts the waitqeue from struct notification into into sruct
> seccomp_filter. This is cleaner overall and let's us avoid having to
> take the notifier mutex since we neither need to read nor modify the
> notifier specific aspects of the seccomp filter. In the exit path I'd
> very much like to avoid having to take the notifier mutex for each
> filter in the task's filter hierarchy.

I guess this is a minor size/speed trade-off (every seccomp_filter
struct grows by 1 pointer regardless of the presence of USER_NOTIF
rules attached...). But I think this is an optimization detail, and I
need to understand why we can't just close the file on filter free.

-- 
Kees Cook

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 21:43 ` Kees Cook
@ 2020-05-27 21:52   ` Tycho Andersen
  2020-05-27 22:36     ` Kees Cook
  2020-05-27 22:05   ` Christian Brauner
  1 sibling, 1 reply; 22+ messages in thread
From: Tycho Andersen @ 2020-05-27 21:52 UTC (permalink / raw)
  To: Kees Cook
  Cc: Christian Brauner, linux-kernel, Andy Lutomirski, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 02:43:49PM -0700, Kees Cook wrote:
> (While I'm here -- why can there be only one listener per task? The
> notifications are filter-specific, not task-specific?)

Not sure what you mean here?

> > To fix this, we introduce a new "live" reference counter that tracks the
> > live tasks making use of a given filter and when a notifier is
> > registered waiting tasks will be notified that the filter is now empty
> > by receiving a (E)POLLHUP event.
> > The concept in this patch introduces is the same as for signal_struct,
> > i.e. reference counting for life-cycle management is decoupled from
> > reference counting live taks using the object.
> 
> I will need convincing that life-cycle ref-counting needs to be decoupled
> from usage ref-counting.

I think it does, since the refcount is no longer 1:1 with the number
of tasks that have it (a notification fd's struct file has a reference
too).

We could also do it the reverse way, and keep track of how many
notification fds point to a particular file. But somehow we need two
counts.

Maybe it's best to decouple them entirely, and have usage go back to
just being the number of tasks, and introduce a new counter for
notification fds.

> I see what you're saying here and in the other
> reply about where the notification is coming from (release vs put, etc),
> but I think it'd be better if the EPOLLHUP was handled internally to the
> VFS due to the kernel end of the file being closed.
> 
> > There's probably some trickery possible but the second counter is just
> > the correct way of doing this imho and has precedence. The patch also
> > lifts the waitqeue from struct notification into into sruct
> > seccomp_filter. This is cleaner overall and let's us avoid having to
> > take the notifier mutex since we neither need to read nor modify the
> > notifier specific aspects of the seccomp filter. In the exit path I'd
> > very much like to avoid having to take the notifier mutex for each
> > filter in the task's filter hierarchy.
> 
> I guess this is a minor size/speed trade-off (every seccomp_filter
> struct grows by 1 pointer regardless of the presence of USER_NOTIF
> rules attached...). But I think this is an optimization detail, and I
> need to understand why we can't just close the file on filter free.

That seems nicest, agreed.

Tycho

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 21:43 ` Kees Cook
  2020-05-27 21:52   ` Tycho Andersen
@ 2020-05-27 22:05   ` Christian Brauner
  2020-05-27 22:37     ` Kees Cook
  1 sibling, 1 reply; 22+ messages in thread
From: Christian Brauner @ 2020-05-27 22:05 UTC (permalink / raw)
  To: Kees Cook
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 02:43:49PM -0700, Kees Cook wrote:
> On Wed, May 27, 2020 at 01:19:01PM +0200, Christian Brauner wrote:
> > loop. But this is problematic since we don't get a notification when the
> > seccomp filter has become unused and so we currently never remove the
> > seccomp notifier fd from the event loop and just keep accumulating fds
> > in the event loop. We've had this issue for a while but it has recently
> > become more pressing as more and larger users make use of this.
> 
> I had been under the (seemingly very wrong) understanding that when
> all the tasks associated with a filter cease to exist, the notif fd is
> effectively closed. But I see now where I got confused: this is only
> half implemented: if the userspace end of the fd is closed, it'll get
> cleaned up in the kernel, but we have nothing going the other direction
> except the general object lifetime management on the filter itself.
> 
> So, yes, I accept the basic problem statement, "we have fds hanging
> around that will never be used again, we need to notice that". :)
> 
> Why is EPOLLHUP needed? Can't the fd just get closed on the kernel end?
> I would expect that to be visible as EPOLLHUP internally (though I
> haven't looked through the VFS yet). And I haven't found how to
> close/detach a anon file from the listener task. It strikes me that this
> would actually be much cleaner: then we actually don't need the
> additional __get_seccomp_filter() in init_listener() -- we simply
> invalidate the file during __put_seccomp_filter().
> 
> (While I'm here -- why can there be only one listener per task? The
> notifications are filter-specific, not task-specific?)
> 
> > To fix this, we introduce a new "live" reference counter that tracks the
> > live tasks making use of a given filter and when a notifier is
> > registered waiting tasks will be notified that the filter is now empty
> > by receiving a (E)POLLHUP event.
> > The concept in this patch introduces is the same as for signal_struct,
> > i.e. reference counting for life-cycle management is decoupled from
> > reference counting live taks using the object.
> 
> I will need convincing that life-cycle ref-counting needs to be decoupled
> from usage ref-counting. I see what you're saying here and in the other
> reply about where the notification is coming from (release vs put, etc),
> but I think it'd be better if the EPOLLHUP was handled internally to the
> VFS due to the kernel end of the file being closed.
> 
> > There's probably some trickery possible but the second counter is just
> > the correct way of doing this imho and has precedence. The patch also
> > lifts the waitqeue from struct notification into into sruct
> > seccomp_filter. This is cleaner overall and let's us avoid having to
> > take the notifier mutex since we neither need to read nor modify the
> > notifier specific aspects of the seccomp filter. In the exit path I'd
> > very much like to avoid having to take the notifier mutex for each
> > filter in the task's filter hierarchy.
> 
> I guess this is a minor size/speed trade-off (every seccomp_filter
> struct grows by 1 pointer regardless of the presence of USER_NOTIF
> rules attached...). But I think this is an optimization detail, and I
> need to understand why we can't just close the file on filter free.

The usage count is not just notify + tasks, it's also incremented when
e.g. ptrace is inspecting the filter and everytime someone takes a
refrence to it that is not using the filter. So "usage" never had a 1:1
correspondence with tasks in the first place. Filter free can happen way
after any task _used_ that filter, either because of where that filter
free happens due to call_rcu, or because someone is mucking with the
filter but not using it (ptrace etc. pp.). So a separate counter doesn't
seem wrong. We also need it even if we were to do the kernel close thing
you suggested.

The main question also is, is there precedence where the kernel just
closes the file descriptor for userspace behind it's back? I'm not sure
I've heard of this before. That's not how that works afaict; it's also
not how we do pidfds. We don't just close the fd when the task
associated with it goes away, we notify and then userspace can close.

Christian

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 21:52   ` Tycho Andersen
@ 2020-05-27 22:36     ` Kees Cook
  2020-05-27 22:56       ` Tycho Andersen
  0 siblings, 1 reply; 22+ messages in thread
From: Kees Cook @ 2020-05-27 22:36 UTC (permalink / raw)
  To: Tycho Andersen
  Cc: Christian Brauner, linux-kernel, Andy Lutomirski, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 03:52:03PM -0600, Tycho Andersen wrote:
> On Wed, May 27, 2020 at 02:43:49PM -0700, Kees Cook wrote:
> > (While I'm here -- why can there be only one listener per task? The
> > notifications are filter-specific, not task-specific?)
> 
> Not sure what you mean here?

tatic struct file *init_listener(struct seccomp_filter *filter)
{
        struct file *ret = ERR_PTR(-EBUSY);
        struct seccomp_filter *cur;

        for (cur = current->seccomp.filter; cur; cur = cur->prev) {
                if (cur->notif)
                        goto out;
        }

...

        /* Installing a second listener in the chain should EBUSY */
        EXPECT_EQ(user_trap_syscall(__NR_getpid,
                                    SECCOMP_FILTER_FLAG_NEW_LISTENER),
                  -1);
        EXPECT_EQ(errno, EBUSY);


Why does this limit exist? Since the fd is tied to a specific filter,
I don't see conflicts about having multiple USER_NOTIF filters on one
task -- the monitor's response will either fake it or continue it, so
there is no "composition" needed? I must be missing something.

> Maybe it's best to decouple them entirely, and have usage go back to
> just being the number of tasks, and introduce a new counter for
> notification fds.

But isn't that already tracked by the VFS? i.e. there is a one-to-one
mapping from the "struct file *" returned by "anon_inode_getfile()" and
the "struct filter" (though we do not presently save it in the filter)
and the VFS tracks how many userspace fds are attached to that struct
file via ->f_count (i.e. f_count reaching zero is what triggers calling
seccomp_notify_release()).

In trying to write up an example patch for this, though, yeah, I don't
see how to do the locking. There is the "file" part, which is effectively
held by both any task's fd table and by the seccomp filter.

I suspect the issue is that the private_data can't be the
seccomp_filter. The "validity" of the mapping between kernel and user
needs to be tracked externally:

struct seccomp_notification_pipe
{
	struct filter *filter;
	struct file *file;
};

But I still can't see where to put the lock or refcount....

-- 
Kees Cook

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 22:05   ` Christian Brauner
@ 2020-05-27 22:37     ` Kees Cook
  2020-05-27 22:45       ` Christian Brauner
  0 siblings, 1 reply; 22+ messages in thread
From: Kees Cook @ 2020-05-27 22:37 UTC (permalink / raw)
  To: Christian Brauner
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Thu, May 28, 2020 at 12:05:32AM +0200, Christian Brauner wrote:
> The main question also is, is there precedence where the kernel just
> closes the file descriptor for userspace behind it's back? I'm not sure
> I've heard of this before. That's not how that works afaict; it's also
> not how we do pidfds. We don't just close the fd when the task
> associated with it goes away, we notify and then userspace can close.

But there's a mapping between pidfd and task struct that is separate
from task struct itself, yes? I.e. keeping a pidfd open doesn't pin
struct task in memory forever, right?

-- 
Kees Cook

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 22:37     ` Kees Cook
@ 2020-05-27 22:45       ` Christian Brauner
  2020-05-27 23:16         ` Christian Brauner
  2020-05-28  1:49         ` Kees Cook
  0 siblings, 2 replies; 22+ messages in thread
From: Christian Brauner @ 2020-05-27 22:45 UTC (permalink / raw)
  To: Kees Cook
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 03:37:58PM -0700, Kees Cook wrote:
> On Thu, May 28, 2020 at 12:05:32AM +0200, Christian Brauner wrote:
> > The main question also is, is there precedence where the kernel just
> > closes the file descriptor for userspace behind it's back? I'm not sure
> > I've heard of this before. That's not how that works afaict; it's also
> > not how we do pidfds. We don't just close the fd when the task
> > associated with it goes away, we notify and then userspace can close.
> 
> But there's a mapping between pidfd and task struct that is separate
> from task struct itself, yes? I.e. keeping a pidfd open doesn't pin
> struct task in memory forever, right?

No, but that's an implementation detail and we discussed that. It pins
struct pid instead of task_struct. Once the process is fully gone you
just get ESRCH.
For example, fds to /proc/<pid>/<tid>/ fds aren't just closed once the
task has gone away, userspace will just get ESRCH when it tries to open
files under there but the fd remains valid until close() is called.

In addition, of all the anon inode fds, none of them have the "close the
file behind userspace back" behavior: io_uring, signalfd, timerfd, btf,
perf_event, bpf-prog, bpf-link, bpf-map, pidfd, userfaultfd, fanotify,
inotify, eventpoll, fscontext, eventfd. These are just core kernel ones.
I'm pretty sure that it'd be very odd behavior if we did that. I'd
rather just notify userspace and leave the close to them. But maybe I'm
missing something.

Christian

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 22:36     ` Kees Cook
@ 2020-05-27 22:56       ` Tycho Andersen
  2020-05-28  1:50         ` Kees Cook
  0 siblings, 1 reply; 22+ messages in thread
From: Tycho Andersen @ 2020-05-27 22:56 UTC (permalink / raw)
  To: Kees Cook
  Cc: Christian Brauner, linux-kernel, Andy Lutomirski, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 03:36:09PM -0700, Kees Cook wrote:
> On Wed, May 27, 2020 at 03:52:03PM -0600, Tycho Andersen wrote:
> > On Wed, May 27, 2020 at 02:43:49PM -0700, Kees Cook wrote:
> > > (While I'm here -- why can there be only one listener per task? The
> > > notifications are filter-specific, not task-specific?)
> > 
> > Not sure what you mean here?
> 
> tatic struct file *init_listener(struct seccomp_filter *filter)
> {
>         struct file *ret = ERR_PTR(-EBUSY);
>         struct seccomp_filter *cur;
> 
>         for (cur = current->seccomp.filter; cur; cur = cur->prev) {
>                 if (cur->notif)
>                         goto out;
>         }
> 
> ...
> 
>         /* Installing a second listener in the chain should EBUSY */
>         EXPECT_EQ(user_trap_syscall(__NR_getpid,
>                                     SECCOMP_FILTER_FLAG_NEW_LISTENER),
>                   -1);
>         EXPECT_EQ(errno, EBUSY);
> 
> 
> Why does this limit exist? Since the fd is tied to a specific filter,
> I don't see conflicts about having multiple USER_NOTIF filters on one
> task -- the monitor's response will either fake it or continue it, so
> there is no "composition" needed? I must be missing something.

It exists because Andy asked for it :)

I agree that there's no technical reason for it to be there. I think
it's just that the semantics were potentially confusing, and it wasn't
a requirement anyone had to have multiples attached.

Tycho

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 22:45       ` Christian Brauner
@ 2020-05-27 23:16         ` Christian Brauner
  2020-05-28  1:59           ` Kees Cook
  2020-05-28  1:49         ` Kees Cook
  1 sibling, 1 reply; 22+ messages in thread
From: Christian Brauner @ 2020-05-27 23:16 UTC (permalink / raw)
  To: Kees Cook
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Thu, May 28, 2020 at 12:45:02AM +0200, Christian Brauner wrote:
> On Wed, May 27, 2020 at 03:37:58PM -0700, Kees Cook wrote:
> > On Thu, May 28, 2020 at 12:05:32AM +0200, Christian Brauner wrote:
> > > The main question also is, is there precedence where the kernel just
> > > closes the file descriptor for userspace behind it's back? I'm not sure
> > > I've heard of this before. That's not how that works afaict; it's also
> > > not how we do pidfds. We don't just close the fd when the task
> > > associated with it goes away, we notify and then userspace can close.
> > 
> > But there's a mapping between pidfd and task struct that is separate
> > from task struct itself, yes? I.e. keeping a pidfd open doesn't pin
> > struct task in memory forever, right?
> 
> No, but that's an implementation detail and we discussed that. It pins
> struct pid instead of task_struct. Once the process is fully gone you
> just get ESRCH.
> For example, fds to /proc/<pid>/<tid>/ fds aren't just closed once the
> task has gone away, userspace will just get ESRCH when it tries to open
> files under there but the fd remains valid until close() is called.
> 
> In addition, of all the anon inode fds, none of them have the "close the
> file behind userspace back" behavior: io_uring, signalfd, timerfd, btf,
> perf_event, bpf-prog, bpf-link, bpf-map, pidfd, userfaultfd, fanotify,
> inotify, eventpoll, fscontext, eventfd. These are just core kernel ones.
> I'm pretty sure that it'd be very odd behavior if we did that. I'd
> rather just notify userspace and leave the close to them. But maybe I'm
> missing something.

I'm also starting to think this isn't even possible or currently doable
safely.
The fdtable in the kernel would end up with a dangling pointer, I would
think. Unless you backtrack all fds that still have a reference into the
fdtable and refer to that file and close them all in the kernel which I
don't think is possible and also sounds very dodgy. This also really
seems like we would be breaking a major contract, namely that fds stay
valid until userspace calls close, execve(), or exits.

Christian

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 22:45       ` Christian Brauner
  2020-05-27 23:16         ` Christian Brauner
@ 2020-05-28  1:49         ` Kees Cook
  1 sibling, 0 replies; 22+ messages in thread
From: Kees Cook @ 2020-05-28  1:49 UTC (permalink / raw)
  To: Christian Brauner
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Thu, May 28, 2020 at 12:45:01AM +0200, Christian Brauner wrote:
> On Wed, May 27, 2020 at 03:37:58PM -0700, Kees Cook wrote:
> > But there's a mapping between pidfd and task struct that is separate
> > from task struct itself, yes? I.e. keeping a pidfd open doesn't pin
> > struct task in memory forever, right?
> 
> No, but that's an implementation detail and we discussed that. It pins
> struct pid instead of task_struct. Once the process is fully gone you
> just get ESRCH.

Oh right! struct pid, yes. Okay, that's quite a bit smaller.

> For example, fds to /proc/<pid>/<tid>/ fds aren't just closed once the
> task has gone away, userspace will just get ESRCH when it tries to open
> files under there but the fd remains valid until close() is called.
> 
> In addition, of all the anon inode fds, none of them have the "close the
> file behind userspace back" behavior: io_uring, signalfd, timerfd, btf,
> perf_event, bpf-prog, bpf-link, bpf-map, pidfd, userfaultfd, fanotify,
> inotify, eventpoll, fscontext, eventfd. These are just core kernel ones.
> I'm pretty sure that it'd be very odd behavior if we did that. I'd
> rather just notify userspace and leave the close to them. But maybe I'm
> missing something.

Well, they have a "you are now disconnected" state, which I was thinking
could be done entirely entirely on the VFS side of things, but it looks
like it's not.

So, yes, okay, thank you for walking me through all that. I still want
to take a closer look at all the notify calls in here. It seems strange
that seccomp has to do all the wakeups (but I guess there are no
"generic" poll helpers?)

-- 
Kees Cook

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 22:56       ` Tycho Andersen
@ 2020-05-28  1:50         ` Kees Cook
  0 siblings, 0 replies; 22+ messages in thread
From: Kees Cook @ 2020-05-28  1:50 UTC (permalink / raw)
  To: Tycho Andersen
  Cc: Christian Brauner, linux-kernel, Andy Lutomirski, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 04:56:00PM -0600, Tycho Andersen wrote:
> On Wed, May 27, 2020 at 03:36:09PM -0700, Kees Cook wrote:
> > On Wed, May 27, 2020 at 03:52:03PM -0600, Tycho Andersen wrote:
> > > On Wed, May 27, 2020 at 02:43:49PM -0700, Kees Cook wrote:
> > > > (While I'm here -- why can there be only one listener per task? The
> > > > notifications are filter-specific, not task-specific?)
> > > 
> > > Not sure what you mean here?
> > 
> > tatic struct file *init_listener(struct seccomp_filter *filter)
> > {
> >         struct file *ret = ERR_PTR(-EBUSY);
> >         struct seccomp_filter *cur;
> > 
> >         for (cur = current->seccomp.filter; cur; cur = cur->prev) {
> >                 if (cur->notif)
> >                         goto out;
> >         }
> > 
> > ...
> > 
> >         /* Installing a second listener in the chain should EBUSY */
> >         EXPECT_EQ(user_trap_syscall(__NR_getpid,
> >                                     SECCOMP_FILTER_FLAG_NEW_LISTENER),
> >                   -1);
> >         EXPECT_EQ(errno, EBUSY);
> > 
> > 
> > Why does this limit exist? Since the fd is tied to a specific filter,
> > I don't see conflicts about having multiple USER_NOTIF filters on one
> > task -- the monitor's response will either fake it or continue it, so
> > there is no "composition" needed? I must be missing something.
> 
> It exists because Andy asked for it :)
> 
> I agree that there's no technical reason for it to be there. I think
> it's just that the semantics were potentially confusing, and it wasn't
> a requirement anyone had to have multiples attached.

Okay, sounds good. It just seems seccomp continues to grow "layers", so
I'm eyeing this aspect of user_notif. i.e. what if systemd decides to
add a user_notif for something and now suddenly the containers can't use
it. Or if some complex thing inside a container tries to use user_notif
and it can't because the container manager is doing it, etc.

Future work! :)

-- 
Kees Cook

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 23:16         ` Christian Brauner
@ 2020-05-28  1:59           ` Kees Cook
  2020-05-28  4:14             ` Jann Horn
  2020-05-28 14:16             ` Christian Brauner
  0 siblings, 2 replies; 22+ messages in thread
From: Kees Cook @ 2020-05-28  1:59 UTC (permalink / raw)
  To: Christian Brauner
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Thu, May 28, 2020 at 01:16:46AM +0200, Christian Brauner wrote:
> I'm also starting to think this isn't even possible or currently doable
> safely.
> The fdtable in the kernel would end up with a dangling pointer, I would
> think. Unless you backtrack all fds that still have a reference into the
> fdtable and refer to that file and close them all in the kernel which I
> don't think is possible and also sounds very dodgy. This also really
> seems like we would be breaking a major contract, namely that fds stay
> valid until userspace calls close, execve(), or exits.

Right, I think I was just using the wrong words? I was looking at it
like a pipe, or a socket, where you still have an fd, but reads return
0, you might get SIGPIPE, etc. The VFS clearly knows what a
"disconnected" fd is, and I had assumed there was general logic for it
to indicate "I'm not here any more".

I recently did something very similar to the pstore filesystem, but I got
to cheat with some massive subsystem locks. In that case I needed to clear
all the inodes out of the tmpfs, so I unlink them all and manage the data
lifetimes pointing back into the (waiting to be unloaded) backend module
by NULLing the pointer back, which is safe because of the how the locking
there happens to work. Any open readers, when they close, will have the
last ref count dropped, at which point the record itself is released too.

Back to the seccomp subject: should "all tasks died" be distinguishable
from "I can't find that notification" in the ioctl()? (i.e. is ENOENT
sufficient, or does there need to be an EIO or ESRCH there?)

-- 
Kees Cook

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-27 11:19 [PATCH 1/2] seccomp: notify user trap about unused filter Christian Brauner
                   ` (3 preceding siblings ...)
  2020-05-27 21:43 ` Kees Cook
@ 2020-05-28  4:04 ` Jann Horn
  2020-05-28  9:57   ` Christian Brauner
  4 siblings, 1 reply; 22+ messages in thread
From: Jann Horn @ 2020-05-28  4:04 UTC (permalink / raw)
  To: Christian Brauner
  Cc: kernel list, Kees Cook, Andy Lutomirski, Tycho Andersen,
	Matt Denton, Sargun Dhillon, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 1:19 PM Christian Brauner
<christian.brauner@ubuntu.com> wrote:
> We've been making heavy use of the seccomp notifier to intercept and
> handle certain syscalls for containers. This patch allows a syscall
> supervisor listening on a given notifier to be notified when a seccomp
> filter has become unused.
[...]
> To fix this, we introduce a new "live" reference counter that tracks the
> live tasks making use of a given filter and when a notifier is
> registered waiting tasks will be notified that the filter is now empty
> by receiving a (E)POLLHUP event.
> The concept in this patch introduces is the same as for signal_struct,
> i.e. reference counting for life-cycle management is decoupled from
> reference counting live taks using the object.
[...]
> + * @live: tasks that actually use this filter, only to be altered
> + *          during fork(), exit()/free_task(), and filter installation

This comment is a bit off. Actually, @live counts the number of tasks
that use the filter directly plus the number of dependent filters that
have non-zero @live.

[...]
> +void seccomp_filter_notify(const struct task_struct *tsk)
> +{
> +       struct seccomp_filter *orig = tsk->seccomp.filter;
> +
> +       while (orig && refcount_dec_and_test(&orig->live)) {
> +               if (waitqueue_active(&orig->wqh))
> +                       wake_up_poll(&orig->wqh, EPOLLHUP);
> +               orig = orig->prev;
> +       }
> +}

/me fetches the paint bucket

Maybe name this seccomp_filter_unuse() or
seccomp_filter_unuse_notify() or something like that? The current name
isn't very descriptive.

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-28  1:59           ` Kees Cook
@ 2020-05-28  4:14             ` Jann Horn
  2020-05-28 14:16             ` Christian Brauner
  1 sibling, 0 replies; 22+ messages in thread
From: Jann Horn @ 2020-05-28  4:14 UTC (permalink / raw)
  To: Kees Cook
  Cc: Christian Brauner, kernel list, Andy Lutomirski, Tycho Andersen,
	Matt Denton, Sargun Dhillon, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Thu, May 28, 2020 at 3:59 AM Kees Cook <keescook@chromium.org> wrote:
> On Thu, May 28, 2020 at 01:16:46AM +0200, Christian Brauner wrote:
> > I'm also starting to think this isn't even possible or currently doable
> > safely.
> > The fdtable in the kernel would end up with a dangling pointer, I would
> > think. Unless you backtrack all fds that still have a reference into the
> > fdtable and refer to that file and close them all in the kernel which I
> > don't think is possible and also sounds very dodgy. This also really
> > seems like we would be breaking a major contract, namely that fds stay
> > valid until userspace calls close, execve(), or exits.
>
> Right, I think I was just using the wrong words? I was looking at it
> like a pipe, or a socket, where you still have an fd, but reads return
> 0, you might get SIGPIPE, etc. The VFS clearly knows what a
> "disconnected" fd is, and I had assumed there was general logic for it
> to indicate "I'm not here any more".

Nope. For example, pipes have manual checks based on pipe->readers and
pipe->writers, and manually send SIGPIPE and stuff from inside
fs/pipe.c. And pipes are not actually permanently "disconnected" -
someone can e.g. open a pipe that previously had no readers in read
mode, and suddenly you can write to it again.

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-28  4:04 ` Jann Horn
@ 2020-05-28  9:57   ` Christian Brauner
  0 siblings, 0 replies; 22+ messages in thread
From: Christian Brauner @ 2020-05-28  9:57 UTC (permalink / raw)
  To: Jann Horn
  Cc: kernel list, Kees Cook, Andy Lutomirski, Tycho Andersen,
	Matt Denton, Sargun Dhillon, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Thu, May 28, 2020 at 06:04:48AM +0200, Jann Horn wrote:
> On Wed, May 27, 2020 at 1:19 PM Christian Brauner
> <christian.brauner@ubuntu.com> wrote:
> > We've been making heavy use of the seccomp notifier to intercept and
> > handle certain syscalls for containers. This patch allows a syscall
> > supervisor listening on a given notifier to be notified when a seccomp
> > filter has become unused.
> [...]
> > To fix this, we introduce a new "live" reference counter that tracks the
> > live tasks making use of a given filter and when a notifier is
> > registered waiting tasks will be notified that the filter is now empty
> > by receiving a (E)POLLHUP event.
> > The concept in this patch introduces is the same as for signal_struct,
> > i.e. reference counting for life-cycle management is decoupled from
> > reference counting live taks using the object.
> [...]
> > + * @live: tasks that actually use this filter, only to be altered
> > + *          during fork(), exit()/free_task(), and filter installation
> 
> This comment is a bit off. Actually, @live counts the number of tasks
> that use the filter directly plus the number of dependent filters that
> have non-zero @live.

I'll update the comment.

> 
> [...]
> > +void seccomp_filter_notify(const struct task_struct *tsk)
> > +{
> > +       struct seccomp_filter *orig = tsk->seccomp.filter;
> > +
> > +       while (orig && refcount_dec_and_test(&orig->live)) {
> > +               if (waitqueue_active(&orig->wqh))
> > +                       wake_up_poll(&orig->wqh, EPOLLHUP);
> > +               orig = orig->prev;
> > +       }
> > +}
> 
> /me fetches the paint bucket
> 
> Maybe name this seccomp_filter_unuse() or
> seccomp_filter_unuse_notify() or something like that? The current name
> isn't very descriptive.

I think seccomp_filter_release() might be the right color. It would also
line-up nicely with:
- cgroup_release()
- exit_mm_release()
- exec_mm_release()
- futex_exec_release()
- ptrace_release_task()
and others.

Christian

^ permalink raw reply	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-28  1:59           ` Kees Cook
  2020-05-28  4:14             ` Jann Horn
@ 2020-05-28 14:16             ` Christian Brauner
  2020-05-28 14:39               ` Christian Brauner
  1 sibling, 1 reply; 22+ messages in thread
From: Christian Brauner @ 2020-05-28 14:16 UTC (permalink / raw)
  To: Kees Cook
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Wed, May 27, 2020 at 06:59:54PM -0700, Kees Cook wrote:
> On Thu, May 28, 2020 at 01:16:46AM +0200, Christian Brauner wrote:
> > I'm also starting to think this isn't even possible or currently doable
> > safely.
> > The fdtable in the kernel would end up with a dangling pointer, I would
> > think. Unless you backtrack all fds that still have a reference into the
> > fdtable and refer to that file and close them all in the kernel which I
> > don't think is possible and also sounds very dodgy. This also really
> > seems like we would be breaking a major contract, namely that fds stay
> > valid until userspace calls close, execve(), or exits.
> 
> Right, I think I was just using the wrong words? I was looking at it
> like a pipe, or a socket, where you still have an fd, but reads return
> 0, you might get SIGPIPE, etc. The VFS clearly knows what a
> "disconnected" fd is, and I had assumed there was general logic for it
> to indicate "I'm not here any more".
> 
> I recently did something very similar to the pstore filesystem, but I got
> to cheat with some massive subsystem locks. In that case I needed to clear
> all the inodes out of the tmpfs, so I unlink them all and manage the data
> lifetimes pointing back into the (waiting to be unloaded) backend module
> by NULLing the pointer back, which is safe because of the how the locking
> there happens to work. Any open readers, when they close, will have the
> last ref count dropped, at which point the record itself is released too.
> 
> Back to the seccomp subject: should "all tasks died" be distinguishable
> from "I can't find that notification" in the ioctl()? (i.e. is ENOENT
> sufficient, or does there need to be an EIO or ESRCH there?)

I personally think it's fine as it is but as it might help users if we
reported ESRCH something like the patch below might do.
Actual cleanup of the notifier should still happen in
seccomp_notify_release() imho, and not in __poll_t both conceptually and
also because f_op->release() happens on finaly fput() which punts it to
task_work() which finishes when the task returns from kernel mode (or
exits) - or - if the task is not alive anymore just puts it on the
kernel global workqueue which is perfect for non-high-priority cleanup
stuff. It's better than making __poll_t heavier than it needs to be.
Unless there's an obvious reason not to.

diff --git a/kernel/seccomp.c b/kernel/seccomp.c
index 9fa642d6d549..e6fa03cc4840 100644
--- a/kernel/seccomp.c
+++ b/kernel/seccomp.c
@@ -1221,6 +1221,9 @@ static __poll_t seccomp_notify_poll(struct file *file,

        poll_wait(file, &filter->wqh, poll_tab);

+       if (refcount_read(&filter->live) == 0)
+               ret |= EPOLLHUP;
+
        if (mutex_lock_interruptible(&filter->notify_lock) < 0)
                return EPOLLERR;

@@ -1231,13 +1234,17 @@ static __poll_t seccomp_notify_poll(struct file *file,
                        ret |= EPOLLOUT | EPOLLWRNORM;
                if ((ret & EPOLLIN) && (ret & EPOLLOUT))
                        break;
+
+               if ((ret & EPOLLHUP) && cur->state != SECCOMP_NOTIFY_REPLIED) {
+                       knotif->state = SECCOMP_NOTIFY_REPLIED;
+                       knotif->error = -ESRCH;
+                       knotif->val = 0;
+                       complete(&knotif->ready);
+               }
        }

        mutex_unlock(&filter->notify_lock);

-       if (refcount_read(&filter->live) == 0)
-               ret |= EPOLLHUP;
-
        return ret;
 }

Christian

^ permalink raw reply related	[flat|nested] 22+ messages in thread

* Re: [PATCH 1/2] seccomp: notify user trap about unused filter
  2020-05-28 14:16             ` Christian Brauner
@ 2020-05-28 14:39               ` Christian Brauner
  0 siblings, 0 replies; 22+ messages in thread
From: Christian Brauner @ 2020-05-28 14:39 UTC (permalink / raw)
  To: Kees Cook
  Cc: linux-kernel, Andy Lutomirski, Tycho Andersen, Matt Denton,
	Sargun Dhillon, Jann Horn, Chris Palmer, Aleksa Sarai,
	Robert Sesek, Jeffrey Vander Stoep, Linux Containers

On Thu, May 28, 2020 at 04:17:00PM +0200, Christian Brauner wrote:
> On Wed, May 27, 2020 at 06:59:54PM -0700, Kees Cook wrote:
> > On Thu, May 28, 2020 at 01:16:46AM +0200, Christian Brauner wrote:
> > > I'm also starting to think this isn't even possible or currently doable
> > > safely.
> > > The fdtable in the kernel would end up with a dangling pointer, I would
> > > think. Unless you backtrack all fds that still have a reference into the
> > > fdtable and refer to that file and close them all in the kernel which I
> > > don't think is possible and also sounds very dodgy. This also really
> > > seems like we would be breaking a major contract, namely that fds stay
> > > valid until userspace calls close, execve(), or exits.
> > 
> > Right, I think I was just using the wrong words? I was looking at it
> > like a pipe, or a socket, where you still have an fd, but reads return
> > 0, you might get SIGPIPE, etc. The VFS clearly knows what a
> > "disconnected" fd is, and I had assumed there was general logic for it
> > to indicate "I'm not here any more".
> > 
> > I recently did something very similar to the pstore filesystem, but I got
> > to cheat with some massive subsystem locks. In that case I needed to clear
> > all the inodes out of the tmpfs, so I unlink them all and manage the data
> > lifetimes pointing back into the (waiting to be unloaded) backend module
> > by NULLing the pointer back, which is safe because of the how the locking
> > there happens to work. Any open readers, when they close, will have the
> > last ref count dropped, at which point the record itself is released too.
> > 
> > Back to the seccomp subject: should "all tasks died" be distinguishable
> > from "I can't find that notification" in the ioctl()? (i.e. is ENOENT
> > sufficient, or does there need to be an EIO or ESRCH there?)
> 
> I personally think it's fine as it is but as it might help users if we
> reported ESRCH something like the patch below might do.
> Actual cleanup of the notifier should still happen in
> seccomp_notify_release() imho, and not in __poll_t both conceptually and
> also because f_op->release() happens on finaly fput() which punts it to
> task_work() which finishes when the task returns from kernel mode (or
> exits) - or - if the task is not alive anymore just puts it on the
> kernel global workqueue which is perfect for non-high-priority cleanup
> stuff. It's better than making __poll_t heavier than it needs to be.
> Unless there's an obvious reason not to.

Scratch the patch I posted before here; it's garbage of course.

^ permalink raw reply	[flat|nested] 22+ messages in thread

end of thread, other threads:[~2020-05-28 14:40 UTC | newest]

Thread overview: 22+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-05-27 11:19 [PATCH 1/2] seccomp: notify user trap about unused filter Christian Brauner
2020-05-27 11:19 ` [PATCH 2/2] tests: test seccomp filter notifications Christian Brauner
2020-05-27 15:25 ` [PATCH 1/2] seccomp: notify user trap about unused filter Tycho Andersen
2020-05-27 15:35   ` Christian Brauner
2020-05-27 17:37 ` Sargun Dhillon
2020-05-27 19:13   ` Christian Brauner
2020-05-27 21:43 ` Kees Cook
2020-05-27 21:52   ` Tycho Andersen
2020-05-27 22:36     ` Kees Cook
2020-05-27 22:56       ` Tycho Andersen
2020-05-28  1:50         ` Kees Cook
2020-05-27 22:05   ` Christian Brauner
2020-05-27 22:37     ` Kees Cook
2020-05-27 22:45       ` Christian Brauner
2020-05-27 23:16         ` Christian Brauner
2020-05-28  1:59           ` Kees Cook
2020-05-28  4:14             ` Jann Horn
2020-05-28 14:16             ` Christian Brauner
2020-05-28 14:39               ` Christian Brauner
2020-05-28  1:49         ` Kees Cook
2020-05-28  4:04 ` Jann Horn
2020-05-28  9:57   ` Christian Brauner

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).