linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [RFC PATCH 1/8] perf: Save PMU specific data in task_struct
@ 2019-11-28 15:14 kan.liang
  2019-11-28 15:14 ` [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data kan.liang
                   ` (6 more replies)
  0 siblings, 7 replies; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

Some PMU specific data has to be saved/restored during context switch,
e.g. LBR call stack data. Currently, the data is saved in event context
structure, but only for per-process event. For system-wide event,
because of missing the LBR call stack data after context switch, LBR
callstacks are always shorter in comparison to per-process mode.

For example,
  Per-process mode:
  $perf record --call-graph lbr -- taskset -c 0 ./tchain_edit

  -   99.90%    99.86%  tchain_edit  tchain_edit       [.] f3
       99.86% _start
          __libc_start_main
          generic_start_main
          main
          f1
        - f2
             f3

  System-wide mode:
  $perf record --call-graph lbr -a -- taskset -c 0 ./tchain_edit

  -   99.88%    99.82%  tchain_edit  tchain_edit        [.] f3
   - 62.02% main
        f1
        f2
        f3
   - 28.83% f1
      - f2
        f3
   - 28.83% f1
      - f2
           f3
   - 8.88% generic_start_main
        main
        f1
        f2
        f3

It isn't practical to simply allocate the data for system-wide event in
CPU context structure for all tasks. We have no idea which CPU a task
will be scheduled to. The duplicated LBR data has to be maintained on
every CPU context structure. That's a huge waste. Otherwise, the LBR
data still lost if the task is scheduled to another CPU.

Save the pmu specific data in task_struct. The size of pmu specific data
is 788 bytes for LBR call stack. Usually, the overall amount of threads
doesn't exceed a few thousands. For 10K threads, keeping LBR data would
consume additional ~8MB. The additional space will only be allocated
during LBR call stack monitoring. It will be released when the
monitoring is finished.

Furthermore, moving task_ctx_data from perf_event_context to task_struct
can reduce complexity and make things clearer. E.g. perf doesn't need to
swap task_ctx_data on optimized context switch path.
This patch set is just the first step. There could be other
optimization/extension on top of this patch set. E.g. for cgroup
profiling, perf just needs to save/store the LBR call stack information
for tasks in specific cgroup. That could reduce the additional space.
Also, the LBR call stack can be available for software events, or allow
even debugging use cases, like LBRs on crash later.

The data can be shared among events. To sync the writers of
perf_ctx_data RCU pointer, add a lock in task_struct as well.

The size of pmu specific data is saved in struct perf_ctx_data. It's
required when child task allocates the space.
The refcount in struct perf_ctx_data is used to track the users of pmu
specific data.

Reviewed-by: Alexey Budankov <alexey.budankov@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 include/linux/perf_event.h | 28 ++++++++++++++++++++++++++++
 include/linux/sched.h      |  4 ++++
 kernel/events/core.c       |  2 ++
 3 files changed, 34 insertions(+)

diff --git a/include/linux/perf_event.h b/include/linux/perf_event.h
index 34c7c69..a6abefb 100644
--- a/include/linux/perf_event.h
+++ b/include/linux/perf_event.h
@@ -821,6 +821,34 @@ struct perf_event_context {
 	struct rcu_head			rcu_head;
 };
 
+/**
+ * struct perf_ctx_data - PMU specific data for a task
+ * @rcu_head:  To avoid the race on free PMU specific data
+ * @refcount:  To track users
+ * @data_size: Size of PMU specific data
+ * @data:      PMU specific data
+ *
+ * Currently, the struct is only used in Intel LBR call stack mode to
+ * save/restore the call stack of a task on context switches.
+ * The data only be allocated when Intel LBR call stack mode is enabled.
+ * The data will be freed when the mode is disabled. The rcu_head is
+ * used to prevent the race on free the data.
+ * The content of the data will only be accessed in context switch, which
+ * should be protected by rcu_read_lock().
+ *
+ * Careful: Struct perf_ctx_data is added as a pointor in struct task_struct.
+ * When system-wide Intel LBR call stack mode is enabled, a buffer with
+ * constant size will be allocated for each task.
+ * Also, system memory consumption can further grow when the size of
+ * struct perf_ctx_data enlarges.
+ */
+struct perf_ctx_data {
+	struct rcu_head			rcu_head;
+	u64				refcount;
+	size_t				data_size;
+	void				*data;
+};
+
 /*
  * Number of contexts where an event can trigger:
  *	task, softirq, hardirq, nmi.
diff --git a/include/linux/sched.h b/include/linux/sched.h
index 67a1d86..51397e2 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -46,6 +46,7 @@ struct mempolicy;
 struct nameidata;
 struct nsproxy;
 struct perf_event_context;
+struct perf_ctx_data;
 struct pid_namespace;
 struct pipe_inode_info;
 struct rcu_node;
@@ -1059,6 +1060,9 @@ struct task_struct {
 	struct perf_event_context	*perf_event_ctxp[perf_nr_task_contexts];
 	struct mutex			perf_event_mutex;
 	struct list_head		perf_event_list;
+	/* Sync the writers of perf_ctx_data RCU pointer */
+	raw_spinlock_t			perf_ctx_data_lock;
+	struct perf_ctx_data __rcu	*perf_ctx_data;
 #endif
 #ifdef CONFIG_DEBUG_PREEMPT
 	unsigned long			preempt_disable_ip;
diff --git a/kernel/events/core.c b/kernel/events/core.c
index 059ee71..43567d1 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -12321,6 +12321,8 @@ int perf_event_init_task(struct task_struct *child)
 	memset(child->perf_event_ctxp, 0, sizeof(child->perf_event_ctxp));
 	mutex_init(&child->perf_event_mutex);
 	INIT_LIST_HEAD(&child->perf_event_list);
+	child->perf_ctx_data = NULL;
+	raw_spin_lock_init(&child->perf_ctx_data_lock);
 
 	for_each_task_context_nr(ctxn) {
 		ret = perf_event_init_context(child, ctxn);
-- 
2.7.4


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

* [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data
  2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
@ 2019-11-28 15:14 ` kan.liang
  2019-12-02 13:16   ` Peter Zijlstra
  2019-11-28 15:14 ` [RFC PATCH 3/8] perf: Init/fini " kan.liang
                   ` (5 subsequent siblings)
  6 siblings, 1 reply; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

The PMU specific data for the monitored tasks is
allocated/initialized/freed during LBR call stack monitoring. Several
helper functions are provided.

alloc_task_ctx_data_rcu() is used to allocate the perf_ctx_data for a
task when RCU protected perf_ctx_data is NULL. It doesn't update the
refcount if the perf_ctx_data has already been allocated.

init_task_ctx_data_rcu() is similar as alloc_task_ctx_data_rcu(). But it
updates the refcount if the perf_ctx_data was already allocated.

fini_task_ctx_data_rcu() is to free the RCU protected perf_ctx_data when
there are no users, or force flag is set.

A lock is required for these functions, which is used to sync the
writers of perf_ctx_data RCU pointer and refcount.

The functions will be used by the following patch.

Reviewed-by: Alexey Budankov <alexey.budankov@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 kernel/events/core.c | 154 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 154 insertions(+)

diff --git a/kernel/events/core.c b/kernel/events/core.c
index 43567d1..b4976e0 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -4440,6 +4440,160 @@ static void unaccount_freq_event(void)
 		atomic_dec(&nr_freq_events);
 }
 
+static int
+alloc_perf_ctx_data(size_t ctx_size, gfp_t flags,
+		    struct perf_ctx_data **task_ctx_data)
+{
+	struct perf_ctx_data *ctx_data;
+
+	ctx_data = kzalloc(sizeof(struct perf_ctx_data), flags);
+	if (!ctx_data)
+		return -ENOMEM;
+
+	ctx_data->data = kzalloc(ctx_size, flags);
+	if (!ctx_data->data) {
+		kfree(ctx_data);
+		return -ENOMEM;
+	}
+
+	ctx_data->data_size = ctx_size;
+	*task_ctx_data = ctx_data;
+
+	return 0;
+}
+
+static int
+__alloc_task_ctx_data_rcu(struct task_struct *task,
+			  size_t ctx_size, gfp_t flags)
+{
+	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
+	int ret;
+
+	lockdep_assert_held_once(&task->perf_ctx_data_lock);
+
+	ret = alloc_perf_ctx_data(ctx_size, flags, &ctx_data);
+	if (ret)
+		return ret;
+
+	ctx_data->refcount = 1;
+
+	rcu_assign_pointer(task->perf_ctx_data, ctx_data);
+
+	return 0;
+}
+
+/**
+ * alloc perf_ctx_data for a task
+ * @task:        Target Task
+ * @ctx_size:    Size of PMU specific data
+ * @flags:       Allocation flags
+ *
+ * Allocate perf_ctx_data and update the RCU pointer.
+ * If the perf_ctx_data has been allocated, return 0.
+ * Lock is required to sync the writers of perf_ctx_data RCU pointer
+ * and refcount.
+ */
+static int
+alloc_task_ctx_data_rcu(struct task_struct *task,
+			size_t ctx_size, gfp_t flags)
+{
+	unsigned long lock_flags;
+	int ret = 0;
+
+	raw_spin_lock_irqsave(&task->perf_ctx_data_lock, lock_flags);
+
+	if (task->perf_ctx_data)
+		goto unlock;
+
+	ret = __alloc_task_ctx_data_rcu(task, ctx_size, flags);
+
+unlock:
+	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, lock_flags);
+
+	return ret;
+}
+
+static int
+__init_task_ctx_data_rcu(struct task_struct *task, size_t ctx_size, gfp_t flags)
+{
+	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
+
+	lockdep_assert_held_once(&task->perf_ctx_data_lock);
+
+	if (ctx_data) {
+		ctx_data->refcount++;
+		return 0;
+	}
+
+	return __alloc_task_ctx_data_rcu(task, ctx_size, flags);
+}
+
+/**
+ * Init perf_ctx_data for a task
+ * @task:        Target Task
+ * @ctx_size:    Size of PMU specific data
+ * @flags:       Allocation flags
+ *
+ * If the perf_ctx_data has been allocated, update the refcount.
+ * Otherwise, allocate perf_ctx_data and update the RCU pointer.
+ * Lock is required to sync the writers of perf_ctx_data RCU pointer
+ * and refcount.
+ */
+static int
+init_task_ctx_data_rcu(struct task_struct *task, size_t ctx_size, gfp_t flags)
+{
+	unsigned long lock_flags;
+	int ret;
+
+	raw_spin_lock_irqsave(&task->perf_ctx_data_lock, lock_flags);
+	ret = __init_task_ctx_data_rcu(task, ctx_size, flags);
+	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, lock_flags);
+
+	return ret;
+}
+
+static void
+free_perf_ctx_data(struct rcu_head *rcu_head)
+{
+	struct perf_ctx_data *ctx_data;
+
+	ctx_data = container_of(rcu_head, struct perf_ctx_data, rcu_head);
+	kfree(ctx_data->data);
+	kfree(ctx_data);
+}
+
+/**
+ * Free perf_ctx_data RCU pointer for a task
+ * @task:        Target Task
+ * @force:       Unconditionally free perf_ctx_data
+ *
+ * If force is set, free perf_ctx_data unconditionally.
+ * Otherwise, free perf_ctx_data when there are no users.
+ * Lock is required to sync the writers of perf_ctx_data RCU pointer
+ * and refcount.
+ */
+static void
+fini_task_ctx_data_rcu(struct task_struct *task, bool force)
+{
+	struct perf_ctx_data *ctx_data;
+	unsigned long flags;
+
+	raw_spin_lock_irqsave(&task->perf_ctx_data_lock, flags);
+
+	ctx_data = task->perf_ctx_data;
+	if (!ctx_data)
+		goto unlock;
+
+	if (!force && --ctx_data->refcount)
+		goto unlock;
+
+	RCU_INIT_POINTER(task->perf_ctx_data, NULL);
+	call_rcu(&ctx_data->rcu_head, free_perf_ctx_data);
+
+unlock:
+	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, flags);
+}
+
 static void unaccount_event(struct perf_event *event)
 {
 	bool dec = false;
-- 
2.7.4


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

* [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
  2019-11-28 15:14 ` [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data kan.liang
@ 2019-11-28 15:14 ` kan.liang
  2019-12-02 12:40   ` Peter Zijlstra
  2019-11-28 15:14 ` [RFC PATCH 4/8] perf: Supply task information to sched_task() kan.liang
                   ` (4 subsequent siblings)
  6 siblings, 1 reply; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

For per-process event, only allocate the space for current task.
The space will be allocated by the first user.

For system-wide event, allocation for all the existing tasks and
upcoming tasks are required. Add variable nr_task_data_events to track
the number of system-wide event.
In perf_event_alloc(), the space for all the existing tasks will be
allocated.
The space for new tasks will be allocated in perf_event_fork().

The allocation may be failed. For per-process event, it error out.
For system-wide event, it doesn't error out, a debug message will be
dumped to system log instead. LBR callstack may be cutoff for the task
which doesn't have the space allocated.

Reviewed-by: Alexey Budankov <alexey.budankov@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 kernel/events/core.c | 209 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 209 insertions(+)

diff --git a/kernel/events/core.c b/kernel/events/core.c
index b4976e0..e519720 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -387,6 +387,10 @@ static atomic_t nr_switch_events __read_mostly;
 static atomic_t nr_ksymbol_events __read_mostly;
 static atomic_t nr_bpf_events __read_mostly;
 
+/* Track the number of system-wide event which requires pmu specific data */
+static atomic_t nr_task_data_events;
+static DEFINE_RAW_SPINLOCK(task_data_events_lock);
+
 static LIST_HEAD(pmus);
 static DEFINE_MUTEX(pmus_lock);
 static struct srcu_struct pmus_srcu;
@@ -4552,6 +4556,104 @@ init_task_ctx_data_rcu(struct task_struct *task, size_t ctx_size, gfp_t flags)
 	return ret;
 }
 
+static int
+init_task_ctx_data(struct task_struct *task, size_t ctx_size)
+{
+	struct perf_ctx_data *ctx_data;
+	unsigned long flags;
+	int ret = 0;
+
+	raw_spin_lock_irqsave(&task->perf_ctx_data_lock, flags);
+
+	ret = __init_task_ctx_data_rcu(task, ctx_size, GFP_KERNEL);
+	if (ret)
+		goto unlock;
+
+	ctx_data = task->perf_ctx_data;
+	/* System-wide event is active as well */
+	if ((ctx_data->refcount == 1) && atomic_read(&nr_task_data_events))
+		ctx_data->refcount++;
+
+unlock:
+	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, flags);
+	return ret;
+}
+
+static int
+init_system_wide_ctx_data(size_t ctx_size)
+{
+	struct task_struct *g, *p;
+	int failed_alloc = 0;
+	unsigned long flags;
+
+	/*
+	 * Allocate perf_ctx_data for all existing threads by the first event.
+	 *
+	 * The perf_ctx_data for new thread will be allocated in
+	 * perf_event_fork(). The perf_event_fork() is called after the thread
+	 * is added into the tasklist. It guarantees that any new threads will
+	 * not be missed.
+	 */
+	raw_spin_lock_irqsave(&task_data_events_lock, flags);
+	if (atomic_inc_return(&nr_task_data_events) > 1)
+		goto unlock;
+
+	read_lock(&tasklist_lock);
+
+	for_each_process_thread(g, p) {
+		/*
+		 * The PMU specific data may already be allocated by
+		 * per-process event. Need to update refcounter.
+		 * init_task_ctx_data_rcu() is called here.
+		 * Do a quick allocation in first round with GFP_ATOMIC.
+		 */
+		if (init_task_ctx_data_rcu(p, ctx_size, GFP_ATOMIC))
+			failed_alloc++;
+	}
+
+	/*
+	 * Failed to allocate the ctx data for some tasks.
+	 * Repeat the allocation.
+	 */
+	if (!failed_alloc)
+		goto tasklist_unlock;
+
+	failed_alloc = 0;
+	for_each_process_thread(g, p) {
+		/*
+		 * Doesn't need to update refcounter for the task which
+		 * is monitored by per-process event, or new created
+		 * PMU specific data. They are done in first round allocation.
+		 * alloc_task_ctx_data_rcu() is called here.
+		 */
+		if (alloc_task_ctx_data_rcu(p, ctx_size, GFP_KERNEL) &&
+		    ((++failed_alloc) == 1)) {
+			printk(KERN_DEBUG
+			       "Failed to allocate space for LBR callstack for some tasks. "
+			       "Their LBR callstack may be cutoff.\n");
+		}
+	}
+
+tasklist_unlock:
+	read_unlock(&tasklist_lock);
+unlock:
+	raw_spin_unlock_irqrestore(&task_data_events_lock, flags);
+
+	return 0;
+}
+
+static int
+init_perf_ctx_data(struct perf_event *event)
+{
+	struct task_struct *task = event->hw.target;
+	size_t ctx_size = event->pmu->task_ctx_size;
+
+	if (task)
+		return init_task_ctx_data(task, ctx_size);
+	else
+		return init_system_wide_ctx_data(ctx_size);
+}
+
 static void
 free_perf_ctx_data(struct rcu_head *rcu_head)
 {
@@ -4594,6 +4696,40 @@ fini_task_ctx_data_rcu(struct task_struct *task, bool force)
 	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, flags);
 }
 
+static void fini_task_ctx_data(struct task_struct *task)
+{
+	fini_task_ctx_data_rcu(task, false);
+}
+
+static void fini_system_wide_ctx_data(void)
+{
+	struct task_struct *g, *p;
+	unsigned long flags;
+
+	raw_spin_lock_irqsave(&task_data_events_lock, flags);
+	if (!atomic_dec_and_test(&nr_task_data_events))
+		goto unlock;
+
+	read_lock(&tasklist_lock);
+	for_each_process_thread(g, p)
+		fini_task_ctx_data_rcu(p, false);
+
+	read_unlock(&tasklist_lock);
+
+unlock:
+	raw_spin_unlock_irqrestore(&task_data_events_lock, flags);
+}
+
+static void fini_perf_ctx_data(struct perf_event *event)
+{
+	struct task_struct *task = event->hw.target;
+
+	if (task)
+		fini_task_ctx_data(task);
+	else
+		fini_system_wide_ctx_data();
+}
+
 static void unaccount_event(struct perf_event *event)
 {
 	bool dec = false;
@@ -4625,6 +4761,8 @@ static void unaccount_event(struct perf_event *event)
 		atomic_dec(&nr_ksymbol_events);
 	if (event->attr.bpf_event)
 		atomic_dec(&nr_bpf_events);
+	if (event->attach_state & PERF_ATTACH_TASK_DATA)
+		fini_perf_ctx_data(event);
 
 	if (dec) {
 		if (!atomic_add_unless(&perf_sched_count, -1, 1))
@@ -7451,10 +7589,72 @@ static void perf_event_task(struct task_struct *task,
 		       task_ctx);
 }
 
+/*
+ * Allocate data for a new task when profiling system-wide
+ * events which require PMU specific data
+ */
+static void perf_event_alloc_task_data(struct task_struct *child,
+				       struct task_struct *parent)
+{
+	struct perf_ctx_data *ctx_data;
+	size_t ctx_size = 0;
+	unsigned long flags;
+
+	if (!atomic_read(&nr_task_data_events))
+		return;
+
+	rcu_read_lock();
+	ctx_data = rcu_dereference(parent->perf_ctx_data);
+	if (ctx_data)
+		ctx_size = ctx_data->data_size;
+	rcu_read_unlock();
+
+	if (!ctx_size)
+		return;
+
+	/*
+	 * The refcount of a new task may not be updated correctly for some
+	 * rare case as below.
+	 *                 CPU A                              CPU B
+	 *        perf_event_alloc_task_data():    init_system_wide_ctx_data():
+	 *                                           inc(&nr_task_data_events)
+	 *          read(nr_task_data_events)
+	 *          alloc_task_ctx_data_rcu()
+	 *                                           init_task_ctx_data_rcu()
+	 * The refcount of new task is double count.
+	 * Lock is required to prevent the case by serializing the allocation.
+	 */
+	raw_spin_lock_irqsave(&task_data_events_lock, flags);
+
+	/*
+	 * System-wide event may be unaccount when attaching the perf_ctx_data.
+	 * For example,
+	 *                CPU A                              CPU B
+	 *        perf_event_alloc_task_data():    fini_system_wide_ctx_data():
+	 *          read(nr_task_data_events)
+	 *                                         fini_task_ctx_data_rcu()
+	 *          alloc_task_ctx_data_rcu()
+	 *
+	 * The perf_ctx_data may never be freed until the task is terminated.
+	 */
+	if (unlikely(!atomic_read(&nr_task_data_events)))
+		goto unlock;
+
+	if (alloc_task_ctx_data_rcu(child, ctx_size, GFP_KERNEL)) {
+		printk_once(KERN_DEBUG
+			    "LBR callstack may be cutoff for task %s (%d) ctx_size %zu\n",
+			    child->comm, task_pid_nr(child), ctx_size);
+	}
+
+unlock:
+	raw_spin_unlock_irqrestore(&task_data_events_lock, flags);
+}
+
 void perf_event_fork(struct task_struct *task)
 {
 	perf_event_task(task, NULL, 1);
 	perf_event_namespaces(task);
+	perf_event_alloc_task_data(task, current);
 }
 
 /*
@@ -10980,11 +11180,18 @@ perf_event_alloc(struct perf_event_attr *attr, int cpu,
 	if (err)
 		goto err_callchain_buffer;
 
+	if ((event->attach_state & PERF_ATTACH_TASK_DATA) &&
+	    init_perf_ctx_data(event))
+		goto err_task_ctx_data;
+
 	/* symmetric to unaccount_event() in _free_event() */
 	account_event(event);
 
 	return event;
 
+err_task_ctx_data:
+	if (!event->parent && (event->attr.sample_type & PERF_SAMPLE_CALLCHAIN))
+		put_callchain_buffers();
 err_callchain_buffer:
 	if (!event->parent) {
 		if (event->attr.sample_type & PERF_SAMPLE_CALLCHAIN)
@@ -12046,6 +12253,8 @@ void perf_event_exit_task(struct task_struct *child)
 	 * At this point we need to send EXIT events to cpu contexts.
 	 */
 	perf_event_task(child, NULL, 0);
+
+	fini_task_ctx_data_rcu(child, true);
 }
 
 static void perf_free_event(struct perf_event *event,
-- 
2.7.4


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

* [RFC PATCH 4/8] perf: Supply task information to sched_task()
  2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
  2019-11-28 15:14 ` [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data kan.liang
  2019-11-28 15:14 ` [RFC PATCH 3/8] perf: Init/fini " kan.liang
@ 2019-11-28 15:14 ` kan.liang
  2019-11-28 15:14 ` [RFC PATCH 5/8] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode kan.liang
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

To save/restore LBR call stack data in system-wide mode, the task_struct
information is required.

Extend the parameters of sched_task() to supply task_struct information.

When schedule in, the LBR call stack data for new task will be restored.
When schedule out, the LBR call stack data for old task will be saved.
Only need to pass the required task_struct information.

Reviewed-by: Andi Kleen <ak@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 arch/powerpc/perf/core-book3s.c | 8 ++++++--
 arch/x86/events/core.c          | 5 +++--
 arch/x86/events/intel/core.c    | 4 ++--
 arch/x86/events/intel/lbr.c     | 3 ++-
 arch/x86/events/perf_event.h    | 5 +++--
 include/linux/perf_event.h      | 2 +-
 kernel/events/core.c            | 2 +-
 7 files changed, 18 insertions(+), 11 deletions(-)

diff --git a/arch/powerpc/perf/core-book3s.c b/arch/powerpc/perf/core-book3s.c
index 4860462..99cb809 100644
--- a/arch/powerpc/perf/core-book3s.c
+++ b/arch/powerpc/perf/core-book3s.c
@@ -126,7 +126,10 @@ static unsigned long ebb_switch_in(bool ebb, struct cpu_hw_events *cpuhw)
 
 static inline void power_pmu_bhrb_enable(struct perf_event *event) {}
 static inline void power_pmu_bhrb_disable(struct perf_event *event) {}
-static void power_pmu_sched_task(struct perf_event_context *ctx, bool sched_in) {}
+static void power_pmu_sched_task(struct perf_event_context *ctx,
+				 struct task_struct *task, bool sched_in)
+{
+}
 static inline void power_pmu_bhrb_read(struct perf_event *event, struct cpu_hw_events *cpuhw) {}
 static void pmao_restore_workaround(bool ebb) { }
 #endif /* CONFIG_PPC32 */
@@ -403,7 +406,8 @@ static void power_pmu_bhrb_disable(struct perf_event *event)
 /* Called from ctxsw to prevent one process's branch entries to
  * mingle with the other process's entries during context switch.
  */
-static void power_pmu_sched_task(struct perf_event_context *ctx, bool sched_in)
+static void power_pmu_sched_task(struct perf_event_context *ctx,
+				 struct task_struct *task, bool sched_in)
 {
 	if (!ppmu->bhrb_nr)
 		return;
diff --git a/arch/x86/events/core.c b/arch/x86/events/core.c
index 6e3f0c1..3874a2d 100644
--- a/arch/x86/events/core.c
+++ b/arch/x86/events/core.c
@@ -2237,10 +2237,11 @@ static const struct attribute_group *x86_pmu_attr_groups[] = {
 	NULL,
 };
 
-static void x86_pmu_sched_task(struct perf_event_context *ctx, bool sched_in)
+static void x86_pmu_sched_task(struct perf_event_context *ctx,
+			       struct task_struct *task, bool sched_in)
 {
 	if (x86_pmu.sched_task)
-		x86_pmu.sched_task(ctx, sched_in);
+		x86_pmu.sched_task(ctx, task, sched_in);
 }
 
 static void x86_pmu_swap_task_ctx(struct perf_event_context *prev,
diff --git a/arch/x86/events/intel/core.c b/arch/x86/events/intel/core.c
index dc64b16..439306b 100644
--- a/arch/x86/events/intel/core.c
+++ b/arch/x86/events/intel/core.c
@@ -3814,10 +3814,10 @@ static void intel_pmu_cpu_dead(int cpu)
 }
 
 static void intel_pmu_sched_task(struct perf_event_context *ctx,
-				 bool sched_in)
+				 struct task_struct *task, bool sched_in)
 {
 	intel_pmu_pebs_sched_task(ctx, sched_in);
-	intel_pmu_lbr_sched_task(ctx, sched_in);
+	intel_pmu_lbr_sched_task(ctx, task, sched_in);
 }
 
 static void intel_pmu_swap_task_ctx(struct perf_event_context *prev,
diff --git a/arch/x86/events/intel/lbr.c b/arch/x86/events/intel/lbr.c
index 534c7660..dbf31f9 100644
--- a/arch/x86/events/intel/lbr.c
+++ b/arch/x86/events/intel/lbr.c
@@ -440,7 +440,8 @@ void intel_pmu_lbr_swap_task_ctx(struct perf_event_context *prev,
 	     next_ctx_data->lbr_callstack_users);
 }
 
-void intel_pmu_lbr_sched_task(struct perf_event_context *ctx, bool sched_in)
+void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
+			      struct task_struct *task, bool sched_in)
 {
 	struct cpu_hw_events *cpuc = this_cpu_ptr(&cpu_hw_events);
 	struct x86_perf_task_context *task_ctx;
diff --git a/arch/x86/events/perf_event.h b/arch/x86/events/perf_event.h
index 930611d..55c4812 100644
--- a/arch/x86/events/perf_event.h
+++ b/arch/x86/events/perf_event.h
@@ -639,7 +639,7 @@ struct x86_pmu {
 
 	void		(*check_microcode)(void);
 	void		(*sched_task)(struct perf_event_context *ctx,
-				      bool sched_in);
+				      struct task_struct *task, bool sched_in);
 
 	/*
 	 * Intel Arch Perfmon v2+
@@ -1027,7 +1027,8 @@ void intel_ds_init(void);
 void intel_pmu_lbr_swap_task_ctx(struct perf_event_context *prev,
 				 struct perf_event_context *next);
 
-void intel_pmu_lbr_sched_task(struct perf_event_context *ctx, bool sched_in);
+void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
+			      struct task_struct *task, bool sched_in);
 
 u64 lbr_from_signext_quirk_wr(u64 val);
 
diff --git a/include/linux/perf_event.h b/include/linux/perf_event.h
index a6abefb..56d5fea 100644
--- a/include/linux/perf_event.h
+++ b/include/linux/perf_event.h
@@ -406,7 +406,7 @@ struct pmu {
 	 * context-switches callback
 	 */
 	void (*sched_task)		(struct perf_event_context *ctx,
-					bool sched_in);
+					 struct task_struct *task, bool sched_in);
 	/*
 	 * PMU specific data size
 	 */
diff --git a/kernel/events/core.c b/kernel/events/core.c
index e519720..9b7aa0d 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -3335,7 +3335,7 @@ static void perf_pmu_sched_task(struct task_struct *prev,
 		perf_ctx_lock(cpuctx, cpuctx->task_ctx);
 		perf_pmu_disable(pmu);
 
-		pmu->sched_task(cpuctx->task_ctx, sched_in);
+		pmu->sched_task(cpuctx->task_ctx, sched_in ? next : prev, sched_in);
 
 		perf_pmu_enable(pmu);
 		perf_ctx_unlock(cpuctx, cpuctx->task_ctx);
-- 
2.7.4


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

* [RFC PATCH 5/8] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode
  2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
                   ` (2 preceding siblings ...)
  2019-11-28 15:14 ` [RFC PATCH 4/8] perf: Supply task information to sched_task() kan.liang
@ 2019-11-28 15:14 ` kan.liang
  2019-11-28 15:14 ` [RFC PATCH 6/8] perf/x86: Remove swap_task_ctx() kan.liang
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

In system-wide mode, LBR callstacks are shorter in comparison to
per-process mode.

LBR MSRs are reset during context switch in system-wide mode. For LBR
call stack, the LBRs should be always saved/restored during context
switch.

Use the space in task_struct to save/restore the LBR call stack data.

For system-wide event, it's unnecessagy to update the
lbr_callstack_users for each threads. Add a variable in x86_pmu to
indicate if the system-wide event is active.

Fixes: 76cb2c617f12 ("perf/x86/intel: Save/restore LBR stack during context switch")
Reported-by: Alexey Budankov <alexey.budankov@linux.intel.com>
Debugged-by: Alexey Budankov <alexey.budankov@linux.intel.com>
Reviewed-by: Andi Kleen <ak@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 arch/x86/events/intel/lbr.c  | 57 ++++++++++++++++++++++++++++++++++----------
 arch/x86/events/perf_event.h |  1 +
 2 files changed, 45 insertions(+), 13 deletions(-)

diff --git a/arch/x86/events/intel/lbr.c b/arch/x86/events/intel/lbr.c
index dbf31f9..855628a 100644
--- a/arch/x86/events/intel/lbr.c
+++ b/arch/x86/events/intel/lbr.c
@@ -337,6 +337,12 @@ static inline u64 rdlbr_to(unsigned int idx)
 	return val;
 }
 
+static bool has_lbr_callstack_users(struct x86_perf_task_context *task_ctx)
+{
+	return task_ctx->lbr_callstack_users ||
+	       x86_pmu.lbr_callstack_users;
+}
+
 static void __intel_pmu_lbr_restore(struct x86_perf_task_context *task_ctx)
 {
 	struct cpu_hw_events *cpuc = this_cpu_ptr(&cpu_hw_events);
@@ -344,7 +350,7 @@ static void __intel_pmu_lbr_restore(struct x86_perf_task_context *task_ctx)
 	unsigned lbr_idx, mask;
 	u64 tos;
 
-	if (task_ctx->lbr_callstack_users == 0 ||
+	if (!has_lbr_callstack_users(task_ctx) ||
 	    task_ctx->lbr_stack_state == LBR_NONE) {
 		intel_pmu_lbr_reset();
 		return;
@@ -392,7 +398,7 @@ static void __intel_pmu_lbr_save(struct x86_perf_task_context *task_ctx)
 	u64 tos, from;
 	int i;
 
-	if (task_ctx->lbr_callstack_users == 0) {
+	if (!has_lbr_callstack_users(task_ctx)) {
 		task_ctx->lbr_stack_state = LBR_NONE;
 		return;
 	}
@@ -445,6 +451,7 @@ void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
 {
 	struct cpu_hw_events *cpuc = this_cpu_ptr(&cpu_hw_events);
 	struct x86_perf_task_context *task_ctx;
+	struct perf_ctx_data *ctx_data;
 
 	if (!cpuc->lbr_users)
 		return;
@@ -454,15 +461,18 @@ void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
 	 * the task was scheduled out, restore the stack. Otherwise flush
 	 * the LBR stack.
 	 */
-	task_ctx = ctx ? ctx->task_ctx_data : NULL;
+	rcu_read_lock();
+	ctx_data = rcu_dereference(task->perf_ctx_data);
+	task_ctx = ctx_data ? (struct x86_perf_task_context *) ctx_data->data : NULL;
 	if (task_ctx) {
 		if (sched_in)
 			__intel_pmu_lbr_restore(task_ctx);
 		else
 			__intel_pmu_lbr_save(task_ctx);
+		rcu_read_unlock();
 		return;
 	}
-
+	rcu_read_unlock();
 	/*
 	 * Since a context switch can flip the address space and LBR entries
 	 * are not tagged with an identifier, we need to wipe the LBR, even for
@@ -481,16 +491,27 @@ static inline bool branch_user_callstack(unsigned br_sel)
 void intel_pmu_lbr_add(struct perf_event *event)
 {
 	struct cpu_hw_events *cpuc = this_cpu_ptr(&cpu_hw_events);
-	struct x86_perf_task_context *task_ctx;
 
 	if (!x86_pmu.lbr_nr)
 		return;
 
 	cpuc->br_sel = event->hw.branch_reg.reg;
 
-	if (branch_user_callstack(cpuc->br_sel) && event->ctx->task_ctx_data) {
-		task_ctx = event->ctx->task_ctx_data;
-		task_ctx->lbr_callstack_users++;
+	if (branch_user_callstack(cpuc->br_sel)) {
+		if (event->attach_state & PERF_ATTACH_TASK) {
+			struct x86_perf_task_context *task_ctx;
+			struct task_struct *task = event->hw.target;
+			struct perf_ctx_data *ctx_data;
+
+			rcu_read_lock();
+			ctx_data = rcu_dereference(task->perf_ctx_data);
+			if (ctx_data) {
+				task_ctx = (struct x86_perf_task_context *)ctx_data->data;
+				task_ctx->lbr_callstack_users++;
+			}
+			rcu_read_unlock();
+		} else
+			x86_pmu.lbr_callstack_users++;
 	}
 
 	/*
@@ -522,15 +543,25 @@ void intel_pmu_lbr_add(struct perf_event *event)
 void intel_pmu_lbr_del(struct perf_event *event)
 {
 	struct cpu_hw_events *cpuc = this_cpu_ptr(&cpu_hw_events);
-	struct x86_perf_task_context *task_ctx;
 
 	if (!x86_pmu.lbr_nr)
 		return;
 
-	if (branch_user_callstack(cpuc->br_sel) &&
-	    event->ctx->task_ctx_data) {
-		task_ctx = event->ctx->task_ctx_data;
-		task_ctx->lbr_callstack_users--;
+	if (branch_user_callstack(cpuc->br_sel)) {
+		if (event->attach_state & PERF_ATTACH_TASK) {
+			struct task_struct *task = event->hw.target;
+			struct x86_perf_task_context *task_ctx;
+			struct perf_ctx_data *ctx_data;
+
+			rcu_read_lock();
+			ctx_data = rcu_dereference(task->perf_ctx_data);
+			if (ctx_data) {
+				task_ctx = (struct x86_perf_task_context *)ctx_data->data;
+				task_ctx->lbr_callstack_users--;
+			}
+			rcu_read_unlock();
+		} else
+			x86_pmu.lbr_callstack_users--;
 	}
 
 	if (x86_pmu.intel_cap.pebs_baseline && event->attr.precise_ip > 0)
diff --git a/arch/x86/events/perf_event.h b/arch/x86/events/perf_event.h
index 55c4812..b8b7280 100644
--- a/arch/x86/events/perf_event.h
+++ b/arch/x86/events/perf_event.h
@@ -674,6 +674,7 @@ struct x86_pmu {
 	int		lbr_nr;			   /* hardware stack size */
 	u64		lbr_sel_mask;		   /* LBR_SELECT valid bits */
 	const int	*lbr_sel_map;		   /* lbr_select mappings */
+	u64		lbr_callstack_users;	   /* lbr callstack system wide users */
 	bool		lbr_double_abort;	   /* duplicated lbr aborts */
 	bool		lbr_pt_coexist;		   /* (LBR|BTS) may coexist with PT */
 
-- 
2.7.4


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

* [RFC PATCH 6/8] perf/x86: Remove swap_task_ctx()
  2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
                   ` (3 preceding siblings ...)
  2019-11-28 15:14 ` [RFC PATCH 5/8] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode kan.liang
@ 2019-11-28 15:14 ` kan.liang
  2019-11-28 15:14 ` [RFC PATCH 7/8] perf: Clean up pmu specific data kan.liang
  2019-11-28 15:14 ` [RFC PATCH 8/8] perf: Clean up event context from sched_task() kan.liang
  6 siblings, 0 replies; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

The pmu specific data is saved in task_struct now. It doesn't need to
swap between context.

Remove swap_task_ctx() support.

Reviewed-by: Andi Kleen <ak@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 arch/x86/events/core.c       |  8 --------
 arch/x86/events/intel/core.c |  7 -------
 arch/x86/events/intel/lbr.c  | 23 -----------------------
 arch/x86/events/perf_event.h | 11 -----------
 4 files changed, 49 deletions(-)

diff --git a/arch/x86/events/core.c b/arch/x86/events/core.c
index 3874a2d..7046a59 100644
--- a/arch/x86/events/core.c
+++ b/arch/x86/events/core.c
@@ -2244,13 +2244,6 @@ static void x86_pmu_sched_task(struct perf_event_context *ctx,
 		x86_pmu.sched_task(ctx, task, sched_in);
 }
 
-static void x86_pmu_swap_task_ctx(struct perf_event_context *prev,
-				  struct perf_event_context *next)
-{
-	if (x86_pmu.swap_task_ctx)
-		x86_pmu.swap_task_ctx(prev, next);
-}
-
 void perf_check_microcode(void)
 {
 	if (x86_pmu.check_microcode)
@@ -2305,7 +2298,6 @@ static struct pmu pmu = {
 	.event_idx		= x86_pmu_event_idx,
 	.sched_task		= x86_pmu_sched_task,
 	.task_ctx_size          = sizeof(struct x86_perf_task_context),
-	.swap_task_ctx		= x86_pmu_swap_task_ctx,
 	.check_period		= x86_pmu_check_period,
 
 	.aux_output_match	= x86_pmu_aux_output_match,
diff --git a/arch/x86/events/intel/core.c b/arch/x86/events/intel/core.c
index 439306b..bd18c83 100644
--- a/arch/x86/events/intel/core.c
+++ b/arch/x86/events/intel/core.c
@@ -3820,12 +3820,6 @@ static void intel_pmu_sched_task(struct perf_event_context *ctx,
 	intel_pmu_lbr_sched_task(ctx, task, sched_in);
 }
 
-static void intel_pmu_swap_task_ctx(struct perf_event_context *prev,
-				    struct perf_event_context *next)
-{
-	intel_pmu_lbr_swap_task_ctx(prev, next);
-}
-
 static int intel_pmu_check_period(struct perf_event *event, u64 value)
 {
 	return intel_pmu_has_bts_period(event, value) ? -EINVAL : 0;
@@ -3961,7 +3955,6 @@ static __initconst const struct x86_pmu intel_pmu = {
 
 	.guest_get_msrs		= intel_guest_get_msrs,
 	.sched_task		= intel_pmu_sched_task,
-	.swap_task_ctx		= intel_pmu_swap_task_ctx,
 
 	.check_period		= intel_pmu_check_period,
 
diff --git a/arch/x86/events/intel/lbr.c b/arch/x86/events/intel/lbr.c
index 855628a..20c1d7e 100644
--- a/arch/x86/events/intel/lbr.c
+++ b/arch/x86/events/intel/lbr.c
@@ -423,29 +423,6 @@ static void __intel_pmu_lbr_save(struct x86_perf_task_context *task_ctx)
 	cpuc->last_log_id = ++task_ctx->log_id;
 }
 
-void intel_pmu_lbr_swap_task_ctx(struct perf_event_context *prev,
-				 struct perf_event_context *next)
-{
-	struct x86_perf_task_context *prev_ctx_data, *next_ctx_data;
-
-	swap(prev->task_ctx_data, next->task_ctx_data);
-
-	/*
-	 * Architecture specific synchronization makes sense in
-	 * case both prev->task_ctx_data and next->task_ctx_data
-	 * pointers are allocated.
-	 */
-
-	prev_ctx_data = next->task_ctx_data;
-	next_ctx_data = prev->task_ctx_data;
-
-	if (!prev_ctx_data || !next_ctx_data)
-		return;
-
-	swap(prev_ctx_data->lbr_callstack_users,
-	     next_ctx_data->lbr_callstack_users);
-}
-
 void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
 			      struct task_struct *task, bool sched_in)
 {
diff --git a/arch/x86/events/perf_event.h b/arch/x86/events/perf_event.h
index b8b7280..ed287ba 100644
--- a/arch/x86/events/perf_event.h
+++ b/arch/x86/events/perf_event.h
@@ -684,14 +684,6 @@ struct x86_pmu {
 	atomic_t	lbr_exclusive[x86_lbr_exclusive_max];
 
 	/*
-	 * perf task context (i.e. struct perf_event_context::task_ctx_data)
-	 * switch helper to bridge calls from perf/core to perf/x86.
-	 * See struct pmu::swap_task_ctx() usage for examples;
-	 */
-	void		(*swap_task_ctx)(struct perf_event_context *prev,
-					 struct perf_event_context *next);
-
-	/*
 	 * AMD bits
 	 */
 	unsigned int	amd_nb_constraints : 1;
@@ -1025,9 +1017,6 @@ void intel_pmu_store_pebs_lbrs(struct pebs_lbr *lbr);
 
 void intel_ds_init(void);
 
-void intel_pmu_lbr_swap_task_ctx(struct perf_event_context *prev,
-				 struct perf_event_context *next);
-
 void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
 			      struct task_struct *task, bool sched_in);
 
-- 
2.7.4


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

* [RFC PATCH 7/8] perf: Clean up pmu specific data
  2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
                   ` (4 preceding siblings ...)
  2019-11-28 15:14 ` [RFC PATCH 6/8] perf/x86: Remove swap_task_ctx() kan.liang
@ 2019-11-28 15:14 ` kan.liang
  2019-11-28 15:14 ` [RFC PATCH 8/8] perf: Clean up event context from sched_task() kan.liang
  6 siblings, 0 replies; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

The pmu specific data is saved in task_struct now. Remove it from event
context structure.

Remove swap_task_ctx() as well.

Reviewed-by: Andi Kleen <ak@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 include/linux/perf_event.h | 11 ----------
 kernel/events/core.c       | 53 ++--------------------------------------------
 2 files changed, 2 insertions(+), 62 deletions(-)

diff --git a/include/linux/perf_event.h b/include/linux/perf_event.h
index 56d5fea..44280d7 100644
--- a/include/linux/perf_event.h
+++ b/include/linux/perf_event.h
@@ -413,16 +413,6 @@ struct pmu {
 	size_t				task_ctx_size;
 
 	/*
-	 * PMU specific parts of task perf event context (i.e. ctx->task_ctx_data)
-	 * can be synchronized using this function. See Intel LBR callstack support
-	 * implementation and Perf core context switch handling callbacks for usage
-	 * examples.
-	 */
-	void (*swap_task_ctx)		(struct perf_event_context *prev,
-					 struct perf_event_context *next);
-					/* optional */
-
-	/*
 	 * Set up pmu-private data structures for an AUX area
 	 */
 	void *(*setup_aux)		(struct perf_event *event, void **pages,
@@ -817,7 +807,6 @@ struct perf_event_context {
 #ifdef CONFIG_CGROUP_PERF
 	int				nr_cgroups;	 /* cgroup evts */
 #endif
-	void				*task_ctx_data; /* pmu specific data */
 	struct rcu_head			rcu_head;
 };
 
diff --git a/kernel/events/core.c b/kernel/events/core.c
index 9b7aa0d..037f360 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -1184,7 +1184,6 @@ static void free_ctx(struct rcu_head *head)
 	struct perf_event_context *ctx;
 
 	ctx = container_of(head, struct perf_event_context, rcu_head);
-	kfree(ctx->task_ctx_data);
 	kfree(ctx);
 }
 
@@ -3242,28 +3241,14 @@ static void perf_event_context_sched_out(struct task_struct *task, int ctxn,
 		raw_spin_lock(&ctx->lock);
 		raw_spin_lock_nested(&next_ctx->lock, SINGLE_DEPTH_NESTING);
 		if (context_equiv(ctx, next_ctx)) {
-			struct pmu *pmu = ctx->pmu;
-
 			WRITE_ONCE(ctx->task, next);
 			WRITE_ONCE(next_ctx->task, task);
 
 			/*
-			 * PMU specific parts of task perf context can require
-			 * additional synchronization. As an example of such
-			 * synchronization see implementation details of Intel
-			 * LBR call stack data profiling;
-			 */
-			if (pmu->swap_task_ctx)
-				pmu->swap_task_ctx(ctx, next_ctx);
-			else
-				swap(ctx->task_ctx_data, next_ctx->task_ctx_data);
-
-			/*
 			 * RCU_INIT_POINTER here is safe because we've not
 			 * modified the ctx and the above modification of
-			 * ctx->task and ctx->task_ctx_data are immaterial
-			 * since those values are always verified under
-			 * ctx->lock which we're now holding.
+			 * ctx->task is immaterial since this value is always
+			 * verified under ctx->lock which we're now holding.
 			 */
 			RCU_INIT_POINTER(task->perf_event_ctxp[ctxn], next_ctx);
 			RCU_INIT_POINTER(next->perf_event_ctxp[ctxn], ctx);
@@ -4271,7 +4256,6 @@ find_get_context(struct pmu *pmu, struct task_struct *task,
 {
 	struct perf_event_context *ctx, *clone_ctx = NULL;
 	struct perf_cpu_context *cpuctx;
-	void *task_ctx_data = NULL;
 	unsigned long flags;
 	int ctxn, err;
 	int cpu = event->cpu;
@@ -4295,24 +4279,12 @@ find_get_context(struct pmu *pmu, struct task_struct *task,
 	if (ctxn < 0)
 		goto errout;
 
-	if (event->attach_state & PERF_ATTACH_TASK_DATA) {
-		task_ctx_data = kzalloc(pmu->task_ctx_size, GFP_KERNEL);
-		if (!task_ctx_data) {
-			err = -ENOMEM;
-			goto errout;
-		}
-	}
-
 retry:
 	ctx = perf_lock_task_context(task, ctxn, &flags);
 	if (ctx) {
 		clone_ctx = unclone_ctx(ctx);
 		++ctx->pin_count;
 
-		if (task_ctx_data && !ctx->task_ctx_data) {
-			ctx->task_ctx_data = task_ctx_data;
-			task_ctx_data = NULL;
-		}
 		raw_spin_unlock_irqrestore(&ctx->lock, flags);
 
 		if (clone_ctx)
@@ -4323,11 +4295,6 @@ find_get_context(struct pmu *pmu, struct task_struct *task,
 		if (!ctx)
 			goto errout;
 
-		if (task_ctx_data) {
-			ctx->task_ctx_data = task_ctx_data;
-			task_ctx_data = NULL;
-		}
-
 		err = 0;
 		mutex_lock(&task->perf_event_mutex);
 		/*
@@ -4354,11 +4321,9 @@ find_get_context(struct pmu *pmu, struct task_struct *task,
 		}
 	}
 
-	kfree(task_ctx_data);
 	return ctx;
 
 errout:
-	kfree(task_ctx_data);
 	return ERR_PTR(err);
 }
 
@@ -12408,19 +12373,6 @@ inherit_event(struct perf_event *parent_event,
 	if (IS_ERR(child_event))
 		return child_event;
 
-
-	if ((child_event->attach_state & PERF_ATTACH_TASK_DATA) &&
-	    !child_ctx->task_ctx_data) {
-		struct pmu *pmu = child_event->pmu;
-
-		child_ctx->task_ctx_data = kzalloc(pmu->task_ctx_size,
-						   GFP_KERNEL);
-		if (!child_ctx->task_ctx_data) {
-			free_event(child_event);
-			return ERR_PTR(-ENOMEM);
-		}
-	}
-
 	/*
 	 * is_orphaned_event() and list_add_tail(&parent_event->child_list)
 	 * must be under the same lock in order to serialize against
@@ -12431,7 +12383,6 @@ inherit_event(struct perf_event *parent_event,
 	if (is_orphaned_event(parent_event) ||
 	    !atomic_long_inc_not_zero(&parent_event->refcount)) {
 		mutex_unlock(&parent_event->child_mutex);
-		/* task_ctx_data is freed with child_ctx */
 		free_event(child_event);
 		return NULL;
 	}
-- 
2.7.4


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

* [RFC PATCH 8/8] perf: Clean up event context from sched_task()
  2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
                   ` (5 preceding siblings ...)
  2019-11-28 15:14 ` [RFC PATCH 7/8] perf: Clean up pmu specific data kan.liang
@ 2019-11-28 15:14 ` kan.liang
  6 siblings, 0 replies; 23+ messages in thread
From: kan.liang @ 2019-11-28 15:14 UTC (permalink / raw)
  To: peterz, mingo, acme, tglx, bp, linux-kernel
  Cc: eranian, alexey.budankov, vitaly.slobodskoy, ak, Kan Liang

From: Kan Liang <kan.liang@linux.intel.com>

No one uses event context in sched_task() anymore.

Remove the event context parameters from sched_task().
ctx_lock is useless as well.

Reviewed-by: Andi Kleen <ak@linux.intel.com>
Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---
 arch/powerpc/perf/core-book3s.c | 8 ++------
 arch/x86/events/core.c          | 5 ++---
 arch/x86/events/intel/core.c    | 7 +++----
 arch/x86/events/intel/ds.c      | 2 +-
 arch/x86/events/intel/lbr.c     | 3 +--
 arch/x86/events/perf_event.h    | 8 +++-----
 include/linux/perf_event.h      | 3 +--
 kernel/events/core.c            | 4 +---
 8 files changed, 14 insertions(+), 26 deletions(-)

diff --git a/arch/powerpc/perf/core-book3s.c b/arch/powerpc/perf/core-book3s.c
index 99cb809..3c3631d 100644
--- a/arch/powerpc/perf/core-book3s.c
+++ b/arch/powerpc/perf/core-book3s.c
@@ -126,10 +126,7 @@ static unsigned long ebb_switch_in(bool ebb, struct cpu_hw_events *cpuhw)
 
 static inline void power_pmu_bhrb_enable(struct perf_event *event) {}
 static inline void power_pmu_bhrb_disable(struct perf_event *event) {}
-static void power_pmu_sched_task(struct perf_event_context *ctx,
-				 struct task_struct *task, bool sched_in)
-{
-}
+static void power_pmu_sched_task(struct task_struct *task, bool sched_in) {}
 static inline void power_pmu_bhrb_read(struct perf_event *event, struct cpu_hw_events *cpuhw) {}
 static void pmao_restore_workaround(bool ebb) { }
 #endif /* CONFIG_PPC32 */
@@ -406,8 +403,7 @@ static void power_pmu_bhrb_disable(struct perf_event *event)
 /* Called from ctxsw to prevent one process's branch entries to
  * mingle with the other process's entries during context switch.
  */
-static void power_pmu_sched_task(struct perf_event_context *ctx,
-				 struct task_struct *task, bool sched_in)
+static void power_pmu_sched_task(struct task_struct *task, bool sched_in)
 {
 	if (!ppmu->bhrb_nr)
 		return;
diff --git a/arch/x86/events/core.c b/arch/x86/events/core.c
index 7046a59..43d0918 100644
--- a/arch/x86/events/core.c
+++ b/arch/x86/events/core.c
@@ -2237,11 +2237,10 @@ static const struct attribute_group *x86_pmu_attr_groups[] = {
 	NULL,
 };
 
-static void x86_pmu_sched_task(struct perf_event_context *ctx,
-			       struct task_struct *task, bool sched_in)
+static void x86_pmu_sched_task(struct task_struct *task, bool sched_in)
 {
 	if (x86_pmu.sched_task)
-		x86_pmu.sched_task(ctx, task, sched_in);
+		x86_pmu.sched_task(task, sched_in);
 }
 
 void perf_check_microcode(void)
diff --git a/arch/x86/events/intel/core.c b/arch/x86/events/intel/core.c
index bd18c83..f43ec24 100644
--- a/arch/x86/events/intel/core.c
+++ b/arch/x86/events/intel/core.c
@@ -3813,11 +3813,10 @@ static void intel_pmu_cpu_dead(int cpu)
 	intel_cpuc_finish(&per_cpu(cpu_hw_events, cpu));
 }
 
-static void intel_pmu_sched_task(struct perf_event_context *ctx,
-				 struct task_struct *task, bool sched_in)
+static void intel_pmu_sched_task(struct task_struct *task, bool sched_in)
 {
-	intel_pmu_pebs_sched_task(ctx, sched_in);
-	intel_pmu_lbr_sched_task(ctx, task, sched_in);
+	intel_pmu_pebs_sched_task(sched_in);
+	intel_pmu_lbr_sched_task(task, sched_in);
 }
 
 static int intel_pmu_check_period(struct perf_event *event, u64 value)
diff --git a/arch/x86/events/intel/ds.c b/arch/x86/events/intel/ds.c
index ce83950..d22af4b 100644
--- a/arch/x86/events/intel/ds.c
+++ b/arch/x86/events/intel/ds.c
@@ -908,7 +908,7 @@ static inline bool pebs_needs_sched_cb(struct cpu_hw_events *cpuc)
 	return cpuc->n_pebs && (cpuc->n_pebs == cpuc->n_large_pebs);
 }
 
-void intel_pmu_pebs_sched_task(struct perf_event_context *ctx, bool sched_in)
+void intel_pmu_pebs_sched_task(bool sched_in)
 {
 	struct cpu_hw_events *cpuc = this_cpu_ptr(&cpu_hw_events);
 
diff --git a/arch/x86/events/intel/lbr.c b/arch/x86/events/intel/lbr.c
index 20c1d7e..97fadfa 100644
--- a/arch/x86/events/intel/lbr.c
+++ b/arch/x86/events/intel/lbr.c
@@ -423,8 +423,7 @@ static void __intel_pmu_lbr_save(struct x86_perf_task_context *task_ctx)
 	cpuc->last_log_id = ++task_ctx->log_id;
 }
 
-void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
-			      struct task_struct *task, bool sched_in)
+void intel_pmu_lbr_sched_task(struct task_struct *task, bool sched_in)
 {
 	struct cpu_hw_events *cpuc = this_cpu_ptr(&cpu_hw_events);
 	struct x86_perf_task_context *task_ctx;
diff --git a/arch/x86/events/perf_event.h b/arch/x86/events/perf_event.h
index ed287ba..84822cb 100644
--- a/arch/x86/events/perf_event.h
+++ b/arch/x86/events/perf_event.h
@@ -638,8 +638,7 @@ struct x86_pmu {
 	void		(*cpu_dead)(int cpu);
 
 	void		(*check_microcode)(void);
-	void		(*sched_task)(struct perf_event_context *ctx,
-				      struct task_struct *task, bool sched_in);
+	void		(*sched_task)(struct task_struct *task, bool sched_in);
 
 	/*
 	 * Intel Arch Perfmon v2+
@@ -1009,7 +1008,7 @@ void intel_pmu_pebs_enable_all(void);
 
 void intel_pmu_pebs_disable_all(void);
 
-void intel_pmu_pebs_sched_task(struct perf_event_context *ctx, bool sched_in);
+void intel_pmu_pebs_sched_task(bool sched_in);
 
 void intel_pmu_auto_reload_read(struct perf_event *event);
 
@@ -1017,8 +1016,7 @@ void intel_pmu_store_pebs_lbrs(struct pebs_lbr *lbr);
 
 void intel_ds_init(void);
 
-void intel_pmu_lbr_sched_task(struct perf_event_context *ctx,
-			      struct task_struct *task, bool sched_in);
+void intel_pmu_lbr_sched_task(struct task_struct *task, bool sched_in);
 
 u64 lbr_from_signext_quirk_wr(u64 val);
 
diff --git a/include/linux/perf_event.h b/include/linux/perf_event.h
index 44280d7..64bf396 100644
--- a/include/linux/perf_event.h
+++ b/include/linux/perf_event.h
@@ -405,8 +405,7 @@ struct pmu {
 	/*
 	 * context-switches callback
 	 */
-	void (*sched_task)		(struct perf_event_context *ctx,
-					 struct task_struct *task, bool sched_in);
+	void (*sched_task)		(struct task_struct *task, bool sched_in);
 	/*
 	 * PMU specific data size
 	 */
diff --git a/kernel/events/core.c b/kernel/events/core.c
index 037f360..49cff4e 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -3317,13 +3317,11 @@ static void perf_pmu_sched_task(struct task_struct *prev,
 		if (WARN_ON_ONCE(!pmu->sched_task))
 			continue;
 
-		perf_ctx_lock(cpuctx, cpuctx->task_ctx);
 		perf_pmu_disable(pmu);
 
-		pmu->sched_task(cpuctx->task_ctx, sched_in ? next : prev, sched_in);
+		pmu->sched_task(sched_in ? next : prev, sched_in);
 
 		perf_pmu_enable(pmu);
-		perf_ctx_unlock(cpuctx, cpuctx->task_ctx);
 	}
 }
 
-- 
2.7.4


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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-11-28 15:14 ` [RFC PATCH 3/8] perf: Init/fini " kan.liang
@ 2019-12-02 12:40   ` Peter Zijlstra
  2019-12-02 14:59     ` Andi Kleen
  0 siblings, 1 reply; 23+ messages in thread
From: Peter Zijlstra @ 2019-12-02 12:40 UTC (permalink / raw)
  To: kan.liang
  Cc: mingo, acme, tglx, bp, linux-kernel, eranian, alexey.budankov,
	vitaly.slobodskoy, ak

On Thu, Nov 28, 2019 at 07:14:26AM -0800, kan.liang@linux.intel.com wrote:
> +static int
> +init_system_wide_ctx_data(size_t ctx_size)
> +{
> +	struct task_struct *g, *p;
> +	int failed_alloc = 0;
> +	unsigned long flags;
> +
> +	/*
> +	 * Allocate perf_ctx_data for all existing threads by the first event.
> +	 *
> +	 * The perf_ctx_data for new thread will be allocated in
> +	 * perf_event_fork(). The perf_event_fork() is called after the thread
> +	 * is added into the tasklist. It guarantees that any new threads will
> +	 * not be missed.
> +	 */
> +	raw_spin_lock_irqsave(&task_data_events_lock, flags);
> +	if (atomic_inc_return(&nr_task_data_events) > 1)
> +		goto unlock;
> +
> +	read_lock(&tasklist_lock);
> +
> +	for_each_process_thread(g, p) {
> +		/*
> +		 * The PMU specific data may already be allocated by
> +		 * per-process event. Need to update refcounter.
> +		 * init_task_ctx_data_rcu() is called here.
> +		 * Do a quick allocation in first round with GFP_ATOMIC.
> +		 */
> +		if (init_task_ctx_data_rcu(p, ctx_size, GFP_ATOMIC))
> +			failed_alloc++;
> +	}

This is atricous crap. Also it is completely broken for -RT.

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

* Re: [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data
  2019-11-28 15:14 ` [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data kan.liang
@ 2019-12-02 13:16   ` Peter Zijlstra
  2019-12-02 15:25     ` Alexey Budankov
  2019-12-02 20:35     ` Liang, Kan
  0 siblings, 2 replies; 23+ messages in thread
From: Peter Zijlstra @ 2019-12-02 13:16 UTC (permalink / raw)
  To: kan.liang
  Cc: mingo, acme, tglx, bp, linux-kernel, eranian, alexey.budankov,
	vitaly.slobodskoy, ak

On Thu, Nov 28, 2019 at 07:14:25AM -0800, kan.liang@linux.intel.com wrote:

> +static int
> +__alloc_task_ctx_data_rcu(struct task_struct *task,
> +			  size_t ctx_size, gfp_t flags)
> +{
> +	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
> +	int ret;
> +
> +	lockdep_assert_held_once(&task->perf_ctx_data_lock);
> +
> +	ret = alloc_perf_ctx_data(ctx_size, flags, &ctx_data);
> +	if (ret)
> +		return ret;
> +
> +	ctx_data->refcount = 1;
> +
> +	rcu_assign_pointer(task->perf_ctx_data, ctx_data);
> +
> +	return 0;
> +}

> +static int
> +__init_task_ctx_data_rcu(struct task_struct *task, size_t ctx_size, gfp_t flags)
> +{
> +	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
> +
> +	lockdep_assert_held_once(&task->perf_ctx_data_lock);
> +
> +	if (ctx_data) {
> +		ctx_data->refcount++;
> +		return 0;
> +	}
> +
> +	return __alloc_task_ctx_data_rcu(task, ctx_size, flags);
> +}

> +/**
> + * Free perf_ctx_data RCU pointer for a task
> + * @task:        Target Task
> + * @force:       Unconditionally free perf_ctx_data
> + *
> + * If force is set, free perf_ctx_data unconditionally.
> + * Otherwise, free perf_ctx_data when there are no users.
> + * Lock is required to sync the writers of perf_ctx_data RCU pointer
> + * and refcount.
> + */
> +static void
> +fini_task_ctx_data_rcu(struct task_struct *task, bool force)
> +{
> +	struct perf_ctx_data *ctx_data;
> +	unsigned long flags;
> +
> +	raw_spin_lock_irqsave(&task->perf_ctx_data_lock, flags);
> +
> +	ctx_data = task->perf_ctx_data;
> +	if (!ctx_data)
> +		goto unlock;
> +
> +	if (!force && --ctx_data->refcount)
> +		goto unlock;
> +
> +	RCU_INIT_POINTER(task->perf_ctx_data, NULL);
> +	call_rcu(&ctx_data->rcu_head, free_perf_ctx_data);
> +
> +unlock:
> +	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, flags);
> +}

All this refcount under lock is an anti-pattern. Also the naming is
insane.

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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 12:40   ` Peter Zijlstra
@ 2019-12-02 14:59     ` Andi Kleen
  2019-12-02 16:21       ` Peter Zijlstra
  2019-12-02 16:38       ` Alexey Budankov
  0 siblings, 2 replies; 23+ messages in thread
From: Andi Kleen @ 2019-12-02 14:59 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: kan.liang, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy

> 
> This is atricous crap. Also it is completely broken for -RT.

Well can you please suggest how you would implement it instead?

-Andi

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

* Re: [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data
  2019-12-02 13:16   ` Peter Zijlstra
@ 2019-12-02 15:25     ` Alexey Budankov
  2019-12-02 20:35     ` Liang, Kan
  1 sibling, 0 replies; 23+ messages in thread
From: Alexey Budankov @ 2019-12-02 15:25 UTC (permalink / raw)
  To: Peter Zijlstra, kan.liang
  Cc: mingo, acme, tglx, bp, linux-kernel, eranian, vitaly.slobodskoy, ak


On 02.12.2019 16:16, Peter Zijlstra wrote:
> On Thu, Nov 28, 2019 at 07:14:25AM -0800, kan.liang@linux.intel.com wrote:
> 
>> +static int
>> +__alloc_task_ctx_data_rcu(struct task_struct *task,
>> +			  size_t ctx_size, gfp_t flags)
>> +{
>> +	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
>> +	int ret;
>> +
>> +	lockdep_assert_held_once(&task->perf_ctx_data_lock);
>> +
>> +	ret = alloc_perf_ctx_data(ctx_size, flags, &ctx_data);
>> +	if (ret)
>> +		return ret;
>> +
>> +	ctx_data->refcount = 1;
>> +
>> +	rcu_assign_pointer(task->perf_ctx_data, ctx_data);
>> +
>> +	return 0;
>> +}
> 
>> +static int
>> +__init_task_ctx_data_rcu(struct task_struct *task, size_t ctx_size, gfp_t flags)
>> +{
>> +	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
>> +
>> +	lockdep_assert_held_once(&task->perf_ctx_data_lock);
>> +
>> +	if (ctx_data) {
>> +		ctx_data->refcount++;
>> +		return 0;
>> +	}
>> +
>> +	return __alloc_task_ctx_data_rcu(task, ctx_size, flags);
>> +}
> 
>> +/**
>> + * Free perf_ctx_data RCU pointer for a task
>> + * @task:        Target Task
>> + * @force:       Unconditionally free perf_ctx_data
>> + *
>> + * If force is set, free perf_ctx_data unconditionally.
>> + * Otherwise, free perf_ctx_data when there are no users.
>> + * Lock is required to sync the writers of perf_ctx_data RCU pointer
>> + * and refcount.
>> + */
>> +static void
>> +fini_task_ctx_data_rcu(struct task_struct *task, bool force)
>> +{
>> +	struct perf_ctx_data *ctx_data;
>> +	unsigned long flags;
>> +
>> +	raw_spin_lock_irqsave(&task->perf_ctx_data_lock, flags);
>> +
>> +	ctx_data = task->perf_ctx_data;
>> +	if (!ctx_data)
>> +		goto unlock;
>> +
>> +	if (!force && --ctx_data->refcount)
>> +		goto unlock;
>> +
>> +	RCU_INIT_POINTER(task->perf_ctx_data, NULL);
>> +	call_rcu(&ctx_data->rcu_head, free_perf_ctx_data);
>> +
>> +unlock:
>> +	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, flags);
>> +}
> 
> All this refcount under lock is an anti-pattern. Also the naming is
> insane.

Could you please provide proper patterning examples for such or similar cases?

Thanks,
Alexey

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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 14:59     ` Andi Kleen
@ 2019-12-02 16:21       ` Peter Zijlstra
  2019-12-02 19:15         ` Andi Kleen
  2019-12-02 16:38       ` Alexey Budankov
  1 sibling, 1 reply; 23+ messages in thread
From: Peter Zijlstra @ 2019-12-02 16:21 UTC (permalink / raw)
  To: Andi Kleen
  Cc: kan.liang, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy

On Mon, Dec 02, 2019 at 06:59:57AM -0800, Andi Kleen wrote:
> > 
> > This is atricous crap. Also it is completely broken for -RT.
> 
> Well can you please suggest how you would implement it instead?

I don't think that is on me; at best I get to explain why it is
completely unacceptible to have O(nr_tasks) and allocations under a
raw_spinlock_t, but I was thinking you'd already know that.



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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 14:59     ` Andi Kleen
  2019-12-02 16:21       ` Peter Zijlstra
@ 2019-12-02 16:38       ` Alexey Budankov
  2019-12-02 16:43         ` Peter Zijlstra
  1 sibling, 1 reply; 23+ messages in thread
From: Alexey Budankov @ 2019-12-02 16:38 UTC (permalink / raw)
  To: Andi Kleen, Peter Zijlstra
  Cc: kan.liang, mingo, acme, tglx, bp, linux-kernel, eranian,
	vitaly.slobodskoy


On 02.12.2019 17:59, Andi Kleen wrote:
>>
>> This is atricous crap. Also it is completely broken for -RT.
> 
> Well can you please suggest how you would implement it instead?

FWIW,
An alternative could probably be to make task_ctx_data allocations
on the nearest context switch in, and obvious drawback is slowdown on
this critical path, but it could be amortized by static branches.

Thanks,
Alexey

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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 16:38       ` Alexey Budankov
@ 2019-12-02 16:43         ` Peter Zijlstra
  2019-12-02 17:42           ` Alexey Budankov
  0 siblings, 1 reply; 23+ messages in thread
From: Peter Zijlstra @ 2019-12-02 16:43 UTC (permalink / raw)
  To: Alexey Budankov
  Cc: Andi Kleen, kan.liang, mingo, acme, tglx, bp, linux-kernel,
	eranian, vitaly.slobodskoy

On Mon, Dec 02, 2019 at 07:38:00PM +0300, Alexey Budankov wrote:
> 
> On 02.12.2019 17:59, Andi Kleen wrote:
> >>
> >> This is atricous crap. Also it is completely broken for -RT.
> > 
> > Well can you please suggest how you would implement it instead?
> 
> FWIW,
> An alternative could probably be to make task_ctx_data allocations
> on the nearest context switch in, and obvious drawback is slowdown on
> this critical path, but it could be amortized by static branches.

Context switch is under a raw_spinlock_t too.

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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 16:43         ` Peter Zijlstra
@ 2019-12-02 17:42           ` Alexey Budankov
  0 siblings, 0 replies; 23+ messages in thread
From: Alexey Budankov @ 2019-12-02 17:42 UTC (permalink / raw)
  To: Peter Zijlstra, Andi Kleen
  Cc: kan.liang, mingo, acme, tglx, bp, linux-kernel, eranian,
	vitaly.slobodskoy


On 02.12.2019 19:43, Peter Zijlstra wrote:
> On Mon, Dec 02, 2019 at 07:38:00PM +0300, Alexey Budankov wrote:
>>
>> On 02.12.2019 17:59, Andi Kleen wrote:
>>>>
>>>> This is atricous crap. Also it is completely broken for -RT.
>>>
>>> Well can you please suggest how you would implement it instead?
>>
>> FWIW,
>> An alternative could probably be to make task_ctx_data allocations
>> on the nearest context switch in, and obvious drawback is slowdown on
>> this critical path, but it could be amortized by static branches.
> 
> Context switch is under a raw_spinlock_t too.

Indeed, under rq->lock (some of runqueue locks, I suppose), but 
as far locking order is not violated a thread shouldn't deadlock.
On the other side it could probably hurt concurrency and 
it is more preferable to have task_ctx_data memory pre-allocated 
by the time it is assigned on a context switch in.

What if we would create some pool of preallocated task_ctx_data 
objects on the first system wide perf_event_open() syscall
and after that:
- already existing threads would take task_ctx_data objects from
  the pool without additional locking on the nearest
  context switch in;
- newly created threads would allocate task_ctx_data themselves,
  atomically checking some global state, possibly at PMU object
- task_ctx_data deallocation would be performed by threads
  themselves, at some safe points in time;

Thanks,
Alexey

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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 16:21       ` Peter Zijlstra
@ 2019-12-02 19:15         ` Andi Kleen
  2019-12-02 20:13           ` Liang, Kan
  0 siblings, 1 reply; 23+ messages in thread
From: Andi Kleen @ 2019-12-02 19:15 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: kan.liang, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy

On Mon, Dec 02, 2019 at 05:21:52PM +0100, Peter Zijlstra wrote:
> On Mon, Dec 02, 2019 at 06:59:57AM -0800, Andi Kleen wrote:
> > > 
> > > This is atricous crap. Also it is completely broken for -RT.
> > 
> > Well can you please suggest how you would implement it instead?
> 
> I don't think that is on me; at best I get to explain why it is

Normally code review is expected to be constructive.

> completely unacceptible to have O(nr_tasks) and allocations under a
> raw_spinlock_t, but I was thinking you'd already know that.

Ok if that's the only problem then a lock breaker + retry 
if rescheduling is needed + some limit against live lock 
should be sufficient.

-Andi


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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 19:15         ` Andi Kleen
@ 2019-12-02 20:13           ` Liang, Kan
  2019-12-02 20:25             ` Andi Kleen
  0 siblings, 1 reply; 23+ messages in thread
From: Liang, Kan @ 2019-12-02 20:13 UTC (permalink / raw)
  To: Andi Kleen, Peter Zijlstra
  Cc: mingo, acme, tglx, bp, linux-kernel, eranian, alexey.budankov,
	vitaly.slobodskoy



On 12/2/2019 2:15 PM, Andi Kleen wrote:
> On Mon, Dec 02, 2019 at 05:21:52PM +0100, Peter Zijlstra wrote:
>> On Mon, Dec 02, 2019 at 06:59:57AM -0800, Andi Kleen wrote:
>>>>
>>>> This is atricous crap. Also it is completely broken for -RT.
>>>
>>> Well can you please suggest how you would implement it instead?
>>
>> I don't think that is on me; at best I get to explain why it is
> 
> Normally code review is expected to be constructive.
> 
>> completely unacceptible to have O(nr_tasks) and allocations under a
>> raw_spinlock_t, but I was thinking you'd already know that.
> 
> Ok if that's the only problem then a lock breaker + retry
> if rescheduling is needed + some limit against live lock
> should be sufficient.
> 

OK. I will move the allocation out of critical sections.
Here is some pseudo code.

if (atomic_read(&nr_task_data_events))
	return;

//get current number of threads
read_lock(&tasklist_lock);
for_each_process_thread(g, p)
	num_thread++;
read_unlock(&tasklist_lock);

//allocate the space for them
for (i = 0; i < num_thread; i++)
	data[i] = kzalloc(ctx_size, flags);
i = 0;

/*
  * Assign the space to tasks
  * There may be some new threads created when we allocate space.
  * new_task will track its number.
  */
raw_spin_lock_irqsave(&task_data_events_lock, flags);

if (atomic_inc_return(&nr_task_data_events) > 1)
	goto unlock;

for_each_process_thread(g, p) {
	if (i < num_thread)
		p->perf_ctx_data = data[i++];
	else
		new_task++;
}
raw_spin_unlock_irqrestore(&task_data_events_lock, flags);

if (i < num_thread)
	goto end;

/*
  * Try again to allocate the space for the task created when
  * we first allocate space.
  * We don't need to worry about the task created after
  * atomic_inc_return(). It will be handled in perf_event_fork().
  * Retry one is enough.
  */
for (i = 0; i < new_task; i++)
	data[i] = kzalloc(ctx_size, flags);

raw_spin_lock_irqsave(&task_data_events_lock, flags);

for_each_process_thread(g, p) {
	if (i < unallocated)
		p->perf_ctx_data = data[i++];
	else
		WARN_ON
}
raw_spin_unlock_irqrestore(&task_data_events_lock, flags);

unlock:
	raw_spin_unlock_irqrestore(&task_data_events_lock, flags);

end:
	free unused data[]

Thanks,
Kan

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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 20:13           ` Liang, Kan
@ 2019-12-02 20:25             ` Andi Kleen
  2019-12-02 20:44               ` Liang, Kan
  0 siblings, 1 reply; 23+ messages in thread
From: Andi Kleen @ 2019-12-02 20:25 UTC (permalink / raw)
  To: Liang, Kan
  Cc: Peter Zijlstra, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy


Looks reasonable to me.

> //get current number of threads
> read_lock(&tasklist_lock);
> for_each_process_thread(g, p)
> 	num_thread++;
> read_unlock(&tasklist_lock);

I'm sure we have that count somewhere.

> 
> //allocate the space for them
> for (i = 0; i < num_thread; i++)
> 	data[i] = kzalloc(ctx_size, flags);
> i = 0;
> 
> /*
>  * Assign the space to tasks
>  * There may be some new threads created when we allocate space.
>  * new_task will track its number.
>  */
> raw_spin_lock_irqsave(&task_data_events_lock, flags);
> 
> if (atomic_inc_return(&nr_task_data_events) > 1)
> 	goto unlock;
> 
> for_each_process_thread(g, p) {
> 	if (i < num_thread)
> 		p->perf_ctx_data = data[i++];
> 	else
> 		new_task++;
> }
> raw_spin_unlock_irqrestore(&task_data_events_lock, flags);

Is that lock taken in the context switch?

If not could be a normal spinlock, thus be more RT friendly.

-Andi

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

* Re: [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data
  2019-12-02 13:16   ` Peter Zijlstra
  2019-12-02 15:25     ` Alexey Budankov
@ 2019-12-02 20:35     ` Liang, Kan
  2019-12-04 12:36       ` Peter Zijlstra
  1 sibling, 1 reply; 23+ messages in thread
From: Liang, Kan @ 2019-12-02 20:35 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: mingo, acme, tglx, bp, linux-kernel, eranian, alexey.budankov,
	vitaly.slobodskoy, ak



On 12/2/2019 8:16 AM, Peter Zijlstra wrote:
> On Thu, Nov 28, 2019 at 07:14:25AM -0800, kan.liang@linux.intel.com wrote:
> 
>> +static int
>> +__alloc_task_ctx_data_rcu(struct task_struct *task,
>> +			  size_t ctx_size, gfp_t flags)
>> +{
>> +	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
>> +	int ret;
>> +
>> +	lockdep_assert_held_once(&task->perf_ctx_data_lock);
>> +
>> +	ret = alloc_perf_ctx_data(ctx_size, flags, &ctx_data);
>> +	if (ret)
>> +		return ret;
>> +
>> +	ctx_data->refcount = 1;
>> +
>> +	rcu_assign_pointer(task->perf_ctx_data, ctx_data);
>> +
>> +	return 0;
>> +}
> 
>> +static int
>> +__init_task_ctx_data_rcu(struct task_struct *task, size_t ctx_size, gfp_t flags)
>> +{
>> +	struct perf_ctx_data *ctx_data = task->perf_ctx_data;
>> +
>> +	lockdep_assert_held_once(&task->perf_ctx_data_lock);
>> +
>> +	if (ctx_data) {
>> +		ctx_data->refcount++;
>> +		return 0;
>> +	}
>> +
>> +	return __alloc_task_ctx_data_rcu(task, ctx_size, flags);
>> +}
> 
>> +/**
>> + * Free perf_ctx_data RCU pointer for a task
>> + * @task:        Target Task
>> + * @force:       Unconditionally free perf_ctx_data
>> + *
>> + * If force is set, free perf_ctx_data unconditionally.
>> + * Otherwise, free perf_ctx_data when there are no users.
>> + * Lock is required to sync the writers of perf_ctx_data RCU pointer
>> + * and refcount.
>> + */
>> +static void
>> +fini_task_ctx_data_rcu(struct task_struct *task, bool force)
>> +{
>> +	struct perf_ctx_data *ctx_data;
>> +	unsigned long flags;
>> +
>> +	raw_spin_lock_irqsave(&task->perf_ctx_data_lock, flags);
>> +
>> +	ctx_data = task->perf_ctx_data;
>> +	if (!ctx_data)
>> +		goto unlock;
>> +
>> +	if (!force && --ctx_data->refcount)
>> +		goto unlock;
>> +
>> +	RCU_INIT_POINTER(task->perf_ctx_data, NULL);
>> +	call_rcu(&ctx_data->rcu_head, free_perf_ctx_data);
>> +
>> +unlock:
>> +	raw_spin_unlock_irqrestore(&task->perf_ctx_data_lock, flags);
>> +}
> 
> All this refcount under lock is an anti-pattern. Also the naming is
> insane.
> 

Could you please give me an example?

I think we do need something to protect the refcount. Are you suggesting 
atomic_*?

Thanks,
Kan


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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 20:25             ` Andi Kleen
@ 2019-12-02 20:44               ` Liang, Kan
  2019-12-04 13:12                 ` Peter Zijlstra
  0 siblings, 1 reply; 23+ messages in thread
From: Liang, Kan @ 2019-12-02 20:44 UTC (permalink / raw)
  To: Andi Kleen
  Cc: Peter Zijlstra, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy



On 12/2/2019 3:25 PM, Andi Kleen wrote:
> 
> Looks reasonable to me.
> 
>> //get current number of threads
>> read_lock(&tasklist_lock);
>> for_each_process_thread(g, p)
>> 	num_thread++;
>> read_unlock(&tasklist_lock);
> 
> I'm sure we have that count somewhere.
>

It looks like we can get the number from global variable "nr_threads"
I will use it in v2.

>>
>> //allocate the space for them
>> for (i = 0; i < num_thread; i++)
>> 	data[i] = kzalloc(ctx_size, flags);
>> i = 0;
>>
>> /*
>>   * Assign the space to tasks
>>   * There may be some new threads created when we allocate space.
>>   * new_task will track its number.
>>   */
>> raw_spin_lock_irqsave(&task_data_events_lock, flags);
>>
>> if (atomic_inc_return(&nr_task_data_events) > 1)
>> 	goto unlock;
>>
>> for_each_process_thread(g, p) {
>> 	if (i < num_thread)
>> 		p->perf_ctx_data = data[i++];
>> 	else
>> 		new_task++;
>> }
>> raw_spin_unlock_irqrestore(&task_data_events_lock, flags);
> 
> Is that lock taken in the context switch? >
> If not could be a normal spinlock, thus be more RT friendly.
> 

It's not in context switch. I will use the normal spinlock to instead.

Thanks,
Kan

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

* Re: [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data
  2019-12-02 20:35     ` Liang, Kan
@ 2019-12-04 12:36       ` Peter Zijlstra
  0 siblings, 0 replies; 23+ messages in thread
From: Peter Zijlstra @ 2019-12-04 12:36 UTC (permalink / raw)
  To: Liang, Kan
  Cc: mingo, acme, tglx, bp, linux-kernel, eranian, alexey.budankov,
	vitaly.slobodskoy, ak

On Mon, Dec 02, 2019 at 03:35:00PM -0500, Liang, Kan wrote:

> Could you please give me an example?

There's a ton of refcounting in perf, none of it follows this pattern.

> I think we do need something to protect the refcount. Are you suggesting
> atomic_*?

refcount_t comes to mind.

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

* Re: [RFC PATCH 3/8] perf: Init/fini PMU specific data
  2019-12-02 20:44               ` Liang, Kan
@ 2019-12-04 13:12                 ` Peter Zijlstra
  0 siblings, 0 replies; 23+ messages in thread
From: Peter Zijlstra @ 2019-12-04 13:12 UTC (permalink / raw)
  To: Liang, Kan
  Cc: Andi Kleen, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy

On Mon, Dec 02, 2019 at 03:44:34PM -0500, Liang, Kan wrote:

> It's not in context switch. I will use the normal spinlock to instead.

Mutex would make even more sense. And we already have a per-task
perf_event_mutex.

Also, I don't think you need tasklist_lock here, if you set the state
before the iteration, any new clone()s will observe the state and
allocate the storage themselves. Then all you need is RCU iteration of
the tasklist.




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

end of thread, other threads:[~2019-12-04 13:12 UTC | newest]

Thread overview: 23+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-11-28 15:14 [RFC PATCH 1/8] perf: Save PMU specific data in task_struct kan.liang
2019-11-28 15:14 ` [RFC PATCH 2/8] perf: Helpers for alloc/init/fini PMU specific data kan.liang
2019-12-02 13:16   ` Peter Zijlstra
2019-12-02 15:25     ` Alexey Budankov
2019-12-02 20:35     ` Liang, Kan
2019-12-04 12:36       ` Peter Zijlstra
2019-11-28 15:14 ` [RFC PATCH 3/8] perf: Init/fini " kan.liang
2019-12-02 12:40   ` Peter Zijlstra
2019-12-02 14:59     ` Andi Kleen
2019-12-02 16:21       ` Peter Zijlstra
2019-12-02 19:15         ` Andi Kleen
2019-12-02 20:13           ` Liang, Kan
2019-12-02 20:25             ` Andi Kleen
2019-12-02 20:44               ` Liang, Kan
2019-12-04 13:12                 ` Peter Zijlstra
2019-12-02 16:38       ` Alexey Budankov
2019-12-02 16:43         ` Peter Zijlstra
2019-12-02 17:42           ` Alexey Budankov
2019-11-28 15:14 ` [RFC PATCH 4/8] perf: Supply task information to sched_task() kan.liang
2019-11-28 15:14 ` [RFC PATCH 5/8] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode kan.liang
2019-11-28 15:14 ` [RFC PATCH 6/8] perf/x86: Remove swap_task_ctx() kan.liang
2019-11-28 15:14 ` [RFC PATCH 7/8] perf: Clean up pmu specific data kan.liang
2019-11-28 15:14 ` [RFC PATCH 8/8] perf: Clean up event context from sched_task() kan.liang

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