linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v6] perf: Sharing PMU counters across compatible events
@ 2019-09-19  5:23 Song Liu
  2019-09-30  5:36 ` Song Liu
                   ` (2 more replies)
  0 siblings, 3 replies; 13+ messages in thread
From: Song Liu @ 2019-09-19  5:23 UTC (permalink / raw)
  To: linux-kernel
  Cc: Song Liu, kernel-team, acme, Peter Zijlstra,
	Arnaldo Carvalho de Melo, Jiri Olsa, Alexey Budankov,
	Namhyung Kim, Tejun Heo

This patch tries to enable PMU sharing. To make perf event scheduling
fast, we use special data structures.

An array of "struct perf_event_dup" is added to the perf_event_context,
to remember all the duplicated events under this ctx. All the events
under this ctx has a "dup_id" pointing to its perf_event_dup. Compatible
events under the same ctx share the same perf_event_dup. The following
figure shows a simplified version of the data structure.

      ctx ->  perf_event_dup -> master
                     ^
                     |
         perf_event /|
                     |
         perf_event /

Connection among perf_event and perf_event_dup are built when events are
added or removed from the ctx. So these are not on the critical path of
schedule or perf_rotate_context().

On the critical paths (add, del read), sharing PMU counters doesn't
increase the complexity. Helper functions event_pmu_[add|del|read]() are
introduced to cover these cases. All these functions have O(1) time
complexity.

We allocate a separate perf_event for perf_event_dup->master. This needs
extra attention, because perf_event_alloc() may sleep. To allocate the
master event properly, a new pointer, tmp_master, is added to perf_event.
tmp_master carries a separate perf_event into list_[add|del]_event().
The master event has valid ->ctx and holds ctx->refcount.

Details about the handling of the master event is added to
include/linux/perf_event.h, before struct perf_event_dup.

Cc: Peter Zijlstra <peterz@infradead.org>
Cc: Arnaldo Carvalho de Melo <acme@redhat.com>
Cc: Jiri Olsa <jolsa@kernel.org>
Cc: Alexey Budankov <alexey.budankov@linux.intel.com>
Cc: Namhyung Kim <namhyung@kernel.org>
Cc: Tejun Heo <tj@kernel.org>
Signed-off-by: Song Liu <songliubraving@fb.com>
---
 include/linux/perf_event.h |  61 ++++++++
 kernel/events/core.c       | 294 ++++++++++++++++++++++++++++++++++---
 2 files changed, 332 insertions(+), 23 deletions(-)

diff --git a/include/linux/perf_event.h b/include/linux/perf_event.h
index 61448c19a132..a694e5eee80a 100644
--- a/include/linux/perf_event.h
+++ b/include/linux/perf_event.h
@@ -722,6 +722,12 @@ struct perf_event {
 #endif
 
 	struct list_head		sb_list;
+
+	/* for PMU sharing */
+	struct perf_event		*tmp_master;
+	int				dup_id;
+	u64				dup_base_count;
+	u64				dup_base_child_count;
 #endif /* CONFIG_PERF_EVENTS */
 };
 
@@ -731,6 +737,58 @@ struct perf_event_groups {
 	u64		index;
 };
 
+/*
+ * Sharing PMU across compatible events
+ *
+ * If two perf_events in the same perf_event_context are counting same
+ * hardware events (instructions, cycles, etc.), they could share the
+ * hardware PMU counter.
+ *
+ * When a perf_event is added to the ctx (list_add_event), it is compared
+ * against other events in the ctx. If they can share the PMU counter,
+ * a perf_event_dup is allocated to represent the sharing.
+ *
+ * Each perf_event_dup has a virtual master event, which is called by
+ * pmu->add() and pmu->del(). We cannot call perf_event_alloc() in
+ * list_add_event(), so it is allocated and carried by event->tmp_master
+ * into list_add_event().
+ *
+ * Virtual master in different cases/paths:
+ *
+ * < I > perf_event_open() -> close() path:
+ *
+ * 1. Allocated by perf_event_alloc() in sys_perf_event_open();
+ * 2. event->tmp_master->ctx assigned in perf_install_in_context();
+ * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
+ * 3.b. if not used by ctx->dup_events, freed in perf_event_open().
+ *
+ * < II > inherit_event() path:
+ *
+ * 1. Allocated by perf_event_alloc() in inherit_event();
+ * 2. tmp_master->ctx assigned in inherit_event();
+ * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
+ * 3.b. if not used by ctx->dup_events, freed in inherit_event().
+ *
+ * < III > perf_pmu_migrate_context() path:
+ * all dup_events removed during migration (no sharing after the move).
+ *
+ * < IV > perf_event_create_kernel_counter() path:
+ * not supported yet.
+ */
+struct perf_event_dup {
+	/*
+	 * master event being called by pmu->add() and pmu->del().
+	 * This event is allocated with perf_event_alloc(). When
+	 * attached to a ctx, this event should hold ctx->refcount.
+	 */
+	struct perf_event       *master;
+	/* number of events in the ctx that shares the master */
+	int			total_event_count;
+	/* number of active events of the master */
+	int			active_event_count;
+};
+
+#define MAX_PERF_EVENT_DUP_PER_CTX 4
 /**
  * struct perf_event_context - event context structure
  *
@@ -791,6 +849,9 @@ struct perf_event_context {
 #endif
 	void				*task_ctx_data; /* pmu specific data */
 	struct rcu_head			rcu_head;
+
+	/* for PMU sharing. array is needed for O(1) access */
+	struct perf_event_dup		dup_events[MAX_PERF_EVENT_DUP_PER_CTX];
 };
 
 /*
diff --git a/kernel/events/core.c b/kernel/events/core.c
index 4f08b17d6426..973425e9de9b 100644
--- a/kernel/events/core.c
+++ b/kernel/events/core.c
@@ -1657,6 +1657,142 @@ perf_event_groups_next(struct perf_event *event)
 		event = rb_entry_safe(rb_next(&event->group_node),	\
 				typeof(*event), group_node))
 
+static void _free_event(struct perf_event *event);
+
+/* free event->tmp_master */
+static inline void perf_event_free_tmp_master(struct perf_event *event)
+{
+	if (event->tmp_master) {
+		_free_event(event->tmp_master);
+		event->tmp_master = NULL;
+	}
+}
+
+static inline u64 dup_read_count(struct perf_event_dup *dup)
+{
+	return local64_read(&dup->master->count);
+}
+
+static inline u64 dup_read_child_count(struct perf_event_dup *dup)
+{
+	return atomic64_read(&dup->master->child_count);
+}
+
+/* Returns whether a perf_event can share PMU counter with other events */
+static inline bool perf_event_can_share(struct perf_event *event)
+{
+	/* only do sharing for hardware events */
+	if (is_software_event(event))
+		return false;
+
+	/*
+	 * limit sharing to counting events.
+	 * perf-stat sets PERF_SAMPLE_IDENTIFIER for counting events, so
+	 * let that in.
+	 */
+	if (event->attr.sample_type & ~PERF_SAMPLE_IDENTIFIER)
+		return false;
+
+	return true;
+}
+
+/*
+ * Returns whether the two events can share a PMU counter.
+ *
+ * Note: This function does NOT check perf_event_can_share() for
+ * the two events, they should be checked before this function
+ */
+static inline bool perf_event_compatible(struct perf_event *event_a,
+					 struct perf_event *event_b)
+{
+	return event_a->attr.type == event_b->attr.type &&
+		event_a->attr.config == event_b->attr.config &&
+		event_a->attr.config1 == event_b->attr.config1 &&
+		event_a->attr.config2 == event_b->attr.config2;
+}
+
+/*
+ * After adding a event to the ctx, try find compatible event(s).
+ *
+ * This function should only be called at the end of list_add_event().
+ * Master event cannot be allocated or freed within this function. To add
+ * new master event, the event should already have a master event
+ * allocated (event->tmp_master).
+ */
+static inline void perf_event_setup_dup(struct perf_event *event,
+					struct perf_event_context *ctx)
+
+{
+	struct perf_event *tmp;
+	int empty_slot = -1;
+	int match;
+	int i;
+
+	if (!perf_event_can_share(event))
+		return;
+
+	/* look for sharing with existing dup events */
+	for (i = 0; i < MAX_PERF_EVENT_DUP_PER_CTX; i++) {
+		if (ctx->dup_events[i].master == NULL) {
+			if (empty_slot == -1)
+				empty_slot = i;
+			continue;
+		}
+
+		if (perf_event_compatible(event, ctx->dup_events[i].master)) {
+			event->dup_id = i;
+			ctx->dup_events[i].total_event_count++;
+			return;
+		}
+	}
+
+	if (!event->tmp_master ||  /* perf_event_alloc() failed */
+	    empty_slot == -1)      /* no available dup_event */
+		return;
+
+	match = 0;
+	/* look for dup with other events */
+	list_for_each_entry(tmp, &ctx->event_list, event_entry) {
+		if (tmp == event || tmp->dup_id != -1 ||
+		    !perf_event_can_share(tmp) ||
+		    !perf_event_compatible(event, tmp))
+			continue;
+
+		tmp->dup_id = empty_slot;
+		match++;
+	}
+
+	/* found at least one dup */
+	if (match) {
+		ctx->dup_events[empty_slot].master = event->tmp_master;
+		ctx->dup_events[empty_slot].total_event_count = match + 1;
+		event->dup_id = empty_slot;
+		event->tmp_master = NULL;
+		return;
+	}
+}
+
+/*
+ * Remove the event from ctx->dup_events.
+ * This function should only be called from list_del_event(). Similar to
+ * perf_event_setup_dup(), we cannot call _free_event(master). If a master
+ * event should be freed, it is carried out of this function by the event
+ * (event->tmp_master).
+ */
+static void perf_event_remove_dup(struct perf_event *event,
+				  struct perf_event_context *ctx)
+
+{
+	if (event->dup_id == -1)
+		return;
+
+	if (--ctx->dup_events[event->dup_id].total_event_count == 0) {
+		event->tmp_master = ctx->dup_events[event->dup_id].master;
+		ctx->dup_events[event->dup_id].master = NULL;
+	}
+	event->dup_id = -1;
+}
+
 /*
  * Add an event from the lists for its context.
  * Must be called with ctx->mutex and ctx->lock held.
@@ -1689,6 +1825,7 @@ list_add_event(struct perf_event *event, struct perf_event_context *ctx)
 		ctx->nr_stat++;
 
 	ctx->generation++;
+	perf_event_setup_dup(event, ctx);
 }
 
 /*
@@ -1855,6 +1992,7 @@ list_del_event(struct perf_event *event, struct perf_event_context *ctx)
 	WARN_ON_ONCE(event->ctx != ctx);
 	lockdep_assert_held(&ctx->lock);
 
+	perf_event_remove_dup(event, ctx);
 	/*
 	 * We can have double detach due to exit/hot-unplug + close.
 	 */
@@ -2069,6 +2207,84 @@ event_filter_match(struct perf_event *event)
 	       perf_cgroup_match(event) && pmu_filter_match(event);
 }
 
+/* PMU sharing aware version of event->pmu->add() */
+static int event_pmu_add(struct perf_event *event,
+			 struct perf_event_context *ctx)
+{
+	struct perf_event_dup *dup;
+	int ret;
+
+	/* no sharing, just do event->pmu->add() */
+	if (event->dup_id == -1)
+		return event->pmu->add(event, PERF_EF_START);
+
+	dup = &ctx->dup_events[event->dup_id];
+
+	if (!dup->active_event_count) {
+		ret = event->pmu->add(dup->master, PERF_EF_START);
+		if (ret)
+			return ret;
+	}
+
+	dup->active_event_count++;
+	dup->master->pmu->read(dup->master);
+	event->dup_base_count = dup_read_count(dup);
+	event->dup_base_child_count = dup_read_child_count(dup);
+	return 0;
+}
+
+/*
+ * sync data count from dup->master to event, called on event_pmu_read()
+ * and event_pmu_del()
+ */
+static void event_sync_dup_count(struct perf_event *event,
+				 struct perf_event_dup *dup)
+{
+	u64 new_count;
+	u64 new_child_count;
+
+	event->pmu->read(dup->master);
+	new_count = dup_read_count(dup);
+	new_child_count = dup_read_child_count(dup);
+
+	local64_add(new_count - event->dup_base_count, &event->count);
+	atomic64_add(new_child_count - event->dup_base_child_count,
+		     &event->child_count);
+
+	event->dup_base_count = new_count;
+	event->dup_base_child_count = new_child_count;
+}
+
+/* PMU sharing aware version of event->pmu->del() */
+static void event_pmu_del(struct perf_event *event,
+			  struct perf_event_context *ctx)
+{
+	struct perf_event_dup *dup;
+
+	if (event->dup_id == -1) {
+		event->pmu->del(event, 0);
+		return;
+	}
+
+	dup = &ctx->dup_events[event->dup_id];
+	event_sync_dup_count(event, dup);
+	if (--dup->active_event_count == 0)
+		event->pmu->del(dup->master, 0);
+}
+
+/* PMU sharing aware version of event->pmu->read() */
+static void event_pmu_read(struct perf_event *event)
+{
+	struct perf_event_dup *dup;
+
+	if (event->dup_id == -1) {
+		event->pmu->read(event);
+		return;
+	}
+	dup = &event->ctx->dup_events[event->dup_id];
+	event_sync_dup_count(event, dup);
+}
+
 static void
 event_sched_out(struct perf_event *event,
 		  struct perf_cpu_context *cpuctx,
@@ -2091,7 +2307,7 @@ event_sched_out(struct perf_event *event,
 
 	perf_pmu_disable(event->pmu);
 
-	event->pmu->del(event, 0);
+	event_pmu_del(event, ctx);
 	event->oncpu = -1;
 
 	if (READ_ONCE(event->pending_disable) >= 0) {
@@ -2364,7 +2580,7 @@ event_sched_in(struct perf_event *event,
 
 	perf_log_itrace_start(event);
 
-	if (event->pmu->add(event, PERF_EF_START)) {
+	if (event_pmu_add(event, ctx)) {
 		perf_event_set_state(event, PERF_EVENT_STATE_INACTIVE);
 		event->oncpu = -1;
 		ret = -EAGAIN;
@@ -2612,20 +2828,9 @@ static int  __perf_install_in_context(void *info)
 		raw_spin_lock(&task_ctx->lock);
 	}
 
-#ifdef CONFIG_CGROUP_PERF
-	if (is_cgroup_event(event)) {
-		/*
-		 * If the current cgroup doesn't match the event's
-		 * cgroup, we should not try to schedule it.
-		 */
-		struct perf_cgroup *cgrp = perf_cgroup_from_task(current, ctx);
-		reprogram = cgroup_is_descendant(cgrp->css.cgroup,
-					event->cgrp->css.cgroup);
-	}
-#endif
-
 	if (reprogram) {
-		ctx_sched_out(ctx, cpuctx, EVENT_TIME);
+		/* schedule out all events to set up dup properly */
+		ctx_sched_out(ctx, cpuctx, EVENT_ALL);
 		add_event_to_ctx(event, ctx);
 		ctx_resched(cpuctx, task_ctx, get_event_type(event));
 	} else {
@@ -2664,6 +2869,10 @@ perf_install_in_context(struct perf_event_context *ctx,
 	 * Ensures that if we can observe event->ctx, both the event and ctx
 	 * will be 'complete'. See perf_iterate_sb_cpu().
 	 */
+	if (event->tmp_master) {
+		event->tmp_master->ctx = ctx;
+		get_ctx(ctx);
+	}
 	smp_store_release(&event->ctx, ctx);
 
 	if (!task) {
@@ -3115,7 +3324,7 @@ static void __perf_event_sync_stat(struct perf_event *event,
 	 * don't need to use it.
 	 */
 	if (event->state == PERF_EVENT_STATE_ACTIVE)
-		event->pmu->read(event);
+		event_pmu_read(event);
 
 	perf_event_update_time(event);
 
@@ -3967,14 +4176,14 @@ static void __perf_event_read(void *info)
 		goto unlock;
 
 	if (!data->group) {
-		pmu->read(event);
+		event_pmu_read(event);
 		data->ret = 0;
 		goto unlock;
 	}
 
 	pmu->start_txn(pmu, PERF_PMU_TXN_READ);
 
-	pmu->read(event);
+	event_pmu_read(event);
 
 	for_each_sibling_event(sub, event) {
 		if (sub->state == PERF_EVENT_STATE_ACTIVE) {
@@ -3982,7 +4191,7 @@ static void __perf_event_read(void *info)
 			 * Use sibling's PMU rather than @event's since
 			 * sibling could be on different (eg: software) PMU.
 			 */
-			sub->pmu->read(sub);
+			event_pmu_read(sub);
 		}
 	}
 
@@ -4052,7 +4261,7 @@ int perf_event_read_local(struct perf_event *event, u64 *value,
 	 * oncpu == -1).
 	 */
 	if (event->oncpu == smp_processor_id())
-		event->pmu->read(event);
+		event_pmu_read(event);
 
 	*value = local64_read(&event->count);
 	if (enabled || running) {
@@ -4587,6 +4796,7 @@ static void free_event(struct perf_event *event)
 		return;
 	}
 
+	perf_event_free_tmp_master(event);
 	_free_event(event);
 }
 
@@ -4646,6 +4856,7 @@ static void put_event(struct perf_event *event)
 	if (!atomic_long_dec_and_test(&event->refcount))
 		return;
 
+	perf_event_free_tmp_master(event);
 	_free_event(event);
 }
 
@@ -6261,7 +6472,7 @@ static void perf_output_read_group(struct perf_output_handle *handle,
 
 	if ((leader != event) &&
 	    (leader->state == PERF_EVENT_STATE_ACTIVE))
-		leader->pmu->read(leader);
+		event_pmu_read(leader);
 
 	values[n++] = perf_event_count(leader);
 	if (read_format & PERF_FORMAT_ID)
@@ -6274,7 +6485,7 @@ static void perf_output_read_group(struct perf_output_handle *handle,
 
 		if ((sub != event) &&
 		    (sub->state == PERF_EVENT_STATE_ACTIVE))
-			sub->pmu->read(sub);
+			event_pmu_read(sub);
 
 		values[n++] = perf_event_count(sub);
 		if (read_format & PERF_FORMAT_ID)
@@ -9539,7 +9750,7 @@ static enum hrtimer_restart perf_swevent_hrtimer(struct hrtimer *hrtimer)
 	if (event->state != PERF_EVENT_STATE_ACTIVE)
 		return HRTIMER_NORESTART;
 
-	event->pmu->read(event);
+	event_pmu_read(event);
 
 	perf_sample_data_init(&data, 0, event->hw.last_period);
 	regs = get_irq_regs();
@@ -10430,6 +10641,7 @@ perf_event_alloc(struct perf_event_attr *attr, int cpu,
 	event->id		= atomic64_inc_return(&perf_event_id);
 
 	event->state		= PERF_EVENT_STATE_INACTIVE;
+	event->dup_id		= -1;
 
 	if (task) {
 		event->attach_state = PERF_ATTACH_TASK;
@@ -10986,6 +11198,14 @@ SYSCALL_DEFINE5(perf_event_open,
 		goto err_cred;
 	}
 
+	if (perf_event_can_share(event)) {
+		event->tmp_master = perf_event_alloc(&event->attr, cpu,
+						     task, NULL, NULL,
+						     NULL, NULL, -1);
+		if (IS_ERR(event->tmp_master))
+			event->tmp_master = NULL;
+	}
+
 	if (is_sampling_event(event)) {
 		if (event->pmu->capabilities & PERF_PMU_CAP_NO_INTERRUPT) {
 			err = -EOPNOTSUPP;
@@ -11197,9 +11417,17 @@ SYSCALL_DEFINE5(perf_event_open,
 		perf_remove_from_context(group_leader, 0);
 		put_ctx(gctx);
 
+		/*
+		 * move_group only happens to sw events, from sw ctx to hw
+		 * ctx. The sw events should not have valid dup_id. So it
+		 * is not necessary to handle dup_events.
+		 */
+		WARN_ON_ONCE(group_leader->dup_id != -1);
+
 		for_each_sibling_event(sibling, group_leader) {
 			perf_remove_from_context(sibling, 0);
 			put_ctx(gctx);
+			WARN_ON_ONCE(sibling->dup_id != -1);
 		}
 
 		/*
@@ -11257,6 +11485,9 @@ SYSCALL_DEFINE5(perf_event_open,
 		put_task_struct(task);
 	}
 
+	/* if event->tmp_master is not used by ctx->dup_events, free it */
+	perf_event_free_tmp_master(event);
+
 	mutex_lock(&current->perf_event_mutex);
 	list_add_tail(&event->owner_entry, &current->perf_event_list);
 	mutex_unlock(&current->perf_event_mutex);
@@ -11401,6 +11632,7 @@ void perf_pmu_migrate_context(struct pmu *pmu, int src_cpu, int dst_cpu)
 		perf_remove_from_context(event, 0);
 		unaccount_event_cpu(event, src_cpu);
 		put_ctx(src_ctx);
+		perf_event_free_tmp_master(event);
 		list_add(&event->migrate_entry, &events);
 	}
 
@@ -11773,6 +12005,14 @@ inherit_event(struct perf_event *parent_event,
 	if (IS_ERR(child_event))
 		return child_event;
 
+	if (perf_event_can_share(child_event)) {
+		child_event->tmp_master = perf_event_alloc(&parent_event->attr,
+							   parent_event->cpu,
+							   child, NULL, NULL,
+							   NULL, NULL, -1);
+		if (IS_ERR(child_event->tmp_master))
+			child_event->tmp_master = NULL;
+	}
 
 	if ((child_event->attach_state & PERF_ATTACH_TASK_DATA) &&
 	    !child_ctx->task_ctx_data) {
@@ -11827,6 +12067,10 @@ inherit_event(struct perf_event *parent_event,
 	child_event->overflow_handler = parent_event->overflow_handler;
 	child_event->overflow_handler_context
 		= parent_event->overflow_handler_context;
+	if (child_event->tmp_master) {
+		child_event->tmp_master->ctx = child_ctx;
+		get_ctx(child_ctx);
+	}
 
 	/*
 	 * Precalculate sample_data sizes
@@ -11841,6 +12085,7 @@ inherit_event(struct perf_event *parent_event,
 	add_event_to_ctx(child_event, child_ctx);
 	raw_spin_unlock_irqrestore(&child_ctx->lock, flags);
 
+	perf_event_free_tmp_master(child_event);
 	/*
 	 * Link this into the parent event's child list
 	 */
@@ -12112,6 +12357,7 @@ static void perf_event_exit_cpu_context(int cpu)
 {
 	struct perf_cpu_context *cpuctx;
 	struct perf_event_context *ctx;
+	struct perf_event *event;
 	struct pmu *pmu;
 
 	mutex_lock(&pmus_lock);
@@ -12123,6 +12369,8 @@ static void perf_event_exit_cpu_context(int cpu)
 		smp_call_function_single(cpu, __perf_event_exit_context, ctx, 1);
 		cpuctx->online = 0;
 		mutex_unlock(&ctx->mutex);
+		list_for_each_entry(event, &ctx->event_list, event_entry)
+			perf_event_free_tmp_master(event);
 	}
 	cpumask_clear_cpu(cpu, perf_online_mask);
 	mutex_unlock(&pmus_lock);
-- 
2.17.1


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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-09-19  5:23 [PATCH v6] perf: Sharing PMU counters across compatible events Song Liu
@ 2019-09-30  5:36 ` Song Liu
  2019-10-31 12:43 ` Peter Zijlstra
  2019-11-05 23:51 ` Song Liu
  2 siblings, 0 replies; 13+ messages in thread
From: Song Liu @ 2019-09-30  5:36 UTC (permalink / raw)
  To: open list, Peter Zijlstra
  Cc: Kernel Team, Arnaldo Carvalho de Melo, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo

Hi Peter,

> On Sep 18, 2019, at 10:23 PM, Song Liu <songliubraving@fb.com> wrote:
> 
> This patch tries to enable PMU sharing. To make perf event scheduling
> fast, we use special data structures.
> 
> An array of "struct perf_event_dup" is added to the perf_event_context,
> to remember all the duplicated events under this ctx. All the events
> under this ctx has a "dup_id" pointing to its perf_event_dup. Compatible
> events under the same ctx share the same perf_event_dup. The following
> figure shows a simplified version of the data structure.
> 
>      ctx ->  perf_event_dup -> master
>                     ^
>                     |
>         perf_event /|
>                     |
>         perf_event /
> 
> Connection among perf_event and perf_event_dup are built when events are
> added or removed from the ctx. So these are not on the critical path of
> schedule or perf_rotate_context().
> 
> On the critical paths (add, del read), sharing PMU counters doesn't
> increase the complexity. Helper functions event_pmu_[add|del|read]() are
> introduced to cover these cases. All these functions have O(1) time
> complexity.
> 
> We allocate a separate perf_event for perf_event_dup->master. This needs
> extra attention, because perf_event_alloc() may sleep. To allocate the
> master event properly, a new pointer, tmp_master, is added to perf_event.
> tmp_master carries a separate perf_event into list_[add|del]_event().
> The master event has valid ->ctx and holds ctx->refcount.
> 
> Details about the handling of the master event is added to
> include/linux/perf_event.h, before struct perf_event_dup.

Could you please share your comments/suggestions on this work?

Thanks,
Song

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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-09-19  5:23 [PATCH v6] perf: Sharing PMU counters across compatible events Song Liu
  2019-09-30  5:36 ` Song Liu
@ 2019-10-31 12:43 ` Peter Zijlstra
  2019-10-31 16:29   ` Song Liu
  2019-11-05 23:51 ` Song Liu
  2 siblings, 1 reply; 13+ messages in thread
From: Peter Zijlstra @ 2019-10-31 12:43 UTC (permalink / raw)
  To: Song Liu
  Cc: linux-kernel, kernel-team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo

On Wed, Sep 18, 2019 at 10:23:14PM -0700, Song Liu wrote:
> This patch tries to enable PMU sharing. To make perf event scheduling
> fast, we use special data structures.
> 
> An array of "struct perf_event_dup" is added to the perf_event_context,
> to remember all the duplicated events under this ctx. All the events
> under this ctx has a "dup_id" pointing to its perf_event_dup. Compatible
> events under the same ctx share the same perf_event_dup. The following
> figure shows a simplified version of the data structure.
> 
>       ctx ->  perf_event_dup -> master
>                      ^
>                      |
>          perf_event /|
>                      |
>          perf_event /
> 
> Connection among perf_event and perf_event_dup are built when events are
> added or removed from the ctx. So these are not on the critical path of
> schedule or perf_rotate_context().
> 
> On the critical paths (add, del read), sharing PMU counters doesn't
> increase the complexity. Helper functions event_pmu_[add|del|read]() are
> introduced to cover these cases. All these functions have O(1) time
> complexity.
> 
> We allocate a separate perf_event for perf_event_dup->master. This needs
> extra attention, because perf_event_alloc() may sleep. To allocate the
> master event properly, a new pointer, tmp_master, is added to perf_event.
> tmp_master carries a separate perf_event into list_[add|del]_event().
> The master event has valid ->ctx and holds ctx->refcount.

That is realy nasty and expensive, it basically means every !sampling
event carries a double allocate.

Why can't we use one of the actual events as master?

> +/*
> + * Sharing PMU across compatible events
> + *
> + * If two perf_events in the same perf_event_context are counting same
> + * hardware events (instructions, cycles, etc.), they could share the
> + * hardware PMU counter.
> + *
> + * When a perf_event is added to the ctx (list_add_event), it is compared
> + * against other events in the ctx. If they can share the PMU counter,
> + * a perf_event_dup is allocated to represent the sharing.
> + *
> + * Each perf_event_dup has a virtual master event, which is called by
> + * pmu->add() and pmu->del(). We cannot call perf_event_alloc() in
> + * list_add_event(), so it is allocated and carried by event->tmp_master
> + * into list_add_event().
> + *
> + * Virtual master in different cases/paths:
> + *
> + * < I > perf_event_open() -> close() path:
> + *
> + * 1. Allocated by perf_event_alloc() in sys_perf_event_open();
> + * 2. event->tmp_master->ctx assigned in perf_install_in_context();
> + * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
> + * 3.b. if not used by ctx->dup_events, freed in perf_event_open().
> + *
> + * < II > inherit_event() path:
> + *
> + * 1. Allocated by perf_event_alloc() in inherit_event();
> + * 2. tmp_master->ctx assigned in inherit_event();
> + * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
> + * 3.b. if not used by ctx->dup_events, freed in inherit_event().
> + *
> + * < III > perf_pmu_migrate_context() path:
> + * all dup_events removed during migration (no sharing after the move).
> + *
> + * < IV > perf_event_create_kernel_counter() path:
> + * not supported yet.
> + */
> +struct perf_event_dup {
> +	/*
> +	 * master event being called by pmu->add() and pmu->del().
> +	 * This event is allocated with perf_event_alloc(). When
> +	 * attached to a ctx, this event should hold ctx->refcount.
> +	 */
> +	struct perf_event       *master;
> +	/* number of events in the ctx that shares the master */
> +	int			total_event_count;
> +	/* number of active events of the master */
> +	int			active_event_count;
> +};
> +
> +#define MAX_PERF_EVENT_DUP_PER_CTX 4
>  /**
>   * struct perf_event_context - event context structure
>   *
> @@ -791,6 +849,9 @@ struct perf_event_context {
>  #endif
>  	void				*task_ctx_data; /* pmu specific data */
>  	struct rcu_head			rcu_head;
> +
> +	/* for PMU sharing. array is needed for O(1) access */
> +	struct perf_event_dup		dup_events[MAX_PERF_EVENT_DUP_PER_CTX];

Yuck!

event_pmu_{add,del,read}() appear to be the consumer of this array
thing, but I'm not seeing why we need it.

That is, again, why can't we use one of the actual events as master and
have a dup_master pointer per event and then do something like:

event_pmu_add()
{
	if (event->dup_master != event)
		return;

	event->pmu->add(event, PERF_EF_START);
}

Such that we only schedule the master events and ignore all duplicates.

Then on read it can do something like:

event_pmu_read()
{
	if (event->dup_master == event)
		return;

	/* use event->dup_master as counter */
again:
	prev_count = local64_read(&hwc->prev_count);
	count = local64_read(&event->dup_master->count);
	if (local64_cmpxchg(&hwc->prev_count, prev_count, count) != prev_count)
		goto again;

	delta = count - prev_count;
	local64_add(delta, &event->count);
}

>  };

> +/* Returns whether a perf_event can share PMU counter with other events */
> +static inline bool perf_event_can_share(struct perf_event *event)
> +{
> +	/* only do sharing for hardware events */
> +	if (is_software_event(event))
> +		return false;
> +
> +	/*
> +	 * limit sharing to counting events.
> +	 * perf-stat sets PERF_SAMPLE_IDENTIFIER for counting events, so
> +	 * let that in.
> +	 */
> +	if (event->attr.sample_type & ~PERF_SAMPLE_IDENTIFIER)
> +		return false;

Why is is_sampling_event() not usable?

> +
> +	return true;
> +}
> +
> +/*
> + * Returns whether the two events can share a PMU counter.
> + *
> + * Note: This function does NOT check perf_event_can_share() for
> + * the two events, they should be checked before this function
> + */
> +static inline bool perf_event_compatible(struct perf_event *event_a,
> +					 struct perf_event *event_b)
> +{
> +	return event_a->attr.type == event_b->attr.type &&
> +		event_a->attr.config == event_b->attr.config &&
> +		event_a->attr.config1 == event_b->attr.config1 &&
> +		event_a->attr.config2 == event_b->attr.config2;
> +}

Slightly scared by this one.


> @@ -2612,20 +2828,9 @@ static int  __perf_install_in_context(void *info)
>  		raw_spin_lock(&task_ctx->lock);
>  	}
>  
> -#ifdef CONFIG_CGROUP_PERF
> -	if (is_cgroup_event(event)) {
> -		/*
> -		 * If the current cgroup doesn't match the event's
> -		 * cgroup, we should not try to schedule it.
> -		 */
> -		struct perf_cgroup *cgrp = perf_cgroup_from_task(current, ctx);
> -		reprogram = cgroup_is_descendant(cgrp->css.cgroup,
> -					event->cgrp->css.cgroup);
> -	}
> -#endif

Why is this removed?

> @@ -10986,6 +11198,14 @@ SYSCALL_DEFINE5(perf_event_open,
>  		goto err_cred;
>  	}
>  
> +	if (perf_event_can_share(event)) {
> +		event->tmp_master = perf_event_alloc(&event->attr, cpu,
> +						     task, NULL, NULL,
> +						     NULL, NULL, -1);
> +		if (IS_ERR(event->tmp_master))
> +			event->tmp_master = NULL;
> +	}


> @@ -11773,6 +12005,14 @@ inherit_event(struct perf_event *parent_event,
>  	if (IS_ERR(child_event))
>  		return child_event;
>  
> +	if (perf_event_can_share(child_event)) {
> +		child_event->tmp_master = perf_event_alloc(&parent_event->attr,
> +							   parent_event->cpu,
> +							   child, NULL, NULL,
> +							   NULL, NULL, -1);
> +		if (IS_ERR(child_event->tmp_master))
> +			child_event->tmp_master = NULL;
> +	}

So this is terrible!

> 

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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-10-31 12:43 ` Peter Zijlstra
@ 2019-10-31 16:29   ` Song Liu
  2019-11-05 17:11     ` Song Liu
  0 siblings, 1 reply; 13+ messages in thread
From: Song Liu @ 2019-10-31 16:29 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo



> On Oct 31, 2019, at 5:43 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Wed, Sep 18, 2019 at 10:23:14PM -0700, Song Liu wrote:
>> This patch tries to enable PMU sharing. To make perf event scheduling
>> fast, we use special data structures.
>> 
>> An array of "struct perf_event_dup" is added to the perf_event_context,
>> to remember all the duplicated events under this ctx. All the events
>> under this ctx has a "dup_id" pointing to its perf_event_dup. Compatible
>> events under the same ctx share the same perf_event_dup. The following
>> figure shows a simplified version of the data structure.
>> 
>>      ctx ->  perf_event_dup -> master
>>                     ^
>>                     |
>>         perf_event /|
>>                     |
>>         perf_event /
>> 
>> Connection among perf_event and perf_event_dup are built when events are
>> added or removed from the ctx. So these are not on the critical path of
>> schedule or perf_rotate_context().
>> 
>> On the critical paths (add, del read), sharing PMU counters doesn't
>> increase the complexity. Helper functions event_pmu_[add|del|read]() are
>> introduced to cover these cases. All these functions have O(1) time
>> complexity.
>> 
>> We allocate a separate perf_event for perf_event_dup->master. This needs
>> extra attention, because perf_event_alloc() may sleep. To allocate the
>> master event properly, a new pointer, tmp_master, is added to perf_event.
>> tmp_master carries a separate perf_event into list_[add|del]_event().
>> The master event has valid ->ctx and holds ctx->refcount.
> 
> That is realy nasty and expensive, it basically means every !sampling
> event carries a double allocate.
> 
> Why can't we use one of the actual events as master?

I think we can use one of the event as master. We need to be careful when
the master event is removed, but it should be doable. Let me try. 

> 
>> +/*
>> + * Sharing PMU across compatible events
>> + *
>> + * If two perf_events in the same perf_event_context are counting same
>> + * hardware events (instructions, cycles, etc.), they could share the
>> + * hardware PMU counter.
>> + *
>> + * When a perf_event is added to the ctx (list_add_event), it is compared
>> + * against other events in the ctx. If they can share the PMU counter,
>> + * a perf_event_dup is allocated to represent the sharing.
>> + *
>> + * Each perf_event_dup has a virtual master event, which is called by
>> + * pmu->add() and pmu->del(). We cannot call perf_event_alloc() in
>> + * list_add_event(), so it is allocated and carried by event->tmp_master
>> + * into list_add_event().
>> + *
>> + * Virtual master in different cases/paths:
>> + *
>> + * < I > perf_event_open() -> close() path:
>> + *
>> + * 1. Allocated by perf_event_alloc() in sys_perf_event_open();
>> + * 2. event->tmp_master->ctx assigned in perf_install_in_context();
>> + * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
>> + * 3.b. if not used by ctx->dup_events, freed in perf_event_open().
>> + *
>> + * < II > inherit_event() path:
>> + *
>> + * 1. Allocated by perf_event_alloc() in inherit_event();
>> + * 2. tmp_master->ctx assigned in inherit_event();
>> + * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
>> + * 3.b. if not used by ctx->dup_events, freed in inherit_event().
>> + *
>> + * < III > perf_pmu_migrate_context() path:
>> + * all dup_events removed during migration (no sharing after the move).
>> + *
>> + * < IV > perf_event_create_kernel_counter() path:
>> + * not supported yet.
>> + */
>> +struct perf_event_dup {
>> +	/*
>> +	 * master event being called by pmu->add() and pmu->del().
>> +	 * This event is allocated with perf_event_alloc(). When
>> +	 * attached to a ctx, this event should hold ctx->refcount.
>> +	 */
>> +	struct perf_event       *master;
>> +	/* number of events in the ctx that shares the master */
>> +	int			total_event_count;
>> +	/* number of active events of the master */
>> +	int			active_event_count;
>> +};
>> +
>> +#define MAX_PERF_EVENT_DUP_PER_CTX 4
>> /**
>>  * struct perf_event_context - event context structure
>>  *
>> @@ -791,6 +849,9 @@ struct perf_event_context {
>> #endif
>> 	void				*task_ctx_data; /* pmu specific data */
>> 	struct rcu_head			rcu_head;
>> +
>> +	/* for PMU sharing. array is needed for O(1) access */
>> +	struct perf_event_dup		dup_events[MAX_PERF_EVENT_DUP_PER_CTX];
> 
> Yuck!
> 
> event_pmu_{add,del,read}() appear to be the consumer of this array
> thing, but I'm not seeing why we need it.
> 
> That is, again, why can't we use one of the actual events as master and
> have a dup_master pointer per event and then do something like:
> 
> event_pmu_add()
> {
> 	if (event->dup_master != event)
> 		return;
> 
> 	event->pmu->add(event, PERF_EF_START);
> }
> 
> Such that we only schedule the master events and ignore all duplicates.
> 
> Then on read it can do something like:
> 
> event_pmu_read()
> {
> 	if (event->dup_master == event)
> 		return;
> 
> 	/* use event->dup_master as counter */
> again:
> 	prev_count = local64_read(&hwc->prev_count);
> 	count = local64_read(&event->dup_master->count);
> 	if (local64_cmpxchg(&hwc->prev_count, prev_count, count) != prev_count)
> 		goto again;
> 
> 	delta = count - prev_count;
> 	local64_add(delta, &event->count);
> }
> 
>> };
> 
>> +/* Returns whether a perf_event can share PMU counter with other events */
>> +static inline bool perf_event_can_share(struct perf_event *event)
>> +{
>> +	/* only do sharing for hardware events */
>> +	if (is_software_event(event))
>> +		return false;
>> +
>> +	/*
>> +	 * limit sharing to counting events.
>> +	 * perf-stat sets PERF_SAMPLE_IDENTIFIER for counting events, so
>> +	 * let that in.
>> +	 */
>> +	if (event->attr.sample_type & ~PERF_SAMPLE_IDENTIFIER)
>> +		return false;
> 
> Why is is_sampling_event() not usable?

Hmm... let me try it. Thanks for the pointer. 

> 
>> +
>> +	return true;
>> +}
>> +
>> +/*
>> + * Returns whether the two events can share a PMU counter.
>> + *
>> + * Note: This function does NOT check perf_event_can_share() for
>> + * the two events, they should be checked before this function
>> + */
>> +static inline bool perf_event_compatible(struct perf_event *event_a,
>> +					 struct perf_event *event_b)
>> +{
>> +	return event_a->attr.type == event_b->attr.type &&
>> +		event_a->attr.config == event_b->attr.config &&
>> +		event_a->attr.config1 == event_b->attr.config1 &&
>> +		event_a->attr.config2 == event_b->attr.config2;
>> +}
> 
> Slightly scared by this one.

I feel a little nervous too. Maybe we should memcmp the two attr?

> 
> 
>> @@ -2612,20 +2828,9 @@ static int  __perf_install_in_context(void *info)
>> 		raw_spin_lock(&task_ctx->lock);
>> 	}
>> 
>> -#ifdef CONFIG_CGROUP_PERF
>> -	if (is_cgroup_event(event)) {
>> -		/*
>> -		 * If the current cgroup doesn't match the event's
>> -		 * cgroup, we should not try to schedule it.
>> -		 */
>> -		struct perf_cgroup *cgrp = perf_cgroup_from_task(current, ctx);
>> -		reprogram = cgroup_is_descendant(cgrp->css.cgroup,
>> -					event->cgrp->css.cgroup);
>> -	}
>> -#endif
> 
> Why is this removed?

e... I bet I messed this up during a rebase... Sorry..

> 
>> @@ -10986,6 +11198,14 @@ SYSCALL_DEFINE5(perf_event_open,
>> 		goto err_cred;
>> 	}
>> 
>> +	if (perf_event_can_share(event)) {
>> +		event->tmp_master = perf_event_alloc(&event->attr, cpu,
>> +						     task, NULL, NULL,
>> +						     NULL, NULL, -1);
>> +		if (IS_ERR(event->tmp_master))
>> +			event->tmp_master = NULL;
>> +	}
> 
> 
>> @@ -11773,6 +12005,14 @@ inherit_event(struct perf_event *parent_event,
>> 	if (IS_ERR(child_event))
>> 		return child_event;
>> 
>> +	if (perf_event_can_share(child_event)) {
>> +		child_event->tmp_master = perf_event_alloc(&parent_event->attr,
>> +							   parent_event->cpu,
>> +							   child, NULL, NULL,
>> +							   NULL, NULL, -1);
>> +		if (IS_ERR(child_event->tmp_master))
>> +			child_event->tmp_master = NULL;
>> +	}
> 
> So this is terrible!

Let me try get rid of the double alloc. 

Thanks for these feedback!
Song


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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-10-31 16:29   ` Song Liu
@ 2019-11-05 17:11     ` Song Liu
  2019-11-05 20:16       ` Peter Zijlstra
  0 siblings, 1 reply; 13+ messages in thread
From: Song Liu @ 2019-11-05 17:11 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo


Hi Peter, 

> On Oct 31, 2019, at 9:29 AM, Song Liu <songliubraving@fb.com> wrote:
> 
>> On Oct 31, 2019, at 5:43 AM, Peter Zijlstra <peterz@infradead.org> wrote:
>> 
>> On Wed, Sep 18, 2019 at 10:23:14PM -0700, Song Liu wrote:
>>> This patch tries to enable PMU sharing. To make perf event scheduling
>>> fast, we use special data structures.
>>> 
>>> An array of "struct perf_event_dup" is added to the perf_event_context,
>>> to remember all the duplicated events under this ctx. All the events
>>> under this ctx has a "dup_id" pointing to its perf_event_dup. Compatible
>>> events under the same ctx share the same perf_event_dup. The following
>>> figure shows a simplified version of the data structure.
>>> 
>>>     ctx ->  perf_event_dup -> master
>>>                    ^
>>>                    |
>>>        perf_event /|
>>>                    |
>>>        perf_event /
>>> 
>>> Connection among perf_event and perf_event_dup are built when events are
>>> added or removed from the ctx. So these are not on the critical path of
>>> schedule or perf_rotate_context().
>>> 
>>> On the critical paths (add, del read), sharing PMU counters doesn't
>>> increase the complexity. Helper functions event_pmu_[add|del|read]() are
>>> introduced to cover these cases. All these functions have O(1) time
>>> complexity.
>>> 
>>> We allocate a separate perf_event for perf_event_dup->master. This needs
>>> extra attention, because perf_event_alloc() may sleep. To allocate the
>>> master event properly, a new pointer, tmp_master, is added to perf_event.
>>> tmp_master carries a separate perf_event into list_[add|del]_event().
>>> The master event has valid ->ctx and holds ctx->refcount.
>> 
>> That is realy nasty and expensive, it basically means every !sampling
>> event carries a double allocate.
>> 
>> Why can't we use one of the actual events as master?
> 
> I think we can use one of the event as master. We need to be careful when
> the master event is removed, but it should be doable. Let me try. 

Actually, there is a bigger issue when we use one event as the master: what
shall we do if the master event is not running? Say it is an cgroup event, 
and the cgroup is not running on this cpu. An extra master (and all these
array hacks) help us get O(1) complexity in such scenario. 

Do you have suggestions on how to solve this problem? Maybe we can keep the 
extra master, and try get rid of the double alloc? 

Thanks,
Song



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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-11-05 17:11     ` Song Liu
@ 2019-11-05 20:16       ` Peter Zijlstra
  2019-11-05 23:06         ` Song Liu
  0 siblings, 1 reply; 13+ messages in thread
From: Peter Zijlstra @ 2019-11-05 20:16 UTC (permalink / raw)
  To: Song Liu
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo

On Tue, Nov 05, 2019 at 05:11:08PM +0000, Song Liu wrote:

> > I think we can use one of the event as master. We need to be careful when
> > the master event is removed, but it should be doable. Let me try. 
> 
> Actually, there is a bigger issue when we use one event as the master: what
> shall we do if the master event is not running? Say it is an cgroup event, 
> and the cgroup is not running on this cpu. An extra master (and all these
> array hacks) help us get O(1) complexity in such scenario. 
> 
> Do you have suggestions on how to solve this problem? Maybe we can keep the 
> extra master, and try get rid of the double alloc? 

Right, you have to consider scope when sharing. The master should be the
largest scope event and any slaves should be complete subsets.

Without much thought this seems a fairly straight forward constraint;
that is, given cgroups I'm not immediately seeing how we can violate
that.

Basically, pick the cgroup event nearest to the root as the master.
We have to have logic to re-elect the master anyway for deletion, so
changing it on add shouldn't be different.

(obviously the root-cgroup is cpu-wide and always on, and if you have
two events from disjoint subtrees they have no overlap, so it doesn't
make sense to share anyway)


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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-11-05 20:16       ` Peter Zijlstra
@ 2019-11-05 23:06         ` Song Liu
  2019-11-06  9:14           ` Peter Zijlstra
  0 siblings, 1 reply; 13+ messages in thread
From: Song Liu @ 2019-11-05 23:06 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo



> On Nov 5, 2019, at 12:16 PM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Nov 05, 2019 at 05:11:08PM +0000, Song Liu wrote:
> 
>>> I think we can use one of the event as master. We need to be careful when
>>> the master event is removed, but it should be doable. Let me try. 
>> 
>> Actually, there is a bigger issue when we use one event as the master: what
>> shall we do if the master event is not running? Say it is an cgroup event, 
>> and the cgroup is not running on this cpu. An extra master (and all these
>> array hacks) help us get O(1) complexity in such scenario. 
>> 
>> Do you have suggestions on how to solve this problem? Maybe we can keep the 
>> extra master, and try get rid of the double alloc? 
> 
> Right, you have to consider scope when sharing. The master should be the
> largest scope event and any slaves should be complete subsets.
> 
> Without much thought this seems a fairly straight forward constraint;
> that is, given cgroups I'm not immediately seeing how we can violate
> that.
> 
> Basically, pick the cgroup event nearest to the root as the master.
> We have to have logic to re-elect the master anyway for deletion, so
> changing it on add shouldn't be different.
> 
> (obviously the root-cgroup is cpu-wide and always on, and if you have
> two events from disjoint subtrees they have no overlap, so it doesn't
> make sense to share anyway)

Hmm... I didn't think about cgroup structure with this much detail. And 
this is very interesting idea. 

OTOH, non-cgroup event could also be inactive. For example, when we have 
to rotate events, we may schedule slave before master. And if the master
is in an event group, it will be more complicated...

Currently, we already have two separate scopes in sharing: one for cpu_ctx, 
the other for task_ctx. I would like to enable as much sharing as possible
with in each ctx. 

Let me double check whether we can make the code with extra master clearer, 
namely, get rid of double alloc and the ugly array. 

Thanks,
Song


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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-09-19  5:23 [PATCH v6] perf: Sharing PMU counters across compatible events Song Liu
  2019-09-30  5:36 ` Song Liu
  2019-10-31 12:43 ` Peter Zijlstra
@ 2019-11-05 23:51 ` Song Liu
  2019-11-06  8:56   ` Peter Zijlstra
  2 siblings, 1 reply; 13+ messages in thread
From: Song Liu @ 2019-11-05 23:51 UTC (permalink / raw)
  To: open list
  Cc: Kernel Team, acme, Peter Zijlstra, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo

More details on where I am heading...

> On Sep 18, 2019, at 10:23 PM, Song Liu <songliubraving@fb.com> wrote:
> 
> This patch tries to enable PMU sharing. To make perf event scheduling
> fast, we use special data structures.
> 
> An array of "struct perf_event_dup" is added to the perf_event_context,
> to remember all the duplicated events under this ctx. All the events
> under this ctx has a "dup_id" pointing to its perf_event_dup. Compatible
> events under the same ctx share the same perf_event_dup. The following
> figure shows a simplified version of the data structure.
> 
>      ctx ->  perf_event_dup -> master
>                     ^
>                     |
>         perf_event /|
>                     |
>         perf_event /
> 
> Connection among perf_event and perf_event_dup are built when events are
> added or removed from the ctx. So these are not on the critical path of
> schedule or perf_rotate_context().
> 
> On the critical paths (add, del read), sharing PMU counters doesn't
> increase the complexity. Helper functions event_pmu_[add|del|read]() are
> introduced to cover these cases. All these functions have O(1) time
> complexity.
> 
> We allocate a separate perf_event for perf_event_dup->master. This needs
> extra attention, because perf_event_alloc() may sleep. To allocate the
> master event properly, a new pointer, tmp_master, is added to perf_event.
> tmp_master carries a separate perf_event into list_[add|del]_event().
> The master event has valid ->ctx and holds ctx->refcount.

If we do GFP_ATOMIC in perf_event_alloc(), maybe with an extra option, we
don't need the tmp_master hack. So we only allocate master when we will 
use it. 

> 
> Details about the handling of the master event is added to
> include/linux/perf_event.h, before struct perf_event_dup.
> 
> Cc: Peter Zijlstra <peterz@infradead.org>
> Cc: Arnaldo Carvalho de Melo <acme@redhat.com>
> Cc: Jiri Olsa <jolsa@kernel.org>
> Cc: Alexey Budankov <alexey.budankov@linux.intel.com>
> Cc: Namhyung Kim <namhyung@kernel.org>
> Cc: Tejun Heo <tj@kernel.org>
> Signed-off-by: Song Liu <songliubraving@fb.com>
> ---
> include/linux/perf_event.h |  61 ++++++++
> kernel/events/core.c       | 294 ++++++++++++++++++++++++++++++++++---
> 2 files changed, 332 insertions(+), 23 deletions(-)
> 
> diff --git a/include/linux/perf_event.h b/include/linux/perf_event.h
> index 61448c19a132..a694e5eee80a 100644
> --- a/include/linux/perf_event.h
> +++ b/include/linux/perf_event.h
> @@ -722,6 +722,12 @@ struct perf_event {
> #endif
> 
> 	struct list_head		sb_list;
> +
> +	/* for PMU sharing */
> +	struct perf_event		*tmp_master;
> +	int				dup_id;

I guess we can get rid of dup_id here, and just have 
struct perf_event *dup_master. 

> +	u64				dup_base_count;
> +	u64				dup_base_child_count;
> #endif /* CONFIG_PERF_EVENTS */
> };
> 
> @@ -731,6 +737,58 @@ struct perf_event_groups {
> 	u64		index;
> };
> 
> +/*
> + * Sharing PMU across compatible events
> + *
> + * If two perf_events in the same perf_event_context are counting same
> + * hardware events (instructions, cycles, etc.), they could share the
> + * hardware PMU counter.
> + *
> + * When a perf_event is added to the ctx (list_add_event), it is compared
> + * against other events in the ctx. If they can share the PMU counter,
> + * a perf_event_dup is allocated to represent the sharing.
> + *
> + * Each perf_event_dup has a virtual master event, which is called by
> + * pmu->add() and pmu->del(). We cannot call perf_event_alloc() in
> + * list_add_event(), so it is allocated and carried by event->tmp_master
> + * into list_add_event().
> + *
> + * Virtual master in different cases/paths:
> + *
> + * < I > perf_event_open() -> close() path:
> + *
> + * 1. Allocated by perf_event_alloc() in sys_perf_event_open();
> + * 2. event->tmp_master->ctx assigned in perf_install_in_context();
> + * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
> + * 3.b. if not used by ctx->dup_events, freed in perf_event_open().
> + *
> + * < II > inherit_event() path:
> + *
> + * 1. Allocated by perf_event_alloc() in inherit_event();
> + * 2. tmp_master->ctx assigned in inherit_event();
> + * 3.a. if used by ctx->dup_events, freed in perf_event_release_kernel();
> + * 3.b. if not used by ctx->dup_events, freed in inherit_event().
> + *
> + * < III > perf_pmu_migrate_context() path:
> + * all dup_events removed during migration (no sharing after the move).
> + *
> + * < IV > perf_event_create_kernel_counter() path:
> + * not supported yet.
> + */
> +struct perf_event_dup {
> +	/*
> +	 * master event being called by pmu->add() and pmu->del().
> +	 * This event is allocated with perf_event_alloc(). When
> +	 * attached to a ctx, this event should hold ctx->refcount.
> +	 */
> +	struct perf_event       *master;
> +	/* number of events in the ctx that shares the master */
> +	int			total_event_count;
> +	/* number of active events of the master */
> +	int			active_event_count;
> +};

And hopefully get rid of this. 

Please let me know if this doesn't work. 

Thanks,
Song



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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-11-05 23:51 ` Song Liu
@ 2019-11-06  8:56   ` Peter Zijlstra
  0 siblings, 0 replies; 13+ messages in thread
From: Peter Zijlstra @ 2019-11-06  8:56 UTC (permalink / raw)
  To: Song Liu
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo

On Tue, Nov 05, 2019 at 11:51:42PM +0000, Song Liu wrote:

> > We allocate a separate perf_event for perf_event_dup->master. This needs
> > extra attention, because perf_event_alloc() may sleep. To allocate the
> > master event properly, a new pointer, tmp_master, is added to perf_event.
> > tmp_master carries a separate perf_event into list_[add|del]_event().
> > The master event has valid ->ctx and holds ctx->refcount.
> 
> If we do GFP_ATOMIC in perf_event_alloc(), maybe with an extra option, we
> don't need the tmp_master hack. So we only allocate master when we will 
> use it. 

You can't, that's broken on -RT. ctx->lock is a raw_spinlock_t and
allocator locks are spinlock_t.

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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-11-05 23:06         ` Song Liu
@ 2019-11-06  9:14           ` Peter Zijlstra
  2019-11-06 17:40             ` Song Liu
  0 siblings, 1 reply; 13+ messages in thread
From: Peter Zijlstra @ 2019-11-06  9:14 UTC (permalink / raw)
  To: Song Liu
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo

On Tue, Nov 05, 2019 at 11:06:06PM +0000, Song Liu wrote:
> 
> 
> > On Nov 5, 2019, at 12:16 PM, Peter Zijlstra <peterz@infradead.org> wrote:
> > 
> > On Tue, Nov 05, 2019 at 05:11:08PM +0000, Song Liu wrote:
> > 
> >>> I think we can use one of the event as master. We need to be careful when
> >>> the master event is removed, but it should be doable. Let me try. 
> >> 
> >> Actually, there is a bigger issue when we use one event as the master: what
> >> shall we do if the master event is not running? Say it is an cgroup event, 
> >> and the cgroup is not running on this cpu. An extra master (and all these
> >> array hacks) help us get O(1) complexity in such scenario. 
> >> 
> >> Do you have suggestions on how to solve this problem? Maybe we can keep the 
> >> extra master, and try get rid of the double alloc? 
> > 
> > Right, you have to consider scope when sharing. The master should be the
> > largest scope event and any slaves should be complete subsets.
> > 
> > Without much thought this seems a fairly straight forward constraint;
> > that is, given cgroups I'm not immediately seeing how we can violate
> > that.
> > 
> > Basically, pick the cgroup event nearest to the root as the master.
> > We have to have logic to re-elect the master anyway for deletion, so
> > changing it on add shouldn't be different.
> > 
> > (obviously the root-cgroup is cpu-wide and always on, and if you have
> > two events from disjoint subtrees they have no overlap, so it doesn't
> > make sense to share anyway)
> 
> Hmm... I didn't think about cgroup structure with this much detail. And 
> this is very interesting idea. 
> 
> OTOH, non-cgroup event could also be inactive. For example, when we have 
> to rotate events, we may schedule slave before master. 

Right, although I suppose in that case you can do what you did in your
patch here. If someone did IOC_DISABLE on the master, we'd have to
re-elect a master -- obviously (and IOC_ENABLE).

> And if the master is in an event group, it will be more complicated...

Hurmph, do you actually have that use-case? And yes, this one is tricky.

Would it be sufficient if we disallow group events to be master (but
allow them to be slaves) ?

> Currently, we already have two separate scopes in sharing: one for cpu_ctx, 
> the other for task_ctx. I would like to enable as much sharing as possible
> with in each ctx. 

Right, although at plumbers you mentioned the idea of sticking
per-task-per-cpu events on the cpu context (as opposed on the task
context where they live today), which is interesting (it's basically an
extention of the cgroup scheduling to per-task scope).

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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-11-06  9:14           ` Peter Zijlstra
@ 2019-11-06 17:40             ` Song Liu
  2019-11-06 20:44               ` Peter Zijlstra
  0 siblings, 1 reply; 13+ messages in thread
From: Song Liu @ 2019-11-06 17:40 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo



> On Nov 6, 2019, at 1:14 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Tue, Nov 05, 2019 at 11:06:06PM +0000, Song Liu wrote:
>> 
>> 
>>> On Nov 5, 2019, at 12:16 PM, Peter Zijlstra <peterz@infradead.org> wrote:
>>> 
>>> On Tue, Nov 05, 2019 at 05:11:08PM +0000, Song Liu wrote:
>>> 
>>>>> I think we can use one of the event as master. We need to be careful when
>>>>> the master event is removed, but it should be doable. Let me try. 
>>>> 
>>>> Actually, there is a bigger issue when we use one event as the master: what
>>>> shall we do if the master event is not running? Say it is an cgroup event, 
>>>> and the cgroup is not running on this cpu. An extra master (and all these
>>>> array hacks) help us get O(1) complexity in such scenario. 
>>>> 
>>>> Do you have suggestions on how to solve this problem? Maybe we can keep the 
>>>> extra master, and try get rid of the double alloc? 
>>> 
>>> Right, you have to consider scope when sharing. The master should be the
>>> largest scope event and any slaves should be complete subsets.
>>> 
>>> Without much thought this seems a fairly straight forward constraint;
>>> that is, given cgroups I'm not immediately seeing how we can violate
>>> that.
>>> 
>>> Basically, pick the cgroup event nearest to the root as the master.
>>> We have to have logic to re-elect the master anyway for deletion, so
>>> changing it on add shouldn't be different.
>>> 
>>> (obviously the root-cgroup is cpu-wide and always on, and if you have
>>> two events from disjoint subtrees they have no overlap, so it doesn't
>>> make sense to share anyway)
>> 
>> Hmm... I didn't think about cgroup structure with this much detail. And 
>> this is very interesting idea. 
>> 
>> OTOH, non-cgroup event could also be inactive. For example, when we have 
>> to rotate events, we may schedule slave before master. 
> 
> Right, although I suppose in that case you can do what you did in your
> patch here. If someone did IOC_DISABLE on the master, we'd have to
> re-elect a master -- obviously (and IOC_ENABLE).

Re-elect master on IOC_DISABLE is good. But we still need work for ctx
rotation. Otherwise, we need keep the master on at all time. 

> 
>> And if the master is in an event group, it will be more complicated...
> 
> Hurmph, do you actually have that use-case? And yes, this one is tricky.
> 
> Would it be sufficient if we disallow group events to be master (but
> allow them to be slaves) ?

Maybe we can solve this with an extra "first_active" pointer in perf_event.
first_active points to the first event that being added by event_pmu_add(). 
Then we need something like:

event_pmu_add(event)
{
	if (event->dup_master->first_active) {
		/* sync with first_active */
	} else {
		/* this event will be the first_active */
		event->dup_master->first_active = event;
		pmu->add(event);
	}
}

However, I just realized the event_pmu_del() path need some more thoughts, 
because first_active is likely the first one get sched_out(). 

Merging another email here:

>> If we do GFP_ATOMIC in perf_event_alloc(), maybe with an extra option, we
>> don't need the tmp_master hack. So we only allocate master when we will 
>> use it. 
> 
> You can't, that's broken on -RT. ctx->lock is a raw_spinlock_t and
> allocator locks are spinlock_t.

How about we add another step in __perf_install_in_context(), like

__perf_install_in_context()
{
	bool alloc_master;

	perf_ctx_lock();
	alloc_master = find_new_sharing(event, ctx);
	perf_ctx_unlock();
	
	if (alloc_master)
		event->dup_master = perf_event_alloc();

	/* existing logic of __perf_install_in_context() */

}

In this way, we only allocate the master event when necessary, and it
is outside of the locks. 

Thanks,
Song



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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-11-06 17:40             ` Song Liu
@ 2019-11-06 20:44               ` Peter Zijlstra
  2019-11-06 22:23                 ` Song Liu
  0 siblings, 1 reply; 13+ messages in thread
From: Peter Zijlstra @ 2019-11-06 20:44 UTC (permalink / raw)
  To: Song Liu
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo

On Wed, Nov 06, 2019 at 05:40:29PM +0000, Song Liu wrote:
> > On Nov 6, 2019, at 1:14 AM, Peter Zijlstra <peterz@infradead.org> wrote:

> >> OTOH, non-cgroup event could also be inactive. For example, when we have 
> >> to rotate events, we may schedule slave before master. 
> > 
> > Right, although I suppose in that case you can do what you did in your
> > patch here. If someone did IOC_DISABLE on the master, we'd have to
> > re-elect a master -- obviously (and IOC_ENABLE).
> 
> Re-elect master on IOC_DISABLE is good. But we still need work for ctx
> rotation. Otherwise, we need keep the master on at all time. 

I meant to says that for the rotation case we can do as you did here, if
we do add() on a slave, add the master if it wasn't add()'ed yet.

> >> And if the master is in an event group, it will be more complicated...
> > 
> > Hurmph, do you actually have that use-case? And yes, this one is tricky.
> > 
> > Would it be sufficient if we disallow group events to be master (but
> > allow them to be slaves) ?
> 
> Maybe we can solve this with an extra "first_active" pointer in perf_event.
> first_active points to the first event that being added by event_pmu_add(). 
> Then we need something like:
> 
> event_pmu_add(event)
> {
> 	if (event->dup_master->first_active) {
> 		/* sync with first_active */
> 	} else {
> 		/* this event will be the first_active */
> 		event->dup_master->first_active = event;
> 		pmu->add(event);
> 	}
> }

I'm confused on what exactly you're trying to solve with the
first_active thing. The problem with the group event as master is that
you then _must_ schedule the whole group, which is obviously difficult.

> >> If we do GFP_ATOMIC in perf_event_alloc(), maybe with an extra option, we
> >> don't need the tmp_master hack. So we only allocate master when we will 
> >> use it. 
> > 
> > You can't, that's broken on -RT. ctx->lock is a raw_spinlock_t and
> > allocator locks are spinlock_t.
> 
> How about we add another step in __perf_install_in_context(), like
> 
> __perf_install_in_context()
> {
> 	bool alloc_master;
> 
> 	perf_ctx_lock();
> 	alloc_master = find_new_sharing(event, ctx);
> 	perf_ctx_unlock();
> 	
> 	if (alloc_master)
> 		event->dup_master = perf_event_alloc();
> 	/* existing logic of __perf_install_in_context() */
> 
> }
> 
> In this way, we only allocate the master event when necessary, and it
> is outside of the locks. 

It's still broken on -RT, because __perf_install_in_context() is in
hardirq context (IPI) and the allocator locks are spinlock_t.


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

* Re: [PATCH v6] perf: Sharing PMU counters across compatible events
  2019-11-06 20:44               ` Peter Zijlstra
@ 2019-11-06 22:23                 ` Song Liu
  0 siblings, 0 replies; 13+ messages in thread
From: Song Liu @ 2019-11-06 22:23 UTC (permalink / raw)
  To: Peter Zijlstra
  Cc: open list, Kernel Team, acme, Arnaldo Carvalho de Melo,
	Jiri Olsa, Alexey Budankov, Namhyung Kim, Tejun Heo



> On Nov 6, 2019, at 12:44 PM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
> On Wed, Nov 06, 2019 at 05:40:29PM +0000, Song Liu wrote:
>>> On Nov 6, 2019, at 1:14 AM, Peter Zijlstra <peterz@infradead.org> wrote:
> 
>>>> OTOH, non-cgroup event could also be inactive. For example, when we have 
>>>> to rotate events, we may schedule slave before master. 
>>> 
>>> Right, although I suppose in that case you can do what you did in your
>>> patch here. If someone did IOC_DISABLE on the master, we'd have to
>>> re-elect a master -- obviously (and IOC_ENABLE).
>> 
>> Re-elect master on IOC_DISABLE is good. But we still need work for ctx
>> rotation. Otherwise, we need keep the master on at all time. 
> 
> I meant to says that for the rotation case we can do as you did here, if
> we do add() on a slave, add the master if it wasn't add()'ed yet.

Maybe an "add-but-don't-count" state would solve this, even with event groups?
Say "PERF_EVENT_STATE_ACTIVE_NOT_COUNTING". Let me think more about it. 

> 
>>>> And if the master is in an event group, it will be more complicated...
>>> 
>>> Hurmph, do you actually have that use-case? And yes, this one is tricky.
>>> 
>>> Would it be sufficient if we disallow group events to be master (but
>>> allow them to be slaves) ?
>> 
>> Maybe we can solve this with an extra "first_active" pointer in perf_event.
>> first_active points to the first event that being added by event_pmu_add(). 
>> Then we need something like:
>> 
>> event_pmu_add(event)
>> {
>> 	if (event->dup_master->first_active) {
>> 		/* sync with first_active */
>> 	} else {
>> 		/* this event will be the first_active */
>> 		event->dup_master->first_active = event;
>> 		pmu->add(event);
>> 	}
>> }
> 
> I'm confused on what exactly you're trying to solve with the
> first_active thing. The problem with the group event as master is that
> you then _must_ schedule the whole group, which is obviously difficult.

With first_active, we are not required to schedule the master. A slave 
could be the first_active, and other slaves could read data from it. 

For group event use cases, I think only allowing non-group event to be
the master would be a good start. 

> 
>>>> If we do GFP_ATOMIC in perf_event_alloc(), maybe with an extra option, we
>>>> don't need the tmp_master hack. So we only allocate master when we will 
>>>> use it. 
>>> 
>>> You can't, that's broken on -RT. ctx->lock is a raw_spinlock_t and
>>> allocator locks are spinlock_t.
>> 
>> How about we add another step in __perf_install_in_context(), like
>> 
>> __perf_install_in_context()
>> {
>> 	bool alloc_master;
>> 
>> 	perf_ctx_lock();
>> 	alloc_master = find_new_sharing(event, ctx);
>> 	perf_ctx_unlock();
>> 	
>> 	if (alloc_master)
>> 		event->dup_master = perf_event_alloc();
>> 	/* existing logic of __perf_install_in_context() */
>> 
>> }
>> 
>> In this way, we only allocate the master event when necessary, and it
>> is outside of the locks. 
> 
> It's still broken on -RT, because __perf_install_in_context() is in
> hardirq context (IPI) and the allocator locks are spinlock_t.

Hmm... how about perf_install_in_context()? Something like:

diff --git i/kernel/events/core.c w/kernel/events/core.c
index e8bec0823763..f55a7a8b9de4 100644
--- i/kernel/events/core.c
+++ w/kernel/events/core.c
@@ -2860,6 +2860,13 @@ perf_install_in_context(struct perf_event_context *ctx,
         */
        smp_store_release(&event->ctx, ctx);

+       raw_spin_lock_irq(&ctx->lock);
+       alloc_master = find_new_sharing(event, ctx);
+       raw_spin_unlock_irq(&ctx->lock);
+
+       if (alloc_master)
+               event->dup_master = perf_event_alloc(xxx);
+

If this works, we won't need PERF_EVENT_STATE_ACTIVE_NOT_COUNTING. 

Thanks,
Song



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

end of thread, other threads:[~2019-11-06 22:23 UTC | newest]

Thread overview: 13+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2019-09-19  5:23 [PATCH v6] perf: Sharing PMU counters across compatible events Song Liu
2019-09-30  5:36 ` Song Liu
2019-10-31 12:43 ` Peter Zijlstra
2019-10-31 16:29   ` Song Liu
2019-11-05 17:11     ` Song Liu
2019-11-05 20:16       ` Peter Zijlstra
2019-11-05 23:06         ` Song Liu
2019-11-06  9:14           ` Peter Zijlstra
2019-11-06 17:40             ` Song Liu
2019-11-06 20:44               ` Peter Zijlstra
2019-11-06 22:23                 ` Song Liu
2019-11-05 23:51 ` Song Liu
2019-11-06  8:56   ` Peter Zijlstra

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).