All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCH v3] user_events: Enable user processes to create and write to trace events
@ 2021-10-18 23:09 Beau Belgrave
  2021-10-22 13:38 ` Masami Hiramatsu
  2021-10-26  0:46   ` kernel test robot
  0 siblings, 2 replies; 9+ messages in thread
From: Beau Belgrave @ 2021-10-18 23:09 UTC (permalink / raw)
  To: rostedt, mhiramat; +Cc: linux-trace-devel, linux-kernel, beaub

User mode processes that wish to use trace events to get data into
ftrace, perf, eBPF, etc are limited to uprobes today. The user events
features enables an ABI for user mode processes to create and write to
trace events that are isolated from kernel level trace events. This
enables a faster path for tracing from user mode data as well as opens
managed code to participate in trace events, where stub locations are
dynamic.

User processes often want to trace only when it's useful. To enable this
a set of pages are mapped into the user process space that indicate the
current state of the user events that have been registered. User
processes can check if their event is hooked to a trace/probe, and if it
is, emit the event data out via the write() syscall.

Two new files are introduced into tracefs to accomplish this:
user_events_status - This file is mmap'd into participating user mode
processes to indicate event status.

user_events_data - This file is opened and register/delete ioctl's are
issued to create/open/delete trace events that can be used for tracing.

The typical scenario is on process start to mmap user_events_status. Processes
then register the events they plan to use via the REG ioctl. The return value
of the ioctl indicates the byte in the mmap to use for status and the index
to use for writing. Both are 16-bits in size and are accessible via the
INDEX_STATUS and INDEX_WRITE macros within user_events.h. The fd that was
used for the ioctl can now be used to write out data. The data must always
include an int describing which event the data is for at the start of the
payload.

For example, in memory:
int index;
char data[];

Psuedo code example of typical usage:
int page_fd = open("user_events_status", O_RDWR);
char *page_data = mmap(NULL, PAGE_SIZE, PROT_READ, MAP_SHARED, page_fd, 0);

int data_fd = open("user_events_data", O_RDWR);
long result = ioctl(data_fd, DIAG_IOCSREG, "test");
int status_id = INDEX_STATUS(result);
int write_id = INDEX_WRITE(result);

struct iovec io[2];
io[0].iov_base = &write_id;
io[0].iov_len = sizeof(write_id);
io[1].iov_base = payload;
io[1].iov_len = sizeof(payload);

if (page_data[status_id])
	writev(data_fd, io, 2);

User events are also exposed via the dynamic_events tracefs file for
both create and delete. Current status is exposed via the user_events_status
tracefs file.

Simple example to register a user event via dynamic_events:
	echo u:test >> dynamic_events
	cat dynamic_events
	u:test

If an event is hooked to a probe, the probe hooked shows up:
	echo 1 > events/user_events/test/enable
	cat user_events_status
	1:test # Used by ftrace

	Active: 1
	Busy: 1
	Max: 4096

If an event is not hooked to a probe, no probe status shows up:
	echo 0 > events/user_events/test/enable
	cat user_events_status
	1:test

	Active: 1
	Busy: 0
	Max: 4096

Users can describe the trace event format via the following format:
	name [field1;field2]

Each field has the following format:
	type name size

Example for char array with a size of 20 named msg:
	echo 'u:detailed char[20] msg 20' >> dynamic_events
	cat dynamic_events
	u:detailed char[20] msg

Data offsets are based on the data written out via write() and will be
updated to reflect the correct offset in the trace_event fields. __data_loc
types must be aware of the size of trace_entry/common properties to ensure
proper decoding.

The above format is valid for both the ioctl and the dynamic_events file.

V2:
Fixed kmalloc vs kzalloc for register_page.
Renamed user_event_mmap to user_event_status.
Renamed user_event prefix from ue to u.
Added seq_* operations to user_event_status to enable cat output.
Aligned field parsing to synth_events format (+ size specifier for
custom/user types).
Added uapi header user_events.h to align kernel and user ABI definitions.

V3:
Updated ABI to handle single FD into many events via an int header.
Added iovec/writev support to enable int header without payload changes.
Updated bpf context to describe if data is coming from user, kernel or
raw iovec.
Added flag support for registering event, allows forcing BPF to always
recieve the direct iovecs for sensitive code paths that do not want
copies.

Signed-off-by: Beau Belgrave <beaub@linux.microsoft.com>
---
 include/uapi/linux/user_events.h |   57 ++
 kernel/trace/Kconfig             |   15 +
 kernel/trace/Makefile            |    1 +
 kernel/trace/trace_events_user.c | 1040 ++++++++++++++++++++++++++++++
 4 files changed, 1113 insertions(+)
 create mode 100644 include/uapi/linux/user_events.h
 create mode 100644 kernel/trace/trace_events_user.c

diff --git a/include/uapi/linux/user_events.h b/include/uapi/linux/user_events.h
new file mode 100644
index 000000000000..00122624ff26
--- /dev/null
+++ b/include/uapi/linux/user_events.h
@@ -0,0 +1,57 @@
+/* SPDX-License-Identifier: GPL-2.0 WITH Linux-syscall-note */
+/*
+ * Copyright (c) 2021, Microsoft Corporation.
+ *
+ * Authors:
+ *   Beau Belgrave <beaub@linux.microsoft.com>
+ */
+#ifndef _UAPI_LINUX_USER_EVENTS_H
+#define _UAPI_LINUX_USER_EVENTS_H
+
+#include <linux/types.h>
+#include <linux/ioctl.h>
+#include <linux/uio.h>
+
+#define USER_EVENTS_SYSTEM "user_events"
+#define USER_EVENTS_PREFIX "u:"
+
+/* Bits 0-6 are for known probe types, Bit 7 is for unknown probes */
+#define EVENT_BIT_FTRACE 0
+#define EVENT_BIT_PERF 1
+#define EVENT_BIT_OTHER 7
+
+#define EVENT_STATUS_FTRACE (1 << EVENT_BIT_FTRACE)
+#define EVENT_STATUS_PERF (1 << EVENT_BIT_PERF)
+#define EVENT_STATUS_OTHER (1 << EVENT_BIT_OTHER)
+
+#define DIAG_IOC_MAGIC '*'
+#define DIAG_IOCSREG _IOW(DIAG_IOC_MAGIC, 0, char*)
+#define DIAG_IOCSDEL _IOW(DIAG_IOC_MAGIC, 1, char*)
+
+#define INDEX_WRITE(index) (index & 0xFFFF)
+#define INDEX_STATUS(index) ((index >> 16) & 0xFFFF)
+#define INDEX_COMBINE(write, status) (status << 16 | write)
+
+enum {
+	USER_BPF_DATA_KERNEL,
+	USER_BPF_DATA_USER,
+	USER_BPF_DATA_ITER,
+};
+
+struct user_bpf_iter {
+	size_t iov_offset;
+	const struct iovec *iov;
+	unsigned long nr_segs;
+};
+
+struct user_bpf_context {
+	int data_type;
+	int data_len;
+	union {
+		void *kdata;
+		void *udata;
+		struct user_bpf_iter *iter;
+	};
+};
+
+#endif /* _UAPI_LINUX_USER_EVENTS_H */
diff --git a/kernel/trace/Kconfig b/kernel/trace/Kconfig
index 3ee23f4d437f..deaaad421be4 100644
--- a/kernel/trace/Kconfig
+++ b/kernel/trace/Kconfig
@@ -725,6 +725,21 @@ config SYNTH_EVENTS
 
 	  If in doubt, say N.
 
+config USER_EVENTS
+	bool "User trace events"
+	select TRACING
+	select DYNAMIC_EVENTS
+	default n
+	help
+	  User trace events are user-defined trace events that
+	  can be used like an existing kernel trace event.  User trace
+	  events are generated by writing to a tracefs file.  User
+	  processes can determine if their tracing events should be
+	  generated by memory mapping a tracefs file and checking for
+	  an associated byte being non-zero.
+
+	  If in doubt, say N.
+
 config HIST_TRIGGERS
 	bool "Histogram triggers"
 	depends on ARCH_HAVE_NMI_SAFE_CMPXCHG
diff --git a/kernel/trace/Makefile b/kernel/trace/Makefile
index b1c47ccf4f73..a653b255e89c 100644
--- a/kernel/trace/Makefile
+++ b/kernel/trace/Makefile
@@ -80,6 +80,7 @@ obj-$(CONFIG_EVENT_TRACING) += trace_events_trigger.o
 obj-$(CONFIG_TRACE_EVENT_INJECT) += trace_events_inject.o
 obj-$(CONFIG_SYNTH_EVENTS) += trace_events_synth.o
 obj-$(CONFIG_HIST_TRIGGERS) += trace_events_hist.o
+obj-$(CONFIG_USER_EVENTS) += trace_events_user.o
 obj-$(CONFIG_BPF_EVENTS) += bpf_trace.o
 obj-$(CONFIG_KPROBE_EVENTS) += trace_kprobe.o
 obj-$(CONFIG_TRACEPOINTS) += error_report-traces.o
diff --git a/kernel/trace/trace_events_user.c b/kernel/trace/trace_events_user.c
new file mode 100644
index 000000000000..4a90d383bab7
--- /dev/null
+++ b/kernel/trace/trace_events_user.c
@@ -0,0 +1,1040 @@
+// SPDX-License-Identifier: GPL-2.0-only
+/*
+ * Copyright (c) 2021, Microsoft Corporation.
+ *
+ * Authors:
+ *   Beau Belgrave <beaub@linux.microsoft.com>
+ */
+
+#include <linux/bitmap.h>
+#include <linux/cdev.h>
+#include <linux/hashtable.h>
+#include <linux/io.h>
+#include <linux/uio.h>
+#include <linux/ioctl.h>
+#include <linux/jhash.h>
+#include <linux/trace_events.h>
+#include <linux/tracefs.h>
+#include <linux/types.h>
+#include <linux/uaccess.h>
+#include <uapi/linux/user_events.h>
+#include "trace.h"
+#include "trace_dynevent.h"
+
+#define USER_EVENTS_PREFIX_LEN (sizeof(USER_EVENTS_PREFIX)-1)
+
+#define FIELD_DEPTH_TYPE 0
+#define FIELD_DEPTH_NAME 1
+#define FIELD_DEPTH_SIZE 2
+
+/*
+ * Limits how many trace_event calls user processes can create:
+ * Must be multiple of PAGE_SIZE.
+ */
+#define MAX_PAGES 1
+#define MAX_EVENTS (MAX_PAGES * PAGE_SIZE)
+#define MAX_BPF_COPY_SIZE PAGE_SIZE
+#define MAX_STACK_BPF_DATA 512
+
+/* Limit how long of an event name plus args within the subsystem. */
+#define MAX_EVENT_DESC 512
+#define EVENT_NAME(user_event) ((user_event)->tracepoint.name)
+
+#define FLAG_BPF_ITER 1
+
+static char *register_page_data;
+
+static DEFINE_HASHTABLE(register_table, 4);
+static DECLARE_BITMAP(page_bitmap, MAX_EVENTS);
+
+struct user_event {
+	struct tracepoint tracepoint;
+	struct trace_event_call call;
+	struct trace_event_class class;
+	struct dyn_event devent;
+	struct hlist_node node;
+	atomic_t refs;
+	int index;
+	char *args;
+	int flags;
+};
+
+struct user_event_refs {
+	struct rcu_head rcu;
+	int count;
+	struct user_event *events[];
+};
+
+typedef void (*user_event_func_t) (struct user_event *user,
+				   struct iov_iter *i,
+				   void *tpdata);
+
+static int register_user_event(char *name, char *args,
+			       struct user_event **newuser);
+
+/*
+ * Parses a register command for user_events
+ * Format: event_name [field1;field2;...]
+ *
+ * Example event named test with a 20 char msg field with a unsigned int after:
+ * test char[20] msg 20;unsigned int id 4;
+ *
+ * NOTE: Offsets are from the user data perspective, they are not from the
+ * trace_entry/buffer perspective. We automatically add the common properties
+ * sizes to the offset for the user.
+ */
+static int user_event_parse_cmd(char *raw_command, struct user_event **newuser)
+{
+	char *name = raw_command;
+	char *args = strpbrk(name, " ");
+
+	if (args)
+		*args++ = 0;
+
+	return register_user_event(name, args, newuser);
+}
+
+/*
+ * Parses the values of a field within the description
+ * Format: type name size [future]
+ */
+static int user_event_parse_field(char *field, struct user_event *user,
+				  u32 *offset)
+{
+	char *part, *type, *name;
+	u32 size = 0, depth = 0, saved_offset = *offset;
+
+	field = skip_spaces(field);
+
+	if (strstr(field, "unsigned ") == field) {
+		type = field;
+		field = strpbrk(field + sizeof("unsigned"), " ");
+
+		if (field == NULL)
+			return -EINVAL;
+
+		*field++ = 0;
+		depth++;
+	} else if (strstr(field, "print_fmt ") == field) {
+		user->call.print_fmt = field + sizeof("print_fmt");
+		return 0;
+	} else if (strstr(field, "flag ") == field) {
+		field += sizeof("flag");
+
+		if (!strcmp(field, "bpf_iter"))
+			user->flags |= FLAG_BPF_ITER;
+
+		return 0;
+	}
+
+	while ((part = strsep(&field, " ")) != NULL) {
+		switch (depth++) {
+		case FIELD_DEPTH_TYPE:
+			type = part;
+			break;
+		case FIELD_DEPTH_NAME:
+			name = part;
+			break;
+		case FIELD_DEPTH_SIZE:
+			if (kstrtou32(part, 10, &size))
+				return -EINVAL;
+			break;
+		default:
+			/* Forward compatibility, ignore */
+			goto end;
+		}
+	}
+end:
+	if (depth < FIELD_DEPTH_SIZE)
+		return -EINVAL;
+
+	*offset = saved_offset + size;
+
+	return trace_define_field(&user->call, type, name, saved_offset, size,
+				  type[0] != 'u', FILTER_OTHER);
+}
+
+/*
+ * Parses the fields that were described for the event
+ */
+static int user_event_parse_fields(struct user_event *user)
+{
+	char *field;
+	u32 offset = sizeof(struct trace_entry);
+	int ret = -EINVAL;
+
+	while ((field = strsep(&user->args, ";")) != NULL) {
+		ret = user_event_parse_field(field, user, &offset);
+
+		if (ret)
+			break;
+	}
+
+	return ret;
+}
+
+static int user_event_define_fields(struct trace_event_call *call)
+{
+	struct user_event *user = (struct user_event *)call->data;
+
+	/* User chose to not disclose arguments */
+	if (user->args == NULL)
+		return 0;
+
+	return user_event_parse_fields(user);
+}
+
+static struct trace_event_fields user_event_fields_array[] = {
+	{ .type = TRACE_FUNCTION_TYPE,
+	  .define_fields = user_event_define_fields },
+	{}
+};
+
+static enum print_line_t user_event_print_trace(struct trace_iterator *iter,
+						int flags,
+						struct trace_event *event)
+{
+	/* Unsafe to try to decode user provided print_fmt, use hex */
+	trace_print_hex_dump_seq(&iter->seq, "", DUMP_PREFIX_OFFSET, 16,
+				 1, iter->ent, iter->ent_size, true);
+
+	return trace_handle_return(&iter->seq);
+}
+
+static struct trace_event_functions user_event_funcs = {
+	.trace = user_event_print_trace,
+};
+
+static int destroy_user_event(struct user_event *user)
+{
+	int ret = 0;
+
+	/*
+	 * trace_remove_event_call invokes unregister_trace_event:
+	 * Pick the correct one based on if we set the data or not
+	 */
+	if (user->index != 0) {
+		ret = trace_remove_event_call(&user->call);
+
+		if (ret)
+			return ret;
+
+		dyn_event_remove(&user->devent);
+
+		register_page_data[user->index] = 0;
+		clear_bit(user->index, page_bitmap);
+		hash_del(&user->node);
+	} else {
+		unregister_trace_event(&user->call.event);
+	}
+
+	kfree(EVENT_NAME(user));
+	kfree(user);
+
+	return ret;
+}
+
+static struct user_event *find_user_event(u32 key, char *name)
+{
+	struct user_event *user;
+
+	hash_for_each_possible(register_table, user, node, key)
+		if (!strcmp(EVENT_NAME(user), name))
+			return user;
+
+	return NULL;
+}
+
+/*
+ * Writes the user supplied payload out to a trace file.
+ */
+static void user_event_ftrace(struct user_event *user, struct iov_iter *i,
+			      void *tpdata)
+{
+	struct trace_event_file *file;
+	struct trace_entry *entry;
+	struct trace_event_buffer event_buffer;
+	size_t count = i->count;
+
+	file = (struct trace_event_file *)tpdata;
+
+	if (!file ||
+	    !(file->flags & EVENT_FILE_FL_ENABLED) ||
+	    trace_trigger_soft_disabled(file))
+		return;
+
+	entry = trace_event_buffer_reserve(&event_buffer, file,
+					   sizeof(*entry) + count);
+
+	if (unlikely(!entry))
+		return;
+
+	if (unlikely(copy_from_iter(entry + 1, count, i) != count))
+		return;
+
+	trace_event_buffer_commit(&event_buffer);
+}
+
+#ifdef CONFIG_PERF_EVENTS
+/*
+ * Writes the user supplied payload out to perf ring buffer or eBPF program.
+ */
+static void user_event_perf(struct user_event *user, struct iov_iter *i,
+			    void *tpdata)
+{
+	struct hlist_head *perf_head;
+	size_t count = i->count;
+
+	if (bpf_prog_array_valid(&user->call)) {
+		struct user_bpf_context context = {0};
+		struct user_bpf_iter bpf_i = {0};
+		char fast_data[MAX_STACK_BPF_DATA];
+		void *temp = NULL;
+
+		context.data_len = count;
+
+		if ((user->flags & FLAG_BPF_ITER) && iter_is_iovec(i)) {
+			/* Raw iovec to BPF, no copies */
+			bpf_i.iov_offset = i->iov_offset;
+			bpf_i.iov = i->iov;
+			bpf_i.nr_segs = i->nr_segs;
+
+			context.data_type = USER_BPF_DATA_ITER;
+			context.iter = &bpf_i;
+		} else if (i->nr_segs == 1 && iter_is_iovec(i)) {
+			/* Single vector, no copies */
+			context.data_type = USER_BPF_DATA_USER;
+			context.udata = i->iov->iov_base + i->iov_offset;
+		} else {
+			/* Multi vector, copy on stack or heap */
+			struct iov_iter copy;
+			size_t copy_size = count;
+
+			context.data_type = USER_BPF_DATA_KERNEL;
+
+			if (copy_size > MAX_BPF_COPY_SIZE)
+				copy_size = MAX_BPF_COPY_SIZE;
+
+			if (copy_size <= sizeof(fast_data))
+				context.kdata = fast_data;
+			else {
+				temp = kmalloc(copy_size, GFP_KERNEL);
+				context.kdata = temp;
+			}
+
+			/* Must keep in case perf_events is hooked as well */
+			copy = *i;
+
+			context.data_len = copy_from_iter(context.kdata,
+							  copy_size, &copy);
+		}
+
+		trace_call_bpf(&user->call, &context);
+
+		kfree(temp);
+	}
+
+	perf_head = this_cpu_ptr(user->call.perf_events);
+
+	if (perf_head && !hlist_empty(perf_head)) {
+		struct trace_entry *perf_entry;
+		struct pt_regs *regs;
+		size_t size = sizeof(*perf_entry) + count;
+		int context;
+
+		perf_entry = perf_trace_buf_alloc(ALIGN(size, 8),
+						  &regs, &context);
+
+		if (unlikely(!perf_entry))
+			return;
+
+		perf_fetch_caller_regs(regs);
+
+		if (unlikely(copy_from_iter(perf_entry + 1, count, i) != count))
+			return;
+
+		perf_trace_buf_submit(perf_entry, size, context,
+				      user->call.event.type, 1, regs,
+				      perf_head, NULL);
+	}
+}
+#endif
+
+/*
+ * Update the register page that is shared between user processes.
+ */
+static void update_reg_page_for(struct user_event *user)
+{
+	struct tracepoint *tp = &user->tracepoint;
+	char status = 0;
+
+	if (atomic_read(&tp->key.enabled) > 0) {
+		struct tracepoint_func *probe_func_ptr;
+		user_event_func_t probe_func;
+
+		probe_func_ptr = rcu_dereference_sched(tp->funcs);
+
+		if (probe_func_ptr) {
+			do {
+				probe_func = probe_func_ptr->func;
+
+				if (probe_func == user_event_ftrace)
+					status |= EVENT_STATUS_FTRACE;
+#ifdef CONFIG_PERF_EVENTS
+				else if (probe_func == user_event_perf)
+					status |= EVENT_STATUS_PERF;
+#endif
+				else
+					status |= EVENT_STATUS_OTHER;
+			} while ((++probe_func_ptr)->func);
+		}
+	}
+
+	register_page_data[user->index] = status;
+}
+
+/*
+ * Register callback for our events from tracing sub-systems.
+ */
+static int user_event_reg(struct trace_event_call *call,
+			  enum trace_reg type,
+			  void *data)
+{
+	struct user_event *user = (struct user_event *)call->data;
+	int ret = 0;
+
+	if (!user)
+		return -ENOENT;
+
+	switch (type) {
+	case TRACE_REG_REGISTER:
+		ret = tracepoint_probe_register(call->tp,
+						call->class->probe,
+						data);
+		if (!ret)
+			goto inc;
+		break;
+
+	case TRACE_REG_UNREGISTER:
+		tracepoint_probe_unregister(call->tp,
+					    call->class->probe,
+					    data);
+		goto dec;
+
+#ifdef CONFIG_PERF_EVENTS
+	case TRACE_REG_PERF_REGISTER:
+		ret = tracepoint_probe_register(call->tp,
+						call->class->perf_probe,
+						data);
+		if (!ret)
+			goto inc;
+		break;
+
+	case TRACE_REG_PERF_UNREGISTER:
+		tracepoint_probe_unregister(call->tp,
+					    call->class->perf_probe,
+					    data);
+		goto dec;
+
+	case TRACE_REG_PERF_OPEN:
+	case TRACE_REG_PERF_CLOSE:
+	case TRACE_REG_PERF_ADD:
+	case TRACE_REG_PERF_DEL:
+		break;
+#endif
+	}
+
+	return ret;
+inc:
+	atomic_inc(&user->refs);
+	update_reg_page_for(user);
+	return 0;
+dec:
+	update_reg_page_for(user);
+	atomic_dec(&user->refs);
+	return 0;
+}
+
+static u32 user_event_key(char *name)
+{
+	return jhash(name, strlen(name), 0);
+}
+
+static int user_event_create(const char *raw_command)
+{
+	struct user_event *user;
+	char *name;
+	int ret;
+
+	if (strstr(raw_command, USER_EVENTS_PREFIX) != raw_command)
+		return -ECANCELED;
+
+	raw_command += USER_EVENTS_PREFIX_LEN;
+	raw_command = skip_spaces(raw_command);
+
+	name = kstrdup(raw_command, GFP_KERNEL);
+
+	if (!name)
+		return -ENOMEM;
+
+	mutex_lock(&event_mutex);
+	ret = user_event_parse_cmd(name, &user);
+	mutex_unlock(&event_mutex);
+
+	return ret;
+}
+
+static int user_event_show(struct seq_file *m, struct dyn_event *ev)
+{
+	struct user_event *user = container_of(ev, struct user_event, devent);
+	struct ftrace_event_field *field, *next;
+	struct list_head *head;
+	int depth = 0;
+
+	seq_printf(m, "%s%s", USER_EVENTS_PREFIX, EVENT_NAME(user));
+
+	head = trace_get_fields(&user->call);
+
+	list_for_each_entry_safe_reverse(field, next, head, link) {
+		if (depth == 0)
+			seq_puts(m, " ");
+		else
+			seq_puts(m, "; ");
+		seq_printf(m, "%s %s", field->type, field->name);
+		depth++;
+	}
+
+	seq_puts(m, "\n");
+
+	return 0;
+}
+
+static bool user_event_is_busy(struct dyn_event *ev)
+{
+	struct user_event *user = container_of(ev, struct user_event, devent);
+
+	return atomic_read(&user->refs) != 0;
+}
+
+static int user_event_free(struct dyn_event *ev)
+{
+	struct user_event *user = container_of(ev, struct user_event, devent);
+
+	return destroy_user_event(user);
+}
+
+static bool user_event_match(const char *system, const char *event,
+			     int argc, const char **argv, struct dyn_event *ev)
+{
+	struct user_event *user = container_of(ev, struct user_event, devent);
+
+	return strcmp(EVENT_NAME(user), event) == 0 &&
+		(!system || strcmp(system, USER_EVENTS_SYSTEM) == 0);
+}
+
+static struct dyn_event_operations user_event_dops = {
+	.create = user_event_create,
+	.show = user_event_show,
+	.is_busy = user_event_is_busy,
+	.free = user_event_free,
+	.match = user_event_match,
+};
+
+/*
+ * Register a trace_event into the system, either find or create.
+ */
+static int register_user_event(char *name, char *args,
+			       struct user_event **newuser)
+{
+	int ret;
+	int index;
+	u32 key = user_event_key(name);
+	struct user_event *user = find_user_event(key, name);
+
+	if (user) {
+		*newuser = user;
+		ret = 0;
+		goto put_name;
+	}
+
+	index = find_first_zero_bit(page_bitmap, MAX_EVENTS);
+
+	if (index == MAX_EVENTS) {
+		ret = -EMFILE;
+		goto put_name;
+	}
+
+	user = kzalloc(sizeof(*user), GFP_KERNEL);
+
+	if (!user) {
+		ret = -ENOMEM;
+		goto put_name;
+	}
+
+	INIT_LIST_HEAD(&user->class.fields);
+
+	user->tracepoint.name = name;
+	user->args = args;
+
+	user->call.data = user;
+	user->call.class = &user->class;
+	user->call.name = name;
+	user->call.flags = TRACE_EVENT_FL_TRACEPOINT;
+	user->call.tp = &user->tracepoint;
+	user->call.event.funcs = &user_event_funcs;
+
+	/* libtraceevent requires atleast double quotes */
+	user->call.print_fmt = "\"\"";
+
+	user->class.system = USER_EVENTS_SYSTEM;
+	user->class.fields_array = user_event_fields_array;
+	user->class.reg = user_event_reg;
+	user->class.probe = user_event_ftrace;
+#ifdef CONFIG_PERF_EVENTS
+	user->class.perf_probe = user_event_perf;
+#endif
+
+	ret = register_trace_event(&user->call.event);
+
+	if (!ret) {
+		ret = -ENODEV;
+		goto put_user;
+	}
+
+	ret = trace_add_event_call(&user->call);
+
+	if (ret) {
+		destroy_user_event(user);
+		goto out;
+	}
+
+	user->index = index;
+	dyn_event_init(&user->devent, &user_event_dops);
+	dyn_event_add(&user->devent);
+	set_bit(user->index, page_bitmap);
+	hash_add(register_table, &user->node, key);
+
+	*newuser = user;
+	return 0;
+put_user:
+	kfree(user);
+put_name:
+	kfree(name);
+out:
+	return ret;
+}
+
+/*
+ * Deletes a previously created event if it is no longer being used.
+ */
+static int delete_user_event(char *name)
+{
+	u32 key = user_event_key(name);
+	struct user_event *user = find_user_event(key, name);
+
+	if (!user)
+		return -ENOENT;
+
+	if (atomic_read(&user->refs) != 0)
+		return -EBUSY;
+
+	return destroy_user_event(user);
+}
+
+/*
+ * Validates the user payload and writes via iterator.
+ */
+static ssize_t user_events_write_core(struct file *file, struct iov_iter *i)
+{
+	struct user_event_refs *refs;
+	struct user_event *user = NULL;
+	struct tracepoint *tp;
+	ssize_t ret = i->count;
+	int idx;
+
+	preempt_disable();
+
+	if (unlikely(!(cpu_online(raw_smp_processor_id()))))
+		goto out;
+
+	if (unlikely(copy_from_iter(&idx, sizeof(idx), i) != sizeof(idx))) {
+		ret = -EFAULT;
+		goto out;
+	}
+
+	idx = INDEX_WRITE(idx);
+
+	refs = rcu_dereference_sched(file->private_data);
+
+	if (likely(refs && idx < refs->count))
+		user = refs->events[idx];
+
+	if (unlikely(user == NULL)) {
+		ret = -ENOENT;
+		goto out;
+	}
+
+	tp = &user->tracepoint;
+
+	if (likely(atomic_read(&tp->key.enabled) > 0)) {
+		struct tracepoint_func *probe_func_ptr;
+		user_event_func_t probe_func;
+		struct iov_iter copy;
+		void *tpdata;
+
+		probe_func_ptr = rcu_dereference_sched(tp->funcs);
+
+		if (probe_func_ptr) {
+			do {
+				/*
+				 * Probes advance the iterator so we
+				 * need to have a copy for each probe.
+				 */
+				copy = *i;
+
+				probe_func = probe_func_ptr->func;
+				tpdata = probe_func_ptr->data;
+				probe_func(user, &copy, tpdata);
+			} while ((++probe_func_ptr)->func);
+		}
+	}
+out:
+	preempt_enable();
+
+	return ret;
+}
+
+static ssize_t user_events_write(struct file *file, const char __user *ubuf,
+				 size_t count, loff_t *ppos)
+{
+	struct iovec iov;
+	struct iov_iter i;
+
+	if (unlikely(*ppos != 0))
+		return -EFAULT;
+
+	if (unlikely(import_single_range(READ, (char *)ubuf, count, &iov, &i)))
+		return -EFAULT;
+
+	return user_events_write_core(file, &i);
+}
+
+static ssize_t user_events_write_iter(struct kiocb *kp, struct iov_iter *i)
+{
+	return user_events_write_core(kp->ki_filp, i);
+}
+
+static int user_events_ref_add(struct file *file, struct user_event *user)
+{
+	struct user_event_refs *refs, *new_refs;
+	int i, size, count = 0;
+
+	refs = rcu_dereference_sched(file->private_data);
+
+	if (refs) {
+		count = refs->count;
+
+		for (i = 0; i < count; ++i)
+			if (refs->events[i] == user)
+				return i;
+	}
+
+	size = sizeof(*refs) + (sizeof(struct user_event *) * (count + 1));
+
+	new_refs = kzalloc(size, GFP_KERNEL);
+
+	if (!new_refs)
+		return -ENOMEM;
+
+	new_refs->count = count + 1;
+
+	for (i = 0; i < count; ++i)
+		new_refs->events[i] = refs->events[i];
+
+	new_refs->events[i] = user;
+
+	atomic_inc(&user->refs);
+
+	rcu_assign_pointer(file->private_data, new_refs);
+
+	if (refs)
+		kfree_rcu(refs, rcu);
+
+	return i;
+}
+
+/*
+ * Handles the ioctl from user mode to register or alter operations.
+ */
+static long user_events_ioctl(struct file *file, unsigned int cmd,
+			      unsigned long uarg)
+{
+	void __user *ubuf = (void __user *)uarg;
+	struct user_event *user;
+	char *name;
+	long ret;
+
+	switch (cmd) {
+	case DIAG_IOCSREG:
+		/* Register/lookup on behalf of user process */
+		name = strndup_user(ubuf, MAX_EVENT_DESC);
+
+		if (IS_ERR(name)) {
+			ret = PTR_ERR(name);
+			goto out;
+		}
+
+		mutex_lock(&event_mutex);
+
+		ret = user_event_parse_cmd(name, &user);
+
+		if (ret < 0)
+			goto reg_out;
+
+		preempt_disable();
+		ret = user_events_ref_add(file, user);
+		preempt_enable();
+
+		if (ret < 0)
+			goto reg_out;
+
+		ret = INDEX_COMBINE(ret, user->index);
+reg_out:
+		mutex_unlock(&event_mutex);
+
+		break;
+
+	case DIAG_IOCSDEL:
+		/* Delete on behalf of user process */
+		name = strndup_user(ubuf, MAX_EVENT_DESC);
+
+		if (IS_ERR(name)) {
+			ret = PTR_ERR(name);
+			goto out;
+		}
+
+		mutex_lock(&event_mutex);
+		ret = delete_user_event(name);
+		mutex_unlock(&event_mutex);
+
+		kfree(name);
+		break;
+
+	default:
+		ret = -ENOTTY;
+		break;
+	}
+out:
+	return ret;
+}
+
+/*
+ * Handles the final close of the file from user mode.
+ */
+static int user_events_release(struct inode *node, struct file *file)
+{
+	struct user_event_refs *refs;
+	struct user_event *user;
+	int i;
+
+	preempt_disable();
+
+	refs = rcu_dereference_sched(file->private_data);
+
+	if (!refs)
+		goto out;
+
+	for (i = 0; i < refs->count; ++i) {
+		user = refs->events[i];
+
+		if (user)
+			atomic_dec(&user->refs);
+	}
+
+	kfree_rcu(refs, rcu);
+out:
+	preempt_enable();
+
+	return 0;
+}
+
+static const struct file_operations user_data_fops = {
+	.write = user_events_write,
+	.write_iter = user_events_write_iter,
+	.unlocked_ioctl	= user_events_ioctl,
+	.release = user_events_release,
+};
+
+/*
+ * Maps the shared page into the user process for checking if event is enabled.
+ */
+static int user_status_mmap(struct file *file, struct vm_area_struct *vma)
+{
+	unsigned long size = vma->vm_end - vma->vm_start;
+
+	if (size != MAX_EVENTS)
+		return -EINVAL;
+
+	return remap_pfn_range(vma, vma->vm_start,
+			       virt_to_phys(register_page_data) >> PAGE_SHIFT,
+			       size, PAGE_READONLY);
+}
+
+static int user_status_show(struct seq_file *m, void *p)
+{
+	struct user_event *user;
+	char status;
+	int i, active = 0, busy = 0;
+
+	mutex_lock(&event_mutex);
+
+	hash_for_each(register_table, i, user, node) {
+		status = register_page_data[user->index];
+
+		seq_printf(m, "%d:%s", user->index, EVENT_NAME(user));
+
+		if (status != 0) {
+			seq_puts(m, " # Used by");
+			if (status & EVENT_STATUS_FTRACE)
+				seq_puts(m, " ftrace");
+			if (status & EVENT_STATUS_PERF)
+				seq_puts(m, " perf");
+			if (status & EVENT_STATUS_OTHER)
+				seq_puts(m, " other");
+			busy++;
+		}
+
+		seq_puts(m, "\n");
+		active++;
+	}
+
+	mutex_unlock(&event_mutex);
+
+	seq_puts(m, "\n");
+	seq_printf(m, "Active: %d\n", active);
+	seq_printf(m, "Busy: %d\n", busy);
+	seq_printf(m, "Max: %ld\n", MAX_EVENTS);
+
+	return 0;
+}
+
+static ssize_t user_status_read(struct file *file, char __user *ubuf,
+				size_t count, loff_t *ppos)
+{
+	/*
+	 * Delay allocation of seq data until requested, most callers
+	 * will never read the status file. They will only mmap.
+	 */
+	if (file->private_data == NULL) {
+		int ret;
+
+		if (*ppos != 0)
+			return -EINVAL;
+
+		ret = single_open(file, user_status_show, NULL);
+
+		if (ret)
+			return ret;
+	}
+
+	return seq_read(file, ubuf, count, ppos);
+}
+
+static loff_t user_status_seek(struct file *file, loff_t offset, int whence)
+{
+	if (file->private_data == NULL)
+		return 0;
+
+	return seq_lseek(file, offset, whence);
+}
+
+static int user_status_release(struct inode *node, struct file *file)
+{
+	if (file->private_data == NULL)
+		return 0;
+
+	return single_release(node, file);
+}
+
+static const struct file_operations user_status_fops = {
+	.mmap = user_status_mmap,
+	.read = user_status_read,
+	.llseek  = user_status_seek,
+	.release = user_status_release,
+};
+
+/*
+ * Creates a set of tracefs files to allow user mode interactions.
+ */
+static int create_user_tracefs(void)
+{
+	struct dentry *edata, *emmap;
+
+	edata = tracefs_create_file("user_events_data", 0644, NULL,
+				    NULL, &user_data_fops);
+
+	if (!edata) {
+		pr_warn("Could not create tracefs 'user_events_data' entry\n");
+		goto err;
+	}
+
+	/* mmap with MAP_SHARED requires writable fd */
+	emmap = tracefs_create_file("user_events_status", 0644, NULL,
+				    NULL, &user_status_fops);
+
+	if (!emmap) {
+		tracefs_remove(edata);
+		pr_warn("Could not create tracefs 'user_events_mmap' entry\n");
+		goto err;
+	}
+
+	return 0;
+err:
+	return -ENODEV;
+}
+
+static void set_page_reservations(bool set)
+{
+	int page;
+
+	for (page = 0; page < MAX_PAGES; ++page) {
+		void *addr = register_page_data + (PAGE_SIZE * page);
+
+		if (set)
+			SetPageReserved(virt_to_page(addr));
+		else
+			ClearPageReserved(virt_to_page(addr));
+	}
+}
+
+static int __init trace_events_user_init(void)
+{
+	int ret;
+
+	/* Zero all bits beside 0 (which is reserved for failures) */
+	bitmap_zero(page_bitmap, MAX_EVENTS);
+	set_bit(0, page_bitmap);
+
+	register_page_data = kzalloc(MAX_EVENTS, GFP_KERNEL);
+
+	if (!register_page_data)
+		return -ENOMEM;
+
+	set_page_reservations(true);
+
+	ret = create_user_tracefs();
+
+	if (ret) {
+		pr_warn("user_events could not register with tracefs\n");
+		set_page_reservations(false);
+		kfree(register_page_data);
+		return ret;
+	}
+
+	if (dyn_event_register(&user_event_dops))
+		pr_warn("user_events could not register with dyn_events\n");
+
+	return 0;
+}
+
+fs_initcall(trace_events_user_init);
-- 
2.17.1


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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
  2021-10-18 23:09 [PATCH v3] user_events: Enable user processes to create and write to trace events Beau Belgrave
@ 2021-10-22 13:38 ` Masami Hiramatsu
  2021-10-22 22:42   ` Beau Belgrave
  2021-10-26  0:46   ` kernel test robot
  1 sibling, 1 reply; 9+ messages in thread
From: Masami Hiramatsu @ 2021-10-22 13:38 UTC (permalink / raw)
  To: Beau Belgrave; +Cc: rostedt, linux-trace-devel, linux-kernel

Hi Beau,

On Mon, 18 Oct 2021 16:09:57 -0700
Beau Belgrave <beaub@linux.microsoft.com> wrote:

> User mode processes that wish to use trace events to get data into
> ftrace, perf, eBPF, etc are limited to uprobes today. The user events
> features enables an ABI for user mode processes to create and write to
> trace events that are isolated from kernel level trace events. This
> enables a faster path for tracing from user mode data as well as opens
> managed code to participate in trace events, where stub locations are
> dynamic.
> 
> User processes often want to trace only when it's useful. To enable this
> a set of pages are mapped into the user process space that indicate the
> current state of the user events that have been registered. User
> processes can check if their event is hooked to a trace/probe, and if it
> is, emit the event data out via the write() syscall.
> 
> Two new files are introduced into tracefs to accomplish this:
> user_events_status - This file is mmap'd into participating user mode
> processes to indicate event status.
> 
> user_events_data - This file is opened and register/delete ioctl's are
> issued to create/open/delete trace events that can be used for tracing.
> 
> The typical scenario is on process start to mmap user_events_status. Processes
> then register the events they plan to use via the REG ioctl. The return value
> of the ioctl indicates the byte in the mmap to use for status and the index
> to use for writing. Both are 16-bits in size and are accessible via the
> INDEX_STATUS and INDEX_WRITE macros within user_events.h.

Hmm, if we uses the ioctl(), the return value must be int because it can
return -1 and kernel return value is stored into the errno if there is
any error. I think you can introduce a data structure to return the
id and offset. (Or make id 15-bits :))  

> The fd that was
> used for the ioctl can now be used to write out data. The data must always
> include an int describing which event the data is for at the start of the
> payload.
> 
> For example, in memory:
> int index;
> char data[];
> 
> Psuedo code example of typical usage:
> int page_fd = open("user_events_status", O_RDWR);
> char *page_data = mmap(NULL, PAGE_SIZE, PROT_READ, MAP_SHARED, page_fd, 0);
> 
> int data_fd = open("user_events_data", O_RDWR);
> long result = ioctl(data_fd, DIAG_IOCSREG, "test");
> int status_id = INDEX_STATUS(result);
> int write_id = INDEX_WRITE(result);
> 
> struct iovec io[2];
> io[0].iov_base = &write_id;
> io[0].iov_len = sizeof(write_id);
> io[1].iov_base = payload;
> io[1].iov_len = sizeof(payload);
> 
> if (page_data[status_id])
> 	writev(data_fd, io, 2);

OK, so you switch to the 1:n model. Using writev() is a good idea.
(BTW, what happen if I use write()?)

> 
> User events are also exposed via the dynamic_events tracefs file for
> both create and delete. Current status is exposed via the user_events_status
> tracefs file.
> 
> Simple example to register a user event via dynamic_events:
> 	echo u:test >> dynamic_events
> 	cat dynamic_events
> 	u:test
> 
> If an event is hooked to a probe, the probe hooked shows up:
> 	echo 1 > events/user_events/test/enable
> 	cat user_events_status
> 	1:test # Used by ftrace
> 
> 	Active: 1
> 	Busy: 1
> 	Max: 4096
> 
> If an event is not hooked to a probe, no probe status shows up:
> 	echo 0 > events/user_events/test/enable
> 	cat user_events_status
> 	1:test
> 
> 	Active: 1
> 	Busy: 0
> 	Max: 4096

OK, this looks good to me.

> 
> Users can describe the trace event format via the following format:
> 	name [field1;field2]

Does this mean it always need 2 fields?

> 
> Each field has the following format:
> 	type name size

Why do you need the "size"? Since "type" must have the size information,
it is not needed. (or, user can define 1 byte "int" type...)

> 
> Example for char array with a size of 20 named msg:
> 	echo 'u:detailed char[20] msg 20' >> dynamic_events
> 	cat dynamic_events
> 	u:detailed char[20] msg
> 
> Data offsets are based on the data written out via write() and will be
> updated to reflect the correct offset in the trace_event fields. __data_loc
> types must be aware of the size of trace_entry/common properties to ensure
> proper decoding.

As I said that the you have to add __rel_loc type (yes, you can do it)
so that the user don't need to consider the data location. That new type
decodeer code may need to be implemented afterwards in user-space tools
but anyway, that can be postponed.

OK, let me review the code.

[...]
> ---
>  include/uapi/linux/user_events.h |   57 ++
>  kernel/trace/Kconfig             |   15 +
>  kernel/trace/Makefile            |    1 +
>  kernel/trace/trace_events_user.c | 1040 ++++++++++++++++++++++++++++++
>  4 files changed, 1113 insertions(+)
>  create mode 100644 include/uapi/linux/user_events.h
>  create mode 100644 kernel/trace/trace_events_user.c

BTW, if you can split this patch into several patches, it is easier to us
to review it. E.g. basic support for ftrace, add perf support, add bpf support,
documentation, optimize performance etc.

> 
> diff --git a/include/uapi/linux/user_events.h b/include/uapi/linux/user_events.h
> new file mode 100644
> index 000000000000..00122624ff26
> --- /dev/null
> +++ b/include/uapi/linux/user_events.h
> @@ -0,0 +1,57 @@
> +/* SPDX-License-Identifier: GPL-2.0 WITH Linux-syscall-note */
> +/*
> + * Copyright (c) 2021, Microsoft Corporation.
> + *
> + * Authors:
> + *   Beau Belgrave <beaub@linux.microsoft.com>
> + */
> +#ifndef _UAPI_LINUX_USER_EVENTS_H
> +#define _UAPI_LINUX_USER_EVENTS_H
> +
> +#include <linux/types.h>
> +#include <linux/ioctl.h>
> +#include <linux/uio.h>
> +
> +#define USER_EVENTS_SYSTEM "user_events"
> +#define USER_EVENTS_PREFIX "u:"
> +
> +/* Bits 0-6 are for known probe types, Bit 7 is for unknown probes */
> +#define EVENT_BIT_FTRACE 0
> +#define EVENT_BIT_PERF 1
> +#define EVENT_BIT_OTHER 7
> +
> +#define EVENT_STATUS_FTRACE (1 << EVENT_BIT_FTRACE)
> +#define EVENT_STATUS_PERF (1 << EVENT_BIT_PERF)
> +#define EVENT_STATUS_OTHER (1 << EVENT_BIT_OTHER)
> +
> +#define DIAG_IOC_MAGIC '*'
> +#define DIAG_IOCSREG _IOW(DIAG_IOC_MAGIC, 0, char*)
> +#define DIAG_IOCSDEL _IOW(DIAG_IOC_MAGIC, 1, char*)
> +
> +#define INDEX_WRITE(index) (index & 0xFFFF)
> +#define INDEX_STATUS(index) ((index >> 16) & 0xFFFF)
> +#define INDEX_COMBINE(write, status) (status << 16 | write)
> +
> +enum {
> +	USER_BPF_DATA_KERNEL,
> +	USER_BPF_DATA_USER,
> +	USER_BPF_DATA_ITER,
> +};
> +
> +struct user_bpf_iter {
> +	size_t iov_offset;
> +	const struct iovec *iov;
> +	unsigned long nr_segs;
> +};
> +
> +struct user_bpf_context {
> +	int data_type;
> +	int data_len;
> +	union {
> +		void *kdata;
> +		void *udata;
> +		struct user_bpf_iter *iter;
> +	};
> +};
> +
> +#endif /* _UAPI_LINUX_USER_EVENTS_H */
> diff --git a/kernel/trace/Kconfig b/kernel/trace/Kconfig
> index 3ee23f4d437f..deaaad421be4 100644
> --- a/kernel/trace/Kconfig
> +++ b/kernel/trace/Kconfig
> @@ -725,6 +725,21 @@ config SYNTH_EVENTS
>  
>  	  If in doubt, say N.
>  
> +config USER_EVENTS
> +	bool "User trace events"
> +	select TRACING
> +	select DYNAMIC_EVENTS
> +	default n
> +	help
> +	  User trace events are user-defined trace events that
> +	  can be used like an existing kernel trace event.  User trace
> +	  events are generated by writing to a tracefs file.  User
> +	  processes can determine if their tracing events should be
> +	  generated by memory mapping a tracefs file and checking for
> +	  an associated byte being non-zero.
> +
> +	  If in doubt, say N.
> +
>  config HIST_TRIGGERS
>  	bool "Histogram triggers"
>  	depends on ARCH_HAVE_NMI_SAFE_CMPXCHG
> diff --git a/kernel/trace/Makefile b/kernel/trace/Makefile
> index b1c47ccf4f73..a653b255e89c 100644
> --- a/kernel/trace/Makefile
> +++ b/kernel/trace/Makefile
> @@ -80,6 +80,7 @@ obj-$(CONFIG_EVENT_TRACING) += trace_events_trigger.o
>  obj-$(CONFIG_TRACE_EVENT_INJECT) += trace_events_inject.o
>  obj-$(CONFIG_SYNTH_EVENTS) += trace_events_synth.o
>  obj-$(CONFIG_HIST_TRIGGERS) += trace_events_hist.o
> +obj-$(CONFIG_USER_EVENTS) += trace_events_user.o
>  obj-$(CONFIG_BPF_EVENTS) += bpf_trace.o
>  obj-$(CONFIG_KPROBE_EVENTS) += trace_kprobe.o
>  obj-$(CONFIG_TRACEPOINTS) += error_report-traces.o
> diff --git a/kernel/trace/trace_events_user.c b/kernel/trace/trace_events_user.c
> new file mode 100644
> index 000000000000..4a90d383bab7
> --- /dev/null
> +++ b/kernel/trace/trace_events_user.c
> @@ -0,0 +1,1040 @@
> +// SPDX-License-Identifier: GPL-2.0-only
> +/*
> + * Copyright (c) 2021, Microsoft Corporation.
> + *
> + * Authors:
> + *   Beau Belgrave <beaub@linux.microsoft.com>
> + */
> +
> +#include <linux/bitmap.h>
> +#include <linux/cdev.h>
> +#include <linux/hashtable.h>
> +#include <linux/io.h>
> +#include <linux/uio.h>
> +#include <linux/ioctl.h>
> +#include <linux/jhash.h>
> +#include <linux/trace_events.h>
> +#include <linux/tracefs.h>
> +#include <linux/types.h>
> +#include <linux/uaccess.h>
> +#include <uapi/linux/user_events.h>
> +#include "trace.h"
> +#include "trace_dynevent.h"
> +
> +#define USER_EVENTS_PREFIX_LEN (sizeof(USER_EVENTS_PREFIX)-1)
> +
> +#define FIELD_DEPTH_TYPE 0
> +#define FIELD_DEPTH_NAME 1
> +#define FIELD_DEPTH_SIZE 2
> +
> +/*
> + * Limits how many trace_event calls user processes can create:
> + * Must be multiple of PAGE_SIZE.
> + */
> +#define MAX_PAGES 1
> +#define MAX_EVENTS (MAX_PAGES * PAGE_SIZE)
> +#define MAX_BPF_COPY_SIZE PAGE_SIZE
> +#define MAX_STACK_BPF_DATA 512
> +
> +/* Limit how long of an event name plus args within the subsystem. */
> +#define MAX_EVENT_DESC 512
> +#define EVENT_NAME(user_event) ((user_event)->tracepoint.name)
> +
> +#define FLAG_BPF_ITER 1
> +
> +static char *register_page_data;
> +
> +static DEFINE_HASHTABLE(register_table, 4);
> +static DECLARE_BITMAP(page_bitmap, MAX_EVENTS);
> +
> +struct user_event {
> +	struct tracepoint tracepoint;
> +	struct trace_event_call call;
> +	struct trace_event_class class;
> +	struct dyn_event devent;
> +	struct hlist_node node;
> +	atomic_t refs;
> +	int index;
> +	char *args;
> +	int flags;
> +};
> +
> +struct user_event_refs {
> +	struct rcu_head rcu;
> +	int count;
> +	struct user_event *events[];
> +};
> +
> +typedef void (*user_event_func_t) (struct user_event *user,
> +				   struct iov_iter *i,
> +				   void *tpdata);
> +
> +static int register_user_event(char *name, char *args,
> +			       struct user_event **newuser);
> +
> +/*
> + * Parses a register command for user_events
> + * Format: event_name [field1;field2;...]
> + *
> + * Example event named test with a 20 char msg field with a unsigned int after:
> + * test char[20] msg 20;unsigned int id 4;
> + *
> + * NOTE: Offsets are from the user data perspective, they are not from the
> + * trace_entry/buffer perspective. We automatically add the common properties
> + * sizes to the offset for the user.
> + */
> +static int user_event_parse_cmd(char *raw_command, struct user_event **newuser)
> +{
> +	char *name = raw_command;
> +	char *args = strpbrk(name, " ");
> +
> +	if (args)
> +		*args++ = 0;
> +
> +	return register_user_event(name, args, newuser);
> +}
> +
> +/*
> + * Parses the values of a field within the description
> + * Format: type name size [future]
> + */

As I said that the size is non-sense. And this should finally unified
with synthetic event field definition. 

> +static int user_event_parse_field(char *field, struct user_event *user,
> +				  u32 *offset)
> +{
> +	char *part, *type, *name;
> +	u32 size = 0, depth = 0, saved_offset = *offset;
> +
> +	field = skip_spaces(field);
> +
> +	if (strstr(field, "unsigned ") == field) {
> +		type = field;
> +		field = strpbrk(field + sizeof("unsigned"), " ");
> +
> +		if (field == NULL)
> +			return -EINVAL;
> +
> +		*field++ = 0;
> +		depth++;
> +	} else if (strstr(field, "print_fmt ") == field) {
> +		user->call.print_fmt = field + sizeof("print_fmt");
> +		return 0;

What is this ?

> +	} else if (strstr(field, "flag ") == field) {
> +		field += sizeof("flag");
> +
> +		if (!strcmp(field, "bpf_iter"))
> +			user->flags |= FLAG_BPF_ITER;
> +

What is this flag?

> +		return 0;
> +	}
> +
> +	while ((part = strsep(&field, " ")) != NULL) {
> +		switch (depth++) {
> +		case FIELD_DEPTH_TYPE:
> +			type = part;
> +			break;
> +		case FIELD_DEPTH_NAME:
> +			name = part;
> +			break;
> +		case FIELD_DEPTH_SIZE:
> +			if (kstrtou32(part, 10, &size))
> +				return -EINVAL;
> +			break;
> +		default:
> +			/* Forward compatibility, ignore */
> +			goto end;
> +		}
> +	}
> +end:
> +	if (depth < FIELD_DEPTH_SIZE)
> +		return -EINVAL;
> +
> +	*offset = saved_offset + size;
> +
> +	return trace_define_field(&user->call, type, name, saved_offset, size,
> +				  type[0] != 'u', FILTER_OTHER);
> +}
> +
> +/*
> + * Parses the fields that were described for the event
> + */
> +static int user_event_parse_fields(struct user_event *user)
> +{
> +	char *field;
> +	u32 offset = sizeof(struct trace_entry);
> +	int ret = -EINVAL;
> +
> +	while ((field = strsep(&user->args, ";")) != NULL) {
> +		ret = user_event_parse_field(field, user, &offset);
> +
> +		if (ret)
> +			break;
> +	}
> +
> +	return ret;
> +}
> +
> +static int user_event_define_fields(struct trace_event_call *call)
> +{
> +	struct user_event *user = (struct user_event *)call->data;
> +
> +	/* User chose to not disclose arguments */
> +	if (user->args == NULL)
> +		return 0;
> +
> +	return user_event_parse_fields(user);
> +}
> +
> +static struct trace_event_fields user_event_fields_array[] = {
> +	{ .type = TRACE_FUNCTION_TYPE,
> +	  .define_fields = user_event_define_fields },
> +	{}
> +};
> +
> +static enum print_line_t user_event_print_trace(struct trace_iterator *iter,
> +						int flags,
> +						struct trace_event *event)
> +{
> +	/* Unsafe to try to decode user provided print_fmt, use hex */
> +	trace_print_hex_dump_seq(&iter->seq, "", DUMP_PREFIX_OFFSET, 16,
> +				 1, iter->ent, iter->ent_size, true);
> +
> +	return trace_handle_return(&iter->seq);
> +}
> +
> +static struct trace_event_functions user_event_funcs = {
> +	.trace = user_event_print_trace,
> +};
> +
> +static int destroy_user_event(struct user_event *user)
> +{
> +	int ret = 0;
> +
> +	/*
> +	 * trace_remove_event_call invokes unregister_trace_event:
> +	 * Pick the correct one based on if we set the data or not
> +	 */
> +	if (user->index != 0) {
> +		ret = trace_remove_event_call(&user->call);
> +
> +		if (ret)
> +			return ret;
> +
> +		dyn_event_remove(&user->devent);
> +
> +		register_page_data[user->index] = 0;
> +		clear_bit(user->index, page_bitmap);
> +		hash_del(&user->node);
> +	} else {
> +		unregister_trace_event(&user->call.event);
> +	}
> +
> +	kfree(EVENT_NAME(user));
> +	kfree(user);
> +
> +	return ret;
> +}
> +
> +static struct user_event *find_user_event(u32 key, char *name)
> +{
> +	struct user_event *user;
> +
> +	hash_for_each_possible(register_table, user, node, key)
> +		if (!strcmp(EVENT_NAME(user), name))
> +			return user;
> +
> +	return NULL;
> +}
> +
> +/*
> + * Writes the user supplied payload out to a trace file.
> + */
> +static void user_event_ftrace(struct user_event *user, struct iov_iter *i,
> +			      void *tpdata)
> +{
> +	struct trace_event_file *file;
> +	struct trace_entry *entry;
> +	struct trace_event_buffer event_buffer;
> +	size_t count = i->count;
> +
> +	file = (struct trace_event_file *)tpdata;
> +
> +	if (!file ||
> +	    !(file->flags & EVENT_FILE_FL_ENABLED) ||
> +	    trace_trigger_soft_disabled(file))
> +		return;
> +
> +	entry = trace_event_buffer_reserve(&event_buffer, file,
> +					   sizeof(*entry) + count);
> +
> +	if (unlikely(!entry))
> +		return;
> +
> +	if (unlikely(copy_from_iter(entry + 1, count, i) != count))
> +		return;
> +
> +	trace_event_buffer_commit(&event_buffer);
> +}
> +
> +#ifdef CONFIG_PERF_EVENTS
> +/*
> + * Writes the user supplied payload out to perf ring buffer or eBPF program.
> + */
> +static void user_event_perf(struct user_event *user, struct iov_iter *i,
> +			    void *tpdata)
> +{
> +	struct hlist_head *perf_head;
> +	size_t count = i->count;
> +
> +	if (bpf_prog_array_valid(&user->call)) {
> +		struct user_bpf_context context = {0};
> +		struct user_bpf_iter bpf_i = {0};
> +		char fast_data[MAX_STACK_BPF_DATA];
> +		void *temp = NULL;
> +
> +		context.data_len = count;
> +
> +		if ((user->flags & FLAG_BPF_ITER) && iter_is_iovec(i)) {
> +			/* Raw iovec to BPF, no copies */
> +			bpf_i.iov_offset = i->iov_offset;
> +			bpf_i.iov = i->iov;
> +			bpf_i.nr_segs = i->nr_segs;
> +
> +			context.data_type = USER_BPF_DATA_ITER;
> +			context.iter = &bpf_i;
> +		} else if (i->nr_segs == 1 && iter_is_iovec(i)) {
> +			/* Single vector, no copies */
> +			context.data_type = USER_BPF_DATA_USER;
> +			context.udata = i->iov->iov_base + i->iov_offset;
> +		} else {
> +			/* Multi vector, copy on stack or heap */
> +			struct iov_iter copy;
> +			size_t copy_size = count;
> +
> +			context.data_type = USER_BPF_DATA_KERNEL;
> +
> +			if (copy_size > MAX_BPF_COPY_SIZE)
> +				copy_size = MAX_BPF_COPY_SIZE;
> +
> +			if (copy_size <= sizeof(fast_data))
> +				context.kdata = fast_data;
> +			else {
> +				temp = kmalloc(copy_size, GFP_KERNEL);
> +				context.kdata = temp;
> +			}
> +
> +			/* Must keep in case perf_events is hooked as well */
> +			copy = *i;
> +
> +			context.data_len = copy_from_iter(context.kdata,
> +							  copy_size, &copy);
> +		}
> +
> +		trace_call_bpf(&user->call, &context);
> +
> +		kfree(temp);
> +	}
> +
> +	perf_head = this_cpu_ptr(user->call.perf_events);
> +
> +	if (perf_head && !hlist_empty(perf_head)) {
> +		struct trace_entry *perf_entry;
> +		struct pt_regs *regs;
> +		size_t size = sizeof(*perf_entry) + count;
> +		int context;
> +
> +		perf_entry = perf_trace_buf_alloc(ALIGN(size, 8),
> +						  &regs, &context);
> +
> +		if (unlikely(!perf_entry))
> +			return;
> +
> +		perf_fetch_caller_regs(regs);
> +
> +		if (unlikely(copy_from_iter(perf_entry + 1, count, i) != count))
> +			return;
> +
> +		perf_trace_buf_submit(perf_entry, size, context,
> +				      user->call.event.type, 1, regs,
> +				      perf_head, NULL);
> +	}
> +}
> +#endif
> +
> +/*
> + * Update the register page that is shared between user processes.
> + */
> +static void update_reg_page_for(struct user_event *user)
> +{
> +	struct tracepoint *tp = &user->tracepoint;
> +	char status = 0;
> +
> +	if (atomic_read(&tp->key.enabled) > 0) {
> +		struct tracepoint_func *probe_func_ptr;
> +		user_event_func_t probe_func;
> +
> +		probe_func_ptr = rcu_dereference_sched(tp->funcs);
> +
> +		if (probe_func_ptr) {
> +			do {
> +				probe_func = probe_func_ptr->func;
> +
> +				if (probe_func == user_event_ftrace)
> +					status |= EVENT_STATUS_FTRACE;
> +#ifdef CONFIG_PERF_EVENTS
> +				else if (probe_func == user_event_perf)
> +					status |= EVENT_STATUS_PERF;
> +#endif
> +				else
> +					status |= EVENT_STATUS_OTHER;
> +			} while ((++probe_func_ptr)->func);
> +		}
> +	}
> +
> +	register_page_data[user->index] = status;
> +}
> +
> +/*
> + * Register callback for our events from tracing sub-systems.
> + */
> +static int user_event_reg(struct trace_event_call *call,
> +			  enum trace_reg type,
> +			  void *data)
> +{
> +	struct user_event *user = (struct user_event *)call->data;
> +	int ret = 0;
> +
> +	if (!user)
> +		return -ENOENT;
> +
> +	switch (type) {
> +	case TRACE_REG_REGISTER:
> +		ret = tracepoint_probe_register(call->tp,
> +						call->class->probe,
> +						data);
> +		if (!ret)
> +			goto inc;
> +		break;
> +
> +	case TRACE_REG_UNREGISTER:
> +		tracepoint_probe_unregister(call->tp,
> +					    call->class->probe,
> +					    data);
> +		goto dec;
> +
> +#ifdef CONFIG_PERF_EVENTS
> +	case TRACE_REG_PERF_REGISTER:
> +		ret = tracepoint_probe_register(call->tp,
> +						call->class->perf_probe,
> +						data);
> +		if (!ret)
> +			goto inc;
> +		break;
> +
> +	case TRACE_REG_PERF_UNREGISTER:
> +		tracepoint_probe_unregister(call->tp,
> +					    call->class->perf_probe,
> +					    data);
> +		goto dec;
> +
> +	case TRACE_REG_PERF_OPEN:
> +	case TRACE_REG_PERF_CLOSE:
> +	case TRACE_REG_PERF_ADD:
> +	case TRACE_REG_PERF_DEL:
> +		break;
> +#endif
> +	}
> +
> +	return ret;
> +inc:
> +	atomic_inc(&user->refs);
> +	update_reg_page_for(user);
> +	return 0;
> +dec:
> +	update_reg_page_for(user);
> +	atomic_dec(&user->refs);
> +	return 0;
> +}
> +
> +static u32 user_event_key(char *name)
> +{
> +	return jhash(name, strlen(name), 0);
> +}
> +
> +static int user_event_create(const char *raw_command)
> +{
> +	struct user_event *user;
> +	char *name;
> +	int ret;
> +
> +	if (strstr(raw_command, USER_EVENTS_PREFIX) != raw_command)
> +		return -ECANCELED;
> +
> +	raw_command += USER_EVENTS_PREFIX_LEN;
> +	raw_command = skip_spaces(raw_command);
> +
> +	name = kstrdup(raw_command, GFP_KERNEL);
> +
> +	if (!name)
> +		return -ENOMEM;
> +
> +	mutex_lock(&event_mutex);
> +	ret = user_event_parse_cmd(name, &user);

I recommend you to split "parse" and "register" function, and
locks event_mutex only in the "register" function, since parsing
doesn't need to hold other events.

> +	mutex_unlock(&event_mutex);
> +
> +	return ret;
> +}
> +
> +static int user_event_show(struct seq_file *m, struct dyn_event *ev)
> +{
> +	struct user_event *user = container_of(ev, struct user_event, devent);
> +	struct ftrace_event_field *field, *next;
> +	struct list_head *head;
> +	int depth = 0;
> +
> +	seq_printf(m, "%s%s", USER_EVENTS_PREFIX, EVENT_NAME(user));
> +
> +	head = trace_get_fields(&user->call);
> +
> +	list_for_each_entry_safe_reverse(field, next, head, link) {
> +		if (depth == 0)
> +			seq_puts(m, " ");
> +		else
> +			seq_puts(m, "; ");
> +		seq_printf(m, "%s %s", field->type, field->name);
> +		depth++;
> +	}
> +
> +	seq_puts(m, "\n");
> +
> +	return 0;
> +}
> +
> +static bool user_event_is_busy(struct dyn_event *ev)
> +{
> +	struct user_event *user = container_of(ev, struct user_event, devent);
> +
> +	return atomic_read(&user->refs) != 0;
> +}
> +
> +static int user_event_free(struct dyn_event *ev)
> +{
> +	struct user_event *user = container_of(ev, struct user_event, devent);
> +
> +	return destroy_user_event(user);
> +}
> +
> +static bool user_event_match(const char *system, const char *event,
> +			     int argc, const char **argv, struct dyn_event *ev)
> +{
> +	struct user_event *user = container_of(ev, struct user_event, devent);
> +
> +	return strcmp(EVENT_NAME(user), event) == 0 &&
> +		(!system || strcmp(system, USER_EVENTS_SYSTEM) == 0);

For the minimal implementation, this is OK.

Note that it is better to check the rest of arguments with the event
parameters. E.g. if you have "myuevent int var", user cant try to remove
"-:myuevent char var".

> +}
> +
> +static struct dyn_event_operations user_event_dops = {
> +	.create = user_event_create,
> +	.show = user_event_show,
> +	.is_busy = user_event_is_busy,
> +	.free = user_event_free,
> +	.match = user_event_match,
> +};
> +
> +/*
> + * Register a trace_event into the system, either find or create.
> + */
> +static int register_user_event(char *name, char *args,
> +			       struct user_event **newuser)
> +{
> +	int ret;
> +	int index;
> +	u32 key = user_event_key(name);
> +	struct user_event *user = find_user_event(key, name);
> +
> +	if (user) {
> +		*newuser = user;
> +		ret = 0;
> +		goto put_name;
> +	}
> +
> +	index = find_first_zero_bit(page_bitmap, MAX_EVENTS);
> +
> +	if (index == MAX_EVENTS) {
> +		ret = -EMFILE;
> +		goto put_name;
> +	}
> +
> +	user = kzalloc(sizeof(*user), GFP_KERNEL);
> +
> +	if (!user) {
> +		ret = -ENOMEM;
> +		goto put_name;
> +	}
> +
> +	INIT_LIST_HEAD(&user->class.fields);
> +
> +	user->tracepoint.name = name;
> +	user->args = args;
> +
> +	user->call.data = user;
> +	user->call.class = &user->class;
> +	user->call.name = name;
> +	user->call.flags = TRACE_EVENT_FL_TRACEPOINT;
> +	user->call.tp = &user->tracepoint;
> +	user->call.event.funcs = &user_event_funcs;
> +
> +	/* libtraceevent requires atleast double quotes */
> +	user->call.print_fmt = "\"\"";

I strongly recommend you to generate the default print_fmt.
e.g. traceprobe_set_print_fmt()@kernel/trace/trace_probe.c
But anyway, for the minimal implementation, this is OK.


> +
> +	user->class.system = USER_EVENTS_SYSTEM;
> +	user->class.fields_array = user_event_fields_array;
> +	user->class.reg = user_event_reg;
> +	user->class.probe = user_event_ftrace;
> +#ifdef CONFIG_PERF_EVENTS
> +	user->class.perf_probe = user_event_perf;
> +#endif
> +
> +	ret = register_trace_event(&user->call.event);
> +
> +	if (!ret) {
> +		ret = -ENODEV;
> +		goto put_user;
> +	}
> +
> +	ret = trace_add_event_call(&user->call);
> +
> +	if (ret) {
> +		destroy_user_event(user);
> +		goto out;
> +	}
> +
> +	user->index = index;
> +	dyn_event_init(&user->devent, &user_event_dops);
> +	dyn_event_add(&user->devent);
> +	set_bit(user->index, page_bitmap);
> +	hash_add(register_table, &user->node, key);
> +
> +	*newuser = user;
> +	return 0;
> +put_user:
> +	kfree(user);
> +put_name:
> +	kfree(name);
> +out:
> +	return ret;
> +}
> +
> +/*
> + * Deletes a previously created event if it is no longer being used.
> + */
> +static int delete_user_event(char *name)
> +{
> +	u32 key = user_event_key(name);
> +	struct user_event *user = find_user_event(key, name);
> +
> +	if (!user)
> +		return -ENOENT;
> +
> +	if (atomic_read(&user->refs) != 0)
> +		return -EBUSY;
> +
> +	return destroy_user_event(user);
> +}
> +
> +/*
> + * Validates the user payload and writes via iterator.
> + */
> +static ssize_t user_events_write_core(struct file *file, struct iov_iter *i)
> +{
> +	struct user_event_refs *refs;
> +	struct user_event *user = NULL;
> +	struct tracepoint *tp;
> +	ssize_t ret = i->count;
> +	int idx;
> +
> +	preempt_disable();

Why do you disable preemption here?

> +
> +	if (unlikely(!(cpu_online(raw_smp_processor_id()))))
> +		goto out;

This seems meaningless because this function is called in the
user process context. If the cpu is going to offline, the process
itself will be migrated before reaching here.

> +
> +	if (unlikely(copy_from_iter(&idx, sizeof(idx), i) != sizeof(idx))) {
> +		ret = -EFAULT;
> +		goto out;
> +	}

Isn't this copy_from_iter() sleepable function?

> +
> +	idx = INDEX_WRITE(idx);
> +
> +	refs = rcu_dereference_sched(file->private_data);
> +
> +	if (likely(refs && idx < refs->count))
> +		user = refs->events[idx];
> +
> +	if (unlikely(user == NULL)) {
> +		ret = -ENOENT;
> +		goto out;
> +	}
> +
> +	tp = &user->tracepoint;
> +
> +	if (likely(atomic_read(&tp->key.enabled) > 0)) {
> +		struct tracepoint_func *probe_func_ptr;
> +		user_event_func_t probe_func;
> +		struct iov_iter copy;
> +		void *tpdata;
> +
> +		probe_func_ptr = rcu_dereference_sched(tp->funcs);
> +
> +		if (probe_func_ptr) {
> +			do {
> +				/*
> +				 * Probes advance the iterator so we
> +				 * need to have a copy for each probe.
> +				 */
> +				copy = *i;
> +
> +				probe_func = probe_func_ptr->func;
> +				tpdata = probe_func_ptr->data;
> +				probe_func(user, &copy, tpdata);

You seems to try to copy in from user space in each probe func, but
please copy it here to the temporary buffer and pass it to the
each probe function. Such performacne optimization can postpone.
Start with simple implementation.

> +			} while ((++probe_func_ptr)->func);
> +		}
> +	}
> +out:
> +	preempt_enable();
> +
> +	return ret;
> +}
> +
> +static ssize_t user_events_write(struct file *file, const char __user *ubuf,
> +				 size_t count, loff_t *ppos)
> +{
> +	struct iovec iov;
> +	struct iov_iter i;
> +
> +	if (unlikely(*ppos != 0))
> +		return -EFAULT;
> +
> +	if (unlikely(import_single_range(READ, (char *)ubuf, count, &iov, &i)))
> +		return -EFAULT;
> +
> +	return user_events_write_core(file, &i);
> +}
> +
> +static ssize_t user_events_write_iter(struct kiocb *kp, struct iov_iter *i)
> +{
> +	return user_events_write_core(kp->ki_filp, i);
> +}
> +
> +static int user_events_ref_add(struct file *file, struct user_event *user)
> +{
> +	struct user_event_refs *refs, *new_refs;
> +	int i, size, count = 0;
> +
> +	refs = rcu_dereference_sched(file->private_data);
> +
> +	if (refs) {
> +		count = refs->count;
> +
> +		for (i = 0; i < count; ++i)
> +			if (refs->events[i] == user)
> +				return i;
> +	}
> +
> +	size = sizeof(*refs) + (sizeof(struct user_event *) * (count + 1));
> +
> +	new_refs = kzalloc(size, GFP_KERNEL);
> +
> +	if (!new_refs)
> +		return -ENOMEM;
> +
> +	new_refs->count = count + 1;
> +
> +	for (i = 0; i < count; ++i)
> +		new_refs->events[i] = refs->events[i];
> +
> +	new_refs->events[i] = user;
> +
> +	atomic_inc(&user->refs);
> +
> +	rcu_assign_pointer(file->private_data, new_refs);
> +
> +	if (refs)
> +		kfree_rcu(refs, rcu);

It seems you introduced non-needed complexity with the events lists.
Why not just adding new events on the current refs?
Please make this simple at the first step. Forget about performance,
but just start with simple code.

> +
> +	return i;
> +}
> +
> +/*
> + * Handles the ioctl from user mode to register or alter operations.
> + */
> +static long user_events_ioctl(struct file *file, unsigned int cmd,
> +			      unsigned long uarg)
> +{
> +	void __user *ubuf = (void __user *)uarg;
> +	struct user_event *user;
> +	char *name;
> +	long ret;
> +
> +	switch (cmd) {
> +	case DIAG_IOCSREG:
> +		/* Register/lookup on behalf of user process */
> +		name = strndup_user(ubuf, MAX_EVENT_DESC);
> +
> +		if (IS_ERR(name)) {
> +			ret = PTR_ERR(name);
> +			goto out;
> +		}
> +
> +		mutex_lock(&event_mutex);
> +
> +		ret = user_event_parse_cmd(name, &user);
> +
> +		if (ret < 0)
> +			goto reg_out;
> +
> +		preempt_disable();

Why do you need to disable preemption here? It is enough to
use a mutex.

> +		ret = user_events_ref_add(file, user);
> +		preempt_enable();
> +
> +		if (ret < 0)
> +			goto reg_out;
> +
> +		ret = INDEX_COMBINE(ret, user->index);
> +reg_out:
> +		mutex_unlock(&event_mutex);
> +
> +		break;
> +
> +	case DIAG_IOCSDEL:
> +		/* Delete on behalf of user process */
> +		name = strndup_user(ubuf, MAX_EVENT_DESC);
> +
> +		if (IS_ERR(name)) {
> +			ret = PTR_ERR(name);
> +			goto out;
> +		}
> +
> +		mutex_lock(&event_mutex);
> +		ret = delete_user_event(name);
> +		mutex_unlock(&event_mutex);
> +
> +		kfree(name);
> +		break;
> +
> +	default:
> +		ret = -ENOTTY;
> +		break;
> +	}
> +out:
> +	return ret;
> +}
> +
> +/*
> + * Handles the final close of the file from user mode.
> + */
> +static int user_events_release(struct inode *node, struct file *file)
> +{
> +	struct user_event_refs *refs;
> +	struct user_event *user;
> +	int i;
> +
> +	preempt_disable();

I'm not sure why do you stick on using the RCU here.
Since the user-events are only used in the user-context,
which is not a critical section.
For example, if you use kprobes, it may be called from anywhere
in the kernel including critical section, and the handler will
run in the interrupt handler. Thus I should use RCU. But here,
the user events are written by syscalls, no critical section
at all...

Can you try to remove RCU from this code?

Thank you,

> +
> +	refs = rcu_dereference_sched(file->private_data);
> +
> +	if (!refs)
> +		goto out;
> +
> +	for (i = 0; i < refs->count; ++i) {
> +		user = refs->events[i];
> +
> +		if (user)
> +			atomic_dec(&user->refs);
> +	}
> +
> +	kfree_rcu(refs, rcu);
> +out:
> +	preempt_enable();
> +
> +	return 0;
> +}
> +
> +static const struct file_operations user_data_fops = {
> +	.write = user_events_write,
> +	.write_iter = user_events_write_iter,
> +	.unlocked_ioctl	= user_events_ioctl,
> +	.release = user_events_release,
> +};
> +
> +/*
> + * Maps the shared page into the user process for checking if event is enabled.
> + */
> +static int user_status_mmap(struct file *file, struct vm_area_struct *vma)
> +{
> +	unsigned long size = vma->vm_end - vma->vm_start;
> +
> +	if (size != MAX_EVENTS)
> +		return -EINVAL;
> +
> +	return remap_pfn_range(vma, vma->vm_start,
> +			       virt_to_phys(register_page_data) >> PAGE_SHIFT,
> +			       size, PAGE_READONLY);
> +}
> +
> +static int user_status_show(struct seq_file *m, void *p)
> +{
> +	struct user_event *user;
> +	char status;
> +	int i, active = 0, busy = 0;
> +
> +	mutex_lock(&event_mutex);
> +
> +	hash_for_each(register_table, i, user, node) {
> +		status = register_page_data[user->index];
> +
> +		seq_printf(m, "%d:%s", user->index, EVENT_NAME(user));
> +
> +		if (status != 0) {
> +			seq_puts(m, " # Used by");
> +			if (status & EVENT_STATUS_FTRACE)
> +				seq_puts(m, " ftrace");
> +			if (status & EVENT_STATUS_PERF)
> +				seq_puts(m, " perf");
> +			if (status & EVENT_STATUS_OTHER)
> +				seq_puts(m, " other");
> +			busy++;
> +		}
> +
> +		seq_puts(m, "\n");
> +		active++;
> +	}
> +
> +	mutex_unlock(&event_mutex);
> +
> +	seq_puts(m, "\n");
> +	seq_printf(m, "Active: %d\n", active);
> +	seq_printf(m, "Busy: %d\n", busy);
> +	seq_printf(m, "Max: %ld\n", MAX_EVENTS);
> +
> +	return 0;
> +}
> +
> +static ssize_t user_status_read(struct file *file, char __user *ubuf,
> +				size_t count, loff_t *ppos)
> +{
> +	/*
> +	 * Delay allocation of seq data until requested, most callers
> +	 * will never read the status file. They will only mmap.
> +	 */
> +	if (file->private_data == NULL) {
> +		int ret;
> +
> +		if (*ppos != 0)
> +			return -EINVAL;
> +
> +		ret = single_open(file, user_status_show, NULL);
> +
> +		if (ret)
> +			return ret;
> +	}
> +
> +	return seq_read(file, ubuf, count, ppos);
> +}
> +
> +static loff_t user_status_seek(struct file *file, loff_t offset, int whence)
> +{
> +	if (file->private_data == NULL)
> +		return 0;
> +
> +	return seq_lseek(file, offset, whence);
> +}
> +
> +static int user_status_release(struct inode *node, struct file *file)
> +{
> +	if (file->private_data == NULL)
> +		return 0;
> +
> +	return single_release(node, file);
> +}
> +
> +static const struct file_operations user_status_fops = {
> +	.mmap = user_status_mmap,
> +	.read = user_status_read,
> +	.llseek  = user_status_seek,
> +	.release = user_status_release,
> +};
> +
> +/*
> + * Creates a set of tracefs files to allow user mode interactions.
> + */
> +static int create_user_tracefs(void)
> +{
> +	struct dentry *edata, *emmap;
> +
> +	edata = tracefs_create_file("user_events_data", 0644, NULL,
> +				    NULL, &user_data_fops);
> +
> +	if (!edata) {
> +		pr_warn("Could not create tracefs 'user_events_data' entry\n");
> +		goto err;
> +	}
> +
> +	/* mmap with MAP_SHARED requires writable fd */
> +	emmap = tracefs_create_file("user_events_status", 0644, NULL,
> +				    NULL, &user_status_fops);
> +
> +	if (!emmap) {
> +		tracefs_remove(edata);
> +		pr_warn("Could not create tracefs 'user_events_mmap' entry\n");
> +		goto err;
> +	}
> +
> +	return 0;
> +err:
> +	return -ENODEV;
> +}
> +
> +static void set_page_reservations(bool set)
> +{
> +	int page;
> +
> +	for (page = 0; page < MAX_PAGES; ++page) {
> +		void *addr = register_page_data + (PAGE_SIZE * page);
> +
> +		if (set)
> +			SetPageReserved(virt_to_page(addr));
> +		else
> +			ClearPageReserved(virt_to_page(addr));
> +	}
> +}
> +
> +static int __init trace_events_user_init(void)
> +{
> +	int ret;
> +
> +	/* Zero all bits beside 0 (which is reserved for failures) */
> +	bitmap_zero(page_bitmap, MAX_EVENTS);
> +	set_bit(0, page_bitmap);
> +
> +	register_page_data = kzalloc(MAX_EVENTS, GFP_KERNEL);
> +
> +	if (!register_page_data)
> +		return -ENOMEM;
> +
> +	set_page_reservations(true);
> +
> +	ret = create_user_tracefs();
> +
> +	if (ret) {
> +		pr_warn("user_events could not register with tracefs\n");
> +		set_page_reservations(false);
> +		kfree(register_page_data);
> +		return ret;
> +	}
> +
> +	if (dyn_event_register(&user_event_dops))
> +		pr_warn("user_events could not register with dyn_events\n");
> +
> +	return 0;
> +}
> +
> +fs_initcall(trace_events_user_init);
> -- 
> 2.17.1
> 


-- 
Masami Hiramatsu <mhiramat@kernel.org>

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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
  2021-10-22 13:38 ` Masami Hiramatsu
@ 2021-10-22 22:42   ` Beau Belgrave
  2021-10-25  1:40     ` Masami Hiramatsu
  0 siblings, 1 reply; 9+ messages in thread
From: Beau Belgrave @ 2021-10-22 22:42 UTC (permalink / raw)
  To: Masami Hiramatsu; +Cc: rostedt, linux-trace-devel, linux-kernel

On Fri, Oct 22, 2021 at 10:38:11PM +0900, Masami Hiramatsu wrote:
> On Mon, 18 Oct 2021 16:09:57 -0700
> Beau Belgrave <beaub@linux.microsoft.com> wrote:
> > of the ioctl indicates the byte in the mmap to use for status and the index
> > to use for writing. Both are 16-bits in size and are accessible via the
> > INDEX_STATUS and INDEX_WRITE macros within user_events.h.
> 
> Hmm, if we uses the ioctl(), the return value must be int because it can
> return -1 and kernel return value is stored into the errno if there is
> any error. I think you can introduce a data structure to return the
> id and offset. (Or make id 15-bits :))  
> 
Yeah, I think I will introduce a struct to aid with this and some others
things like flags / print_fmt being specified (See below for those).

> > int data_fd = open("user_events_data", O_RDWR);
> > long result = ioctl(data_fd, DIAG_IOCSREG, "test");
> > int status_id = INDEX_STATUS(result);
> > int write_id = INDEX_WRITE(result);
> > 
> > struct iovec io[2];
> > io[0].iov_base = &write_id;
> > io[0].iov_len = sizeof(write_id);
> > io[1].iov_base = payload;
> > io[1].iov_len = sizeof(payload);
> > 
> > if (page_data[status_id])
> > 	writev(data_fd, io, 2);
> 
> OK, so you switch to the 1:n model. Using writev() is a good idea.
> (BTW, what happen if I use write()?)
> 
The first 4 bytes are used regardless if writev() or write() is used.
Both turn into an iterator to make the code simpler instead of handling
each case differently.

> > 
> > Users can describe the trace event format via the following format:
> > 	name [field1;field2]
> 
> Does this mean it always need 2 fields?
> 
No, they need 0-N fields, perhaps I should leave off the second field in
the description.

> > 
> > Each field has the following format:
> > 	type name size
> 
> Why do you need the "size"? Since "type" must have the size information,
> it is not needed. (or, user can define 1 byte "int" type...)
> 
The idea is to allow user mode processes to describe their own data
types if required. For that we need offset or size to be passed. If we
require all user mode to only describe in types the kernel understands
then I can remove the size field.

> > 
> > Example for char array with a size of 20 named msg:
> > 	echo 'u:detailed char[20] msg 20' >> dynamic_events
> > 	cat dynamic_events
> > 	u:detailed char[20] msg
> > 
> > Data offsets are based on the data written out via write() and will be
> > updated to reflect the correct offset in the trace_event fields. __data_loc
> > types must be aware of the size of trace_entry/common properties to ensure
> > proper decoding.
> 
> As I said that the you have to add __rel_loc type (yes, you can do it)
> so that the user don't need to consider the data location. That new type
> decodeer code may need to be implemented afterwards in user-space tools
> but anyway, that can be postponed.
> 
Got it, does that mean I should just reference it here as a new type?
I'm not sure what it means to add __rel_loc type since it appears anyone
can pass any type down to trace_event and it will accept it.

> > ---
> >  include/uapi/linux/user_events.h |   57 ++
> >  kernel/trace/Kconfig             |   15 +
> >  kernel/trace/Makefile            |    1 +
> >  kernel/trace/trace_events_user.c | 1040 ++++++++++++++++++++++++++++++
> >  4 files changed, 1113 insertions(+)
> >  create mode 100644 include/uapi/linux/user_events.h
> >  create mode 100644 kernel/trace/trace_events_user.c
> 
> BTW, if you can split this patch into several patches, it is easier to us
> to review it. E.g. basic support for ftrace, add perf support, add bpf support,
> documentation, optimize performance etc.
> 
Sorry, are you asking for several isolated patches? Or a single patch
with multiple commits to the same file?

Typically after addition I am running a set of tests and benchmarks
against the code, so having full functioning code is required to ensure
I haven't broken anything.

We are actively using these patches for various projects, which require
ftrace, perf and eBPF to be functional.

> > +
> > +/*
> > + * Parses the values of a field within the description
> > + * Format: type name size [future]
> > + */
> 
> As I said that the size is non-sense. And this should finally unified
> with synthetic event field definition. 
> 
I have this for custom user types the kernel may not know the size of.
Another reason is 32-bit process on a 64-bit kernel. The size will always be
the kernel size of that type.

If we don't care about either, then I can remove this. For custom types
we are fine with using __rel_loc instead.

> > +static int user_event_parse_field(char *field, struct user_event *user,
> > +				  u32 *offset)
> > +{
> > +	char *part, *type, *name;
> > +	u32 size = 0, depth = 0, saved_offset = *offset;
> > +
> > +	field = skip_spaces(field);
> > +
> > +	if (strstr(field, "unsigned ") == field) {
> > +		type = field;
> > +		field = strpbrk(field + sizeof("unsigned"), " ");
> > +
> > +		if (field == NULL)
> > +			return -EINVAL;
> > +
> > +		*field++ = 0;
> > +		depth++;
> > +	} else if (strstr(field, "print_fmt ") == field) {
> > +		user->call.print_fmt = field + sizeof("print_fmt");
> > +		return 0;
> 
> What is this ?
> 
We have several events that describe the print format to use instead of
using the default, like we allow in the kernel. This is currently how it
is passed, it could be described in the new struct to the reg ioctl
instead.

> > +	} else if (strstr(field, "flag ") == field) {
> > +		field += sizeof("flag");
> > +
> > +		if (!strcmp(field, "bpf_iter"))
> > +			user->flags |= FLAG_BPF_ITER;
> > +
> 
> What is this flag?
> 
We want to enable certain sensitive events the ability to mark that
there should never be a buffer copy. When FLAG_BPF_ITER is used the raw
iovecs are exposed out to eBPF instead of any sort of copy to reduce
latency. We run user_events in some highly performant code and want to
monitor things with the least amount of overhead possible.

> > +	name = kstrdup(raw_command, GFP_KERNEL);
> > +
> > +	if (!name)
> > +		return -ENOMEM;
> > +
> > +	mutex_lock(&event_mutex);
> > +	ret = user_event_parse_cmd(name, &user);
> 
> I recommend you to split "parse" and "register" function, and
> locks event_mutex only in the "register" function, since parsing
> doesn't need to hold other events.
> 
Sure

> > +static bool user_event_match(const char *system, const char *event,
> > +			     int argc, const char **argv, struct dyn_event *ev)
> > +{
> > +	struct user_event *user = container_of(ev, struct user_event, devent);
> > +
> > +	return strcmp(EVENT_NAME(user), event) == 0 &&
> > +		(!system || strcmp(system, USER_EVENTS_SYSTEM) == 0);
> 
> For the minimal implementation, this is OK.
> 
> Note that it is better to check the rest of arguments with the event
> parameters. E.g. if you have "myuevent int var", user cant try to remove
> "-:myuevent char var".
> 
Ok, the reason we do this is because the events are going to be
versioned. They might change their arguments and having a way to clear
without knowing the exact arguments is not a bad thing in our cases.

No one can remove an event while it's being used and the name cannot
conflict since all user events go to the user_events sub-system and
cannot be overriden.

> > +/*
> > + * Validates the user payload and writes via iterator.
> > + */
> > +static ssize_t user_events_write_core(struct file *file, struct iov_iter *i)
> > +{
> > +	struct user_event_refs *refs;
> > +	struct user_event *user = NULL;
> > +	struct tracepoint *tp;
> > +	ssize_t ret = i->count;
> > +	int idx;
> > +
> > +	preempt_disable();
> 
> Why do you disable preemption here?
> 
My understanding was that preemption is to be disabled when using rcu
protected by sched, which tracepoints and the refs are. We also want the
correct processor to log the event (as near as we can).

Did I misunderstand rcu_dereference_sched_check()? I may be off in the
weeds.

I also ran with CONFIG_PROVE_RCU and didn't see anything show up in
dmesg.

> > +
> > +	if (unlikely(!(cpu_online(raw_smp_processor_id()))))
> > +		goto out;
> 
> This seems meaningless because this function is called in the
> user process context. If the cpu is going to offline, the process
> itself will be migrated before reaching here.
> 
Sure, everywhere else had this and wasn't sure if this was possible to
happen or not. I'll remove it.

> > +
> > +	if (unlikely(copy_from_iter(&idx, sizeof(idx), i) != sizeof(idx))) {
> > +		ret = -EFAULT;
> > +		goto out;
> > +	}
> 
> Isn't this copy_from_iter() sleepable function?
> 
Yes, if we need to fault in data we want the data to come in. This is in
the user process context of write() and writev() which I believe is the
expected target for copy_from_iter().

During the probe copies we are not holding a lock or have interrupts
disabled, under what circumstances would a page fault cause an issue in
these paths? What is recommended here if we want to have page faults be
allowed (Hoping they are rare)?

> > +				/*
> > +				 * Probes advance the iterator so we
> > +				 * need to have a copy for each probe.
> > +				 */
> > +				copy = *i;
> > +
> > +				probe_func = probe_func_ptr->func;
> > +				tpdata = probe_func_ptr->data;
> > +				probe_func(user, &copy, tpdata);
> 
> You seems to try to copy in from user space in each probe func, but
> please copy it here to the temporary buffer and pass it to the
> each probe function. Such performacne optimization can postpone.
> Start with simple implementation.
> 
Yes, this avoids double copying of the data in the normal paths. Moving
to a temp buffer only really changes 1 line in the probe functions
(copy_from_iter to copy_from_user).

If I were to create a temp buffer for simplicity I guess I would have to
kmalloc on each call or move to a per-cpu buffer or use stack memory and
limit how much data can be copied.

These would likely add more lines than they remove from the code.

> > +	refs = rcu_dereference_sched(file->private_data);
> > +
> > +	if (refs) {
> > +		count = refs->count;
> > +
> > +		for (i = 0; i < count; ++i)
> > +			if (refs->events[i] == user)
> > +				return i;
> > +	}
> > +
> > +	size = sizeof(*refs) + (sizeof(struct user_event *) * (count + 1));
> > +
> > +	new_refs = kzalloc(size, GFP_KERNEL);
> > +
> > +	if (!new_refs)
> > +		return -ENOMEM;
> > +
> > +	new_refs->count = count + 1;
> > +
> > +	for (i = 0; i < count; ++i)
> > +		new_refs->events[i] = refs->events[i];
> > +
> > +	new_refs->events[i] = user;
> > +
> > +	atomic_inc(&user->refs);
> > +
> > +	rcu_assign_pointer(file->private_data, new_refs);
> > +
> > +	if (refs)
> > +		kfree_rcu(refs, rcu);
> 
> It seems you introduced non-needed complexity with the events lists.
> Why not just adding new events on the current refs?
> Please make this simple at the first step. Forget about performance,
> but just start with simple code.
> 
Callers are free to call the reg ioctl while a write is occurring on the
same file. This can cause the kernel to access garbage. Either RCU or
locking is required in these paths or the ABI needs a way to lock the
file into a mode where no more events can be added (IE: Change to write
mode).

We want RCU because we have places where multiple processors will be
emitting data out to the same file. We don't want the data/cache
contention across processors to slow things down, which we have seen in
other telemetry data.

> > +	case DIAG_IOCSREG:
> > +		/* Register/lookup on behalf of user process */
> > +		name = strndup_user(ubuf, MAX_EVENT_DESC);
> > +
> > +		if (IS_ERR(name)) {
> > +			ret = PTR_ERR(name);
> > +			goto out;
> > +		}
> > +
> > +		mutex_lock(&event_mutex);
> > +
> > +		ret = user_event_parse_cmd(name, &user);
> > +
> > +		if (ret < 0)
> > +			goto reg_out;
> > +
> > +		preempt_disable();
> 
> Why do you need to disable preemption here? It is enough to
> use a mutex.
> 
Because user_events_ref_add use rcu protected by sched. It appears that
preemption is supposed to be disabled when using that from my read of
the code. Is this not correct or not required?

> > +/*
> > + * Handles the final close of the file from user mode.
> > + */
> > +static int user_events_release(struct inode *node, struct file *file)
> > +{
> > +	struct user_event_refs *refs;
> > +	struct user_event *user;
> > +	int i;
> > +
> > +	preempt_disable();
> 
> I'm not sure why do you stick on using the RCU here.
> Since the user-events are only used in the user-context,
> which is not a critical section.
> For example, if you use kprobes, it may be called from anywhere
> in the kernel including critical section, and the handler will
> run in the interrupt handler. Thus I should use RCU. But here,
> the user events are written by syscalls, no critical section
> at all...
> 
> Can you try to remove RCU from this code?
> 
If you are talking about this specific call site (user_events_release)
then yeah I believe I can remove it. If you talking in general, we use
RCU to protect against some scenarios where events are getting
registered while previous events are getting written out.

We chose RCU over readwrite locking due to potential for shared cache
contention related slowdowns.

> Thank you,
> -- 
> Masami Hiramatsu <mhiramat@kernel.org>
Thank you for taking the time to review, appreciate it!

Thanks,
-Beau

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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
  2021-10-22 22:42   ` Beau Belgrave
@ 2021-10-25  1:40     ` Masami Hiramatsu
  2021-10-25 17:26       ` Beau Belgrave
  0 siblings, 1 reply; 9+ messages in thread
From: Masami Hiramatsu @ 2021-10-25  1:40 UTC (permalink / raw)
  To: Beau Belgrave
  Cc: rostedt, linux-trace-devel, linux-kernel, Tzvetomir Stoyanov,
	Paul E. McKenney

On Fri, 22 Oct 2021 15:42:02 -0700
Beau Belgrave <beaub@linux.microsoft.com> wrote:

> On Fri, Oct 22, 2021 at 10:38:11PM +0900, Masami Hiramatsu wrote:
> > On Mon, 18 Oct 2021 16:09:57 -0700
> > Beau Belgrave <beaub@linux.microsoft.com> wrote:
> > > of the ioctl indicates the byte in the mmap to use for status and the index
> > > to use for writing. Both are 16-bits in size and are accessible via the
> > > INDEX_STATUS and INDEX_WRITE macros within user_events.h.
> > 
> > Hmm, if we uses the ioctl(), the return value must be int because it can
> > return -1 and kernel return value is stored into the errno if there is
> > any error. I think you can introduce a data structure to return the
> > id and offset. (Or make id 15-bits :))  
> > 
> Yeah, I think I will introduce a struct to aid with this and some others
> things like flags / print_fmt being specified (See below for those).
> 
> > > int data_fd = open("user_events_data", O_RDWR);
> > > long result = ioctl(data_fd, DIAG_IOCSREG, "test");
> > > int status_id = INDEX_STATUS(result);
> > > int write_id = INDEX_WRITE(result);
> > > 
> > > struct iovec io[2];
> > > io[0].iov_base = &write_id;
> > > io[0].iov_len = sizeof(write_id);
> > > io[1].iov_base = payload;
> > > io[1].iov_len = sizeof(payload);
> > > 
> > > if (page_data[status_id])
> > > 	writev(data_fd, io, 2);
> > 
> > OK, so you switch to the 1:n model. Using writev() is a good idea.
> > (BTW, what happen if I use write()?)
> > 
> The first 4 bytes are used regardless if writev() or write() is used.
> Both turn into an iterator to make the code simpler instead of handling
> each case differently.

Ah OK. 

> 
> > > 
> > > Users can describe the trace event format via the following format:
> > > 	name [field1;field2]
> > 
> > Does this mean it always need 2 fields?
> > 
> No, they need 0-N fields, perhaps I should leave off the second field in
> the description.

OK, in that case, 

 name [field1[;field2...]]

will be good.

> 
> > > 
> > > Each field has the following format:
> > > 	type name size
> > 
> > Why do you need the "size"? Since "type" must have the size information,
> > it is not needed. (or, user can define 1 byte "int" type...)
> > 
> The idea is to allow user mode processes to describe their own data
> types if required. For that we need offset or size to be passed. If we
> require all user mode to only describe in types the kernel understands
> then I can remove the size field.

Hmm, I got it. I think it should be optional, and only for the types
which has "struct" prefix. Basic types like char, int, long etc should
keep the size as it is.

> > > 
> > > Example for char array with a size of 20 named msg:
> > > 	echo 'u:detailed char[20] msg 20' >> dynamic_events
> > > 	cat dynamic_events
> > > 	u:detailed char[20] msg
> > > 
> > > Data offsets are based on the data written out via write() and will be
> > > updated to reflect the correct offset in the trace_event fields. __data_loc
> > > types must be aware of the size of trace_entry/common properties to ensure
> > > proper decoding.
> > 
> > As I said that the you have to add __rel_loc type (yes, you can do it)
> > so that the user don't need to consider the data location. That new type
> > decodeer code may need to be implemented afterwards in user-space tools
> > but anyway, that can be postponed.
> > 
> Got it, does that mean I should just reference it here as a new type?
> I'm not sure what it means to add __rel_loc type since it appears anyone
> can pass any type down to trace_event and it will accept it.

The __data_loc (and __rel_loc) is a bit special type attribute, which is
required for "parsing" the trace buffer entry. Thus it is used also in the
libtraceevent (and perf tools to pass the parsed event to python/perl script).
Let me add the __rel_loc support.

> > > ---
> > >  include/uapi/linux/user_events.h |   57 ++
> > >  kernel/trace/Kconfig             |   15 +
> > >  kernel/trace/Makefile            |    1 +
> > >  kernel/trace/trace_events_user.c | 1040 ++++++++++++++++++++++++++++++
> > >  4 files changed, 1113 insertions(+)
> > >  create mode 100644 include/uapi/linux/user_events.h
> > >  create mode 100644 kernel/trace/trace_events_user.c
> > 
> > BTW, if you can split this patch into several patches, it is easier to us
> > to review it. E.g. basic support for ftrace, add perf support, add bpf support,
> > documentation, optimize performance etc.
> > 
> Sorry, are you asking for several isolated patches? Or a single patch
> with multiple commits to the same file?

I mean the several isolated patches which commit to the same file.
(like upgrading it step by step.) You can make it a series.

> Typically after addition I am running a set of tests and benchmarks
> against the code, so having full functioning code is required to ensure
> I haven't broken anything.

Yes, you can put the test in the end of the series :)

> We are actively using these patches for various projects, which require
> ftrace, perf and eBPF to be functional.

The series usually merged at once :) I just asked you to break it
down to the several patches, so that I can review and discuss on
each feature in different threads.

> > > +
> > > +/*
> > > + * Parses the values of a field within the description
> > > + * Format: type name size [future]
> > > + */
> > 
> > As I said that the size is non-sense. And this should finally unified
> > with synthetic event field definition. 
> > 
> I have this for custom user types the kernel may not know the size of.
> Another reason is 32-bit process on a 64-bit kernel. The size will always be
> the kernel size of that type.

Hmm, I see. In that case, the user process should not use such types because
other analysis tool will run as a 64-bit process.
(For example, for this reason kprobe event doesn't use 'int' or 'long' type,
 but only allow 'u32', 's64' etc.)

> If we don't care about either, then I can remove this. For custom types
> we are fine with using __rel_loc instead.

Or, as I said, you can pass the size when it has "struct" prefix. :)
The __rel_loc and __data_loc is usually used for storing arrays, especially
dynamic size array like null terminated string. But of course you can use
it for storing a big data structure.
Let's check how the libtraceevent(tools/lib/traceevent) parses the recorded
data.

> > > +static int user_event_parse_field(char *field, struct user_event *user,
> > > +				  u32 *offset)
> > > +{
> > > +	char *part, *type, *name;
> > > +	u32 size = 0, depth = 0, saved_offset = *offset;
> > > +
> > > +	field = skip_spaces(field);
> > > +
> > > +	if (strstr(field, "unsigned ") == field) {
> > > +		type = field;
> > > +		field = strpbrk(field + sizeof("unsigned"), " ");
> > > +
> > > +		if (field == NULL)
> > > +			return -EINVAL;
> > > +
> > > +		*field++ = 0;
> > > +		depth++;
> > > +	} else if (strstr(field, "print_fmt ") == field) {
> > > +		user->call.print_fmt = field + sizeof("print_fmt");
> > > +		return 0;
> > 
> > What is this ?
> > 
> We have several events that describe the print format to use instead of
> using the default, like we allow in the kernel. This is currently how it
> is passed, it could be described in the new struct to the reg ioctl
> instead.

OK, the 'print_fmt' in 'format' file will show how the trace entry will be
printed in the trace buffer (in human readable style). But that is not
done automatically. You have to write the corresponding printout function
for the event. This means that user-specific format needs to be verified
and you need a programmable printout function. This is why it is hard to
support the custom data structure in ftrace.
Anyway, I think you'd better to start without this option. (and please
start with describing how the user-event arguments are shown in trace
buffer.)

> 
> > > +	} else if (strstr(field, "flag ") == field) {
> > > +		field += sizeof("flag");
> > > +
> > > +		if (!strcmp(field, "bpf_iter"))
> > > +			user->flags |= FLAG_BPF_ITER;
> > > +
> > 
> > What is this flag?
> > 
> We want to enable certain sensitive events the ability to mark that
> there should never be a buffer copy. When FLAG_BPF_ITER is used the raw
> iovecs are exposed out to eBPF instead of any sort of copy to reduce
> latency. We run user_events in some highly performant code and want to
> monitor things with the least amount of overhead possible.

Would you mean the event with this flag is only available from eBPF?

> 
> > > +	name = kstrdup(raw_command, GFP_KERNEL);
> > > +
> > > +	if (!name)
> > > +		return -ENOMEM;
> > > +
> > > +	mutex_lock(&event_mutex);
> > > +	ret = user_event_parse_cmd(name, &user);
> > 
> > I recommend you to split "parse" and "register" function, and
> > locks event_mutex only in the "register" function, since parsing
> > doesn't need to hold other events.
> > 
> Sure
> 
> > > +static bool user_event_match(const char *system, const char *event,
> > > +			     int argc, const char **argv, struct dyn_event *ev)
> > > +{
> > > +	struct user_event *user = container_of(ev, struct user_event, devent);
> > > +
> > > +	return strcmp(EVENT_NAME(user), event) == 0 &&
> > > +		(!system || strcmp(system, USER_EVENTS_SYSTEM) == 0);
> > 
> > For the minimal implementation, this is OK.
> > 
> > Note that it is better to check the rest of arguments with the event
> > parameters. E.g. if you have "myuevent int var", user cant try to remove
> > "-:myuevent char var".
> > 
> Ok, the reason we do this is because the events are going to be
> versioned. They might change their arguments and having a way to clear
> without knowing the exact arguments is not a bad thing in our cases.

Actually, as other dynamic events do, user can remove only by name, or
specifically with the arguments. So argument check is optional but
if user specified it, we should verify it.

> 
> No one can remove an event while it's being used and the name cannot
> conflict since all user events go to the user_events sub-system and
> cannot be overriden.
> 
> > > +/*
> > > + * Validates the user payload and writes via iterator.
> > > + */
> > > +static ssize_t user_events_write_core(struct file *file, struct iov_iter *i)
> > > +{
> > > +	struct user_event_refs *refs;
> > > +	struct user_event *user = NULL;
> > > +	struct tracepoint *tp;
> > > +	ssize_t ret = i->count;
> > > +	int idx;
> > > +
> > > +	preempt_disable();
> > 
> > Why do you disable preemption here?
> > 
> My understanding was that preemption is to be disabled when using rcu
> protected by sched, which tracepoints and the refs are. We also want the
> correct processor to log the event (as near as we can).
> 
> Did I misunderstand rcu_dereference_sched_check()? I may be off in the
> weeds.

I see that you want to fix the processor id, but preempt_disable()
section should be minimum as possible and should not involve the
function which can access to user memeory.

> 
> I also ran with CONFIG_PROVE_RCU and didn't see anything show up in
> dmesg.

Hmm, that's strange, because copy_from_iter(user) may cause a fault
and yielded. Isn't it an iovec?

> > > +
> > > +	if (unlikely(!(cpu_online(raw_smp_processor_id()))))
> > > +		goto out;
> > 
> > This seems meaningless because this function is called in the
> > user process context. If the cpu is going to offline, the process
> > itself will be migrated before reaching here.
> > 
> Sure, everywhere else had this and wasn't sure if this was possible to
> happen or not. I'll remove it.
> 
> > > +
> > > +	if (unlikely(copy_from_iter(&idx, sizeof(idx), i) != sizeof(idx))) {
> > > +		ret = -EFAULT;
> > > +		goto out;
> > > +	}
> > 
> > Isn't this copy_from_iter() sleepable function?
> > 
> Yes, if we need to fault in data we want the data to come in. This is in
> the user process context of write() and writev() which I believe is the
> expected target for copy_from_iter().
> 
> During the probe copies we are not holding a lock or have interrupts
> disabled, under what circumstances would a page fault cause an issue in
> these paths? What is recommended here if we want to have page faults be
> allowed (Hoping they are rare)?

They are rare, but possibly the process is yielded if it is under heavy
memory pressure. I recommend you to copy the user write data in the stack
or thread-local buffer at once and pass it to probe functions. That is
safe way.

So, can you make it something like below?;

/* Copy event from user space */
copy_from_iter(&idx, sizeof(idx), i);
buf_len = copy_from_iter(local_buf, i->count, &i);

/* loop on recording functions */
preempt_disable();
user_event = get_user_event(file, idx);
if (user_event && user_event_is_enabled(user_event)) {
	user_event_for_each_probe_func(user_event, probe_func, tpdata) {
		probe_func(user_event, thread_buf, buf_len, tpdata);
	}
}
preempt_enable();

> 
> > > +				/*
> > > +				 * Probes advance the iterator so we
> > > +				 * need to have a copy for each probe.
> > > +				 */
> > > +				copy = *i;
> > > +
> > > +				probe_func = probe_func_ptr->func;
> > > +				tpdata = probe_func_ptr->data;
> > > +				probe_func(user, &copy, tpdata);
> > 
> > You seems to try to copy in from user space in each probe func, but
> > please copy it here to the temporary buffer and pass it to the
> > each probe function. Such performacne optimization can postpone.
> > Start with simple implementation.
> > 
> Yes, this avoids double copying of the data in the normal paths. Moving
> to a temp buffer only really changes 1 line in the probe functions
> (copy_from_iter to copy_from_user).
> 
> If I were to create a temp buffer for simplicity I guess I would have to
> kmalloc on each call or move to a per-cpu buffer or use stack memory and
> limit how much data can be copied.

Anyway, it should be limited. You can not write more than 1 page, and
do you really need it? And allocating kmalloc object is relatively low
cost compared with a system call.

> 
> These would likely add more lines than they remove from the code.

That's OK.

> 
> > > +	refs = rcu_dereference_sched(file->private_data);
> > > +
> > > +	if (refs) {
> > > +		count = refs->count;
> > > +
> > > +		for (i = 0; i < count; ++i)
> > > +			if (refs->events[i] == user)
> > > +				return i;
> > > +	}
> > > +
> > > +	size = sizeof(*refs) + (sizeof(struct user_event *) * (count + 1));
> > > +
> > > +	new_refs = kzalloc(size, GFP_KERNEL);
> > > +
> > > +	if (!new_refs)
> > > +		return -ENOMEM;
> > > +
> > > +	new_refs->count = count + 1;
> > > +
> > > +	for (i = 0; i < count; ++i)
> > > +		new_refs->events[i] = refs->events[i];
> > > +
> > > +	new_refs->events[i] = user;
> > > +
> > > +	atomic_inc(&user->refs);

BTW, I recommend you to use a different name for this reference counter,
e.g. refcnt, since refs is used for "file-local reference user event tables".


> > > +
> > > +	rcu_assign_pointer(file->private_data, new_refs);
> > > +
> > > +	if (refs)
> > > +		kfree_rcu(refs, rcu);
> > 
> > It seems you introduced non-needed complexity with the events lists.
> > Why not just adding new events on the current refs?
> > Please make this simple at the first step. Forget about performance,
> > but just start with simple code.
> > 
> Callers are free to call the reg ioctl while a write is occurring on the
> same file. This can cause the kernel to access garbage. Either RCU or
> locking is required in these paths or the ABI needs a way to lock the
> file into a mode where no more events can be added (IE: Change to write
> mode).
> 
> We want RCU because we have places where multiple processors will be
> emitting data out to the same file. We don't want the data/cache
> contention across processors to slow things down, which we have seen in
> other telemetry data.

Hmm, OK. So the events are not serialized, multiple threads can write the
events concurrently. Got it.

> 
> > > +	case DIAG_IOCSREG:
> > > +		/* Register/lookup on behalf of user process */
> > > +		name = strndup_user(ubuf, MAX_EVENT_DESC);
> > > +
> > > +		if (IS_ERR(name)) {
> > > +			ret = PTR_ERR(name);
> > > +			goto out;
> > > +		}
> > > +
> > > +		mutex_lock(&event_mutex);
> > > +
> > > +		ret = user_event_parse_cmd(name, &user);
> > > +
> > > +		if (ret < 0)
> > > +			goto reg_out;
> > > +
> > > +		preempt_disable();
> > 
> > Why do you need to disable preemption here? It is enough to
> > use a mutex.
> > 
> Because user_events_ref_add use rcu protected by sched. It appears that
> preemption is supposed to be disabled when using that from my read of
> the code. Is this not correct or not required?

No not required, the "event_mutex" prevents other user from invoking
the user_events_ref_add(), thus the "file->private_data" will not be
updated from others. Thus you don't need to care the scenario of
"file->private_data" is changed between dereference and reassign it.
I think what you need is "lockdep_assert_held(&event_mutex)" in the
user_events_ref_add().

> > > +/*
> > > + * Handles the final close of the file from user mode.
> > > + */
> > > +static int user_events_release(struct inode *node, struct file *file)
> > > +{
> > > +	struct user_event_refs *refs;
> > > +	struct user_event *user;
> > > +	int i;
> > > +
> > > +	preempt_disable();
> > 
> > I'm not sure why do you stick on using the RCU here.
> > Since the user-events are only used in the user-context,
> > which is not a critical section.
> > For example, if you use kprobes, it may be called from anywhere
> > in the kernel including critical section, and the handler will
> > run in the interrupt handler. Thus I should use RCU. But here,
> > the user events are written by syscalls, no critical section
> > at all...
> > 
> > Can you try to remove RCU from this code?
> > 
> If you are talking about this specific call site (user_events_release)
> then yeah I believe I can remove it. If you talking in general, we use
> RCU to protect against some scenarios where events are getting
> registered while previous events are getting written out.
> 
> We chose RCU over readwrite locking due to potential for shared cache
> contention related slowdowns.
> 

OK, I understand why you need to use RCU at least reader part. For the
updater part, you don't need the preemption disabled but need a mutex.
(And when closing a file, the fd is already invalidated.)


Thank you,

-- 
Masami Hiramatsu <mhiramat@kernel.org>

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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
  2021-10-25  1:40     ` Masami Hiramatsu
@ 2021-10-25 17:26       ` Beau Belgrave
  2021-10-26  8:26         ` Masami Hiramatsu
  0 siblings, 1 reply; 9+ messages in thread
From: Beau Belgrave @ 2021-10-25 17:26 UTC (permalink / raw)
  To: Masami Hiramatsu
  Cc: rostedt, linux-trace-devel, linux-kernel, Tzvetomir Stoyanov,
	Paul E. McKenney

On Mon, Oct 25, 2021 at 10:40:06AM +0900, Masami Hiramatsu wrote:
> On Fri, 22 Oct 2021 15:42:02 -0700
> Beau Belgrave <beaub@linux.microsoft.com> wrote:
> 
> > On Fri, Oct 22, 2021 at 10:38:11PM +0900, Masami Hiramatsu wrote:
> > The idea is to allow user mode processes to describe their own data
> > types if required. For that we need offset or size to be passed. If we
> > require all user mode to only describe in types the kernel understands
> > then I can remove the size field.
> 
> Hmm, I got it. I think it should be optional, and only for the types
> which has "struct" prefix. Basic types like char, int, long etc should
> keep the size as it is.
> 
Ok, sounds good.

> > Sorry, are you asking for several isolated patches? Or a single patch
> > with multiple commits to the same file?
> 
> I mean the several isolated patches which commit to the same file.
> (like upgrading it step by step.) You can make it a series.
> 
> > Typically after addition I am running a set of tests and benchmarks
> > against the code, so having full functioning code is required to ensure
> > I haven't broken anything.
> 
> Yes, you can put the test in the end of the series :)
> 
> > We are actively using these patches for various projects, which require
> > ftrace, perf and eBPF to be functional.
> 
> The series usually merged at once :) I just asked you to break it
> down to the several patches, so that I can review and discuss on
> each feature in different threads.
> 
Got it, that's what I thought, but wanted to make sure.

> > If we don't care about either, then I can remove this. For custom types
> > we are fine with using __rel_loc instead.
> 
> Or, as I said, you can pass the size when it has "struct" prefix. :)
> The __rel_loc and __data_loc is usually used for storing arrays, especially
> dynamic size array like null terminated string. But of course you can use
> it for storing a big data structure.
> Let's check how the libtraceevent(tools/lib/traceevent) parses the recorded
> data.
> 
Ok, I'll go for struct prefix.

> > We have several events that describe the print format to use instead of
> > using the default, like we allow in the kernel. This is currently how it
> > is passed, it could be described in the new struct to the reg ioctl
> > instead.
> 
> OK, the 'print_fmt' in 'format' file will show how the trace entry will be
> printed in the trace buffer (in human readable style). But that is not
> done automatically. You have to write the corresponding printout function
> for the event. This means that user-specific format needs to be verified
> and you need a programmable printout function. This is why it is hard to
> support the custom data structure in ftrace.
> Anyway, I think you'd better to start without this option. (and please
> start with describing how the user-event arguments are shown in trace
> buffer.)
> 
Got it, by that I assume you mean auto-generating a print_fmt statement
for the user like the other probes. I will do that.

> > 
> > > > +	} else if (strstr(field, "flag ") == field) {
> > > > +		field += sizeof("flag");
> > > > +
> > > > +		if (!strcmp(field, "bpf_iter"))
> > > > +			user->flags |= FLAG_BPF_ITER;
> > > > +
> > > 
> > > What is this flag?
> > > 
> > We want to enable certain sensitive events the ability to mark that
> > there should never be a buffer copy. When FLAG_BPF_ITER is used the raw
> > iovecs are exposed out to eBPF instead of any sort of copy to reduce
> > latency. We run user_events in some highly performant code and want to
> > monitor things with the least amount of overhead possible.
> 
> Would you mean the event with this flag is only available from eBPF?
> 
It means that if eBPF attaches we will honor the users request to make
the data as cheap as possible to them. If a user with proper access
enables ftrace or perf on these high performant events they will still
come through (we don't want to hide them).

We will not be able to do that at all if we copy to heap or stack. At
that point we've lost the ability to delay copy/probing up until the
eBPF states it is actually required.

> > > > +	return strcmp(EVENT_NAME(user), event) == 0 &&
> > > > +		(!system || strcmp(system, USER_EVENTS_SYSTEM) == 0);
> > > 
> > > For the minimal implementation, this is OK.
> > > 
> > > Note that it is better to check the rest of arguments with the event
> > > parameters. E.g. if you have "myuevent int var", user cant try to remove
> > > "-:myuevent char var".
> > > 
> > Ok, the reason we do this is because the events are going to be
> > versioned. They might change their arguments and having a way to clear
> > without knowing the exact arguments is not a bad thing in our cases.
> 
> Actually, as other dynamic events do, user can remove only by name, or
> specifically with the arguments. So argument check is optional but
> if user specified it, we should verify it.
> 
Ok, I will look into that.

> > 
> > No one can remove an event while it's being used and the name cannot
> > conflict since all user events go to the user_events sub-system and
> > cannot be overriden.
> > 
> > > > +/*
> > > > + * Validates the user payload and writes via iterator.
> > > > + */
> > > > +static ssize_t user_events_write_core(struct file *file, struct iov_iter *i)
> > > > +{
> > > > +	struct user_event_refs *refs;
> > > > +	struct user_event *user = NULL;
> > > > +	struct tracepoint *tp;
> > > > +	ssize_t ret = i->count;
> > > > +	int idx;
> > > > +
> > > > +	preempt_disable();
> > > 
> > > Why do you disable preemption here?
> > > 
> > My understanding was that preemption is to be disabled when using rcu
> > protected by sched, which tracepoints and the refs are. We also want the
> > correct processor to log the event (as near as we can).
> > 
> > Did I misunderstand rcu_dereference_sched_check()? I may be off in the
> > weeds.
> 
> I see that you want to fix the processor id, but preempt_disable()
> section should be minimum as possible and should not involve the
> function which can access to user memeory.
> 
I guess I could use disable migration or just live with the rare event
of a processor swap.

> > 
> > I also ran with CONFIG_PROVE_RCU and didn't see anything show up in
> > dmesg.
> 
> Hmm, that's strange, because copy_from_iter(user) may cause a fault
> and yielded. Isn't it an iovec?
> 
Yeah, likely I just haven't hit a page fault case. I'll try to force one
in our testing to ensure this case is properly covered and doesn't cause
issues.

> > > > +
> > > > +	if (unlikely(copy_from_iter(&idx, sizeof(idx), i) != sizeof(idx))) {
> > > > +		ret = -EFAULT;
> > > > +		goto out;
> > > > +	}
> > > 
> > > Isn't this copy_from_iter() sleepable function?
> > > 
> > Yes, if we need to fault in data we want the data to come in. This is in
> > the user process context of write() and writev() which I believe is the
> > expected target for copy_from_iter().
> > 
> > During the probe copies we are not holding a lock or have interrupts
> > disabled, under what circumstances would a page fault cause an issue in
> > these paths? What is recommended here if we want to have page faults be
> > allowed (Hoping they are rare)?
> 
> They are rare, but possibly the process is yielded if it is under heavy
> memory pressure. I recommend you to copy the user write data in the stack
> or thread-local buffer at once and pass it to probe functions. That is
> safe way.
> 
> So, can you make it something like below?;
> 
> /* Copy event from user space */
> copy_from_iter(&idx, sizeof(idx), i);
> buf_len = copy_from_iter(local_buf, i->count, &i);
> 
> /* loop on recording functions */
> preempt_disable();
> user_event = get_user_event(file, idx);
> if (user_event && user_event_is_enabled(user_event)) {
> 	user_event_for_each_probe_func(user_event, probe_func, tpdata) {
> 		probe_func(user_event, thread_buf, buf_len, tpdata);
> 	}
> }
> preempt_enable();
> 
Yeah, I will start with the simple version above.

Ultimately I would like to do more of the following in the series:
iov_iter_fault_in_readable(&i, i->count);

And then pass the iter to each probe and use copy_from_iter_nocache.

That would pre-fault in the data and then copy_from_iter_nocache cannot
take a fault and if for some reason it did, it would zero fill.

I can disable migrations instead of preempt disable (eBPF does this).
I could also disable page faults entirely as well to further protect
against this.

> > 
> > > > +				/*
> > > > +				 * Probes advance the iterator so we
> > > > +				 * need to have a copy for each probe.
> > > > +				 */
> > > > +				copy = *i;
> > > > +
> > > > +				probe_func = probe_func_ptr->func;
> > > > +				tpdata = probe_func_ptr->data;
> > > > +				probe_func(user, &copy, tpdata);
> > > 
> > > You seems to try to copy in from user space in each probe func, but
> > > please copy it here to the temporary buffer and pass it to the
> > > each probe function. Such performacne optimization can postpone.
> > > Start with simple implementation.
> > > 
> > Yes, this avoids double copying of the data in the normal paths. Moving
> > to a temp buffer only really changes 1 line in the probe functions
> > (copy_from_iter to copy_from_user).
> > 
> > If I were to create a temp buffer for simplicity I guess I would have to
> > kmalloc on each call or move to a per-cpu buffer or use stack memory and
> > limit how much data can be copied.
> 
> Anyway, it should be limited. You can not write more than 1 page, and
> do you really need it? And allocating kmalloc object is relatively low
> cost compared with a system call.
> 
Really, it's that low?

We are tracking cycles counts to compare user_events with other
telemetry data we have. Some people care a lot about that number, some
don't.

Anyway, I can break it up in the series so we'll have a simple version
and a faster version.

> > > > +	atomic_inc(&user->refs);
> 
> BTW, I recommend you to use a different name for this reference counter,
> e.g. refcnt, since refs is used for "file-local reference user event tables".
> 
Sure.

> > > > +	case DIAG_IOCSREG:
> > > > +		/* Register/lookup on behalf of user process */
> > > > +		name = strndup_user(ubuf, MAX_EVENT_DESC);
> > > > +
> > > > +		if (IS_ERR(name)) {
> > > > +			ret = PTR_ERR(name);
> > > > +			goto out;
> > > > +		}
> > > > +
> > > > +		mutex_lock(&event_mutex);
> > > > +
> > > > +		ret = user_event_parse_cmd(name, &user);
> > > > +
> > > > +		if (ret < 0)
> > > > +			goto reg_out;
> > > > +
> > > > +		preempt_disable();
> > > 
> > > Why do you need to disable preemption here? It is enough to
> > > use a mutex.
> > > 
> > Because user_events_ref_add use rcu protected by sched. It appears that
> > preemption is supposed to be disabled when using that from my read of
> > the code. Is this not correct or not required?
> 
> No not required, the "event_mutex" prevents other user from invoking
> the user_events_ref_add(), thus the "file->private_data" will not be
> updated from others. Thus you don't need to care the scenario of
> "file->private_data" is changed between dereference and reassign it.
> I think what you need is "lockdep_assert_held(&event_mutex)" in the
> user_events_ref_add().
> 
I assume each rcu_dereference_sched should be wrapped with preemption
disabled for a very short period of time? It seems to imply that's
expected. I'm not sure if that's really required or just guidance.

The reason I'm unsure is that the sched held check looks for preemption
being disabled (!preemptible();). rcu_read_lock_sched() disables
preeemption. For our case, I'm not sure if we really need
rcu_read_lock_sched or if we can just dereference it with preemption
disabled.

My guess is I should make these clear and have rcu_read_lock_sched() /
unlock around each rcu_dereference_sched() call, it seems to enable
debugging bad callers easier if things go bad.

Thanks,
-Beau

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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
  2021-10-18 23:09 [PATCH v3] user_events: Enable user processes to create and write to trace events Beau Belgrave
@ 2021-10-26  0:46   ` kernel test robot
  2021-10-26  0:46   ` kernel test robot
  1 sibling, 0 replies; 9+ messages in thread
From: kernel test robot @ 2021-10-26  0:46 UTC (permalink / raw)
  To: Beau Belgrave, rostedt, mhiramat
  Cc: kbuild-all, linux-trace-devel, linux-kernel, beaub

[-- Attachment #1: Type: text/plain, Size: 2888 bytes --]

Hi Beau,

Thank you for the patch! Yet something to improve:

[auto build test ERROR on tip/perf/core]
[If your patch is applied to the wrong git tree, kindly drop us a note.
And when submitting patch, we suggest to use '--base' as documented in
https://git-scm.com/docs/git-format-patch]

url:    https://github.com/0day-ci/linux/commits/Beau-Belgrave/user_events-Enable-user-processes-to-create-and-write-to-trace-events/20211019-071039
base:   https://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git 79df45731da68772d2285265864a52c900b8c65f
config: mips-allyesconfig (attached as .config)
compiler: mips-linux-gcc (GCC) 11.2.0
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # https://github.com/0day-ci/linux/commit/c165bf4b91374ac0ffac76a9f93be466d37a5545
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review Beau-Belgrave/user_events-Enable-user-processes-to-create-and-write-to-trace-events/20211019-071039
        git checkout c165bf4b91374ac0ffac76a9f93be466d37a5545
        # save the attached .config to linux build tree
        mkdir build_dir
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=gcc-11.2.0 make.cross O=build_dir ARCH=mips SHELL=/bin/bash

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All errors (new ones prefixed by >>):

   kernel/trace/trace_events_user.c: In function 'user_status_mmap':
>> kernel/trace/trace_events_user.c:880:38: error: 'PAGE_READONLY' undeclared (first use in this function); did you mean 'MNT_READONLY'?
     880 |                                size, PAGE_READONLY);
         |                                      ^~~~~~~~~~~~~
         |                                      MNT_READONLY
   kernel/trace/trace_events_user.c:880:38: note: each undeclared identifier is reported only once for each function it appears in
   kernel/trace/trace_events_user.c:881:1: error: control reaches end of non-void function [-Werror=return-type]
     881 | }
         | ^
   cc1: some warnings being treated as errors


vim +880 kernel/trace/trace_events_user.c

   867	
   868	/*
   869	 * Maps the shared page into the user process for checking if event is enabled.
   870	 */
   871	static int user_status_mmap(struct file *file, struct vm_area_struct *vma)
   872	{
   873		unsigned long size = vma->vm_end - vma->vm_start;
   874	
   875		if (size != MAX_EVENTS)
   876			return -EINVAL;
   877	
   878		return remap_pfn_range(vma, vma->vm_start,
   879				       virt_to_phys(register_page_data) >> PAGE_SHIFT,
 > 880				       size, PAGE_READONLY);
   881	}
   882	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all@lists.01.org

[-- Attachment #2: .config.gz --]
[-- Type: application/gzip, Size: 70961 bytes --]

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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
@ 2021-10-26  0:46   ` kernel test robot
  0 siblings, 0 replies; 9+ messages in thread
From: kernel test robot @ 2021-10-26  0:46 UTC (permalink / raw)
  To: kbuild-all

[-- Attachment #1: Type: text/plain, Size: 2953 bytes --]

Hi Beau,

Thank you for the patch! Yet something to improve:

[auto build test ERROR on tip/perf/core]
[If your patch is applied to the wrong git tree, kindly drop us a note.
And when submitting patch, we suggest to use '--base' as documented in
https://git-scm.com/docs/git-format-patch]

url:    https://github.com/0day-ci/linux/commits/Beau-Belgrave/user_events-Enable-user-processes-to-create-and-write-to-trace-events/20211019-071039
base:   https://git.kernel.org/pub/scm/linux/kernel/git/tip/tip.git 79df45731da68772d2285265864a52c900b8c65f
config: mips-allyesconfig (attached as .config)
compiler: mips-linux-gcc (GCC) 11.2.0
reproduce (this is a W=1 build):
        wget https://raw.githubusercontent.com/intel/lkp-tests/master/sbin/make.cross -O ~/bin/make.cross
        chmod +x ~/bin/make.cross
        # https://github.com/0day-ci/linux/commit/c165bf4b91374ac0ffac76a9f93be466d37a5545
        git remote add linux-review https://github.com/0day-ci/linux
        git fetch --no-tags linux-review Beau-Belgrave/user_events-Enable-user-processes-to-create-and-write-to-trace-events/20211019-071039
        git checkout c165bf4b91374ac0ffac76a9f93be466d37a5545
        # save the attached .config to linux build tree
        mkdir build_dir
        COMPILER_INSTALL_PATH=$HOME/0day COMPILER=gcc-11.2.0 make.cross O=build_dir ARCH=mips SHELL=/bin/bash

If you fix the issue, kindly add following tag as appropriate
Reported-by: kernel test robot <lkp@intel.com>

All errors (new ones prefixed by >>):

   kernel/trace/trace_events_user.c: In function 'user_status_mmap':
>> kernel/trace/trace_events_user.c:880:38: error: 'PAGE_READONLY' undeclared (first use in this function); did you mean 'MNT_READONLY'?
     880 |                                size, PAGE_READONLY);
         |                                      ^~~~~~~~~~~~~
         |                                      MNT_READONLY
   kernel/trace/trace_events_user.c:880:38: note: each undeclared identifier is reported only once for each function it appears in
   kernel/trace/trace_events_user.c:881:1: error: control reaches end of non-void function [-Werror=return-type]
     881 | }
         | ^
   cc1: some warnings being treated as errors


vim +880 kernel/trace/trace_events_user.c

   867	
   868	/*
   869	 * Maps the shared page into the user process for checking if event is enabled.
   870	 */
   871	static int user_status_mmap(struct file *file, struct vm_area_struct *vma)
   872	{
   873		unsigned long size = vma->vm_end - vma->vm_start;
   874	
   875		if (size != MAX_EVENTS)
   876			return -EINVAL;
   877	
   878		return remap_pfn_range(vma, vma->vm_start,
   879				       virt_to_phys(register_page_data) >> PAGE_SHIFT,
 > 880				       size, PAGE_READONLY);
   881	}
   882	

---
0-DAY CI Kernel Test Service, Intel Corporation
https://lists.01.org/hyperkitty/list/kbuild-all(a)lists.01.org

[-- Attachment #2: config.gz --]
[-- Type: application/gzip, Size: 70961 bytes --]

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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
  2021-10-25 17:26       ` Beau Belgrave
@ 2021-10-26  8:26         ` Masami Hiramatsu
  2021-10-27 19:14           ` Beau Belgrave
  0 siblings, 1 reply; 9+ messages in thread
From: Masami Hiramatsu @ 2021-10-26  8:26 UTC (permalink / raw)
  To: Beau Belgrave
  Cc: rostedt, linux-trace-devel, linux-kernel, Tzvetomir Stoyanov,
	Paul E. McKenney, bpf

On Mon, 25 Oct 2021 10:26:55 -0700
Beau Belgrave <beaub@linux.microsoft.com> wrote:

> On Mon, Oct 25, 2021 at 10:40:06AM +0900, Masami Hiramatsu wrote:
> > On Fri, 22 Oct 2021 15:42:02 -0700
> > Beau Belgrave <beaub@linux.microsoft.com> wrote:
> > 
> > > On Fri, Oct 22, 2021 at 10:38:11PM +0900, Masami Hiramatsu wrote:
> > > The idea is to allow user mode processes to describe their own data
> > > types if required. For that we need offset or size to be passed. If we
> > > require all user mode to only describe in types the kernel understands
> > > then I can remove the size field.
> > 
> > Hmm, I got it. I think it should be optional, and only for the types
> > which has "struct" prefix. Basic types like char, int, long etc should
> > keep the size as it is.
> > 
> Ok, sounds good.
> 
> > > Sorry, are you asking for several isolated patches? Or a single patch
> > > with multiple commits to the same file?
> > 
> > I mean the several isolated patches which commit to the same file.
> > (like upgrading it step by step.) You can make it a series.
> > 
> > > Typically after addition I am running a set of tests and benchmarks
> > > against the code, so having full functioning code is required to ensure
> > > I haven't broken anything.
> > 
> > Yes, you can put the test in the end of the series :)
> > 
> > > We are actively using these patches for various projects, which require
> > > ftrace, perf and eBPF to be functional.
> > 
> > The series usually merged at once :) I just asked you to break it
> > down to the several patches, so that I can review and discuss on
> > each feature in different threads.
> > 
> Got it, that's what I thought, but wanted to make sure.
> 
> > > If we don't care about either, then I can remove this. For custom types
> > > we are fine with using __rel_loc instead.
> > 
> > Or, as I said, you can pass the size when it has "struct" prefix. :)
> > The __rel_loc and __data_loc is usually used for storing arrays, especially
> > dynamic size array like null terminated string. But of course you can use
> > it for storing a big data structure.
> > Let's check how the libtraceevent(tools/lib/traceevent) parses the recorded
> > data.
> > 
> Ok, I'll go for struct prefix.
> 
> > > We have several events that describe the print format to use instead of
> > > using the default, like we allow in the kernel. This is currently how it
> > > is passed, it could be described in the new struct to the reg ioctl
> > > instead.
> > 
> > OK, the 'print_fmt' in 'format' file will show how the trace entry will be
> > printed in the trace buffer (in human readable style). But that is not
> > done automatically. You have to write the corresponding printout function
> > for the event. This means that user-specific format needs to be verified
> > and you need a programmable printout function. This is why it is hard to
> > support the custom data structure in ftrace.
> > Anyway, I think you'd better to start without this option. (and please
> > start with describing how the user-event arguments are shown in trace
> > buffer.)
> > 
> Got it, by that I assume you mean auto-generating a print_fmt statement
> for the user like the other probes. I will do that.

Yes, thanks :)

> 
> > > 
> > > > > +	} else if (strstr(field, "flag ") == field) {
> > > > > +		field += sizeof("flag");
> > > > > +
> > > > > +		if (!strcmp(field, "bpf_iter"))
> > > > > +			user->flags |= FLAG_BPF_ITER;
> > > > > +
> > > > 
> > > > What is this flag?
> > > > 
> > > We want to enable certain sensitive events the ability to mark that
> > > there should never be a buffer copy. When FLAG_BPF_ITER is used the raw
> > > iovecs are exposed out to eBPF instead of any sort of copy to reduce
> > > latency. We run user_events in some highly performant code and want to
> > > monitor things with the least amount of overhead possible.
> > 
> > Would you mean the event with this flag is only available from eBPF?
> > 
> It means that if eBPF attaches we will honor the users request to make
> the data as cheap as possible to them. If a user with proper access
> enables ftrace or perf on these high performant events they will still
> come through (we don't want to hide them).
> 
> We will not be able to do that at all if we copy to heap or stack. At
> that point we've lost the ability to delay copy/probing up until the
> eBPF states it is actually required.

I think the bpf optimization should be discussed in the other thread.

Anyway, here I would like to know is that the syntax of this flag. 
If the flag is for the user event itself, it would be better to add the flag
with a special separator, not the "flag", so that user puts the flags
after fieldN.

name[:FLAG1[,FLAG2...]] [field1[;field2...]] 

> 
> > > > > +	return strcmp(EVENT_NAME(user), event) == 0 &&
> > > > > +		(!system || strcmp(system, USER_EVENTS_SYSTEM) == 0);
> > > > 
> > > > For the minimal implementation, this is OK.
> > > > 
> > > > Note that it is better to check the rest of arguments with the event
> > > > parameters. E.g. if you have "myuevent int var", user cant try to remove
> > > > "-:myuevent char var".
> > > > 
> > > Ok, the reason we do this is because the events are going to be
> > > versioned. They might change their arguments and having a way to clear
> > > without knowing the exact arguments is not a bad thing in our cases.
> > 
> > Actually, as other dynamic events do, user can remove only by name, or
> > specifically with the arguments. So argument check is optional but
> > if user specified it, we should verify it.
> > 
> Ok, I will look into that.
> 
> > > 
> > > No one can remove an event while it's being used and the name cannot
> > > conflict since all user events go to the user_events sub-system and
> > > cannot be overriden.
> > > 
> > > > > +/*
> > > > > + * Validates the user payload and writes via iterator.
> > > > > + */
> > > > > +static ssize_t user_events_write_core(struct file *file, struct iov_iter *i)
> > > > > +{
> > > > > +	struct user_event_refs *refs;
> > > > > +	struct user_event *user = NULL;
> > > > > +	struct tracepoint *tp;
> > > > > +	ssize_t ret = i->count;
> > > > > +	int idx;
> > > > > +
> > > > > +	preempt_disable();
> > > > 
> > > > Why do you disable preemption here?
> > > > 
> > > My understanding was that preemption is to be disabled when using rcu
> > > protected by sched, which tracepoints and the refs are. We also want the
> > > correct processor to log the event (as near as we can).
> > > 
> > > Did I misunderstand rcu_dereference_sched_check()? I may be off in the
> > > weeds.
> > 
> > I see that you want to fix the processor id, but preempt_disable()
> > section should be minimum as possible and should not involve the
> > function which can access to user memeory.
> > 
> I guess I could use disable migration or just live with the rare event
> of a processor swap.

I think you can keep using preempt_disable(), of course it should avoid
including faultable code.

> 
> > > 
> > > I also ran with CONFIG_PROVE_RCU and didn't see anything show up in
> > > dmesg.
> > 
> > Hmm, that's strange, because copy_from_iter(user) may cause a fault
> > and yielded. Isn't it an iovec?
> > 
> Yeah, likely I just haven't hit a page fault case. I'll try to force one
> in our testing to ensure this case is properly covered and doesn't cause
> issues.

If you can suppress the fault (just skip copying when the fault occurs),
I think it is OK. e.g. copy_from_user_nofault().

> > > > > +
> > > > > +	if (unlikely(copy_from_iter(&idx, sizeof(idx), i) != sizeof(idx))) {
> > > > > +		ret = -EFAULT;
> > > > > +		goto out;
> > > > > +	}
> > > > 
> > > > Isn't this copy_from_iter() sleepable function?
> > > > 
> > > Yes, if we need to fault in data we want the data to come in. This is in
> > > the user process context of write() and writev() which I believe is the
> > > expected target for copy_from_iter().
> > > 
> > > During the probe copies we are not holding a lock or have interrupts
> > > disabled, under what circumstances would a page fault cause an issue in
> > > these paths? What is recommended here if we want to have page faults be
> > > allowed (Hoping they are rare)?
> > 
> > They are rare, but possibly the process is yielded if it is under heavy
> > memory pressure. I recommend you to copy the user write data in the stack
> > or thread-local buffer at once and pass it to probe functions. That is
> > safe way.
> > 
> > So, can you make it something like below?;
> > 
> > /* Copy event from user space */
> > copy_from_iter(&idx, sizeof(idx), i);
> > buf_len = copy_from_iter(local_buf, i->count, &i);
> > 
> > /* loop on recording functions */
> > preempt_disable();
> > user_event = get_user_event(file, idx);
> > if (user_event && user_event_is_enabled(user_event)) {
> > 	user_event_for_each_probe_func(user_event, probe_func, tpdata) {
> > 		probe_func(user_event, thread_buf, buf_len, tpdata);
> > 	}
> > }
> > preempt_enable();
> > 
> Yeah, I will start with the simple version above.
> 
> Ultimately I would like to do more of the following in the series:
> iov_iter_fault_in_readable(&i, i->count);
> 
> And then pass the iter to each probe and use copy_from_iter_nocache.

Good, that's easier for me to focus on each topic. :-)

> 
> That would pre-fault in the data and then copy_from_iter_nocache cannot
> take a fault and if for some reason it did, it would zero fill.

Shouldn't we drop the whole event? or leave the null data?
Anyway, it is switchable by the tracing option. For making histogram,
it is better to drop the null data. But for recording RAS data, user
might want to keep the event 'existance' recorded.

> I can disable migrations instead of preempt disable (eBPF does this).
> I could also disable page faults entirely as well to further protect
> against this.

yes, in that case it is OK to me.

> 
> > > 
> > > > > +				/*
> > > > > +				 * Probes advance the iterator so we
> > > > > +				 * need to have a copy for each probe.
> > > > > +				 */
> > > > > +				copy = *i;
> > > > > +
> > > > > +				probe_func = probe_func_ptr->func;
> > > > > +				tpdata = probe_func_ptr->data;
> > > > > +				probe_func(user, &copy, tpdata);
> > > > 
> > > > You seems to try to copy in from user space in each probe func, but
> > > > please copy it here to the temporary buffer and pass it to the
> > > > each probe function. Such performacne optimization can postpone.
> > > > Start with simple implementation.
> > > > 
> > > Yes, this avoids double copying of the data in the normal paths. Moving
> > > to a temp buffer only really changes 1 line in the probe functions
> > > (copy_from_iter to copy_from_user).
> > > 
> > > If I were to create a temp buffer for simplicity I guess I would have to
> > > kmalloc on each call or move to a per-cpu buffer or use stack memory and
> > > limit how much data can be copied.
> > 
> > Anyway, it should be limited. You can not write more than 1 page, and
> > do you really need it? And allocating kmalloc object is relatively low
> > cost compared with a system call.
> > 
> Really, it's that low?
> 
> We are tracking cycles counts to compare user_events with other
> telemetry data we have. Some people care a lot about that number, some
> don't.

OK, then you can use a static per-cpu buffer for copying.

> 
> Anyway, I can break it up in the series so we'll have a simple version
> and a faster version.
> 
> > > > > +	atomic_inc(&user->refs);
> > 
> > BTW, I recommend you to use a different name for this reference counter,
> > e.g. refcnt, since refs is used for "file-local reference user event tables".
> > 
> Sure.
> 
> > > > > +	case DIAG_IOCSREG:
> > > > > +		/* Register/lookup on behalf of user process */
> > > > > +		name = strndup_user(ubuf, MAX_EVENT_DESC);
> > > > > +
> > > > > +		if (IS_ERR(name)) {
> > > > > +			ret = PTR_ERR(name);
> > > > > +			goto out;
> > > > > +		}
> > > > > +
> > > > > +		mutex_lock(&event_mutex);
> > > > > +
> > > > > +		ret = user_event_parse_cmd(name, &user);
> > > > > +
> > > > > +		if (ret < 0)
> > > > > +			goto reg_out;
> > > > > +
> > > > > +		preempt_disable();
> > > > 
> > > > Why do you need to disable preemption here? It is enough to
> > > > use a mutex.
> > > > 
> > > Because user_events_ref_add use rcu protected by sched. It appears that
> > > preemption is supposed to be disabled when using that from my read of
> > > the code. Is this not correct or not required?
> > 
> > No not required, the "event_mutex" prevents other user from invoking
> > the user_events_ref_add(), thus the "file->private_data" will not be
> > updated from others. Thus you don't need to care the scenario of
> > "file->private_data" is changed between dereference and reassign it.
> > I think what you need is "lockdep_assert_held(&event_mutex)" in the
> > user_events_ref_add().
> > 
> I assume each rcu_dereference_sched should be wrapped with preemption
> disabled for a very short period of time? It seems to imply that's
> expected. I'm not sure if that's really required or just guidance.

Hm, indeed. rcu_dereference_sched() requires rcu_read_lock() but the
rcu_assign_pointer() doesn't. I guess we don't need rcu_dereference_sched()
for update side because we are sure no others update it. (and event_mutex
makes a memory barrier) Maybe READ_ONCE() is enough??

I mean if the update is protected by the 'event_mutex', we don't need
to disable preemption, because no one can update that. Or, if you concern
about warning, maybe you can write it as;

static int user_events_ref_add(struct file *file, struct user_event *user)
{
...
	lockdep_assert_held(&event_mutex);
	rcu_read_lock_sched();
        refs = rcu_dereference_sched(file->private_data);
	rcu_read_unlock_sched();

        if (refs) {
...
        rcu_assign_pointer(file->private_data, new_refs);


> 
> The reason I'm unsure is that the sched held check looks for preemption
> being disabled (!preemptible();). rcu_read_lock_sched() disables
> preeemption. For our case, I'm not sure if we really need
> rcu_read_lock_sched or if we can just dereference it with preemption
> disabled.

Yes, so you can just disable preemption only when accessing file->private_data as above :)

> 
> My guess is I should make these clear and have rcu_read_lock_sched() /
> unlock around each rcu_dereference_sched() call, it seems to enable
> debugging bad callers easier if things go bad.

Yeah, for the probing function side, you can keep using the preempt_disable().
For the other parts, rcu_read_lock_sched() will be more clear.

Thank you,

-- 
Masami Hiramatsu <mhiramat@kernel.org>

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

* Re: [PATCH v3] user_events: Enable user processes to create and write to trace events
  2021-10-26  8:26         ` Masami Hiramatsu
@ 2021-10-27 19:14           ` Beau Belgrave
  0 siblings, 0 replies; 9+ messages in thread
From: Beau Belgrave @ 2021-10-27 19:14 UTC (permalink / raw)
  To: Masami Hiramatsu
  Cc: rostedt, linux-trace-devel, linux-kernel, Tzvetomir Stoyanov,
	Paul E. McKenney, bpf

On Tue, Oct 26, 2021 at 05:26:02PM +0900, Masami Hiramatsu wrote:
> > > > > > +	} else if (strstr(field, "flag ") == field) {
> > > > > > +		field += sizeof("flag");
> > > > > > +
> > > > > > +		if (!strcmp(field, "bpf_iter"))
> > > > > > +			user->flags |= FLAG_BPF_ITER;
> > > > > > +
> > > > > 
> > > > > What is this flag?
> > > > > 
> > > > We want to enable certain sensitive events the ability to mark that
> > > > there should never be a buffer copy. When FLAG_BPF_ITER is used the raw
> > > > iovecs are exposed out to eBPF instead of any sort of copy to reduce
> > > > latency. We run user_events in some highly performant code and want to
> > > > monitor things with the least amount of overhead possible.
> > > 
> > > Would you mean the event with this flag is only available from eBPF?
> > > 
> > It means that if eBPF attaches we will honor the users request to make
> > the data as cheap as possible to them. If a user with proper access
> > enables ftrace or perf on these high performant events they will still
> > come through (we don't want to hide them).
> > 
> > We will not be able to do that at all if we copy to heap or stack. At
> > that point we've lost the ability to delay copy/probing up until the
> > eBPF states it is actually required.
> 
> I think the bpf optimization should be discussed in the other thread.
> 

Yep

> Anyway, here I would like to know is that the syntax of this flag. 
> If the flag is for the user event itself, it would be better to add the flag
> with a special separator, not the "flag", so that user puts the flags
> after fieldN.
> 
> name[:FLAG1[,FLAG2...]] [field1[;field2...]] 
> 

Agreed, will do that.

> > > > I also ran with CONFIG_PROVE_RCU and didn't see anything show up in
> > > > dmesg.
> > > 
> > > Hmm, that's strange, because copy_from_iter(user) may cause a fault
> > > and yielded. Isn't it an iovec?
> > > 
> > Yeah, likely I just haven't hit a page fault case. I'll try to force one
> > in our testing to ensure this case is properly covered and doesn't cause
> > issues.
> 
> If you can suppress the fault (just skip copying when the fault occurs),
> I think it is OK. e.g. copy_from_user_nofault().
> 

We want to handle faults in these paths, which means handling them
outside of preemption disabled. This limits where we can have the buffer.

> > > > > > +				/*
> > > > > > +				 * Probes advance the iterator so we
> > > > > > +				 * need to have a copy for each probe.
> > > > > > +				 */
> > > > > > +				copy = *i;
> > > > > > +
> > > > > > +				probe_func = probe_func_ptr->func;
> > > > > > +				tpdata = probe_func_ptr->data;
> > > > > > +				probe_func(user, &copy, tpdata);
> > > > > 
> > > > > You seems to try to copy in from user space in each probe func, but
> > > > > please copy it here to the temporary buffer and pass it to the
> > > > > each probe function. Such performacne optimization can postpone.
> > > > > Start with simple implementation.
> > > > > 
> > > > Yes, this avoids double copying of the data in the normal paths. Moving
> > > > to a temp buffer only really changes 1 line in the probe functions
> > > > (copy_from_iter to copy_from_user).
> > > > 
> > > > If I were to create a temp buffer for simplicity I guess I would have to
> > > > kmalloc on each call or move to a per-cpu buffer or use stack memory and
> > > > limit how much data can be copied.
> > > 
> > > Anyway, it should be limited. You can not write more than 1 page, and
> > > do you really need it? And allocating kmalloc object is relatively low
> > > cost compared with a system call.
> > > 
> > Really, it's that low?
> > 
> > We are tracking cycles counts to compare user_events with other
> > telemetry data we have. Some people care a lot about that number, some
> > don't.
> 
> OK, then you can use a static per-cpu buffer for copying.
> 

I can only use static per-cpu buffers if preemption is disabled during
the copy. This limits to not being able to fault in data. For example
simple migration disabled could still see another user_event getting
traced on the same processor and corrupt / partial fill that per-CPU
buffer.

For the simple version I will use kmalloc and then we can talk on the
other threads about better ways to go about it.

Thanks,
-Beau

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

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

Thread overview: 9+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2021-10-18 23:09 [PATCH v3] user_events: Enable user processes to create and write to trace events Beau Belgrave
2021-10-22 13:38 ` Masami Hiramatsu
2021-10-22 22:42   ` Beau Belgrave
2021-10-25  1:40     ` Masami Hiramatsu
2021-10-25 17:26       ` Beau Belgrave
2021-10-26  8:26         ` Masami Hiramatsu
2021-10-27 19:14           ` Beau Belgrave
2021-10-26  0:46 ` kernel test robot
2021-10-26  0:46   ` kernel test robot

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