live-patching.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus
@ 2021-09-22 11:05 Peter Zijlstra
  2021-09-22 11:05 ` [RFC][PATCH 1/7] sched,rcu: Rework try_invoke_on_locked_down_task() Peter Zijlstra
                   ` (7 more replies)
  0 siblings, 8 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

Hi,

Compile tested only, please consider carefully, esp. the last few patches that
concern context_tracking and nohz_full.


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

* [RFC][PATCH 1/7] sched,rcu: Rework try_invoke_on_locked_down_task()
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
@ 2021-09-22 11:05 ` Peter Zijlstra
  2021-09-22 11:05 ` [RFC][PATCH 2/7] sched: Fix task_try_func() Peter Zijlstra
                   ` (6 subsequent siblings)
  7 siblings, 0 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

Give try_invoke_on_locked_down_task() a saner name and have it return
an int so that the caller might distinguish between different reasons
of failure.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 include/linux/wait.h    |    3 ++-
 kernel/rcu/tasks.h      |   12 ++++++------
 kernel/rcu/tree_stall.h |    8 ++++----
 kernel/sched/core.c     |   11 +++++------
 4 files changed, 17 insertions(+), 17 deletions(-)

--- a/include/linux/wait.h
+++ b/include/linux/wait.h
@@ -1160,6 +1160,7 @@ int autoremove_wake_function(struct wait
 		(wait)->flags = 0;						\
 	} while (0)
 
-bool try_invoke_on_locked_down_task(struct task_struct *p, bool (*func)(struct task_struct *t, void *arg), void *arg);
+typedef int (*task_try_f)(struct task_struct *p, void *arg);
+extern int task_try_func(struct task_struct *p, task_try_f func, void *arg);
 
 #endif /* _LINUX_WAIT_H */
--- a/kernel/rcu/tasks.h
+++ b/kernel/rcu/tasks.h
@@ -928,7 +928,7 @@ static void trc_read_check_handler(void
 }
 
 /* Callback function for scheduler to check locked-down task.  */
-static bool trc_inspect_reader(struct task_struct *t, void *arg)
+static int trc_inspect_reader(struct task_struct *t, void *arg)
 {
 	int cpu = task_cpu(t);
 	bool in_qs = false;
@@ -939,7 +939,7 @@ static bool trc_inspect_reader(struct ta
 
 		// If no chance of heavyweight readers, do it the hard way.
 		if (!ofl && !IS_ENABLED(CONFIG_TASKS_TRACE_RCU_READ_MB))
-			return false;
+			return -EINVAL;
 
 		// If heavyweight readers are enabled on the remote task,
 		// we can inspect its state despite its currently running.
@@ -947,7 +947,7 @@ static bool trc_inspect_reader(struct ta
 		n_heavy_reader_attempts++;
 		if (!ofl && // Check for "running" idle tasks on offline CPUs.
 		    !rcu_dynticks_zero_in_eqs(cpu, &t->trc_reader_nesting))
-			return false; // No quiescent state, do it the hard way.
+			return -EINVAL; // No quiescent state, do it the hard way.
 		n_heavy_reader_updates++;
 		if (ofl)
 			n_heavy_reader_ofl_updates++;
@@ -962,7 +962,7 @@ static bool trc_inspect_reader(struct ta
 	t->trc_reader_checked = true;
 
 	if (in_qs)
-		return true;  // Already in quiescent state, done!!!
+		return 0;  // Already in quiescent state, done!!!
 
 	// The task is in a read-side critical section, so set up its
 	// state so that it will awaken the grace-period kthread upon exit
@@ -970,7 +970,7 @@ static bool trc_inspect_reader(struct ta
 	atomic_inc(&trc_n_readers_need_end); // One more to wait on.
 	WARN_ON_ONCE(READ_ONCE(t->trc_reader_special.b.need_qs));
 	WRITE_ONCE(t->trc_reader_special.b.need_qs, true);
-	return true;
+	return 0;
 }
 
 /* Attempt to extract the state for the specified task. */
@@ -992,7 +992,7 @@ static void trc_wait_for_one_reader(stru
 
 	// Attempt to nail down the task for inspection.
 	get_task_struct(t);
-	if (try_invoke_on_locked_down_task(t, trc_inspect_reader, NULL)) {
+	if (!task_try_func(t, trc_inspect_reader, NULL)) {
 		put_task_struct(t);
 		return;
 	}
--- a/kernel/rcu/tree_stall.h
+++ b/kernel/rcu/tree_stall.h
@@ -240,16 +240,16 @@ struct rcu_stall_chk_rdr {
  * Report out the state of a not-running task that is stalling the
  * current RCU grace period.
  */
-static bool check_slow_task(struct task_struct *t, void *arg)
+static int check_slow_task(struct task_struct *t, void *arg)
 {
 	struct rcu_stall_chk_rdr *rscrp = arg;
 
 	if (task_curr(t))
-		return false; // It is running, so decline to inspect it.
+		return -EBUSY; // It is running, so decline to inspect it.
 	rscrp->nesting = t->rcu_read_lock_nesting;
 	rscrp->rs = t->rcu_read_unlock_special;
 	rscrp->on_blkd_list = !list_empty(&t->rcu_node_entry);
-	return true;
+	return 0;
 }
 
 /*
@@ -283,7 +283,7 @@ static int rcu_print_task_stall(struct r
 	raw_spin_unlock_irqrestore_rcu_node(rnp, flags);
 	while (i) {
 		t = ts[--i];
-		if (!try_invoke_on_locked_down_task(t, check_slow_task, &rscr))
+		if (task_try_func(t, check_slow_task, &rscr))
 			pr_cont(" P%d", t->pid);
 		else
 			pr_cont(" P%d/%d:%c%c%c%c",
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -4106,7 +4106,7 @@ try_to_wake_up(struct task_struct *p, un
 }
 
 /**
- * try_invoke_on_locked_down_task - Invoke a function on task in fixed state
+ * task_try_func - Invoke a function on task in fixed state
  * @p: Process for which the function is to be invoked, can be @current.
  * @func: Function to invoke.
  * @arg: Argument to function.
@@ -4119,14 +4119,13 @@ try_to_wake_up(struct task_struct *p, un
  * lightweight.
  *
  * Returns:
- *	@false if the task slipped out from under the locks.
- *	@true if the task was locked onto a runqueue or is sleeping.
- *		However, @func can override this by returning @false.
+ *   -EAGAIN: we raced against task movement/state
+ *   *: as returned by @func
  */
-bool try_invoke_on_locked_down_task(struct task_struct *p, bool (*func)(struct task_struct *t, void *arg), void *arg)
+int task_try_func(struct task_struct *p, task_try_f func, void *arg)
 {
 	struct rq_flags rf;
-	bool ret = false;
+	int ret = -EAGAIN; /* raced, try again later */
 	struct rq *rq;
 
 	raw_spin_lock_irqsave(&p->pi_lock, rf.flags);



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

* [RFC][PATCH 2/7] sched: Fix task_try_func()
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
  2021-09-22 11:05 ` [RFC][PATCH 1/7] sched,rcu: Rework try_invoke_on_locked_down_task() Peter Zijlstra
@ 2021-09-22 11:05 ` Peter Zijlstra
  2021-09-22 11:05 ` [RFC][PATCH 3/7] sched,livepatch: Use task_try_func() Peter Zijlstra
                   ` (5 subsequent siblings)
  7 siblings, 0 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

Clarify and fix task_try_func(). Move the smp_rmb() up to avoid
re-loading p->on_rq in the false case, but add a p->on_rq reload after
acquiring rq->lock, after all, it could have gotten dequeued while
waiting for the lock.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/sched/core.c |   37 ++++++++++++++++++++++++++++++-------
 1 file changed, 30 insertions(+), 7 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -4005,7 +4005,7 @@ try_to_wake_up(struct task_struct *p, un
 	 * Pairs with the LOCK+smp_mb__after_spinlock() on rq->lock in
 	 * __schedule().  See the comment for smp_mb__after_spinlock().
 	 *
-	 * A similar smb_rmb() lives in try_invoke_on_locked_down_task().
+	 * A similar smb_rmb() lives in task_try_func().
 	 */
 	smp_rmb();
 	if (READ_ONCE(p->on_rq) && ttwu_runnable(p, wake_flags))
@@ -4124,25 +4124,48 @@ try_to_wake_up(struct task_struct *p, un
  */
 int task_try_func(struct task_struct *p, task_try_f func, void *arg)
 {
+	unsigned int state;
 	struct rq_flags rf;
-	int ret = -EAGAIN; /* raced, try again later */
 	struct rq *rq;
+	int ret = -EAGAIN; /* raced, try again later */
 
 	raw_spin_lock_irqsave(&p->pi_lock, rf.flags);
+
+	state = READ_ONCE(p->__state);
+
+	/*
+	 * Ensure we load p->on_rq after p->__state, otherwise it would be
+	 * possible to, falsely, observe p->on_rq == 0.
+	 *
+	 * See try_to_wake_up() for a longer comment.
+	 */
+	smp_rmb();
+
 	if (p->on_rq) {
 		rq = __task_rq_lock(p, &rf);
-		if (task_rq(p) == rq)
+
+		/* re-check p->on_rq now that we hold rq->lock */
+		if (p->on_rq && task_rq(p) == rq)
 			ret = func(p, arg);
+
 		rq_unlock(rq, &rf);
+
 	} else {
-		switch (READ_ONCE(p->__state)) {
+
+		switch (state) {
 		case TASK_RUNNING:
 		case TASK_WAKING:
+			/*
+			 * We raced against wakeup, try again later.
+			 */
 			break;
+
 		default:
-			smp_rmb(); // See smp_rmb() comment in try_to_wake_up().
-			if (!p->on_rq)
-				ret = func(p, arg);
+			/*
+			 * Since we hold ->pi_lock, we serialize against
+			 * try_to_wake_up() and any blocked state must remain.
+			 */
+			ret = func(p, arg);
 		}
 	}
 	raw_spin_unlock_irqrestore(&p->pi_lock, rf.flags);



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

* [RFC][PATCH 3/7] sched,livepatch: Use task_try_func()
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
  2021-09-22 11:05 ` [RFC][PATCH 1/7] sched,rcu: Rework try_invoke_on_locked_down_task() Peter Zijlstra
  2021-09-22 11:05 ` [RFC][PATCH 2/7] sched: Fix task_try_func() Peter Zijlstra
@ 2021-09-22 11:05 ` Peter Zijlstra
  2021-09-23 12:05   ` Petr Mladek
  2021-09-22 11:05 ` [RFC][PATCH 4/7] sched: Simplify wake_up_*idle*() Peter Zijlstra
                   ` (4 subsequent siblings)
  7 siblings, 1 reply; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

Instead of frobbing around with scheduler internals, use the shiny new
task_try_func() interface.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/livepatch/transition.c |   84 ++++++++++++++++++------------------------
 1 file changed, 37 insertions(+), 47 deletions(-)

--- a/kernel/livepatch/transition.c
+++ b/kernel/livepatch/transition.c
@@ -13,7 +13,6 @@
 #include "core.h"
 #include "patch.h"
 #include "transition.h"
-#include "../sched/sched.h"
 
 #define MAX_STACK_ENTRIES  100
 #define STACK_ERR_BUF_SIZE 128
@@ -240,7 +239,7 @@ static int klp_check_stack_func(struct k
  * Determine whether it's safe to transition the task to the target patch state
  * by looking for any to-be-patched or to-be-unpatched functions on its stack.
  */
-static int klp_check_stack(struct task_struct *task, char *err_buf)
+static int klp_check_stack(struct task_struct *task, const char **oldname)
 {
 	static unsigned long entries[MAX_STACK_ENTRIES];
 	struct klp_object *obj;
@@ -248,12 +247,8 @@ static int klp_check_stack(struct task_s
 	int ret, nr_entries;
 
 	ret = stack_trace_save_tsk_reliable(task, entries, ARRAY_SIZE(entries));
-	if (ret < 0) {
-		snprintf(err_buf, STACK_ERR_BUF_SIZE,
-			 "%s: %s:%d has an unreliable stack\n",
-			 __func__, task->comm, task->pid);
-		return ret;
-	}
+	if (ret < 0)
+		return -EINVAL;
 	nr_entries = ret;
 
 	klp_for_each_object(klp_transition_patch, obj) {
@@ -262,11 +257,8 @@ static int klp_check_stack(struct task_s
 		klp_for_each_func(obj, func) {
 			ret = klp_check_stack_func(func, entries, nr_entries);
 			if (ret) {
-				snprintf(err_buf, STACK_ERR_BUF_SIZE,
-					 "%s: %s:%d is sleeping on function %s\n",
-					 __func__, task->comm, task->pid,
-					 func->old_name);
-				return ret;
+				*oldname = func->old_name;
+				return -EADDRINUSE;
 			}
 		}
 	}
@@ -274,6 +266,22 @@ static int klp_check_stack(struct task_s
 	return 0;
 }
 
+static int klp_check_task(struct task_struct *task, void *arg)
+{
+	int ret;
+
+	if (task_curr(task))
+		return -EBUSY;
+
+	ret = klp_check_stack(task, arg);
+	if (ret)
+		return ret;
+
+	clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
+	task->patch_state = klp_target_state;
+	return 0;
+}
+
 /*
  * Try to safely switch a task to the target patch state.  If it's currently
  * running, or it's sleeping on a to-be-patched or to-be-unpatched function, or
@@ -281,13 +289,8 @@ static int klp_check_stack(struct task_s
  */
 static bool klp_try_switch_task(struct task_struct *task)
 {
-	static char err_buf[STACK_ERR_BUF_SIZE];
-	struct rq *rq;
-	struct rq_flags flags;
+	const char *old_name;
 	int ret;
-	bool success = false;
-
-	err_buf[0] = '\0';
 
 	/* check if this task has already switched over */
 	if (task->patch_state == klp_target_state)
@@ -305,36 +308,23 @@ static bool klp_try_switch_task(struct t
 	 * functions.  If all goes well, switch the task to the target patch
 	 * state.
 	 */
-	rq = task_rq_lock(task, &flags);
-
-	if (task_running(rq, task) && task != current) {
-		snprintf(err_buf, STACK_ERR_BUF_SIZE,
-			 "%s: %s:%d is running\n", __func__, task->comm,
-			 task->pid);
-		goto done;
+	ret = task_try_func(task, klp_check_task, &old_name);
+	switch (ret) {
+	case -EBUSY:
+		pr_debug("%s: %s:%d is running\n",
+			 __func__, task->comm, task->pid);
+		break;
+	case -EINVAL:
+		pr_debug("%s: %s:%d has an unreliable stack\n",
+			 __func__, task->comm, task->pid);
+		break;
+	case -EADDRINUSE:
+		pr_debug("%s: %s:%d is sleeping on function %s\n",
+			 __func__, task->comm, task->pid, old_name);
+		break;
 	}
 
-	ret = klp_check_stack(task, err_buf);
-	if (ret)
-		goto done;
-
-	success = true;
-
-	clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
-	task->patch_state = klp_target_state;
-
-done:
-	task_rq_unlock(rq, task, &flags);
-
-	/*
-	 * Due to console deadlock issues, pr_debug() can't be used while
-	 * holding the task rq lock.  Instead we have to use a temporary buffer
-	 * and print the debug message after releasing the lock.
-	 */
-	if (err_buf[0] != '\0')
-		pr_debug("%s", err_buf);
-
-	return success;
+	return !ret;
 }
 
 /*



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

* [RFC][PATCH 4/7] sched: Simplify wake_up_*idle*()
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
                   ` (2 preceding siblings ...)
  2021-09-22 11:05 ` [RFC][PATCH 3/7] sched,livepatch: Use task_try_func() Peter Zijlstra
@ 2021-09-22 11:05 ` Peter Zijlstra
  2021-09-22 11:05 ` [RFC][PATCH 5/7] sched,livepatch: Use wake_up_if_idle() Peter Zijlstra
                   ` (3 subsequent siblings)
  7 siblings, 0 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

Simplify and make wake_up_if_idle() more robust, also don't iterate
the whole machine with preempt_disable() in it's caller:
wake_up_all_idle_cpus().

This prepares for another wake_up_if_idle() user that needs a full
do_idle() cycle.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/sched/core.c |   14 +++++---------
 kernel/smp.c        |    6 +++---
 2 files changed, 8 insertions(+), 12 deletions(-)

--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -3691,15 +3691,11 @@ void wake_up_if_idle(int cpu)
 	if (!is_idle_task(rcu_dereference(rq->curr)))
 		goto out;
 
-	if (set_nr_if_polling(rq->idle)) {
-		trace_sched_wake_idle_without_ipi(cpu);
-	} else {
-		rq_lock_irqsave(rq, &rf);
-		if (is_idle_task(rq->curr))
-			smp_send_reschedule(cpu);
-		/* Else CPU is not idle, do nothing here: */
-		rq_unlock_irqrestore(rq, &rf);
-	}
+	rq_lock_irqsave(rq, &rf);
+	if (is_idle_task(rq->curr))
+		resched_curr(rq);
+	/* Else CPU is not idle, do nothing here: */
+	rq_unlock_irqrestore(rq, &rf);
 
 out:
 	rcu_read_unlock();
--- a/kernel/smp.c
+++ b/kernel/smp.c
@@ -1170,14 +1170,14 @@ void wake_up_all_idle_cpus(void)
 {
 	int cpu;
 
-	preempt_disable();
+	cpus_read_lock();
 	for_each_online_cpu(cpu) {
-		if (cpu == smp_processor_id())
+		if (cpu == raw_smp_processor_id())
 			continue;
 
 		wake_up_if_idle(cpu);
 	}
-	preempt_enable();
+	cpus_read_unlock();
 }
 EXPORT_SYMBOL_GPL(wake_up_all_idle_cpus);
 



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

* [RFC][PATCH 5/7] sched,livepatch: Use wake_up_if_idle()
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
                   ` (3 preceding siblings ...)
  2021-09-22 11:05 ` [RFC][PATCH 4/7] sched: Simplify wake_up_*idle*() Peter Zijlstra
@ 2021-09-22 11:05 ` Peter Zijlstra
  2021-09-22 13:05   ` Miroslav Benes
  2021-09-22 11:05 ` [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU Peter Zijlstra
                   ` (2 subsequent siblings)
  7 siblings, 1 reply; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

Make sure to prod idle CPUs so they call klp_update_patch_state().

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/livepatch/transition.c |   16 +++++++++-------
 1 file changed, 9 insertions(+), 7 deletions(-)

--- a/kernel/livepatch/transition.c
+++ b/kernel/livepatch/transition.c
@@ -287,21 +287,21 @@ static int klp_check_task(struct task_st
  * running, or it's sleeping on a to-be-patched or to-be-unpatched function, or
  * if the stack is unreliable, return false.
  */
-static bool klp_try_switch_task(struct task_struct *task)
+static int klp_try_switch_task(struct task_struct *task)
 {
 	const char *old_name;
 	int ret;
 
 	/* check if this task has already switched over */
 	if (task->patch_state == klp_target_state)
-		return true;
+		return 0;
 
 	/*
 	 * For arches which don't have reliable stack traces, we have to rely
 	 * on other methods (e.g., switching tasks at kernel exit).
 	 */
 	if (!klp_have_reliable_stack())
-		return false;
+		return -EINVAL;
 
 	/*
 	 * Now try to check the stack for any to-be-patched or to-be-unpatched
@@ -324,7 +324,7 @@ static bool klp_try_switch_task(struct t
 		break;
 	}
 
-	return !ret;
+	return ret;
 }
 
 /*
@@ -394,7 +394,7 @@ void klp_try_complete_transition(void)
 	 */
 	read_lock(&tasklist_lock);
 	for_each_process_thread(g, task)
-		if (!klp_try_switch_task(task))
+		if (klp_try_switch_task(task))
 			complete = false;
 	read_unlock(&tasklist_lock);
 
@@ -405,8 +405,10 @@ void klp_try_complete_transition(void)
 	for_each_possible_cpu(cpu) {
 		task = idle_task(cpu);
 		if (cpu_online(cpu)) {
-			if (!klp_try_switch_task(task))
-				complete = false;
+			int ret = klp_try_switch_task(task);
+			if (ret == -EBUSY)
+				wake_up_if_idle(cpu);
+			complete = !ret;
 		} else if (task->patch_state != klp_target_state) {
 			/* offline idle tasks can be switched immediately */
 			clear_tsk_thread_flag(task, TIF_PATCH_PENDING);



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

* [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
                   ` (4 preceding siblings ...)
  2021-09-22 11:05 ` [RFC][PATCH 5/7] sched,livepatch: Use wake_up_if_idle() Peter Zijlstra
@ 2021-09-22 11:05 ` Peter Zijlstra
  2021-09-22 15:17   ` Paul E. McKenney
                     ` (2 more replies)
  2021-09-22 11:05 ` [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks Peter Zijlstra
  2021-09-23 13:22 ` [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Petr Mladek
  7 siblings, 3 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

Use rcu_user_{enter,exit}() calls to provide SMP ordering on context
tracking state stores:

__context_tracking_exit()
  __this_cpu_write(context_tracking.state, CONTEXT_KERNEL)
  rcu_user_exit()
    rcu_eqs_exit()
      rcu_dynticks_eqs_eit()
        rcu_dynticks_inc()
          atomic_add_return() /* smp_mb */

__context_tracking_enter()
  rcu_user_enter()
    rcu_eqs_enter()
      rcu_dynticks_eqs_enter()
        rcu_dynticks_inc()
	  atomic_add_return() /* smp_mb */
  __this_cpu_write(context_tracking.state, state)

This separates USER/KERNEL state with an smp_mb() on each side,
therefore, a user of context_tracking_state_cpu() can say the CPU must
pass through an smp_mb() before changing.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 include/linux/context_tracking_state.h |   12 ++++++++++++
 kernel/context_tracking.c              |    7 ++++---
 2 files changed, 16 insertions(+), 3 deletions(-)

--- a/include/linux/context_tracking_state.h
+++ b/include/linux/context_tracking_state.h
@@ -45,11 +45,23 @@ static __always_inline bool context_trac
 {
 	return __this_cpu_read(context_tracking.state) == CONTEXT_USER;
 }
+
+static __always_inline bool context_tracking_state_cpu(int cpu)
+{
+	struct context_tracking *ct = per_cpu_ptr(&context_tracking);
+
+	if (!context_tracking_enabled() || !ct->active)
+		return CONTEXT_DISABLED;
+
+	return ct->state;
+}
+
 #else
 static inline bool context_tracking_in_user(void) { return false; }
 static inline bool context_tracking_enabled(void) { return false; }
 static inline bool context_tracking_enabled_cpu(int cpu) { return false; }
 static inline bool context_tracking_enabled_this_cpu(void) { return false; }
+static inline bool context_tracking_state_cpu(int cpu) { return CONTEXT_DISABLED; }
 #endif /* CONFIG_CONTEXT_TRACKING */
 
 #endif
--- a/kernel/context_tracking.c
+++ b/kernel/context_tracking.c
@@ -82,7 +82,7 @@ void noinstr __context_tracking_enter(en
 				vtime_user_enter(current);
 				instrumentation_end();
 			}
-			rcu_user_enter();
+			rcu_user_enter(); /* smp_mb */
 		}
 		/*
 		 * Even if context tracking is disabled on this CPU, because it's outside
@@ -149,12 +149,14 @@ void noinstr __context_tracking_exit(enu
 		return;
 
 	if (__this_cpu_read(context_tracking.state) == state) {
+		__this_cpu_write(context_tracking.state, CONTEXT_KERNEL);
+
 		if (__this_cpu_read(context_tracking.active)) {
 			/*
 			 * We are going to run code that may use RCU. Inform
 			 * RCU core about that (ie: we may need the tick again).
 			 */
-			rcu_user_exit();
+			rcu_user_exit(); /* smp_mb */
 			if (state == CONTEXT_USER) {
 				instrumentation_begin();
 				vtime_user_exit(current);
@@ -162,7 +164,6 @@ void noinstr __context_tracking_exit(enu
 				instrumentation_end();
 			}
 		}
-		__this_cpu_write(context_tracking.state, CONTEXT_KERNEL);
 	}
 	context_tracking_recursion_exit();
 }



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

* [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
                   ` (5 preceding siblings ...)
  2021-09-22 11:05 ` [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU Peter Zijlstra
@ 2021-09-22 11:05 ` Peter Zijlstra
  2021-09-23 13:14   ` Petr Mladek
  2021-09-23 13:22 ` [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Petr Mladek
  7 siblings, 1 reply; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 11:05 UTC (permalink / raw)
  To: gor, jpoimboe, jikos, mbenes, pmladek, mingo
  Cc: linux-kernel, peterz, joe.lawrence, fweisbec, tglx, hca, svens,
	sumanthk, live-patching, paulmck

When a task is stuck in NOHZ_FULL usermode, we can simply mark the
livepatch state complete.

Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
---
 kernel/livepatch/transition.c |   13 ++++++++++++-
 1 file changed, 12 insertions(+), 1 deletion(-)

--- a/kernel/livepatch/transition.c
+++ b/kernel/livepatch/transition.c
@@ -270,13 +270,24 @@ static int klp_check_task(struct task_st
 {
 	int ret;
 
-	if (task_curr(task))
+	if (task_curr(task)) {
+		if (context_tracking_state_cpu(task_cpu(task)) == CONTEXT_USER) {
+			/*
+			 * If we observe the CPU being in USER context, they
+			 * must issue an smp_mb() before doing much kernel
+			 * space and as such will observe the patched state,
+			 * mark it clean.
+			 */
+			goto complete;
+		}
 		return -EBUSY;
+	}
 
 	ret = klp_check_stack(task, arg);
 	if (ret)
 		return ret;
 
+complete:
 	clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
 	task->patch_state = klp_target_state;
 	return 0;



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

* Re: [RFC][PATCH 5/7] sched,livepatch: Use wake_up_if_idle()
  2021-09-22 11:05 ` [RFC][PATCH 5/7] sched,livepatch: Use wake_up_if_idle() Peter Zijlstra
@ 2021-09-22 13:05   ` Miroslav Benes
  2021-09-23 12:19     ` Petr Mladek
  0 siblings, 1 reply; 26+ messages in thread
From: Miroslav Benes @ 2021-09-22 13:05 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, pmladek, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

> @@ -405,8 +405,10 @@ void klp_try_complete_transition(void)
>  	for_each_possible_cpu(cpu) {
>  		task = idle_task(cpu);
>  		if (cpu_online(cpu)) {
> -			if (!klp_try_switch_task(task))
> -				complete = false;
> +			int ret = klp_try_switch_task(task);
> +			if (ret == -EBUSY)
> +				wake_up_if_idle(cpu);
> +			complete = !ret;

This is broken. You can basically change "complete" only to false (when it 
applies). This could leave some tasks in the old patching state.

Anyway, I like the patch set a lot. It moves our infrastructure to a 
proper (I hope so) API and it removes few quirks we have along the way. 
I'll play with it some more.

Thanks

Miroslav

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 11:05 ` [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU Peter Zijlstra
@ 2021-09-22 15:17   ` Paul E. McKenney
  2021-09-22 19:33     ` Peter Zijlstra
  2021-09-23 12:10   ` Petr Mladek
  2021-09-24 18:57   ` Joel Savitz
  2 siblings, 1 reply; 26+ messages in thread
From: Paul E. McKenney @ 2021-09-22 15:17 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, pmladek, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching

On Wed, Sep 22, 2021 at 01:05:12PM +0200, Peter Zijlstra wrote:
> Use rcu_user_{enter,exit}() calls to provide SMP ordering on context
> tracking state stores:
> 
> __context_tracking_exit()
>   __this_cpu_write(context_tracking.state, CONTEXT_KERNEL)
>   rcu_user_exit()
>     rcu_eqs_exit()
>       rcu_dynticks_eqs_eit()
>         rcu_dynticks_inc()
>           atomic_add_return() /* smp_mb */
> 
> __context_tracking_enter()
>   rcu_user_enter()
>     rcu_eqs_enter()
>       rcu_dynticks_eqs_enter()
>         rcu_dynticks_inc()
> 	  atomic_add_return() /* smp_mb */
>   __this_cpu_write(context_tracking.state, state)
> 
> This separates USER/KERNEL state with an smp_mb() on each side,
> therefore, a user of context_tracking_state_cpu() can say the CPU must
> pass through an smp_mb() before changing.
> 
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>

For the transformation to negative errno return value and name change
from an RCU perspective:

Acked-by: Paul E. McKenney <paulmck@kernel.org>

For the sampling of nohz_full userspace state:

Another approach is for the rcu_data structure's ->dynticks variable to
use the lower two bits to differentiate between idle, nohz_full userspace
and kernel.  In theory, inlining should make this zero cost for idle
transition, and should allow you to safely sample nohz_full userspace
state with a load and a couple of memory barriers instead of an IPI.

To make this work nicely, the low-order bits have to be 00 for kernel,
and (say) 01 for idle and 10 for nohz_full userspace.  11 would be an
error.

The trick would be for rcu_user_enter() and rcu_user_exit() to atomically
increment ->dynticks by 2, for rcu_nmi_exit() to increment by 1 and
rcu_nmi_enter() to increment by 3.  The state sampling would need to
change accordingly.

Does this make sense, or am I missing something?

							Thanx, Paul

> ---
>  include/linux/context_tracking_state.h |   12 ++++++++++++
>  kernel/context_tracking.c              |    7 ++++---
>  2 files changed, 16 insertions(+), 3 deletions(-)
> 
> --- a/include/linux/context_tracking_state.h
> +++ b/include/linux/context_tracking_state.h
> @@ -45,11 +45,23 @@ static __always_inline bool context_trac
>  {
>  	return __this_cpu_read(context_tracking.state) == CONTEXT_USER;
>  }
> +
> +static __always_inline bool context_tracking_state_cpu(int cpu)
> +{
> +	struct context_tracking *ct = per_cpu_ptr(&context_tracking);
> +
> +	if (!context_tracking_enabled() || !ct->active)
> +		return CONTEXT_DISABLED;
> +
> +	return ct->state;
> +}
> +
>  #else
>  static inline bool context_tracking_in_user(void) { return false; }
>  static inline bool context_tracking_enabled(void) { return false; }
>  static inline bool context_tracking_enabled_cpu(int cpu) { return false; }
>  static inline bool context_tracking_enabled_this_cpu(void) { return false; }
> +static inline bool context_tracking_state_cpu(int cpu) { return CONTEXT_DISABLED; }
>  #endif /* CONFIG_CONTEXT_TRACKING */
>  
>  #endif
> --- a/kernel/context_tracking.c
> +++ b/kernel/context_tracking.c
> @@ -82,7 +82,7 @@ void noinstr __context_tracking_enter(en
>  				vtime_user_enter(current);
>  				instrumentation_end();
>  			}
> -			rcu_user_enter();
> +			rcu_user_enter(); /* smp_mb */
>  		}
>  		/*
>  		 * Even if context tracking is disabled on this CPU, because it's outside
> @@ -149,12 +149,14 @@ void noinstr __context_tracking_exit(enu
>  		return;
>  
>  	if (__this_cpu_read(context_tracking.state) == state) {
> +		__this_cpu_write(context_tracking.state, CONTEXT_KERNEL);
> +
>  		if (__this_cpu_read(context_tracking.active)) {
>  			/*
>  			 * We are going to run code that may use RCU. Inform
>  			 * RCU core about that (ie: we may need the tick again).
>  			 */
> -			rcu_user_exit();
> +			rcu_user_exit(); /* smp_mb */
>  			if (state == CONTEXT_USER) {
>  				instrumentation_begin();
>  				vtime_user_exit(current);
> @@ -162,7 +164,6 @@ void noinstr __context_tracking_exit(enu
>  				instrumentation_end();
>  			}
>  		}
> -		__this_cpu_write(context_tracking.state, CONTEXT_KERNEL);
>  	}
>  	context_tracking_recursion_exit();
>  }
> 
> 

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 15:17   ` Paul E. McKenney
@ 2021-09-22 19:33     ` Peter Zijlstra
  2021-09-22 19:47       ` Peter Zijlstra
  2021-09-22 19:53       ` Paul E. McKenney
  0 siblings, 2 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 19:33 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: gor, jpoimboe, jikos, mbenes, pmladek, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching

On Wed, Sep 22, 2021 at 08:17:21AM -0700, Paul E. McKenney wrote:
> On Wed, Sep 22, 2021 at 01:05:12PM +0200, Peter Zijlstra wrote:
> > Use rcu_user_{enter,exit}() calls to provide SMP ordering on context
> > tracking state stores:
> > 
> > __context_tracking_exit()
> >   __this_cpu_write(context_tracking.state, CONTEXT_KERNEL)
> >   rcu_user_exit()
> >     rcu_eqs_exit()
> >       rcu_dynticks_eqs_eit()
> >         rcu_dynticks_inc()
> >           atomic_add_return() /* smp_mb */
> > 
> > __context_tracking_enter()
> >   rcu_user_enter()
> >     rcu_eqs_enter()
> >       rcu_dynticks_eqs_enter()
> >         rcu_dynticks_inc()
> > 	  atomic_add_return() /* smp_mb */
> >   __this_cpu_write(context_tracking.state, state)
> > 
> > This separates USER/KERNEL state with an smp_mb() on each side,
> > therefore, a user of context_tracking_state_cpu() can say the CPU must
> > pass through an smp_mb() before changing.
> > 
> > Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> 
> For the transformation to negative errno return value and name change
> from an RCU perspective:
> 
> Acked-by: Paul E. McKenney <paulmck@kernel.org>

Thanks!

> For the sampling of nohz_full userspace state:
> 
> Another approach is for the rcu_data structure's ->dynticks variable to
> use the lower two bits to differentiate between idle, nohz_full userspace
> and kernel.  In theory, inlining should make this zero cost for idle
> transition, and should allow you to safely sample nohz_full userspace
> state with a load and a couple of memory barriers instead of an IPI.

That's what I do now, it's like:

  <user code>

  state = KERNEL
  smp_mb()

  <kernel code>

  smp_mb()
  state = USER

  <user core>

vs

  <patch kernel code>
  smp_mb()
  if (state == USER)
    // then we're guaranteed any subsequent kernel code execution
    // will see the modified kernel code

more-or-less

> To make this work nicely, the low-order bits have to be 00 for kernel,
> and (say) 01 for idle and 10 for nohz_full userspace.  11 would be an
> error.
> 
> The trick would be for rcu_user_enter() and rcu_user_exit() to atomically
> increment ->dynticks by 2, for rcu_nmi_exit() to increment by 1 and
> rcu_nmi_enter() to increment by 3.  The state sampling would need to
> change accordingly.
> 
> Does this make sense, or am I missing something?

Why doesn't the proposed patch work? Also, ISTR sampling of remote
context state coming up before. And as is, it's a weird mix between
context_tracking and rcu.

AFAICT there is very little useful in context_tracking as is, but it's
also very weird to have to ask RCU about this. Is there any way to slice
this this code differently? Perhaps move some of the state RCU now keeps
into context_tracking ?

Anyway, lemme see if I get your proposal; lets say the counter starts at
0 and is in kernel space.

 0x00(0) - kernel
 0x02(2) - user
 0x04(0) - kernel

So far so simple, then NMI on top of that goes:

 0x00(0) - kernel
 0x03(3) - kernel + nmi
 0x04(0) - kernel
 0x06(2) - user
 0x09(1) - user + nmi
 0x0a(2) - user

Which then gives us:

 (0) := kernel
 (1) := nmi-from-user
 (2) := user
 (3) := nmi-from-kernel

Which should work I suppose. But like I said above, I'd be happier if
this counter would live in context_tracking rather than RCU.

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 19:33     ` Peter Zijlstra
@ 2021-09-22 19:47       ` Peter Zijlstra
  2021-09-22 19:59         ` Paul E. McKenney
  2021-09-22 19:53       ` Paul E. McKenney
  1 sibling, 1 reply; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 19:47 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: gor, jpoimboe, jikos, mbenes, pmladek, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching

On Wed, Sep 22, 2021 at 09:33:43PM +0200, Peter Zijlstra wrote:

> Anyway, lemme see if I get your proposal; lets say the counter starts at
> 0 and is in kernel space.
> 
>  0x00(0) - kernel
>  0x02(2) - user
>  0x04(0) - kernel
> 
> So far so simple, then NMI on top of that goes:
> 
>  0x00(0) - kernel
>  0x03(3) - kernel + nmi
>  0x04(0) - kernel
>  0x06(2) - user
>  0x09(1) - user + nmi
>  0x0a(2) - user
> 
> Which then gives us:
> 
>  (0) := kernel
>  (1) := nmi-from-user
>  (2) := user
>  (3) := nmi-from-kernel
> 
> Which should work I suppose. But like I said above, I'd be happier if
> this counter would live in context_tracking rather than RCU.

Furthermore, if we have this counter, the we can also do things like:

  seq = context_tracking_seq_cpu(that_cpu);
  if ((seq & 3) != USER)
    // nohz_fail, do something
  set_tsk_thread_flag(curr_task(that_cpu), TIF_DO_SOME_WORK);
  if (seq == context_tracking_seq_cpu(that_cpu))
    // success!!

To remotely set pending state. Allowing yet more NOHZ_FULL fixes, like,
for example, eliding the text_poke IPIs.

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 19:33     ` Peter Zijlstra
  2021-09-22 19:47       ` Peter Zijlstra
@ 2021-09-22 19:53       ` Paul E. McKenney
  2021-09-22 20:02         ` Peter Zijlstra
  1 sibling, 1 reply; 26+ messages in thread
From: Paul E. McKenney @ 2021-09-22 19:53 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, pmladek, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching

On Wed, Sep 22, 2021 at 09:33:43PM +0200, Peter Zijlstra wrote:
> On Wed, Sep 22, 2021 at 08:17:21AM -0700, Paul E. McKenney wrote:
> > On Wed, Sep 22, 2021 at 01:05:12PM +0200, Peter Zijlstra wrote:
> > > Use rcu_user_{enter,exit}() calls to provide SMP ordering on context
> > > tracking state stores:
> > > 
> > > __context_tracking_exit()
> > >   __this_cpu_write(context_tracking.state, CONTEXT_KERNEL)
> > >   rcu_user_exit()
> > >     rcu_eqs_exit()
> > >       rcu_dynticks_eqs_eit()
> > >         rcu_dynticks_inc()
> > >           atomic_add_return() /* smp_mb */
> > > 
> > > __context_tracking_enter()
> > >   rcu_user_enter()
> > >     rcu_eqs_enter()
> > >       rcu_dynticks_eqs_enter()
> > >         rcu_dynticks_inc()
> > > 	  atomic_add_return() /* smp_mb */
> > >   __this_cpu_write(context_tracking.state, state)
> > > 
> > > This separates USER/KERNEL state with an smp_mb() on each side,
> > > therefore, a user of context_tracking_state_cpu() can say the CPU must
> > > pass through an smp_mb() before changing.
> > > 
> > > Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> > 
> > For the transformation to negative errno return value and name change
> > from an RCU perspective:
> > 
> > Acked-by: Paul E. McKenney <paulmck@kernel.org>
> 
> Thanks!
> 
> > For the sampling of nohz_full userspace state:
> > 
> > Another approach is for the rcu_data structure's ->dynticks variable to
> > use the lower two bits to differentiate between idle, nohz_full userspace
> > and kernel.  In theory, inlining should make this zero cost for idle
> > transition, and should allow you to safely sample nohz_full userspace
> > state with a load and a couple of memory barriers instead of an IPI.
> 
> That's what I do now, it's like:
> 
>   <user code>
> 
>   state = KERNEL
>   smp_mb()
> 
>   <kernel code>
> 
>   smp_mb()
>   state = USER
> 
>   <user core>
> 
> vs
> 
>   <patch kernel code>
>   smp_mb()
>   if (state == USER)
>     // then we're guaranteed any subsequent kernel code execution
>     // will see the modified kernel code
> 
> more-or-less

OK.

> > To make this work nicely, the low-order bits have to be 00 for kernel,
> > and (say) 01 for idle and 10 for nohz_full userspace.  11 would be an
> > error.
> > 
> > The trick would be for rcu_user_enter() and rcu_user_exit() to atomically
> > increment ->dynticks by 2, for rcu_nmi_exit() to increment by 1 and
> > rcu_nmi_enter() to increment by 3.  The state sampling would need to
> > change accordingly.
> > 
> > Does this make sense, or am I missing something?
> 
> Why doesn't the proposed patch work? Also, ISTR sampling of remote
> context state coming up before. And as is, it's a weird mix between
> context_tracking and rcu.

I wasn't saying that the patch doesn't work.  But doesn't it add an IPI?
Or was I looking at it too early this morning?

As to RCU's ->dynticks and context-tracking state, something about RCU
being there first by many years.  ;-)  Plus, does context-tracking
track idleness within the kernel?  RCU needs that as well.

> AFAICT there is very little useful in context_tracking as is, but it's
> also very weird to have to ask RCU about this. Is there any way to slice
> this this code differently? Perhaps move some of the state RCU now keeps
> into context_tracking ?
> 
> Anyway, lemme see if I get your proposal; lets say the counter starts at
> 0 and is in kernel space.
> 
>  0x00(0) - kernel
>  0x02(2) - user
>  0x04(0) - kernel
> 
> So far so simple, then NMI on top of that goes:
> 
>  0x00(0) - kernel
>  0x03(3) - kernel + nmi

This would stay 0x00 because the NMI is interrupting kernel code.  The
check of rcu_dynticks_curr_cpu_in_eqs() avoids this additional increment.

>  0x04(0) - kernel

And same here, still zero.

>  0x06(2) - user

And now 0x02.

>  0x09(1) - user + nmi

This would be 0x04, back in the kernel.  Which is the area of concern,
because the amount to increment depends on the counter value, requiring
an additional arithmetic operation on the from-idle fastpath.  Probably
not visible even in microbenchmarks, but still a potential issue.

>  0x0a(2) - user

Now 0x06, back in nohz_full userspace.

> Which then gives us:
> 
>  (0) := kernel
>  (1) := nmi-from-user
>  (2) := user
>  (3) := nmi-from-kernel

You need to know NMI as a separate state?

> Which should work I suppose. But like I said above, I'd be happier if
> this counter would live in context_tracking rather than RCU.

This would be the first non-RCU user of the counter.  The various
rcu_*_{enter,exit}() functions are still required, though, in order
to handle things like deferred wakeups.  Plus RCU makes heavy use
of that counter.  So it is not clear that moving the counter to the
context-tracking subsystem really buys you anything.

But it would be good to avoid maintaining duplicate information,
that is assuming that the information really is duplicate...

							Thanx, Paul

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 19:47       ` Peter Zijlstra
@ 2021-09-22 19:59         ` Paul E. McKenney
  0 siblings, 0 replies; 26+ messages in thread
From: Paul E. McKenney @ 2021-09-22 19:59 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, pmladek, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching

On Wed, Sep 22, 2021 at 09:47:59PM +0200, Peter Zijlstra wrote:
> On Wed, Sep 22, 2021 at 09:33:43PM +0200, Peter Zijlstra wrote:
> 
> > Anyway, lemme see if I get your proposal; lets say the counter starts at
> > 0 and is in kernel space.
> > 
> >  0x00(0) - kernel
> >  0x02(2) - user
> >  0x04(0) - kernel
> > 
> > So far so simple, then NMI on top of that goes:
> > 
> >  0x00(0) - kernel
> >  0x03(3) - kernel + nmi
> >  0x04(0) - kernel
> >  0x06(2) - user
> >  0x09(1) - user + nmi
> >  0x0a(2) - user
> > 
> > Which then gives us:
> > 
> >  (0) := kernel
> >  (1) := nmi-from-user
> >  (2) := user
> >  (3) := nmi-from-kernel
> > 
> > Which should work I suppose. But like I said above, I'd be happier if
> > this counter would live in context_tracking rather than RCU.
> 
> Furthermore, if we have this counter, the we can also do things like:
> 
>   seq = context_tracking_seq_cpu(that_cpu);
>   if ((seq & 3) != USER)
>     // nohz_fail, do something
>   set_tsk_thread_flag(curr_task(that_cpu), TIF_DO_SOME_WORK);
>   if (seq == context_tracking_seq_cpu(that_cpu))
>     // success!!
> 
> To remotely set pending state. Allowing yet more NOHZ_FULL fixes, like,
> for example, eliding the text_poke IPIs.

Nice!

There have been several instances where I thought that the extra state
would help RCU, but each time there turned out to be a simpler way to
get things done.  Or that it eventually turned out that RCU didn't need
to care about the difference between idle and nohz_full userspace.

							Thanx, Paul

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 19:53       ` Paul E. McKenney
@ 2021-09-22 20:02         ` Peter Zijlstra
  0 siblings, 0 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-22 20:02 UTC (permalink / raw)
  To: Paul E. McKenney
  Cc: gor, jpoimboe, jikos, mbenes, pmladek, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching

On Wed, Sep 22, 2021 at 12:53:50PM -0700, Paul E. McKenney wrote:

> I wasn't saying that the patch doesn't work.  But doesn't it add an IPI?
> Or was I looking at it too early this morning?

Ah, no. The patch allows a user-bound NOHZ_FULL task to be transitioned
remotely. Unlike today, where they'll eventually poke it with a signal
to force a kernel entry, which is bad m'kay :-)

The code in question skips transitioning running tasks, seeing as you
can't tell what they're doing etc.. Howver, with context tracking on
you're supposedly able to tell they're in userspace without disturbing
them -- except you really can't today.

So if you can tell that a current running task is in userspace (hence my
patch) you can allow the task to transition without any further ado,
userspace is a safe state vs kernel text patching.

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

* Re: [RFC][PATCH 3/7] sched,livepatch: Use task_try_func()
  2021-09-22 11:05 ` [RFC][PATCH 3/7] sched,livepatch: Use task_try_func() Peter Zijlstra
@ 2021-09-23 12:05   ` Petr Mladek
  2021-09-23 13:17     ` Peter Zijlstra
  0 siblings, 1 reply; 26+ messages in thread
From: Petr Mladek @ 2021-09-23 12:05 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Wed 2021-09-22 13:05:09, Peter Zijlstra wrote:
> Instead of frobbing around with scheduler internals, use the shiny new
> task_try_func() interface.
> 
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> ---
>  kernel/livepatch/transition.c |   84 ++++++++++++++++++------------------------
>  1 file changed, 37 insertions(+), 47 deletions(-)
> 
> --- a/kernel/livepatch/transition.c
> +++ b/kernel/livepatch/transition.c
> @@ -274,6 +266,22 @@ static int klp_check_stack(struct task_s
>  	return 0;
>  }
>  
> +static int klp_check_task(struct task_struct *task, void *arg)

Please, call this klp_check_and_switch_task() to make it clear
that it actually does the switch.

> +{
> +	int ret;
> +
> +	if (task_curr(task))
> +		return -EBUSY;
> +
> +	ret = klp_check_stack(task, arg);
> +	if (ret)
> +		return ret;
> +
> +	clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
> +	task->patch_state = klp_target_state;
> +	return 0;
> +}
> +
>  /*
>   * Try to safely switch a task to the target patch state.  If it's currently
>   * running, or it's sleeping on a to-be-patched or to-be-unpatched function, or
> @@ -305,36 +308,23 @@ static bool klp_try_switch_task(struct t
>  	 * functions.  If all goes well, switch the task to the target patch
>  	 * state.
>  	 */
> -	rq = task_rq_lock(task, &flags);
> -
> -	if (task_running(rq, task) && task != current) {
> -		snprintf(err_buf, STACK_ERR_BUF_SIZE,
> -			 "%s: %s:%d is running\n", __func__, task->comm,
> -			 task->pid);
> -		goto done;
> +	ret = task_try_func(task, klp_check_task, &old_name);
> +	switch (ret) {
> +	case -EBUSY:
> +		pr_debug("%s: %s:%d is running\n",
> +			 __func__, task->comm, task->pid);
> +		break;
> +	case -EINVAL:
> +		pr_debug("%s: %s:%d has an unreliable stack\n",
> +			 __func__, task->comm, task->pid);
> +		break;
> +	case -EADDRINUSE:
> +		pr_debug("%s: %s:%d is sleeping on function %s\n",
> +			 __func__, task->comm, task->pid, old_name);
> +		break;

I would prefer to be on the safe side and catch error codes that might
eventually appear in the future.

	case 0:
		/* success */
		break;
	default:
		pr_debug("%s: Unknown error code (%d) when trying to switch %s:%d\n",
			 __func__, ret, task->comm, task->pid);

>  	}
>  
> -	ret = klp_check_stack(task, err_buf);
> -	if (ret)
> -		goto done;
> -
> -	success = true;
> -
> -	clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
> -	task->patch_state = klp_target_state;
> -
> -done:
> -	task_rq_unlock(rq, task, &flags);
> -
> -	/*
> -	 * Due to console deadlock issues, pr_debug() can't be used while
> -	 * holding the task rq lock.  Instead we have to use a temporary buffer
> -	 * and print the debug message after releasing the lock.
> -	 */
> -	if (err_buf[0] != '\0')
> -		pr_debug("%s", err_buf);
> -
> -	return success;
> +	return !ret;
>  }

Otherwise, it is great improvement!

Best Regards,
Petr

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 11:05 ` [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU Peter Zijlstra
  2021-09-22 15:17   ` Paul E. McKenney
@ 2021-09-23 12:10   ` Petr Mladek
  2021-09-24 18:57   ` Joel Savitz
  2 siblings, 0 replies; 26+ messages in thread
From: Petr Mladek @ 2021-09-23 12:10 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Wed 2021-09-22 13:05:12, Peter Zijlstra wrote:
> Use rcu_user_{enter,exit}() calls to provide SMP ordering on context
> tracking state stores:
> 
> __context_tracking_exit()
>   __this_cpu_write(context_tracking.state, CONTEXT_KERNEL)
>   rcu_user_exit()
>     rcu_eqs_exit()
>       rcu_dynticks_eqs_eit()
>         rcu_dynticks_inc()
>           atomic_add_return() /* smp_mb */
> 
> __context_tracking_enter()
>   rcu_user_enter()
>     rcu_eqs_enter()
>       rcu_dynticks_eqs_enter()
>         rcu_dynticks_inc()
> 	  atomic_add_return() /* smp_mb */
>   __this_cpu_write(context_tracking.state, state)
> 
> This separates USER/KERNEL state with an smp_mb() on each side,
> therefore, a user of context_tracking_state_cpu() can say the CPU must
> pass through an smp_mb() before changing.
> 
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> ---
>  include/linux/context_tracking_state.h |   12 ++++++++++++
>  kernel/context_tracking.c              |    7 ++++---
>  2 files changed, 16 insertions(+), 3 deletions(-)
> 
> --- a/include/linux/context_tracking_state.h
> +++ b/include/linux/context_tracking_state.h
> @@ -45,11 +45,23 @@ static __always_inline bool context_trac
>  {
>  	return __this_cpu_read(context_tracking.state) == CONTEXT_USER;
>  }
> +
> +static __always_inline bool context_tracking_state_cpu(int cpu)
> +{
> +	struct context_tracking *ct = per_cpu_ptr(&context_tracking);

Missing cpu parameter:

	struct context_tracking *ct = per_cpu_ptr(&context_tracking, cpu);

> +
> +	if (!context_tracking_enabled() || !ct->active)
> +		return CONTEXT_DISABLED;
> +
> +	return ct->state;
> +}

Best Regards,
Petr

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

* Re: [RFC][PATCH 5/7] sched,livepatch: Use wake_up_if_idle()
  2021-09-22 13:05   ` Miroslav Benes
@ 2021-09-23 12:19     ` Petr Mladek
  0 siblings, 0 replies; 26+ messages in thread
From: Petr Mladek @ 2021-09-23 12:19 UTC (permalink / raw)
  To: Miroslav Benes
  Cc: Peter Zijlstra, gor, jpoimboe, jikos, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching, paulmck

On Wed 2021-09-22 15:05:03, Miroslav Benes wrote:
> > @@ -405,8 +405,10 @@ void klp_try_complete_transition(void)
> >  	for_each_possible_cpu(cpu) {
> >  		task = idle_task(cpu);
> >  		if (cpu_online(cpu)) {
> > -			if (!klp_try_switch_task(task))
> > -				complete = false;
> > +			int ret = klp_try_switch_task(task);
> > +			if (ret == -EBUSY)
> > +				wake_up_if_idle(cpu);
> > +			complete = !ret;
> 
> This is broken. You can basically change "complete" only to false (when it 
> applies). This could leave some tasks in the old patching state.

I was a bit confused by Mirek's comment ;-) Anyway, the following works for me:

@@ -406,9 +406,12 @@ void klp_try_complete_transition(void)
 		task = idle_task(cpu);
 		if (cpu_online(cpu)) {
 			int ret = klp_try_switch_task(task);
-			if (ret == -EBUSY)
-				wake_up_if_idle(cpu);
-			complete = !ret;
+			if (ret) {
+				complete = false;
+				/* Make idle task go through the main loop. */
+				if (ret == -EBUSY)
+					wake_up_if_idle(cpu);
+			}
 		} else if (task->patch_state != klp_target_state) {
 			/* offline idle tasks can be switched immediately */
 			clear_tsk_thread_flag(task, TIF_PATCH_PENDING);

Best Regards,
Petr

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

* Re: [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks
  2021-09-22 11:05 ` [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks Peter Zijlstra
@ 2021-09-23 13:14   ` Petr Mladek
  2021-09-23 13:28     ` Peter Zijlstra
  0 siblings, 1 reply; 26+ messages in thread
From: Petr Mladek @ 2021-09-23 13:14 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Wed 2021-09-22 13:05:13, Peter Zijlstra wrote:
> When a task is stuck in NOHZ_FULL usermode, we can simply mark the
> livepatch state complete.
> 
> Signed-off-by: Peter Zijlstra (Intel) <peterz@infradead.org>
> ---
>  kernel/livepatch/transition.c |   13 ++++++++++++-
>  1 file changed, 12 insertions(+), 1 deletion(-)
> 
> --- a/kernel/livepatch/transition.c
> +++ b/kernel/livepatch/transition.c
> @@ -270,13 +270,24 @@ static int klp_check_task(struct task_st
>  {
>  	int ret;
>  
> -	if (task_curr(task))
> +	if (task_curr(task)) {
> +		if (context_tracking_state_cpu(task_cpu(task)) == CONTEXT_USER) {
> +			/*
> +			 * If we observe the CPU being in USER context, they
> +			 * must issue an smp_mb() before doing much kernel
> +			 * space and as such will observe the patched state,
> +			 * mark it clean.
> +			 */
> +			goto complete;

IMHO, this is not safe:

CPU0				CPU1

klp_check_task(A)
  if (context_tracking_state_cpu(task_cpu(task)) == CONTEXT_USER)
     goto complete;

  clear_tsk_thread_flag(task, TIF_PATCH_PENDING);

				# task switching to kernel space
				klp_update_patch_state(A)
				       if (test_and_clear_tsk_thread_flag(task,	TIF_PATCH_PENDING))
				       //false

				# calling kernel code with old task->patch_state

	task->patch_state = klp_target_state;

BANG: CPU0 sets task->patch_state when task A is already running
	kernel code on CPU1.

> +		}
>  		return -EBUSY;
> +	}
>  
>  	ret = klp_check_stack(task, arg);
>  	if (ret)
>  		return ret;
>  
> +complete:
>  	clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
>  	task->patch_state = klp_target_state;

A solution might be to switch ordering and add a barrier here.

>  	return 0;


The code might look like:

static int klp_check_task(struct task_struct *task, void *arg)
{
	int ret;

	if (task_curr(task)) {
		if (context_tracking_state_cpu(task_cpu(task)) == CONTEXT_USER) {
			/*
			 * Task running in USER mode might get switched
			 * immediately. They are switched when entering
			 * kernel code anyway.
			 */
			goto complete;
		}
		return -EBUSY;
	}

	ret = klp_check_stack(task, arg);
	if (ret)
		return ret;

complete:
	WRITE_ONCE(task->patch_state, klp_target_state);
	/*
	 * We switch also tasks running in USER mode here. They must
	 * see the new state before clearing the pending flag.
	 * Otherwise, they might enter kernel mode without switching
	 * the state in klp_update_patch_state().
	 */
	smp_wmb();
	clear_tsk_thread_flag(task, TIF_PATCH_PENDING);

	return 0;
}

The only problem is that the corresponding read barrier is not clear.
It will make more sense if it is paired with some read barrier
in the scheduler after handling TIF flags.

But we should be on the safe side because klp_ftrace_handler() always
does read barrier before reading the state. Though, it is done
there from other reasons.

Best Regards,
Petr

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

* Re: [RFC][PATCH 3/7] sched,livepatch: Use task_try_func()
  2021-09-23 12:05   ` Petr Mladek
@ 2021-09-23 13:17     ` Peter Zijlstra
  2021-09-23 13:47       ` Peter Zijlstra
  0 siblings, 1 reply; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-23 13:17 UTC (permalink / raw)
  To: Petr Mladek
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Thu, Sep 23, 2021 at 02:05:00PM +0200, Petr Mladek wrote:
> On Wed 2021-09-22 13:05:09, Peter Zijlstra wrote:

> > +static int klp_check_task(struct task_struct *task, void *arg)
> 
> Please, call this klp_check_and_switch_task() to make it clear
> that it actually does the switch.

Sure.


> > +	ret = task_try_func(task, klp_check_task, &old_name);
> > +	switch (ret) {
> > +	case -EBUSY:
> > +		pr_debug("%s: %s:%d is running\n",
> > +			 __func__, task->comm, task->pid);
> > +		break;
> > +	case -EINVAL:
> > +		pr_debug("%s: %s:%d has an unreliable stack\n",
> > +			 __func__, task->comm, task->pid);
> > +		break;
> > +	case -EADDRINUSE:
> > +		pr_debug("%s: %s:%d is sleeping on function %s\n",
> > +			 __func__, task->comm, task->pid, old_name);
> > +		break;
> 
> I would prefer to be on the safe side and catch error codes that might
> eventually appear in the future.
> 
> 	case 0:
> 		/* success */
> 		break;

	case -EAGAIN:
		/* task_try_func() raced */
		break;

> 	default:
> 		pr_debug("%s: Unknown error code (%d) when trying to switch %s:%d\n",
> 			 __func__, ret, task->comm, task->pid);
> 
> >  	}

Done.

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

* Re: [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus
  2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
                   ` (6 preceding siblings ...)
  2021-09-22 11:05 ` [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks Peter Zijlstra
@ 2021-09-23 13:22 ` Petr Mladek
  7 siblings, 0 replies; 26+ messages in thread
From: Petr Mladek @ 2021-09-23 13:22 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Wed 2021-09-22 13:05:06, Peter Zijlstra wrote:
> Hi,
> 
> Compile tested only, please consider carefully, esp. the last few patches that
> concern context_tracking and nohz_full.

BTW: The patchset seems to significantly speed up livepatch selftests
     in tools/testing/selftests/livepatch. I haven't measured it.
     But I do not longer have to switch screen and do anything else.

     The transition speed is not our priority. But it is another nice
     win of this patchset. I only hope that we did not introduce
     any race ;-)

Best Regards,
Petr

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

* Re: [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks
  2021-09-23 13:14   ` Petr Mladek
@ 2021-09-23 13:28     ` Peter Zijlstra
  2021-09-24  7:33       ` Petr Mladek
  0 siblings, 1 reply; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-23 13:28 UTC (permalink / raw)
  To: Petr Mladek
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Thu, Sep 23, 2021 at 03:14:48PM +0200, Petr Mladek wrote:

> IMHO, this is not safe:
> 
> CPU0				CPU1
> 
> klp_check_task(A)
>   if (context_tracking_state_cpu(task_cpu(task)) == CONTEXT_USER)
>      goto complete;
> 
>   clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
> 
> 				# task switching to kernel space
> 				klp_update_patch_state(A)
> 				       if (test_and_clear_tsk_thread_flag(task,	TIF_PATCH_PENDING))
> 				       //false
> 
> 				# calling kernel code with old task->patch_state
> 
> 	task->patch_state = klp_target_state;
> 
> BANG: CPU0 sets task->patch_state when task A is already running
> 	kernel code on CPU1.

Why is that a problem? That is, who actually cares about
task->patch_state ? I was under the impression that state was purely for
klp itself, to track which task has observed the new state.

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

* Re: [RFC][PATCH 3/7] sched,livepatch: Use task_try_func()
  2021-09-23 13:17     ` Peter Zijlstra
@ 2021-09-23 13:47       ` Peter Zijlstra
  0 siblings, 0 replies; 26+ messages in thread
From: Peter Zijlstra @ 2021-09-23 13:47 UTC (permalink / raw)
  To: Petr Mladek
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Thu, Sep 23, 2021 at 03:17:17PM +0200, Peter Zijlstra wrote:
> On Thu, Sep 23, 2021 at 02:05:00PM +0200, Petr Mladek wrote:
> > I would prefer to be on the safe side and catch error codes that might
> > eventually appear in the future.
> > 
> > 	case 0:
> > 		/* success */
> > 		break;
> 
> 	case -EAGAIN:
> 		/* task_try_func() raced */
> 		break;

Also, I'll try and see if I can get rid of that one.

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

* Re: [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks
  2021-09-23 13:28     ` Peter Zijlstra
@ 2021-09-24  7:33       ` Petr Mladek
  0 siblings, 0 replies; 26+ messages in thread
From: Petr Mladek @ 2021-09-24  7:33 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, mingo, linux-kernel, joe.lawrence,
	fweisbec, tglx, hca, svens, sumanthk, live-patching, paulmck

On Thu 2021-09-23 15:28:56, Peter Zijlstra wrote:
> On Thu, Sep 23, 2021 at 03:14:48PM +0200, Petr Mladek wrote:
> 
> > IMHO, this is not safe:
> > 
> > CPU0				CPU1
> > 
> > klp_check_task(A)
> >   if (context_tracking_state_cpu(task_cpu(task)) == CONTEXT_USER)
> >      goto complete;
> > 
> >   clear_tsk_thread_flag(task, TIF_PATCH_PENDING);
> > 
> > 				# task switching to kernel space
> > 				klp_update_patch_state(A)
> > 				       if (test_and_clear_tsk_thread_flag(task,	TIF_PATCH_PENDING))
> > 				       //false
> > 
> > 				# calling kernel code with old task->patch_state
> > 
> > 	task->patch_state = klp_target_state;
> > 
> > BANG: CPU0 sets task->patch_state when task A is already running
> > 	kernel code on CPU1.
> 
> Why is that a problem? That is, who actually cares about
> task->patch_state ? I was under the impression that state was purely for
> klp itself, to track which task has observed the new state.

It is the other way. The patch_state is used in klp_ftrace_handler()
to decide which code must be used (old or new).

The state must change only when the given task is _not_ using
any patched function. Hence we do it when:

     + no patched function is on the stack  (needed primary for kthreads)
     + entering/leaving kernel              (reliable way for user space)

See "Consistency model" in Documentation/livepatch/livepatch.rst
for more details.

Best Regards,
Petr

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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-22 11:05 ` [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU Peter Zijlstra
  2021-09-22 15:17   ` Paul E. McKenney
  2021-09-23 12:10   ` Petr Mladek
@ 2021-09-24 18:57   ` Joel Savitz
  2021-09-27 14:33     ` Petr Mladek
  2 siblings, 1 reply; 26+ messages in thread
From: Joel Savitz @ 2021-09-24 18:57 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: gor, jpoimboe, jikos, mbenes, pmladek, mingo, linux-kernel,
	joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching, paulmck

On Wed, Sep 22, 2021 at 01:05:12PM +0200, Peter Zijlstra wrote:
> ---
>  include/linux/context_tracking_state.h |   12 ++++++++++++
>  kernel/context_tracking.c              |    7 ++++---
>  2 files changed, 16 insertions(+), 3 deletions(-)
> 
> --- a/include/linux/context_tracking_state.h
> +++ b/include/linux/context_tracking_state.h
> @@ -45,11 +45,23 @@ static __always_inline bool context_trac
>  {
>  	return __this_cpu_read(context_tracking.state) == CONTEXT_USER;
>  }
> +
> +static __always_inline bool context_tracking_state_cpu(int cpu)
> +{
> +	struct context_tracking *ct = per_cpu_ptr(&context_tracking);
> +
> +	if (!context_tracking_enabled() || !ct->active)
> +		return CONTEXT_DISABLED;
> +
> +	return ct->state;
> +}
> +
>  #else
>  static inline bool context_tracking_in_user(void) { return false; }
>  static inline bool context_tracking_enabled(void) { return false; }
>  static inline bool context_tracking_enabled_cpu(int cpu) { return false; }
>  static inline bool context_tracking_enabled_this_cpu(void) { return false; }
> +static inline bool context_tracking_state_cpu(int cpu) { return CONTEXT_DISABLED; }
>  #endif /* CONFIG_CONTEXT_TRACKING */
>  
>  #endif

Should context_tracking_state_cpu return an enum ctx_state rather than a
bool? It appears to be doing an implicit cast.

I don't know if it possible to run livepatch with
CONFIG_CONTEXT_TRACKING disabled, but if so, then klp_check_task() as
modified by patch 7 will always consider the transition complete even if
the current task is in kernel mode. Also in the general case, the CPU
will consider the task complete if has ctx_state CONTEXT_GUEST though the
condition does not make it explicit.

I'm not sure what the correct behavior should be here as I am not very
experienced with this sybsystem but the patch looks a bit odd to me.

> --- a/kernel/context_tracking.c
> +++ b/kernel/context_tracking.c
> @@ -82,7 +82,7 @@ void noinstr __context_tracking_enter(en
>  				vtime_user_enter(current);
>  				instrumentation_end();
>  			}
> -			rcu_user_enter();
> +			rcu_user_enter(); /* smp_mb */
>  		}
>  		/*
>  		 * Even if context tracking is disabled on this CPU, because it's outside
> @@ -149,12 +149,14 @@ void noinstr __context_tracking_exit(enu
>  		return;
>  
>  	if (__this_cpu_read(context_tracking.state) == state) {
> +		__this_cpu_write(context_tracking.state, CONTEXT_KERNEL);
> +
>  		if (__this_cpu_read(context_tracking.active)) {
>  			/*
>  			 * We are going to run code that may use RCU. Inform
>  			 * RCU core about that (ie: we may need the tick again).
>  			 */
> -			rcu_user_exit();
> +			rcu_user_exit(); /* smp_mb */
>  			if (state == CONTEXT_USER) {
>  				instrumentation_begin();
>  				vtime_user_exit(current);
> @@ -162,7 +164,6 @@ void noinstr __context_tracking_exit(enu
>  				instrumentation_end();
>  			}
>  		}
> -		__this_cpu_write(context_tracking.state, CONTEXT_KERNEL);
>  	}
>  	context_tracking_recursion_exit();
>  }
> 
> 


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

* Re: [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU
  2021-09-24 18:57   ` Joel Savitz
@ 2021-09-27 14:33     ` Petr Mladek
  0 siblings, 0 replies; 26+ messages in thread
From: Petr Mladek @ 2021-09-27 14:33 UTC (permalink / raw)
  To: Joel Savitz
  Cc: Peter Zijlstra, gor, jpoimboe, jikos, mbenes, mingo,
	linux-kernel, joe.lawrence, fweisbec, tglx, hca, svens, sumanthk,
	live-patching, paulmck

On Fri 2021-09-24 14:57:05, Joel Savitz wrote:
> On Wed, Sep 22, 2021 at 01:05:12PM +0200, Peter Zijlstra wrote:
> > ---
> >  include/linux/context_tracking_state.h |   12 ++++++++++++
> >  kernel/context_tracking.c              |    7 ++++---
> >  2 files changed, 16 insertions(+), 3 deletions(-)
> > 
> > --- a/include/linux/context_tracking_state.h
> > +++ b/include/linux/context_tracking_state.h
> > @@ -45,11 +45,23 @@ static __always_inline bool context_trac
> >  {
> >  	return __this_cpu_read(context_tracking.state) == CONTEXT_USER;
> >  }
> > +
> > +static __always_inline bool context_tracking_state_cpu(int cpu)
> > +{
> > +	struct context_tracking *ct = per_cpu_ptr(&context_tracking);
> > +
> > +	if (!context_tracking_enabled() || !ct->active)
> > +		return CONTEXT_DISABLED;
> > +
> > +	return ct->state;
> > +}
> > +
> >  #else
> >  static inline bool context_tracking_in_user(void) { return false; }
> >  static inline bool context_tracking_enabled(void) { return false; }
> >  static inline bool context_tracking_enabled_cpu(int cpu) { return false; }
> >  static inline bool context_tracking_enabled_this_cpu(void) { return false; }
> > +static inline bool context_tracking_state_cpu(int cpu) { return CONTEXT_DISABLED; }
> >  #endif /* CONFIG_CONTEXT_TRACKING */
> >  
> >  #endif
> 
> Should context_tracking_state_cpu return an enum ctx_state rather than a
> bool? It appears to be doing an implicit cast.

Great catch!

> I don't know if it possible to run livepatch with
> CONFIG_CONTEXT_TRACKING disabled,

It should work with CONFIG_CONTEXT_TRACKING. The original code
migrates the task only when it is not running on any CPU and patched
functions are not on the stack. The stack check covers also
the user context.

> modified by patch 7 will always consider the transition complete even if
> the current task is in kernel mode. Also in the general case, the CPU
> will consider the task complete if has ctx_state CONTEXT_GUEST though the
> condition does not make it explicit.

Yup, we should avoid the enum -> bool cast, definitely.

Best Regards,
Petr

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

end of thread, other threads:[~2021-09-27 14:33 UTC | newest]

Thread overview: 26+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-09-22 11:05 [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Peter Zijlstra
2021-09-22 11:05 ` [RFC][PATCH 1/7] sched,rcu: Rework try_invoke_on_locked_down_task() Peter Zijlstra
2021-09-22 11:05 ` [RFC][PATCH 2/7] sched: Fix task_try_func() Peter Zijlstra
2021-09-22 11:05 ` [RFC][PATCH 3/7] sched,livepatch: Use task_try_func() Peter Zijlstra
2021-09-23 12:05   ` Petr Mladek
2021-09-23 13:17     ` Peter Zijlstra
2021-09-23 13:47       ` Peter Zijlstra
2021-09-22 11:05 ` [RFC][PATCH 4/7] sched: Simplify wake_up_*idle*() Peter Zijlstra
2021-09-22 11:05 ` [RFC][PATCH 5/7] sched,livepatch: Use wake_up_if_idle() Peter Zijlstra
2021-09-22 13:05   ` Miroslav Benes
2021-09-23 12:19     ` Petr Mladek
2021-09-22 11:05 ` [RFC][PATCH 6/7] context_tracking: Provide SMP ordering using RCU Peter Zijlstra
2021-09-22 15:17   ` Paul E. McKenney
2021-09-22 19:33     ` Peter Zijlstra
2021-09-22 19:47       ` Peter Zijlstra
2021-09-22 19:59         ` Paul E. McKenney
2021-09-22 19:53       ` Paul E. McKenney
2021-09-22 20:02         ` Peter Zijlstra
2021-09-23 12:10   ` Petr Mladek
2021-09-24 18:57   ` Joel Savitz
2021-09-27 14:33     ` Petr Mladek
2021-09-22 11:05 ` [RFC][PATCH 7/7] livepatch,context_tracking: Avoid disturbing NOHZ_FULL tasks Peter Zijlstra
2021-09-23 13:14   ` Petr Mladek
2021-09-23 13:28     ` Peter Zijlstra
2021-09-24  7:33       ` Petr Mladek
2021-09-23 13:22 ` [RFC][PATCH 0/7] sched,rcu,context_tracking,livepatch: Improve livepatch task transitions for idle and NOHZ_FULL cpus Petr Mladek

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