public inbox for io-uring@vger.kernel.org
 help / color / mirror / Atom feed
* [PATCHSET RFC v4] Inherited restrictions and BPF filtering
@ 2026-01-16 22:38 Jens Axboe
  2026-01-16 22:38 ` [PATCH 1/5] io_uring: add support for BPF filtering for opcode restrictions Jens Axboe
                   ` (4 more replies)
  0 siblings, 5 replies; 6+ messages in thread
From: Jens Axboe @ 2026-01-16 22:38 UTC (permalink / raw)
  To: io-uring

Hi,

Followup to v3 here:

https://lore.kernel.org/io-uring/20260115165244.1037465-1-axboe@kernel.dk/

which has all the gory details. The main new thing in v4 is proper
support for stacking of BPF filters, when a task goes through fork.
These are referenced, and COW'ed if new filters are attempted installed
in the referenced filter.

Performance and functionality is otherwise the same. See the changelog
for more details.

Comments welcome! Kernel branch can be found here:

https://git.kernel.org/pub/scm/linux/kernel/git/axboe/linux.git/log/?h=io_uring-bpf-restrictions

and sits on top of for-7.0/io_uring.

Changes since v3:
- Move fork.c logic to an io_uring helper, io_uring_fork(), rather than
  open code it.
- Sort out the filter and filter table references, and drop the patch
  that made io_restriction dynamically allocated.
- Drop IORING_REGISTER_RESTRICTIONS_TASK, IORING_REGISTER_RESTRICTIONS
  can just get reused with fd == -1 like other "blind" reg opcodes.
- Split IORING_OP_SOCKET filter support patch out as a separate patch
- Fix a few style issues
- Do proper cloning of registered BPF filters, if a task is forked.
  These are done COW style, where initially a new task will just
  inherit the existing filter with a reference to it. If new filters
  are added in the new task, then the existing table is COW'ed.
- Drop ctx->bpf_restricted, just check for the presence of the filter.
- Drop various alloc helpers, not needed with other cleanups.
- Add and improve a bunch of comments.

 include/linux/bpf.h            |   1 +
 include/linux/bpf_types.h      |   4 +
 include/linux/io_uring.h       |  14 +-
 include/linux/io_uring_types.h |  16 ++
 include/linux/sched.h          |   1 +
 include/uapi/linux/bpf.h       |   1 +
 include/uapi/linux/io_uring.h  |  52 +++++
 io_uring/Makefile              |   1 +
 io_uring/bpf_filter.c          | 372 +++++++++++++++++++++++++++++++++
 io_uring/bpf_filter.h          |  46 ++++
 io_uring/io_uring.c            |  27 +++
 io_uring/io_uring.h            |   1 +
 io_uring/net.c                 |   9 +
 io_uring/net.h                 |   5 +
 io_uring/register.c            |  73 +++++++
 io_uring/tctx.c                |  40 +++-
 kernel/bpf/syscall.c           |   9 +
 kernel/fork.c                  |   5 +
 18 files changed, 667 insertions(+), 10 deletions(-)

-- 
Jens Axboe


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

* [PATCH 1/5] io_uring: add support for BPF filtering for opcode restrictions
  2026-01-16 22:38 [PATCHSET RFC v4] Inherited restrictions and BPF filtering Jens Axboe
@ 2026-01-16 22:38 ` Jens Axboe
  2026-01-16 22:38 ` [PATCH 2/5] io_uring/net: allow filtering on IORING_OP_SOCKET data Jens Axboe
                   ` (3 subsequent siblings)
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2026-01-16 22:38 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

This adds support for loading BPF programs with io_uring, which can
restrict the opcodes executed. Unlike IORING_REGISTER_RESTRICTIONS,
using BPF programs allow fine grained control over both the opcode in
question, as well as other data associated with the request. This
initial patch just supports whatever is in the io_kiocb for filtering,
but shortly opcode specific support will be added.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 include/linux/bpf.h            |   1 +
 include/linux/bpf_types.h      |   4 +
 include/linux/io_uring_types.h |  14 ++
 include/uapi/linux/bpf.h       |   1 +
 include/uapi/linux/io_uring.h  |  38 +++++
 io_uring/Makefile              |   1 +
 io_uring/bpf_filter.c          | 285 +++++++++++++++++++++++++++++++++
 io_uring/bpf_filter.h          |  40 +++++
 io_uring/io_uring.c            |   8 +
 io_uring/register.c            |   8 +
 kernel/bpf/syscall.c           |   9 ++
 11 files changed, 409 insertions(+)
 create mode 100644 io_uring/bpf_filter.c
 create mode 100644 io_uring/bpf_filter.h

diff --git a/include/linux/bpf.h b/include/linux/bpf.h
index e5be698256d1..9b4435452458 100644
--- a/include/linux/bpf.h
+++ b/include/linux/bpf.h
@@ -31,6 +31,7 @@
 #include <linux/static_call.h>
 #include <linux/memcontrol.h>
 #include <linux/cfi.h>
+#include <linux/io_uring_types.h>
 #include <asm/rqspinlock.h>
 
 struct bpf_verifier_env;
diff --git a/include/linux/bpf_types.h b/include/linux/bpf_types.h
index b13de31e163f..c5d58806a1cf 100644
--- a/include/linux/bpf_types.h
+++ b/include/linux/bpf_types.h
@@ -83,6 +83,10 @@ BPF_PROG_TYPE(BPF_PROG_TYPE_SYSCALL, bpf_syscall,
 BPF_PROG_TYPE(BPF_PROG_TYPE_NETFILTER, netfilter,
 	      struct bpf_nf_ctx, struct bpf_nf_ctx)
 #endif
+#ifdef CONFIG_IO_URING
+BPF_PROG_TYPE(BPF_PROG_TYPE_IO_URING, io_uring_filter,
+	      struct io_uring_bpf_ctx, struct io_uring_bpf_ctx)
+#endif
 
 BPF_MAP_TYPE(BPF_MAP_TYPE_ARRAY, array_map_ops)
 BPF_MAP_TYPE(BPF_MAP_TYPE_PERCPU_ARRAY, percpu_array_map_ops)
diff --git a/include/linux/io_uring_types.h b/include/linux/io_uring_types.h
index 211686ad89fd..1e91fa7ecbaf 100644
--- a/include/linux/io_uring_types.h
+++ b/include/linux/io_uring_types.h
@@ -219,9 +219,23 @@ struct io_rings {
 	struct io_uring_cqe	cqes[] ____cacheline_aligned_in_smp;
 };
 
+#ifdef CONFIG_BPF
+extern const struct bpf_prog_ops io_uring_filter_prog_ops;
+extern const struct bpf_verifier_ops io_uring_filter_verifier_ops;
+#endif
+
+struct io_bpf_filter;
+struct io_bpf_filters {
+	refcount_t refs;	/* ref for ->bpf_filters */
+	spinlock_t lock;	/* protects ->bpf_filters modifications */
+	struct io_bpf_filter __rcu **filters;
+	struct rcu_head rcu_head;
+};
+
 struct io_restriction {
 	DECLARE_BITMAP(register_op, IORING_REGISTER_LAST);
 	DECLARE_BITMAP(sqe_op, IORING_OP_LAST);
+	struct io_bpf_filters *bpf_filters;
 	u8 sqe_flags_allowed;
 	u8 sqe_flags_required;
 	/* IORING_OP_* restrictions exist */
diff --git a/include/uapi/linux/bpf.h b/include/uapi/linux/bpf.h
index f8d8513eda27..4d43ec003887 100644
--- a/include/uapi/linux/bpf.h
+++ b/include/uapi/linux/bpf.h
@@ -1072,6 +1072,7 @@ enum bpf_prog_type {
 	BPF_PROG_TYPE_SK_LOOKUP,
 	BPF_PROG_TYPE_SYSCALL, /* a program that can execute syscalls */
 	BPF_PROG_TYPE_NETFILTER,
+	BPF_PROG_TYPE_IO_URING,
 	__MAX_BPF_PROG_TYPE
 };
 
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index b5b23c0d5283..768b0e7c0b57 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -700,6 +700,9 @@ enum io_uring_register_op {
 	/* auxiliary zcrx configuration, see enum zcrx_ctrl_op */
 	IORING_REGISTER_ZCRX_CTRL		= 36,
 
+	/* register bpf filtering programs */
+	IORING_REGISTER_BPF_FILTER		= 37,
+
 	/* this goes last */
 	IORING_REGISTER_LAST,
 
@@ -1113,6 +1116,41 @@ struct zcrx_ctrl {
 	};
 };
 
+struct io_uring_bpf_ctx {
+	__u8	opcode;
+	__u8	sqe_flags;
+	__u8	pad[6];
+	__u64	user_data;
+	__u64	resv[6];
+};
+
+enum {
+	/*
+	 * If set, any currently unset opcode will have a deny filter attached
+	 */
+	IO_URING_BPF_FILTER_DENY_REST	= 1,
+};
+
+struct io_uring_bpf_filter {
+	__u32	opcode;		/* io_uring opcode to filter */
+	__u32	flags;
+	__s32	prog_fd;	/* BPF program fd */
+	__u32	reserved[3];
+};
+
+enum {
+	IO_URING_BPF_CMD_FILTER	= 1,
+};
+
+struct io_uring_bpf {
+	__u16	cmd_type;	/* IO_URING_BPF_* values */
+	__u16	cmd_flags;	/* none so far */
+	__u32	resv;
+	union {
+		struct io_uring_bpf_filter	filter;
+	};
+};
+
 #ifdef __cplusplus
 }
 #endif
diff --git a/io_uring/Makefile b/io_uring/Makefile
index bc4e4a3fa0a5..d89bd0cf6363 100644
--- a/io_uring/Makefile
+++ b/io_uring/Makefile
@@ -22,3 +22,4 @@ obj-$(CONFIG_NET_RX_BUSY_POLL)	+= napi.o
 obj-$(CONFIG_NET) += net.o cmd_net.o
 obj-$(CONFIG_PROC_FS) += fdinfo.o
 obj-$(CONFIG_IO_URING_MOCK_FILE) += mock_file.o
+obj-$(CONFIG_BPF) += bpf_filter.o
diff --git a/io_uring/bpf_filter.c b/io_uring/bpf_filter.c
new file mode 100644
index 000000000000..f63a8e9e85db
--- /dev/null
+++ b/io_uring/bpf_filter.c
@@ -0,0 +1,285 @@
+// SPDX-License-Identifier: GPL-2.0
+/*
+ * BPF filter support for io_uring. Supports SQE opcodes for now.
+ */
+#include <linux/kernel.h>
+#include <linux/errno.h>
+#include <linux/io_uring.h>
+#include <linux/filter.h>
+#include <linux/bpf.h>
+#include <uapi/linux/io_uring.h>
+
+#include "io_uring.h"
+#include "bpf_filter.h"
+#include "net.h"
+
+struct io_bpf_filter {
+	struct bpf_prog		*prog;
+	struct io_bpf_filter	*next;
+};
+
+/* Deny if this is set as the filter */
+static const struct io_bpf_filter dummy_filter;
+
+static bool io_uring_filter_is_valid_access(int off, int size,
+					    enum bpf_access_type type,
+					    const struct bpf_prog *prog,
+					    struct bpf_insn_access_aux *info)
+{
+	if (type != BPF_READ)
+		return false;
+	if (off < 0 || off >= sizeof(struct io_uring_bpf_ctx))
+		return false;
+	if (off % size != 0)
+		return false;
+
+	return true;
+}
+
+/* Convert context field access if needed */
+static u32 io_uring_filter_convert_ctx_access(enum bpf_access_type type,
+					      const struct bpf_insn *si,
+					      struct bpf_insn *insn_buf,
+					      struct bpf_prog *prog,
+					      u32 *target_size)
+{
+	struct bpf_insn *insn = insn_buf;
+
+	/* Direct access is fine - context is read-only and passed directly */
+	switch (si->off) {
+	case offsetof(struct io_uring_bpf_ctx, opcode):
+	case offsetof(struct io_uring_bpf_ctx, sqe_flags):
+	case offsetof(struct io_uring_bpf_ctx, user_data):
+		*insn++ = BPF_LDX_MEM(BPF_SIZE(si->code), si->dst_reg,
+				      si->src_reg, si->off);
+		break;
+	default:
+		/* Union fields - also direct access */
+		*insn++ = BPF_LDX_MEM(BPF_SIZE(si->code), si->dst_reg,
+				      si->src_reg, si->off);
+		break;
+	}
+
+	return insn - insn_buf;
+}
+
+const struct bpf_prog_ops io_uring_filter_prog_ops = { };
+
+const struct bpf_verifier_ops io_uring_filter_verifier_ops = {
+	.get_func_proto		= bpf_base_func_proto,
+	.is_valid_access	= io_uring_filter_is_valid_access,
+	.convert_ctx_access	= io_uring_filter_convert_ctx_access,
+};
+
+static void io_uring_populate_bpf_ctx(struct io_uring_bpf_ctx *bctx,
+				      struct io_kiocb *req)
+{
+	memset(bctx, 0, sizeof(*bctx));
+	bctx->opcode = req->opcode;
+	bctx->sqe_flags = (__force int) req->flags & SQE_VALID_FLAGS;
+	bctx->user_data = req->cqe.user_data;
+}
+
+/*
+ * Run registered filters for a given opcode. For filters, a return of 0 denies
+ * execution of the request, a return of 1 allows it. If any filter for an
+ * opcode returns 0, filter processing is stopped, and the request is denied.
+ * This also stops the processing of filters.
+ *
+ * __io_uring_run_bpf_filters() returns 0 on success, allow running the
+ * request, and -EACCES when a request is denied.
+ */
+int __io_uring_run_bpf_filters(struct io_restriction *res, struct io_kiocb *req)
+{
+	struct io_bpf_filter *filter;
+	struct io_uring_bpf_ctx bpf_ctx;
+	int ret;
+
+	/*
+	 * req->opcode has already been validated to be within the range
+	 * of what we expect, io_init_req() does this.
+	 */
+	rcu_read_lock();
+	filter = rcu_dereference(res->bpf_filters->filters[req->opcode]);
+	if (!filter) {
+		ret = 1;
+		goto out;
+	} else if (filter == &dummy_filter) {
+		ret = 0;
+		goto out;
+	}
+
+	io_uring_populate_bpf_ctx(&bpf_ctx, req);
+
+	/*
+	 * Iterate registered filters. The opcode is allowed IFF all filters
+	 * return 1. If any filter returns denied, opcode will be denied.
+	 */
+	do {
+		ret = bpf_prog_run(filter->prog, &bpf_ctx);
+		if (!ret)
+			break;
+		filter = filter->next;
+	} while (filter);
+out:
+	rcu_read_unlock();
+	return ret ? 0 : -EACCES;
+}
+
+static void io_free_bpf_filters(struct rcu_head *head)
+{
+	struct io_bpf_filter __rcu **filter;
+	struct io_bpf_filters *filters;
+	int i;
+
+	filters = container_of(head, struct io_bpf_filters, rcu_head);
+	spin_lock(&filters->lock);
+	filter = filters->filters;
+	if (!filter) {
+		spin_unlock(&filters->lock);
+		return;
+	}
+	spin_unlock(&filters->lock);
+
+	for (i = 0; i < IORING_OP_LAST; i++) {
+		struct io_bpf_filter *f;
+
+		rcu_read_lock();
+		f = rcu_dereference(filter[i]);
+		while (f) {
+			struct io_bpf_filter *next = f->next;
+
+			/*
+			 * Even if stacked, dummy filter will always be last
+			 * as it can only get installed into an empty spot.
+			 */
+			if (f == &dummy_filter)
+				break;
+			if (f->prog)
+				bpf_prog_put(f->prog);
+			kfree(f);
+			f = next;
+		}
+		rcu_read_unlock();
+	}
+	kfree(filters->filters);
+	kfree(filters);
+}
+
+static void __io_put_bpf_filters(struct io_bpf_filters *filters)
+{
+	if (refcount_dec_and_test(&filters->refs))
+		call_rcu(&filters->rcu_head, io_free_bpf_filters);
+}
+
+void io_put_bpf_filters(struct io_restriction *res)
+{
+	if (res->bpf_filters)
+		__io_put_bpf_filters(res->bpf_filters);
+}
+
+static struct io_bpf_filters *io_new_bpf_filters(void)
+{
+	struct io_bpf_filters *filters;
+
+	filters = kzalloc(sizeof(*filters), GFP_KERNEL_ACCOUNT);
+	if (!filters)
+		return ERR_PTR(-ENOMEM);
+
+	filters->filters = kcalloc(IORING_OP_LAST,
+				   sizeof(struct io_bpf_filter *),
+				   GFP_KERNEL_ACCOUNT);
+	if (!filters->filters) {
+		kfree(filters);
+		return ERR_PTR(-ENOMEM);
+	}
+
+	refcount_set(&filters->refs, 1);
+	spin_lock_init(&filters->lock);
+	return filters;
+}
+
+int io_register_bpf_filter(struct io_restriction *res,
+			   struct io_uring_bpf __user *arg)
+{
+	struct io_bpf_filter *filter, *old_filter;
+	struct io_bpf_filters *filters;
+	struct io_uring_bpf reg;
+	struct bpf_prog *prog;
+	int ret;
+
+	if (copy_from_user(&reg, arg, sizeof(reg)))
+		return -EFAULT;
+	if (reg.cmd_type != IO_URING_BPF_CMD_FILTER)
+		return -EINVAL;
+	if (reg.cmd_flags || reg.resv)
+		return -EINVAL;
+
+	if (reg.filter.opcode >= IORING_OP_LAST)
+		return -EINVAL;
+	if ((reg.filter.flags & ~IO_URING_BPF_FILTER_DENY_REST) ||
+	    !mem_is_zero(reg.filter.reserved, sizeof(reg.filter.reserved)))
+		return -EINVAL;
+	if (reg.filter.prog_fd < 0)
+		return -EBADF;
+
+	/*
+	 * No existing filters, allocate set.
+	 */
+	filters = res->bpf_filters;
+	if (!filters) {
+		filters = io_new_bpf_filters();
+		if (IS_ERR(filters))
+			return PTR_ERR(filters);
+	}
+
+	prog = bpf_prog_get_type(reg.filter.prog_fd, BPF_PROG_TYPE_IO_URING);
+	if (IS_ERR(prog)) {
+		ret = PTR_ERR(prog);
+		goto err;
+	}
+
+	filter = kzalloc(sizeof(*filter), GFP_KERNEL_ACCOUNT);
+	if (!filter) {
+		ret = -ENOMEM;
+		goto err;
+	}
+	filter->prog = prog;
+	res->bpf_filters = filters;
+
+	/*
+	 * Insert filter - if the current opcode already has a filter
+	 * attached, add to the set.
+	 */
+	rcu_read_lock();
+	spin_lock_bh(&filters->lock);
+	old_filter = rcu_dereference(filters->filters[reg.filter.opcode]);
+	if (old_filter)
+		filter->next = old_filter;
+	rcu_assign_pointer(filters->filters[reg.filter.opcode], filter);
+
+	/*
+	 * If IO_URING_BPF_FILTER_DENY_REST is set, fill any unregistered
+	 * opcode with the dummy filter. That will cause them to be denied.
+	 */
+	if (reg.filter.flags & IO_URING_BPF_FILTER_DENY_REST) {
+		for (int i = 0; i < IORING_OP_LAST; i++) {
+			if (i == reg.filter.opcode)
+				continue;
+			old_filter = rcu_dereference(filters->filters[i]);
+			if (old_filter)
+				continue;
+			rcu_assign_pointer(filters->filters[i], &dummy_filter);
+		}
+	}
+
+	spin_unlock_bh(&filters->lock);
+	rcu_read_unlock();
+	return 0;
+err:
+	if (filters != res->bpf_filters)
+		__io_put_bpf_filters(filters);
+	if (!IS_ERR(prog))
+		bpf_prog_put(prog);
+	return ret;
+}
diff --git a/io_uring/bpf_filter.h b/io_uring/bpf_filter.h
new file mode 100644
index 000000000000..a131953ce950
--- /dev/null
+++ b/io_uring/bpf_filter.h
@@ -0,0 +1,40 @@
+// SPDX-License-Identifier: GPL-2.0
+#ifndef IO_URING_BPF_FILTER_H
+#define IO_URING_BPF_FILTER_H
+
+#ifdef CONFIG_BPF
+
+int __io_uring_run_bpf_filters(struct io_restriction *res, struct io_kiocb *req);
+
+int io_register_bpf_filter(struct io_restriction *res,
+			   struct io_uring_bpf __user *arg);
+
+void io_put_bpf_filters(struct io_restriction *res);
+
+static inline int io_uring_run_bpf_filters(struct io_restriction *res,
+					   struct io_kiocb *req)
+{
+	if (res->bpf_filters)
+		return __io_uring_run_bpf_filters(res, req);
+
+	return 0;
+}
+
+#else
+
+static inline int io_register_bpf_filter(struct io_restriction *res,
+					 struct io_uring_bpf __user *arg)
+{
+	return -EINVAL;
+}
+static inline int io_uring_run_bpf_filters(struct io_restriction *res,
+					   struct io_kiocb *req)
+{
+	return 0;
+}
+static inline void io_put_bpf_filters(struct io_restriction *res)
+{
+}
+#endif /* CONFIG_IO_URING */
+
+#endif
diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index 2cde22af78a3..67533e494836 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -93,6 +93,7 @@
 #include "rw.h"
 #include "alloc_cache.h"
 #include "eventfd.h"
+#include "bpf_filter.h"
 
 #define SQE_COMMON_FLAGS (IOSQE_FIXED_FILE | IOSQE_IO_LINK | \
 			  IOSQE_IO_HARDLINK | IOSQE_ASYNC)
@@ -2261,6 +2262,12 @@ static inline int io_submit_sqe(struct io_ring_ctx *ctx, struct io_kiocb *req,
 	if (unlikely(ret))
 		return io_submit_fail_init(sqe, req, ret);
 
+	if (unlikely(ctx->restrictions.bpf_filters)) {
+		ret = io_uring_run_bpf_filters(&ctx->restrictions, req);
+		if (ret)
+			return io_submit_fail_init(sqe, req, ret);
+	}
+
 	trace_io_uring_submit_req(req);
 
 	/*
@@ -2850,6 +2857,7 @@ static __cold void io_ring_ctx_free(struct io_ring_ctx *ctx)
 	percpu_ref_exit(&ctx->refs);
 	free_uid(ctx->user);
 	io_req_caches_free(ctx);
+	io_put_bpf_filters(&ctx->restrictions);
 
 	WARN_ON_ONCE(ctx->nr_req_allocated);
 
diff --git a/io_uring/register.c b/io_uring/register.c
index 8551f13920dc..30957c2cb5eb 100644
--- a/io_uring/register.c
+++ b/io_uring/register.c
@@ -33,6 +33,7 @@
 #include "memmap.h"
 #include "zcrx.h"
 #include "query.h"
+#include "bpf_filter.h"
 
 #define IORING_MAX_RESTRICTIONS	(IORING_RESTRICTION_LAST + \
 				 IORING_REGISTER_LAST + IORING_OP_LAST)
@@ -830,6 +831,13 @@ static int __io_uring_register(struct io_ring_ctx *ctx, unsigned opcode,
 	case IORING_REGISTER_ZCRX_CTRL:
 		ret = io_zcrx_ctrl(ctx, arg, nr_args);
 		break;
+	case IORING_REGISTER_BPF_FILTER:
+		ret = -EINVAL;
+
+		if (nr_args != 1)
+			break;
+		ret = io_register_bpf_filter(&ctx->restrictions, arg);
+		break;
 	default:
 		ret = -EINVAL;
 		break;
diff --git a/kernel/bpf/syscall.c b/kernel/bpf/syscall.c
index 4ff82144f885..d12537d918f7 100644
--- a/kernel/bpf/syscall.c
+++ b/kernel/bpf/syscall.c
@@ -2752,6 +2752,10 @@ bpf_prog_load_check_attach(enum bpf_prog_type prog_type,
 		if (expected_attach_type == BPF_NETFILTER)
 			return 0;
 		return -EINVAL;
+	case BPF_PROG_TYPE_IO_URING:
+		if (expected_attach_type)
+			return -EINVAL;
+		return 0;
 	case BPF_PROG_TYPE_SYSCALL:
 	case BPF_PROG_TYPE_EXT:
 		if (expected_attach_type)
@@ -2934,6 +2938,7 @@ static int bpf_prog_load(union bpf_attr *attr, bpfptr_t uattr, u32 uattr_size)
 	}
 	if (type != BPF_PROG_TYPE_SOCKET_FILTER &&
 	    type != BPF_PROG_TYPE_CGROUP_SKB &&
+	    type != BPF_PROG_TYPE_IO_URING &&
 	    !bpf_cap)
 		goto put_token;
 
@@ -4403,6 +4408,10 @@ static int bpf_prog_attach_check_attach_type(const struct bpf_prog *prog,
 		if (attach_type != BPF_NETFILTER)
 			return -EINVAL;
 		return 0;
+	case BPF_PROG_TYPE_IO_URING:
+		if (attach_type != 0)
+			return -EINVAL;
+		return 0;
 	case BPF_PROG_TYPE_PERF_EVENT:
 	case BPF_PROG_TYPE_TRACEPOINT:
 		if (attach_type != BPF_PERF_EVENT)
-- 
2.51.0


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

* [PATCH 2/5] io_uring/net: allow filtering on IORING_OP_SOCKET data
  2026-01-16 22:38 [PATCHSET RFC v4] Inherited restrictions and BPF filtering Jens Axboe
  2026-01-16 22:38 ` [PATCH 1/5] io_uring: add support for BPF filtering for opcode restrictions Jens Axboe
@ 2026-01-16 22:38 ` Jens Axboe
  2026-01-16 22:38 ` [PATCH 3/5] io_uring/bpf_filter: add ref counts to struct io_bpf_filter Jens Axboe
                   ` (2 subsequent siblings)
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2026-01-16 22:38 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

Example population method for the BPF based opcode filtering. This
exposes the socket family, type, and protocol to a registered BPF
filter. This in turn enables the filter to make decisions based on
what was passed in to the IORING_OP_SOCKET request type.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 include/uapi/linux/io_uring.h |  9 ++++++++-
 io_uring/bpf_filter.c         | 10 ++++++++++
 io_uring/net.c                |  9 +++++++++
 io_uring/net.h                |  5 +++++
 4 files changed, 32 insertions(+), 1 deletion(-)

diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index 768b0e7c0b57..27839318c43e 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -1121,7 +1121,14 @@ struct io_uring_bpf_ctx {
 	__u8	sqe_flags;
 	__u8	pad[6];
 	__u64	user_data;
-	__u64	resv[6];
+	union {
+		__u64	resv[6];
+		struct {
+			__u32	family;
+			__u32	type;
+			__u32	protocol;
+		} socket;
+	};
 };
 
 enum {
diff --git a/io_uring/bpf_filter.c b/io_uring/bpf_filter.c
index f63a8e9e85db..74969e9bc3b4 100644
--- a/io_uring/bpf_filter.c
+++ b/io_uring/bpf_filter.c
@@ -78,6 +78,16 @@ static void io_uring_populate_bpf_ctx(struct io_uring_bpf_ctx *bctx,
 	bctx->opcode = req->opcode;
 	bctx->sqe_flags = (__force int) req->flags & SQE_VALID_FLAGS;
 	bctx->user_data = req->cqe.user_data;
+
+	/*
+	 * Opcodes can provide a handler fo populating more data into bctx,
+	 * for filters to use.
+	 */
+	switch (req->opcode) {
+	case IORING_OP_SOCKET:
+		io_socket_bpf_populate(bctx, req);
+		break;
+	}
 }
 
 /*
diff --git a/io_uring/net.c b/io_uring/net.c
index 519ea055b761..4fcba36bd0bb 100644
--- a/io_uring/net.c
+++ b/io_uring/net.c
@@ -1699,6 +1699,15 @@ int io_accept(struct io_kiocb *req, unsigned int issue_flags)
 	return IOU_COMPLETE;
 }
 
+void io_socket_bpf_populate(struct io_uring_bpf_ctx *bctx, struct io_kiocb *req)
+{
+	struct io_socket *sock = io_kiocb_to_cmd(req, struct io_socket);
+
+	bctx->socket.family = sock->domain;
+	bctx->socket.type = sock->type;
+	bctx->socket.protocol = sock->protocol;
+}
+
 int io_socket_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe)
 {
 	struct io_socket *sock = io_kiocb_to_cmd(req, struct io_socket);
diff --git a/io_uring/net.h b/io_uring/net.h
index 43e5ce5416b7..eef6b4272d01 100644
--- a/io_uring/net.h
+++ b/io_uring/net.h
@@ -44,6 +44,7 @@ int io_accept(struct io_kiocb *req, unsigned int issue_flags);
 
 int io_socket_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
 int io_socket(struct io_kiocb *req, unsigned int issue_flags);
+void io_socket_bpf_populate(struct io_uring_bpf_ctx *bctx, struct io_kiocb *req);
 
 int io_connect_prep(struct io_kiocb *req, const struct io_uring_sqe *sqe);
 int io_connect(struct io_kiocb *req, unsigned int issue_flags);
@@ -64,4 +65,8 @@ void io_netmsg_cache_free(const void *entry);
 static inline void io_netmsg_cache_free(const void *entry)
 {
 }
+static inline void io_socket_bpf_populate(struct io_uring_bpf_ctx *bctx,
+					  struct io_kiocb *req)
+{
+}
 #endif
-- 
2.51.0


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

* [PATCH 3/5] io_uring/bpf_filter: add ref counts to struct io_bpf_filter
  2026-01-16 22:38 [PATCHSET RFC v4] Inherited restrictions and BPF filtering Jens Axboe
  2026-01-16 22:38 ` [PATCH 1/5] io_uring: add support for BPF filtering for opcode restrictions Jens Axboe
  2026-01-16 22:38 ` [PATCH 2/5] io_uring/net: allow filtering on IORING_OP_SOCKET data Jens Axboe
@ 2026-01-16 22:38 ` Jens Axboe
  2026-01-16 22:38 ` [PATCH 4/5] io_uring: add task fork hook Jens Axboe
  2026-01-16 22:38 ` [PATCH 5/5] io_uring: allow registration of per-task restrictions Jens Axboe
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2026-01-16 22:38 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

In preparation for allowing inheritance of BPF filters and filter
tables, add a reference count to the filter. This allows multiple tables
to safely include the same filter.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 io_uring/bpf_filter.c | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/io_uring/bpf_filter.c b/io_uring/bpf_filter.c
index 74969e9bc3b4..8ed5b913005a 100644
--- a/io_uring/bpf_filter.c
+++ b/io_uring/bpf_filter.c
@@ -14,6 +14,7 @@
 #include "net.h"
 
 struct io_bpf_filter {
+	refcount_t		refs;
 	struct bpf_prog		*prog;
 	struct io_bpf_filter	*next;
 };
@@ -165,6 +166,11 @@ static void io_free_bpf_filters(struct rcu_head *head)
 			 */
 			if (f == &dummy_filter)
 				break;
+
+			/* Someone still holds a ref, stop iterating. */
+			if (!refcount_dec_and_test(&f->refs))
+				break;
+
 			if (f->prog)
 				bpf_prog_put(f->prog);
 			kfree(f);
@@ -254,6 +260,7 @@ int io_register_bpf_filter(struct io_restriction *res,
 		ret = -ENOMEM;
 		goto err;
 	}
+	refcount_set(&filter->refs, 1);
 	filter->prog = prog;
 	res->bpf_filters = filters;
 
-- 
2.51.0


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

* [PATCH 4/5] io_uring: add task fork hook
  2026-01-16 22:38 [PATCHSET RFC v4] Inherited restrictions and BPF filtering Jens Axboe
                   ` (2 preceding siblings ...)
  2026-01-16 22:38 ` [PATCH 3/5] io_uring/bpf_filter: add ref counts to struct io_bpf_filter Jens Axboe
@ 2026-01-16 22:38 ` Jens Axboe
  2026-01-16 22:38 ` [PATCH 5/5] io_uring: allow registration of per-task restrictions Jens Axboe
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2026-01-16 22:38 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

Called when copy_process() is called to copy state to a new child.
Right now this is just a stub, but will be used shortly to properly
handle fork'ing of task based io_uring restrictions.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 include/linux/io_uring.h | 14 +++++++++++++-
 include/linux/sched.h    |  1 +
 io_uring/tctx.c          |  5 +++++
 kernel/fork.c            |  5 +++++
 4 files changed, 24 insertions(+), 1 deletion(-)

diff --git a/include/linux/io_uring.h b/include/linux/io_uring.h
index 85fe4e6b275c..d1aa4edfc2a5 100644
--- a/include/linux/io_uring.h
+++ b/include/linux/io_uring.h
@@ -12,6 +12,7 @@ void __io_uring_free(struct task_struct *tsk);
 void io_uring_unreg_ringfd(void);
 const char *io_uring_get_opcode(u8 opcode);
 bool io_is_uring_fops(struct file *file);
+int __io_uring_fork(struct task_struct *tsk);
 
 static inline void io_uring_files_cancel(void)
 {
@@ -25,9 +26,16 @@ static inline void io_uring_task_cancel(void)
 }
 static inline void io_uring_free(struct task_struct *tsk)
 {
-	if (tsk->io_uring)
+	if (tsk->io_uring || tsk->io_uring_restrict)
 		__io_uring_free(tsk);
 }
+static inline int io_uring_fork(struct task_struct *tsk)
+{
+	if (tsk->io_uring_restrict)
+		return __io_uring_fork(tsk);
+
+	return 0;
+}
 #else
 static inline void io_uring_task_cancel(void)
 {
@@ -46,6 +54,10 @@ static inline bool io_is_uring_fops(struct file *file)
 {
 	return false;
 }
+static inline int io_uring_fork(struct task_struct *tsk)
+{
+	return 0;
+}
 #endif
 
 #endif
diff --git a/include/linux/sched.h b/include/linux/sched.h
index d395f2810fac..9abbd11bb87c 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1190,6 +1190,7 @@ struct task_struct {
 
 #ifdef CONFIG_IO_URING
 	struct io_uring_task		*io_uring;
+	struct io_restriction		*io_uring_restrict;
 #endif
 
 	/* Namespaces: */
diff --git a/io_uring/tctx.c b/io_uring/tctx.c
index 5b66755579c0..cca13d291cfd 100644
--- a/io_uring/tctx.c
+++ b/io_uring/tctx.c
@@ -351,3 +351,8 @@ int io_ringfd_unregister(struct io_ring_ctx *ctx, void __user *__arg,
 
 	return i ? i : ret;
 }
+
+int __io_uring_fork(struct task_struct *tsk)
+{
+	return 0;
+}
diff --git a/kernel/fork.c b/kernel/fork.c
index b1f3915d5f8e..08a2515380ec 100644
--- a/kernel/fork.c
+++ b/kernel/fork.c
@@ -97,6 +97,7 @@
 #include <linux/kasan.h>
 #include <linux/scs.h>
 #include <linux/io_uring.h>
+#include <linux/io_uring_types.h>
 #include <linux/bpf.h>
 #include <linux/stackprotector.h>
 #include <linux/user_events.h>
@@ -2129,6 +2130,9 @@ __latent_entropy struct task_struct *copy_process(
 
 #ifdef CONFIG_IO_URING
 	p->io_uring = NULL;
+	retval = io_uring_fork(p);
+	if (unlikely(retval))
+		goto bad_fork_cleanup_delayacct;
 #endif
 
 	p->default_timer_slack_ns = current->timer_slack_ns;
@@ -2525,6 +2529,7 @@ __latent_entropy struct task_struct *copy_process(
 	mpol_put(p->mempolicy);
 #endif
 bad_fork_cleanup_delayacct:
+	io_uring_free(p);
 	delayacct_tsk_free(p);
 bad_fork_cleanup_count:
 	dec_rlimit_ucounts(task_ucounts(p), UCOUNT_RLIMIT_NPROC, 1);
-- 
2.51.0


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

* [PATCH 5/5] io_uring: allow registration of per-task restrictions
  2026-01-16 22:38 [PATCHSET RFC v4] Inherited restrictions and BPF filtering Jens Axboe
                   ` (3 preceding siblings ...)
  2026-01-16 22:38 ` [PATCH 4/5] io_uring: add task fork hook Jens Axboe
@ 2026-01-16 22:38 ` Jens Axboe
  4 siblings, 0 replies; 6+ messages in thread
From: Jens Axboe @ 2026-01-16 22:38 UTC (permalink / raw)
  To: io-uring; +Cc: Jens Axboe

Currently io_uring supports restricting operations on a per-ring basis.
To use those, the ring must be setup in a disabled state by setting
IORING_SETUP_R_DISABLED. Then restrictions can be set for the ring, and
the ring can then be enabled.

This commit adds support for IORING_REGISTER_RESTRICTIONS with ring_fd
== -1, like the other "blind" register opcodes which work on the task
rather than a specific ring. This allows registration of the same kind
of restrictions as can been done on a specific ring, but with the task
itself. Once done, any ring created will inherit these restrictions.

If a restriction filter is registered with a task, then it's inherited
on fork for its children. Children may only further restrict operations,
not extend them.

Inheriting restrictions include both the classic
IORING_REGISTER_RESTRICTIONS based restrictions, as well as the BPF
filters that have been registered with the task via
IORING_REGISTER_BPF_FILTER.

Signed-off-by: Jens Axboe <axboe@kernel.dk>
---
 include/linux/io_uring_types.h |  2 +
 include/uapi/linux/io_uring.h  |  7 ++++
 io_uring/bpf_filter.c          | 70 ++++++++++++++++++++++++++++++++++
 io_uring/bpf_filter.h          |  6 +++
 io_uring/io_uring.c            | 19 +++++++++
 io_uring/io_uring.h            |  1 +
 io_uring/register.c            | 65 +++++++++++++++++++++++++++++++
 io_uring/tctx.c                | 35 ++++++++++++-----
 8 files changed, 196 insertions(+), 9 deletions(-)

diff --git a/include/linux/io_uring_types.h b/include/linux/io_uring_types.h
index 1e91fa7ecbaf..f4a55c104825 100644
--- a/include/linux/io_uring_types.h
+++ b/include/linux/io_uring_types.h
@@ -236,6 +236,8 @@ struct io_restriction {
 	DECLARE_BITMAP(register_op, IORING_REGISTER_LAST);
 	DECLARE_BITMAP(sqe_op, IORING_OP_LAST);
 	struct io_bpf_filters *bpf_filters;
+	/* ->bpf_filters needs COW on modification */
+	bool bpf_filters_cow;
 	u8 sqe_flags_allowed;
 	u8 sqe_flags_required;
 	/* IORING_OP_* restrictions exist */
diff --git a/include/uapi/linux/io_uring.h b/include/uapi/linux/io_uring.h
index 27839318c43e..419bdfb48b9c 100644
--- a/include/uapi/linux/io_uring.h
+++ b/include/uapi/linux/io_uring.h
@@ -808,6 +808,13 @@ struct io_uring_restriction {
 	__u32 resv2[3];
 };
 
+struct io_uring_task_restriction {
+	__u16 flags;
+	__u16 nr_res;
+	__u32 resv[3];
+	__DECLARE_FLEX_ARRAY(struct io_uring_restriction, restrictions);
+};
+
 struct io_uring_clock_register {
 	__u32	clockid;
 	__u32	__resv[3];
diff --git a/io_uring/bpf_filter.c b/io_uring/bpf_filter.c
index 8ed5b913005a..30a9d7355cd7 100644
--- a/io_uring/bpf_filter.c
+++ b/io_uring/bpf_filter.c
@@ -215,6 +215,70 @@ static struct io_bpf_filters *io_new_bpf_filters(void)
 	return filters;
 }
 
+void io_bpf_filter_clone(struct io_restriction *dst, struct io_restriction *src)
+{
+	if (!src->bpf_filters)
+		return;
+
+	rcu_read_lock();
+	/*
+	 * If the src filter is going away, just ignore it.
+	 */
+	if (refcount_inc_not_zero(&src->bpf_filters->refs)) {
+		dst->bpf_filters = src->bpf_filters;
+		dst->bpf_filters_cow = true;
+	}
+	rcu_read_unlock();
+}
+
+/*
+ * Allocate a new struct io_bpf_filters. Used when a filter is cloned and
+ * modifications need to be made.
+ */
+static struct io_bpf_filters *io_bpf_filter_cow(struct io_restriction *src)
+{
+	struct io_bpf_filters *filters;
+	struct io_bpf_filter *srcf;
+	int i;
+
+	filters = io_new_bpf_filters();
+	if (IS_ERR(filters))
+		return filters;
+
+	/*
+	 * Iterate filters from src and assign in destination. Grabbing
+	 * a reference is enough, we don't need to duplicate the memory.
+	 * This is safe because filters are only ever appended to the
+	 * front of the list, hence the only memory ever touched inside
+	 * a filter is the refcount.
+	 */
+	rcu_read_lock();
+	for (i = 0; i < IORING_OP_LAST; i++) {
+		srcf = rcu_dereference(src->bpf_filters->filters[i]);
+		if (!srcf) {
+			continue;
+		} else if (srcf == &dummy_filter) {
+			rcu_assign_pointer(filters->filters[i], &dummy_filter);
+			continue;
+		}
+
+		/*
+		 * Getting a ref on the first node is enough, putting the
+		 * filter and iterating nodes to free will stop on the first
+		 * one that doesn't hit zero when dropping.
+		 */
+		if (!refcount_inc_not_zero(&srcf->refs))
+			goto err;
+		rcu_assign_pointer(filters->filters[i], srcf);
+	}
+	rcu_read_unlock();
+	return filters;
+err:
+	rcu_read_unlock();
+	__io_put_bpf_filters(filters);
+	return ERR_PTR(-EBUSY);
+}
+
 int io_register_bpf_filter(struct io_restriction *res,
 			   struct io_uring_bpf __user *arg)
 {
@@ -247,6 +311,12 @@ int io_register_bpf_filter(struct io_restriction *res,
 		filters = io_new_bpf_filters();
 		if (IS_ERR(filters))
 			return PTR_ERR(filters);
+	} else if (res->bpf_filters_cow) {
+		filters = io_bpf_filter_cow(res);
+		if (IS_ERR(filters))
+			return PTR_ERR(filters);
+		__io_put_bpf_filters(res->bpf_filters);
+		res->bpf_filters_cow = false;
 	}
 
 	prog = bpf_prog_get_type(reg.filter.prog_fd, BPF_PROG_TYPE_IO_URING);
diff --git a/io_uring/bpf_filter.h b/io_uring/bpf_filter.h
index a131953ce950..3f117a4c8752 100644
--- a/io_uring/bpf_filter.h
+++ b/io_uring/bpf_filter.h
@@ -11,6 +11,8 @@ int io_register_bpf_filter(struct io_restriction *res,
 
 void io_put_bpf_filters(struct io_restriction *res);
 
+void io_bpf_filter_clone(struct io_restriction *dst, struct io_restriction *src);
+
 static inline int io_uring_run_bpf_filters(struct io_restriction *res,
 					   struct io_kiocb *req)
 {
@@ -35,6 +37,10 @@ static inline int io_uring_run_bpf_filters(struct io_restriction *res,
 static inline void io_put_bpf_filters(struct io_restriction *res)
 {
 }
+static inline void io_bpf_filter_clone(struct io_restriction *dst,
+				       struct io_restriction *src)
+{
+}
 #endif /* CONFIG_IO_URING */
 
 #endif
diff --git a/io_uring/io_uring.c b/io_uring/io_uring.c
index 67533e494836..8e9d300b8604 100644
--- a/io_uring/io_uring.c
+++ b/io_uring/io_uring.c
@@ -3562,6 +3562,18 @@ int io_prepare_config(struct io_ctx_config *config)
 	return 0;
 }
 
+void io_restriction_clone(struct io_restriction *dst, struct io_restriction *src)
+{
+	memcpy(&dst->register_op, &src->register_op, sizeof(dst->register_op));
+	memcpy(&dst->sqe_op, &src->sqe_op, sizeof(dst->sqe_op));
+	dst->sqe_flags_allowed = src->sqe_flags_allowed;
+	dst->sqe_flags_required = src->sqe_flags_required;
+	dst->op_registered = src->op_registered;
+	dst->reg_registered = src->reg_registered;
+
+	io_bpf_filter_clone(dst, src);
+}
+
 static __cold int io_uring_create(struct io_ctx_config *config)
 {
 	struct io_uring_params *p = &config->p;
@@ -3622,6 +3634,13 @@ static __cold int io_uring_create(struct io_ctx_config *config)
 	else
 		ctx->notify_method = TWA_SIGNAL;
 
+	/*
+	 * If the current task has restrictions enabled, then copy them to
+	 * our newly created ring and mark it as registered.
+	 */
+	if (current->io_uring_restrict)
+		io_restriction_clone(&ctx->restrictions, current->io_uring_restrict);
+
 	/*
 	 * This is just grabbed for accounting purposes. When a process exits,
 	 * the mm is exited and dropped before the files, hence we need to hang
diff --git a/io_uring/io_uring.h b/io_uring/io_uring.h
index c5bbb43b5842..feb9f76761e9 100644
--- a/io_uring/io_uring.h
+++ b/io_uring/io_uring.h
@@ -195,6 +195,7 @@ void io_task_refs_refill(struct io_uring_task *tctx);
 bool __io_alloc_req_refill(struct io_ring_ctx *ctx);
 
 void io_activate_pollwq(struct io_ring_ctx *ctx);
+void io_restriction_clone(struct io_restriction *dst, struct io_restriction *src);
 
 static inline void io_lockdep_assert_cq_locked(struct io_ring_ctx *ctx)
 {
diff --git a/io_uring/register.c b/io_uring/register.c
index 30957c2cb5eb..12164b4e03aa 100644
--- a/io_uring/register.c
+++ b/io_uring/register.c
@@ -190,6 +190,67 @@ static __cold int io_register_restrictions(struct io_ring_ctx *ctx,
 	return 0;
 }
 
+static int io_register_restrictions_task(void __user *arg, unsigned int nr_args)
+{
+	struct io_uring_task_restriction __user *ures = arg;
+	struct io_uring_task_restriction tres;
+	struct io_restriction *res;
+	int ret;
+
+	/* Disallow if task already has registered restrictions */
+	if (current->io_uring_restrict)
+		return -EPERM;
+	if (nr_args != 1)
+		return -EINVAL;
+
+	if (copy_from_user(&tres, arg, sizeof(tres)))
+		return -EFAULT;
+
+	if (tres.flags)
+		return -EINVAL;
+	if (!mem_is_zero(tres.resv, sizeof(tres.resv)))
+		return -EINVAL;
+
+	res = kzalloc(sizeof(*res), GFP_KERNEL_ACCOUNT);
+	if (!res)
+		return -ENOMEM;
+
+	ret = io_parse_restrictions(ures->restrictions, tres.nr_res, res);
+	if (ret < 0) {
+		kfree(res);
+		return ret;
+	}
+	current->io_uring_restrict = res;
+	return 0;
+}
+
+static int io_register_bpf_filter_task(void __user *arg, unsigned int nr_args)
+{
+	struct io_restriction *res;
+	int ret;
+
+	if (nr_args != 1)
+		return -EINVAL;
+
+	/* If no task restrictions exist, setup a new set */
+	res = current->io_uring_restrict;
+	if (!res) {
+		res = kzalloc(sizeof(*res), GFP_KERNEL_ACCOUNT);
+		if (!res)
+			return -ENOMEM;
+	}
+
+	ret = io_register_bpf_filter(res, arg);
+	if (ret) {
+		if (res != current->io_uring_restrict)
+			kfree(res);
+		return ret;
+	}
+	if (!current->io_uring_restrict)
+		current->io_uring_restrict = res;
+	return 0;
+}
+
 static int io_register_enable_rings(struct io_ring_ctx *ctx)
 {
 	if (!(ctx->flags & IORING_SETUP_R_DISABLED))
@@ -909,6 +970,10 @@ static int io_uring_register_blind(unsigned int opcode, void __user *arg,
 		return io_uring_register_send_msg_ring(arg, nr_args);
 	case IORING_REGISTER_QUERY:
 		return io_query(arg, nr_args);
+	case IORING_REGISTER_RESTRICTIONS:
+		return io_register_restrictions_task(arg, nr_args);
+	case IORING_REGISTER_BPF_FILTER:
+		return io_register_bpf_filter_task(arg, nr_args);
 	}
 	return -EINVAL;
 }
diff --git a/io_uring/tctx.c b/io_uring/tctx.c
index cca13d291cfd..2c05e8f66172 100644
--- a/io_uring/tctx.c
+++ b/io_uring/tctx.c
@@ -11,6 +11,8 @@
 
 #include "io_uring.h"
 #include "tctx.h"
+#include "register.h"
+#include "bpf_filter.h"
 
 static struct io_wq *io_init_wq_offload(struct io_ring_ctx *ctx,
 					struct task_struct *task)
@@ -54,16 +56,23 @@ void __io_uring_free(struct task_struct *tsk)
 	 * node is stored in the xarray. Until that gets sorted out, attempt
 	 * an iteration here and warn if any entries are found.
 	 */
-	xa_for_each(&tctx->xa, index, node) {
-		WARN_ON_ONCE(1);
-		break;
-	}
-	WARN_ON_ONCE(tctx->io_wq);
-	WARN_ON_ONCE(tctx->cached_refs);
+	if (tctx) {
+		xa_for_each(&tctx->xa, index, node) {
+			WARN_ON_ONCE(1);
+			break;
+		}
+		WARN_ON_ONCE(tctx->io_wq);
+		WARN_ON_ONCE(tctx->cached_refs);
 
-	percpu_counter_destroy(&tctx->inflight);
-	kfree(tctx);
-	tsk->io_uring = NULL;
+		percpu_counter_destroy(&tctx->inflight);
+		kfree(tctx);
+		tsk->io_uring = NULL;
+	}
+	if (tsk->io_uring_restrict) {
+		io_put_bpf_filters(tsk->io_uring_restrict);
+		kfree(tsk->io_uring_restrict);
+		tsk->io_uring_restrict = NULL;
+	}
 }
 
 __cold int io_uring_alloc_task_context(struct task_struct *task,
@@ -354,5 +363,13 @@ int io_ringfd_unregister(struct io_ring_ctx *ctx, void __user *__arg,
 
 int __io_uring_fork(struct task_struct *tsk)
 {
+	struct io_restriction *res, *src = tsk->io_uring_restrict;
+
+	res = kzalloc(sizeof(*res), GFP_KERNEL_ACCOUNT);
+	if (!res)
+		return -ENOMEM;
+
+	tsk->io_uring_restrict = res;
+	io_restriction_clone(res, src);
 	return 0;
 }
-- 
2.51.0


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

end of thread, other threads:[~2026-01-16 22:44 UTC | newest]

Thread overview: 6+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2026-01-16 22:38 [PATCHSET RFC v4] Inherited restrictions and BPF filtering Jens Axboe
2026-01-16 22:38 ` [PATCH 1/5] io_uring: add support for BPF filtering for opcode restrictions Jens Axboe
2026-01-16 22:38 ` [PATCH 2/5] io_uring/net: allow filtering on IORING_OP_SOCKET data Jens Axboe
2026-01-16 22:38 ` [PATCH 3/5] io_uring/bpf_filter: add ref counts to struct io_bpf_filter Jens Axboe
2026-01-16 22:38 ` [PATCH 4/5] io_uring: add task fork hook Jens Axboe
2026-01-16 22:38 ` [PATCH 5/5] io_uring: allow registration of per-task restrictions Jens Axboe

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox