linux-mm.kvack.org archive mirror
 help / color / mirror / Atom feed
* [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups
@ 2022-01-20 15:55 Peter Zijlstra
  2022-01-20 15:55 ` [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages Peter Zijlstra
                   ` (6 more replies)
  0 siblings, 7 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-20 15:55 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, peterz, pjt, posk,
	avagin, jannh, tdelisle, mark.rutland, posk

Latest version, many changes since last time, still under heavy discussion.

Seems to work with the test-case I have (below), but that still has a few gaps,
coverage wise.

Still haven't done the SMP wakeup thing, finally get the idea with
worker-timeouts but haven't yet implemented that.

Happy hacking..

----

#define _GNU_SOURCE
#include <unistd.h>
#include <sys/types.h>
#include <sys/syscall.h>
#include <pthread.h>
#include <stdio.h>
#include <stdlib.h>
#include <errno.h>
#include <signal.h>

#ifndef __NR_umcg_ctl
#define __NR_umcg_ctl  450
#define __NR_umcg_wait 451
#define __NR_umcg_kick 452
#endif

#include <linux/list.h>
#include "include/uapi/linux/umcg.h"

/* syscall wrappers */

static inline int
sys_umcg_ctl(u32 flags, struct umcg_task *self, clockid_t which_clock)
{
	return syscall(__NR_umcg_ctl, flags, self, which_clock);
}

static inline int
sys_umcg_wait(u32 flags, u64 timo)
{
	return syscall(__NR_umcg_wait, flags, timo);
}

static inline int
sys_umcg_kick(u32 flags, pid_t tid)
{
	return syscall(__NR_umcg_kick, flags, tid);
}

/* the 'foo' scheduler */

struct foo_task {
	struct umcg_task	task;
	struct list_head	node;
	pid_t			tid;
};

struct foo_server {
	struct umcg_task	task;
	struct list_head	node;
	pid_t			tid;
	struct foo_task		*cur;
	int			workers;
};

void foo_add(struct foo_server *server, struct umcg_task *t)
{
	struct foo_task *foo = container_of(t, struct foo_task, task);

	t->runnable_workers_ptr = 0ULL;
	list_add_tail(&foo->node, &server->node);
}

struct foo_task *foo_pick_next(struct foo_server *server)
{
	struct foo_task *first = NULL;

	if (list_empty(&server->node))
		return first;

	first = list_first_entry(&server->node, struct foo_task, node);
	list_del(&first->node);
	return first;
}

#define NSEC_PER_SEC 1000000000ULL

u64 foo_time(void)
{
	struct timespec ts;
	clock_gettime(CLOCK_MONOTONIC, &ts);
	return (unsigned long long)ts.tv_sec * NSEC_PER_SEC + ts.tv_nsec;
}

void foo_yield(struct umcg_task *self)
{
	self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
	sys_umcg_wait(0, 0);
}

#define TICK_NSEC NSEC_PER_SEC

static volatile bool foo_preemptible = false;

/* our workers */

static volatile bool done = false;

static void umcg_signal(int signr)
{
	done = true;
}

/* always running worker */
void *worker_fn0(void *arg)
{
	struct foo_server *server = arg;
	struct foo_task task = { };
	unsigned long i;
	int ret;

	task.tid = gettid();
	task.task.server_tid = server->tid;
	task.task.state = UMCG_TASK_BLOCKED;

	printf("A == %d\n", gettid());

	ret = sys_umcg_ctl(UMCG_CTL_REGISTER|UMCG_CTL_WORKER, &task.task, CLOCK_MONOTONIC);
	if (ret) {
		perror("umcg_ctl(A): ");
		exit(-1);
	}

	__atomic_add_fetch(&server->workers, 1, __ATOMIC_RELAXED);

	while (!done) {
		int x = i++;

		if (!(x % 1000000)) {
			putchar('.');
			fflush(stdout);
		}

		/* co-operative or preemptible */
		if (!foo_preemptible && !(x % 10000000))
			foo_yield(&task.task);
	}

	printf("A == done\n");

	__atomic_add_fetch(&server->workers, -1, __ATOMIC_RELAXED);

	ret = sys_umcg_ctl(UMCG_CTL_UNREGISTER|UMCG_CTL_WORKER, &task.task, 0);
	if (ret) {
		perror("umcg_ctl(~A): ");
		exit(-1);
	}

	return NULL;
}

/* event driven worker */
void *worker_fn1(void *arg)
{
	struct foo_server *server = arg;
	struct foo_task task = { };
	int ret;

	task.tid = gettid();
	task.task.server_tid = server->tid;
	task.task.state = UMCG_TASK_BLOCKED;

	printf("B == %d\n", gettid());

	ret = sys_umcg_ctl(UMCG_CTL_REGISTER|UMCG_CTL_WORKER, &task.task, CLOCK_MONOTONIC);
	if (ret) {
		perror("umcg_ctl(B): ");
		exit(-1);
	}

	__atomic_add_fetch(&server->workers, 1, __ATOMIC_RELAXED);

	while (!done) {
		printf("B\n");
		fflush(stdout);

		sleep(1);
	}

	printf("B == done\n");

	__atomic_add_fetch(&server->workers, -1, __ATOMIC_RELAXED);

	ret = sys_umcg_ctl(UMCG_CTL_UNREGISTER|UMCG_CTL_WORKER, &task.task, 0);
	if (ret) {
		perror("umcg_ctl(~B): ");
		exit(-1);
	}

	return NULL;
}

void *worker_fn2(void *arg)
{
	struct foo_server *server = arg;
	struct foo_task task = { };
	int ret;

	task.tid = gettid();
	task.task.server_tid = server->tid;
	task.task.state = UMCG_TASK_BLOCKED;

	printf("C == %d\n", gettid());

	ret = sys_umcg_ctl(UMCG_CTL_REGISTER|UMCG_CTL_WORKER, &task.task, CLOCK_MONOTONIC);
	if (ret) {
		perror("umcg_ctl(C): ");
		exit(-1);
	}

	__atomic_add_fetch(&server->workers, 1, __ATOMIC_RELAXED);

	while (!done) {
		printf("C\n");
		fflush(stdout);

		sleep(2);
	}

	printf("C == done\n");

	__atomic_add_fetch(&server->workers, -1, __ATOMIC_RELAXED);

	ret = sys_umcg_ctl(UMCG_CTL_UNREGISTER|UMCG_CTL_WORKER, &task.task, 0);
	if (ret) {
		perror("umcg_ctl(~C): ");
		exit(-1);
	}

	return NULL;
}

/* the server */

int main(int argc, char **argv)
{
	struct umcg_task *runnable_ptr, *next;
	struct foo_server server = { };
	pthread_t worker[3];
	u64 timeout = 0;
	u32 tid;
	int ret;

	struct sigaction sa = {
		.sa_handler = umcg_signal,
	};

	sigaction(SIGINT, &sa, NULL);

	printf("server == %d\n", gettid());
	fflush(stdout);

	server.tid = gettid();
	INIT_LIST_HEAD(&server.node);
	server.task.server_tid = gettid();
	server.task.state = UMCG_TASK_RUNNING;

	ret = sys_umcg_ctl(UMCG_CTL_REGISTER, &server.task, CLOCK_MONOTONIC);
	if (ret) {
		perror("umcg_ctl: ");
		exit(-1);
	}

	pthread_create(&worker[0], NULL, worker_fn0, &server);
	pthread_create(&worker[1], NULL, worker_fn1, &server);
	pthread_create(&worker[2], NULL, worker_fn2, &server);

	if (argc > 1) {
		foo_preemptible = true;
		/*
		 * setup preemption tick
		 */
		timeout = foo_time() + TICK_NSEC;
	}

	while (!(done && !__atomic_load_n(&server.workers, __ATOMIC_RELAXED))) {
		/*
		 * Mark the server as runnable first, so we can detect
		 * additions to the runnable list after we read it.
		 */
		__atomic_store_n(&server.task.state,
				 UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT,
				 __ATOMIC_RELAXED);

		/*
		 * comsume the runnable notification list and add
		 * the tasks to our local runqueue.
		 */
		runnable_ptr = (void*)__atomic_exchange_n(&server.task.runnable_workers_ptr,
							  NULL, __ATOMIC_SEQ_CST);
		while (runnable_ptr) {
			next = (void *)runnable_ptr->runnable_workers_ptr;
			foo_add(&server, runnable_ptr);
			runnable_ptr = next;
		}

		if (server.cur && server.cur->task.state == UMCG_TASK_RUNNING) {
			/*
			 * Assert ::next_tid still points there and has RUNNING bit on
			 */
			if (server.task.next_tid != (server.cur->tid | UMCG_TID_RUNNING)) {
				printf("current not running: %d %x\n",
				       server.task.next_tid & UMCG_TID_MASK,
				       server.task.next_tid & ~UMCG_TID_MASK);
				exit(-1);
			}

			putchar('x');
		} else {
			tid = 0;
			server.cur = foo_pick_next(&server);
			if (server.cur)
				tid = server.cur->tid;

			__atomic_store_n(&server.task.next_tid, tid, __ATOMIC_RELAXED);

			printf("pick: %d\n", tid);
		}
		fflush(stdout);

		ret = sys_umcg_wait(0, timeout);

		/*
		 * If we set ::next_tid but it hasn't been consumed by the
		 * syscall due to failure, make sure to put the task back on
		 * the queue, lest we leak it.
		 */
		tid = __atomic_load_n(&server.task.next_tid, __ATOMIC_RELAXED);
		if (tid && !(tid & UMCG_TID_RUNNING)) {
			foo_add(&server, &server.cur->task);
			server.cur = NULL;
			putchar('*');
		}

		if (!ret)
			continue;

		switch (errno) {
		case EAGAIN:
			/*
			 * Got a wakeup, try again.
			 */
			continue;

		case ETIMEDOUT:
			/*
			 * timeout: drive preemption
			 */
			putchar('t');
			fflush(stdout);

			/*
			 * Next tick..
			 */
			timeout += TICK_NSEC;

			/*
			 * If we have a current, cmpxchg set TF_PREEMPT and on success
			 * send it a signal to kick it into the kernel such that
			 * it might re-report itself runnable.
			 */
			if (server.cur) {
				struct foo_task *t = server.cur;
				u32 val = UMCG_TASK_RUNNING;
				u32 new = UMCG_TASK_RUNNING | UMCG_TF_PREEMPT;

				if (__atomic_compare_exchange_n(&t->task.state, &val, new,
								false, __ATOMIC_SEQ_CST, __ATOMIC_SEQ_CST)) {
					sys_umcg_kick(0, t->tid);
				}
			}
			/*
			 * Either way around, if the cmpxchg
			 * failed the task will have blocked
			 * and we should re-start the loop.
			 */
			continue;

		default:
			printf("errno: %d\n", errno);
			perror("wait:");
			exit(-1);
		}
	}

	pthread_join(worker[0], NULL);
	pthread_join(worker[1], NULL);
	pthread_join(worker[2], NULL);

	return 0;
}




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

* [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
@ 2022-01-20 15:55 ` Peter Zijlstra
  2022-01-20 18:03   ` Nadav Amit
  2022-01-20 18:25   ` David Hildenbrand
  2022-01-20 15:55 ` [RFC][PATCH v2 2/5] entry,x86: Create common IRQ operations for exceptions Peter Zijlstra
                   ` (5 subsequent siblings)
  6 siblings, 2 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-20 15:55 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, peterz, pjt, posk,
	avagin, jannh, tdelisle, mark.rutland, posk

Add a guarantee for Anon pages that pin_user_page*() ensures the
user-mapping of these pages stay preserved. In order to ensure this
all rmap users have been audited:

 vmscan:	already fails eviction due to page_maybe_dma_pinned()

 migrate:	migration will fail on pinned pages due to
		expected_page_refs() not matching, however that is
		*after* try_to_migrate() has already destroyed the
		user mapping of these pages. Add an early exit for
		this case.

 numa-balance:	as per the above, pinned pages cannot be migrated,
		however numa balancing scanning will happily PROT_NONE
		them to get usage information on these pages. Avoid
		this for pinned pages.

None of the other rmap users (damon,page-idle,mlock,..) unmap the
page, they mostly just muck about with reference,dirty flags etc.

This same guarantee cannot be provided for Shared (file) pages due to
dirty page tracking.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 mm/migrate.c  |   10 +++++++++-
 mm/mprotect.c |    6 ++++++
 2 files changed, 15 insertions(+), 1 deletion(-)

--- a/mm/migrate.c
+++ b/mm/migrate.c
@@ -1472,7 +1472,15 @@ int migrate_pages(struct list_head *from
 			nr_subpages = thp_nr_pages(page);
 			cond_resched();
 
-			if (PageHuge(page))
+			/*
+			 * If the page has a pin then expected_page_refs() will
+			 * not match and the whole migration will fail later
+			 * anyway, fail early and preserve the mappings.
+			 */
+			if (page_maybe_dma_pinned(page))
+				rc = -EAGAIN;
+
+			else if (PageHuge(page))
 				rc = unmap_and_move_huge_page(get_new_page,
 						put_new_page, private, page,
 						pass > 2, mode, reason,
--- a/mm/mprotect.c
+++ b/mm/mprotect.c
@@ -106,6 +106,12 @@ static unsigned long change_pte_range(st
 					continue;
 
 				/*
+				 * Can't migrate pinned pages, avoid touching them.
+				 */
+				if (page_maybe_dma_pinned(page))
+					continue;
+
+				/*
 				 * Don't mess with PTEs if page is already on the node
 				 * a single-threaded process is running on.
 				 */




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

* [RFC][PATCH v2 2/5] entry,x86: Create common IRQ operations for exceptions
  2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
  2022-01-20 15:55 ` [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages Peter Zijlstra
@ 2022-01-20 15:55 ` Peter Zijlstra
  2022-01-21 16:34   ` Mark Rutland
  2022-01-20 15:55 ` [RFC][PATCH v2 3/5] sched/umcg: add WF_CURRENT_CPU and externise ttwu Peter Zijlstra
                   ` (4 subsequent siblings)
  6 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-20 15:55 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, peterz, pjt, posk,
	avagin, jannh, tdelisle, mark.rutland, posk

A number of exceptions can re-enable IRQs and schedule (provided the
context taking the exception has IRQs enabled etc.). Currently this
isn't standardized and each architecture does it slightly different.

(Notably, ARM64 restores/inherits more than just the IRQ state,
suggesting at least some form of arch_ hook might be appropriate on
top of this).

Create a single set of functions for this and convert x86 to use
these. The purpose is to have a single (common) place to hook in order
to cover all the exceptions that can schedule().

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 arch/x86/kernel/traps.c      |   48 ++++++++++++++++---------------------------
 arch/x86/mm/fault.c          |   28 ++++++++++++-------------
 include/linux/entry-common.h |   24 +++++++++++++++++++++
 3 files changed, 56 insertions(+), 44 deletions(-)

--- a/arch/x86/kernel/traps.c
+++ b/arch/x86/kernel/traps.c
@@ -73,18 +73,6 @@
 
 DECLARE_BITMAP(system_vectors, NR_VECTORS);
 
-static inline void cond_local_irq_enable(struct pt_regs *regs)
-{
-	if (regs->flags & X86_EFLAGS_IF)
-		local_irq_enable();
-}
-
-static inline void cond_local_irq_disable(struct pt_regs *regs)
-{
-	if (regs->flags & X86_EFLAGS_IF)
-		local_irq_disable();
-}
-
 __always_inline int is_valid_bugaddr(unsigned long addr)
 {
 	if (addr < TASK_SIZE_MAX)
@@ -177,9 +165,9 @@ static void do_error_trap(struct pt_regs
 
 	if (notify_die(DIE_TRAP, str, regs, error_code, trapnr, signr) !=
 			NOTIFY_STOP) {
-		cond_local_irq_enable(regs);
+		irqentry_irq_enable(regs);
 		do_trap(trapnr, signr, str, regs, error_code, sicode, addr);
-		cond_local_irq_disable(regs);
+		irqentry_irq_disable(regs);
 	}
 }
 
@@ -300,7 +288,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_alignment_
 	if (!user_mode(regs))
 		die("Split lock detected\n", regs, error_code);
 
-	local_irq_enable();
+	irqentry_irq_enable(regs);
 
 	if (handle_user_split_lock(regs, error_code))
 		goto out;
@@ -309,7 +297,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_alignment_
 		error_code, BUS_ADRALN, NULL);
 
 out:
-	local_irq_disable();
+	irqentry_irq_disable(regs);
 }
 
 #ifdef CONFIG_VMAP_STACK
@@ -473,14 +461,14 @@ DEFINE_IDTENTRY(exc_bounds)
 	if (notify_die(DIE_TRAP, "bounds", regs, 0,
 			X86_TRAP_BR, SIGSEGV) == NOTIFY_STOP)
 		return;
-	cond_local_irq_enable(regs);
+	irqentry_irq_enable(regs);
 
 	if (!user_mode(regs))
 		die("bounds", regs, 0);
 
 	do_trap(X86_TRAP_BR, SIGSEGV, "bounds", regs, 0, 0, NULL);
 
-	cond_local_irq_disable(regs);
+	irqentry_irq_disable(regs);
 }
 
 enum kernel_gp_hint {
@@ -567,7 +555,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_general_pr
 	unsigned long gp_addr;
 	int ret;
 
-	cond_local_irq_enable(regs);
+	irqentry_irq_enable(regs);
 
 	if (static_cpu_has(X86_FEATURE_UMIP)) {
 		if (user_mode(regs) && fixup_umip_exception(regs))
@@ -638,7 +626,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_general_pr
 	die_addr(desc, regs, error_code, gp_addr);
 
 exit:
-	cond_local_irq_disable(regs);
+	irqentry_irq_disable(regs);
 }
 
 static bool do_int3(struct pt_regs *regs)
@@ -665,9 +653,9 @@ static void do_int3_user(struct pt_regs
 	if (do_int3(regs))
 		return;
 
-	cond_local_irq_enable(regs);
+	irqentry_irq_enable(regs);
 	do_trap(X86_TRAP_BP, SIGTRAP, "int3", regs, 0, 0, NULL);
-	cond_local_irq_disable(regs);
+	irqentry_irq_disable(regs);
 }
 
 DEFINE_IDTENTRY_RAW(exc_int3)
@@ -1003,7 +991,7 @@ static __always_inline void exc_debug_us
 		goto out;
 
 	/* It's safe to allow irq's after DR6 has been saved */
-	local_irq_enable();
+	irqentry_irq_enable(regs);
 
 	if (v8086_mode(regs)) {
 		handle_vm86_trap((struct kernel_vm86_regs *)regs, 0, X86_TRAP_DB);
@@ -1020,7 +1008,7 @@ static __always_inline void exc_debug_us
 		send_sigtrap(regs, 0, get_si_code(dr6));
 
 out_irq:
-	local_irq_disable();
+	irqentry_irq_disable(regs);
 out:
 	instrumentation_end();
 	irqentry_exit_to_user_mode(regs);
@@ -1064,7 +1052,7 @@ static void math_error(struct pt_regs *r
 	char *str = (trapnr == X86_TRAP_MF) ? "fpu exception" :
 						"simd exception";
 
-	cond_local_irq_enable(regs);
+	irqentry_irq_enable(regs);
 
 	if (!user_mode(regs)) {
 		if (fixup_exception(regs, trapnr, 0, 0))
@@ -1099,7 +1087,7 @@ static void math_error(struct pt_regs *r
 	force_sig_fault(SIGFPE, si_code,
 			(void __user *)uprobe_get_trap_addr(regs));
 exit:
-	cond_local_irq_disable(regs);
+	irqentry_irq_disable(regs);
 }
 
 DEFINE_IDTENTRY(exc_coprocessor_error)
@@ -1160,7 +1148,7 @@ static bool handle_xfd_event(struct pt_r
 	if (WARN_ON(!user_mode(regs)))
 		return false;
 
-	local_irq_enable();
+	irqentry_irq_enable(regs);
 
 	err = xfd_enable_feature(xfd_err);
 
@@ -1173,7 +1161,7 @@ static bool handle_xfd_event(struct pt_r
 		break;
 	}
 
-	local_irq_disable();
+	irqentry_irq_disable(regs);
 	return true;
 }
 
@@ -1188,12 +1176,12 @@ DEFINE_IDTENTRY(exc_device_not_available
 	if (!boot_cpu_has(X86_FEATURE_FPU) && (cr0 & X86_CR0_EM)) {
 		struct math_emu_info info = { };
 
-		cond_local_irq_enable(regs);
+		irqentry_irq_enable(regs);
 
 		info.regs = regs;
 		math_emulate(&info);
 
-		cond_local_irq_disable(regs);
+		irqentry_irq_disable(regs);
 		return;
 	}
 #endif
--- a/arch/x86/mm/fault.c
+++ b/arch/x86/mm/fault.c
@@ -1209,6 +1209,12 @@ do_kern_addr_fault(struct pt_regs *regs,
 NOKPROBE_SYMBOL(do_kern_addr_fault);
 
 /*
+ * EFLAGS[3] is unused and ABI defined to be 0, use it to store IRQ state,
+ * because do_user_addr_fault() is too convoluted to track things.
+ */
+#define X86_EFLAGS_MISC		(1UL << 3)
+
+/*
  * Handle faults in the user portion of the address space.  Nothing in here
  * should check X86_PF_USER without a specific justification: for almost
  * all purposes, we should treat a normal kernel access to user memory
@@ -1290,13 +1296,11 @@ void do_user_addr_fault(struct pt_regs *
 	 * User-mode registers count as a user access even for any
 	 * potential system fault or CPU buglet:
 	 */
-	if (user_mode(regs)) {
-		local_irq_enable();
+	if (user_mode(regs))
 		flags |= FAULT_FLAG_USER;
-	} else {
-		if (regs->flags & X86_EFLAGS_IF)
-			local_irq_enable();
-	}
+
+	irqentry_irq_enable(regs);
+	regs->flags |= X86_EFLAGS_MISC;
 
 	perf_sw_event(PERF_COUNT_SW_PAGE_FAULTS, 1, regs, address);
 
@@ -1483,14 +1487,10 @@ handle_page_fault(struct pt_regs *regs,
 		do_kern_addr_fault(regs, error_code, address);
 	} else {
 		do_user_addr_fault(regs, error_code, address);
-		/*
-		 * User address page fault handling might have reenabled
-		 * interrupts. Fixing up all potential exit points of
-		 * do_user_addr_fault() and its leaf functions is just not
-		 * doable w/o creating an unholy mess or turning the code
-		 * upside down.
-		 */
-		local_irq_disable();
+		if (regs->flags & X86_EFLAGS_MISC) {
+			regs->flags &= ~X86_EFLAGS_MISC;
+			irqentry_irq_disable(regs);
+		}
 	}
 }
 
--- a/include/linux/entry-common.h
+++ b/include/linux/entry-common.h
@@ -7,6 +7,7 @@
 #include <linux/syscalls.h>
 #include <linux/seccomp.h>
 #include <linux/sched.h>
+#include <asm/ptrace.h>
 
 #include <asm/entry-common.h>
 
@@ -213,6 +214,29 @@ static inline void local_irq_disable_exi
 #endif
 
 /**
+ * irqentry_irq_enable - Conditionally enable IRQs from exceptions
+ *
+ * Common code for exceptions to (re)enable IRQs, typically done to allow
+ * from-user exceptions to schedule (since they run on the task stack).
+ */
+static inline void irqentry_irq_enable(struct pt_regs *regs)
+{
+	if (!regs_irqs_disabled(regs))
+		local_irq_enable();
+}
+
+/**
+ * irqentry_irq_disable - Conditionally disable IRQs from exceptions
+ *
+ * Counterpart of irqentry_irq_enable().
+ */
+static inline void irqentry_irq_disable(struct pt_regs *regs)
+{
+	if (!regs_irqs_disabled(regs))
+		local_irq_disable();
+}
+
+/**
  * arch_exit_to_user_mode_work - Architecture specific TIF work for exit
  *				 to user mode.
  * @regs:	Pointer to currents pt_regs




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

* [RFC][PATCH v2 3/5] sched/umcg: add WF_CURRENT_CPU and externise ttwu
  2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
  2022-01-20 15:55 ` [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages Peter Zijlstra
  2022-01-20 15:55 ` [RFC][PATCH v2 2/5] entry,x86: Create common IRQ operations for exceptions Peter Zijlstra
@ 2022-01-20 15:55 ` Peter Zijlstra
  2022-01-20 15:55 ` [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user() Peter Zijlstra
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-20 15:55 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, peterz, pjt, posk,
	avagin, jannh, tdelisle, mark.rutland, posk

From: Peter Oskolkov <posk@posk.io>

Add WF_CURRENT_CPU wake flag that advices the scheduler to
move the wakee to the current CPU. This is useful for fast on-CPU
context switching use cases such as UMCG.

In addition, make ttwu external rather than static so that
the flag could be passed to it from outside of sched/core.c.

Signed-off-by: Peter Oskolkov <posk@google.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lkml.kernel.org/r/20211122211327.5931-2-posk@google.com
---
 kernel/sched/core.c  |    3 +--
 kernel/sched/fair.c  |    4 ++++
 kernel/sched/sched.h |   17 ++++++++++-------
 3 files changed, 15 insertions(+), 9 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -3980,8 +3980,7 @@ bool ttwu_state_match(struct task_struct
  * Return: %true if @p->state changes (an actual wakeup was done),
  *	   %false otherwise.
  */
-static int
-try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
+int try_to_wake_up(struct task_struct *p, unsigned int state, int wake_flags)
 {
 	unsigned long flags;
 	int cpu, success = 0;
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -6838,6 +6838,10 @@ select_task_rq_fair(struct task_struct *
 	if (wake_flags & WF_TTWU) {
 		record_wakee(p);
 
+		if ((wake_flags & WF_CURRENT_CPU) &&
+		    cpumask_test_cpu(cpu, p->cpus_ptr))
+			return cpu;
+
 		if (sched_energy_enabled()) {
 			new_cpu = find_energy_efficient_cpu(p, prev_cpu);
 			if (new_cpu >= 0)
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -2052,13 +2052,14 @@ static inline int task_on_rq_migrating(s
 }
 
 /* Wake flags. The first three directly map to some SD flag value */
-#define WF_EXEC     0x02 /* Wakeup after exec; maps to SD_BALANCE_EXEC */
-#define WF_FORK     0x04 /* Wakeup after fork; maps to SD_BALANCE_FORK */
-#define WF_TTWU     0x08 /* Wakeup;            maps to SD_BALANCE_WAKE */
-
-#define WF_SYNC     0x10 /* Waker goes to sleep after wakeup */
-#define WF_MIGRATED 0x20 /* Internal use, task got migrated */
-#define WF_ON_CPU   0x40 /* Wakee is on_cpu */
+#define WF_EXEC         0x02 /* Wakeup after exec; maps to SD_BALANCE_EXEC */
+#define WF_FORK         0x04 /* Wakeup after fork; maps to SD_BALANCE_FORK */
+#define WF_TTWU         0x08 /* Wakeup;            maps to SD_BALANCE_WAKE */
+
+#define WF_SYNC         0x10 /* Waker goes to sleep after wakeup */
+#define WF_MIGRATED     0x20 /* Internal use, task got migrated */
+#define WF_ON_CPU       0x40 /* Wakee is on_cpu */
+#define WF_CURRENT_CPU  0x80 /* Prefer to move the wakee to the current CPU. */
 
 #ifdef CONFIG_SMP
 static_assert(WF_EXEC == SD_BALANCE_EXEC);
@@ -3112,6 +3113,8 @@ static inline bool is_per_cpu_kthread(st
 extern void swake_up_all_locked(struct swait_queue_head *q);
 extern void __prepare_to_swait(struct swait_queue_head *q, struct swait_queue *wait);
 
+extern int try_to_wake_up(struct task_struct *tsk, unsigned int state, int wake_flags);
+
 #ifdef CONFIG_PREEMPT_DYNAMIC
 extern int preempt_dynamic_mode;
 extern int sched_dynamic_mode(const char *str);




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

* [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
                   ` (2 preceding siblings ...)
  2022-01-20 15:55 ` [RFC][PATCH v2 3/5] sched/umcg: add WF_CURRENT_CPU and externise ttwu Peter Zijlstra
@ 2022-01-20 15:55 ` Peter Zijlstra
  2022-01-27  2:17   ` Sean Christopherson
  2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
                   ` (2 subsequent siblings)
  6 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-20 15:55 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, peterz, pjt, posk,
	avagin, jannh, tdelisle, mark.rutland, posk, Sean Christopherson

Do try_cmpxchg() loops on userspace addresses.

Cc: Sean Christopherson <seanjc@google.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 arch/x86/include/asm/uaccess.h |   67 +++++++++++++++++++++++++++++++++++++++++
 1 file changed, 67 insertions(+)

--- a/arch/x86/include/asm/uaccess.h
+++ b/arch/x86/include/asm/uaccess.h
@@ -342,6 +342,24 @@ do {									\
 		     : [umem] "m" (__m(addr))				\
 		     : : label)
 
+#define __try_cmpxchg_user_asm(itype, ltype, _ptr, _pold, _new, label)	({ \
+	bool success;							\
+	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
+	__typeof__(*(_ptr)) __old = *_old;				\
+	__typeof__(*(_ptr)) __new = (_new);				\
+	asm_volatile_goto("\n"						\
+		     "1: " LOCK_PREFIX "cmpxchg"itype" %[new], %[ptr]\n"\
+		     _ASM_EXTABLE_UA(1b, %l[label])			\
+		     : CC_OUT(z) (success),				\
+		       [ptr] "+m" (*_ptr),				\
+		       [old] "+a" (__old)				\
+		     : [new] ltype (__new)				\
+		     : "memory", "cc"					\
+		     : label);						\
+	if (unlikely(!success))						\
+		*_old = __old;						\
+	likely(success);					})
+
 #else // !CONFIG_CC_HAS_ASM_GOTO_OUTPUT
 
 #ifdef CONFIG_X86_32
@@ -407,6 +425,30 @@ do {									\
 		     : [umem] "m" (__m(addr)),				\
 		       "0" (err))
 
+#define __try_cmpxchg_user_asm(itype, ltype, _ptr, _pold, _new, label)	({ \
+	int __err = 0;							\
+	bool success;							\
+	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
+	__typeof__(*(_ptr)) __old = *_old;				\
+	__typeof__(*(_ptr)) __new = (_new);				\
+	asm volatile("\n"						\
+		     "1: " LOCK_PREFIX "cmpxchg"itype" %[new], %[ptr]\n"\
+		     CC_SET(z)						\
+		     "2:\n"						\
+		     _ASM_EXTABLE_TYPE_REG(1b, 2b, EX_TYPE_EFAULT_REG,	\
+					   %[errout])			\
+		     : CC_OUT(z) (success),				\
+		       [errout] "+r" (__err),				\
+		       [ptr] "+m" (*_ptr),				\
+		       [old] "+a" (__old)				\
+		     : [new] ltype (__new)				\
+		     : "memory", "cc");					\
+	if (unlikely(__err))						\
+		goto label;						\
+	if (unlikely(!success))						\
+		*_old = __old;						\
+	likely(success);					})
+
 #endif // CONFIG_CC_HAS_ASM_GOTO_OUTPUT
 
 /* FIXME: this hack is definitely wrong -AK */
@@ -501,6 +543,31 @@ do {										\
 } while (0)
 #endif // CONFIG_CC_HAS_ASM_GOTO_OUTPUT
 
+extern void __try_cmpxchg_user_wrong_size(void);
+
+#define unsafe_try_cmpxchg_user(_ptr, _oldp, _nval, _label) ({		\
+	__typeof__(*(_ptr)) __ret;					\
+	switch (sizeof(__ret)) {					\
+	case 1:	__ret = __try_cmpxchg_user_asm("b", "q",		\
+					       (_ptr), (_oldp),		\
+					       (_nval), _label);	\
+		break;							\
+	case 2:	__ret = __try_cmpxchg_user_asm("w", "r",		\
+					       (_ptr), (_oldp),		\
+					       (_nval), _label);	\
+		break;							\
+	case 4:	__ret = __try_cmpxchg_user_asm("l", "r",		\
+					       (_ptr), (_oldp),		\
+					       (_nval), _label);	\
+		break;							\
+	case 8:	__ret = __try_cmpxchg_user_asm("q", "r",		\
+					       (_ptr), (_oldp),		\
+					       (_nval), _label);	\
+		break;							\
+	default: __try_cmpxchg_user_wrong_size();			\
+	}								\
+	__ret;						})
+
 /*
  * We want the unsafe accessors to always be inlined and use
  * the error labels - thus the macro games.




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

* [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
                   ` (3 preceding siblings ...)
  2022-01-20 15:55 ` [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user() Peter Zijlstra
@ 2022-01-20 15:55 ` Peter Zijlstra
  2022-01-21 11:47   ` Peter Zijlstra
                     ` (4 more replies)
  2022-01-20 17:28 ` [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Oskolkov
  2022-01-21 18:01 ` Steven Rostedt
  6 siblings, 5 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-20 15:55 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, peterz, pjt, posk,
	avagin, jannh, tdelisle, mark.rutland, posk

User Managed Concurrency Groups is an M:N threading toolkit that allows
constructing user space schedulers designed to efficiently manage
heterogeneous in-process workloads while maintaining high CPU
utilization (95%+).

XXX moar changelog explaining how this is moar awesome than
traditional user-space threading.

The big thing that's still missing is the SMP wake-to-remote-idle.

Originally-by: Peter Oskolkov <posk@google.com>
Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Link: https://lkml.kernel.org/r/20211214205358.701701555@infradead.org
---
 arch/x86/Kconfig                       |    1 
 arch/x86/entry/syscalls/syscall_64.tbl |    3 
 arch/x86/include/asm/thread_info.h     |    2 
 fs/exec.c                              |    1 
 include/linux/entry-common.h           |   16 
 include/linux/sched.h                  |   82 ++
 include/linux/syscalls.h               |    4 
 include/linux/thread_info.h            |    2 
 include/uapi/asm-generic/unistd.h      |    9 
 include/uapi/linux/umcg.h              |  141 ++++
 init/Kconfig                           |   15 
 kernel/entry/common.c                  |   18 
 kernel/exit.c                          |    5 
 kernel/sched/Makefile                  |    1 
 kernel/sched/core.c                    |    9 
 kernel/sched/umcg.c                    |  954 +++++++++++++++++++++++++++++++++
 kernel/sys_ni.c                        |    5 
 17 files changed, 1259 insertions(+), 9 deletions(-)

--- a/arch/x86/Kconfig
+++ b/arch/x86/Kconfig
@@ -248,6 +248,7 @@ config X86
 	select HAVE_RSEQ
 	select HAVE_SYSCALL_TRACEPOINTS
 	select HAVE_UNSTABLE_SCHED_CLOCK
+	select HAVE_UMCG			if X86_64
 	select HAVE_USER_RETURN_NOTIFIER
 	select HAVE_GENERIC_VDSO
 	select HOTPLUG_SMT			if SMP
--- a/arch/x86/entry/syscalls/syscall_64.tbl
+++ b/arch/x86/entry/syscalls/syscall_64.tbl
@@ -371,6 +371,9 @@
 447	common	memfd_secret		sys_memfd_secret
 448	common	process_mrelease	sys_process_mrelease
 449	common	futex_waitv		sys_futex_waitv
+450	common	umcg_ctl		sys_umcg_ctl
+451	common	umcg_wait		sys_umcg_wait
+452	common	umcg_kick		sys_umcg_kick
 
 #
 # Due to a historical design error, certain syscalls are numbered differently
--- a/arch/x86/include/asm/thread_info.h
+++ b/arch/x86/include/asm/thread_info.h
@@ -83,6 +83,7 @@ struct thread_info {
 #define TIF_NEED_RESCHED	3	/* rescheduling necessary */
 #define TIF_SINGLESTEP		4	/* reenable singlestep on user return*/
 #define TIF_SSBD		5	/* Speculative store bypass disable */
+#define TIF_UMCG		6	/* UMCG return to user hook */
 #define TIF_SPEC_IB		9	/* Indirect branch speculation mitigation */
 #define TIF_SPEC_L1D_FLUSH	10	/* Flush L1D on mm switches (processes) */
 #define TIF_USER_RETURN_NOTIFY	11	/* notify kernel of userspace return */
@@ -107,6 +108,7 @@ struct thread_info {
 #define _TIF_NEED_RESCHED	(1 << TIF_NEED_RESCHED)
 #define _TIF_SINGLESTEP		(1 << TIF_SINGLESTEP)
 #define _TIF_SSBD		(1 << TIF_SSBD)
+#define _TIF_UMCG		(1 << TIF_UMCG)
 #define _TIF_SPEC_IB		(1 << TIF_SPEC_IB)
 #define _TIF_SPEC_L1D_FLUSH	(1 << TIF_SPEC_L1D_FLUSH)
 #define _TIF_USER_RETURN_NOTIFY	(1 << TIF_USER_RETURN_NOTIFY)
--- a/fs/exec.c
+++ b/fs/exec.c
@@ -1838,6 +1838,7 @@ static int bprm_execve(struct linux_binp
 	current->fs->in_exec = 0;
 	current->in_execve = 0;
 	rseq_execve(current);
+	umcg_execve(current);
 	acct_update_integrals(current);
 	task_numa_free(current, false);
 	return retval;
--- a/include/linux/entry-common.h
+++ b/include/linux/entry-common.h
@@ -23,6 +23,10 @@
 # define _TIF_UPROBE			(0)
 #endif
 
+#ifndef _TIF_UMCG
+# define _TIF_UMCG			(0)
+#endif
+
 /*
  * SYSCALL_WORK flags handled in syscall_enter_from_user_mode()
  */
@@ -43,11 +47,13 @@
 				 SYSCALL_WORK_SYSCALL_EMU |		\
 				 SYSCALL_WORK_SYSCALL_AUDIT |		\
 				 SYSCALL_WORK_SYSCALL_USER_DISPATCH |	\
+				 SYSCALL_WORK_SYSCALL_UMCG |		\
 				 ARCH_SYSCALL_WORK_ENTER)
 #define SYSCALL_WORK_EXIT	(SYSCALL_WORK_SYSCALL_TRACEPOINT |	\
 				 SYSCALL_WORK_SYSCALL_TRACE |		\
 				 SYSCALL_WORK_SYSCALL_AUDIT |		\
 				 SYSCALL_WORK_SYSCALL_USER_DISPATCH |	\
+				 SYSCALL_WORK_SYSCALL_UMCG |		\
 				 SYSCALL_WORK_SYSCALL_EXIT_TRAP	|	\
 				 ARCH_SYSCALL_WORK_EXIT)
 
@@ -221,8 +227,11 @@ static inline void local_irq_disable_exi
  */
 static inline void irqentry_irq_enable(struct pt_regs *regs)
 {
-	if (!regs_irqs_disabled(regs))
+	if (!regs_irqs_disabled(regs)) {
 		local_irq_enable();
+		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
+			umcg_sys_enter(regs, -1);
+	}
 }
 
 /**
@@ -232,8 +241,11 @@ static inline void irqentry_irq_enable(s
  */
 static inline void irqentry_irq_disable(struct pt_regs *regs)
 {
-	if (!regs_irqs_disabled(regs))
+	if (!regs_irqs_disabled(regs)) {
+		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
+			umcg_sys_exit(regs);
 		local_irq_disable();
+	}
 }
 
 /**
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -67,6 +67,7 @@ struct sighand_struct;
 struct signal_struct;
 struct task_delay_info;
 struct task_group;
+struct umcg_task;
 
 /*
  * Task state bitmask. NOTE! These bits are also
@@ -1294,6 +1295,19 @@ struct task_struct {
 	unsigned long rseq_event_mask;
 #endif
 
+#ifdef CONFIG_UMCG
+	/* setup by sys_umcg_ctrl() */
+	clockid_t		umcg_clock;
+	struct umcg_task __user	*umcg_task;
+
+	/* setup by umcg_pin_enter() */
+	struct page		*umcg_page;
+
+	struct task_struct	*umcg_server;
+	struct umcg_task __user *umcg_server_task;
+	struct page		*umcg_server_page;
+#endif
+
 	struct tlbflush_unmap_batch	tlb_ubc;
 
 	union {
@@ -1687,6 +1701,13 @@ extern struct pid *cad_pid;
 #define PF_KTHREAD		0x00200000	/* I am a kernel thread */
 #define PF_RANDOMIZE		0x00400000	/* Randomize virtual address space */
 #define PF_SWAPWRITE		0x00800000	/* Allowed to write to swap */
+
+#ifdef CONFIG_UMCG
+#define PF_UMCG_WORKER		0x01000000	/* UMCG worker */
+#else
+#define PF_UMCG_WORKER		0x00000000
+#endif
+
 #define PF_NO_SETAFFINITY	0x04000000	/* Userland is not allowed to meddle with cpus_mask */
 #define PF_MCE_EARLY		0x08000000      /* Early kill for mce process policy */
 #define PF_MEMALLOC_PIN		0x10000000	/* Allocation context constrained to zones which allow long term pinning. */
@@ -2294,6 +2315,67 @@ static inline void rseq_execve(struct ta
 {
 }
 
+#endif
+
+#ifdef CONFIG_UMCG
+
+extern void umcg_sys_enter(struct pt_regs *regs, long syscall);
+extern void umcg_sys_exit(struct pt_regs *regs);
+extern void umcg_notify_resume(struct pt_regs *regs);
+extern void umcg_worker_exit(void);
+extern void umcg_clear_child(struct task_struct *tsk);
+
+/* Called by bprm_execve() in fs/exec.c. */
+static inline void umcg_execve(struct task_struct *tsk)
+{
+	if (tsk->umcg_task)
+		umcg_clear_child(tsk);
+}
+
+/* Called by do_exit() in kernel/exit.c. */
+static inline void umcg_handle_exit(void)
+{
+	if (current->flags & PF_UMCG_WORKER)
+		umcg_worker_exit();
+}
+
+/*
+ * umcg_wq_worker_[sleeping|running] are called in core.c by
+ * sched_submit_work() and sched_update_worker().
+ */
+extern void umcg_wq_worker_sleeping(struct task_struct *tsk);
+extern void umcg_wq_worker_running(struct task_struct *tsk);
+
+#else  /* CONFIG_UMCG */
+
+static inline void umcg_sys_enter(struct pt_regs *regs, long syscall)
+{
+}
+
+static inline void umcg_sys_exit(struct pt_regs *regs)
+{
+}
+
+static inline void umcg_notify_resume(struct pt_regs *regs)
+{
+}
+
+static inline void umcg_clear_child(struct task_struct *tsk)
+{
+}
+static inline void umcg_execve(struct task_struct *tsk)
+{
+}
+static inline void umcg_handle_exit(void)
+{
+}
+static inline void umcg_wq_worker_sleeping(struct task_struct *tsk)
+{
+}
+static inline void umcg_wq_worker_running(struct task_struct *tsk)
+{
+}
+
 #endif
 
 #ifdef CONFIG_DEBUG_RSEQ
--- a/include/linux/syscalls.h
+++ b/include/linux/syscalls.h
@@ -72,6 +72,7 @@ struct open_how;
 struct mount_attr;
 struct landlock_ruleset_attr;
 enum landlock_rule_type;
+struct umcg_task;
 
 #include <linux/types.h>
 #include <linux/aio_abi.h>
@@ -1057,6 +1058,9 @@ asmlinkage long sys_landlock_add_rule(in
 		const void __user *rule_attr, __u32 flags);
 asmlinkage long sys_landlock_restrict_self(int ruleset_fd, __u32 flags);
 asmlinkage long sys_memfd_secret(unsigned int flags);
+asmlinkage long sys_umcg_ctl(u32 flags, struct umcg_task __user *self, clockid_t which_clock);
+asmlinkage long sys_umcg_wait(u32 flags, u64 abs_timeout);
+asmlinkage long sys_umcg_kick(u32 flags, pid_t tid);
 
 /*
  * Architecture-specific system calls
--- a/include/linux/thread_info.h
+++ b/include/linux/thread_info.h
@@ -46,6 +46,7 @@ enum syscall_work_bit {
 	SYSCALL_WORK_BIT_SYSCALL_AUDIT,
 	SYSCALL_WORK_BIT_SYSCALL_USER_DISPATCH,
 	SYSCALL_WORK_BIT_SYSCALL_EXIT_TRAP,
+	SYSCALL_WORK_BIT_SYSCALL_UMCG,
 };
 
 #define SYSCALL_WORK_SECCOMP		BIT(SYSCALL_WORK_BIT_SECCOMP)
@@ -55,6 +56,7 @@ enum syscall_work_bit {
 #define SYSCALL_WORK_SYSCALL_AUDIT	BIT(SYSCALL_WORK_BIT_SYSCALL_AUDIT)
 #define SYSCALL_WORK_SYSCALL_USER_DISPATCH BIT(SYSCALL_WORK_BIT_SYSCALL_USER_DISPATCH)
 #define SYSCALL_WORK_SYSCALL_EXIT_TRAP	BIT(SYSCALL_WORK_BIT_SYSCALL_EXIT_TRAP)
+#define SYSCALL_WORK_SYSCALL_UMCG	BIT(SYSCALL_WORK_BIT_SYSCALL_UMCG)
 #endif
 
 #include <asm/thread_info.h>
--- a/include/uapi/asm-generic/unistd.h
+++ b/include/uapi/asm-generic/unistd.h
@@ -883,8 +883,15 @@ __SYSCALL(__NR_process_mrelease, sys_pro
 #define __NR_futex_waitv 449
 __SYSCALL(__NR_futex_waitv, sys_futex_waitv)
 
+#define __NR_umcg_ctl 450
+__SYSCALL(__NR_umcg_ctl, sys_umcg_ctl)
+#define __NR_umcg_wait 451
+__SYSCALL(__NR_umcg_wait, sys_umcg_wait)
+#define __NR_umcg_kick 452
+__SYSCALL(__NR_umcg_kick, sys_umcg_kick)
+
 #undef __NR_syscalls
-#define __NR_syscalls 450
+#define __NR_syscalls 453
 
 /*
  * 32 bit systems traditionally used different
--- /dev/null
+++ b/include/uapi/linux/umcg.h
@@ -0,0 +1,141 @@
+/* SPDX-License-Identifier: GPL-2.0+ WITH Linux-syscall-note */
+#ifndef _UAPI_LINUX_UMCG_H
+#define _UAPI_LINUX_UMCG_H
+
+#include <linux/types.h>
+
+/*
+ * UMCG: User Managed Concurrency Groups.
+ *
+ * Syscalls (see kernel/sched/umcg.c):
+ *      sys_umcg_ctl()  - register/unregister UMCG tasks;
+ *      sys_umcg_wait() - wait/wake/context-switch.
+ *      sys_umcg_kick() - prod a UMCG task
+ *
+ * struct umcg_task (below): controls the state of UMCG tasks.
+ */
+
+/*
+ * UMCG task states, the first 8 bits of struct umcg_task.state.
+ *
+ *   ,--------(TF_PREEMPT + notify_resume)-------.   ,----------.
+ *   |                                           v   |          |
+ * RUNNING -(schedule)-> BLOCKED -(sys_exit)-> RUNNABLE  (signal + notify_resume)
+ *   ^                      ^                    | | ^          |
+ *   |                      `-----(schedule)-----' | |          |
+ *   `--------------(sys_umcg_wait)----------------' `----------'
+ *
+ */
+#define UMCG_TASK_NONE			0x0000U
+#define UMCG_TASK_RUNNING		0x0001U
+#define UMCG_TASK_RUNNABLE		0x0002U
+#define UMCG_TASK_BLOCKED		0x0003U
+
+#define UMCG_TASK_MASK			0x00ffU
+
+/*
+ * UMCG_TF_PREEMPT: userspace indicates the worker should be preempted.
+ *
+ * Must only be set on UMCG_TASK_RUNNING; once set, any subsequent
+ * return-to-user (eg sys_umcg_kick()) will perform the equivalent of
+ * sys_umcg_wait() on it. That is, it will wake next_tid/server_tid, transfer
+ * to RUNNABLE and enqueue on the server's runnable list.
+ */
+#define UMCG_TF_PREEMPT			0x0100U
+/*
+ * UMCG_TF_COND_WAIT: indicate the task *will* call sys_umcg_wait()
+ *
+ * Enables server loops like (vs umcg_sys_exit()):
+ *
+ *   for(;;) {
+ *	self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
+ *
+ *	// smp_mb() implied by xchg()
+ *	runnable_ptr = (void *)xchg(self->runnable_workers_ptr, NULL);
+ *	while (runnable_ptr) {
+ *		next = (void *)runnable_ptr->runnable_workers_ptr;
+ *		umcg_server_add_runnable(self, runnable_ptr);
+ *		runnable_ptr = next;
+ *	}
+ *
+ *	self->next_tid = umcg_server_pick_next(self);
+ *	sys_umcg_wait(0, 0);
+ *   }
+ *
+ * without a signal or interrupt in between setting umcg_task::state and
+ * sys_umcg_wait() resulting in an infinite wait in umcg_notify_resume().
+ */
+#define UMCG_TF_COND_WAIT		0x0200U
+
+#define UMCG_TF_MASK			0xff00U
+
+#define UMCG_TASK_ALIGN			64
+
+/**
+ * struct umcg_task - controls the state of UMCG tasks.
+ *
+ * The struct is aligned at 64 bytes to ensure that it fits into
+ * a single cache line.
+ */
+struct umcg_task {
+	/**
+	 * @state: the current state of the UMCG task described by
+	 *         this struct.
+	 *
+	 * Readable/writable by both the kernel and the userspace.
+	 *
+	 * UMCG task state:
+	 *   bits  0 -  7: task state;
+	 *   bits  8 - 15: state flags;
+	 *   bits 16 - 31: for userspace use;
+	 */
+	__u32	state;				/* r/w */
+
+#define UMCG_TID_RUNNING	0x80000000U
+#define UMCG_TID_MASK		0x3fffffffU
+	/**
+	 * @next_tid: the TID of the UMCG task that should be context-switched
+	 *            into in sys_umcg_wait(). Can be zero.
+	 *
+	 * @server_tid: the TID of the UMCG server that hosts this task,
+	 *		when RUNNABLE this task will get added to it's
+	 *		runnable_workers_ptr list.
+	 *
+	 * Read-only for the kernel, read/write for the userspace.
+	 */
+	__u32	next_tid;			/* r   */
+	__u32	server_tid;			/* r   */
+
+	__u32	__hole[1];
+
+	/*
+	 * Timestamps for when last we became BLOCKED, RUNNABLE.
+	 */
+	__u64	blocked_ts;			/*   w */
+	__u64   runnable_ts;			/*   w */
+
+	/**
+	 * @runnable_workers_ptr: a single-linked list of runnable workers.
+	 *
+	 * Readable/writable by both the kernel and the userspace: the
+	 * kernel adds items to the list, userspace removes them.
+	 */
+	__u64	runnable_workers_ptr;		/* r/w */
+
+	__u64	__zero[3];
+
+} __attribute__((packed, aligned(UMCG_TASK_ALIGN)));
+
+/**
+ * enum umcg_ctl_flag - flags to pass to sys_umcg_ctl
+ * @UMCG_CTL_REGISTER:   register the current task as a UMCG task
+ * @UMCG_CTL_UNREGISTER: unregister the current task as a UMCG task
+ * @UMCG_CTL_WORKER:     register the current task as a UMCG worker
+ */
+enum umcg_ctl_flag {
+	UMCG_CTL_REGISTER	= 0x00001,
+	UMCG_CTL_UNREGISTER	= 0x00002,
+	UMCG_CTL_WORKER		= 0x10000,
+};
+
+#endif /* _UAPI_LINUX_UMCG_H */
--- a/init/Kconfig
+++ b/init/Kconfig
@@ -1686,6 +1686,21 @@ config MEMBARRIER
 
 	  If unsure, say Y.
 
+config HAVE_UMCG
+	bool
+
+config UMCG
+	bool "Enable User Managed Concurrency Groups API"
+	depends on 64BIT
+	depends on GENERIC_ENTRY
+	depends on HAVE_UMCG
+	default n
+	help
+	  Enable User Managed Concurrency Groups API, which form the basis
+	  for an in-process M:N userspace scheduling framework.
+	  At the moment this is an experimental/RFC feature that is not
+	  guaranteed to be backward-compatible.
+
 config KALLSYMS
 	bool "Load all symbols for debugging/ksymoops" if EXPERT
 	default y
--- a/kernel/entry/common.c
+++ b/kernel/entry/common.c
@@ -6,6 +6,7 @@
 #include <linux/livepatch.h>
 #include <linux/audit.h>
 #include <linux/tick.h>
+#include <linux/sched.h>
 
 #include "common.h"
 
@@ -76,6 +77,9 @@ static long syscall_trace_enter(struct p
 	if (unlikely(work & SYSCALL_WORK_SYSCALL_TRACEPOINT))
 		trace_sys_enter(regs, syscall);
 
+	if (work & SYSCALL_WORK_SYSCALL_UMCG)
+		umcg_sys_enter(regs, syscall);
+
 	syscall_enter_audit(regs, syscall);
 
 	return ret ? : syscall;
@@ -155,8 +159,7 @@ static unsigned long exit_to_user_mode_l
 	 * Before returning to user space ensure that all pending work
 	 * items have been completed.
 	 */
-	while (ti_work & EXIT_TO_USER_MODE_WORK) {
-
+	do {
 		local_irq_enable_exit_to_user(ti_work);
 
 		if (ti_work & _TIF_NEED_RESCHED)
@@ -168,6 +171,10 @@ static unsigned long exit_to_user_mode_l
 		if (ti_work & _TIF_PATCH_PENDING)
 			klp_update_patch_state(current);
 
+		/* must be before handle_signal_work(); terminates on sigpending */
+		if (ti_work & _TIF_UMCG)
+			umcg_notify_resume(regs);
+
 		if (ti_work & (_TIF_SIGPENDING | _TIF_NOTIFY_SIGNAL))
 			handle_signal_work(regs, ti_work);
 
@@ -188,7 +195,7 @@ static unsigned long exit_to_user_mode_l
 		tick_nohz_user_enter_prepare();
 
 		ti_work = read_thread_flags();
-	}
+	} while (ti_work & EXIT_TO_USER_MODE_WORK);
 
 	/* Return the latest work state for arch_exit_to_user_mode() */
 	return ti_work;
@@ -203,7 +210,7 @@ static void exit_to_user_mode_prepare(st
 	/* Flush pending rcuog wakeup before the last need_resched() check */
 	tick_nohz_user_enter_prepare();
 
-	if (unlikely(ti_work & EXIT_TO_USER_MODE_WORK))
+	if (unlikely(ti_work & (EXIT_TO_USER_MODE_WORK | _TIF_UMCG)))
 		ti_work = exit_to_user_mode_loop(regs, ti_work);
 
 	arch_exit_to_user_mode_prepare(regs, ti_work);
@@ -253,6 +260,9 @@ static void syscall_exit_work(struct pt_
 	step = report_single_step(work);
 	if (step || work & SYSCALL_WORK_SYSCALL_TRACE)
 		arch_syscall_exit_tracehook(regs, step);
+
+	if (work & SYSCALL_WORK_SYSCALL_UMCG)
+		umcg_sys_exit(regs);
 }
 
 /*
--- a/kernel/exit.c
+++ b/kernel/exit.c
@@ -749,6 +749,10 @@ void __noreturn do_exit(long code)
 	if (unlikely(!tsk->pid))
 		panic("Attempted to kill the idle task!");
 
+	/* Turn off UMCG sched hooks. */
+	if (unlikely(tsk->flags & PF_UMCG_WORKER))
+		tsk->flags &= ~PF_UMCG_WORKER;
+
 	/*
 	 * If do_exit is called because this processes oopsed, it's possible
 	 * that get_fs() was left as KERNEL_DS, so reset it to USER_DS before
@@ -786,6 +790,7 @@ void __noreturn do_exit(long code)
 
 	io_uring_files_cancel();
 	exit_signals(tsk);  /* sets PF_EXITING */
+	umcg_handle_exit();
 
 	/* sync mm's RSS info before statistics gathering */
 	if (tsk->mm)
--- a/kernel/sched/Makefile
+++ b/kernel/sched/Makefile
@@ -41,3 +41,4 @@ obj-$(CONFIG_MEMBARRIER) += membarrier.o
 obj-$(CONFIG_CPU_ISOLATION) += isolation.o
 obj-$(CONFIG_PSI) += psi.o
 obj-$(CONFIG_SCHED_CORE) += core_sched.o
+obj-$(CONFIG_UMCG) += umcg.o
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -4272,6 +4272,7 @@ static void __sched_fork(unsigned long c
 	p->wake_entry.u_flags = CSD_TYPE_TTWU;
 	p->migration_pending = NULL;
 #endif
+	umcg_clear_child(p);
 }
 
 DEFINE_STATIC_KEY_FALSE(sched_numa_balancing);
@@ -6330,9 +6331,11 @@ static inline void sched_submit_work(str
 	 * If a worker goes to sleep, notify and ask workqueue whether it
 	 * wants to wake up a task to maintain concurrency.
 	 */
-	if (task_flags & (PF_WQ_WORKER | PF_IO_WORKER)) {
+	if (task_flags & (PF_WQ_WORKER | PF_IO_WORKER | PF_UMCG_WORKER)) {
 		if (task_flags & PF_WQ_WORKER)
 			wq_worker_sleeping(tsk);
+		else if (task_flags & PF_UMCG_WORKER)
+			umcg_wq_worker_sleeping(tsk);
 		else
 			io_wq_worker_sleeping(tsk);
 	}
@@ -6350,9 +6353,11 @@ static inline void sched_submit_work(str
 
 static void sched_update_worker(struct task_struct *tsk)
 {
-	if (tsk->flags & (PF_WQ_WORKER | PF_IO_WORKER)) {
+	if (tsk->flags & (PF_WQ_WORKER | PF_IO_WORKER | PF_UMCG_WORKER)) {
 		if (tsk->flags & PF_WQ_WORKER)
 			wq_worker_running(tsk);
+		else if (tsk->flags & PF_UMCG_WORKER)
+			umcg_wq_worker_running(tsk);
 		else
 			io_wq_worker_running(tsk);
 	}
--- /dev/null
+++ b/kernel/sched/umcg.c
@@ -0,0 +1,954 @@
+// SPDX-License-Identifier: GPL-2.0-only
+
+/*
+ * User Managed Concurrency Groups (UMCG).
+ *
+ */
+
+#include <linux/syscalls.h>
+#include <linux/types.h>
+#include <linux/uaccess.h>
+#include <linux/umcg.h>
+#include <linux/mm.h>
+
+#include <asm/syscall.h>
+#include <asm/ptrace.h>
+
+#include "sched.h"
+
+static struct task_struct *umcg_get_task(u32 tid)
+{
+	struct task_struct *tsk = NULL;
+
+	if (tid) {
+		rcu_read_lock();
+		tsk = find_task_by_vpid(tid & UMCG_TID_MASK);
+		if (tsk && current->mm == tsk->mm && tsk->umcg_task)
+			get_task_struct(tsk);
+		else
+			tsk = NULL;
+		rcu_read_unlock();
+	}
+
+	return tsk;
+}
+
+/*
+ * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
+ * through the user mapping ensures the user mapping exists.
+ */
+#define umcg_pin_and_load(_self, _pagep, _member)				\
+({										\
+	__label__ __out;							\
+	int __ret = -EFAULT;							\
+										\
+	if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)	\
+		goto __out;							\
+										\
+	if (!PageAnon(_pagep) ||						\
+	    get_user(_member, &(_self)->_member)) {				\
+		unpin_user_page(_pagep);					\
+		goto __out;							\
+	}									\
+	__ret = 0;								\
+__out:	__ret;									\
+})
+
+/**
+ * umcg_pin_pages: pin pages containing struct umcg_task of
+ *		   this task and its server (possibly this task again).
+ */
+static int umcg_pin_pages(void)
+{
+	struct task_struct *server = NULL, *tsk = current;
+	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
+	int server_tid;
+	int ret;
+
+	/* must not have stale state */
+	if (WARN_ON_ONCE(tsk->umcg_page ||
+			 tsk->umcg_server_page ||
+			 tsk->umcg_server_task ||
+			 tsk->umcg_server))
+		return -EBUSY;
+
+	ret = umcg_pin_and_load(self, tsk->umcg_page, server_tid);
+	if (ret)
+		goto clear_self;
+
+	ret = -ESRCH;
+	server = umcg_get_task(server_tid);
+	if (!server)
+		goto unpin_self;
+
+	/* must cache due to possible concurrent change */
+	tsk->umcg_server_task = READ_ONCE(server->umcg_task);
+	ret = umcg_pin_and_load(tsk->umcg_server_task, tsk->umcg_server_page, server_tid);
+	if (ret)
+		goto clear_server;
+
+	tsk->umcg_server = server;
+
+	return 0;
+
+clear_server:
+	tsk->umcg_server_task = NULL;
+	tsk->umcg_server_page = NULL;
+
+unpin_self:
+	unpin_user_page(tsk->umcg_page);
+clear_self:
+	tsk->umcg_page = NULL;
+
+	return ret;
+}
+
+static void umcg_unpin_pages(void)
+{
+	struct task_struct *tsk = current;
+
+	if (tsk->umcg_server) {
+		unpin_user_page(tsk->umcg_page);
+		tsk->umcg_page = NULL;
+
+		unpin_user_page(tsk->umcg_server_page);
+		tsk->umcg_server_page = NULL;
+		tsk->umcg_server_task = NULL;
+
+		put_task_struct(tsk->umcg_server);
+		tsk->umcg_server = NULL;
+	}
+}
+
+static void umcg_clear_task(struct task_struct *tsk)
+{
+	/*
+	 * This is either called for the current task, or for a newly forked
+	 * task that is not yet running, so we don't need strict atomicity
+	 * below.
+	 */
+	if (tsk->umcg_task) {
+		WRITE_ONCE(tsk->umcg_task, NULL);
+		tsk->umcg_page = NULL;
+
+		tsk->umcg_server = NULL;
+		tsk->umcg_server_page = NULL;
+		tsk->umcg_server_task = NULL;
+
+		tsk->flags &= ~PF_UMCG_WORKER;
+		clear_task_syscall_work(tsk, SYSCALL_UMCG);
+		clear_tsk_thread_flag(tsk, TIF_UMCG);
+	}
+}
+
+/* Called for a forked or execve-ed child. */
+void umcg_clear_child(struct task_struct *tsk)
+{
+	umcg_clear_task(tsk);
+}
+
+/* Called both by normally (unregister) and abnormally exiting workers. */
+void umcg_worker_exit(void)
+{
+	umcg_unpin_pages();
+	umcg_clear_task(current);
+}
+
+/*
+ * Do a state transition: @from -> @to.
+ *
+ * Will clear UMCG_TF_PREEMPT, UMCG_TF_COND_WAIT.
+ *
+ * When @to == {BLOCKED,RUNNABLE}, update timestamps.
+ *
+ * Returns:
+ *   0: success
+ *   -EAGAIN: when self->state != @from
+ *   -EFAULT
+ */
+static int umcg_update_state(struct task_struct *tsk,
+			     struct umcg_task __user *self,
+			     u32 from, u32 to)
+{
+	u32 old, new;
+	u64 now;
+
+	if (to >= UMCG_TASK_RUNNABLE) {
+		switch (tsk->umcg_clock) {
+		case CLOCK_REALTIME:      now = ktime_get_real_ns();     break;
+		case CLOCK_MONOTONIC:     now = ktime_get_ns();          break;
+		case CLOCK_BOOTTIME:      now = ktime_get_boottime_ns(); break;
+		case CLOCK_TAI:           now = ktime_get_clocktai_ns(); break;
+		}
+	}
+
+	if (!user_access_begin(self, sizeof(*self)))
+		return -EFAULT;
+
+	unsafe_get_user(old, &self->state, Efault);
+	do {
+		if ((old & UMCG_TASK_MASK) != from)
+			goto fail;
+
+		new = old & ~(UMCG_TASK_MASK |
+			      UMCG_TF_PREEMPT | UMCG_TF_COND_WAIT);
+		new |= to & UMCG_TASK_MASK;
+
+	} while (!unsafe_try_cmpxchg_user(&self->state, &old, new, Efault));
+
+	if (to == UMCG_TASK_BLOCKED)
+		unsafe_put_user(now, &self->blocked_ts, Efault);
+	if (to == UMCG_TASK_RUNNABLE)
+		unsafe_put_user(now, &self->runnable_ts, Efault);
+
+	user_access_end();
+	return 0;
+
+fail:
+	user_access_end();
+	return -EAGAIN;
+
+Efault:
+	user_access_end();
+	return -EFAULT;
+}
+
+#define __UMCG_DIE(stmt, reason)	do {				\
+	stmt;								\
+	pr_warn_ratelimited("%s: killing task %s/%d because: " reason "\n",\
+			    __func__, current->comm, current->pid);	\
+	force_sig(SIGKILL);						\
+	return;								\
+} while (0)
+
+#define UMCG_DIE(reason)	__UMCG_DIE(,reason)
+#define UMCG_DIE_PF(reason)	__UMCG_DIE(pagefault_enable(), reason)
+#define UMCG_DIE_UNPIN(reason)	__UMCG_DIE(umcg_unpin_pages(), reason)
+
+/* Called from syscall enter path and exceptions that can schedule */
+void umcg_sys_enter(struct pt_regs *regs, long syscall)
+{
+	/* avoid recursion vs our own syscalls */
+	if (syscall == __NR_umcg_wait ||
+	    syscall == __NR_umcg_ctl)
+		return;
+
+	/* avoid recursion vs schedule() */
+	current->flags &= ~PF_UMCG_WORKER;
+
+	/*
+	 * Pin all the state on sys_enter() such that we can rely on it
+	 * from dodgy contexts. It is either unpinned from pre-schedule()
+	 * or sys_exit(), whichever comes first, thereby ensuring the pin
+	 * is temporary.
+	 */
+	if (umcg_pin_pages())
+		UMCG_DIE("pin");
+
+	current->flags |= PF_UMCG_WORKER;
+}
+
+static int umcg_wake_task(struct task_struct *tsk, struct umcg_task __user *self)
+{
+	int ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNING);
+	if (ret)
+		return ret;
+
+	try_to_wake_up(tsk, TASK_NORMAL, WF_CURRENT_CPU);
+	return 0;
+}
+
+static int umcg_wake_server(struct task_struct *tsk)
+{
+	int ret = umcg_wake_task(tsk->umcg_server, tsk->umcg_server_task);
+	if (ret == -EAGAIN) {
+		/*
+		 * Server could have timed-out or already be running
+		 * due to a runnable enqueue. See umcg_sys_exit().
+		 */
+		ret = 0;
+	}
+	return ret;
+}
+
+/* pre-schedule() */
+void umcg_wq_worker_sleeping(struct task_struct *tsk)
+{
+	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
+	int ret;
+
+	if (!tsk->umcg_server) {
+		/*
+		 * Already blocked before, the pages are unpinned.
+		 */
+		return;
+	}
+
+	/* Must not fault, mmap_sem might be held. */
+	pagefault_disable();
+
+	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNING, UMCG_TASK_BLOCKED);
+	if (ret == -EAGAIN) {
+		/*
+		 * Consider:
+		 *
+		 *   self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
+		 *   ...
+		 *   sys_umcg_wait();
+		 *
+		 * and the '...' code doing a blocking syscall/fault. This
+		 * ensures that returns with UMCG_TASK_RUNNING, which will make
+		 * sys_umcg_wait() return with -EAGAIN.
+		 */
+		ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_BLOCKED);
+	}
+	if (ret)
+		UMCG_DIE_PF("state");
+
+	if (umcg_wake_server(tsk))
+		UMCG_DIE_PF("wake");
+
+	pagefault_enable();
+
+	/*
+	 * We're going to sleep, make sure to unpin the pages, this ensures
+	 * the pins are temporary. Also see umcg_sys_exit().
+	 */
+	umcg_unpin_pages();
+}
+
+/* post-schedule() */
+void umcg_wq_worker_running(struct task_struct *tsk)
+{
+	/* nothing here, see umcg_sys_exit() */
+}
+
+/*
+ * Enqueue @tsk on it's server's runnable list
+ *
+ * Must be called in umcg_pin_pages() context, relies on tsk->umcg_server.
+ *
+ * cmpxchg based single linked list add such that list integrity is never
+ * violated.  Userspace *MUST* remove it from the list before changing ->state.
+ * As such, we must change state to RUNNABLE before enqueue.
+ *
+ * Returns:
+ *   0: success
+ *   -EFAULT
+ */
+static int umcg_enqueue_runnable(struct task_struct *tsk)
+{
+	struct umcg_task __user *server = tsk->umcg_server_task;
+	struct umcg_task __user *self = tsk->umcg_task;
+	u64 first_ptr, *head = &server->runnable_workers_ptr;
+	u64 self_ptr = (unsigned long)self;
+
+	/*
+	 * umcg_pin_pages() did access_ok() on both pointers, use self here
+	 * only because __user_access_begin() isn't available in generic code.
+	 */
+	if (!user_access_begin(self, sizeof(*self)))
+		return -EFAULT;
+
+	unsafe_get_user(first_ptr, head, Efault);
+	do {
+		unsafe_put_user(first_ptr, &self->runnable_workers_ptr, Efault);
+	} while (!unsafe_try_cmpxchg_user(head, &first_ptr, self_ptr, Efault));
+
+	user_access_end();
+	return 0;
+
+Efault:
+	user_access_end();
+	return -EFAULT;
+}
+
+static int umcg_enqueue_and_wake(struct task_struct *tsk)
+{
+	int ret;
+
+	ret = umcg_enqueue_runnable(tsk);
+	if (!ret)
+		ret = umcg_wake_server(tsk);
+
+	return ret;
+}
+
+/*
+ * umcg_wait: Wait for ->state to become RUNNING
+ *
+ * Returns:
+ * 0		- success
+ * -EINTR	- pending signal
+ * -EINVAL	- ::state is not {RUNNABLE,RUNNING}
+ * -ETIMEDOUT
+ * -EFAULT
+ */
+static int umcg_wait(u64 timo)
+{
+	struct task_struct *tsk = current;
+	struct umcg_task __user *self = tsk->umcg_task;
+	struct page *page = NULL;
+	u32 state;
+	int ret;
+
+	for (;;) {
+		set_current_state(TASK_INTERRUPTIBLE);
+
+		ret = -EINTR;
+		if (signal_pending(current))
+			break;
+
+		/*
+		 * Faults can block and scribble our wait state.
+		 */
+		pagefault_disable();
+		if (get_user(state, &self->state)) {
+			pagefault_enable();
+			__set_current_state(TASK_RUNNING);
+
+			ret = -EFAULT;
+			if (page) {
+				unpin_user_page(page);
+				page = NULL;
+				break;
+			}
+
+			ret = umcg_pin_and_load(self, page, state);
+			if (ret) {
+				page = NULL;
+				break;
+			}
+
+			continue;
+		}
+
+		if (page) {
+			unpin_user_page(page);
+			page = NULL;
+		}
+		pagefault_enable();
+
+		state &= UMCG_TASK_MASK;
+		if (state != UMCG_TASK_RUNNABLE) {
+			ret = 0;
+			if (state == UMCG_TASK_RUNNING)
+				break;
+
+			ret = -EINVAL;
+			break;
+		}
+
+		if (!schedule_hrtimeout_range_clock(timo ? &timo : NULL,
+						    tsk->timer_slack_ns,
+						    HRTIMER_MODE_ABS,
+						    tsk->umcg_clock)) {
+			ret = -ETIMEDOUT;
+			break;
+		}
+	}
+	__set_current_state(TASK_RUNNING);
+
+	return ret;
+}
+
+/*
+ * Blocked case for umcg_sys_exit(), shared with sys_umcg_ctl().
+ */
+static void umcg_unblock_and_wait(void)
+{
+	struct task_struct *tsk = current;
+	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
+
+	/* avoid recursion vs schedule() */
+	tsk->flags &= ~PF_UMCG_WORKER;
+
+	if (umcg_pin_pages())
+		UMCG_DIE("pin");
+
+	if (umcg_update_state(tsk, self, UMCG_TASK_BLOCKED, UMCG_TASK_RUNNABLE))
+		UMCG_DIE_UNPIN("state");
+
+	if (umcg_enqueue_and_wake(tsk))
+		UMCG_DIE_UNPIN("enqueue-wake");
+
+	umcg_unpin_pages();
+
+	switch (umcg_wait(0)) {
+	case 0:
+	case -EINTR:
+		/* notify_resume will continue the wait after the signal */
+		break;
+
+	default:
+		UMCG_DIE("wait");
+	}
+
+	tsk->flags |= PF_UMCG_WORKER;
+}
+
+/* Called from syscall exit path and exceptions that can schedule */
+void umcg_sys_exit(struct pt_regs *regs)
+{
+	struct task_struct *tsk = current;
+	long syscall = syscall_get_nr(tsk, regs);
+
+	if (syscall == __NR_umcg_wait ||
+	    syscall == __NR_umcg_ctl)
+		return;
+
+	if (tsk->umcg_server) {
+		/*
+		 * Didn't block, we done.
+		 */
+		umcg_unpin_pages();
+		return;
+	}
+
+	umcg_unblock_and_wait();
+}
+
+/* return-to-user path */
+void umcg_notify_resume(struct pt_regs *regs)
+{
+	struct task_struct *tsk = current;
+	struct umcg_task __user *self = tsk->umcg_task;
+	bool worker = tsk->flags & PF_UMCG_WORKER;
+	u32 state;
+
+	/* avoid recursion vs schedule() */
+	if (worker)
+		current->flags &= ~PF_UMCG_WORKER;
+
+	if (get_user(state, &self->state))
+		UMCG_DIE("get-state");
+
+	state &= UMCG_TASK_MASK | UMCG_TF_MASK;
+	if (state == UMCG_TASK_RUNNING)
+		goto done;
+
+	/*
+	 * See comment at UMCG_TF_COND_WAIT; TL;DR: user *will* call
+	 * sys_umcg_wait() and signals/interrupts shouldn't block
+	 * return-to-user.
+	 */
+	if (state == (UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT))
+		goto done;
+
+	if (state & UMCG_TF_PREEMPT) {
+		if (umcg_pin_pages())
+			UMCG_DIE("pin");
+
+		if (umcg_update_state(tsk, self,
+				      UMCG_TASK_RUNNING,
+				      UMCG_TASK_RUNNABLE))
+			UMCG_DIE_UNPIN("state");
+
+		if (umcg_enqueue_and_wake(tsk))
+			UMCG_DIE_UNPIN("enqueue-wake");
+
+		umcg_unpin_pages();
+	}
+
+	switch (umcg_wait(0)) {
+	case 0:
+	case -EINTR:
+		/* we will resume the wait after the signal */
+		break;
+
+	default:
+		UMCG_DIE("wait");
+	}
+
+done:
+	if (worker)
+		current->flags |= PF_UMCG_WORKER;
+}
+
+/**
+ * sys_umcg_kick: makes a UMCG task cycle through umcg_notify_resume()
+ *
+ * Returns:
+ * 0		- Ok;
+ * -ESRCH	- not a related UMCG task
+ * -EINVAL	- another error happened (unknown flags, etc..)
+ */
+SYSCALL_DEFINE2(umcg_kick, u32, flags, pid_t, tid)
+{
+	struct task_struct *task = umcg_get_task(tid);
+	if (!task)
+		return -ESRCH;
+
+	if (flags)
+		return -EINVAL;
+
+#ifdef CONFIG_SMP
+	smp_send_reschedule(task_cpu(task));
+#endif
+
+	return 0;
+}
+
+/*
+ * Handles ::next_tid as per sys_umcg_wait().
+ *
+ * ::next_tid		- return
+ * -----------------------------
+ * 0			- 0 (success)
+ *
+ * tid			- -ESRCH (no such task, or not of this UMCG)
+ *			- -EAGAIN (next::state != RUNNABLE)
+ *			- 0 (success, ::next_tid |= RUNNING)
+ *
+ * tid|RUNNING		- -EAGAIN (next::state != RUNNING)
+ *			- 0 (success)
+ *
+ * Returns:
+ *  0: success
+ *  -EFAULT
+ *  -ESRCH
+ *  -EAGAIN
+ */
+static int umcg_wake_next(struct task_struct *tsk, struct umcg_task __user *self)
+{
+	struct umcg_task __user *next_task;
+	struct task_struct *next;
+	u32 next_tid, state;
+	int ret;
+
+	if (get_user(next_tid, &self->next_tid))
+		return -EFAULT;
+
+	if (!next_tid)
+		return 0;
+
+	next = umcg_get_task(next_tid);
+	if (!next)
+		return -ESRCH;
+
+	next_task = READ_ONCE(next->umcg_task);
+
+	if (next_tid & UMCG_TID_RUNNING) {
+		ret = -EFAULT;
+		if (get_user(state, &next_task->state))
+			goto put_next;
+
+		ret = 0;
+		if ((state & UMCG_TASK_MASK) != UMCG_TASK_RUNNING)
+			ret = -EAGAIN;
+
+	} else {
+		ret = umcg_wake_task(next, next_task);
+		if (ret)
+			goto put_next;
+
+		ret = -EFAULT;
+		if (put_user(next_tid | UMCG_TID_RUNNING, &self->next_tid))
+			goto put_next;
+
+		/*
+		 * If this is a worker doing sys_umcg_wait() switching to
+		 * another worker, userspace has the responsibility to update
+		 * server::next_tid.
+		 */
+
+		ret = 0;
+	}
+
+put_next:
+	put_task_struct(next);
+	return ret;
+}
+
+/**
+ * sys_umcg_wait: transfer running context
+ *
+ * Called like:
+ *
+ *	self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
+ *	...
+ *	sys_umcg_wait(0, time);
+ *
+ * The syscall will clear TF_COND_WAIT and wait until state becomes RUNNING.
+ * The code '...' must not contain syscalls
+ *
+ * If self->next_tid is set and indicates a valid UMCG task with RUNNABLE state
+ * that task will be made RUNNING and woken -- transfering the running context
+ * to that task. In this case self->next_tid is modified with TID_RUNNING to
+ * indicate self->next_tid is consumed.
+ *
+ * If self->next has TID_RUNNING set, it is validated the related task has
+ * RUNNING state, otherwise -EAGAIN is returned to indicate a new task needs to
+ * be selected.
+ *
+ * If the caller is a worker:
+ *
+ *  - it will be enqueued on the associated server's runnable_workers_ptr list
+ *    and the server will be woken.
+ *
+ *  - when ::next_tid is used to affect a worker-to-worker transfer, it is up
+ *    to userspace to keep server::next_tid consistent.
+ *
+ * The corrolary is that a server setting ::next_tid to 0 will idle.
+ *
+ * Returns:
+ * 0		- OK;
+ * -ETIMEDOUT	- the timeout expired;
+ * -ERANGE	- the timeout is out of range (worker);
+ * -EAGAIN	- ::state wasn't RUNNABLE, concurrent wakeup;
+ * -EFAULT	- failed accessing struct umcg_task __user of the current
+ *		  task, the server or next;
+ * -ESRCH	- the task to wake not found or not a UMCG task;
+ * -EINVAL	- another error happened (e.g. the current task is not a
+ *		  UMCG task, etc.)
+ */
+SYSCALL_DEFINE2(umcg_wait, u32, flags, u64, timo)
+{
+	struct task_struct *tsk = current;
+	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
+	bool worker = tsk->flags & PF_UMCG_WORKER;
+	int ret;
+
+	if (!self || flags)
+		return -EINVAL;
+
+	if (worker) {
+		tsk->flags &= ~PF_UMCG_WORKER;
+		if (timo)
+			return -ERANGE;
+	}
+
+	/* see umcg_sys_{enter,exit}() syscall exceptions */
+	ret = umcg_pin_pages();
+	if (ret)
+		goto unblock;
+
+	/*
+	 * Clear UMCG_TF_COND_WAIT *and* check state == RUNNABLE.
+	 */
+	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNABLE);
+	if (ret)
+		goto unpin;
+
+	ret = umcg_wake_next(tsk, self);
+	if (ret)
+		goto unpin;
+
+	if (worker) {
+		/*
+		 * If this fails it is possible ::next_tid is already running
+		 * while this task is not going to block. This violates our
+		 * constraints.
+		 *
+		 * That said, pretty much the only way to make this fail is by
+		 * force munmap()'ing things. In which case one is most welcome
+		 * to the pieces.
+		 */
+		ret = umcg_enqueue_and_wake(tsk);
+		if (ret)
+			goto unpin;
+	}
+
+	umcg_unpin_pages();
+
+	ret = umcg_wait(timo);
+	switch (ret) {
+	case 0:		/* all done */
+	case -EINTR:	/* umcg_notify_resume() will continue the wait */
+		ret = 0;
+		break;
+
+	default:
+		goto unblock;
+	}
+out:
+	if (worker)
+		tsk->flags |= PF_UMCG_WORKER;
+	return ret;
+
+unpin:
+	umcg_unpin_pages();
+unblock:
+	umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNING);
+	goto out;
+}
+
+static int umcg_register(struct umcg_task __user *self, u32 flags, clockid_t which_clock)
+{
+	struct task_struct *server;
+	struct umcg_task ut;
+
+	switch (which_clock) {
+	case CLOCK_REALTIME:
+	case CLOCK_MONOTONIC:
+	case CLOCK_BOOTTIME:
+	case CLOCK_TAI:
+		current->umcg_clock = which_clock;
+		break;
+
+	default:
+		return -EINVAL;
+	}
+
+	if (current->umcg_task || !self)
+		return -EINVAL;
+
+	if (copy_from_user(&ut, self, sizeof(ut)))
+		return -EFAULT;
+
+	if (ut.next_tid || ut.__hole[0] || ut.__zero[0] || ut.__zero[1] || ut.__zero[2])
+		return -EINVAL;
+
+	rcu_read_lock();
+	server = find_task_by_vpid(ut.server_tid);
+	if (server && server->mm == current->mm) {
+		if (flags == UMCG_CTL_WORKER) {
+			if (!server->umcg_task ||
+			    (server->flags & PF_UMCG_WORKER))
+				server = NULL;
+		} else {
+			if (server != current)
+				server = NULL;
+		}
+	} else {
+		server = NULL;
+	}
+	rcu_read_unlock();
+
+	if (!server)
+		return -ESRCH;
+
+	if (flags == UMCG_CTL_WORKER) {
+		if ((ut.state & (UMCG_TASK_MASK | UMCG_TF_MASK)) != UMCG_TASK_BLOCKED)
+			return -EINVAL;
+
+		WRITE_ONCE(current->umcg_task, self);
+		current->flags |= PF_UMCG_WORKER;	/* hook schedule() */
+		set_syscall_work(SYSCALL_UMCG);		/* hook syscall */
+		set_thread_flag(TIF_UMCG);		/* hook return-to-user */
+
+		umcg_unblock_and_wait();
+
+	} else {
+		if ((ut.state & (UMCG_TASK_MASK | UMCG_TF_MASK)) != UMCG_TASK_RUNNING)
+			return -EINVAL;
+
+		WRITE_ONCE(current->umcg_task, self);
+		set_thread_flag(TIF_UMCG);		/* hook return-to-user */
+
+		/* umcg_notify_resume() would block if not RUNNING */
+	}
+
+	return 0;
+}
+
+static int umcg_unregister(struct umcg_task __user *self, u32 flags)
+{
+	bool worker = current->flags & PF_UMCG_WORKER;
+	int ret;
+
+	if (!self || self != current->umcg_task)
+		return -EINVAL;
+
+	if (!worker != !(flags & UMCG_CTL_WORKER))
+		return -EINVAL;
+
+	current->flags &= ~PF_UMCG_WORKER;
+
+	ret = umcg_pin_pages();
+	if (ret) {
+		if (worker)
+			current->flags |= PF_UMCG_WORKER;
+		return ret;
+	}
+
+	ret = umcg_update_state(current, self, UMCG_TASK_RUNNING, UMCG_TASK_NONE);
+	if (ret) {
+		if (worker)
+			current->flags |= PF_UMCG_WORKER;
+		return ret;
+	}
+
+	if (worker)
+		umcg_wake_server(current);
+
+	umcg_unpin_pages();
+	umcg_clear_task(current);
+	return 0;
+}
+
+#define UMCG_CTL_CMD	0xff
+
+/**
+ * sys_umcg_ctl: (un)register the current task as a UMCG task.
+ * @flags:       ORed values from enum umcg_ctl_flag; see below;
+ * @self:        a pointer to struct umcg_task that describes this
+ *               task and governs the behavior of sys_umcg_wait.
+ * @which_clock: clockid to use for timestamps and timeouts
+ *
+ * @flags & UMCG_CTL_REGISTER: register a UMCG task:
+ *
+ *	UMCG workers:
+ *	 - @flags & UMCG_CTL_WORKER
+ *	 - self->state must be UMCG_TASK_BLOCKED
+ *
+ *	UMCG servers:
+ *	 - !(@flags & UMCG_CTL_WORKER)
+ *	 - self->state must be UMCG_TASK_RUNNING
+ *
+ *	All tasks:
+ *	 - self->server_tid must be a valid server
+ *	 - self->next_tid must be zero
+ *
+ *	If the conditions above are met, sys_umcg_ctl() immediately returns
+ *	if the registered task is a server. If the registered task is a
+ *	worker it will be added to it's server's runnable_workers_ptr list
+ *	and the server will be woken.
+ *
+ * @flags & UMCG_CTL_UNREGISTER: unregister a UMCG task.
+ *
+ *	UMCG workers:
+ *	 - @flags & UMCG_CTL_WORKER
+ *
+ *	UMCG servers:
+ *	 - !(@flags & UMCG_CTL_WORKER)
+ *
+ *	All tasks:
+ *	 - self must match with UMCG_CTL_REGISTER
+ *	 - self->state must be UMCG_TASK_RUNNING
+ *	 - self->server_tid must be a valid server
+ *
+ *	If the conditions above are met, sys_umcg_ctl() will change state to
+ *	UMCG_TASK_NONE, and for workers, wake either next or server.
+ *
+ * Return:
+ * 0		- success
+ * -EFAULT	- failed to read @self
+ * -EINVAL	- some other error occurred
+ * -ESRCH	- no such server_tid
+ */
+SYSCALL_DEFINE3(umcg_ctl, u32, flags, struct umcg_task __user *, self, clockid_t, which_clock)
+{
+	int cmd = flags & UMCG_CTL_CMD;
+
+	if ((unsigned long)self % UMCG_TASK_ALIGN)
+		return -EINVAL;
+
+	flags &= ~UMCG_CTL_CMD;
+
+	if (flags & ~(UMCG_CTL_WORKER))
+		return -EINVAL;
+
+	switch (cmd) {
+	case UMCG_CTL_REGISTER:
+		return umcg_register(self, flags, which_clock);
+
+	case UMCG_CTL_UNREGISTER:
+		return umcg_unregister(self, flags);
+
+	default:
+		break;
+	}
+
+	return -EINVAL;
+}
--- a/kernel/sys_ni.c
+++ b/kernel/sys_ni.c
@@ -273,6 +273,11 @@ COND_SYSCALL(landlock_create_ruleset);
 COND_SYSCALL(landlock_add_rule);
 COND_SYSCALL(landlock_restrict_self);
 
+/* kernel/sched/umcg.c */
+COND_SYSCALL(umcg_ctl);
+COND_SYSCALL(umcg_wait);
+COND_SYSCALL(umcg_kick);
+
 /* arch/example/kernel/sys_example.c */
 
 /* mm/fadvise.c */




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

* Re: [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups
  2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
                   ` (4 preceding siblings ...)
  2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
@ 2022-01-20 17:28 ` Peter Oskolkov
  2022-01-21  8:01   ` Peter Zijlstra
  2022-01-21 18:01 ` Steven Rostedt
  6 siblings, 1 reply; 47+ messages in thread
From: Peter Oskolkov @ 2022-01-20 17:28 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, avagin, jannh, tdelisle, mark.rutland, posk

On Thu, Jan 20, 2022 at 8:09 AM Peter Zijlstra <peterz@infradead.org> wrote:
>
> Latest version, many changes since last time, still under heavy discussion.

Thanks! I'll work on testing/integrating this patchset. I'll also
assume that my changes that added blocked worker list
(https://lore.kernel.org/lkml/20220113233940.3608440-5-posk@google.com/)
and server kicking
(https://lore.kernel.org/lkml/20220113233940.3608440-6-posk@google.com/)
are acceptable.

[...]


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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-20 15:55 ` [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages Peter Zijlstra
@ 2022-01-20 18:03   ` Nadav Amit
  2022-01-21  7:59     ` Peter Zijlstra
  2022-01-20 18:25   ` David Hildenbrand
  1 sibling, 1 reply; 47+ messages in thread
From: Nadav Amit @ 2022-01-20 18:03 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: Ingo Molnar, Thomas Gleixner, juri.lelli, vincent.guittot,
	dietmar.eggemann, Steven Rostedt, bsegall, Mel Gorman, bristot,
	Linux Kernel Mailing List, Linux-MM, linux-api, X86 ML, pjt,
	posk, avagin, Jann Horn, tdelisle, mark.rutland, posk



> On Jan 20, 2022, at 7:55 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> Add a guarantee for Anon pages that pin_user_page*() ensures the
> user-mapping of these pages stay preserved. In order to ensure this
> all rmap users have been audited:
> 
> vmscan:	already fails eviction due to page_maybe_dma_pinned()
> 
> migrate:	migration will fail on pinned pages due to
> 		expected_page_refs() not matching, however that is
> 		*after* try_to_migrate() has already destroyed the
> 		user mapping of these pages. Add an early exit for
> 		this case.
> 
> numa-balance:	as per the above, pinned pages cannot be migrated,
> 		however numa balancing scanning will happily PROT_NONE
> 		them to get usage information on these pages. Avoid
> 		this for pinned pages.
> 
> None of the other rmap users (damon,page-idle,mlock,..) unmap the
> page, they mostly just muck about with reference,dirty flags etc.
> 
> This same guarantee cannot be provided for Shared (file) pages due to
> dirty page tracking.
> 
> 

[ snip ]

> --- a/mm/mprotect.c
> +++ b/mm/mprotect.c
> @@ -106,6 +106,12 @@ static unsigned long change_pte_range(st
> 					continue;
> 
> 				/*
> +				 * Can't migrate pinned pages, avoid touching them.
> +				 */
> +				if (page_maybe_dma_pinned(page))
> +					continue;
> +
> +				/*
> 

I have a similar problem with userfaultfd changing protection for
DMA-pinned pages. For userfaultfd it is important to know how many
pages were actually modified.

I am working on a vectored UFFDIO_WRITEPROTECTV that aborts once
a pinned page is encountered, but also returns the number of pages
that were properly protected. I still need to do some work to
send patches for that as it requires further changes (to return
the number of pages that were handled).

But for the matter of your patch, is it possible to make this
test generic (not migration specific) and rely on a new flag in
cp_flags? I can of course make this change later if you prefer it
this way.



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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-20 15:55 ` [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages Peter Zijlstra
  2022-01-20 18:03   ` Nadav Amit
@ 2022-01-20 18:25   ` David Hildenbrand
  2022-01-21  7:51     ` Peter Zijlstra
  1 sibling, 1 reply; 47+ messages in thread
From: David Hildenbrand @ 2022-01-20 18:25 UTC (permalink / raw)
  To: Peter Zijlstra, mingo, tglx, juri.lelli, vincent.guittot,
	dietmar.eggemann, rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk

On 20.01.22 16:55, Peter Zijlstra wrote:
> Add a guarantee for Anon pages that pin_user_page*() ensures the
> user-mapping of these pages stay preserved. In order to ensure this
> all rmap users have been audited:
> 
>  vmscan:	already fails eviction due to page_maybe_dma_pinned()
> 
>  migrate:	migration will fail on pinned pages due to
> 		expected_page_refs() not matching, however that is
> 		*after* try_to_migrate() has already destroyed the
> 		user mapping of these pages. Add an early exit for
> 		this case.
> 
>  numa-balance:	as per the above, pinned pages cannot be migrated,
> 		however numa balancing scanning will happily PROT_NONE
> 		them to get usage information on these pages. Avoid
> 		this for pinned pages.

page_maybe_dma_pinned() can race with GUP-fast without
mm->write_protect_seq. This is a real problem for vmscan() with
concurrent GUP-fast as it can result in R/O mappings of pinned pages and
GUP will lose synchronicity to the page table on write faults due to
wrong COW.

If you're just using it as an optimization, that should work just fine.
I assume all migration will freeze the refcount and consequently  bail
out at that point. In that case, LGTM.


-- 
Thanks,

David / dhildenb



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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-20 18:25   ` David Hildenbrand
@ 2022-01-21  7:51     ` Peter Zijlstra
  2022-01-21  8:22       ` David Hildenbrand
  2022-01-21  8:59       ` Peter Zijlstra
  0 siblings, 2 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21  7:51 UTC (permalink / raw)
  To: David Hildenbrand
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Thu, Jan 20, 2022 at 07:25:08PM +0100, David Hildenbrand wrote:
> On 20.01.22 16:55, Peter Zijlstra wrote:
> > Add a guarantee for Anon pages that pin_user_page*() ensures the
> > user-mapping of these pages stay preserved. In order to ensure this
> > all rmap users have been audited:
> > 
> >  vmscan:	already fails eviction due to page_maybe_dma_pinned()
> > 
> >  migrate:	migration will fail on pinned pages due to
> > 		expected_page_refs() not matching, however that is
> > 		*after* try_to_migrate() has already destroyed the
> > 		user mapping of these pages. Add an early exit for
> > 		this case.
> > 
> >  numa-balance:	as per the above, pinned pages cannot be migrated,
> > 		however numa balancing scanning will happily PROT_NONE
> > 		them to get usage information on these pages. Avoid
> > 		this for pinned pages.
> 
> page_maybe_dma_pinned() can race with GUP-fast without
> mm->write_protect_seq. This is a real problem for vmscan() with
> concurrent GUP-fast as it can result in R/O mappings of pinned pages and
> GUP will lose synchronicity to the page table on write faults due to
> wrong COW.

Urgh, so yeah, that might be a problem. Follow up code uses it like
this:

+/*
+ * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
+ * through the user mapping ensures the user mapping exists.
+ */
+#define umcg_pin_and_load(_self, _pagep, _member)                              \
+({                                                                             \
+       __label__ __out;                                                        \
+       int __ret = -EFAULT;                                                    \
+                                                                               \
+       if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)  \
+               goto __out;                                                     \
+                                                                               \
+       if (!PageAnon(_pagep) ||                                                \
+           get_user(_member, &(_self)->_member)) {                             \
+               unpin_user_page(_pagep);                                        \
+               goto __out;                                                     \
+       }                                                                       \
+       __ret = 0;                                                              \
+__out: __ret;                                                                  \
+})

And after that hard assumes (on the penalty of SIGKILL) that direct user
access works. Specifically it does RmW ops on it. So I suppose I'd
better upgrade that load to a RmW at the very least.

But is that sufficient? Let me go find that race you mention...


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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-20 18:03   ` Nadav Amit
@ 2022-01-21  7:59     ` Peter Zijlstra
  0 siblings, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21  7:59 UTC (permalink / raw)
  To: Nadav Amit
  Cc: Ingo Molnar, Thomas Gleixner, juri.lelli, vincent.guittot,
	dietmar.eggemann, Steven Rostedt, bsegall, Mel Gorman, bristot,
	Linux Kernel Mailing List, Linux-MM, linux-api, X86 ML, pjt,
	posk, avagin, Jann Horn, tdelisle, mark.rutland, posk

On Thu, Jan 20, 2022 at 10:03:44AM -0800, Nadav Amit wrote:

> > --- a/mm/mprotect.c
> > +++ b/mm/mprotect.c
> > @@ -106,6 +106,12 @@ static unsigned long change_pte_range(st
> > 					continue;
> > 
> > 				/*
> > +				 * Can't migrate pinned pages, avoid touching them.
> > +				 */
> > +				if (page_maybe_dma_pinned(page))
> > +					continue;
> > +
> > +				/*
> > 
> 
> I have a similar problem with userfaultfd changing protection for
> DMA-pinned pages. For userfaultfd it is important to know how many
> pages were actually modified.
> 
> I am working on a vectored UFFDIO_WRITEPROTECTV that aborts once
> a pinned page is encountered, but also returns the number of pages
> that were properly protected. I still need to do some work to
> send patches for that as it requires further changes (to return
> the number of pages that were handled).
> 
> But for the matter of your patch, is it possible to make this
> test generic (not migration specific) and rely on a new flag in
> cp_flags? I can of course make this change later if you prefer it
> this way.

I have no objection to making it apply more widely, but I'm currently
only interested in the rmap users. If userspace does mprotect() on it's
own pages, it gets to keep whatever pieces are the result of that.


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

* Re: [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups
  2022-01-20 17:28 ` [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Oskolkov
@ 2022-01-21  8:01   ` Peter Zijlstra
  0 siblings, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21  8:01 UTC (permalink / raw)
  To: Peter Oskolkov
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, avagin, jannh, tdelisle, mark.rutland, posk

On Thu, Jan 20, 2022 at 09:28:11AM -0800, Peter Oskolkov wrote:
> On Thu, Jan 20, 2022 at 8:09 AM Peter Zijlstra <peterz@infradead.org> wrote:
> >
> > Latest version, many changes since last time, still under heavy discussion.
> 
> Thanks! I'll work on testing/integrating this patchset. I'll also
> assume that my changes that added blocked worker list
> (https://lore.kernel.org/lkml/20220113233940.3608440-5-posk@google.com/)

So the blocked list I don't mind, but I'm not sure about the use-case
you mentioned... it gets a mess vs PE, but I see why you'd want it, so i
need to think about it more.

> and server kicking
> (https://lore.kernel.org/lkml/20220113233940.3608440-6-posk@google.com/)
> are acceptable.

Still need to look at the server kicking thing...


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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-21  7:51     ` Peter Zijlstra
@ 2022-01-21  8:22       ` David Hildenbrand
  2022-01-21  8:59       ` Peter Zijlstra
  1 sibling, 0 replies; 47+ messages in thread
From: David Hildenbrand @ 2022-01-21  8:22 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On 21.01.22 08:51, Peter Zijlstra wrote:
> On Thu, Jan 20, 2022 at 07:25:08PM +0100, David Hildenbrand wrote:
>> On 20.01.22 16:55, Peter Zijlstra wrote:
>>> Add a guarantee for Anon pages that pin_user_page*() ensures the
>>> user-mapping of these pages stay preserved. In order to ensure this
>>> all rmap users have been audited:
>>>
>>>  vmscan:	already fails eviction due to page_maybe_dma_pinned()
>>>
>>>  migrate:	migration will fail on pinned pages due to
>>> 		expected_page_refs() not matching, however that is
>>> 		*after* try_to_migrate() has already destroyed the
>>> 		user mapping of these pages. Add an early exit for
>>> 		this case.
>>>
>>>  numa-balance:	as per the above, pinned pages cannot be migrated,
>>> 		however numa balancing scanning will happily PROT_NONE
>>> 		them to get usage information on these pages. Avoid
>>> 		this for pinned pages.
>>
>> page_maybe_dma_pinned() can race with GUP-fast without
>> mm->write_protect_seq. This is a real problem for vmscan() with
>> concurrent GUP-fast as it can result in R/O mappings of pinned pages and
>> GUP will lose synchronicity to the page table on write faults due to
>> wrong COW.
> 
> Urgh, so yeah, that might be a problem. Follow up code uses it like
> this:
> 
> +/*
> + * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
> + * through the user mapping ensures the user mapping exists.
> + */
> +#define umcg_pin_and_load(_self, _pagep, _member)                              \
> +({                                                                             \
> +       __label__ __out;                                                        \
> +       int __ret = -EFAULT;                                                    \
> +                                                                               \
> +       if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)  \
> +               goto __out;                                                     \
> +                                                                               \
> +       if (!PageAnon(_pagep) ||                                                \
> +           get_user(_member, &(_self)->_member)) {                             \
> +               unpin_user_page(_pagep);                                        \
> +               goto __out;                                                     \
> +       }                                                                       \
> +       __ret = 0;                                                              \
> +__out: __ret;                                                                  \
> +})
> 
> And after that hard assumes (on the penalty of SIGKILL) that direct user
> access works. Specifically it does RmW ops on it. So I suppose I'd
> better upgrade that load to a RmW at the very least.
> 
> But is that sufficient? Let me go find that race you mention...
> 

It's described in [1] under point 3.

After we put the page into the swapcache, it's still mapped into the
page tables, where GUP can find it. Only after that, we try to unmap the
page (placing swap entries). So it's racy.


Note also point 2. in [1], which is related to O_DIRECT that does
currently not yet use FOLL_PIN but uses FOLL_GET.


[1]
https://lore.kernel.org/r/3ae33b08-d9ef-f846-56fb-645e3b9b4c66@redhat.com

-- 
Thanks,

David / dhildenb



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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-21  7:51     ` Peter Zijlstra
  2022-01-21  8:22       ` David Hildenbrand
@ 2022-01-21  8:59       ` Peter Zijlstra
  2022-01-21  9:04         ` David Hildenbrand
  1 sibling, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21  8:59 UTC (permalink / raw)
  To: David Hildenbrand
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Fri, Jan 21, 2022 at 08:51:57AM +0100, Peter Zijlstra wrote:
> On Thu, Jan 20, 2022 at 07:25:08PM +0100, David Hildenbrand wrote:
> > On 20.01.22 16:55, Peter Zijlstra wrote:
> > > Add a guarantee for Anon pages that pin_user_page*() ensures the
> > > user-mapping of these pages stay preserved. In order to ensure this
> > > all rmap users have been audited:
> > > 
> > >  vmscan:	already fails eviction due to page_maybe_dma_pinned()
> > > 
> > >  migrate:	migration will fail on pinned pages due to
> > > 		expected_page_refs() not matching, however that is
> > > 		*after* try_to_migrate() has already destroyed the
> > > 		user mapping of these pages. Add an early exit for
> > > 		this case.
> > > 
> > >  numa-balance:	as per the above, pinned pages cannot be migrated,
> > > 		however numa balancing scanning will happily PROT_NONE
> > > 		them to get usage information on these pages. Avoid
> > > 		this for pinned pages.
> > 
> > page_maybe_dma_pinned() can race with GUP-fast without
> > mm->write_protect_seq. This is a real problem for vmscan() with
> > concurrent GUP-fast as it can result in R/O mappings of pinned pages and
> > GUP will lose synchronicity to the page table on write faults due to
> > wrong COW.
> 
> Urgh, so yeah, that might be a problem. Follow up code uses it like
> this:
> 
> +/*
> + * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
> + * through the user mapping ensures the user mapping exists.
> + */
> +#define umcg_pin_and_load(_self, _pagep, _member)                              \
> +({                                                                             \
> +       __label__ __out;                                                        \
> +       int __ret = -EFAULT;                                                    \
> +                                                                               \
> +       if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)  \
> +               goto __out;                                                     \
> +                                                                               \
> +       if (!PageAnon(_pagep) ||                                                \
> +           get_user(_member, &(_self)->_member)) {                             \
> +               unpin_user_page(_pagep);                                        \
> +               goto __out;                                                     \
> +       }                                                                       \
> +       __ret = 0;                                                              \
> +__out: __ret;                                                                  \
> +})
> 
> And after that hard assumes (on the penalty of SIGKILL) that direct user
> access works. Specifically it does RmW ops on it. So I suppose I'd
> better upgrade that load to a RmW at the very least.
> 
> But is that sufficient? Let me go find that race you mention...

OK, so copy_page_range() vs lockless_pages_from_mm(). Since I use
FOLL_PIN that should be sorted, it'll fall back the slow path and use
mmap_sem and serialize against the fork().

(Also, can I express my hate for __gup_longterm_unlocked(), that
function name is utter garbage)

However, I'm not quite sure what fork() does with pages that have a pin.
There's been a number of GUP vs fork() problems over the years, but I'm
afraid I have lost track of that and I can't quickly find anything in
the code..

Naively, a page that has async DMA activity should not be CoW'ed, or if
it is, care must be taken to ensure the original pages stays in the
original process, but I realize that's somewhat hard.

Let me dig in a bit more.


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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-21  8:59       ` Peter Zijlstra
@ 2022-01-21  9:04         ` David Hildenbrand
  2022-01-21 11:40           ` Peter Zijlstra
  0 siblings, 1 reply; 47+ messages in thread
From: David Hildenbrand @ 2022-01-21  9:04 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On 21.01.22 09:59, Peter Zijlstra wrote:
> On Fri, Jan 21, 2022 at 08:51:57AM +0100, Peter Zijlstra wrote:
>> On Thu, Jan 20, 2022 at 07:25:08PM +0100, David Hildenbrand wrote:
>>> On 20.01.22 16:55, Peter Zijlstra wrote:
>>>> Add a guarantee for Anon pages that pin_user_page*() ensures the
>>>> user-mapping of these pages stay preserved. In order to ensure this
>>>> all rmap users have been audited:
>>>>
>>>>  vmscan:	already fails eviction due to page_maybe_dma_pinned()
>>>>
>>>>  migrate:	migration will fail on pinned pages due to
>>>> 		expected_page_refs() not matching, however that is
>>>> 		*after* try_to_migrate() has already destroyed the
>>>> 		user mapping of these pages. Add an early exit for
>>>> 		this case.
>>>>
>>>>  numa-balance:	as per the above, pinned pages cannot be migrated,
>>>> 		however numa balancing scanning will happily PROT_NONE
>>>> 		them to get usage information on these pages. Avoid
>>>> 		this for pinned pages.
>>>
>>> page_maybe_dma_pinned() can race with GUP-fast without
>>> mm->write_protect_seq. This is a real problem for vmscan() with
>>> concurrent GUP-fast as it can result in R/O mappings of pinned pages and
>>> GUP will lose synchronicity to the page table on write faults due to
>>> wrong COW.
>>
>> Urgh, so yeah, that might be a problem. Follow up code uses it like
>> this:
>>
>> +/*
>> + * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
>> + * through the user mapping ensures the user mapping exists.
>> + */
>> +#define umcg_pin_and_load(_self, _pagep, _member)                              \
>> +({                                                                             \
>> +       __label__ __out;                                                        \
>> +       int __ret = -EFAULT;                                                    \
>> +                                                                               \
>> +       if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)  \
>> +               goto __out;                                                     \
>> +                                                                               \
>> +       if (!PageAnon(_pagep) ||                                                \
>> +           get_user(_member, &(_self)->_member)) {                             \
>> +               unpin_user_page(_pagep);                                        \
>> +               goto __out;                                                     \
>> +       }                                                                       \
>> +       __ret = 0;                                                              \
>> +__out: __ret;                                                                  \
>> +})
>>
>> And after that hard assumes (on the penalty of SIGKILL) that direct user
>> access works. Specifically it does RmW ops on it. So I suppose I'd
>> better upgrade that load to a RmW at the very least.
>>
>> But is that sufficient? Let me go find that race you mention...
> 
> OK, so copy_page_range() vs lockless_pages_from_mm(). Since I use
> FOLL_PIN that should be sorted, it'll fall back the slow path and use
> mmap_sem and serialize against the fork().
> 
> (Also, can I express my hate for __gup_longterm_unlocked(), that
> function name is utter garbage)

Absolutely, the "_unlocked_ also caused a lot of confusion on my end in
the past.

> 
> However, I'm not quite sure what fork() does with pages that have a pin.

We COW the anon pages always, and we protect against concurrent GUP
using the
* mmap_lock in exclusive mode for ordinary GUP
* mm->write_protect_seq for GUP-fast

> 
> Naively, a page that has async DMA activity should not be CoW'ed, or if
> it is, care must be taken to ensure the original pages stays in the
> original process, but I realize that's somewhat hard.

That's precisely what I'm working on fixing ... and yes, it's hard.

Let me know if you need any other information, I've spent way too much
time on this than I ever panned.

-- 
Thanks,

David / dhildenb



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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-21  9:04         ` David Hildenbrand
@ 2022-01-21 11:40           ` Peter Zijlstra
  2022-01-21 12:04             ` David Hildenbrand
  0 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21 11:40 UTC (permalink / raw)
  To: David Hildenbrand
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Fri, Jan 21, 2022 at 10:04:45AM +0100, David Hildenbrand wrote:
> On 21.01.22 09:59, Peter Zijlstra wrote:

> > However, I'm not quite sure what fork() does with pages that have a pin.
> 
> We COW the anon pages always, and we protect against concurrent GUP
> using the
> * mmap_lock in exclusive mode for ordinary GUP
> * mm->write_protect_seq for GUP-fast

Right, but neither the mmap_sem nor the write_protect_seq help anything
at all vs already extant page pins.

But I just found copy_present_page()'s page_needs_cow_for_dma(), which I
think deals with exactly that case, it avoids doing CoW on pinned pages
and instead feeds the child a full copy while keeping the pinned page in
the original process.

> > Naively, a page that has async DMA activity should not be CoW'ed, or if
> > it is, care must be taken to ensure the original pages stays in the
> > original process, but I realize that's somewhat hard.
> 
> That's precisely what I'm working on fixing ... and yes, it's hard.
> 
> Let me know if you need any other information, I've spent way too much
> time on this than I ever panned.

So let me try and get this right:

 - GUP post-fork breaks CoW for FOLL_WRITE/FOLL_PIN, without either
   there's a problem where one task might observe changes by another.

 - GUP pre-fork prevents CoW and does a full copy.

And that all mostly works, except for a fair amount of 'fun' cases?



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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
@ 2022-01-21 11:47   ` Peter Zijlstra
  2022-01-21 15:18     ` Peter Zijlstra
  2022-01-24 13:59     ` Peter Zijlstra
  2022-01-21 12:26   ` Peter Zijlstra
                     ` (3 subsequent siblings)
  4 siblings, 2 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21 11:47 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk

On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:

> +SYSCALL_DEFINE2(umcg_wait, u32, flags, u64, timo)
> +{
> +	struct task_struct *tsk = current;
> +	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
> +	bool worker = tsk->flags & PF_UMCG_WORKER;
> +	int ret;
> +
> +	if (!self || flags)
> +		return -EINVAL;
> +
> +	if (worker) {
> +		tsk->flags &= ~PF_UMCG_WORKER;
> +		if (timo)
> +			return -ERANGE;
> +	}
> +
> +	/* see umcg_sys_{enter,exit}() syscall exceptions */
> +	ret = umcg_pin_pages();
> +	if (ret)
> +		goto unblock;
> +
> +	/*
> +	 * Clear UMCG_TF_COND_WAIT *and* check state == RUNNABLE.
> +	 */
> +	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNABLE);
> +	if (ret)
> +		goto unpin;
> +
> +	ret = umcg_wake_next(tsk, self);
> +	if (ret)
> +		goto unpin;
> +
> +	if (worker) {
> +		/*
> +		 * If this fails it is possible ::next_tid is already running
> +		 * while this task is not going to block. This violates our
> +		 * constraints.
> +		 *
> +		 * That said, pretty much the only way to make this fail is by
> +		 * force munmap()'ing things. In which case one is most welcome
> +		 * to the pieces.
> +		 */
> +		ret = umcg_enqueue_and_wake(tsk);
> +		if (ret)
> +			goto unpin;
> +	}
> +
> +	umcg_unpin_pages();
> +
> +	ret = umcg_wait(timo);
> +	switch (ret) {
> +	case 0:		/* all done */
> +	case -EINTR:	/* umcg_notify_resume() will continue the wait */

So I was playing with the whole worker timeout thing last night and
realized this is broken. If we get a signal while we have a timeout, the
timeout gets lost.

I think the easiest solution is to have umcg_notify_resume() also resume
the timeout, but the first pass of that was yuck, so I need to try
again.

Related, by moving the whole enqueue-and-wake thing into the timeout, we
get more 'fun' failure cases :-(

Oh well..

> +		ret = 0;
> +		break;
> +
> +	default:
> +		goto unblock;
> +	}
> +out:
> +	if (worker)
> +		tsk->flags |= PF_UMCG_WORKER;
> +	return ret;
> +
> +unpin:
> +	umcg_unpin_pages();
> +unblock:
> +	umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNING);
> +	goto out;
> +}


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

* Re: [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages
  2022-01-21 11:40           ` Peter Zijlstra
@ 2022-01-21 12:04             ` David Hildenbrand
  0 siblings, 0 replies; 47+ messages in thread
From: David Hildenbrand @ 2022-01-21 12:04 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On 21.01.22 12:40, Peter Zijlstra wrote:
> On Fri, Jan 21, 2022 at 10:04:45AM +0100, David Hildenbrand wrote:
>> On 21.01.22 09:59, Peter Zijlstra wrote:
> 
>>> However, I'm not quite sure what fork() does with pages that have a pin.
>>
>> We COW the anon pages always, and we protect against concurrent GUP
>> using the
>> * mmap_lock in exclusive mode for ordinary GUP
>> * mm->write_protect_seq for GUP-fast
> 
> Right, but neither the mmap_sem nor the write_protect_seq help anything
> at all vs already extant page pins.
> 
> But I just found copy_present_page()'s page_needs_cow_for_dma(), which I
> think deals with exactly that case, it avoids doing CoW on pinned pages
> and instead feeds the child a full copy while keeping the pinned page in
> the original process.

Yes, page_needs_cow_for_dma() is the magic bit. The locking I explained
keep its output "reliable".

> 
>>> Naively, a page that has async DMA activity should not be CoW'ed, or if
>>> it is, care must be taken to ensure the original pages stays in the
>>> original process, but I realize that's somewhat hard.
>>
>> That's precisely what I'm working on fixing ... and yes, it's hard.
>>
>> Let me know if you need any other information, I've spent way too much
>> time on this than I ever panned.
> 
> So let me try and get this right:
> 
>  - GUP post-fork breaks CoW for FOLL_WRITE/FOLL_PIN, without either
>    there's a problem where one task might observe changes by another.
> 
>  - GUP pre-fork prevents CoW and does a full copy.

Yes, pretty much.

> 
> And that all mostly works, except for a fair amount of 'fun' cases?

I'd say some obviously broken cases, some racy cases, some fun cases :)


We have three main cases. And usually, trying to tackle one triggers
another.

(1) Missed CoW

If the child R/O pins and unmaps the page, the parent might miss to CoW
and reuse the page. Security issue. Once CVE in that area is currently
still applicable for THP (well, and hugetlb).

(2) Unnecessary CoW

We CoW instead of reusing the page, but there are no relevant pins, so
it's unnecessary.

(3) Wrong CoW

We CoW a page that has relevant pins, losing synchronicity between GUP
and the page tables.


The "criticality" is (1), (3), (2).

(3) can currently get triggered by anything that can map a pinned page
R/O. The racy case is what I described about the swapcache. Other broken
cases are mprotect() and friends (we cannot differentiate between R/O
and R/W pins ...).


-- 
Thanks,

David / dhildenb



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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
  2022-01-21 11:47   ` Peter Zijlstra
@ 2022-01-21 12:26   ` Peter Zijlstra
  2022-01-21 16:57   ` Mark Rutland
                     ` (2 subsequent siblings)
  4 siblings, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21 12:26 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk, david

On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:
> +/*
> + * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
> + * through the user mapping ensures the user mapping exists.
> + */
> +#define umcg_pin_and_load(_self, _pagep, _member)				\
> +({										\
> +	__label__ __out;							\
> +	int __ret = -EFAULT;							\
> +										\
> +	if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)	\
> +		goto __out;							\
> +										\
> +	if (!PageAnon(_pagep) ||						\
> +	    get_user(_member, &(_self)->_member)) {				\
> +		unpin_user_page(_pagep);					\
> +		goto __out;							\
> +	}									\
> +	__ret = 0;								\
> +__out:	__ret;									\
> +})

Per the thread with David, this wants changing like so.

---

--- a/kernel/sched/umcg.c
+++ b/kernel/sched/umcg.c
@@ -34,25 +34,26 @@ static struct task_struct *umcg_get_task
 }
 
 /*
- * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
- * through the user mapping ensures the user mapping exists.
+ * Pinning a page inhibits rmap based unmap for Anon pages. Doing a store
+ * through the user mapping ensures the user mapping exists and is writable.
  */
-#define umcg_pin_and_load(_self, _pagep, _member)				\
-({										\
-	__label__ __out;							\
-	int __ret = -EFAULT;							\
-										\
-	if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)	\
-		goto __out;							\
-										\
-	if (!PageAnon(_pagep) ||						\
-	    get_user(_member, &(_self)->_member)) {				\
-		unpin_user_page(_pagep);					\
-		goto __out;							\
-	}									\
-	__ret = 0;								\
-__out:	__ret;									\
-})
+static int umcg_pin_page(struct umcg_task __user *self, struct page **pagep)
+{
+	int ret = -EFAULT;
+
+	if (pin_user_pages_fast((unsigned long)self, 1, FOLL_WRITE, pagep) != 1)
+		goto out;
+
+	if (!PageAnon(*pagep) ||
+	    put_user(0ULL, &self->__zero[0])) {
+		unpin_user_page(*pagep);
+		goto out;
+	}
+
+	ret = 0;
+out:
+	return ret;
+}
 
 /**
  * umcg_pin_pages: pin pages containing struct umcg_task of
@@ -72,10 +73,13 @@ static int umcg_pin_pages(void)
 			 tsk->umcg_server))
 		return -EBUSY;
 
-	ret = umcg_pin_and_load(self, tsk->umcg_page, server_tid);
+	ret = umcg_pin_page(self, &tsk->umcg_page);
 	if (ret)
 		goto clear_self;
 
+	if (get_user(server_tid, &self->server_tid))
+		goto unpin_self;
+
 	ret = -ESRCH;
 	server = umcg_get_task(server_tid);
 	if (!server)
@@ -83,7 +87,7 @@ static int umcg_pin_pages(void)
 
 	/* must cache due to possible concurrent change */
 	tsk->umcg_server_task = READ_ONCE(server->umcg_task);
-	ret = umcg_pin_and_load(tsk->umcg_server_task, tsk->umcg_server_page, server_tid);
+	ret = umcg_pin_page(tsk->umcg_server_task, &tsk->umcg_server_page);
 	if (ret)
 		goto clear_server;
 
@@ -414,7 +418,7 @@ static int umcg_wait(u64 timo)
 				break;
 			}
 
-			ret = umcg_pin_and_load(self, page, state);
+			ret = umcg_pin_page(self, &page);
 			if (ret) {
 				page = NULL;
 				break;


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-21 11:47   ` Peter Zijlstra
@ 2022-01-21 15:18     ` Peter Zijlstra
  2022-01-24 14:29       ` Peter Zijlstra
  2022-01-24 13:59     ` Peter Zijlstra
  1 sibling, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-21 15:18 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk

On Fri, Jan 21, 2022 at 12:47:58PM +0100, Peter Zijlstra wrote:
> On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:
> 
> > +SYSCALL_DEFINE2(umcg_wait, u32, flags, u64, timo)
> > +{
> > +	struct task_struct *tsk = current;
> > +	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
> > +	bool worker = tsk->flags & PF_UMCG_WORKER;
> > +	int ret;
> > +
> > +	if (!self || flags)
> > +		return -EINVAL;
> > +
> > +	if (worker) {
> > +		tsk->flags &= ~PF_UMCG_WORKER;
> > +		if (timo)
> > +			return -ERANGE;
> > +	}
> > +
> > +	/* see umcg_sys_{enter,exit}() syscall exceptions */
> > +	ret = umcg_pin_pages();
> > +	if (ret)
> > +		goto unblock;
> > +
> > +	/*
> > +	 * Clear UMCG_TF_COND_WAIT *and* check state == RUNNABLE.
> > +	 */
> > +	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNABLE);
> > +	if (ret)
> > +		goto unpin;
> > +
> > +	ret = umcg_wake_next(tsk, self);
> > +	if (ret)
> > +		goto unpin;
> > +
> > +	if (worker) {
> > +		/*
> > +		 * If this fails it is possible ::next_tid is already running
> > +		 * while this task is not going to block. This violates our
> > +		 * constraints.
> > +		 *
> > +		 * That said, pretty much the only way to make this fail is by
> > +		 * force munmap()'ing things. In which case one is most welcome
> > +		 * to the pieces.
> > +		 */
> > +		ret = umcg_enqueue_and_wake(tsk);
> > +		if (ret)
> > +			goto unpin;
> > +	}
> > +
> > +	umcg_unpin_pages();
> > +
> > +	ret = umcg_wait(timo);
> > +	switch (ret) {
> > +	case 0:		/* all done */
> > +	case -EINTR:	/* umcg_notify_resume() will continue the wait */
> 
> So I was playing with the whole worker timeout thing last night and
> realized this is broken. If we get a signal while we have a timeout, the
> timeout gets lost.
> 
> I think the easiest solution is to have umcg_notify_resume() also resume
> the timeout, but the first pass of that was yuck, so I need to try
> again.

Something like this, still yuck though. Also still need to write me a
test for this.

--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1300,12 +1300,14 @@ struct task_struct {
 	clockid_t		umcg_clock;
 	struct umcg_task __user	*umcg_task;
 
-	/* setup by umcg_pin_enter() */
+	/* setup by umcg_pin_pages() */
 	struct page		*umcg_page;
 
 	struct task_struct	*umcg_server;
 	struct umcg_task __user *umcg_server_task;
 	struct page		*umcg_server_page;
+
+	u64			umcg_timeout;
 #endif
 
 	struct tlbflush_unmap_batch	tlb_ubc;
--- a/kernel/sched/umcg.c
+++ b/kernel/sched/umcg.c
@@ -232,6 +232,8 @@ static int umcg_update_state(struct task
 /* Called from syscall enter path and exceptions that can schedule */
 void umcg_sys_enter(struct pt_regs *regs, long syscall)
 {
+	current->umcg_timeout = 0;
+
 	/* avoid recursion vs our own syscalls */
 	if (syscall == __NR_umcg_wait ||
 	    syscall == __NR_umcg_ctl)
@@ -519,6 +521,7 @@ void umcg_notify_resume(struct pt_regs *
 	struct umcg_task __user *self = tsk->umcg_task;
 	bool worker = tsk->flags & PF_UMCG_WORKER;
 	u32 state;
+	int ret;
 
 	/* avoid recursion vs schedule() */
 	if (worker)
@@ -554,12 +557,17 @@ void umcg_notify_resume(struct pt_regs *
 		umcg_unpin_pages();
 	}
 
-	switch (umcg_wait(0)) {
+	ret = umcg_wait(tsk->umcg_timeout);
+	switch (ret) {
 	case 0:
 	case -EINTR:
 		/* we will resume the wait after the signal */
 		break;
 
+	case -ETIMEDOUT:
+		regs_set_return_value(regs, ret);
+		break;
+
 	default:
 		UMCG_DIE("wait");
 	}
@@ -759,6 +767,7 @@ SYSCALL_DEFINE2(umcg_wait, u32, flags, u
 	switch (ret) {
 	case 0:		/* all done */
 	case -EINTR:	/* umcg_notify_resume() will continue the wait */
+		tsk->umcg_timeout = timo;
 		ret = 0;
 		break;
 


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

* Re: [RFC][PATCH v2 2/5] entry,x86: Create common IRQ operations for exceptions
  2022-01-20 15:55 ` [RFC][PATCH v2 2/5] entry,x86: Create common IRQ operations for exceptions Peter Zijlstra
@ 2022-01-21 16:34   ` Mark Rutland
  0 siblings, 0 replies; 47+ messages in thread
From: Mark Rutland @ 2022-01-21 16:34 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, posk

On Thu, Jan 20, 2022 at 04:55:19PM +0100, Peter Zijlstra wrote:
> A number of exceptions can re-enable IRQs and schedule (provided the
> context taking the exception has IRQs enabled etc.). Currently this
> isn't standardized and each architecture does it slightly different.
> 
> (Notably, ARM64 restores/inherits more than just the IRQ state,
> suggesting at least some form of arch_ hook might be appropriate on
> top of this).

For arm64, I suspect we want the arch code to call the umcg hooks directly,
rather than adding arch_* hooks that get called by the generic
irqentry_irq_{enable,disable}() functions, which doesn't really fit the way we
want to conditionally inherit some mask bits upon entry (depending on the
specific exception taken) and unconditionally mask everything prior to exit.

Ignoring that, arm64 also has separate vectors for exceptions taken from EL0
(userspace) and EL1 (kernel), so we don't need to look at the regs to know that
an exception has been taken from userspace (which we always run with IRQs
unmasked). For patch 5, where we actually add hooks, it'd be cleaner for us to
directly invoke UMCG hooks in our enter_from_user_mode() and
exit_to_user_mode() paths.

That said, if this is useful for other architectures, I don't have a problem
for adding them as generic helpers; I just don't think we should be required to
use them directly, and should make sure the underlying primitives are available
and clearly documented.

Thanks,
Mark.

> Create a single set of functions for this and convert x86 to use
> these. The purpose is to have a single (common) place to hook in order
> to cover all the exceptions that can schedule().
> 
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> ---
>  arch/x86/kernel/traps.c      |   48 ++++++++++++++++---------------------------
>  arch/x86/mm/fault.c          |   28 ++++++++++++-------------
>  include/linux/entry-common.h |   24 +++++++++++++++++++++
>  3 files changed, 56 insertions(+), 44 deletions(-)
> 
> --- a/arch/x86/kernel/traps.c
> +++ b/arch/x86/kernel/traps.c
> @@ -73,18 +73,6 @@
>  
>  DECLARE_BITMAP(system_vectors, NR_VECTORS);
>  
> -static inline void cond_local_irq_enable(struct pt_regs *regs)
> -{
> -	if (regs->flags & X86_EFLAGS_IF)
> -		local_irq_enable();
> -}
> -
> -static inline void cond_local_irq_disable(struct pt_regs *regs)
> -{
> -	if (regs->flags & X86_EFLAGS_IF)
> -		local_irq_disable();
> -}
> -
>  __always_inline int is_valid_bugaddr(unsigned long addr)
>  {
>  	if (addr < TASK_SIZE_MAX)
> @@ -177,9 +165,9 @@ static void do_error_trap(struct pt_regs
>  
>  	if (notify_die(DIE_TRAP, str, regs, error_code, trapnr, signr) !=
>  			NOTIFY_STOP) {
> -		cond_local_irq_enable(regs);
> +		irqentry_irq_enable(regs);
>  		do_trap(trapnr, signr, str, regs, error_code, sicode, addr);
> -		cond_local_irq_disable(regs);
> +		irqentry_irq_disable(regs);
>  	}
>  }
>  
> @@ -300,7 +288,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_alignment_
>  	if (!user_mode(regs))
>  		die("Split lock detected\n", regs, error_code);
>  
> -	local_irq_enable();
> +	irqentry_irq_enable(regs);
>  
>  	if (handle_user_split_lock(regs, error_code))
>  		goto out;
> @@ -309,7 +297,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_alignment_
>  		error_code, BUS_ADRALN, NULL);
>  
>  out:
> -	local_irq_disable();
> +	irqentry_irq_disable(regs);
>  }
>  
>  #ifdef CONFIG_VMAP_STACK
> @@ -473,14 +461,14 @@ DEFINE_IDTENTRY(exc_bounds)
>  	if (notify_die(DIE_TRAP, "bounds", regs, 0,
>  			X86_TRAP_BR, SIGSEGV) == NOTIFY_STOP)
>  		return;
> -	cond_local_irq_enable(regs);
> +	irqentry_irq_enable(regs);
>  
>  	if (!user_mode(regs))
>  		die("bounds", regs, 0);
>  
>  	do_trap(X86_TRAP_BR, SIGSEGV, "bounds", regs, 0, 0, NULL);
>  
> -	cond_local_irq_disable(regs);
> +	irqentry_irq_disable(regs);
>  }
>  
>  enum kernel_gp_hint {
> @@ -567,7 +555,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_general_pr
>  	unsigned long gp_addr;
>  	int ret;
>  
> -	cond_local_irq_enable(regs);
> +	irqentry_irq_enable(regs);
>  
>  	if (static_cpu_has(X86_FEATURE_UMIP)) {
>  		if (user_mode(regs) && fixup_umip_exception(regs))
> @@ -638,7 +626,7 @@ DEFINE_IDTENTRY_ERRORCODE(exc_general_pr
>  	die_addr(desc, regs, error_code, gp_addr);
>  
>  exit:
> -	cond_local_irq_disable(regs);
> +	irqentry_irq_disable(regs);
>  }
>  
>  static bool do_int3(struct pt_regs *regs)
> @@ -665,9 +653,9 @@ static void do_int3_user(struct pt_regs
>  	if (do_int3(regs))
>  		return;
>  
> -	cond_local_irq_enable(regs);
> +	irqentry_irq_enable(regs);
>  	do_trap(X86_TRAP_BP, SIGTRAP, "int3", regs, 0, 0, NULL);
> -	cond_local_irq_disable(regs);
> +	irqentry_irq_disable(regs);
>  }
>  
>  DEFINE_IDTENTRY_RAW(exc_int3)
> @@ -1003,7 +991,7 @@ static __always_inline void exc_debug_us
>  		goto out;
>  
>  	/* It's safe to allow irq's after DR6 has been saved */
> -	local_irq_enable();
> +	irqentry_irq_enable(regs);
>  
>  	if (v8086_mode(regs)) {
>  		handle_vm86_trap((struct kernel_vm86_regs *)regs, 0, X86_TRAP_DB);
> @@ -1020,7 +1008,7 @@ static __always_inline void exc_debug_us
>  		send_sigtrap(regs, 0, get_si_code(dr6));
>  
>  out_irq:
> -	local_irq_disable();
> +	irqentry_irq_disable(regs);
>  out:
>  	instrumentation_end();
>  	irqentry_exit_to_user_mode(regs);
> @@ -1064,7 +1052,7 @@ static void math_error(struct pt_regs *r
>  	char *str = (trapnr == X86_TRAP_MF) ? "fpu exception" :
>  						"simd exception";
>  
> -	cond_local_irq_enable(regs);
> +	irqentry_irq_enable(regs);
>  
>  	if (!user_mode(regs)) {
>  		if (fixup_exception(regs, trapnr, 0, 0))
> @@ -1099,7 +1087,7 @@ static void math_error(struct pt_regs *r
>  	force_sig_fault(SIGFPE, si_code,
>  			(void __user *)uprobe_get_trap_addr(regs));
>  exit:
> -	cond_local_irq_disable(regs);
> +	irqentry_irq_disable(regs);
>  }
>  
>  DEFINE_IDTENTRY(exc_coprocessor_error)
> @@ -1160,7 +1148,7 @@ static bool handle_xfd_event(struct pt_r
>  	if (WARN_ON(!user_mode(regs)))
>  		return false;
>  
> -	local_irq_enable();
> +	irqentry_irq_enable(regs);
>  
>  	err = xfd_enable_feature(xfd_err);
>  
> @@ -1173,7 +1161,7 @@ static bool handle_xfd_event(struct pt_r
>  		break;
>  	}
>  
> -	local_irq_disable();
> +	irqentry_irq_disable(regs);
>  	return true;
>  }
>  
> @@ -1188,12 +1176,12 @@ DEFINE_IDTENTRY(exc_device_not_available
>  	if (!boot_cpu_has(X86_FEATURE_FPU) && (cr0 & X86_CR0_EM)) {
>  		struct math_emu_info info = { };
>  
> -		cond_local_irq_enable(regs);
> +		irqentry_irq_enable(regs);
>  
>  		info.regs = regs;
>  		math_emulate(&info);
>  
> -		cond_local_irq_disable(regs);
> +		irqentry_irq_disable(regs);
>  		return;
>  	}
>  #endif
> --- a/arch/x86/mm/fault.c
> +++ b/arch/x86/mm/fault.c
> @@ -1209,6 +1209,12 @@ do_kern_addr_fault(struct pt_regs *regs,
>  NOKPROBE_SYMBOL(do_kern_addr_fault);
>  
>  /*
> + * EFLAGS[3] is unused and ABI defined to be 0, use it to store IRQ state,
> + * because do_user_addr_fault() is too convoluted to track things.
> + */
> +#define X86_EFLAGS_MISC		(1UL << 3)
> +
> +/*
>   * Handle faults in the user portion of the address space.  Nothing in here
>   * should check X86_PF_USER without a specific justification: for almost
>   * all purposes, we should treat a normal kernel access to user memory
> @@ -1290,13 +1296,11 @@ void do_user_addr_fault(struct pt_regs *
>  	 * User-mode registers count as a user access even for any
>  	 * potential system fault or CPU buglet:
>  	 */
> -	if (user_mode(regs)) {
> -		local_irq_enable();
> +	if (user_mode(regs))
>  		flags |= FAULT_FLAG_USER;
> -	} else {
> -		if (regs->flags & X86_EFLAGS_IF)
> -			local_irq_enable();
> -	}
> +
> +	irqentry_irq_enable(regs);
> +	regs->flags |= X86_EFLAGS_MISC;
>  
>  	perf_sw_event(PERF_COUNT_SW_PAGE_FAULTS, 1, regs, address);
>  
> @@ -1483,14 +1487,10 @@ handle_page_fault(struct pt_regs *regs,
>  		do_kern_addr_fault(regs, error_code, address);
>  	} else {
>  		do_user_addr_fault(regs, error_code, address);
> -		/*
> -		 * User address page fault handling might have reenabled
> -		 * interrupts. Fixing up all potential exit points of
> -		 * do_user_addr_fault() and its leaf functions is just not
> -		 * doable w/o creating an unholy mess or turning the code
> -		 * upside down.
> -		 */
> -		local_irq_disable();
> +		if (regs->flags & X86_EFLAGS_MISC) {
> +			regs->flags &= ~X86_EFLAGS_MISC;
> +			irqentry_irq_disable(regs);
> +		}
>  	}
>  }
>  
> --- a/include/linux/entry-common.h
> +++ b/include/linux/entry-common.h
> @@ -7,6 +7,7 @@
>  #include <linux/syscalls.h>
>  #include <linux/seccomp.h>
>  #include <linux/sched.h>
> +#include <asm/ptrace.h>
>  
>  #include <asm/entry-common.h>
>  
> @@ -213,6 +214,29 @@ static inline void local_irq_disable_exi
>  #endif
>  
>  /**
> + * irqentry_irq_enable - Conditionally enable IRQs from exceptions
> + *
> + * Common code for exceptions to (re)enable IRQs, typically done to allow
> + * from-user exceptions to schedule (since they run on the task stack).
> + */
> +static inline void irqentry_irq_enable(struct pt_regs *regs)
> +{
> +	if (!regs_irqs_disabled(regs))
> +		local_irq_enable();
> +}
> +
> +/**
> + * irqentry_irq_disable - Conditionally disable IRQs from exceptions
> + *
> + * Counterpart of irqentry_irq_enable().
> + */
> +static inline void irqentry_irq_disable(struct pt_regs *regs)
> +{
> +	if (!regs_irqs_disabled(regs))
> +		local_irq_disable();
> +}
> +
> +/**
>   * arch_exit_to_user_mode_work - Architecture specific TIF work for exit
>   *				 to user mode.
>   * @regs:	Pointer to currents pt_regs
> 
> 


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
  2022-01-21 11:47   ` Peter Zijlstra
  2022-01-21 12:26   ` Peter Zijlstra
@ 2022-01-21 16:57   ` Mark Rutland
  2022-01-24  9:48     ` Peter Zijlstra
  2022-01-24 10:03     ` Peter Zijlstra
  2022-01-24 14:46   ` Tao Zhou
  2022-01-27 18:31   ` Tao Zhou
  4 siblings, 2 replies; 47+ messages in thread
From: Mark Rutland @ 2022-01-21 16:57 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, posk

On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:
> User Managed Concurrency Groups is an M:N threading toolkit that allows
> constructing user space schedulers designed to efficiently manage
> heterogeneous in-process workloads while maintaining high CPU
> utilization (95%+).
> 
> XXX moar changelog explaining how this is moar awesome than
> traditional user-space threading.

Awaiting a commit message that I can parse, I'm just looking at the entry bits
for now. TBH I have no idea what this is actually trying to do...

[...]

> --- a/include/linux/entry-common.h
> +++ b/include/linux/entry-common.h
> @@ -23,6 +23,10 @@
>  # define _TIF_UPROBE			(0)
>  #endif
>  
> +#ifndef _TIF_UMCG
> +# define _TIF_UMCG			(0)
> +#endif
> +
>  /*
>   * SYSCALL_WORK flags handled in syscall_enter_from_user_mode()
>   */
> @@ -43,11 +47,13 @@
>  				 SYSCALL_WORK_SYSCALL_EMU |		\
>  				 SYSCALL_WORK_SYSCALL_AUDIT |		\
>  				 SYSCALL_WORK_SYSCALL_USER_DISPATCH |	\
> +				 SYSCALL_WORK_SYSCALL_UMCG |		\
>  				 ARCH_SYSCALL_WORK_ENTER)
>  #define SYSCALL_WORK_EXIT	(SYSCALL_WORK_SYSCALL_TRACEPOINT |	\
>  				 SYSCALL_WORK_SYSCALL_TRACE |		\
>  				 SYSCALL_WORK_SYSCALL_AUDIT |		\
>  				 SYSCALL_WORK_SYSCALL_USER_DISPATCH |	\
> +				 SYSCALL_WORK_SYSCALL_UMCG |		\
>  				 SYSCALL_WORK_SYSCALL_EXIT_TRAP	|	\
>  				 ARCH_SYSCALL_WORK_EXIT)
>  
> @@ -221,8 +227,11 @@ static inline void local_irq_disable_exi
>   */
>  static inline void irqentry_irq_enable(struct pt_regs *regs)
>  {
> -	if (!regs_irqs_disabled(regs))
> +	if (!regs_irqs_disabled(regs)) {
>  		local_irq_enable();
> +		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
> +			umcg_sys_enter(regs, -1);
> +	}
>  }

Perhaps it would make sense to have separate umcg_sys_enter(regs) and
umcg_sys_enter_syscall(regs, syscallno)? Even if the former is just a wrapper,
to make the entry/exit bits clearly correspond for all the !syscall cases?

Also, is the syscall case meant to nest within this, or syscall entry paths not
supposed to call irqentry_irq_enable() ?

>  
>  /**
> @@ -232,8 +241,11 @@ static inline void irqentry_irq_enable(s
>   */
>  static inline void irqentry_irq_disable(struct pt_regs *regs)
>  {
> -	if (!regs_irqs_disabled(regs))
> +	if (!regs_irqs_disabled(regs)) {
> +		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
> +			umcg_sys_exit(regs);
>  		local_irq_disable();
> +	}
>  }

Do the umcg_sys_{enter,exit}() calls need to happen with IRQs unmasked?

* If not (and this nests): for arm64 these can live in our
  enter_from_user_mode() and exit_to_user_mode() helpers.

* If so (or this doesn't nest): for arm64 we'd need to rework our
  local_daif_{inherit,restore,mask}() calls to handle this, though I've been
  meaning to do that anyway to handle pseudo-NMI better.

Either way, it looks like we'd need helpers along the lines of:

| static __always_inline void umcg_enter_from_user(struct pt_regs *regs)
| {
| 	if (current->flags & PF_UMCG_WORKER)
| 		umcg_sys_enter(regs, -1);
| }
| 
| static __always_inline void umcg_exit_to_user(struct pt_regs *regs)
| {
| 	if (current->flags & PF_UMCG_WORKER)
| 		umcg_sys_exit(regs);
| }

Thanks,
Mark.


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

* Re: [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups
  2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
                   ` (5 preceding siblings ...)
  2022-01-20 17:28 ` [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Oskolkov
@ 2022-01-21 18:01 ` Steven Rostedt
  2022-01-24  8:20   ` Peter Zijlstra
  6 siblings, 1 reply; 47+ messages in thread
From: Steven Rostedt @ 2022-01-21 18:01 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	bsegall, mgorman, bristot, linux-kernel, linux-mm, linux-api,
	x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland, posk

On Thu, 20 Jan 2022 16:55:17 +0100
Peter Zijlstra <peterz@infradead.org> wrote:

> Latest version, many changes since last time, still under heavy discussion.
> 
> Seems to work with the test-case I have (below), but that still has a few gaps,
> coverage wise.

Nice work, Peter.

> 
> Still haven't done the SMP wakeup thing, finally get the idea with
> worker-timeouts but haven't yet implemented that.
> 
> Happy hacking..
> 
> ----
>

Should the below be added to samples/ ?

-- Steve
 
> #define _GNU_SOURCE
> #include <unistd.h>
> #include <sys/types.h>
> #include <sys/syscall.h>
> #include <pthread.h>
> #include <stdio.h>
> #include <stdlib.h>
> #include <errno.h>
> #include <signal.h>

[..]


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

* Re: [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups
  2022-01-21 18:01 ` Steven Rostedt
@ 2022-01-24  8:20   ` Peter Zijlstra
  0 siblings, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-24  8:20 UTC (permalink / raw)
  To: Steven Rostedt
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	bsegall, mgorman, bristot, linux-kernel, linux-mm, linux-api,
	x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland, posk

On Fri, Jan 21, 2022 at 01:01:45PM -0500, Steven Rostedt wrote:

> Should the below be added to samples/ ?

Maybe eventually, it needs a little TLC to build with clang at the very
least.


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-21 16:57   ` Mark Rutland
@ 2022-01-24  9:48     ` Peter Zijlstra
  2022-01-24 10:03     ` Peter Zijlstra
  1 sibling, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-24  9:48 UTC (permalink / raw)
  To: Mark Rutland
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, posk

On Fri, Jan 21, 2022 at 04:57:29PM +0000, Mark Rutland wrote:
> On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:
> > User Managed Concurrency Groups is an M:N threading toolkit that allows
> > constructing user space schedulers designed to efficiently manage
> > heterogeneous in-process workloads while maintaining high CPU
> > utilization (95%+).
> > 
> > XXX moar changelog explaining how this is moar awesome than
> > traditional user-space threading.
> 
> Awaiting a commit message that I can parse, I'm just looking at the entry bits
> for now. TBH I have no idea what this is actually trying to do...

Ha! yes.. I knew I was going to have to do that eventually :-)

It's basically a user-space scheduler that is subservient to the kernel
scheduler (hierarchical scheduling were a user task is a server for
other user tasks), where a server thread is in charge of selecting which
of it's worker threads gets to run. The original idea was that each
server only ever runs a single worker, but PeterO is currently trying to
reconsider that.

The *big* feature here, over traditional N:M scheduling, is that threads
can block, while traditional userspace threading is limited to
non-blocking system calls (and per later, page-faults).

In order to make that happen we must ovbiously hook schedule() for
these worker threads and inform userspace (the server thread) when this
happens such that it can select another worker thread to go vroom.

Meanwhile, a worker task getting woken from schedule() must not continue
running; instead it must enter the server's ready-queue and await it's
turn again. Instead of dealing with arbitrary delays deep inside the
kernel block chain, we punt and let the task complete until
return-to-user and block it there. The time between schedule() and
return-to-user is unmanaged time.

Now, since we can't readily poke at userspace memory from schedule(), we
could be holding mmap_sem etc., we pin the worker and server page on
sys-enter such that when we hit schedule() we can update state and then
unpin the pages such that page pin time is from sys-enter to first
schedule(), or sys-exit which ever comes first. This ensures the
page-pin is *short* term.

Additionally we must deal with signals :-(, the currnt approach is to
let them bust boundaries and run them as unmanaged time. UMCG userspace
can obviously control this by using pthread_sigmask() and friends.

Now, the reason for irqentry_irq_enable() is mostly #PF.  When a worker
faults and blocks we want the same things to happen.

Anyway, so workers have 3 layers of hooks:

		sys_enter
				schedule()
		sys_exit

	return-to-user

There's a bunch of paths through this:

 - sys_enter -> sys_exit:

	no blocking; nothing changes:
	  - sys_enter:
	    * pin pages

	  - sys_exit:
	    * unpin pages

 - sys_enter -> schedule() -> sys_exit:

	we did block:
	  - sys_enter:
	    * pin pages

	  - schedule():
	    * mark worker BLOCKED
	    * wake server (it will observe it's current worker !RUNNING
	      and select a new worker or idles)
	    * unpin pages

	  - sys_exit():
	    * mark worker RUNNABLE
	    * enqueue worker on server's runnable_list
	    * wake server (which will observe a new runnable task, add
	      it to whatever and if it was idle goes run, otherwise goes
	      back to sleep to let it's current worker finish)
	    * block until RUNNING

 - sys_enter -> schedule() -> sys_exit -> return_to_user:

	As above; except now we got a signal while !RUNNING. sys_exit()
	terminates and return-to-user takes over running the signal and
	on return from the signal we'll again block until RUNNING, or do
	the whole signal dance again if so required.


Does this clarify things a little?


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-21 16:57   ` Mark Rutland
  2022-01-24  9:48     ` Peter Zijlstra
@ 2022-01-24 10:03     ` Peter Zijlstra
  2022-01-24 10:07       ` Peter Zijlstra
  1 sibling, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-24 10:03 UTC (permalink / raw)
  To: Mark Rutland
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, posk

On Fri, Jan 21, 2022 at 04:57:29PM +0000, Mark Rutland wrote:

> > @@ -221,8 +227,11 @@ static inline void local_irq_disable_exi
> >   */
> >  static inline void irqentry_irq_enable(struct pt_regs *regs)
> >  {
> > -	if (!regs_irqs_disabled(regs))
> > +	if (!regs_irqs_disabled(regs)) {
> >  		local_irq_enable();
> > +		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
> > +			umcg_sys_enter(regs, -1);
> > +	}
> >  }
> 
> Perhaps it would make sense to have separate umcg_sys_enter(regs) and
> umcg_sys_enter_syscall(regs, syscallno)? Even if the former is just a wrapper,
> to make the entry/exit bits clearly correspond for all the !syscall cases?

Can do I suppose.

> Also, is the syscall case meant to nest within this, or syscall entry paths not
> supposed to call irqentry_irq_enable() ?

No nesting, syscall_ vs irqentry_. And you can't have a syscall and an
exception both be from user at the same time :-)

> >  /**
> > @@ -232,8 +241,11 @@ static inline void irqentry_irq_enable(s
> >   */
> >  static inline void irqentry_irq_disable(struct pt_regs *regs)
> >  {
> > -	if (!regs_irqs_disabled(regs))
> > +	if (!regs_irqs_disabled(regs)) {
> > +		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
> > +			umcg_sys_exit(regs);
> >  		local_irq_disable();
> > +	}
> >  }
> 
> Do the umcg_sys_{enter,exit}() calls need to happen with IRQs unmasked?

Yes; both can end up blocking.

> * If not (and this nests): for arm64 these can live in our
>   enter_from_user_mode() and exit_to_user_mode() helpers.
> 
> * If so (or this doesn't nest): for arm64 we'd need to rework our
>   local_daif_{inherit,restore,mask}() calls to handle this, though I've been
>   meaning to do that anyway to handle pseudo-NMI better.
> 
> Either way, it looks like we'd need helpers along the lines of:
> 
> | static __always_inline void umcg_enter_from_user(struct pt_regs *regs)
> | {
> | 	if (current->flags & PF_UMCG_WORKER)
> | 		umcg_sys_enter(regs, -1);
> | }
> | 
> | static __always_inline void umcg_exit_to_user(struct pt_regs *regs)
> | {
> | 	if (current->flags & PF_UMCG_WORKER)
> | 		umcg_sys_exit(regs);
> | }

Would something like:

#ifndef arch_irqentry_irq_enter
static __always_inline bool arch_irqentry_irq_enter(struct pt_regs *regs)
{
	if (!regs_irqs_disabled(regs)) {
		local_irq_enable();
		return true;
	}
	return false;
}
#endif

static __always_inline void irqentry_irq_enter(struct pt_regs *regs)
{
	if (arch_irqentry_irq_inherit(regs)) {
		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
			umcg_sys_enter(regs, -1);
	}
}

Work? Then arm64 can do:

static __always_inline bool arch_irqentry_irq_enter(struct pt_regs *regs)
{
	local_daif_inherit();
	return interrupts_enabled(regs);
}

or somesuch...


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 10:03     ` Peter Zijlstra
@ 2022-01-24 10:07       ` Peter Zijlstra
  2022-01-24 10:27         ` Mark Rutland
  0 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-24 10:07 UTC (permalink / raw)
  To: Mark Rutland
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, posk

On Mon, Jan 24, 2022 at 11:03:06AM +0100, Peter Zijlstra wrote:

> > Either way, it looks like we'd need helpers along the lines of:
> > 
> > | static __always_inline void umcg_enter_from_user(struct pt_regs *regs)
> > | {
> > | 	if (current->flags & PF_UMCG_WORKER)
> > | 		umcg_sys_enter(regs, -1);
> > | }
> > | 
> > | static __always_inline void umcg_exit_to_user(struct pt_regs *regs)
> > | {
> > | 	if (current->flags & PF_UMCG_WORKER)
> > | 		umcg_sys_exit(regs);
> > | }
> 
> Would something like:
> 
> #ifndef arch_irqentry_irq_enter
> static __always_inline bool arch_irqentry_irq_enter(struct pt_regs *regs)
> {
> 	if (!regs_irqs_disabled(regs)) {
> 		local_irq_enable();
> 		return true;
> 	}
> 	return false;
> }
> #endif
> 
> static __always_inline void irqentry_irq_enter(struct pt_regs *regs)
> {
> 	if (arch_irqentry_irq_inherit(regs)) {
> 		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
> 			umcg_sys_enter(regs, -1);
> 	}
> }
> 
> Work? Then arm64 can do:
> 
> static __always_inline bool arch_irqentry_irq_enter(struct pt_regs *regs)
> {
> 	local_daif_inherit();
> 	return interrupts_enabled(regs);
> }
> 
> or somesuch...

Ah,.. just read your other email, so your concern is about the
user_mode() thing due to ARM64 taking a different exception path for
from-user vs from-kernel ?

I don't mind too much if arm64 decides to open-code the umcg hooks, but
please do it such that's hard to forget a spot.


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 10:07       ` Peter Zijlstra
@ 2022-01-24 10:27         ` Mark Rutland
  0 siblings, 0 replies; 47+ messages in thread
From: Mark Rutland @ 2022-01-24 10:27 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, posk

On Mon, Jan 24, 2022 at 11:07:04AM +0100, Peter Zijlstra wrote:
> On Mon, Jan 24, 2022 at 11:03:06AM +0100, Peter Zijlstra wrote:
> 
> > > Either way, it looks like we'd need helpers along the lines of:
> > > 
> > > | static __always_inline void umcg_enter_from_user(struct pt_regs *regs)
> > > | {
> > > | 	if (current->flags & PF_UMCG_WORKER)
> > > | 		umcg_sys_enter(regs, -1);
> > > | }
> > > | 
> > > | static __always_inline void umcg_exit_to_user(struct pt_regs *regs)
> > > | {
> > > | 	if (current->flags & PF_UMCG_WORKER)
> > > | 		umcg_sys_exit(regs);
> > > | }
> > 
> > Would something like:
> > 
> > #ifndef arch_irqentry_irq_enter
> > static __always_inline bool arch_irqentry_irq_enter(struct pt_regs *regs)
> > {
> > 	if (!regs_irqs_disabled(regs)) {
> > 		local_irq_enable();
> > 		return true;
> > 	}
> > 	return false;
> > }
> > #endif
> > 
> > static __always_inline void irqentry_irq_enter(struct pt_regs *regs)
> > {
> > 	if (arch_irqentry_irq_inherit(regs)) {
> > 		if (user_mode(regs) && (current->flags & PF_UMCG_WORKER))
> > 			umcg_sys_enter(regs, -1);
> > 	}
> > }
> > 
> > Work? Then arm64 can do:
> > 
> > static __always_inline bool arch_irqentry_irq_enter(struct pt_regs *regs)
> > {
> > 	local_daif_inherit();
> > 	return interrupts_enabled(regs);
> > }
> > 
> > or somesuch...
> 
> Ah,.. just read your other email, so your concern is about the
> user_mode() thing due to ARM64 taking a different exception path for
> from-user vs from-kernel ?

Yup; it's two-fold:

1) We have separate vectors for entry from-user and from-kernel, and I'd like
   to avoid the conditionality (e.g. the user_mode(regs) checks) where possible. 
   Having that unconditional and explicit in the from-user code avoids
   redundant work and is much easier to see that it's correct and balanced.

   We have separate irqentry_from_user() and irqentry_from_kernel() helpers
   today for this.

2) Due to the way we nest classes of exception, on the entry path we manipulate
   the flags differently depending on which specific exception we've taken. On
   the return path we always mask everything (necessary due to the way
   exception return works architecturally).

   Luckily exceptions from-user don't nest, so those cases are simpler than
   exceptions from-kernel.

> I don't mind too much if arm64 decides to open-code the umcg hooks, but
> please do it such that's hard to forget a spot.

I'll see what I can do. :)

Thanks,
Mark.


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-21 11:47   ` Peter Zijlstra
  2022-01-21 15:18     ` Peter Zijlstra
@ 2022-01-24 13:59     ` Peter Zijlstra
  1 sibling, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-24 13:59 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk

On Fri, Jan 21, 2022 at 12:47:58PM +0100, Peter Zijlstra wrote:
> On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:
> 
> > +SYSCALL_DEFINE2(umcg_wait, u32, flags, u64, timo)
> > +{
> > +	struct task_struct *tsk = current;
> > +	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
> > +	bool worker = tsk->flags & PF_UMCG_WORKER;
> > +	int ret;
> > +
> > +	if (!self || flags)
> > +		return -EINVAL;
> > +
> > +	if (worker) {
> > +		tsk->flags &= ~PF_UMCG_WORKER;
> > +		if (timo)
> > +			return -ERANGE;
> > +	}
> > +
> > +	/* see umcg_sys_{enter,exit}() syscall exceptions */
> > +	ret = umcg_pin_pages();
> > +	if (ret)
> > +		goto unblock;
> > +
> > +	/*
> > +	 * Clear UMCG_TF_COND_WAIT *and* check state == RUNNABLE.
> > +	 */
> > +	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNABLE);
> > +	if (ret)
> > +		goto unpin;
> > +
> > +	ret = umcg_wake_next(tsk, self);
> > +	if (ret)
> > +		goto unpin;
> > +
> > +	if (worker) {
> > +		/*
> > +		 * If this fails it is possible ::next_tid is already running
> > +		 * while this task is not going to block. This violates our
> > +		 * constraints.
> > +		 *
> > +		 * That said, pretty much the only way to make this fail is by
> > +		 * force munmap()'ing things. In which case one is most welcome
> > +		 * to the pieces.
> > +		 */
> > +		ret = umcg_enqueue_and_wake(tsk);
> > +		if (ret)
> > +			goto unpin;
> > +	}
> > +
> > +	umcg_unpin_pages();
> > +
> > +	ret = umcg_wait(timo);
> > +	switch (ret) {
> > +	case 0:		/* all done */
> > +	case -EINTR:	/* umcg_notify_resume() will continue the wait */
> 
> So I was playing with the whole worker timeout thing last night and
> realized this is broken. If we get a signal while we have a timeout, the
> timeout gets lost.
> 
> I think the easiest solution is to have umcg_notify_resume() also resume
> the timeout, but the first pass of that was yuck, so I need to try
> again.
> 
> Related, by moving the whole enqueue-and-wake thing into the timeout, we
> get more 'fun' failure cases :-(

This is the best I can come up with,... but it's a hot mess :-(

Still, let me go try this.

---

--- a/include/uapi/linux/umcg.h
+++ b/include/uapi/linux/umcg.h
@@ -127,6 +127,14 @@ struct umcg_task {
 } __attribute__((packed, aligned(UMCG_TASK_ALIGN)));
 
 /**
+ * enum umcg_wait_flag - flags to pass to sys_umcg_wait
+ * @UMCG_WAIT_ENQUEUE:	Enqueue the task on runnable_workers_ptr before waiting
+ */
+enum umcg_wait_flag {
+	UMCG_WAIT_ENQUEUE	= 0x0001,
+};
+
+/**
  * enum umcg_ctl_flag - flags to pass to sys_umcg_ctl
  * @UMCG_CTL_REGISTER:   register the current task as a UMCG task
  * @UMCG_CTL_UNREGISTER: unregister the current task as a UMCG task
--- a/kernel/sched/umcg.c
+++ b/kernel/sched/umcg.c
@@ -227,7 +227,6 @@ static int umcg_update_state(struct task
 
 #define UMCG_DIE(reason)	__UMCG_DIE(,reason)
 #define UMCG_DIE_PF(reason)	__UMCG_DIE(pagefault_enable(), reason)
-#define UMCG_DIE_UNPIN(reason)	__UMCG_DIE(umcg_unpin_pages(), reason)
 
 /* Called from syscall enter path and exceptions that can schedule */
 void umcg_sys_enter(struct pt_regs *regs, long syscall)
@@ -371,15 +370,23 @@ static int umcg_enqueue_runnable(struct
 
 static int umcg_enqueue_and_wake(struct task_struct *tsk)
 {
-	int ret;
-
-	ret = umcg_enqueue_runnable(tsk);
+	int ret = umcg_enqueue_runnable(tsk);
 	if (!ret)
 		ret = umcg_wake_server(tsk);
 
 	return ret;
 }
 
+static int umcg_pin_enqueue_and_wake(struct task_struct *tsk)
+{
+	int ret = umcg_pin_pages();
+	if (!ret) {
+		ret = umcg_enqueue_and_wake(tsk);
+		umcg_unpin_pages();
+	}
+	return ret;
+}
+
 /*
  * umcg_wait: Wait for ->state to become RUNNING
  *
@@ -469,16 +476,11 @@ static void umcg_unblock_and_wait(void)
 	/* avoid recursion vs schedule() */
 	tsk->flags &= ~PF_UMCG_WORKER;
 
-	if (umcg_pin_pages())
-		UMCG_DIE("pin");
-
 	if (umcg_update_state(tsk, self, UMCG_TASK_BLOCKED, UMCG_TASK_RUNNABLE))
-		UMCG_DIE_UNPIN("state");
+		UMCG_DIE("state");
 
-	if (umcg_enqueue_and_wake(tsk))
-		UMCG_DIE_UNPIN("enqueue-wake");
-
-	umcg_unpin_pages();
+	if (umcg_pin_enqueue_and_wake(tsk))
+		UMCG_DIE("pin-enqueue-wake");
 
 	switch (umcg_wait(0)) {
 	case 0:
@@ -544,18 +546,13 @@ void umcg_notify_resume(struct pt_regs *
 		goto done;
 
 	if (state & UMCG_TF_PREEMPT) {
-		if (umcg_pin_pages())
-			UMCG_DIE("pin");
-
 		if (umcg_update_state(tsk, self,
 				      UMCG_TASK_RUNNING,
 				      UMCG_TASK_RUNNABLE))
-			UMCG_DIE_UNPIN("state");
+			UMCG_DIE("state");
 
-		if (umcg_enqueue_and_wake(tsk))
-			UMCG_DIE_UNPIN("enqueue-wake");
-
-		umcg_unpin_pages();
+		if (umcg_pin_enqueue_and_wake(tsk))
+			UMCG_DIE("pin-enqueue-wake");
 	}
 
 	if (WARN_ON_ONCE(timeout && syscall_get_nr(tsk, regs) != __NR_umcg_wait))
@@ -570,6 +567,13 @@ void umcg_notify_resume(struct pt_regs *
 
 	case -ETIMEDOUT:
 		regs_set_return_value(regs, ret);
+		if (worker) {
+			ret = umcg_pin_enqueue_and_wake(tsk);
+			if (ret) {
+				umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNING);
+				regs_set_return_value(regs, ret);
+			}
+		}
 		break;
 
 	default:
@@ -710,7 +714,6 @@ static int umcg_wake_next(struct task_st
  * Returns:
  * 0		- OK;
  * -ETIMEDOUT	- the timeout expired;
- * -ERANGE	- the timeout is out of range (worker);
  * -EAGAIN	- ::state wasn't RUNNABLE, concurrent wakeup;
  * -EFAULT	- failed accessing struct umcg_task __user of the current
  *		  task, the server or next;
@@ -725,48 +728,40 @@ SYSCALL_DEFINE2(umcg_wait, u32, flags, u
 	bool worker = tsk->flags & PF_UMCG_WORKER;
 	int ret;
 
-	if (!self || flags)
+	if (!self || (flags & ~(UMCG_WAIT_ENQUEUE)))
 		return -EINVAL;
 
-	if (worker) {
-		tsk->flags &= ~PF_UMCG_WORKER;
-		if (timo)
-			return -ERANGE;
-	}
+	if ((flags & UMCG_WAIT_ENQUEUE) && (timo || !worker))
+		return -EINVAL;
 
-	/* see umcg_sys_{enter,exit}() syscall exceptions */
-	ret = umcg_pin_pages();
-	if (ret)
-		goto unblock;
+	if (worker)
+		tsk->flags &= ~PF_UMCG_WORKER;
 
 	/*
 	 * Clear UMCG_TF_COND_WAIT *and* check state == RUNNABLE.
 	 */
 	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNABLE);
 	if (ret)
-		goto unpin;
+		goto unblock;
 
 	ret = umcg_wake_next(tsk, self);
 	if (ret)
-		goto unpin;
+		goto unblock;
 
-	if (worker) {
+	if (flags & UMCG_WAIT_ENQUEUE) {
 		/*
 		 * If this fails it is possible ::next_tid is already running
 		 * while this task is not going to block. This violates our
 		 * constraints.
 		 *
-		 * That said, pretty much the only way to make this fail is by
-		 * force munmap()'ing things. In which case one is most welcome
-		 * to the pieces.
+		 * Userspace can detect this case by looking at: ::next_tid &
+		 * TID_RUNNING.
 		 */
-		ret = umcg_enqueue_and_wake(tsk);
+		ret = umcg_pin_enqueue_and_wake(tsk);
 		if (ret)
-			goto unpin;
+			goto unblock;
 	}
 
-	umcg_unpin_pages();
-
 	ret = umcg_wait(timo);
 	switch (ret) {
 	case 0:		/* all done */
@@ -775,6 +770,26 @@ SYSCALL_DEFINE2(umcg_wait, u32, flags, u
 		ret = 0;
 		break;
 
+	case -ETIMEDOUT:
+		if (worker) {
+			/*
+			 * See the UMCG_WAIT_ENQUEUE case above; except this is
+			 * even more complicated because we *did* wait and
+			 * things might have progressed lots.
+			 *
+			 * Still, abort the wait because otherwise nobody would
+			 * ever find us again. Hopefully userspace can make
+			 * sense of things.
+			 */
+			ret = umcg_pin_enqueue_and_wake(tsk);
+			if (ret)
+				goto unblock;
+
+			ret = -ETIMEDOUT;
+			break;
+		}
+		goto unblock;
+
 	default:
 		goto unblock;
 	}
@@ -783,8 +798,6 @@ SYSCALL_DEFINE2(umcg_wait, u32, flags, u
 		tsk->flags |= PF_UMCG_WORKER;
 	return ret;
 
-unpin:
-	umcg_unpin_pages();
 unblock:
 	umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_RUNNING);
 	goto out;


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-21 15:18     ` Peter Zijlstra
@ 2022-01-24 14:29       ` Peter Zijlstra
  2022-01-24 16:44         ` Peter Zijlstra
  2022-01-25 14:59         ` Peter Zijlstra
  0 siblings, 2 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-24 14:29 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk

On Fri, Jan 21, 2022 at 04:18:46PM +0100, Peter Zijlstra wrote:
> Something like this, still yuck though. Also still need to write me a
> test for this.
> 
> --- a/include/linux/sched.h
> +++ b/include/linux/sched.h
> @@ -1300,12 +1300,14 @@ struct task_struct {
>  	clockid_t		umcg_clock;
>  	struct umcg_task __user	*umcg_task;
>  
> -	/* setup by umcg_pin_enter() */
> +	/* setup by umcg_pin_pages() */
>  	struct page		*umcg_page;
>  
>  	struct task_struct	*umcg_server;
>  	struct umcg_task __user *umcg_server_task;
>  	struct page		*umcg_server_page;
> +
> +	u64			umcg_timeout;
>  #endif
>  
>  	struct tlbflush_unmap_batch	tlb_ubc;
> --- a/kernel/sched/umcg.c
> +++ b/kernel/sched/umcg.c
> @@ -232,6 +232,8 @@ static int umcg_update_state(struct task
>  /* Called from syscall enter path and exceptions that can schedule */
>  void umcg_sys_enter(struct pt_regs *regs, long syscall)
>  {
> +	current->umcg_timeout = 0;
> +
>  	/* avoid recursion vs our own syscalls */
>  	if (syscall == __NR_umcg_wait ||
>  	    syscall == __NR_umcg_ctl)
> @@ -519,6 +521,7 @@ void umcg_notify_resume(struct pt_regs *
>  	struct umcg_task __user *self = tsk->umcg_task;
>  	bool worker = tsk->flags & PF_UMCG_WORKER;
>  	u32 state;
> +	int ret;
>  
>  	/* avoid recursion vs schedule() */
>  	if (worker)
> @@ -554,12 +557,17 @@ void umcg_notify_resume(struct pt_regs *
>  		umcg_unpin_pages();
>  	}
>  
> -	switch (umcg_wait(0)) {
> +	ret = umcg_wait(tsk->umcg_timeout);

Oh how I hate signals... this can get scribbled by a syscall/fault from
sigcontext :/

Maybe I can recover the timo argument from the original syscall
pt_regs.. let me try.

> +	switch (ret) {
>  	case 0:
>  	case -EINTR:
>  		/* we will resume the wait after the signal */
>  		break;
>  
> +	case -ETIMEDOUT:
> +		regs_set_return_value(regs, ret);
> +		break;
> +
>  	default:
>  		UMCG_DIE("wait");
>  	}
> @@ -759,6 +767,7 @@ SYSCALL_DEFINE2(umcg_wait, u32, flags, u
>  	switch (ret) {
>  	case 0:		/* all done */
>  	case -EINTR:	/* umcg_notify_resume() will continue the wait */
> +		tsk->umcg_timeout = timo;
>  		ret = 0;
>  		break;
>  


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
                     ` (2 preceding siblings ...)
  2022-01-21 16:57   ` Mark Rutland
@ 2022-01-24 14:46   ` Tao Zhou
  2022-01-27 12:19     ` Peter Zijlstra
                       ` (2 more replies)
  2022-01-27 18:31   ` Tao Zhou
  4 siblings, 3 replies; 47+ messages in thread
From: Tao Zhou @ 2022-01-24 14:46 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk, Tao Zhou

Hi Peter,

On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:

[...]

> +/* pre-schedule() */
> +void umcg_wq_worker_sleeping(struct task_struct *tsk)
> +{
> +	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
> +	int ret;
> +
> +	if (!tsk->umcg_server) {
> +		/*
> +		 * Already blocked before, the pages are unpinned.
> +		 */
> +		return;
> +	}
> +
> +	/* Must not fault, mmap_sem might be held. */
> +	pagefault_disable();
> +
> +	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNING, UMCG_TASK_BLOCKED);
> +	if (ret == -EAGAIN) {
> +		/*
> +		 * Consider:
> +		 *
> +		 *   self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
> +		 *   ...
> +		 *   sys_umcg_wait();
> +		 *
> +		 * and the '...' code doing a blocking syscall/fault. This
> +		 * ensures that returns with UMCG_TASK_RUNNING, which will make

/UMCG_TASK_RUNNING/UMCG_TASK_RUNNABLE/

> +		 * sys_umcg_wait() return with -EAGAIN.
> +		 */
> +		ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_BLOCKED);
> +	}
> +	if (ret)
> +		UMCG_DIE_PF("state");
> +
> +	if (umcg_wake_server(tsk))
> +		UMCG_DIE_PF("wake");
> +
> +	pagefault_enable();
> +
> +	/*
> +	 * We're going to sleep, make sure to unpin the pages, this ensures
> +	 * the pins are temporary. Also see umcg_sys_exit().
> +	 */
> +	umcg_unpin_pages();
> +}

[...]

> +/* Called from syscall exit path and exceptions that can schedule */
> +void umcg_sys_exit(struct pt_regs *regs)
> +{
> +	struct task_struct *tsk = current;
> +	long syscall = syscall_get_nr(tsk, regs);
> +
> +	if (syscall == __NR_umcg_wait ||
> +	    syscall == __NR_umcg_ctl)
> +		return;
> +
> +	if (tsk->umcg_server) {
> +		/*
> +		 * Didn't block, we done.
> +		 */
> +		umcg_unpin_pages();
> +		return;
> +	}
> +
> +	umcg_unblock_and_wait();

umcg_unblock_and_wait() -> umcg_enqueue_and_wake() ->
umcg_wake_server() -> umcg_wake_task(tsk->umcg_server, ...)

tsk->umcg_server is NULL here and umcg_wake_task() use it to update
state in umcg_update_state(NULL, ...), that means tsk->umcg_clock
will happen something i do not know.

There are two places to call umcg_unblock_and_wait(). One is in 
umcg_register() where the server is set. Another one is in
umcg_sys_exit() where the server is not set. May use a bool to
indicate if the server is set.

> +}

[...]

> +/**
> + * sys_umcg_wait: transfer running context
> + *
> + * Called like:
> + *
> + *	self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
> + *	...
> + *	sys_umcg_wait(0, time);
> + *
> + * The syscall will clear TF_COND_WAIT and wait until state becomes RUNNING.
> + * The code '...' must not contain syscalls
> + *
> + * If self->next_tid is set and indicates a valid UMCG task with RUNNABLE state
> + * that task will be made RUNNING and woken -- transfering the running context
> + * to that task. In this case self->next_tid is modified with TID_RUNNING to
> + * indicate self->next_tid is consumed.
> + *
> + * If self->next has TID_RUNNING set, it is validated the related task has

/self->next/self->next_tid/
Things are not clear to me even they are clear now. Nice.

Thanks,
Tao


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 14:29       ` Peter Zijlstra
@ 2022-01-24 16:44         ` Peter Zijlstra
  2022-01-24 17:06           ` Peter Oskolkov
  2022-01-25 14:59         ` Peter Zijlstra
  1 sibling, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-24 16:44 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk

On Mon, Jan 24, 2022 at 03:29:56PM +0100, Peter Zijlstra wrote:
> On Fri, Jan 21, 2022 at 04:18:46PM +0100, Peter Zijlstra wrote:
> > Something like this, still yuck though. Also still need to write me a
> > test for this.
> > 

> > --- a/kernel/sched/umcg.c
> > +++ b/kernel/sched/umcg.c
> > @@ -232,6 +232,8 @@ static int umcg_update_state(struct task
> >  /* Called from syscall enter path and exceptions that can schedule */
> >  void umcg_sys_enter(struct pt_regs *regs, long syscall)
> >  {
> > +	current->umcg_timeout = 0;
> > +
> >  	/* avoid recursion vs our own syscalls */
> >  	if (syscall == __NR_umcg_wait ||
> >  	    syscall == __NR_umcg_ctl)
> > @@ -519,6 +521,7 @@ void umcg_notify_resume(struct pt_regs *
> >  	struct umcg_task __user *self = tsk->umcg_task;
> >  	bool worker = tsk->flags & PF_UMCG_WORKER;
> >  	u32 state;
> > +	int ret;
> >  
> >  	/* avoid recursion vs schedule() */
> >  	if (worker)
> > @@ -554,12 +557,17 @@ void umcg_notify_resume(struct pt_regs *
> >  		umcg_unpin_pages();
> >  	}
> >  
> > -	switch (umcg_wait(0)) {
> > +	ret = umcg_wait(tsk->umcg_timeout);
> 
> Oh how I hate signals... this can get scribbled by a syscall/fault from
> sigcontext :/
> 
> Maybe I can recover the timo argument from the original syscall
> pt_regs.. let me try.

Urgh, recursive hell... If the signal does *anything* that tickles
notify-resume it'll find RUNNABLE and go wait there --- ad infinitum.

I need to go cook dinner, I'll prod more at this later


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 16:44         ` Peter Zijlstra
@ 2022-01-24 17:06           ` Peter Oskolkov
  0 siblings, 0 replies; 47+ messages in thread
From: Peter Oskolkov @ 2022-01-24 17:06 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, avagin, jannh, tdelisle, mark.rutland, posk

On Mon, Jan 24, 2022 at 8:44 AM Peter Zijlstra <peterz@infradead.org> wrote:

[...]
> >
> > Oh how I hate signals... this can get scribbled by a syscall/fault from
> > sigcontext :/
> >
> > Maybe I can recover the timo argument from the original syscall
> > pt_regs.. let me try.

I don't think we need to do anything special with timeouts if a signal
happens - just normally return -EINTR (after a resume from a server)
and let the userspace figure things out.

Alternatively, in my version of the patchset UMCG tasks ignore
non-fatal signals. :)

[...]


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 14:29       ` Peter Zijlstra
  2022-01-24 16:44         ` Peter Zijlstra
@ 2022-01-25 14:59         ` Peter Zijlstra
  1 sibling, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-25 14:59 UTC (permalink / raw)
  To: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot
  Cc: linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk

On Mon, Jan 24, 2022 at 03:29:56PM +0100, Peter Zijlstra wrote:

> Oh how I hate signals... this can get scribbled by a syscall/fault from
> sigcontext :/

OK, the below seems to work. I'll see if I can clean it up some.

--- a/arch/x86/include/asm/syscall.h
+++ b/arch/x86/include/asm/syscall.h
@@ -94,28 +94,44 @@ static inline int syscall_get_arch(struc
 
 #else	 /* CONFIG_X86_64 */
 
-static inline void syscall_get_arguments(struct task_struct *task,
-					 struct pt_regs *regs,
-					 unsigned long *args)
+static inline unsigned long
+syscall_get_argument(struct task_struct *task, struct pt_regs *regs, int nr)
 {
-# ifdef CONFIG_IA32_EMULATION
+#ifdef CONFIG_IA32_EMULATION
 	if (task->thread_info.status & TS_COMPAT) {
-		*args++ = regs->bx;
-		*args++ = regs->cx;
-		*args++ = regs->dx;
-		*args++ = regs->si;
-		*args++ = regs->di;
-		*args   = regs->bp;
+		switch (nr) {
+		case 0: return regs->bx;
+		case 1: return regs->cx;
+		case 2: return regs->dx;
+		case 3: return regs->si;
+		case 4: return regs->di;
+		case 5: return regs->bp;
+		}
 	} else
-# endif
+#endif
 	{
-		*args++ = regs->di;
-		*args++ = regs->si;
-		*args++ = regs->dx;
-		*args++ = regs->r10;
-		*args++ = regs->r8;
-		*args   = regs->r9;
+		switch (nr) {
+		case 0: return regs->di;
+		case 1: return regs->si;
+		case 2: return regs->dx;
+		case 3: return regs->r10;
+		case 4: return regs->r8;
+		case 5: return regs->r9;
+		}
 	}
+
+	WARN_ON_ONCE(1);
+	return 0;
+}
+
+static inline void syscall_get_arguments(struct task_struct *task,
+					 struct pt_regs *regs,
+					 unsigned long *args)
+{
+	int i;
+
+	for (i = 0; i < 6; i++)
+		*args++ = syscall_get_argument(task, regs, i);
 }
 
 static inline int syscall_get_arch(struct task_struct *task)
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1307,6 +1307,9 @@ struct task_struct {
 	struct task_struct	*umcg_server;
 	struct umcg_task __user *umcg_server_task;
 	struct page		*umcg_server_page;
+
+	unsigned long		umcg_stack_pointer;
+	unsigned int		umcg_worker;
 #endif
 
 	struct tlbflush_unmap_batch	tlb_ubc;
--- a/kernel/sched/umcg.c
+++ b/kernel/sched/umcg.c
@@ -459,7 +459,7 @@ static int umcg_wait(u64 timo)
 /*
  * Blocked case for umcg_sys_exit(), shared with sys_umcg_ctl().
  */
-static void umcg_unblock_and_wait(void)
+static void umcg_unblock(void)
 {
 	struct task_struct *tsk = current;
 	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
@@ -478,15 +478,7 @@ static void umcg_unblock_and_wait(void)
 
 	umcg_unpin_pages();
 
-	switch (umcg_wait(0)) {
-	case 0:
-	case -EINTR:
-		/* notify_resume will continue the wait after the signal */
-		break;
-
-	default:
-		UMCG_DIE("wait");
-	}
+	/* notify-resume will wait */
 
 	tsk->flags |= PF_UMCG_WORKER;
 }
@@ -509,7 +501,7 @@ void umcg_sys_exit(struct pt_regs *regs)
 		return;
 	}
 
-	umcg_unblock_and_wait();
+	umcg_unblock();
 }
 
 /* return-to-user path */
@@ -518,11 +510,47 @@ void umcg_notify_resume(struct pt_regs *
 	struct task_struct *tsk = current;
 	struct umcg_task __user *self = tsk->umcg_task;
 	bool worker = tsk->flags & PF_UMCG_WORKER;
+	u64 timeout = 0;
 	u32 state;
+	int ret;
+
+	/*
+	 * Unix signals are horrible, but we have to handle them somehow.
+	 *
+	 * - simply discarding a signal breaks userspace so is not an option.
+	 *
+	 * - returning -EINTR and have userspace deal with it is not an option
+	 *   since we can be blocked here due to !syscall reasons (page-faults
+	 *   for example). But it's also not permissible to have random
+	 *   syscalls return -EINTR that didn't before.
+	 *
+	 * - subjecting signal handlers to UMCG would render existing signal
+	 *   handler code subject to the whims and latencies of UMCG; given that
+	 *   most signal hander code is short and time sensitive, this seems
+	 *   undesirable (consider ^C not working because it got delivered to a
+	 *   blocked task).
+	 *
+	 * Therefore the chosen path is to exclude signal context from UMCG
+	 * entirely and treat it as unmanaged time.
+	 */
+	if (tsk->umcg_stack_pointer) {
+		if (tsk->umcg_stack_pointer != user_stack_pointer(regs))
+			return;
+
+		tsk->umcg_stack_pointer = 0;
+		worker = tsk->umcg_worker;
+		tsk->umcg_worker = 0;
+
+		if (worker) {
+			set_syscall_work(SYSCALL_UMCG);
+			/* and PF_UMCG_SYSCALL at done */
+		}
+		goto resume;
+	}
 
 	/* avoid recursion vs schedule() */
 	if (worker)
-		current->flags &= ~PF_UMCG_WORKER;
+		tsk->flags &= ~PF_UMCG_WORKER;
 
 	if (get_user(state, &self->state))
 		UMCG_DIE("get-state");
@@ -554,10 +582,31 @@ void umcg_notify_resume(struct pt_regs *
 		umcg_unpin_pages();
 	}
 
-	switch (umcg_wait(0)) {
+resume:
+	/*
+	 * Hack alert! Since the return-to-user path must resume waiting it
+	 * needs access to the timeout argument and set the return value.
+	 */
+	if (syscall_get_nr(tsk, regs) == __NR_umcg_wait)
+		timeout = syscall_get_argument(tsk, regs, 1);
+
+	ret = umcg_wait(timeout);
+	switch (ret) {
 	case 0:
+		break;
+
 	case -EINTR:
 		/* we will resume the wait after the signal */
+		WARN_ON_ONCE(tsk->umcg_stack_pointer);
+		tsk->umcg_stack_pointer = user_stack_pointer(regs);
+		tsk->umcg_worker = worker;
+		clear_task_syscall_work(tsk, SYSCALL_UMCG);
+		/* implicitly clears PF_UMCG_WORKER with the early exit */
+		return;
+
+	case -ETIMEDOUT:
+		/* must be __NR_umcg_wait */
+		regs_set_return_value(regs, ret);
 		break;
 
 	default:
@@ -566,7 +615,7 @@ void umcg_notify_resume(struct pt_regs *
 
 done:
 	if (worker)
-		current->flags |= PF_UMCG_WORKER;
+		tsk->flags |= PF_UMCG_WORKER;
 }
 
 /**
@@ -755,16 +804,7 @@ SYSCALL_DEFINE2(umcg_wait, u32, flags, u
 
 	umcg_unpin_pages();
 
-	ret = umcg_wait(timo);
-	switch (ret) {
-	case 0:		/* all done */
-	case -EINTR:	/* umcg_notify_resume() will continue the wait */
-		ret = 0;
-		break;
-
-	default:
-		goto unblock;
-	}
+	/* notify-resume will wait */
 out:
 	if (worker)
 		tsk->flags |= PF_UMCG_WORKER;
@@ -831,7 +871,7 @@ static int umcg_register(struct umcg_tas
 		set_syscall_work(SYSCALL_UMCG);		/* hook syscall */
 		set_thread_flag(TIF_UMCG);		/* hook return-to-user */
 
-		umcg_unblock_and_wait();
+		umcg_unblock();
 
 	} else {
 		if ((ut.state & (UMCG_TASK_MASK | UMCG_TF_MASK)) != UMCG_TASK_RUNNING)


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

* Re: [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-20 15:55 ` [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user() Peter Zijlstra
@ 2022-01-27  2:17   ` Sean Christopherson
  2022-01-27  6:36     ` Sean Christopherson
  2022-01-27  9:55     ` Peter Zijlstra
  0 siblings, 2 replies; 47+ messages in thread
From: Sean Christopherson @ 2022-01-27  2:17 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Thu, Jan 20, 2022, Peter Zijlstra wrote:
> Do try_cmpxchg() loops on userspace addresses.
> 
> Cc: Sean Christopherson <seanjc@google.com>
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> ---
>  arch/x86/include/asm/uaccess.h |   67 +++++++++++++++++++++++++++++++++++++++++
>  1 file changed, 67 insertions(+)
> 
> --- a/arch/x86/include/asm/uaccess.h
> +++ b/arch/x86/include/asm/uaccess.h
> @@ -342,6 +342,24 @@ do {									\
>  		     : [umem] "m" (__m(addr))				\
>  		     : : label)
>  
> +#define __try_cmpxchg_user_asm(itype, ltype, _ptr, _pold, _new, label)	({ \
> +	bool success;							\
> +	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
> +	__typeof__(*(_ptr)) __old = *_old;				\
> +	__typeof__(*(_ptr)) __new = (_new);				\
> +	asm_volatile_goto("\n"						\
> +		     "1: " LOCK_PREFIX "cmpxchg"itype" %[new], %[ptr]\n"\
> +		     _ASM_EXTABLE_UA(1b, %l[label])			\
> +		     : CC_OUT(z) (success),				\
> +		       [ptr] "+m" (*_ptr),				\
> +		       [old] "+a" (__old)				\
> +		     : [new] ltype (__new)				\
> +		     : "memory", "cc"					\

IIUC, the "cc" clobber is unnecessary as CONFIG_CC_HAS_ASM_GOTO_OUTPUT=y implies
__GCC_ASM_FLAG_OUTPUTS__=y, i.e. CC_OUT() will resolve to "=@cc".

> +		     : label);						\
> +	if (unlikely(!success))						\
> +		*_old = __old;						\
> +	likely(success);					})
> +
>  #else // !CONFIG_CC_HAS_ASM_GOTO_OUTPUT

...

> +extern void __try_cmpxchg_user_wrong_size(void);
> +
> +#define unsafe_try_cmpxchg_user(_ptr, _oldp, _nval, _label) ({		\
> +	__typeof__(*(_ptr)) __ret;					\

This should probably be a bool, the return from the lower level helpers is a bool
that's true if the exchange succeed.  Declaring the type of the target implies
that they return the raw result, which is confusing.

> +	switch (sizeof(__ret)) {					\
> +	case 1:	__ret = __try_cmpxchg_user_asm("b", "q",		\
> +					       (_ptr), (_oldp),		\
> +					       (_nval), _label);	\
> +		break;							\
> +	case 2:	__ret = __try_cmpxchg_user_asm("w", "r",		\
> +					       (_ptr), (_oldp),		\
> +					       (_nval), _label);	\
> +		break;							\
> +	case 4:	__ret = __try_cmpxchg_user_asm("l", "r",		\
> +					       (_ptr), (_oldp),		\
> +					       (_nval), _label);	\
> +		break;							\
> +	case 8:	__ret = __try_cmpxchg_user_asm("q", "r",		\
> +					       (_ptr), (_oldp),		\
> +					       (_nval), _label);	\

Doh, I should have specified that KVM needs 8-byte CMPXCHG on 32-bit kernels due
to using it to atomically update guest PAE PTEs and LTR descriptors (yay).

Also, KVM's use case isn't a tight loop, how gross would it be to add a slightly
less unsafe version that does __uaccess_begin_nospec()?  KVM pre-checks the address
way ahead of time, so the access_ok() check can be omitted.  Alternatively, KVM
could add its own macro, but that seems a little silly.  E.g. somethign like this,
though I don't think this is correct (something is getting inverted somewhere and
the assembly output is a nightmare):

/* "Returns" 0 on success, 1 on failure, -EFAULT if the access faults. */
#define ___try_cmpxchg_user(_ptr, _oldp, _nval, _label)	({		\
	int ____ret = -EFAULT;						\
	__uaccess_begin_nospec();					\
	____ret = !unsafe_try_cmpxchg_user(_ptr, _oldp, _nval, _label);	\
_label:									\
	__uaccess_end();						\
	____ret;							\
						})

Lastly, assuming I get my crap working, mind if I post a variant (Cc'd to stable@) in
the context of KVM series?  Turns out KVM has an ugly bug where it completely
botches the pfn calculation of memory it remaps and accesses[*], the easiest fix
is to switch to __try_cmpxchg_user() and purge the nastiness.

[*] https://lore.kernel.org/all/20220124172633.103323-1-tadeusz.struk@linaro.org


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

* Re: [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-27  2:17   ` Sean Christopherson
@ 2022-01-27  6:36     ` Sean Christopherson
  2022-01-27  9:56       ` Peter Zijlstra
  2022-01-27  9:55     ` Peter Zijlstra
  1 sibling, 1 reply; 47+ messages in thread
From: Sean Christopherson @ 2022-01-27  6:36 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Thu, Jan 27, 2022, Sean Christopherson wrote:
> Doh, I should have specified that KVM needs 8-byte CMPXCHG on 32-bit kernels due
> to using it to atomically update guest PAE PTEs and LTR descriptors (yay).
> 
> Also, KVM's use case isn't a tight loop, how gross would it be to add a slightly
> less unsafe version that does __uaccess_begin_nospec()?  KVM pre-checks the address
> way ahead of time, so the access_ok() check can be omitted.  Alternatively, KVM
> could add its own macro, but that seems a little silly.  E.g. somethign like this,
> though I don't think this is correct

*sigh*

Finally realized I forgot to add back the page offset after converting from guest
page frame to host virtual address.  Anyways, this is what I ended up with, will
test more tomorrow.

From: Peter Zijlstra <peterz@infradead.org>
Date: Thu, 20 Jan 2022 16:55:21 +0100
Subject: [PATCH] x86/uaccess: Implement unsafe_try_cmpxchg_user()

Do try_cmpxchg() loops on userspace addresses.  Provide 8-byte versions
for 32-bit kernels so that KVM can do cmpxchg on guest PAE PTEs, which
are accessed via userspace addresses.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
Co-developed-by: Sean Christopherson <seanjc@google.com>
Signed-off-by: Sean Christopherson <seanjc@google.com>
---
 arch/x86/include/asm/uaccess.h | 129 +++++++++++++++++++++++++++++++++
 1 file changed, 129 insertions(+)

diff --git a/arch/x86/include/asm/uaccess.h b/arch/x86/include/asm/uaccess.h
index ac96f9b2d64b..b706008aed28 100644
--- a/arch/x86/include/asm/uaccess.h
+++ b/arch/x86/include/asm/uaccess.h
@@ -342,6 +342,45 @@ do {									\
 		     : [umem] "m" (__m(addr))				\
 		     : : label)

+#define __try_cmpxchg_user_asm(itype, ltype, _ptr, _pold, _new, label)	({ \
+	bool success;							\
+	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
+	__typeof__(*(_ptr)) __old = *_old;				\
+	__typeof__(*(_ptr)) __new = (_new);				\
+	asm_volatile_goto("\n"						\
+		     "1: " LOCK_PREFIX "cmpxchg"itype" %[new], %[ptr]\n"\
+		     _ASM_EXTABLE_UA(1b, %l[label])			\
+		     : CC_OUT(z) (success),				\
+		       [ptr] "+m" (*_ptr),				\
+		       [old] "+a" (__old)				\
+		     : [new] ltype (__new)				\
+		     : "memory"						\
+		     : label);						\
+	if (unlikely(!success))						\
+		*_old = __old;						\
+	likely(success);					})
+
+#ifdef CONFIG_X86_32
+#define __try_cmpxchg64_user_asm(_ptr, _pold, _new, label)	({ \
+	bool success;							\
+	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
+	__typeof__(*(_ptr)) __old = *_old;				\
+	__typeof__(*(_ptr)) __new = (_new);				\
+	asm_volatile_goto("\n"						\
+		     "1: " LOCK_PREFIX "cmpxchg8b %[ptr]\n"		\
+		     _ASM_EXTABLE_UA(1b, %l[label])			\
+		     : CC_OUT(z) (success),				\
+		       "+A" (__old),					\
+		       [ptr] "+m" (*_ptr)				\
+		     : "b" ((u32)__new),				\
+		       "c" ((u32)((u64)__new >> 32))			\
+		     : "memory"						\
+		     : label);						\
+	if (unlikely(!success))						\
+		*_old = __old;						\
+	likely(success);					})
+#endif // CONFIG_X86_32
+
 #else // !CONFIG_CC_HAS_ASM_GOTO_OUTPUT

 #ifdef CONFIG_X86_32
@@ -407,6 +446,57 @@ do {									\
 		     : [umem] "m" (__m(addr)),				\
 		       "0" (err))

+#define __try_cmpxchg_user_asm(itype, ltype, _ptr, _pold, _new, label)	({ \
+	int __err = 0;							\
+	bool success;							\
+	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
+	__typeof__(*(_ptr)) __old = *_old;				\
+	__typeof__(*(_ptr)) __new = (_new);				\
+	asm volatile("\n"						\
+		     "1: " LOCK_PREFIX "cmpxchg"itype" %[new], %[ptr]\n"\
+		     CC_SET(z)						\
+		     "2:\n"						\
+		     _ASM_EXTABLE_TYPE_REG(1b, 2b, EX_TYPE_EFAULT_REG,	\
+					   %[errout])			\
+		     : CC_OUT(z) (success),				\
+		       [errout] "+r" (__err),				\
+		       [ptr] "+m" (*_ptr),				\
+		       [old] "+a" (__old)				\
+		     : [new] ltype (__new)				\
+		     : "memory", "cc");					\
+	if (unlikely(__err))						\
+		goto label;						\
+	if (unlikely(!success))						\
+		*_old = __old;						\
+	likely(success);					})
+
+#ifdef CONFIG_X86_32
+#define __try_cmpxchg64_user_asm(_ptr, _pold, _new, label)	({	\
+	int __err = 0;							\
+	bool success;							\
+	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
+	__typeof__(*(_ptr)) __old = *_old;				\
+	__typeof__(*(_ptr)) __new = (_new);				\
+	asm volatile("\n"						\
+		     "1: " LOCK_PREFIX "cmpxchg8b %[ptr]\n"		\
+		     CC_SET(z)						\
+		     "2:\n"						\
+		     _ASM_EXTABLE_TYPE_REG(1b, 2b, EX_TYPE_EFAULT_REG,	\
+					   %[errout])			\
+		     : CC_OUT(z) (success),				\
+		       [errout] "+r" (__err),				\
+		       "+A" (__old),					\
+		       [ptr] "+m" (*_ptr)				\
+		     : "b" ((u32)__new),				\
+		       "c" ((u32)((u64)__new >> 32))			\
+		     : "memory", "cc");					\
+	if (unlikely(__err))						\
+		goto label;						\
+	if (unlikely(!success))						\
+		*_old = __old;						\
+	likely(success);					})
+#endif // CONFIG_X86_32
+
 #endif // CONFIG_CC_HAS_ASM_GOTO_OUTPUT

 /* FIXME: this hack is definitely wrong -AK */
@@ -501,6 +591,45 @@ do {										\
 } while (0)
 #endif // CONFIG_CC_HAS_ASM_GOTO_OUTPUT

+extern void __try_cmpxchg_user_wrong_size(void);
+
+#ifndef CONFIG_X86_32
+#define __try_cmpxchg64_user_asm(_ptr, _oldp, _nval, _label)		\
+	__try_cmpxchg_user_asm("q", "r", (_ptr), (_oldp), (_nval), _label)
+#endif
+
+#define unsafe_try_cmpxchg_user(_ptr, _oldp, _nval, _label) ({		\
+	bool __ret;							\
+	switch (sizeof(*(_ptr))) {					\
+	case 1:	__ret = __try_cmpxchg_user_asm("b", "q",		\
+					       (_ptr), (_oldp),		\
+					       (_nval), _label);	\
+		break;							\
+	case 2:	__ret = __try_cmpxchg_user_asm("w", "r",		\
+					       (_ptr), (_oldp),		\
+					       (_nval), _label);	\
+		break;							\
+	case 4:	__ret = __try_cmpxchg_user_asm("l", "r",		\
+					       (_ptr), (_oldp),		\
+					       (_nval), _label);	\
+		break;							\
+	case 8:	__ret = __try_cmpxchg64_user_asm((_ptr), (_oldp),	\
+						 (_nval), _label);	\
+		break;							\
+	default: __try_cmpxchg_user_wrong_size();			\
+	}								\
+	__ret;						})
+
+/* "Returns" 0 on success, 1 on failure, -EFAULT if the access faults. */
+#define __try_cmpxchg_user(_ptr, _oldp, _nval, _label)	({		\
+	int __ret = -EFAULT;						\
+	__uaccess_begin_nospec();					\
+	__ret = !unsafe_try_cmpxchg_user(_ptr, _oldp, _nval, _label);	\
+_label:									\
+	__uaccess_end();						\
+	__ret;								\
+							})
+
 /*
  * We want the unsafe accessors to always be inlined and use
  * the error labels - thus the macro games.
--




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

* Re: [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-27  2:17   ` Sean Christopherson
  2022-01-27  6:36     ` Sean Christopherson
@ 2022-01-27  9:55     ` Peter Zijlstra
  1 sibling, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-27  9:55 UTC (permalink / raw)
  To: Sean Christopherson
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Thu, Jan 27, 2022 at 02:17:20AM +0000, Sean Christopherson wrote:
> On Thu, Jan 20, 2022, Peter Zijlstra wrote:
> > Do try_cmpxchg() loops on userspace addresses.
> > 
> > Cc: Sean Christopherson <seanjc@google.com>
> > Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> > ---
> >  arch/x86/include/asm/uaccess.h |   67 +++++++++++++++++++++++++++++++++++++++++
> >  1 file changed, 67 insertions(+)
> > 
> > --- a/arch/x86/include/asm/uaccess.h
> > +++ b/arch/x86/include/asm/uaccess.h
> > @@ -342,6 +342,24 @@ do {									\
> >  		     : [umem] "m" (__m(addr))				\
> >  		     : : label)
> >  
> > +#define __try_cmpxchg_user_asm(itype, ltype, _ptr, _pold, _new, label)	({ \
> > +	bool success;							\
> > +	__typeof__(_ptr) _old = (__typeof__(_ptr))(_pold);		\
> > +	__typeof__(*(_ptr)) __old = *_old;				\
> > +	__typeof__(*(_ptr)) __new = (_new);				\
> > +	asm_volatile_goto("\n"						\
> > +		     "1: " LOCK_PREFIX "cmpxchg"itype" %[new], %[ptr]\n"\
> > +		     _ASM_EXTABLE_UA(1b, %l[label])			\
> > +		     : CC_OUT(z) (success),				\
> > +		       [ptr] "+m" (*_ptr),				\
> > +		       [old] "+a" (__old)				\
> > +		     : [new] ltype (__new)				\
> > +		     : "memory", "cc"					\
> 
> IIUC, the "cc" clobber is unnecessary as CONFIG_CC_HAS_ASM_GOTO_OUTPUT=y implies
> __GCC_ASM_FLAG_OUTPUTS__=y, i.e. CC_OUT() will resolve to "=@cc".

Yeah, even without that GCC always assumes 'cc' is clobbered due to
hysterical raisins.

> > +		     : label);						\
> > +	if (unlikely(!success))						\
> > +		*_old = __old;						\
> > +	likely(success);					})
> > +
> >  #else // !CONFIG_CC_HAS_ASM_GOTO_OUTPUT
> 
> ...
> 
> > +extern void __try_cmpxchg_user_wrong_size(void);
> > +
> > +#define unsafe_try_cmpxchg_user(_ptr, _oldp, _nval, _label) ({		\
> > +	__typeof__(*(_ptr)) __ret;					\
> 
> This should probably be a bool, the return from the lower level helpers is a bool
> that's true if the exchange succeed.  Declaring the type of the target implies
> that they return the raw result, which is confusing.

Fair enough.

> > +	switch (sizeof(__ret)) {					\
> > +	case 1:	__ret = __try_cmpxchg_user_asm("b", "q",		\
> > +					       (_ptr), (_oldp),		\
> > +					       (_nval), _label);	\
> > +		break;							\
> > +	case 2:	__ret = __try_cmpxchg_user_asm("w", "r",		\
> > +					       (_ptr), (_oldp),		\
> > +					       (_nval), _label);	\
> > +		break;							\
> > +	case 4:	__ret = __try_cmpxchg_user_asm("l", "r",		\
> > +					       (_ptr), (_oldp),		\
> > +					       (_nval), _label);	\
> > +		break;							\
> > +	case 8:	__ret = __try_cmpxchg_user_asm("q", "r",		\
> > +					       (_ptr), (_oldp),		\
> > +					       (_nval), _label);	\
> 
> Doh, I should have specified that KVM needs 8-byte CMPXCHG on 32-bit kernels due
> to using it to atomically update guest PAE PTEs and LTR descriptors (yay).

:-) I'm so trying to de-feature 32bit.

> Also, KVM's use case isn't a tight loop, how gross would it be to add a slightly
> less unsafe version that does __uaccess_begin_nospec()?  KVM pre-checks the address
> way ahead of time, so the access_ok() check can be omitted.  Alternatively, KVM
> could add its own macro, but that seems a little silly.  E.g. somethign like this,
> though I don't think this is correct (something is getting inverted somewhere and
> the assembly output is a nightmare):
> 
> /* "Returns" 0 on success, 1 on failure, -EFAULT if the access faults. */
> #define ___try_cmpxchg_user(_ptr, _oldp, _nval, _label)	({		\
> 	int ____ret = -EFAULT;						\
> 	__uaccess_begin_nospec();					\
> 	____ret = !unsafe_try_cmpxchg_user(_ptr, _oldp, _nval, _label);	\
> _label:									\
> 	__uaccess_end();						\
> 	____ret;							\
> 						})

Works for me I suppose, but we really ought to keep usage of that in
arch code.

> Lastly, assuming I get my crap working, mind if I post a variant (Cc'd to stable@) in
> the context of KVM series?  

Not at all.


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

* Re: [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-27  6:36     ` Sean Christopherson
@ 2022-01-27  9:56       ` Peter Zijlstra
  2022-01-27 23:33         ` Sean Christopherson
  0 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-27  9:56 UTC (permalink / raw)
  To: Sean Christopherson
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Thu, Jan 27, 2022 at 06:36:19AM +0000, Sean Christopherson wrote:
> On Thu, Jan 27, 2022, Sean Christopherson wrote:
> > Doh, I should have specified that KVM needs 8-byte CMPXCHG on 32-bit kernels due
> > to using it to atomically update guest PAE PTEs and LTR descriptors (yay).
> > 
> > Also, KVM's use case isn't a tight loop, how gross would it be to add a slightly
> > less unsafe version that does __uaccess_begin_nospec()?  KVM pre-checks the address
> > way ahead of time, so the access_ok() check can be omitted.  Alternatively, KVM
> > could add its own macro, but that seems a little silly.  E.g. somethign like this,
> > though I don't think this is correct
> 
> *sigh*
> 
> Finally realized I forgot to add back the page offset after converting from guest
> page frame to host virtual address.  Anyways, this is what I ended up with, will
> test more tomorrow.

Looks about right :-) (famous last words etc..)


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 14:46   ` Tao Zhou
@ 2022-01-27 12:19     ` Peter Zijlstra
  2022-01-27 18:33       ` Tao Zhou
  2022-01-27 12:25     ` Peter Zijlstra
  2022-01-27 12:26     ` Peter Zijlstra
  2 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-27 12:19 UTC (permalink / raw)
  To: Tao Zhou
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Mon, Jan 24, 2022 at 10:46:17PM +0800, Tao Zhou wrote:
> Hi Peter,
> 
> On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:
> 
> [...]
> 
> > +/* pre-schedule() */
> > +void umcg_wq_worker_sleeping(struct task_struct *tsk)
> > +{
> > +	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
> > +	int ret;
> > +
> > +	if (!tsk->umcg_server) {
> > +		/*
> > +		 * Already blocked before, the pages are unpinned.
> > +		 */
> > +		return;
> > +	}
> > +
> > +	/* Must not fault, mmap_sem might be held. */
> > +	pagefault_disable();
> > +
> > +	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNING, UMCG_TASK_BLOCKED);
> > +	if (ret == -EAGAIN) {
> > +		/*
> > +		 * Consider:
> > +		 *
> > +		 *   self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
> > +		 *   ...
> > +		 *   sys_umcg_wait();
> > +		 *
> > +		 * and the '...' code doing a blocking syscall/fault. This
> > +		 * ensures that returns with UMCG_TASK_RUNNING, which will make
> 
> /UMCG_TASK_RUNNING/UMCG_TASK_RUNNABLE/

So the issue is that:

	self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;

	<#PF>
	  umcg_sys_enter()
	    umcg_pin_user_page()
	  schedule()
	    sched_submit_work()
	      umcg_wq_worker_sleeping()
	        umcg_update_state(tsk, self, UMCG_TASK_RUNNING, UMCG_TASK_BLOCKED) // -EAGAIN
		UMCG_DIE()

Which is clearly not desirable.

So this additinoal thing ensures that:

		umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_BLOCKED) // 0

	  umcg_sys_exit()
	    umcg_update_state(tsk, self, UMCG_TASK_BLOCKED, UMCG_TASK_RUNNABLE);
	    umcg_enqueue_and_wake()

	  umcg_notify_resume()
	    umcg_wait()

	// must be UMCG_TASK_RUNNING here
	</#PF>

So when the pagefault finally does return, it will have:
UMCG_TASK_RUNNING.

Which will then make sys_umcg_wait() return -EAGAIN and around we go.

> > +		 * sys_umcg_wait() return with -EAGAIN.
> > +		 */
> > +		ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_BLOCKED);
> > +	}
> > +	if (ret)
> > +		UMCG_DIE_PF("state");
> > +
> > +	if (umcg_wake_server(tsk))
> > +		UMCG_DIE_PF("wake");
> > +
> > +	pagefault_enable();
> > +
> > +	/*
> > +	 * We're going to sleep, make sure to unpin the pages, this ensures
> > +	 * the pins are temporary. Also see umcg_sys_exit().
> > +	 */
> > +	umcg_unpin_pages();
> > +}


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 14:46   ` Tao Zhou
  2022-01-27 12:19     ` Peter Zijlstra
@ 2022-01-27 12:25     ` Peter Zijlstra
  2022-01-27 18:47       ` Tao Zhou
  2022-01-27 12:26     ` Peter Zijlstra
  2 siblings, 1 reply; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-27 12:25 UTC (permalink / raw)
  To: Tao Zhou
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Mon, Jan 24, 2022 at 10:46:17PM +0800, Tao Zhou wrote:

> > +/* Called from syscall exit path and exceptions that can schedule */
> > +void umcg_sys_exit(struct pt_regs *regs)
> > +{
> > +	struct task_struct *tsk = current;
> > +	long syscall = syscall_get_nr(tsk, regs);
> > +
> > +	if (syscall == __NR_umcg_wait ||
> > +	    syscall == __NR_umcg_ctl)
> > +		return;
> > +
> > +	if (tsk->umcg_server) {
> > +		/*
> > +		 * Didn't block, we done.
> > +		 */
> > +		umcg_unpin_pages();
> > +		return;
> > +	}
> > +
> > +	umcg_unblock_and_wait();
> 
> umcg_unblock_and_wait() -> umcg_enqueue_and_wake() ->
> umcg_wake_server() -> umcg_wake_task(tsk->umcg_server, ...)
> 
> tsk->umcg_server is NULL here and umcg_wake_task() use it to update
> state in umcg_update_state(NULL, ...), that means tsk->umcg_clock
> will happen something i do not know.

I think umcg_unblock_and_wait() will repin, at which point we should
have tsk->umcg_server again.

> There are two places to call umcg_unblock_and_wait(). One is in 
> umcg_register() where the server is set. Another one is in
> umcg_sys_exit() where the server is not set. May use a bool to
> indicate if the server is set.

I'm not sure what you're on about, but I absolutely hate redundant
state, that only leads to problems.



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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-24 14:46   ` Tao Zhou
  2022-01-27 12:19     ` Peter Zijlstra
  2022-01-27 12:25     ` Peter Zijlstra
@ 2022-01-27 12:26     ` Peter Zijlstra
  2 siblings, 0 replies; 47+ messages in thread
From: Peter Zijlstra @ 2022-01-27 12:26 UTC (permalink / raw)
  To: Tao Zhou
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk

On Mon, Jan 24, 2022 at 10:46:17PM +0800, Tao Zhou wrote:
> > +/**
> > + * sys_umcg_wait: transfer running context
> > + *
> > + * Called like:
> > + *
> > + *	self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
> > + *	...
> > + *	sys_umcg_wait(0, time);
> > + *
> > + * The syscall will clear TF_COND_WAIT and wait until state becomes RUNNING.
> > + * The code '...' must not contain syscalls
> > + *
> > + * If self->next_tid is set and indicates a valid UMCG task with RUNNABLE state
> > + * that task will be made RUNNING and woken -- transfering the running context
> > + * to that task. In this case self->next_tid is modified with TID_RUNNING to
> > + * indicate self->next_tid is consumed.
> > + *
> > + * If self->next has TID_RUNNING set, it is validated the related task has
> 
> /self->next/self->next_tid/

Yeah, there's more of that, I'll be sure to go re-read all the comments.



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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
                     ` (3 preceding siblings ...)
  2022-01-24 14:46   ` Tao Zhou
@ 2022-01-27 18:31   ` Tao Zhou
  4 siblings, 0 replies; 47+ messages in thread
From: Tao Zhou @ 2022-01-27 18:31 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk, Tao Zhou

On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:

Another iterator of the patch, some nits below.

[...]

> +/*
> + * Pinning a page inhibits rmap based unmap for Anon pages. Doing a load
> + * through the user mapping ensures the user mapping exists.
> + */
> +#define umcg_pin_and_load(_self, _pagep, _member)				\
> +({										\
> +	__label__ __out;							\
> +	int __ret = -EFAULT;							\
> +										\
> +	if (pin_user_pages_fast((unsigned long)(_self), 1, 0, &(_pagep)) != 1)	\
> +		goto __out;							\
> +										\
> +	if (!PageAnon(_pagep) ||						\
> +	    get_user(_member, &(_self)->_member)) {				\

Here should be 'get_user(_member, &(_self)->##_member))'
if I am not wrong.

> +		unpin_user_page(_pagep);					\
> +		goto __out;							\
> +	}									\
> +	__ret = 0;								\
> +__out:	__ret;									\
> +})

[...]

> +
> +/*
> + * Enqueue @tsk on it's server's runnable list
> + *
> + * Must be called in umcg_pin_pages() context, relies on tsk->umcg_server.
> + *
> + * cmpxchg based single linked list add such that list integrity is never
> + * violated.  Userspace *MUST* remove it from the list before changing ->state.
> + * As such, we must change state to RUNNABLE before enqueue.
> + *
> + * Returns:
> + *   0: success
> + *   -EFAULT
> + */
> +static int umcg_enqueue_runnable(struct task_struct *tsk)
> +{
> +	struct umcg_task __user *server = tsk->umcg_server_task;
> +	struct umcg_task __user *self = tsk->umcg_task;
> +	u64 first_ptr, *head = &server->runnable_workers_ptr;
> +	u64 self_ptr = (unsigned long)self;

Why not 'u64 self_ptr = (u64)self;' ?

> +	/*
> +	 * umcg_pin_pages() did access_ok() on both pointers, use self here
> +	 * only because __user_access_begin() isn't available in generic code.
> +	 */
> +	if (!user_access_begin(self, sizeof(*self)))
> +		return -EFAULT;
> +
> +	unsafe_get_user(first_ptr, head, Efault);
> +	do {
> +		unsafe_put_user(first_ptr, &self->runnable_workers_ptr, Efault);
> +	} while (!unsafe_try_cmpxchg_user(head, &first_ptr, self_ptr, Efault));
> +
> +	user_access_end();
> +	return 0;
> +
> +Efault:
> +	user_access_end();
> +	return -EFAULT;
> +}

[...]

> +/*
> + * Handles ::next_tid as per sys_umcg_wait().
> + *
> + * ::next_tid		- return
> + * -----------------------------
> + * 0			- 0 (success)
> + *
> + * tid			- -ESRCH (no such task, or not of this UMCG)
> + *			- -EAGAIN (next::state != RUNNABLE)
> + *			- 0 (success, ::next_tid |= RUNNING)
> + *
> + * tid|RUNNING		- -EAGAIN (next::state != RUNNING)
> + *			- 0 (success)
> + *
> + * Returns:
> + *  0: success
> + *  -EFAULT
> + *  -ESRCH
> + *  -EAGAIN
> + */
> +static int umcg_wake_next(struct task_struct *tsk, struct umcg_task __user *self)

@tsk is not used in function.

> +{
> +	struct umcg_task __user *next_task;
> +	struct task_struct *next;
> +	u32 next_tid, state;
> +	int ret;
> +
> +	if (get_user(next_tid, &self->next_tid))
> +		return -EFAULT;
> +
> +	if (!next_tid)
> +		return 0;
> +
> +	next = umcg_get_task(next_tid);
> +	if (!next)
> +		return -ESRCH;
> +
> +	next_task = READ_ONCE(next->umcg_task);
> +
> +	if (next_tid & UMCG_TID_RUNNING) {
> +		ret = -EFAULT;
> +		if (get_user(state, &next_task->state))
> +			goto put_next;
> +
> +		ret = 0;
> +		if ((state & UMCG_TASK_MASK) != UMCG_TASK_RUNNING)
> +			ret = -EAGAIN;
> +
> +	} else {
> +		ret = umcg_wake_task(next, next_task);
> +		if (ret)
> +			goto put_next;
> +
> +		ret = -EFAULT;
> +		if (put_user(next_tid | UMCG_TID_RUNNING, &self->next_tid))
> +			goto put_next;
> +
> +		/*
> +		 * If this is a worker doing sys_umcg_wait() switching to
> +		 * another worker, userspace has the responsibility to update
> +		 * server::next_tid.
> +		 */
> +
> +		ret = 0;
> +	}
> +
> +put_next:
> +	put_task_struct(next);
> +	return ret;
> +}
> +


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-27 12:19     ` Peter Zijlstra
@ 2022-01-27 18:33       ` Tao Zhou
  0 siblings, 0 replies; 47+ messages in thread
From: Tao Zhou @ 2022-01-27 18:33 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk, Tao Zhou

On Thu, Jan 27, 2022 at 01:19:43PM +0100, Peter Zijlstra wrote:

> On Mon, Jan 24, 2022 at 10:46:17PM +0800, Tao Zhou wrote:
> > Hi Peter,
> > 
> > On Thu, Jan 20, 2022 at 04:55:22PM +0100, Peter Zijlstra wrote:
> > 
> > [...]
> > 
> > > +/* pre-schedule() */
> > > +void umcg_wq_worker_sleeping(struct task_struct *tsk)
> > > +{
> > > +	struct umcg_task __user *self = READ_ONCE(tsk->umcg_task);
> > > +	int ret;
> > > +
> > > +	if (!tsk->umcg_server) {
> > > +		/*
> > > +		 * Already blocked before, the pages are unpinned.
> > > +		 */
> > > +		return;
> > > +	}
> > > +
> > > +	/* Must not fault, mmap_sem might be held. */
> > > +	pagefault_disable();
> > > +
> > > +	ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNING, UMCG_TASK_BLOCKED);
> > > +	if (ret == -EAGAIN) {
> > > +		/*
> > > +		 * Consider:
> > > +		 *
> > > +		 *   self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
> > > +		 *   ...
> > > +		 *   sys_umcg_wait();
> > > +		 *
> > > +		 * and the '...' code doing a blocking syscall/fault. This
> > > +		 * ensures that returns with UMCG_TASK_RUNNING, which will make
> > 
> > /UMCG_TASK_RUNNING/UMCG_TASK_RUNNABLE/
> 
> So the issue is that:
> 
> 	self->state = UMCG_TASK_RUNNABLE | UMCG_TF_COND_WAIT;
> 
> 	<#PF>
> 	  umcg_sys_enter()
> 	    umcg_pin_user_page()
> 	  schedule()
> 	    sched_submit_work()
> 	      umcg_wq_worker_sleeping()
> 	        umcg_update_state(tsk, self, UMCG_TASK_RUNNING, UMCG_TASK_BLOCKED) // -EAGAIN
> 		UMCG_DIE()
> 
> Which is clearly not desirable.
> 
> So this additinoal thing ensures that:
> 
> 		umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_BLOCKED) // 0
> 
> 	  umcg_sys_exit()
> 	    umcg_update_state(tsk, self, UMCG_TASK_BLOCKED, UMCG_TASK_RUNNABLE);
> 	    umcg_enqueue_and_wake()
> 
> 	  umcg_notify_resume()
> 	    umcg_wait()
> 
> 	// must be UMCG_TASK_RUNNING here
> 	</#PF>
> 
> So when the pagefault finally does return, it will have:
> UMCG_TASK_RUNNING.
> 
> Which will then make sys_umcg_wait() return -EAGAIN and around we go.

Thank you, Peter.

> > > +		 * sys_umcg_wait() return with -EAGAIN.
> > > +		 */
> > > +		ret = umcg_update_state(tsk, self, UMCG_TASK_RUNNABLE, UMCG_TASK_BLOCKED);
> > > +	}
> > > +	if (ret)
> > > +		UMCG_DIE_PF("state");
> > > +
> > > +	if (umcg_wake_server(tsk))
> > > +		UMCG_DIE_PF("wake");
> > > +
> > > +	pagefault_enable();
> > > +
> > > +	/*
> > > +	 * We're going to sleep, make sure to unpin the pages, this ensures
> > > +	 * the pins are temporary. Also see umcg_sys_exit().
> > > +	 */
> > > +	umcg_unpin_pages();
> > > +}


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

* Re: [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups
  2022-01-27 12:25     ` Peter Zijlstra
@ 2022-01-27 18:47       ` Tao Zhou
  0 siblings, 0 replies; 47+ messages in thread
From: Tao Zhou @ 2022-01-27 18:47 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk, Tao Zhou

On Thu, Jan 27, 2022 at 01:25:36PM +0100, Peter Zijlstra wrote:

> On Mon, Jan 24, 2022 at 10:46:17PM +0800, Tao Zhou wrote:
> 
> > > +/* Called from syscall exit path and exceptions that can schedule */
> > > +void umcg_sys_exit(struct pt_regs *regs)
> > > +{
> > > +	struct task_struct *tsk = current;
> > > +	long syscall = syscall_get_nr(tsk, regs);
> > > +
> > > +	if (syscall == __NR_umcg_wait ||
> > > +	    syscall == __NR_umcg_ctl)
> > > +		return;
> > > +
> > > +	if (tsk->umcg_server) {
> > > +		/*
> > > +		 * Didn't block, we done.
> > > +		 */
> > > +		umcg_unpin_pages();
> > > +		return;
> > > +	}
> > > +
> > > +	umcg_unblock_and_wait();
> > 
> > umcg_unblock_and_wait() -> umcg_enqueue_and_wake() ->
> > umcg_wake_server() -> umcg_wake_task(tsk->umcg_server, ...)
> > 
> > tsk->umcg_server is NULL here and umcg_wake_task() use it to update
> > state in umcg_update_state(NULL, ...), that means tsk->umcg_clock
> > will happen something i do not know.
> 
> I think umcg_unblock_and_wait() will repin, at which point we should
> have tsk->umcg_server again.

That's right, I miss that.

> > There are two places to call umcg_unblock_and_wait(). One is in 
> > umcg_register() where the server is set. Another one is in
> > umcg_sys_exit() where the server is not set. May use a bool to
> > indicate if the server is set.
> 
> I'm not sure what you're on about, but I absolutely hate redundant
> state, that only leads to problems.


But, it's my noise though.


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

* Re: [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-27  9:56       ` Peter Zijlstra
@ 2022-01-27 23:33         ` Sean Christopherson
  2022-01-28  0:17           ` Nick Desaulniers
  0 siblings, 1 reply; 47+ messages in thread
From: Sean Christopherson @ 2022-01-27 23:33 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, tglx, juri.lelli, vincent.guittot, dietmar.eggemann,
	rostedt, bsegall, mgorman, bristot, linux-kernel, linux-mm,
	linux-api, x86, pjt, posk, avagin, jannh, tdelisle, mark.rutland,
	posk, Nick Desaulniers

+Nick

On Thu, Jan 27, 2022, Peter Zijlstra wrote:
> On Thu, Jan 27, 2022 at 06:36:19AM +0000, Sean Christopherson wrote:
> > On Thu, Jan 27, 2022, Sean Christopherson wrote:
> > > Doh, I should have specified that KVM needs 8-byte CMPXCHG on 32-bit kernels due
> > > to using it to atomically update guest PAE PTEs and LTR descriptors (yay).
> > > 
> > > Also, KVM's use case isn't a tight loop, how gross would it be to add a slightly
> > > less unsafe version that does __uaccess_begin_nospec()?  KVM pre-checks the address
> > > way ahead of time, so the access_ok() check can be omitted.  Alternatively, KVM
> > > could add its own macro, but that seems a little silly.  E.g. somethign like this,
> > > though I don't think this is correct
> > 
> > *sigh*
> > 
> > Finally realized I forgot to add back the page offset after converting from guest
> > page frame to host virtual address.  Anyways, this is what I ended up with, will
> > test more tomorrow.
> 
> Looks about right :-) (famous last words etc..)

And it was right, but clang-13 ruined the party :-/

clang barfs on asm goto with a "+m" input/output.  Change the "+m" to "=m" and
clang is happy.  Remove usage of the label, clang is happy.

I tried a bunch of different variants to see if anything would squeak by, but
clang found a way to die on everything I threw at it.

$ clang --version

  Debian clang version 13.0.0-9+build1
  Target: x86_64-pc-linux-gnu
  Thread model: posix
  InstalledDir: /usr/bin

As written, with a named label param, clang yields:

  $ echo 'int foo(int *x) { asm goto (".long (%l[bar]) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null
  <stdin>:1:29: error: invalid operand in inline asm: '.long (${1:l}) - .'
  int foo(int *x) { asm goto (".long (%l[bar]) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }
                            ^
  <stdin>:1:29: error: unknown token in expression
  <inline asm>:1:9: note: instantiated into assembly here
          .long () - .
               ^
  2 errors generated.

While clang is perfectly happy switching "+m" to "=m":

  $ echo 'int foo(int *x) { asm goto (".long (%l[bar]) - .\n": "=m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null

Referencing the label with a numbered param yields either the original error:

  $ echo 'int foo(int *x) { asm goto (".long (%l1) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null
  <stdin>:1:29: error: invalid operand in inline asm: '.long (${1:l}) - .'
  int foo(int *x) { asm goto (".long (%l1) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }
                            ^
  <stdin>:1:29: error: unknown token in expression
  <inline asm>:1:9: note: instantiated into assembly here
          .long () - .
                 ^
   2 errors generated.

Bumping the param number (more below) yields a different error (I tried defining
tmp1, that didn't work :-) ).

  $ echo 'int foo(int *x) { asm goto (".long (%l2) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null
  error: Undefined temporary symbol .Ltmp1
  1 error generated.

Regarding the param number, gcc also appears to have a goof with asm goto and "+m",
but bumping the param number in that case remedies its woes.

  $echo 'int foo(int *x) { asm goto (".long (%l1) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | gcc -x c - -c -o /dev/null
  <stdin>: In function ‘foo’:
  <stdin>:1:19: error: invalid 'asm': '%l' operand isn't a label

  $ echo 'int foo(int *x) { asm goto (".long (%l2) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | gcc -x c - -c -o /dev/null


So my immediate question: how do we want to we deal with this in the kernel?  Keeping
in mind that I'd really like to send this to stable@ to fix the KVM mess.

I can think of few options that are varying degrees of gross.

  1) Use a more complex sequence for probing CC_HAS_ASM_GOTO_OUTPUT.

  2) Use an output-only "=m" operand.

  3) Use an input register param.

Option #1 has the obvious downside of the fancier asm goto for  __get_user_asm()
and friends being collateral damage.  The biggest benefit is it'd reduce the
likelihood of someone else having to debug similar errors, which was quite painful.

Options #2 and #3 are quite gross, but I _think_ would be ok since the sequence
is tagged as clobbering memory anyways?


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

* Re: [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-27 23:33         ` Sean Christopherson
@ 2022-01-28  0:17           ` Nick Desaulniers
  2022-01-28 16:29             ` Sean Christopherson
  0 siblings, 1 reply; 47+ messages in thread
From: Nick Desaulniers @ 2022-01-28  0:17 UTC (permalink / raw)
  To: Sean Christopherson
  Cc: Peter Zijlstra, mingo, tglx, juri.lelli, vincent.guittot,
	dietmar.eggemann, rostedt, bsegall, mgorman, bristot,
	linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk, James Y Knight, llvm

On Thu, Jan 27, 2022 at 3:33 PM Sean Christopherson <seanjc@google.com> wrote:
>
> +Nick

(well, you asked the right person; you probably wont like the answer
much though)

>
> On Thu, Jan 27, 2022, Peter Zijlstra wrote:
> > On Thu, Jan 27, 2022 at 06:36:19AM +0000, Sean Christopherson wrote:
> > > On Thu, Jan 27, 2022, Sean Christopherson wrote:
> > > > Doh, I should have specified that KVM needs 8-byte CMPXCHG on 32-bit kernels due
> > > > to using it to atomically update guest PAE PTEs and LTR descriptors (yay).
> > > >
> > > > Also, KVM's use case isn't a tight loop, how gross would it be to add a slightly
> > > > less unsafe version that does __uaccess_begin_nospec()?  KVM pre-checks the address
> > > > way ahead of time, so the access_ok() check can be omitted.  Alternatively, KVM
> > > > could add its own macro, but that seems a little silly.  E.g. somethign like this,
> > > > though I don't think this is correct
> > >
> > > *sigh*
> > >
> > > Finally realized I forgot to add back the page offset after converting from guest
> > > page frame to host virtual address.  Anyways, this is what I ended up with, will
> > > test more tomorrow.
> >
> > Looks about right :-) (famous last words etc..)
>
> And it was right, but clang-13 ruined the party :-/
>
> clang barfs on asm goto with a "+m" input/output.  Change the "+m" to "=m" and
> clang is happy.  Remove usage of the label, clang is happy.

Yep, sorry, we only recently noticed that this was broken.  I fixed
this very recently (over the holidays) in clang-14, and is too risky
and late to backport to clang-13 IMO.
https://reviews.llvm.org/rG4edb9983cb8c8b850083ee5941468f5d0ef6fe2c

>
> I tried a bunch of different variants to see if anything would squeak by, but
> clang found a way to die on everything I threw at it.
>
> $ clang --version
>
>   Debian clang version 13.0.0-9+build1
>   Target: x86_64-pc-linux-gnu
>   Thread model: posix
>   InstalledDir: /usr/bin
>
> As written, with a named label param, clang yields:
>
>   $ echo 'int foo(int *x) { asm goto (".long (%l[bar]) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null
>   <stdin>:1:29: error: invalid operand in inline asm: '.long (${1:l}) - .'
>   int foo(int *x) { asm goto (".long (%l[bar]) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }
>                             ^
>   <stdin>:1:29: error: unknown token in expression
>   <inline asm>:1:9: note: instantiated into assembly here
>           .long () - .
>                ^
>   2 errors generated.
>
> While clang is perfectly happy switching "+m" to "=m":
>
>   $ echo 'int foo(int *x) { asm goto (".long (%l[bar]) - .\n": "=m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null

= constraints should work with older clang releases; + constraints are
what was broken (Not generally, only when using asm goto with
outputs), fixed in clang-14.

>
> Referencing the label with a numbered param yields either the original error:
>
>   $ echo 'int foo(int *x) { asm goto (".long (%l1) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null
>   <stdin>:1:29: error: invalid operand in inline asm: '.long (${1:l}) - .'
>   int foo(int *x) { asm goto (".long (%l1) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }
>                             ^
>   <stdin>:1:29: error: unknown token in expression
>   <inline asm>:1:9: note: instantiated into assembly here
>           .long () - .
>                  ^
>    2 errors generated.

^ That case should not work in either compilers, more info below...

>
> Bumping the param number (more below) yields a different error (I tried defining
> tmp1, that didn't work :-) ).
>
>   $ echo 'int foo(int *x) { asm goto (".long (%l2) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | clang -x c - -c -o /dev/null
>   error: Undefined temporary symbol .Ltmp1
>   1 error generated.

"Bumping the param number" will be required when using numbered
references, more info below...

>
> Regarding the param number, gcc also appears to have a goof with asm goto and "+m",
> but bumping the param number in that case remedies its woes.
>
>   $echo 'int foo(int *x) { asm goto (".long (%l1) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | gcc -x c - -c -o /dev/null
>   <stdin>: In function ‘foo’:
>   <stdin>:1:19: error: invalid 'asm': '%l' operand isn't a label
>
>   $ echo 'int foo(int *x) { asm goto (".long (%l2) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | gcc -x c - -c -o /dev/null

Right, so in fixing the above issue with tied outputs, I noticed that
the GCC implementation of asm goto with outputs had different
behavior. I changed clang's implementation in clang-14 (same patch
series) to match:
https://reviews.llvm.org/rG5c562f62a4ee15592f5d764d0710553a4b07a6f2
This comment summarizes most of my thoughts on the issue:
https://gcc.gnu.org/bugzilla/show_bug.cgi?id=98096#c7
Specifically the quote "It appears to me that the GCC decision here
was accidental, and that when pointed out, the bug was simply
documented rather than fixed."
Though I think compatibility between compilers is ultimately more
important.  There's no standards bodies involved in these extension,
which is simultaneously more flexible, yet can also lead to
differences in implementations like this. Thanks for attending my TED
talk.

>
>
> So my immediate question: how do we want to we deal with this in the kernel?  Keeping
> in mind that I'd really like to send this to stable@ to fix the KVM mess.
>
> I can think of few options that are varying degrees of gross.
>
>   1) Use a more complex sequence for probing CC_HAS_ASM_GOTO_OUTPUT.
>
>   2) Use an output-only "=m" operand.
>
>   3) Use an input register param.
>
> Option #1 has the obvious downside of the fancier asm goto for  __get_user_asm()
> and friends being collateral damage.  The biggest benefit is it'd reduce the
> likelihood of someone else having to debug similar errors, which was quite painful.

Right; I assumed we'd hit this at some point, as soon as people wanted
to used tied outputs with asm goto.  I'd rather have a different
Kconfig test for working tied outputs, and that all uses in the
kernels used the symbolic references which are much more readable and
less confusing than the rules for numbered references (which are bug
prone IMO).

>
> Options #2 and #3 are quite gross, but I _think_ would be ok since the sequence
> is tagged as clobbering memory anyways?

-- 
Thanks,
~Nick Desaulniers


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

* Re: [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user()
  2022-01-28  0:17           ` Nick Desaulniers
@ 2022-01-28 16:29             ` Sean Christopherson
  0 siblings, 0 replies; 47+ messages in thread
From: Sean Christopherson @ 2022-01-28 16:29 UTC (permalink / raw)
  To: Nick Desaulniers
  Cc: Peter Zijlstra, mingo, tglx, juri.lelli, vincent.guittot,
	dietmar.eggemann, rostedt, bsegall, mgorman, bristot,
	linux-kernel, linux-mm, linux-api, x86, pjt, posk, avagin, jannh,
	tdelisle, mark.rutland, posk, James Y Knight, llvm

On Thu, Jan 27, 2022, Nick Desaulniers wrote:
> On Thu, Jan 27, 2022 at 3:33 PM Sean Christopherson <seanjc@google.com> wrote:
> > Regarding the param number, gcc also appears to have a goof with asm goto and "+m",
> > but bumping the param number in that case remedies its woes.
> >
> >   $echo 'int foo(int *x) { asm goto (".long (%l1) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | gcc -x c - -c -o /dev/null
> >   <stdin>: In function ‘foo’:
> >   <stdin>:1:19: error: invalid 'asm': '%l' operand isn't a label
> >
> >   $ echo 'int foo(int *x) { asm goto (".long (%l2) - .\n": "+m"(*x) ::: bar); return *x; bar: return 0; }' | gcc -x c - -c -o /dev/null
> 
> Right, so in fixing the above issue with tied outputs, I noticed that
> the GCC implementation of asm goto with outputs had different
> behavior. I changed clang's implementation in clang-14 (same patch
> series) to match:
> https://reviews.llvm.org/rG5c562f62a4ee15592f5d764d0710553a4b07a6f2
> This comment summarizes most of my thoughts on the issue:
> https://gcc.gnu.org/bugzilla/show_bug.cgi?id=98096#c7
> Specifically the quote "It appears to me that the GCC decision here
> was accidental, and that when pointed out, the bug was simply
> documented rather than fixed."

I guess that makes a certain amount of sense, but wow that is subtle, confusing,
and potentially dangerous.  Looks like the hidden inputs are numbered after all
explicit inputs, otherwise there would be broken code left and right, but it means
that a typo like so:

  echo 'int foo(int x) { asm ("xor %0, %0; xor %2, %2" : "+a"(x) : "b"(x)); return x; return 0; }' | clang -x c - -c -o tmp.o

will compile cleanly.

> Though I think compatibility between compilers is ultimately more
> important.  There's no standards bodies involved in these extension,
> which is simultaneously more flexible, yet can also lead to
> differences in implementations like this. Thanks for attending my TED
> talk.
> 
> >
> >
> > So my immediate question: how do we want to we deal with this in the kernel?  Keeping
> > in mind that I'd really like to send this to stable@ to fix the KVM mess.
> >
> > I can think of few options that are varying degrees of gross.
> >
> >   1) Use a more complex sequence for probing CC_HAS_ASM_GOTO_OUTPUT.
> >
> >   2) Use an output-only "=m" operand.
> >
> >   3) Use an input register param.
> >
> > Option #1 has the obvious downside of the fancier asm goto for  __get_user_asm()
> > and friends being collateral damage.  The biggest benefit is it'd reduce the
> > likelihood of someone else having to debug similar errors, which was quite painful.
> 
> Right; I assumed we'd hit this at some point, as soon as people wanted
> to used tied outputs with asm goto.  I'd rather have a different
> Kconfig test for working tied outputs, 

Is it all tied outputs, or just "+m"?  E.g. using "+r" compiles cleanly.

  echo 'int foo(int x) { asm goto ("xor %0, %0;.long (%l[bar]) - .\n": "+r"(x) ::: bar); return x; bar: return 0; }' | clang -x c - -c -o /dev/null

It might be a moot point as I can't find any instances of "+"-anything in conjunction
with asm_volatile_goto, i.e. adding CC_HAS_ASM_GOTO_OUTPUT_TIED_OUTPUTS won't create
an inconsistency with existing code.

Regardless, I like that idea.

> and that all uses in the kernels used the symbolic references which are much
> more readable and less confusing than the rules for numbered references
> (which are bug prone IMO).

100% agree, even though it takes me twice as long to write because I can never
remember the syntax :-)  Converting all existing usage is probably a fools errand,
but adding a checkpatch rule would get us going in the right direction.


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

end of thread, other threads:[~2022-01-28 16:29 UTC | newest]

Thread overview: 47+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2022-01-20 15:55 [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Zijlstra
2022-01-20 15:55 ` [RFC][PATCH v2 1/5] mm: Avoid unmapping pinned pages Peter Zijlstra
2022-01-20 18:03   ` Nadav Amit
2022-01-21  7:59     ` Peter Zijlstra
2022-01-20 18:25   ` David Hildenbrand
2022-01-21  7:51     ` Peter Zijlstra
2022-01-21  8:22       ` David Hildenbrand
2022-01-21  8:59       ` Peter Zijlstra
2022-01-21  9:04         ` David Hildenbrand
2022-01-21 11:40           ` Peter Zijlstra
2022-01-21 12:04             ` David Hildenbrand
2022-01-20 15:55 ` [RFC][PATCH v2 2/5] entry,x86: Create common IRQ operations for exceptions Peter Zijlstra
2022-01-21 16:34   ` Mark Rutland
2022-01-20 15:55 ` [RFC][PATCH v2 3/5] sched/umcg: add WF_CURRENT_CPU and externise ttwu Peter Zijlstra
2022-01-20 15:55 ` [RFC][PATCH v2 4/5] x86/uaccess: Implement unsafe_try_cmpxchg_user() Peter Zijlstra
2022-01-27  2:17   ` Sean Christopherson
2022-01-27  6:36     ` Sean Christopherson
2022-01-27  9:56       ` Peter Zijlstra
2022-01-27 23:33         ` Sean Christopherson
2022-01-28  0:17           ` Nick Desaulniers
2022-01-28 16:29             ` Sean Christopherson
2022-01-27  9:55     ` Peter Zijlstra
2022-01-20 15:55 ` [RFC][PATCH v2 5/5] sched: User Mode Concurency Groups Peter Zijlstra
2022-01-21 11:47   ` Peter Zijlstra
2022-01-21 15:18     ` Peter Zijlstra
2022-01-24 14:29       ` Peter Zijlstra
2022-01-24 16:44         ` Peter Zijlstra
2022-01-24 17:06           ` Peter Oskolkov
2022-01-25 14:59         ` Peter Zijlstra
2022-01-24 13:59     ` Peter Zijlstra
2022-01-21 12:26   ` Peter Zijlstra
2022-01-21 16:57   ` Mark Rutland
2022-01-24  9:48     ` Peter Zijlstra
2022-01-24 10:03     ` Peter Zijlstra
2022-01-24 10:07       ` Peter Zijlstra
2022-01-24 10:27         ` Mark Rutland
2022-01-24 14:46   ` Tao Zhou
2022-01-27 12:19     ` Peter Zijlstra
2022-01-27 18:33       ` Tao Zhou
2022-01-27 12:25     ` Peter Zijlstra
2022-01-27 18:47       ` Tao Zhou
2022-01-27 12:26     ` Peter Zijlstra
2022-01-27 18:31   ` Tao Zhou
2022-01-20 17:28 ` [RFC][PATCH v2 0/5] sched: User Managed Concurrency Groups Peter Oskolkov
2022-01-21  8:01   ` Peter Zijlstra
2022-01-21 18:01 ` Steven Rostedt
2022-01-24  8:20   ` Peter Zijlstra

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).