All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct
@ 2020-01-08 15:03 kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 2/7] perf: attach/detach PMU specific data kan.liang
                   ` (5 more replies)
  0 siblings, 6 replies; 9+ messages in thread
From: kan.liang @ 2020-01-08 15:03 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>
---

Changes since V2:
- Cannot use mutex inside rcu_read_lock().
  Restore the pin lock perf_ctx_data_lock

 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..d8b871f 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;
+	refcount_t			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] 9+ messages in thread

* [RFC PATCH V3 2/7] perf: attach/detach PMU specific data
  2020-01-08 15:03 [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct kan.liang
@ 2020-01-08 15:03 ` kan.liang
  2020-01-08 16:50   ` Andi Kleen
  2020-01-08 15:03 ` [RFC PATCH V3 3/7] perf: Supply task information to sched_task() kan.liang
                   ` (4 subsequent siblings)
  5 siblings, 1 reply; 9+ messages in thread
From: kan.liang @ 2020-01-08 15:03 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 only be allocated during
LBR call stack monitoring.

When a LBR call stack event is accounted, the perf_ctx_data for related
tasks will be allocated/attached by attach_perf_ctx_data().
When a LBR call stack event is unaccounted, the perf_ctx_data for
related tasks will be detached/freed by detach_perf_ctx_data().

LBR call stack events could be per-task events or system-wide events.
- For per-task event, perf only allocates the perf_ctx_data for current
  task. If the allocation fails, perf will error out.
- For system-wide event, perf has to allocate the perf_ctx_data for
  both existing tasks and upcoming tasks.
  The allocation for the existing tasks is done in perf_event_alloc().
  The allocation for new tasks will be done in perf_event_fork().
  If any allocation fails, perf doesn't error out for system-wide event.
  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.
- The perf_ctx_data only be freed by the last LBR call stack event.
  The number of per-task events is tracked by refcount of each task.
  Since the system-wide events impact all tasks, it's not practical to
  go through the whole task list to update the refcount for each
  system-wide event. The number of system-wide events is tracked by a
  global variable nr_task_data_sys_wide_events.
  Introduce a macro TASK_DATA_SYS_WIDE for refcount to indicate the
  PMU specific data is used by system-wide events.

Signed-off-by: Kan Liang <kan.liang@linux.intel.com>
---

Changes since V2:
- Remove global spin lock task_data_sys_wide_events_lock
  Since the global spin lock has been removed, we cannot guarantee
  that the allocation/assignments for existing threads and free are
  serialized.
  To fix it, in V3, we go through the task list when accounting for
  each system-wide event, and assign the perf_ctx_data pointer if needed.
  (In V2, we only do the assignment for the first system-wide event).
  In V3, we also add a breaker in free process for system-wide event.
  If there is new system-wide event accounted, stop the free process
  immediately.
- Add a macro TASK_DATA_SYS_WIDE to indicate the PMU specific data
  is used by system-wide events.

 kernel/events/core.c | 371 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 371 insertions(+)

diff --git a/kernel/events/core.c b/kernel/events/core.c
index 43567d1..27f5f94 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -47,6 +47,7 @@
 #include <linux/parser.h>
 #include <linux/sched/clock.h>
 #include <linux/sched/mm.h>
+#include <linux/sched/stat.h>
 #include <linux/proc_ns.h>
 #include <linux/mount.h>
 
@@ -387,6 +388,39 @@ 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_sys_wide_events;
+
+/*
+ * There are two types of users for pmu specific data, system-wide event and
+ * per-task event.
+ *
+ * The number of system-wide events is already tracked by global variable
+ * nr_task_data_sys_wide_events. Set TASK_DATA_SYS_WIDE in refcount to
+ * indicate the PMU specific data is used by system-wide events.
+ *
+ * The number of per-task event users is tracked by refcount. Since the
+ * TASK_DATA_SYS_WIDE is already occupied by system-wide events, limit
+ * the max number of per-task event users less than half of TASK_DATA_SYS_WIDE.
+ */
+#define TASK_DATA_SYS_WIDE		0x1000000
+#define MAX_NR_TASK_DATA_EVENTS		(TASK_DATA_SYS_WIDE >> 1)
+
+static inline bool has_task_data_sys_wide(struct perf_ctx_data *perf_ctx_data)
+{
+	return !!(refcount_read(&perf_ctx_data->refcount) & TASK_DATA_SYS_WIDE);
+}
+
+static inline bool exceed_task_data_events_limit(struct perf_ctx_data *perf_ctx_data)
+{
+	unsigned int count = refcount_read(&perf_ctx_data->refcount);
+
+	if (has_task_data_sys_wide(perf_ctx_data))
+		return (count - TASK_DATA_SYS_WIDE) > MAX_NR_TASK_DATA_EVENTS;
+	else
+		return count > MAX_NR_TASK_DATA_EVENTS;
+}
+
 static LIST_HEAD(pmus);
 static DEFINE_MUTEX(pmus_lock);
 static struct srcu_struct pmus_srcu;
@@ -4440,6 +4474,279 @@ 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 void
+free_perf_ctx_data(struct perf_ctx_data *ctx_data)
+{
+	kfree(ctx_data->data);
+	kfree(ctx_data);
+}
+
+static void
+free_perf_ctx_data_rcu(struct rcu_head *rcu_head)
+{
+	struct perf_ctx_data *ctx_data;
+
+	ctx_data = container_of(rcu_head, struct perf_ctx_data, rcu_head);
+	free_perf_ctx_data(ctx_data);
+}
+
+static int
+attach_task_ctx_data(struct task_struct *task, size_t ctx_size)
+{
+	struct perf_ctx_data *ctx_data, *tsk_data;
+
+	/*
+	 * To make the code RT friendly, make the the allocation out of
+	 * the spinlock.
+	 */
+	if (alloc_perf_ctx_data(ctx_size, GFP_KERNEL, &ctx_data))
+		return -ENOMEM;
+
+	raw_spin_lock(&task->perf_ctx_data_lock);
+
+	tsk_data = task->perf_ctx_data;
+	if (tsk_data) {
+		free_perf_ctx_data(ctx_data);
+		if (WARN_ON_ONCE(exceed_task_data_events_limit(tsk_data))) {
+			raw_spin_unlock(&task->perf_ctx_data_lock);
+			return -EINVAL;
+		}
+		refcount_inc(&tsk_data->refcount);
+	} else {
+		refcount_set(&ctx_data->refcount, 1);
+		/* System-wide event is active as well */
+		if (atomic_read(&nr_task_data_sys_wide_events))
+			refcount_add(TASK_DATA_SYS_WIDE, &ctx_data->refcount);
+
+		rcu_assign_pointer(task->perf_ctx_data, ctx_data);
+	}
+
+	raw_spin_unlock(&task->perf_ctx_data_lock);
+	return 0;
+}
+
+static int
+attach_system_wide_ctx_data(size_t ctx_size)
+{
+	int i, num_thread, pos, nr_failed_alloc;
+	unsigned long flags = GFP_ATOMIC;
+	struct perf_ctx_data *tsk_data;
+	struct perf_ctx_data **data;
+	struct task_struct *g, *p;
+	bool re_alloc = true;
+
+	/* Retrieve total number of threads */
+	num_thread = nr_threads;
+
+	data = kcalloc(num_thread, sizeof(*data), GFP_KERNEL);
+	if (!data) {
+		printk_once(KERN_DEBUG
+			    "Failed to allocate space for LBR callstack. "
+			    "The LBR callstack for all tasks may be cutoff.\n");
+		return -ENOMEM;
+	}
+
+	atomic_inc(&nr_task_data_sys_wide_events);
+
+repeat:
+	/*
+	 * Allocate perf_ctx_data for all existing threads.
+	 * The perf_ctx_data for new threads will be allocated in
+	 * perf_event_fork().
+	 * Do a quick allocation in first round with GFP_ATOMIC.
+	 */
+	for (i = 0; i < num_thread; i++) {
+		if (alloc_perf_ctx_data(ctx_size, flags, &data[i]))
+			break;
+	}
+	num_thread = i;
+	nr_failed_alloc = 0;
+	pos = 0;
+
+	rcu_read_lock();
+	for_each_process_thread(g, p) {
+		raw_spin_lock(&p->perf_ctx_data_lock);
+		tsk_data = p->perf_ctx_data;
+		if (tsk_data) {
+			/*
+			 * The perf_ctx_data for this thread may has been
+			 * allocated by per-task event.
+			 * Only update refcount for the case.
+			 */
+			if (!has_task_data_sys_wide(tsk_data))
+				refcount_add(TASK_DATA_SYS_WIDE, &tsk_data->refcount);
+			raw_spin_unlock(&p->perf_ctx_data_lock);
+			continue;
+		}
+
+		if (pos < num_thread) {
+			refcount_set(&data[pos]->refcount, TASK_DATA_SYS_WIDE);
+			rcu_assign_pointer(p->perf_ctx_data, data[pos++]);
+		} else {
+			/*
+			 * The quick allocation in first round may be failed.
+			 * Track the number in nr_failed_alloc.
+			 */
+			nr_failed_alloc++;
+		}
+		raw_spin_unlock(&p->perf_ctx_data_lock);
+	}
+	rcu_read_unlock();
+
+	if (re_alloc && !nr_failed_alloc) {
+		num_thread = nr_failed_alloc;
+		flags = GFP_KERNEL;
+		re_alloc = false;
+		goto repeat;
+	}
+
+	if (nr_failed_alloc) {
+		printk_once(KERN_DEBUG
+			    "Failed to allocate space for LBR callstack. "
+			    "The LBR callstack for some tasks may be cutoff.\n");
+	}
+
+	for (; pos < num_thread; pos++)
+		free_perf_ctx_data(data[pos]);
+
+	kfree(data);
+	return 0;
+}
+
+static int
+attach_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 attach_task_ctx_data(task, ctx_size);
+	else
+		return attach_system_wide_ctx_data(ctx_size);
+}
+
+/**
+ * Detach perf_ctx_data RCU pointer for a task monitored by per-task event
+ * @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
+ */
+static void
+detach_task_ctx_data(struct task_struct *task, bool force)
+{
+	struct perf_ctx_data *ctx_data;
+
+	raw_spin_lock(&task->perf_ctx_data_lock);
+
+	ctx_data = task->perf_ctx_data;
+	if (!ctx_data)
+		goto unlock;
+
+	if (!force) {
+		WARN_ON_ONCE(refcount_read(&ctx_data->refcount) == TASK_DATA_SYS_WIDE);
+
+		if (!refcount_dec_and_test(&ctx_data->refcount))
+			goto unlock;
+	}
+
+	RCU_INIT_POINTER(task->perf_ctx_data, NULL);
+	call_rcu(&ctx_data->rcu_head, free_perf_ctx_data_rcu);
+
+unlock:
+	raw_spin_unlock(&task->perf_ctx_data_lock);
+}
+
+/**
+ * Detach perf_ctx_data RCU pointer for a task monitored by system-wide event
+ * @task:        Target Task
+ *
+ * Free perf_ctx_data when there are no users.
+ */
+static void
+detach_task_ctx_data_sys_wide(struct task_struct *task)
+{
+	struct perf_ctx_data *ctx_data;
+
+	lockdep_assert_held(&task->perf_ctx_data_lock);
+
+	ctx_data = task->perf_ctx_data;
+	if (!ctx_data)
+		return;
+
+	WARN_ON_ONCE(!has_task_data_sys_wide(ctx_data));
+
+	if (!refcount_sub_and_test(TASK_DATA_SYS_WIDE, &ctx_data->refcount))
+		return;
+
+	RCU_INIT_POINTER(task->perf_ctx_data, NULL);
+	call_rcu(&ctx_data->rcu_head, free_perf_ctx_data_rcu);
+}
+
+static void detach_system_wide_ctx_data(void)
+{
+	struct task_struct *g, *p;
+
+	if (!atomic_dec_and_test(&nr_task_data_sys_wide_events))
+		return;
+
+	rcu_read_lock();
+	for_each_process_thread(g, p) {
+		raw_spin_lock(&p->perf_ctx_data_lock);
+
+		/*
+		 * A new system-wide event may be attached while freeing
+		 * everything for the old event.
+		 * If so, stop the free process immediately.
+		 * For the freed threads, attach_system_wide_ctx_data()
+		 * will re-allocate the space.
+		 */
+		if (unlikely(atomic_read(&nr_task_data_sys_wide_events))) {
+			raw_spin_unlock(&p->perf_ctx_data_lock);
+			goto unlock;
+		}
+
+		detach_task_ctx_data_sys_wide(p);
+		raw_spin_unlock(&p->perf_ctx_data_lock);
+	}
+unlock:
+	rcu_read_unlock();
+}
+
+static void detach_perf_ctx_data(struct perf_event *event)
+{
+	struct task_struct *task = event->hw.target;
+
+	if (task)
+		detach_task_ctx_data(task, false);
+	else
+		detach_system_wide_ctx_data();
+}
+
 static void unaccount_event(struct perf_event *event)
 {
 	bool dec = false;
@@ -4471,6 +4778,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)
+		detach_perf_ctx_data(event);
 
 	if (dec) {
 		if (!atomic_add_unless(&perf_sched_count, -1, 1))
@@ -7297,10 +7606,63 @@ 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;
+
+	if (!atomic_read(&nr_task_data_sys_wide_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;
+
+	if (alloc_perf_ctx_data(ctx_size, GFP_KERNEL, &ctx_data))
+		return;
+
+	raw_spin_lock(&child->perf_ctx_data_lock);
+
+	if (child->perf_ctx_data) {
+		free_perf_ctx_data(ctx_data);
+	} else {
+		refcount_set(&ctx_data->refcount, TASK_DATA_SYS_WIDE);
+		rcu_assign_pointer(child->perf_ctx_data, ctx_data);
+	}
+
+	/*
+	 * System-wide event may be unaccount when attaching the perf_ctx_data.
+	 * For example,
+	 *                CPU A                              CPU B
+	 *        perf_event_alloc_task_data():
+	 *          read(nr_task_data_sys_wide_events)
+	 *                                         detach_system_wide_ctx_data()
+	 *          alloc_perf_ctx_data()
+	 *          rcu_assign_pointer(perf_ctx_data);
+	 *
+	 * The perf_ctx_data may never be freed until the task is terminated.
+	 */
+	if (unlikely(!atomic_read(&nr_task_data_sys_wide_events)))
+		detach_task_ctx_data_sys_wide(child);
+
+	raw_spin_unlock(&child->perf_ctx_data_lock);
+}
+
 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);
 }
 
 /*
@@ -10826,11 +11188,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) &&
+	    attach_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)
@@ -11892,6 +12261,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);
+
+	detach_task_ctx_data(child, true);
 }
 
 static void perf_free_event(struct perf_event *event,
-- 
2.7.4


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

* [RFC PATCH V3 3/7] perf: Supply task information to sched_task()
  2020-01-08 15:03 [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 2/7] perf: attach/detach PMU specific data kan.liang
@ 2020-01-08 15:03 ` kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 4/7] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode kan.liang
                   ` (3 subsequent siblings)
  5 siblings, 0 replies; 9+ messages in thread
From: kan.liang @ 2020-01-08 15:03 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>
---

No changes since V2

 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 d8b871f..f271595 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 27f5f94..1835d9b 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -3365,7 +3365,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] 9+ messages in thread

* [RFC PATCH V3 4/7] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode
  2020-01-08 15:03 [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 2/7] perf: attach/detach PMU specific data kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 3/7] perf: Supply task information to sched_task() kan.liang
@ 2020-01-08 15:03 ` kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 5/7] perf/x86: Remove swap_task_ctx() kan.liang
                   ` (2 subsequent siblings)
  5 siblings, 0 replies; 9+ messages in thread
From: kan.liang @ 2020-01-08 15:03 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>
---

No changes since V2

 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] 9+ messages in thread

* [RFC PATCH V3 5/7] perf/x86: Remove swap_task_ctx()
  2020-01-08 15:03 [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct kan.liang
                   ` (2 preceding siblings ...)
  2020-01-08 15:03 ` [RFC PATCH V3 4/7] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode kan.liang
@ 2020-01-08 15:03 ` kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 6/7] perf: Clean up pmu specific data kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 7/7] perf: Clean up event context from sched_task() kan.liang
  5 siblings, 0 replies; 9+ messages in thread
From: kan.liang @ 2020-01-08 15:03 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>
---

No changes since V2

 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] 9+ messages in thread

* [RFC PATCH V3 6/7] perf: Clean up pmu specific data
  2020-01-08 15:03 [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct kan.liang
                   ` (3 preceding siblings ...)
  2020-01-08 15:03 ` [RFC PATCH V3 5/7] perf/x86: Remove swap_task_ctx() kan.liang
@ 2020-01-08 15:03 ` kan.liang
  2020-01-08 15:03 ` [RFC PATCH V3 7/7] perf: Clean up event context from sched_task() kan.liang
  5 siblings, 0 replies; 9+ messages in thread
From: kan.liang @ 2020-01-08 15:03 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>
---

No changes since V2

 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 f271595..364b0d0 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 1835d9b..23db381 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -1214,7 +1214,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);
 }
 
@@ -3272,28 +3271,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);
@@ -4301,7 +4286,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;
@@ -4325,24 +4309,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)
@@ -4353,11 +4325,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);
 		/*
@@ -4384,11 +4351,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);
 }
 
@@ -12416,19 +12381,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
@@ -12439,7 +12391,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] 9+ messages in thread

* [RFC PATCH V3 7/7] perf: Clean up event context from sched_task()
  2020-01-08 15:03 [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct kan.liang
                   ` (4 preceding siblings ...)
  2020-01-08 15:03 ` [RFC PATCH V3 6/7] perf: Clean up pmu specific data kan.liang
@ 2020-01-08 15:03 ` kan.liang
  5 siblings, 0 replies; 9+ messages in thread
From: kan.liang @ 2020-01-08 15:03 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>
---

No changes since V2

 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 364b0d0..91d2d28 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 23db381..623e520 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -3347,13 +3347,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] 9+ messages in thread

* Re: [RFC PATCH V3 2/7] perf: attach/detach PMU specific data
  2020-01-08 15:03 ` [RFC PATCH V3 2/7] perf: attach/detach PMU specific data kan.liang
@ 2020-01-08 16:50   ` Andi Kleen
  2020-01-08 19:52     ` Liang, Kan
  0 siblings, 1 reply; 9+ messages in thread
From: Andi Kleen @ 2020-01-08 16:50 UTC (permalink / raw)
  To: kan.liang
  Cc: peterz, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy

> +static int
> +attach_system_wide_ctx_data(size_t ctx_size)
> +{
> +	int i, num_thread, pos, nr_failed_alloc;
> +	unsigned long flags = GFP_ATOMIC;
> +	struct perf_ctx_data *tsk_data;
> +	struct perf_ctx_data **data;
> +	struct task_struct *g, *p;
> +	bool re_alloc = true;
> +
> +	/* Retrieve total number of threads */
> +	num_thread = nr_threads;
> +
> +	data = kcalloc(num_thread, sizeof(*data), GFP_KERNEL);

This probably needs kvcalloc for reliability and avoiding stalls.

> +	if (!data) {
> +		printk_once(KERN_DEBUG
> +			    "Failed to allocate space for LBR callstack. "
> +			    "The LBR callstack for all tasks may be cutoff.\n");
> +		return -ENOMEM;
> +	}
> +
> +	atomic_inc(&nr_task_data_sys_wide_events);
> +
> +repeat:
> +	/*
> +	 * Allocate perf_ctx_data for all existing threads.
> +	 * The perf_ctx_data for new threads will be allocated in
> +	 * perf_event_fork().
> +	 * Do a quick allocation in first round with GFP_ATOMIC.
> +	 */
> +	for (i = 0; i < num_thread; i++) {
> +		if (alloc_perf_ctx_data(ctx_size, flags, &data[i]))
> +			break;
> +	}
> +	num_thread = i;
> +	nr_failed_alloc = 0;
> +	pos = 0;
> +

> +	rcu_read_lock();
> +	for_each_process_thread(g, p) {
> +		raw_spin_lock(&p->perf_ctx_data_lock);
> +		tsk_data = p->perf_ctx_data;
> +		if (tsk_data) {

That will be a lot of locks even for tasks that don't use perf, but I guess we 
really need it and it's bounded by the number of tasks.

> +		}
> +
> +		if (pos < num_thread) {
> +			refcount_set(&data[pos]->refcount, TASK_DATA_SYS_WIDE);
> +			rcu_assign_pointer(p->perf_ctx_data, data[pos++]);
> +		} else {
> +			/*
> +			 * The quick allocation in first round may be failed.
> +			 * Track the number in nr_failed_alloc.
> +			 */
> +			nr_failed_alloc++;
> +		}
> +		raw_spin_unlock(&p->perf_ctx_data_lock);
> +	}
> +	rcu_read_unlock();


-Andi

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

* Re: [RFC PATCH V3 2/7] perf: attach/detach PMU specific data
  2020-01-08 16:50   ` Andi Kleen
@ 2020-01-08 19:52     ` Liang, Kan
  0 siblings, 0 replies; 9+ messages in thread
From: Liang, Kan @ 2020-01-08 19:52 UTC (permalink / raw)
  To: Andi Kleen
  Cc: peterz, mingo, acme, tglx, bp, linux-kernel, eranian,
	alexey.budankov, vitaly.slobodskoy



On 1/8/2020 11:50 AM, Andi Kleen wrote:
>> +static int
>> +attach_system_wide_ctx_data(size_t ctx_size)
>> +{
>> +	int i, num_thread, pos, nr_failed_alloc;
>> +	unsigned long flags = GFP_ATOMIC;
>> +	struct perf_ctx_data *tsk_data;
>> +	struct perf_ctx_data **data;
>> +	struct task_struct *g, *p;
>> +	bool re_alloc = true;
>> +
>> +	/* Retrieve total number of threads */
>> +	num_thread = nr_threads;
>> +
>> +	data = kcalloc(num_thread, sizeof(*data), GFP_KERNEL);
> 
> This probably needs kvcalloc for reliability and avoiding stalls.
>

Yes, kvcalloc looks better.

>> +	if (!data) {
>> +		printk_once(KERN_DEBUG
>> +			    "Failed to allocate space for LBR callstack. "
>> +			    "The LBR callstack for all tasks may be cutoff.\n");
>> +		return -ENOMEM;
>> +	}
>> +
>> +	atomic_inc(&nr_task_data_sys_wide_events);
>> +
>> +repeat:
>> +	/*
>> +	 * Allocate perf_ctx_data for all existing threads.
>> +	 * The perf_ctx_data for new threads will be allocated in
>> +	 * perf_event_fork().
>> +	 * Do a quick allocation in first round with GFP_ATOMIC.
>> +	 */
>> +	for (i = 0; i < num_thread; i++) {
>> +		if (alloc_perf_ctx_data(ctx_size, flags, &data[i]))
>> +			break;
>> +	}
>> +	num_thread = i;
>> +	nr_failed_alloc = 0;
>> +	pos = 0;
>> +
> 
>> +	rcu_read_lock();
>> +	for_each_process_thread(g, p) {
>> +		raw_spin_lock(&p->perf_ctx_data_lock);
>> +		tsk_data = p->perf_ctx_data;
>> +		if (tsk_data) {
> 
> That will be a lot of locks even for tasks that don't use perf, but I guess we
> really need it and it's bounded by the number of tasks.

Right. We don't know which tasks will be monitored later. So we have to 
attach the perf_ctx_data for all of them. The per-task lock is required 
to sync the writers of perf_ctx_data RCU pointer.


Thanks,
Kan
> 
>> +		}
>> +
>> +		if (pos < num_thread) {
>> +			refcount_set(&data[pos]->refcount, TASK_DATA_SYS_WIDE);
>> +			rcu_assign_pointer(p->perf_ctx_data, data[pos++]);
>> +		} else {
>> +			/*
>> +			 * The quick allocation in first round may be failed.
>> +			 * Track the number in nr_failed_alloc.
>> +			 */
>> +			nr_failed_alloc++;
>> +		}
>> +		raw_spin_unlock(&p->perf_ctx_data_lock);
>> +	}
>> +	rcu_read_unlock();
> 
> 
> -Andi
> 

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

end of thread, other threads:[~2020-01-08 19:53 UTC | newest]

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-01-08 15:03 [RFC PATCH V3 1/7] perf: Save PMU specific data in task_struct kan.liang
2020-01-08 15:03 ` [RFC PATCH V3 2/7] perf: attach/detach PMU specific data kan.liang
2020-01-08 16:50   ` Andi Kleen
2020-01-08 19:52     ` Liang, Kan
2020-01-08 15:03 ` [RFC PATCH V3 3/7] perf: Supply task information to sched_task() kan.liang
2020-01-08 15:03 ` [RFC PATCH V3 4/7] perf/x86/lbr: Fix shorter LBRs call stacks for system-wide mode kan.liang
2020-01-08 15:03 ` [RFC PATCH V3 5/7] perf/x86: Remove swap_task_ctx() kan.liang
2020-01-08 15:03 ` [RFC PATCH V3 6/7] perf: Clean up pmu specific data kan.liang
2020-01-08 15:03 ` [RFC PATCH V3 7/7] perf: Clean up event context from sched_task() kan.liang

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.