linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCHSET v4 0/6] perf inject: Speed build-id injection
@ 2020-10-12  7:02 Namhyung Kim
  2020-10-12  7:02 ` [PATCH 1/6] perf bench: Add build-id injection benchmark Namhyung Kim
                   ` (6 more replies)
  0 siblings, 7 replies; 8+ messages in thread
From: Namhyung Kim @ 2020-10-12  7:02 UTC (permalink / raw)
  To: Arnaldo Carvalho de Melo, Jiri Olsa
  Cc: Ingo Molnar, Peter Zijlstra, Mark Rutland, Alexander Shishkin,
	LKML, Stephane Eranian, Ian Rogers

Hello,

This is the new version of speed up build-id injection.  As this is
to improve performance, I've added a benchmark for it.  Please look at
the usage in the first commit.

By default, it measures average processing time of 100 MMAP2 events
and 10000 SAMPLE events.  Below is the current result on my laptop.

  $ perf bench internals inject-build-id
  # Running 'internals/inject-build-id' benchmark:
    Average build-id injection took: 25.789 msec (+- 0.202 msec)
    Average time per event: 2.528 usec (+- 0.020 usec)
    Average memory usage: 8411 KB (+- 7 KB)

With this patchset applied, it got this:

  $ perf bench internals inject-build-id
  # Running 'internals/inject-build-id' benchmark:
    Average build-id injection took: 20.838 msec (+- 0.093 msec)
    Average time per event: 2.043 usec (+- 0.009 usec)
    Average memory usage: 8261 KB (+- 0 KB)
    Average build-id-all injection took: 19.361 msec (+- 0.118 msec)
    Average time per event: 1.898 usec (+- 0.012 usec)
    Average memory usage: 7440 KB (+- 0 KB)


Real usecases might be different as it depends on the number of
mmap/sample events as well as how many DSOs are actually hit.

The benchmark result now includes memory footprint in terms of maximum
RSS.  Also I've update the benchmark code to use timestamp so that it
can be queued to the ordered_events (and flushed at the end).  It's
also important how well it sorts the input events in the queue so I
randomly chose a timestamp at the beginning of each MMAP event
injection to resemble actual behavior.

As I said in other thread, perf inject currently doesn't flush the
input events and processes all at the end.  This gives a good speedup
but spends more memory (in proprotion to the input size).  While the
build-id-all injection bypasses the queue so it uses less memory as
well as faster processing.  The downside is that it'll mark all DSOs
as hit so later processing steps (like perf report) likely handle them
unnecessarily.


This code is available at 'perf/inject-speedup-v4' branch on

  git://git.kernel.org/pub/scm/linux/kernel/git/namhyung/linux-perf.git


Changes from v3:
 - add timestamp to the synthesized events in the benchmark
 - add a separate thread to read pipe in the benchmark

Changes from v2:
 - fix benchmark to read required data
 - add Acked-by from Jiri and Ian
 - pass map flag to check huge pages  (Jiri)
 - add comments on some functions  (Ian)
 - show memory (max-RSS) usage in the benchmark  (Ian)
 - drop build-id marking patch at the last  (Adrian)


Namhyung Kim (6):
  perf bench: Add build-id injection benchmark
  perf inject: Add missing callbacks in perf_tool
  perf inject: Enter namespace when reading build-id
  perf inject: Do not load map/dso when injecting build-id
  perf inject: Add --buildid-all option
  perf bench: Run inject-build-id with --buildid-all option too

 tools/perf/Documentation/perf-inject.txt |   6 +-
 tools/perf/bench/Build                   |   1 +
 tools/perf/bench/bench.h                 |   1 +
 tools/perf/bench/inject-buildid.c        | 457 +++++++++++++++++++++++
 tools/perf/builtin-bench.c               |   1 +
 tools/perf/builtin-inject.c              | 199 ++++++++--
 tools/perf/util/build-id.h               |   4 +
 tools/perf/util/map.c                    |  17 +-
 tools/perf/util/map.h                    |  14 +
 9 files changed, 645 insertions(+), 55 deletions(-)
 create mode 100644 tools/perf/bench/inject-buildid.c

-- 
2.28.0.681.g6f77f65b4e-goog


*** BLURB HERE ***

Namhyung Kim (6):
  perf bench: Add build-id injection benchmark
  perf inject: Add missing callbacks in perf_tool
  perf inject: Enter namespace when reading build-id
  perf inject: Do not load map/dso when injecting build-id
  perf inject: Add --buildid-all option
  perf bench: Run inject-build-id with --buildid-all option too

 tools/perf/Documentation/perf-inject.txt |   6 +-
 tools/perf/bench/Build                   |   1 +
 tools/perf/bench/bench.h                 |   1 +
 tools/perf/bench/inject-buildid.c        | 476 +++++++++++++++++++++++
 tools/perf/builtin-bench.c               |   1 +
 tools/perf/builtin-inject.c              | 199 ++++++++--
 tools/perf/util/build-id.h               |   4 +
 tools/perf/util/map.c                    |  17 +-
 tools/perf/util/map.h                    |  14 +
 9 files changed, 664 insertions(+), 55 deletions(-)
 create mode 100644 tools/perf/bench/inject-buildid.c

-- 
2.28.0.1011.ga647a8990f-goog


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

* [PATCH 1/6] perf bench: Add build-id injection benchmark
  2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
@ 2020-10-12  7:02 ` Namhyung Kim
  2020-10-12  7:02 ` [PATCH 2/6] perf inject: Add missing callbacks in perf_tool Namhyung Kim
                   ` (5 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Namhyung Kim @ 2020-10-12  7:02 UTC (permalink / raw)
  To: Arnaldo Carvalho de Melo, Jiri Olsa
  Cc: Ingo Molnar, Peter Zijlstra, Mark Rutland, Alexander Shishkin,
	LKML, Stephane Eranian, Ian Rogers

Sometimes I can see perf record piped with perf inject take long time
processing build-id.  So add inject-build-id benchmark to the
internals benchmark suite to measure its overhead regularly.

It runs perf inject command internally and feeds the given number of
synthesized events (MMAP2 + SAMPLE basically).

  Usage: perf bench internals inject-build-id <options>

    -i, --iterations <n>  Number of iterations used to compute average (default: 100)
    -m, --nr-mmaps <n>    Number of mmap events for each iteration (default: 100)
    -n, --nr-samples <n>  Number of sample events per mmap event (default: 100)
    -v, --verbose         be more verbose (show iteration count, DSO name, etc)

By default, it measures average processing time of 100 MMAP2 events
and 10000 SAMPLE events.  Below is a result on my laptop.

  $ perf bench internals inject-build-id
  # Running 'internals/inject-build-id' benchmark:
    Average build-id injection took: 25.789 msec (+- 0.202 msec)
    Average time per event: 2.528 usec (+- 0.020 usec)
    Average memory usage: 8411 KB (+- 7 KB)

Acked-by: Jiri Olsa <jolsa@redhat.com>
Signed-off-by: Namhyung Kim <namhyung@kernel.org>
---
 tools/perf/bench/Build            |   1 +
 tools/perf/bench/bench.h          |   1 +
 tools/perf/bench/inject-buildid.c | 460 ++++++++++++++++++++++++++++++
 tools/perf/builtin-bench.c        |   1 +
 tools/perf/builtin-inject.c       |   9 +-
 tools/perf/util/build-id.h        |   4 +
 6 files changed, 471 insertions(+), 5 deletions(-)
 create mode 100644 tools/perf/bench/inject-buildid.c

diff --git a/tools/perf/bench/Build b/tools/perf/bench/Build
index dd68a40a790c..8b52591338d6 100644
--- a/tools/perf/bench/Build
+++ b/tools/perf/bench/Build
@@ -12,6 +12,7 @@ perf-y += epoll-ctl.o
 perf-y += synthesize.o
 perf-y += kallsyms-parse.o
 perf-y += find-bit-bench.o
+perf-y += inject-buildid.o
 
 perf-$(CONFIG_X86_64) += mem-memcpy-x86-64-lib.o
 perf-$(CONFIG_X86_64) += mem-memcpy-x86-64-asm.o
diff --git a/tools/perf/bench/bench.h b/tools/perf/bench/bench.h
index 2804812d4154..eac36afab2b3 100644
--- a/tools/perf/bench/bench.h
+++ b/tools/perf/bench/bench.h
@@ -47,6 +47,7 @@ int bench_epoll_wait(int argc, const char **argv);
 int bench_epoll_ctl(int argc, const char **argv);
 int bench_synthesize(int argc, const char **argv);
 int bench_kallsyms_parse(int argc, const char **argv);
+int bench_inject_build_id(int argc, const char **argv);
 
 #define BENCH_FORMAT_DEFAULT_STR	"default"
 #define BENCH_FORMAT_DEFAULT		0
diff --git a/tools/perf/bench/inject-buildid.c b/tools/perf/bench/inject-buildid.c
new file mode 100644
index 000000000000..0fccf2a9e95b
--- /dev/null
+++ b/tools/perf/bench/inject-buildid.c
@@ -0,0 +1,460 @@
+// SPDX-License-Identifier: GPL-2.0
+#include <stdlib.h>
+#include <stddef.h>
+#include <ftw.h>
+#include <fcntl.h>
+#include <errno.h>
+#include <unistd.h>
+#include <pthread.h>
+#include <sys/mman.h>
+#include <sys/wait.h>
+#include <linux/kernel.h>
+#include <linux/time64.h>
+#include <linux/list.h>
+#include <linux/err.h>
+#include <internal/lib.h>
+#include <subcmd/parse-options.h>
+
+#include "bench.h"
+#include "util/data.h"
+#include "util/stat.h"
+#include "util/debug.h"
+#include "util/event.h"
+#include "util/symbol.h"
+#include "util/session.h"
+#include "util/build-id.h"
+#include "util/synthetic-events.h"
+
+#define MMAP_DEV_MAJOR  8
+#define DSO_MMAP_RATIO  4
+
+static unsigned int iterations = 100;
+static unsigned int nr_mmaps   = 100;
+static unsigned int nr_samples = 100;  /* samples per mmap */
+
+static u64 bench_sample_type;
+static u16 bench_id_hdr_size;
+
+struct bench_data {
+	int			pid;
+	int			input_pipe[2];
+	int			output_pipe[2];
+	pthread_t		th;
+};
+
+struct bench_dso {
+	struct list_head	list;
+	char			*name;
+	int			ino;
+};
+
+static int nr_dsos;
+static struct bench_dso *dsos;
+
+extern int cmd_inject(int argc, const char *argv[]);
+
+static const struct option options[] = {
+	OPT_UINTEGER('i', "iterations", &iterations,
+		     "Number of iterations used to compute average (default: 100)"),
+	OPT_UINTEGER('m', "nr-mmaps", &nr_mmaps,
+		     "Number of mmap events for each iteration (default: 100)"),
+	OPT_UINTEGER('n', "nr-samples", &nr_samples,
+		     "Number of sample events per mmap event (default: 100)"),
+	OPT_INCR('v', "verbose", &verbose,
+		 "be more verbose (show iteration count, DSO name, etc)"),
+	OPT_END()
+};
+
+static const char *const bench_usage[] = {
+	"perf bench internals inject-build-id <options>",
+	NULL
+};
+
+/*
+ * Helper for collect_dso that adds the given file as a dso to dso_list
+ * if it contains a build-id.  Stops after collecting 4 times more than
+ * we need (for MMAP2 events).
+ */
+static int add_dso(const char *fpath, const struct stat *sb __maybe_unused,
+		   int typeflag, struct FTW *ftwbuf __maybe_unused)
+{
+	struct bench_dso *dso = &dsos[nr_dsos];
+	unsigned char build_id[BUILD_ID_SIZE];
+
+	if (typeflag == FTW_D || typeflag == FTW_SL)
+		return 0;
+
+	if (filename__read_build_id(fpath, build_id, BUILD_ID_SIZE) < 0)
+		return 0;
+
+	dso->name = realpath(fpath, NULL);
+	if (dso->name == NULL)
+		return -1;
+
+	dso->ino = nr_dsos++;
+	pr_debug2("  Adding DSO: %s\n", fpath);
+
+	/* stop if we collected enough DSOs */
+	if ((unsigned int)nr_dsos == DSO_MMAP_RATIO * nr_mmaps)
+		return 1;
+
+	return 0;
+}
+
+static void collect_dso(void)
+{
+	dsos = calloc(nr_mmaps * DSO_MMAP_RATIO, sizeof(*dsos));
+	if (dsos == NULL) {
+		printf("  Memory allocation failed\n");
+		exit(1);
+	}
+
+	if (nftw("/usr/lib/", add_dso, 10, FTW_PHYS) < 0)
+		return;
+
+	pr_debug("  Collected %d DSOs\n", nr_dsos);
+}
+
+static void release_dso(void)
+{
+	int i;
+
+	for (i = 0; i < nr_dsos; i++) {
+		struct bench_dso *dso = &dsos[i];
+
+		free(dso->name);
+	}
+	free(dsos);
+}
+
+/* Fake address used by mmap and sample events */
+static u64 dso_map_addr(struct bench_dso *dso)
+{
+	return 0x400000ULL + dso->ino * 8192ULL;
+}
+
+static u32 synthesize_attr(struct bench_data *data)
+{
+	union perf_event event;
+
+	memset(&event, 0, sizeof(event.attr) + sizeof(u64));
+
+	event.header.type = PERF_RECORD_HEADER_ATTR;
+	event.header.size = sizeof(event.attr) + sizeof(u64);
+
+	event.attr.attr.type = PERF_TYPE_SOFTWARE;
+	event.attr.attr.config = PERF_COUNT_SW_TASK_CLOCK;
+	event.attr.attr.exclude_kernel = 1;
+	event.attr.attr.sample_id_all = 1;
+	event.attr.attr.sample_type = bench_sample_type;
+
+	return writen(data->input_pipe[1], &event, event.header.size);
+}
+
+static u32 synthesize_fork(struct bench_data *data)
+{
+	union perf_event event;
+
+	memset(&event, 0, sizeof(event.fork) + bench_id_hdr_size);
+
+	event.header.type = PERF_RECORD_FORK;
+	event.header.misc = PERF_RECORD_MISC_FORK_EXEC;
+	event.header.size = sizeof(event.fork) + bench_id_hdr_size;
+
+	event.fork.ppid = 1;
+	event.fork.ptid = 1;
+	event.fork.pid = data->pid;
+	event.fork.tid = data->pid;
+
+	return writen(data->input_pipe[1], &event, event.header.size);
+}
+
+static u32 synthesize_mmap(struct bench_data *data, struct bench_dso *dso,
+			   u64 timestamp)
+{
+	union perf_event event;
+	size_t len = offsetof(struct perf_record_mmap2, filename);
+	u64 *id_hdr_ptr = (void *)&event;
+	int ts_idx;
+
+	len += roundup(strlen(dso->name) + 1, 8) + bench_id_hdr_size;
+
+	memset(&event, 0, min(len, sizeof(event.mmap2)));
+
+	event.header.type = PERF_RECORD_MMAP2;
+	event.header.misc = PERF_RECORD_MISC_USER;
+	event.header.size = len;
+
+	event.mmap2.pid = data->pid;
+	event.mmap2.tid = data->pid;
+	event.mmap2.maj = MMAP_DEV_MAJOR;
+	event.mmap2.ino = dso->ino;
+
+	strcpy(event.mmap2.filename, dso->name);
+
+	event.mmap2.start = dso_map_addr(dso);
+	event.mmap2.len = 4096;
+	event.mmap2.prot = PROT_EXEC;
+
+	if (len > sizeof(event.mmap2)) {
+		/* write mmap2 event first */
+		writen(data->input_pipe[1], &event, len - bench_id_hdr_size);
+		/* zero-fill sample id header */
+		memset(id_hdr_ptr, 0, bench_id_hdr_size);
+		/* put timestamp in the right position */
+		ts_idx = (bench_id_hdr_size / sizeof(u64)) - 2;
+		id_hdr_ptr[ts_idx] = timestamp;
+		writen(data->input_pipe[1], id_hdr_ptr, bench_id_hdr_size);
+	} else {
+		ts_idx = (len / sizeof(u64)) - 2;
+		id_hdr_ptr[ts_idx] = timestamp;
+		writen(data->input_pipe[1], &event, len);
+	}
+	return len;
+}
+
+static u32 synthesize_sample(struct bench_data *data, struct bench_dso *dso,
+			     u64 timestamp)
+{
+	union perf_event event;
+	struct perf_sample sample = {
+		.tid = data->pid,
+		.pid = data->pid,
+		.ip = dso_map_addr(dso),
+		.time = timestamp,
+	};
+
+	event.header.type = PERF_RECORD_SAMPLE;
+	event.header.misc = PERF_RECORD_MISC_USER;
+	event.header.size = perf_event__sample_event_size(&sample, bench_sample_type, 0);
+
+	perf_event__synthesize_sample(&event, bench_sample_type, 0, &sample);
+
+	return writen(data->input_pipe[1], &event, event.header.size);
+}
+
+static u32 synthesize_flush(struct bench_data *data)
+{
+	struct perf_event_header header = {
+		.size = sizeof(header),
+		.type = PERF_RECORD_FINISHED_ROUND,
+	};
+
+	return writen(data->input_pipe[1], &header, header.size);
+}
+
+static void *data_reader(void *arg)
+{
+	struct bench_data *data = arg;
+	char buf[8192];
+	int flag;
+	int n;
+
+	flag = fcntl(data->output_pipe[0], F_GETFL);
+	fcntl(data->output_pipe[0], F_SETFL, flag | O_NONBLOCK);
+
+	/* read out data from child */
+	while (true) {
+		n = read(data->output_pipe[0], buf, sizeof(buf));
+		if (n > 0)
+			continue;
+		if (n == 0)
+			break;
+
+		if (errno != EINTR && errno != EAGAIN)
+			break;
+
+		usleep(100);
+	}
+
+	close(data->output_pipe[0]);
+	return NULL;
+}
+
+static int setup_injection(struct bench_data *data)
+{
+	int ready_pipe[2];
+	int dev_null_fd;
+	char buf;
+
+	if (pipe(ready_pipe) < 0)
+		return -1;
+
+	if (pipe(data->input_pipe) < 0)
+		return -1;
+
+	if (pipe(data->output_pipe) < 0)
+		return -1;
+
+	data->pid = fork();
+	if (data->pid < 0)
+		return -1;
+
+	if (data->pid == 0) {
+		const char **inject_argv;
+
+		close(data->input_pipe[1]);
+		close(data->output_pipe[0]);
+		close(ready_pipe[0]);
+
+		dup2(data->input_pipe[0], STDIN_FILENO);
+		close(data->input_pipe[0]);
+		dup2(data->output_pipe[1], STDOUT_FILENO);
+		close(data->output_pipe[1]);
+
+		dev_null_fd = open("/dev/null", O_WRONLY);
+		if (dev_null_fd < 0)
+			exit(1);
+
+		dup2(dev_null_fd, STDERR_FILENO);
+
+		inject_argv = calloc(3, sizeof(*inject_argv));
+		if (inject_argv == NULL)
+			exit(1);
+
+		inject_argv[0] = strdup("inject");
+		inject_argv[1] = strdup("-b");
+
+		/* signal that we're ready to go */
+		close(ready_pipe[1]);
+
+		cmd_inject(2, inject_argv);
+
+		exit(0);
+	}
+
+	pthread_create(&data->th, NULL, data_reader, data);
+
+	close(ready_pipe[1]);
+	close(data->input_pipe[0]);
+	close(data->output_pipe[1]);
+
+	/* wait for child ready */
+	if (read(ready_pipe[0], &buf, 1) < 0)
+		return -1;
+	close(ready_pipe[0]);
+
+	return 0;
+}
+
+static int inject_build_id(struct bench_data *data, u64 *max_rss)
+{
+	int status;
+	unsigned int i, k;
+	struct rusage rusage;
+	u64 len = 0;
+
+	/* this makes the child to run */
+	if (perf_header__write_pipe(data->input_pipe[1]) < 0)
+		return -1;
+
+	len += synthesize_attr(data);
+	len += synthesize_fork(data);
+
+	for (i = 0; i < nr_mmaps; i++) {
+		int idx = rand() % (nr_dsos - 1);
+		struct bench_dso *dso = &dsos[idx];
+		u64 timestamp = rand() % 1000000;
+
+		pr_debug2("   [%d] injecting: %s\n", i+1, dso->name);
+		len += synthesize_mmap(data, dso, timestamp);
+
+		for (k = 0; k < nr_samples; k++)
+			len += synthesize_sample(data, dso, timestamp + k * 1000);
+
+		if ((i + 1) % 10 == 0)
+			len += synthesize_flush(data);
+	}
+
+	/* tihs makes the child to finish */
+	close(data->input_pipe[1]);
+
+	wait4(data->pid, &status, 0, &rusage);
+	*max_rss = rusage.ru_maxrss;
+
+	pr_debug("   Child %d exited with %d\n", data->pid, status);
+
+	return 0;
+}
+
+static int do_inject_loop(struct bench_data *data)
+{
+	unsigned int i;
+	struct stats time_stats, mem_stats;
+	double time_average, time_stddev;
+	double mem_average, mem_stddev;
+
+	srand(time(NULL));
+	init_stats(&time_stats);
+	init_stats(&mem_stats);
+	symbol__init(NULL);
+
+	bench_sample_type  = PERF_SAMPLE_IDENTIFIER | PERF_SAMPLE_IP;
+	bench_sample_type |= PERF_SAMPLE_TID | PERF_SAMPLE_TIME;
+	bench_id_hdr_size  = 32;
+
+	collect_dso();
+	if (nr_dsos == 0) {
+		printf("  Cannot collect DSOs for injection\n");
+		return -1;
+	}
+
+	for (i = 0; i < iterations; i++) {
+		struct timeval start, end, diff;
+		u64 runtime_us, max_rss;
+
+		pr_debug("  Iteration #%d\n", i+1);
+
+		if (setup_injection(data) < 0) {
+			printf("  Build-id injection setup failed\n");
+			break;
+		}
+
+		gettimeofday(&start, NULL);
+		if (inject_build_id(data, &max_rss) < 0) {
+			printf("  Build-id injection failed\n");
+			break;
+		}
+
+		gettimeofday(&end, NULL);
+		timersub(&end, &start, &diff);
+		runtime_us = diff.tv_sec * USEC_PER_SEC + diff.tv_usec;
+		update_stats(&time_stats, runtime_us);
+		update_stats(&mem_stats, max_rss);
+
+		pthread_join(data->th, NULL);
+	}
+
+	time_average = avg_stats(&time_stats) / USEC_PER_MSEC;
+	time_stddev = stddev_stats(&time_stats) / USEC_PER_MSEC;
+	printf("  Average build-id injection took: %.3f msec (+- %.3f msec)\n",
+		time_average, time_stddev);
+
+	/* each iteration, it processes MMAP2 + BUILD_ID + nr_samples * SAMPLE */
+	time_average = avg_stats(&time_stats) / (nr_mmaps * (nr_samples + 2));
+	time_stddev = stddev_stats(&time_stats) / (nr_mmaps * (nr_samples + 2));
+	printf("  Average time per event: %.3f usec (+- %.3f usec)\n",
+		time_average, time_stddev);
+
+	mem_average = avg_stats(&mem_stats);
+	mem_stddev = stddev_stats(&mem_stats);
+	printf("  Average memory usage: %.0f KB (+- %.0f KB)\n",
+		mem_average, mem_stddev);
+
+	release_dso();
+	return 0;
+}
+
+int bench_inject_build_id(int argc, const char **argv)
+{
+	struct bench_data data;
+
+	argc = parse_options(argc, argv, options, bench_usage, 0);
+	if (argc) {
+		usage_with_options(bench_usage, options);
+		exit(EXIT_FAILURE);
+	}
+
+	return do_inject_loop(&data);
+}
+
diff --git a/tools/perf/builtin-bench.c b/tools/perf/builtin-bench.c
index 4f176039fc8f..62a7b7420a44 100644
--- a/tools/perf/builtin-bench.c
+++ b/tools/perf/builtin-bench.c
@@ -87,6 +87,7 @@ static struct bench epoll_benchmarks[] = {
 static struct bench internals_benchmarks[] = {
 	{ "synthesize", "Benchmark perf event synthesis",	bench_synthesize	},
 	{ "kallsyms-parse", "Benchmark kallsyms parsing",	bench_kallsyms_parse	},
+	{ "inject-build-id", "Benchmark build-id injection",	bench_inject_build_id	},
 	{ NULL,		NULL,					NULL			}
 };
 
diff --git a/tools/perf/builtin-inject.c b/tools/perf/builtin-inject.c
index 6d2f410d773a..e4d78f11494e 100644
--- a/tools/perf/builtin-inject.c
+++ b/tools/perf/builtin-inject.c
@@ -441,11 +441,10 @@ static int dso__inject_build_id(struct dso *dso, struct perf_tool *tool,
 	return 0;
 }
 
-static int perf_event__inject_buildid(struct perf_tool *tool,
-				      union perf_event *event,
-				      struct perf_sample *sample,
-				      struct evsel *evsel __maybe_unused,
-				      struct machine *machine)
+int perf_event__inject_buildid(struct perf_tool *tool, union perf_event *event,
+			       struct perf_sample *sample,
+			       struct evsel *evsel __maybe_unused,
+			       struct machine *machine)
 {
 	struct addr_location al;
 	struct thread *thread;
diff --git a/tools/perf/util/build-id.h b/tools/perf/util/build-id.h
index aad419bb165c..949f7e54c9cb 100644
--- a/tools/perf/util/build-id.h
+++ b/tools/perf/util/build-id.h
@@ -29,6 +29,10 @@ int build_id__mark_dso_hit(struct perf_tool *tool, union perf_event *event,
 
 int dsos__hit_all(struct perf_session *session);
 
+int perf_event__inject_buildid(struct perf_tool *tool, union perf_event *event,
+			       struct perf_sample *sample, struct evsel *evsel,
+			       struct machine *machine);
+
 bool perf_session__read_build_ids(struct perf_session *session, bool with_hits);
 int perf_session__write_buildid_table(struct perf_session *session,
 				      struct feat_fd *fd);
-- 
2.28.0.1011.ga647a8990f-goog


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

* [PATCH 2/6] perf inject: Add missing callbacks in perf_tool
  2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
  2020-10-12  7:02 ` [PATCH 1/6] perf bench: Add build-id injection benchmark Namhyung Kim
@ 2020-10-12  7:02 ` Namhyung Kim
  2020-10-12  7:02 ` [PATCH 3/6] perf inject: Enter namespace when reading build-id Namhyung Kim
                   ` (4 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Namhyung Kim @ 2020-10-12  7:02 UTC (permalink / raw)
  To: Arnaldo Carvalho de Melo, Jiri Olsa
  Cc: Ingo Molnar, Peter Zijlstra, Mark Rutland, Alexander Shishkin,
	LKML, Stephane Eranian, Ian Rogers

I found some events (like PERF_RECORD_CGROUP) are not copied by perf
inject due to the missing callbacks.  Let's add them.

While at it, I've changed the order of the callbacks to match with
struct perf_tool so that we can compare them easily.

Acked-by: Jiri Olsa <jolsa@redhat.com>
Signed-off-by: Namhyung Kim <namhyung@kernel.org>
---
 tools/perf/builtin-inject.c | 36 +++++++++++++++++++++++++++++++-----
 1 file changed, 31 insertions(+), 5 deletions(-)

diff --git a/tools/perf/builtin-inject.c b/tools/perf/builtin-inject.c
index e4d78f11494e..2c5e23d73a8a 100644
--- a/tools/perf/builtin-inject.c
+++ b/tools/perf/builtin-inject.c
@@ -97,6 +97,13 @@ static int perf_event__repipe_op2_synth(struct perf_session *session,
 	return perf_event__repipe_synth(session->tool, event);
 }
 
+static int perf_event__repipe_op4_synth(struct perf_session *session,
+					union perf_event *event,
+					u64 data __maybe_unused)
+{
+	return perf_event__repipe_synth(session->tool, event);
+}
+
 static int perf_event__repipe_attr(struct perf_tool *tool,
 				   union perf_event *event,
 				   struct evlist **pevlist)
@@ -115,6 +122,13 @@ static int perf_event__repipe_attr(struct perf_tool *tool,
 	return perf_event__repipe_synth(tool, event);
 }
 
+static int perf_event__repipe_event_update(struct perf_tool *tool,
+					   union perf_event *event,
+					   struct evlist **pevlist __maybe_unused)
+{
+	return perf_event__repipe_synth(tool, event);
+}
+
 #ifdef HAVE_AUXTRACE_SUPPORT
 
 static int copy_bytes(struct perf_inject *inject, int fd, off_t size)
@@ -707,9 +721,12 @@ int cmd_inject(int argc, const char **argv)
 	struct perf_inject inject = {
 		.tool = {
 			.sample		= perf_event__repipe_sample,
+			.read		= perf_event__repipe_sample,
 			.mmap		= perf_event__repipe,
 			.mmap2		= perf_event__repipe,
 			.comm		= perf_event__repipe,
+			.namespaces	= perf_event__repipe,
+			.cgroup		= perf_event__repipe,
 			.fork		= perf_event__repipe,
 			.exit		= perf_event__repipe,
 			.lost		= perf_event__repipe,
@@ -717,19 +734,28 @@ int cmd_inject(int argc, const char **argv)
 			.aux		= perf_event__repipe,
 			.itrace_start	= perf_event__repipe,
 			.context_switch	= perf_event__repipe,
-			.read		= perf_event__repipe_sample,
 			.throttle	= perf_event__repipe,
 			.unthrottle	= perf_event__repipe,
+			.ksymbol	= perf_event__repipe,
+			.bpf		= perf_event__repipe,
+			.text_poke	= perf_event__repipe,
 			.attr		= perf_event__repipe_attr,
+			.event_update	= perf_event__repipe_event_update,
 			.tracing_data	= perf_event__repipe_op2_synth,
-			.auxtrace_info	= perf_event__repipe_op2_synth,
-			.auxtrace	= perf_event__repipe_auxtrace,
-			.auxtrace_error	= perf_event__repipe_op2_synth,
-			.time_conv	= perf_event__repipe_op2_synth,
 			.finished_round	= perf_event__repipe_oe_synth,
 			.build_id	= perf_event__repipe_op2_synth,
 			.id_index	= perf_event__repipe_op2_synth,
+			.auxtrace_info	= perf_event__repipe_op2_synth,
+			.auxtrace_error	= perf_event__repipe_op2_synth,
+			.time_conv	= perf_event__repipe_op2_synth,
+			.thread_map	= perf_event__repipe_op2_synth,
+			.cpu_map	= perf_event__repipe_op2_synth,
+			.stat_config	= perf_event__repipe_op2_synth,
+			.stat		= perf_event__repipe_op2_synth,
+			.stat_round	= perf_event__repipe_op2_synth,
 			.feature	= perf_event__repipe_op2_synth,
+			.compressed	= perf_event__repipe_op4_synth,
+			.auxtrace	= perf_event__repipe_auxtrace,
 		},
 		.input_name  = "-",
 		.samples = LIST_HEAD_INIT(inject.samples),
-- 
2.28.0.1011.ga647a8990f-goog


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

* [PATCH 3/6] perf inject: Enter namespace when reading build-id
  2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
  2020-10-12  7:02 ` [PATCH 1/6] perf bench: Add build-id injection benchmark Namhyung Kim
  2020-10-12  7:02 ` [PATCH 2/6] perf inject: Add missing callbacks in perf_tool Namhyung Kim
@ 2020-10-12  7:02 ` Namhyung Kim
  2020-10-12  7:02 ` [PATCH 4/6] perf inject: Do not load map/dso when injecting build-id Namhyung Kim
                   ` (3 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Namhyung Kim @ 2020-10-12  7:02 UTC (permalink / raw)
  To: Arnaldo Carvalho de Melo, Jiri Olsa
  Cc: Ingo Molnar, Peter Zijlstra, Mark Rutland, Alexander Shishkin,
	LKML, Stephane Eranian, Ian Rogers

It should be in a proper mnt namespace when accessing the file.

I think this had no problem since the build-id was actually read from
map__load() -> dso__load() already.  But I'd like to change it in the
following commit.

Acked-by: Jiri Olsa <jolsa@redhat.com>
Signed-off-by: Namhyung Kim <namhyung@kernel.org>
---
 tools/perf/builtin-inject.c | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)

diff --git a/tools/perf/builtin-inject.c b/tools/perf/builtin-inject.c
index 2c5e23d73a8a..670157db2502 100644
--- a/tools/perf/builtin-inject.c
+++ b/tools/perf/builtin-inject.c
@@ -23,6 +23,7 @@
 #include "util/symbol.h"
 #include "util/synthetic-events.h"
 #include "util/thread.h"
+#include "util/namespaces.h"
 #include <linux/err.h>
 
 #include <subcmd/parse-options.h>
@@ -419,16 +420,19 @@ static int perf_event__repipe_tracing_data(struct perf_session *session,
 
 static int dso__read_build_id(struct dso *dso)
 {
+	struct nscookie nsc;
+
 	if (dso->has_build_id)
 		return 0;
 
+	nsinfo__mountns_enter(dso->nsinfo, &nsc);
 	if (filename__read_build_id(dso->long_name, dso->build_id,
 				    sizeof(dso->build_id)) > 0) {
 		dso->has_build_id = true;
-		return 0;
 	}
+	nsinfo__mountns_exit(&nsc);
 
-	return -1;
+	return dso->has_build_id ? 0 : -1;
 }
 
 static int dso__inject_build_id(struct dso *dso, struct perf_tool *tool,
-- 
2.28.0.1011.ga647a8990f-goog


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

* [PATCH 4/6] perf inject: Do not load map/dso when injecting build-id
  2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
                   ` (2 preceding siblings ...)
  2020-10-12  7:02 ` [PATCH 3/6] perf inject: Enter namespace when reading build-id Namhyung Kim
@ 2020-10-12  7:02 ` Namhyung Kim
  2020-10-12  7:02 ` [PATCH 5/6] perf inject: Add --buildid-all option Namhyung Kim
                   ` (2 subsequent siblings)
  6 siblings, 0 replies; 8+ messages in thread
From: Namhyung Kim @ 2020-10-12  7:02 UTC (permalink / raw)
  To: Arnaldo Carvalho de Melo, Jiri Olsa
  Cc: Ingo Molnar, Peter Zijlstra, Mark Rutland, Alexander Shishkin,
	LKML, Stephane Eranian, Ian Rogers

No need to load symbols in a DSO when injecting build-id.  I guess the
reason was to check the DSO is a special file like anon files.  Use
some helper functions in map.c to check them before reading build-id.
Also pass sample event's cpumode to a new build-id event.

It brought a speedup in the benchmark of 25 -> 21 msec on my laptop.
Also the memory usage (Max RSS) went down by ~200 KB.

  # Running 'internals/inject-build-id' benchmark:
    Average build-id injection took: 21.389 msec (+- 0.138 msec)
    Average time per event: 2.097 usec (+- 0.014 usec)
    Average memory usage: 8225 KB (+- 0 KB)

Original-patch-by: Stephane Eranian <eranian@google.com>
Acked-by: Jiri Olsa <jolsa@redhat.com>
Signed-off-by: Namhyung Kim <namhyung@kernel.org>
---
 tools/perf/builtin-inject.c | 33 ++++++++++++---------------------
 tools/perf/util/map.c       | 17 +----------------
 tools/perf/util/map.h       | 14 ++++++++++++++
 3 files changed, 27 insertions(+), 37 deletions(-)

diff --git a/tools/perf/builtin-inject.c b/tools/perf/builtin-inject.c
index 670157db2502..248cd9f3e5bb 100644
--- a/tools/perf/builtin-inject.c
+++ b/tools/perf/builtin-inject.c
@@ -24,9 +24,10 @@
 #include "util/synthetic-events.h"
 #include "util/thread.h"
 #include "util/namespaces.h"
-#include <linux/err.h>
 
+#include <linux/err.h>
 #include <subcmd/parse-options.h>
+#include <uapi/linux/mman.h> /* To get things like MAP_HUGETLB even on older libc headers */
 
 #include <linux/list.h>
 #include <errno.h>
@@ -436,21 +437,22 @@ static int dso__read_build_id(struct dso *dso)
 }
 
 static int dso__inject_build_id(struct dso *dso, struct perf_tool *tool,
-				struct machine *machine)
+				struct machine *machine, u8 cpumode, u32 flags)
 {
-	u16 misc = PERF_RECORD_MISC_USER;
 	int err;
 
+	if (is_anon_memory(dso->long_name) || flags & MAP_HUGETLB)
+		return 0;
+	if (is_no_dso_memory(dso->long_name))
+		return 0;
+
 	if (dso__read_build_id(dso) < 0) {
 		pr_debug("no build_id found for %s\n", dso->long_name);
 		return -1;
 	}
 
-	if (dso->kernel)
-		misc = PERF_RECORD_MISC_KERNEL;
-
-	err = perf_event__synthesize_build_id(tool, dso, misc, perf_event__repipe,
-					      machine);
+	err = perf_event__synthesize_build_id(tool, dso, cpumode,
+					      perf_event__repipe, machine);
 	if (err) {
 		pr_err("Can't synthesize build_id event for %s\n", dso->long_name);
 		return -1;
@@ -477,19 +479,8 @@ int perf_event__inject_buildid(struct perf_tool *tool, union perf_event *event,
 	if (thread__find_map(thread, sample->cpumode, sample->ip, &al)) {
 		if (!al.map->dso->hit) {
 			al.map->dso->hit = 1;
-			if (map__load(al.map) >= 0) {
-				dso__inject_build_id(al.map->dso, tool, machine);
-				/*
-				 * If this fails, too bad, let the other side
-				 * account this as unresolved.
-				 */
-			} else {
-#ifdef HAVE_LIBELF_SUPPORT
-				pr_warning("no symbols found in %s, maybe "
-					   "install a debug package?\n",
-					   al.map->dso->long_name);
-#endif
-			}
+			dso__inject_build_id(al.map->dso, tool, machine,
+					     sample->cpumode, al.map->flags);
 		}
 	}
 
diff --git a/tools/perf/util/map.c b/tools/perf/util/map.c
index cc0faf8f1321..8b305e624124 100644
--- a/tools/perf/util/map.c
+++ b/tools/perf/util/map.c
@@ -27,21 +27,6 @@
 
 static void __maps__insert(struct maps *maps, struct map *map);
 
-static inline int is_anon_memory(const char *filename, u32 flags)
-{
-	return flags & MAP_HUGETLB ||
-	       !strcmp(filename, "//anon") ||
-	       !strncmp(filename, "/dev/zero", sizeof("/dev/zero") - 1) ||
-	       !strncmp(filename, "/anon_hugepage", sizeof("/anon_hugepage") - 1);
-}
-
-static inline int is_no_dso_memory(const char *filename)
-{
-	return !strncmp(filename, "[stack", 6) ||
-	       !strncmp(filename, "/SYSV",5)   ||
-	       !strcmp(filename, "[heap]");
-}
-
 static inline int is_android_lib(const char *filename)
 {
 	return strstarts(filename, "/data/app-lib/") ||
@@ -158,7 +143,7 @@ struct map *map__new(struct machine *machine, u64 start, u64 len,
 		int anon, no_dso, vdso, android;
 
 		android = is_android_lib(filename);
-		anon = is_anon_memory(filename, flags);
+		anon = is_anon_memory(filename) || flags & MAP_HUGETLB;
 		vdso = is_vdso_map(filename);
 		no_dso = is_no_dso_memory(filename);
 		map->prot = prot;
diff --git a/tools/perf/util/map.h b/tools/perf/util/map.h
index c2f5d28fe73a..b1c0686db1b7 100644
--- a/tools/perf/util/map.h
+++ b/tools/perf/util/map.h
@@ -171,4 +171,18 @@ static inline bool is_bpf_image(const char *name)
 	return strncmp(name, "bpf_trampoline_", sizeof("bpf_trampoline_") - 1) == 0 ||
 	       strncmp(name, "bpf_dispatcher_", sizeof("bpf_dispatcher_") - 1) == 0;
 }
+
+static inline int is_anon_memory(const char *filename)
+{
+	return !strcmp(filename, "//anon") ||
+	       !strncmp(filename, "/dev/zero", sizeof("/dev/zero") - 1) ||
+	       !strncmp(filename, "/anon_hugepage", sizeof("/anon_hugepage") - 1);
+}
+
+static inline int is_no_dso_memory(const char *filename)
+{
+	return !strncmp(filename, "[stack", 6) ||
+	       !strncmp(filename, "/SYSV", 5)  ||
+	       !strcmp(filename, "[heap]");
+}
 #endif /* __PERF_MAP_H */
-- 
2.28.0.1011.ga647a8990f-goog


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

* [PATCH 5/6] perf inject: Add --buildid-all option
  2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
                   ` (3 preceding siblings ...)
  2020-10-12  7:02 ` [PATCH 4/6] perf inject: Do not load map/dso when injecting build-id Namhyung Kim
@ 2020-10-12  7:02 ` Namhyung Kim
  2020-10-12  7:02 ` [PATCH 6/6] perf bench: Run inject-build-id with --buildid-all option too Namhyung Kim
  2020-10-13 12:11 ` [PATCHSET v4 0/6] perf inject: Speed build-id injection Arnaldo Carvalho de Melo
  6 siblings, 0 replies; 8+ messages in thread
From: Namhyung Kim @ 2020-10-12  7:02 UTC (permalink / raw)
  To: Arnaldo Carvalho de Melo, Jiri Olsa
  Cc: Ingo Molnar, Peter Zijlstra, Mark Rutland, Alexander Shishkin,
	LKML, Stephane Eranian, Ian Rogers

Like perf record, we can even more speedup build-id processing by just
using all DSOs.  Then we don't need to look at all the sample events
anymore.  The following patch will update perf bench to show the
result of the --buildid-all option too.

Original-patch-by: Stephane Eranian <eranian@google.com>
Acked-by: Jiri Olsa <jolsa@redhat.com>
Acked-by: Ian Rogers <irogers@google.com>
Signed-off-by: Namhyung Kim <namhyung@kernel.org>
---
 tools/perf/Documentation/perf-inject.txt |   6 +-
 tools/perf/builtin-inject.c              | 113 ++++++++++++++++++++++-
 2 files changed, 113 insertions(+), 6 deletions(-)

diff --git a/tools/perf/Documentation/perf-inject.txt b/tools/perf/Documentation/perf-inject.txt
index 70969ea73e01..a8eccff21281 100644
--- a/tools/perf/Documentation/perf-inject.txt
+++ b/tools/perf/Documentation/perf-inject.txt
@@ -24,8 +24,12 @@ information could make use of this facility.
 OPTIONS
 -------
 -b::
---build-ids=::
+--build-ids::
         Inject build-ids into the output stream
+
+--buildid-all:
+	Inject build-ids of all DSOs into the output stream
+
 -v::
 --verbose::
 	Be more verbose.
diff --git a/tools/perf/builtin-inject.c b/tools/perf/builtin-inject.c
index 248cd9f3e5bb..f3f965157d69 100644
--- a/tools/perf/builtin-inject.c
+++ b/tools/perf/builtin-inject.c
@@ -10,6 +10,7 @@
 
 #include "util/color.h"
 #include "util/dso.h"
+#include "util/vdso.h"
 #include "util/evlist.h"
 #include "util/evsel.h"
 #include "util/map.h"
@@ -37,6 +38,7 @@ struct perf_inject {
 	struct perf_tool	tool;
 	struct perf_session	*session;
 	bool			build_ids;
+	bool			build_id_all;
 	bool			sched_stat;
 	bool			have_auxtrace;
 	bool			strip;
@@ -56,6 +58,9 @@ struct event_entry {
 	union perf_event event[];
 };
 
+static int dso__inject_build_id(struct dso *dso, struct perf_tool *tool,
+				struct machine *machine, u8 cpumode, u32 flags);
+
 static int output_bytes(struct perf_inject *inject, void *buf, size_t sz)
 {
 	ssize_t size;
@@ -319,6 +324,68 @@ static int perf_event__jit_repipe_mmap(struct perf_tool *tool,
 }
 #endif
 
+static struct dso *findnew_dso(int pid, int tid, const char *filename,
+			       struct dso_id *id, struct machine *machine)
+{
+	struct thread *thread;
+	struct nsinfo *nsi = NULL;
+	struct nsinfo *nnsi;
+	struct dso *dso;
+	bool vdso;
+
+	thread = machine__findnew_thread(machine, pid, tid);
+	if (thread == NULL) {
+		pr_err("cannot find or create a task %d/%d.\n", tid, pid);
+		return NULL;
+	}
+
+	vdso = is_vdso_map(filename);
+	nsi = nsinfo__get(thread->nsinfo);
+
+	if (vdso) {
+		/* The vdso maps are always on the host and not the
+		 * container.  Ensure that we don't use setns to look
+		 * them up.
+		 */
+		nnsi = nsinfo__copy(nsi);
+		if (nnsi) {
+			nsinfo__put(nsi);
+			nnsi->need_setns = false;
+			nsi = nnsi;
+		}
+		dso = machine__findnew_vdso(machine, thread);
+	} else {
+		dso = machine__findnew_dso_id(machine, filename, id);
+	}
+
+	if (dso)
+		dso->nsinfo = nsi;
+	else
+		nsinfo__put(nsi);
+
+	thread__put(thread);
+	return dso;
+}
+
+static int perf_event__repipe_buildid_mmap(struct perf_tool *tool,
+					   union perf_event *event,
+					   struct perf_sample *sample,
+					   struct machine *machine)
+{
+	struct dso *dso;
+
+	dso = findnew_dso(event->mmap.pid, event->mmap.tid,
+			  event->mmap.filename, NULL, machine);
+
+	if (dso && !dso->hit) {
+		dso->hit = 1;
+		dso__inject_build_id(dso, tool, machine, sample->cpumode, 0);
+		dso__put(dso);
+	}
+
+	return perf_event__repipe(tool, event, sample, machine);
+}
+
 static int perf_event__repipe_mmap2(struct perf_tool *tool,
 				   union perf_event *event,
 				   struct perf_sample *sample,
@@ -357,6 +424,34 @@ static int perf_event__jit_repipe_mmap2(struct perf_tool *tool,
 }
 #endif
 
+static int perf_event__repipe_buildid_mmap2(struct perf_tool *tool,
+					    union perf_event *event,
+					    struct perf_sample *sample,
+					    struct machine *machine)
+{
+	struct dso_id dso_id = {
+		.maj = event->mmap2.maj,
+		.min = event->mmap2.min,
+		.ino = event->mmap2.ino,
+		.ino_generation = event->mmap2.ino_generation,
+	};
+	struct dso *dso;
+
+	dso = findnew_dso(event->mmap2.pid, event->mmap2.tid,
+			  event->mmap2.filename, &dso_id, machine);
+
+	if (dso && !dso->hit) {
+		dso->hit = 1;
+		dso__inject_build_id(dso, tool, machine, sample->cpumode,
+				     event->mmap2.flags);
+		dso__put(dso);
+	}
+
+	perf_event__repipe(tool, event, sample, machine);
+
+	return 0;
+}
+
 static int perf_event__repipe_fork(struct perf_tool *tool,
 				   union perf_event *event,
 				   struct perf_sample *sample,
@@ -614,7 +709,7 @@ static int __cmd_inject(struct perf_inject *inject)
 	signal(SIGINT, sig_handler);
 
 	if (inject->build_ids || inject->sched_stat ||
-	    inject->itrace_synth_opts.set) {
+	    inject->itrace_synth_opts.set || inject->build_id_all) {
 		inject->tool.mmap	  = perf_event__repipe_mmap;
 		inject->tool.mmap2	  = perf_event__repipe_mmap2;
 		inject->tool.fork	  = perf_event__repipe_fork;
@@ -623,7 +718,10 @@ static int __cmd_inject(struct perf_inject *inject)
 
 	output_data_offset = session->header.data_offset;
 
-	if (inject->build_ids) {
+	if (inject->build_id_all) {
+		inject->tool.mmap	  = perf_event__repipe_buildid_mmap;
+		inject->tool.mmap2	  = perf_event__repipe_buildid_mmap2;
+	} else if (inject->build_ids) {
 		inject->tool.sample = perf_event__inject_buildid;
 	} else if (inject->sched_stat) {
 		struct evsel *evsel;
@@ -767,6 +865,8 @@ int cmd_inject(int argc, const char **argv)
 	struct option options[] = {
 		OPT_BOOLEAN('b', "build-ids", &inject.build_ids,
 			    "Inject build-ids into the output stream"),
+		OPT_BOOLEAN(0, "buildid-all", &inject.build_id_all,
+			    "Inject build-ids of all DSOs into the output stream"),
 		OPT_STRING('i', "input", &inject.input_name, "file",
 			   "input file name"),
 		OPT_STRING('o', "output", &inject.output.path, "file",
@@ -815,8 +915,6 @@ int cmd_inject(int argc, const char **argv)
 		return -1;
 	}
 
-	inject.tool.ordered_events = inject.sched_stat;
-
 	data.path = inject.input_name;
 	inject.session = perf_session__new(&data, true, &inject.tool);
 	if (IS_ERR(inject.session))
@@ -825,7 +923,7 @@ int cmd_inject(int argc, const char **argv)
 	if (zstd_init(&(inject.session->zstd_data), 0) < 0)
 		pr_warning("Decompression initialization failed.\n");
 
-	if (inject.build_ids) {
+	if (inject.build_ids && !inject.build_id_all) {
 		/*
 		 * to make sure the mmap records are ordered correctly
 		 * and so that the correct especially due to jitted code
@@ -835,6 +933,11 @@ int cmd_inject(int argc, const char **argv)
 		inject.tool.ordered_events = true;
 		inject.tool.ordering_requires_timestamps = true;
 	}
+
+	if (inject.sched_stat) {
+		inject.tool.ordered_events = true;
+	}
+
 #ifdef HAVE_JITDUMP
 	if (inject.jit_mode) {
 		inject.tool.mmap2	   = perf_event__jit_repipe_mmap2;
-- 
2.28.0.1011.ga647a8990f-goog


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

* [PATCH 6/6] perf bench: Run inject-build-id with --buildid-all option too
  2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
                   ` (4 preceding siblings ...)
  2020-10-12  7:02 ` [PATCH 5/6] perf inject: Add --buildid-all option Namhyung Kim
@ 2020-10-12  7:02 ` Namhyung Kim
  2020-10-13 12:11 ` [PATCHSET v4 0/6] perf inject: Speed build-id injection Arnaldo Carvalho de Melo
  6 siblings, 0 replies; 8+ messages in thread
From: Namhyung Kim @ 2020-10-12  7:02 UTC (permalink / raw)
  To: Arnaldo Carvalho de Melo, Jiri Olsa
  Cc: Ingo Molnar, Peter Zijlstra, Mark Rutland, Alexander Shishkin,
	LKML, Stephane Eranian, Ian Rogers

For comparison, it now runs the benchmark twice - one if regular -b
and another for --buildid-all.

  $ perf bench internals inject-build-id
  # Running 'internals/inject-build-id' benchmark:
    Average build-id injection took: 21.002 msec (+- 0.172 msec)
    Average time per event: 2.059 usec (+- 0.017 usec)
    Average memory usage: 8169 KB (+- 0 KB)
    Average build-id-all injection took: 19.543 msec (+- 0.124 msec)
    Average time per event: 1.916 usec (+- 0.012 usec)
    Average memory usage: 7348 KB (+- 0 KB)

Acked-by: Jiri Olsa <jolsa@redhat.com>
Acked-by: Ian Rogers <irogers@google.com>
Signed-off-by: Namhyung Kim <namhyung@kernel.org>
---
 tools/perf/bench/inject-buildid.c | 54 ++++++++++++++++++++-----------
 1 file changed, 35 insertions(+), 19 deletions(-)

diff --git a/tools/perf/bench/inject-buildid.c b/tools/perf/bench/inject-buildid.c
index 0fccf2a9e95b..e9a11f4a1109 100644
--- a/tools/perf/bench/inject-buildid.c
+++ b/tools/perf/bench/inject-buildid.c
@@ -271,7 +271,7 @@ static void *data_reader(void *arg)
 	return NULL;
 }
 
-static int setup_injection(struct bench_data *data)
+static int setup_injection(struct bench_data *data, bool build_id_all)
 {
 	int ready_pipe[2];
 	int dev_null_fd;
@@ -292,6 +292,7 @@ static int setup_injection(struct bench_data *data)
 
 	if (data->pid == 0) {
 		const char **inject_argv;
+		int inject_argc = 2;
 
 		close(data->input_pipe[1]);
 		close(data->output_pipe[0]);
@@ -308,17 +309,22 @@ static int setup_injection(struct bench_data *data)
 
 		dup2(dev_null_fd, STDERR_FILENO);
 
-		inject_argv = calloc(3, sizeof(*inject_argv));
+		if (build_id_all)
+			inject_argc++;
+
+		inject_argv = calloc(inject_argc + 1, sizeof(*inject_argv));
 		if (inject_argv == NULL)
 			exit(1);
 
 		inject_argv[0] = strdup("inject");
 		inject_argv[1] = strdup("-b");
+		if (build_id_all)
+			inject_argv[2] = strdup("--buildid-all");
 
 		/* signal that we're ready to go */
 		close(ready_pipe[1]);
 
-		cmd_inject(2, inject_argv);
+		cmd_inject(inject_argc, inject_argv);
 
 		exit(0);
 	}
@@ -377,27 +383,17 @@ static int inject_build_id(struct bench_data *data, u64 *max_rss)
 	return 0;
 }
 
-static int do_inject_loop(struct bench_data *data)
+static void do_inject_loop(struct bench_data *data, bool build_id_all)
 {
 	unsigned int i;
 	struct stats time_stats, mem_stats;
 	double time_average, time_stddev;
 	double mem_average, mem_stddev;
 
-	srand(time(NULL));
 	init_stats(&time_stats);
 	init_stats(&mem_stats);
-	symbol__init(NULL);
 
-	bench_sample_type  = PERF_SAMPLE_IDENTIFIER | PERF_SAMPLE_IP;
-	bench_sample_type |= PERF_SAMPLE_TID | PERF_SAMPLE_TIME;
-	bench_id_hdr_size  = 32;
-
-	collect_dso();
-	if (nr_dsos == 0) {
-		printf("  Cannot collect DSOs for injection\n");
-		return -1;
-	}
+	pr_debug("  Build-id%s injection benchmark\n", build_id_all ? "-all" : "");
 
 	for (i = 0; i < iterations; i++) {
 		struct timeval start, end, diff;
@@ -405,7 +401,7 @@ static int do_inject_loop(struct bench_data *data)
 
 		pr_debug("  Iteration #%d\n", i+1);
 
-		if (setup_injection(data) < 0) {
+		if (setup_injection(data, build_id_all) < 0) {
 			printf("  Build-id injection setup failed\n");
 			break;
 		}
@@ -427,8 +423,8 @@ static int do_inject_loop(struct bench_data *data)
 
 	time_average = avg_stats(&time_stats) / USEC_PER_MSEC;
 	time_stddev = stddev_stats(&time_stats) / USEC_PER_MSEC;
-	printf("  Average build-id injection took: %.3f msec (+- %.3f msec)\n",
-		time_average, time_stddev);
+	printf("  Average build-id%s injection took: %.3f msec (+- %.3f msec)\n",
+	       build_id_all ? "-all" : "", time_average, time_stddev);
 
 	/* each iteration, it processes MMAP2 + BUILD_ID + nr_samples * SAMPLE */
 	time_average = avg_stats(&time_stats) / (nr_mmaps * (nr_samples + 2));
@@ -440,6 +436,26 @@ static int do_inject_loop(struct bench_data *data)
 	mem_stddev = stddev_stats(&mem_stats);
 	printf("  Average memory usage: %.0f KB (+- %.0f KB)\n",
 		mem_average, mem_stddev);
+}
+
+static int do_inject_loops(struct bench_data *data)
+{
+
+	srand(time(NULL));
+	symbol__init(NULL);
+
+	bench_sample_type  = PERF_SAMPLE_IDENTIFIER | PERF_SAMPLE_IP;
+	bench_sample_type |= PERF_SAMPLE_TID | PERF_SAMPLE_TIME;
+	bench_id_hdr_size  = 32;
+
+	collect_dso();
+	if (nr_dsos == 0) {
+		printf("  Cannot collect DSOs for injection\n");
+		return -1;
+	}
+
+	do_inject_loop(data, false);
+	do_inject_loop(data, true);
 
 	release_dso();
 	return 0;
@@ -455,6 +471,6 @@ int bench_inject_build_id(int argc, const char **argv)
 		exit(EXIT_FAILURE);
 	}
 
-	return do_inject_loop(&data);
+	return do_inject_loops(&data);
 }
 
-- 
2.28.0.1011.ga647a8990f-goog


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

* Re: [PATCHSET v4 0/6] perf inject: Speed build-id injection
  2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
                   ` (5 preceding siblings ...)
  2020-10-12  7:02 ` [PATCH 6/6] perf bench: Run inject-build-id with --buildid-all option too Namhyung Kim
@ 2020-10-13 12:11 ` Arnaldo Carvalho de Melo
  6 siblings, 0 replies; 8+ messages in thread
From: Arnaldo Carvalho de Melo @ 2020-10-13 12:11 UTC (permalink / raw)
  To: Namhyung Kim
  Cc: Jiri Olsa, Ingo Molnar, Peter Zijlstra, Mark Rutland,
	Alexander Shishkin, LKML, Stephane Eranian, Ian Rogers

Em Mon, Oct 12, 2020 at 04:02:08PM +0900, Namhyung Kim escreveu:
> Hello,
> 
> This is the new version of speed up build-id injection.  As this is
> to improve performance, I've added a benchmark for it.  Please look at
> the usage in the first commit.
> 
> By default, it measures average processing time of 100 MMAP2 events
> and 10000 SAMPLE events.  Below is the current result on my laptop.
> 
>   $ perf bench internals inject-build-id
>   # Running 'internals/inject-build-id' benchmark:
>     Average build-id injection took: 25.789 msec (+- 0.202 msec)
>     Average time per event: 2.528 usec (+- 0.020 usec)
>     Average memory usage: 8411 KB (+- 7 KB)
> 
> With this patchset applied, it got this:
> 
>   $ perf bench internals inject-build-id
>   # Running 'internals/inject-build-id' benchmark:
>     Average build-id injection took: 20.838 msec (+- 0.093 msec)
>     Average time per event: 2.043 usec (+- 0.009 usec)
>     Average memory usage: 8261 KB (+- 0 KB)
>     Average build-id-all injection took: 19.361 msec (+- 0.118 msec)
>     Average time per event: 1.898 usec (+- 0.012 usec)
>     Average memory usage: 7440 KB (+- 0 KB)
> 
> 
> Real usecases might be different as it depends on the number of
> mmap/sample events as well as how many DSOs are actually hit.
> 
> The benchmark result now includes memory footprint in terms of maximum
> RSS.  Also I've update the benchmark code to use timestamp so that it
> can be queued to the ordered_events (and flushed at the end).  It's
> also important how well it sorts the input events in the queue so I
> randomly chose a timestamp at the beginning of each MMAP event
> injection to resemble actual behavior.
> 
> As I said in other thread, perf inject currently doesn't flush the
> input events and processes all at the end.  This gives a good speedup
> but spends more memory (in proprotion to the input size).  While the
> build-id-all injection bypasses the queue so it uses less memory as
> well as faster processing.  The downside is that it'll mark all DSOs
> as hit so later processing steps (like perf report) likely handle them
> unnecessarily.

Thanks, tested and applied, first patchkit I process using that b4 tool,
cool!

- Arnaldo
 
> 
> This code is available at 'perf/inject-speedup-v4' branch on
> 
>   git://git.kernel.org/pub/scm/linux/kernel/git/namhyung/linux-perf.git
> 
> 
> Changes from v3:
>  - add timestamp to the synthesized events in the benchmark
>  - add a separate thread to read pipe in the benchmark
> 
> Changes from v2:
>  - fix benchmark to read required data
>  - add Acked-by from Jiri and Ian
>  - pass map flag to check huge pages  (Jiri)
>  - add comments on some functions  (Ian)
>  - show memory (max-RSS) usage in the benchmark  (Ian)
>  - drop build-id marking patch at the last  (Adrian)
> 
> 
> Namhyung Kim (6):
>   perf bench: Add build-id injection benchmark
>   perf inject: Add missing callbacks in perf_tool
>   perf inject: Enter namespace when reading build-id
>   perf inject: Do not load map/dso when injecting build-id
>   perf inject: Add --buildid-all option
>   perf bench: Run inject-build-id with --buildid-all option too
> 
>  tools/perf/Documentation/perf-inject.txt |   6 +-
>  tools/perf/bench/Build                   |   1 +
>  tools/perf/bench/bench.h                 |   1 +
>  tools/perf/bench/inject-buildid.c        | 457 +++++++++++++++++++++++
>  tools/perf/builtin-bench.c               |   1 +
>  tools/perf/builtin-inject.c              | 199 ++++++++--
>  tools/perf/util/build-id.h               |   4 +
>  tools/perf/util/map.c                    |  17 +-
>  tools/perf/util/map.h                    |  14 +
>  9 files changed, 645 insertions(+), 55 deletions(-)
>  create mode 100644 tools/perf/bench/inject-buildid.c
> 
> -- 
> 2.28.0.681.g6f77f65b4e-goog
> 
> 
> *** BLURB HERE ***
> 
> Namhyung Kim (6):
>   perf bench: Add build-id injection benchmark
>   perf inject: Add missing callbacks in perf_tool
>   perf inject: Enter namespace when reading build-id
>   perf inject: Do not load map/dso when injecting build-id
>   perf inject: Add --buildid-all option
>   perf bench: Run inject-build-id with --buildid-all option too
> 
>  tools/perf/Documentation/perf-inject.txt |   6 +-
>  tools/perf/bench/Build                   |   1 +
>  tools/perf/bench/bench.h                 |   1 +
>  tools/perf/bench/inject-buildid.c        | 476 +++++++++++++++++++++++
>  tools/perf/builtin-bench.c               |   1 +
>  tools/perf/builtin-inject.c              | 199 ++++++++--
>  tools/perf/util/build-id.h               |   4 +
>  tools/perf/util/map.c                    |  17 +-
>  tools/perf/util/map.h                    |  14 +
>  9 files changed, 664 insertions(+), 55 deletions(-)
>  create mode 100644 tools/perf/bench/inject-buildid.c
> 
> -- 
> 2.28.0.1011.ga647a8990f-goog
> 

-- 

- Arnaldo

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

end of thread, other threads:[~2020-10-13 12:11 UTC | newest]

Thread overview: 8+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2020-10-12  7:02 [PATCHSET v4 0/6] perf inject: Speed build-id injection Namhyung Kim
2020-10-12  7:02 ` [PATCH 1/6] perf bench: Add build-id injection benchmark Namhyung Kim
2020-10-12  7:02 ` [PATCH 2/6] perf inject: Add missing callbacks in perf_tool Namhyung Kim
2020-10-12  7:02 ` [PATCH 3/6] perf inject: Enter namespace when reading build-id Namhyung Kim
2020-10-12  7:02 ` [PATCH 4/6] perf inject: Do not load map/dso when injecting build-id Namhyung Kim
2020-10-12  7:02 ` [PATCH 5/6] perf inject: Add --buildid-all option Namhyung Kim
2020-10-12  7:02 ` [PATCH 6/6] perf bench: Run inject-build-id with --buildid-all option too Namhyung Kim
2020-10-13 12:11 ` [PATCHSET v4 0/6] perf inject: Speed build-id injection Arnaldo Carvalho de Melo

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