linux-trace-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH v8 0/2] Introducing trace buffer mapping by user-space
@ 2023-12-19 18:45 Vincent Donnefort
  2023-12-19 18:45 ` [PATCH v8 1/2] ring-buffer: Introducing ring-buffer mapping functions Vincent Donnefort
                   ` (2 more replies)
  0 siblings, 3 replies; 8+ messages in thread
From: Vincent Donnefort @ 2023-12-19 18:45 UTC (permalink / raw)
  To: rostedt, mhiramat, linux-kernel, linux-trace-kernel
  Cc: kernel-team, Vincent Donnefort

The tracing ring-buffers can be stored on disk or sent to network
without any copy via splice. However the later doesn't allow real time
processing of the traces. A solution is to give userspace direct access
to the ring-buffer pages via a mapping. An application can now become a
consumer of the ring-buffer, in a similar fashion to what trace_pipe
offers.

Attached to this cover letter an example of consuming read for a
ring-buffer, using libtracefs.

Vincent

v7 -> v8:
  * Drop the subbufs renaming into bpages
  * Use subbuf as a name when relevant

v6 -> v7:
  * Rebase onto lore.kernel.org/lkml/20231215175502.106587604@goodmis.org/
  * Support for subbufs
  * Rename subbufs into bpages

v5 -> v6:
  * Rebase on next-20230802.
  * (unsigned long) -> (void *) cast for virt_to_page().
  * Add a wait for the GET_READER_PAGE ioctl.
  * Move writer fields update (overrun/pages_lost/entries/pages_touched)
    in the irq_work.
  * Rearrange id in struct buffer_page.
  * Rearrange the meta-page.
  * ring_buffer_meta_page -> trace_buffer_meta_page.
  * Add meta_struct_len into the meta-page.

v4 -> v5:
  * Trivial rebase onto 6.5-rc3 (previously 6.4-rc3)

v3 -> v4:
  * Add to the meta-page:
       - pages_lost / pages_read (allow to compute how full is the
	 ring-buffer)
       - read (allow to compute how many entries can be read)
       - A reader_page struct.
  * Rename ring_buffer_meta_header -> ring_buffer_meta
  * Rename ring_buffer_get_reader_page -> ring_buffer_map_get_reader_page
  * Properly consume events on ring_buffer_map_get_reader_page() with
    rb_advance_reader().

v2 -> v3:
  * Remove data page list (for non-consuming read)
    ** Implies removing order > 0 meta-page
  * Add a new meta page field ->read
  * Rename ring_buffer_meta_page_header into ring_buffer_meta_header

v1 -> v2:
  * Hide data_pages from the userspace struct
  * Fix META_PAGE_MAX_PAGES
  * Support for order > 0 meta-page
  * Add missing page->mapping.

---

/* Need to access private struct to save counters */
struct kbuffer {
	unsigned long long      timestamp;
	long long               lost_events;
	unsigned long           flags;
	void                    *subbuffer;
	void                    *data;
	unsigned int            index;
	unsigned int            curr;
	unsigned int            next;
	unsigned int            size;
	unsigned int            start;
	unsigned int            first;

	unsigned int (*read_4)(void *ptr);
	unsigned long long (*read_8)(void *ptr);
	unsigned long long (*read_long)(struct kbuffer *kbuf, void *ptr);
	int (*next_event)(struct kbuffer *kbuf);
};

struct trace_buffer_meta {
	unsigned long	entries;
	unsigned long	overrun;
	unsigned long	read;

	unsigned long	subbufs_touched;
	unsigned long	subbufs_lost;
	unsigned long	subbufs_read;

	struct {
		unsigned long	lost_events;	/* Events lost at the time of the reader swap */
		__u32		id;		/* Reader subbuf ID from 0 to nr_subbufs - 1 */
		__u32		read;		/* Number of bytes read on the reader subbuf */
	} reader;

	__u32		subbuf_size;
	__u32		nr_subbufs;		/* Number of subbufs in the ring-buffer */

	__u32		meta_page_size;		/* Size of the meta-page */
	__u32		meta_struct_len;	/* Len of this struct */
};

static char *argv0;
static bool exit_requested;

static char *get_this_name(void)
{
	static char *this_name;
	char *arg;
	char *p;

	if (this_name)
		return this_name;

	arg = argv0;
	p = arg+strlen(arg);

	while (p >= arg && *p != '/')
		p--;
	p++;

	this_name = p;
	return p;
}

static void __vdie(const char *fmt, va_list ap, int err)
{
	int ret = errno;
	char *p = get_this_name();

	if (err && errno)
		perror(p);
	else
		ret = -1;

	fprintf(stderr, "  ");
	vfprintf(stderr, fmt, ap);

	fprintf(stderr, "\n");
	exit(ret);
}

void pdie(const char *fmt, ...)
{
	va_list ap;

	va_start(ap, fmt);
	__vdie(fmt, ap, 1);
	va_end(ap);
}

static void read_subbuf(struct tep_handle *tep, struct kbuffer *kbuf)
{
	static struct trace_seq seq;
	struct tep_record record;

	if (seq.buffer)
		trace_seq_reset(&seq);
	else
		trace_seq_init(&seq);

	while ((record.data = kbuffer_read_event(kbuf, &record.ts))) {
		record.size = kbuffer_event_size(kbuf);
		kbuffer_next_event(kbuf, NULL);
		tep_print_event(tep, &seq, &record,
				"%s-%d %9d\t%s: %s\n", TEP_PRINT_COMM,
				TEP_PRINT_PID, TEP_PRINT_TIME, TEP_PRINT_NAME,
				TEP_PRINT_INFO);
		trace_seq_do_printf(&seq);
		trace_seq_reset(&seq);
	}
}

static int next_reader_subbuf(int fd, struct trace_buffer_meta *meta, unsigned long *read)
{
	__u32 prev_read, prev_reader, new_reader;

	prev_read = READ_ONCE(meta->reader.read);
	prev_reader = READ_ONCE(meta->reader.id);
	if (ioctl(fd, TRACE_MMAP_IOCTL_GET_READER) < 0)
		pdie("ioctl");
	new_reader = READ_ONCE(meta->reader.id);

	if (prev_reader != new_reader)
		*read = 0;
	else
		*read = prev_read;

	return new_reader;
}

static void signal_handler(int unused)
{
	printf("Exit!\n");
	exit_requested = true;
}

int main(int argc, char **argv)
{
	int page_size, meta_len, data_len, subbuf, fd;
	struct trace_buffer_meta *map;
	struct tep_handle *tep;
	struct kbuffer *kbuf;
	unsigned long read;
	void *meta, *data;
	char path[32];
	int cpu;

	if (argc != 2)
		return -EINVAL;

	argv0 = argv[0];
	cpu = atoi(argv[1]);
	snprintf(path, 32, "per_cpu/cpu%d/trace_pipe_raw", cpu);

	tep = tracefs_local_events(NULL);
	kbuf = tep_kbuffer(tep);
	page_size = getpagesize();

	fd = tracefs_instance_file_open(NULL, path, O_RDONLY);
	if (fd < 0)
		pdie("raw");

	meta = mmap(NULL, page_size, PROT_READ, MAP_SHARED, fd, 0);
	if (meta == MAP_FAILED)
		pdie("mmap");
	map = (struct trace_buffer_meta *)meta;
	meta_len = map->meta_page_size;

	printf("entries:	%lu\n", map->entries);
	printf("overrun:	%lu\n", map->overrun);
	printf("read:		%lu\n", map->read);
	printf("subbufs_touched:%lu\n", map->subbufs_touched);
	printf("subbufs_lost:	%lu\n", map->subbufs_lost);
	printf("subbufs_read:	%lu\n", map->subbufs_read);
	printf("nr_subbufs:	%u\n", map->nr_subbufs);

	data_len = map->subbuf_size * map->nr_subbufs;
	data = mmap(NULL, data_len, PROT_READ, MAP_SHARED, fd, meta_len);
	if (data == MAP_FAILED)
		pdie("mmap data");

	signal(SIGINT, signal_handler);

	while (!exit_requested) {
		subbuf = next_reader_subbuf(fd, map, &read);
		kbuffer_load_subbuffer(kbuf, data + map->subbuf_size * subbuf);
		while (kbuf->curr < read)
			kbuffer_next_event(kbuf, NULL);

		read_subbuf(tep, kbuf);
	}

	munmap(data, data_len);
	munmap(meta, page_size);
	close(fd);

	return 0;
}

Vincent Donnefort (2):
  ring-buffer: Introducing ring-buffer mapping functions
  tracing: Allow user-space mapping of the ring-buffer

 include/linux/ring_buffer.h     |   7 +
 include/uapi/linux/trace_mmap.h |  31 +++
 kernel/trace/ring_buffer.c      | 371 +++++++++++++++++++++++++++++++-
 kernel/trace/trace.c            |  79 ++++++-
 4 files changed, 484 insertions(+), 4 deletions(-)
 create mode 100644 include/uapi/linux/trace_mmap.h

-- 
2.43.0.472.g3155946c3a-goog


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

* [PATCH v8 1/2] ring-buffer: Introducing ring-buffer mapping functions
  2023-12-19 18:45 [PATCH v8 0/2] Introducing trace buffer mapping by user-space Vincent Donnefort
@ 2023-12-19 18:45 ` Vincent Donnefort
  2023-12-19 18:45 ` [PATCH v8 2/2] tracing: Allow user-space mapping of the ring-buffer Vincent Donnefort
  2023-12-19 20:39 ` [PATCH v8 0/2] Introducing trace buffer mapping by user-space Steven Rostedt
  2 siblings, 0 replies; 8+ messages in thread
From: Vincent Donnefort @ 2023-12-19 18:45 UTC (permalink / raw)
  To: rostedt, mhiramat, linux-kernel, linux-trace-kernel
  Cc: kernel-team, Vincent Donnefort

In preparation for allowing the user-space to map a ring-buffer, add
a set of mapping functions:

  ring_buffer_{map,unmap}()
  ring_buffer_map_fault()

And controls on the ring-buffer:

  ring_buffer_map_get_reader()  /* swap reader and head */

Mapping the ring-buffer also involves:

  A unique ID for each subbuf of the ring-buffer, currently they are
  only identified through their in-kernel VA.

  A meta-page, where are stored ring-buffer statistics and a
  description for the current reader

The linear mapping exposes the meta-page, and each subbuf of the
ring-buffer, ordered following their unique ID, assigned during the
first mapping.

Once mapped, no subbuf can get in or out of the ring-buffer: the buffer
size will remain unmodified and the splice enabling functions will in
reality simply memcpy the data instead of swapping subbufs.

Signed-off-by: Vincent Donnefort <vdonnefort@google.com>

diff --git a/include/linux/ring_buffer.h b/include/linux/ring_buffer.h
index fa802db216f9..0841ba8bab14 100644
--- a/include/linux/ring_buffer.h
+++ b/include/linux/ring_buffer.h
@@ -6,6 +6,8 @@
 #include <linux/seq_file.h>
 #include <linux/poll.h>
 
+#include <uapi/linux/trace_mmap.h>
+
 struct trace_buffer;
 struct ring_buffer_iter;
 
@@ -221,4 +223,9 @@ int trace_rb_cpu_prepare(unsigned int cpu, struct hlist_node *node);
 #define trace_rb_cpu_prepare	NULL
 #endif
 
+int ring_buffer_map(struct trace_buffer *buffer, int cpu);
+int ring_buffer_unmap(struct trace_buffer *buffer, int cpu);
+struct page *ring_buffer_map_fault(struct trace_buffer *buffer, int cpu,
+				   unsigned long pgoff);
+int ring_buffer_map_get_reader(struct trace_buffer *buffer, int cpu);
 #endif /* _LINUX_RING_BUFFER_H */
diff --git a/include/uapi/linux/trace_mmap.h b/include/uapi/linux/trace_mmap.h
new file mode 100644
index 000000000000..f950648b0ba9
--- /dev/null
+++ b/include/uapi/linux/trace_mmap.h
@@ -0,0 +1,29 @@
+/* SPDX-License-Identifier: GPL-2.0 WITH Linux-syscall-note */
+#ifndef _UAPI_TRACE_MMAP_H_
+#define _UAPI_TRACE_MMAP_H_
+
+#include <linux/types.h>
+
+struct trace_buffer_meta {
+	unsigned long	entries;
+	unsigned long	overrun;
+	unsigned long	read;
+
+	unsigned long	subbufs_touched;
+	unsigned long	subbufs_lost;
+	unsigned long	subbufs_read;
+
+	struct {
+		unsigned long	lost_events;	/* Events lost at the time of the reader swap */
+		__u32		id;		/* Reader subbuf ID from 0 to nr_subbufs - 1 */
+		__u32		read;		/* Number of bytes read on the reader subbuf */
+	} reader;
+
+	__u32		subbuf_size;		/* Size of each subbuf including the header */
+	__u32		nr_subbufs;		/* Number of subbufs in the ring-buffer */
+
+	__u32		meta_page_size;		/* Size of the meta-page */
+	__u32		meta_struct_len;	/* Len of this struct */
+};
+
+#endif /* _UAPI_TRACE_MMAP_H_ */
diff --git a/kernel/trace/ring_buffer.c b/kernel/trace/ring_buffer.c
index 9b95297339b6..ed788721e3c0 100644
--- a/kernel/trace/ring_buffer.c
+++ b/kernel/trace/ring_buffer.c
@@ -337,6 +337,7 @@ struct buffer_page {
 	local_t		 entries;	/* entries on this page */
 	unsigned long	 real_end;	/* real end of data */
 	unsigned	 order;		/* order of the page */
+	u32		 id;		/* ID for external mapping */
 	struct buffer_data_page *page;	/* Actual data page */
 };
 
@@ -483,6 +484,12 @@ struct ring_buffer_per_cpu {
 	u64				read_stamp;
 	/* pages removed since last reset */
 	unsigned long			pages_removed;
+
+	int				mapped;
+	struct mutex			mapping_lock;
+	unsigned long			*subbuf_ids;	/* ID to addr */
+	struct trace_buffer_meta	*meta_page;
+
 	/* ring buffer pages to update, > 0 to add, < 0 to remove */
 	long				nr_pages_to_update;
 	struct list_head		new_pages; /* new pages to add */
@@ -760,6 +767,22 @@ static __always_inline bool full_hit(struct trace_buffer *buffer, int cpu, int f
 	return (dirty * 100) > (full * nr_pages);
 }
 
+static void rb_update_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
+{
+	if (unlikely(READ_ONCE(cpu_buffer->mapped))) {
+		/* Ensure the meta_page is ready */
+		smp_rmb();
+		WRITE_ONCE(cpu_buffer->meta_page->entries,
+			   local_read(&cpu_buffer->entries));
+		WRITE_ONCE(cpu_buffer->meta_page->overrun,
+			   local_read(&cpu_buffer->overrun));
+		WRITE_ONCE(cpu_buffer->meta_page->subbufs_touched,
+			   local_read(&cpu_buffer->pages_touched));
+		WRITE_ONCE(cpu_buffer->meta_page->subbufs_lost,
+			   local_read(&cpu_buffer->pages_lost));
+	}
+}
+
 /*
  * rb_wake_up_waiters - wake up tasks waiting for ring buffer input
  *
@@ -769,6 +792,10 @@ static __always_inline bool full_hit(struct trace_buffer *buffer, int cpu, int f
 static void rb_wake_up_waiters(struct irq_work *work)
 {
 	struct rb_irq_work *rbwork = container_of(work, struct rb_irq_work, work);
+	struct ring_buffer_per_cpu *cpu_buffer =
+		container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
+
+	rb_update_meta_page(cpu_buffer);
 
 	wake_up_all(&rbwork->waiters);
 	if (rbwork->full_waiters_pending || rbwork->wakeup_full) {
@@ -1562,6 +1589,7 @@ rb_allocate_cpu_buffer(struct trace_buffer *buffer, long nr_pages, int cpu)
 	init_irq_work(&cpu_buffer->irq_work.work, rb_wake_up_waiters);
 	init_waitqueue_head(&cpu_buffer->irq_work.waiters);
 	init_waitqueue_head(&cpu_buffer->irq_work.full_waiters);
+	mutex_init(&cpu_buffer->mapping_lock);
 
 	bpage = kzalloc_node(ALIGN(sizeof(*bpage), cache_line_size()),
 			    GFP_KERNEL, cpu_to_node(cpu));
@@ -4474,6 +4502,14 @@ rb_get_reader_page(struct ring_buffer_per_cpu *cpu_buffer)
 		cpu_buffer->last_overrun = overwrite;
 	}
 
+	if (cpu_buffer->mapped) {
+		WRITE_ONCE(cpu_buffer->meta_page->reader.read, 0);
+		WRITE_ONCE(cpu_buffer->meta_page->reader.id, reader->id);
+		WRITE_ONCE(cpu_buffer->meta_page->reader.lost_events, cpu_buffer->lost_events);
+		WRITE_ONCE(cpu_buffer->meta_page->subbufs_read,
+			   local_read(&cpu_buffer->pages_read));
+	}
+
 	goto again;
 
  out:
@@ -4541,6 +4577,12 @@ static void rb_advance_reader(struct ring_buffer_per_cpu *cpu_buffer)
 	length = rb_event_length(event);
 	cpu_buffer->reader_page->read += length;
 	cpu_buffer->read_bytes += length;
+	if (cpu_buffer->mapped) {
+		WRITE_ONCE(cpu_buffer->meta_page->reader.read,
+			   cpu_buffer->reader_page->read);
+		WRITE_ONCE(cpu_buffer->meta_page->read,
+			   cpu_buffer->read);
+	}
 }
 
 static void rb_advance_iter(struct ring_buffer_iter *iter)
@@ -5088,6 +5130,19 @@ static void rb_clear_buffer_page(struct buffer_page *page)
 	page->read = 0;
 }
 
+static void rb_reset_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
+{
+	struct trace_buffer_meta *meta = cpu_buffer->meta_page;
+
+	WRITE_ONCE(meta->entries, 0);
+	WRITE_ONCE(meta->overrun, 0);
+	WRITE_ONCE(meta->read, cpu_buffer->read);
+	WRITE_ONCE(meta->subbufs_touched, 0);
+	WRITE_ONCE(meta->subbufs_lost, 0);
+	WRITE_ONCE(meta->subbufs_read, local_read(&cpu_buffer->pages_read));
+	WRITE_ONCE(meta->reader.read, cpu_buffer->reader_page->read);
+}
+
 static void
 rb_reset_cpu(struct ring_buffer_per_cpu *cpu_buffer)
 {
@@ -5132,6 +5187,9 @@ rb_reset_cpu(struct ring_buffer_per_cpu *cpu_buffer)
 	cpu_buffer->lost_events = 0;
 	cpu_buffer->last_overrun = 0;
 
+	if (cpu_buffer->mapped)
+		rb_reset_meta_page(cpu_buffer);
+
 	rb_head_page_activate(cpu_buffer);
 	cpu_buffer->pages_removed = 0;
 }
@@ -5346,6 +5404,11 @@ int ring_buffer_swap_cpu(struct trace_buffer *buffer_a,
 	cpu_buffer_a = buffer_a->buffers[cpu];
 	cpu_buffer_b = buffer_b->buffers[cpu];
 
+	if (READ_ONCE(cpu_buffer_a->mapped) || READ_ONCE(cpu_buffer_b->mapped)) {
+		ret = -EBUSY;
+		goto out;
+	}
+
 	/* At least make sure the two buffers are somewhat the same */
 	if (cpu_buffer_a->nr_pages != cpu_buffer_b->nr_pages)
 		goto out;
@@ -5609,7 +5672,8 @@ int ring_buffer_read_page(struct trace_buffer *buffer,
 	 * Otherwise, we can simply swap the page with the one passed in.
 	 */
 	if (read || (len < (commit - read)) ||
-	    cpu_buffer->reader_page == cpu_buffer->commit_page) {
+	    cpu_buffer->reader_page == cpu_buffer->commit_page ||
+	    cpu_buffer->mapped) {
 		struct buffer_data_page *rpage = cpu_buffer->reader_page->page;
 		unsigned int rpos = read;
 		unsigned int pos = 0;
@@ -5828,6 +5892,11 @@ int ring_buffer_subbuf_order_set(struct trace_buffer *buffer, int order)
 
 		cpu_buffer = buffer->buffers[cpu];
 
+		if (cpu_buffer->mapped) {
+			err = -EBUSY;
+			goto error;
+		}
+
 		/* Update the number of pages to match the new size */
 		nr_pages = old_size * buffer->buffers[cpu]->nr_pages;
 		nr_pages = DIV_ROUND_UP(nr_pages, buffer->subbuf_size);
@@ -5929,6 +5998,306 @@ int ring_buffer_subbuf_order_set(struct trace_buffer *buffer, int order)
 }
 EXPORT_SYMBOL_GPL(ring_buffer_subbuf_order_set);
 
+#define subbuf_page(off, start) \
+	virt_to_page((void *)(start + (off << PAGE_SHIFT)))
+
+#define foreach_subbuf_page(off, sub_order, start, page)	\
+	for (off = 0, page = subbuf_page(0, start);		\
+	     off < (1 << sub_order);				\
+	     off++, page = subbuf_page(off, start))
+
+static inline void subbuf_map_prepare(unsigned long subbuf_start, int order)
+{
+	struct page *page;
+	int subbuf_off;
+
+	/*
+	 * When allocating order > 0 pages, only the first struct page has a
+	 * refcount > 1. Increasing the refcount here ensures none of the struct
+	 * page composing the sub-buffer is freeed when the mapping is closed.
+	 */
+	foreach_subbuf_page(subbuf_off, order, subbuf_start, page)
+		page_ref_inc(page);
+}
+
+static inline void subbuf_unmap(unsigned long subbuf_start, int order)
+{
+	struct page *page;
+	int subbuf_off;
+
+	foreach_subbuf_page(subbuf_off, order, subbuf_start, page) {
+		page_ref_dec(page);
+		page->mapping = NULL;
+	}
+}
+
+static void rb_free_subbuf_ids(struct ring_buffer_per_cpu *cpu_buffer)
+{
+	int sub_id;
+
+	for (sub_id = 0; sub_id < cpu_buffer->nr_pages + 1; sub_id++)
+		subbuf_unmap(cpu_buffer->subbuf_ids[sub_id],
+			     cpu_buffer->buffer->subbuf_order);
+
+	kfree(cpu_buffer->subbuf_ids);
+	cpu_buffer->subbuf_ids = NULL;
+}
+
+static int rb_alloc_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
+{
+	if (cpu_buffer->meta_page)
+		return 0;
+
+	cpu_buffer->meta_page = page_to_virt(alloc_page(GFP_USER));
+	if (!cpu_buffer->meta_page)
+		return -ENOMEM;
+
+	return 0;
+}
+
+static void rb_free_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
+{
+	unsigned long addr = (unsigned long)cpu_buffer->meta_page;
+
+	virt_to_page((void *)addr)->mapping = NULL;
+	free_page(addr);
+	cpu_buffer->meta_page = NULL;
+}
+
+static void rb_setup_ids_meta_page(struct ring_buffer_per_cpu *cpu_buffer,
+				   unsigned long *subbuf_ids)
+{
+	struct trace_buffer_meta *meta = cpu_buffer->meta_page;
+	unsigned int nr_subbufs = cpu_buffer->nr_pages + 1;
+	struct buffer_page *first_subbuf, *subbuf;
+	int id = 0;
+
+	subbuf_ids[id] = (unsigned long)cpu_buffer->reader_page->page;
+	subbuf_map_prepare(subbuf_ids[id], cpu_buffer->buffer->subbuf_order);
+	cpu_buffer->reader_page->id = id++;
+
+	first_subbuf = subbuf = rb_set_head_page(cpu_buffer);
+	do {
+		if (id >= nr_subbufs) {
+			WARN_ON(1);
+			break;
+		}
+
+		subbuf_ids[id] = (unsigned long)subbuf->page;
+		subbuf->id = id;
+		subbuf_map_prepare(subbuf_ids[id], cpu_buffer->buffer->subbuf_order);
+
+		rb_inc_page(&subbuf);
+		id++;
+	} while (subbuf != first_subbuf);
+
+	/* install subbuf ID to kern VA translation */
+	cpu_buffer->subbuf_ids = subbuf_ids;
+
+	meta->meta_page_size = PAGE_SIZE;
+	meta->meta_struct_len = sizeof(*meta);
+	meta->nr_subbufs = nr_subbufs;
+	meta->subbuf_size = cpu_buffer->buffer->subbuf_size + BUF_PAGE_HDR_SIZE;
+	meta->reader.id = cpu_buffer->reader_page->id;
+	rb_reset_meta_page(cpu_buffer);
+}
+
+static inline struct ring_buffer_per_cpu *
+rb_get_mapped_buffer(struct trace_buffer *buffer, int cpu)
+{
+	struct ring_buffer_per_cpu *cpu_buffer;
+
+	if (!cpumask_test_cpu(cpu, buffer->cpumask))
+		return ERR_PTR(-EINVAL);
+
+	cpu_buffer = buffer->buffers[cpu];
+
+	mutex_lock(&cpu_buffer->mapping_lock);
+
+	if (!cpu_buffer->mapped) {
+		mutex_unlock(&cpu_buffer->mapping_lock);
+		return ERR_PTR(-ENODEV);
+	}
+
+	return cpu_buffer;
+}
+
+static inline void rb_put_mapped_buffer(struct ring_buffer_per_cpu *cpu_buffer)
+{
+	mutex_unlock(&cpu_buffer->mapping_lock);
+}
+
+int ring_buffer_map(struct trace_buffer *buffer, int cpu)
+{
+	struct ring_buffer_per_cpu *cpu_buffer;
+	unsigned long flags, *subbuf_ids;
+	int err = 0;
+
+	if (!cpumask_test_cpu(cpu, buffer->cpumask))
+		return -EINVAL;
+
+	cpu_buffer = buffer->buffers[cpu];
+
+	mutex_lock(&cpu_buffer->mapping_lock);
+
+	if (cpu_buffer->mapped) {
+		WRITE_ONCE(cpu_buffer->mapped, cpu_buffer->mapped + 1);
+		goto unlock;
+	}
+
+	/* prevent another thread from changing buffer sizes */
+	mutex_lock(&buffer->mutex);
+
+	err = rb_alloc_meta_page(cpu_buffer);
+	if (err)
+		goto unlock;
+
+	/* subbuf_ids include the reader while nr_pages does not */
+	subbuf_ids = kzalloc(sizeof(*subbuf_ids) * (cpu_buffer->nr_pages + 1),
+			   GFP_KERNEL);
+	if (!subbuf_ids) {
+		rb_free_meta_page(cpu_buffer);
+		err = -ENOMEM;
+		goto unlock;
+	}
+
+	atomic_inc(&cpu_buffer->resize_disabled);
+
+	/*
+	 * Lock all readers to block any subbuf swap until the subbuf IDs are
+	 * assigned.
+	 */
+	raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+
+	rb_setup_ids_meta_page(cpu_buffer, subbuf_ids);
+	/*
+	 * Ensure rb_update_meta() will observe the meta-page before
+	 * cpu_buffer->mapped.
+	 */
+	smp_wmb();
+	WRITE_ONCE(cpu_buffer->mapped, 1);
+
+	/* Init meta_page values unless the writer did it already */
+	cmpxchg(&cpu_buffer->meta_page->entries, 0,
+		local_read(&cpu_buffer->entries));
+	cmpxchg(&cpu_buffer->meta_page->overrun, 0,
+		local_read(&cpu_buffer->overrun));
+	cmpxchg(&cpu_buffer->meta_page->subbufs_touched, 0,
+		local_read(&cpu_buffer->pages_touched));
+	cmpxchg(&cpu_buffer->meta_page->subbufs_lost, 0,
+		local_read(&cpu_buffer->pages_lost));
+
+	raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+unlock:
+	mutex_unlock(&buffer->mutex);
+	mutex_unlock(&cpu_buffer->mapping_lock);
+
+	return err;
+}
+
+int ring_buffer_unmap(struct trace_buffer *buffer, int cpu)
+{
+	struct ring_buffer_per_cpu *cpu_buffer;
+	int err = 0;
+
+	if (!cpumask_test_cpu(cpu, buffer->cpumask))
+		return -EINVAL;
+
+	cpu_buffer = buffer->buffers[cpu];
+
+	mutex_lock(&cpu_buffer->mapping_lock);
+
+	if (!cpu_buffer->mapped) {
+		err = -ENODEV;
+		goto unlock;
+	}
+
+	WRITE_ONCE(cpu_buffer->mapped, cpu_buffer->mapped - 1);
+	if (!cpu_buffer->mapped) {
+		/* Wait for the writer and readers to observe !mapped */
+		synchronize_rcu();
+
+		rb_free_subbuf_ids(cpu_buffer);
+		rb_free_meta_page(cpu_buffer);
+		atomic_dec(&cpu_buffer->resize_disabled);
+	}
+unlock:
+	mutex_unlock(&cpu_buffer->mapping_lock);
+
+	return err;
+}
+
+/*
+ *   +--------------+  pgoff == 0
+ *   |   meta page  |
+ *   +--------------+  pgoff == 1
+ *   | subbuffer 0  |
+ *   +--------------+  pgoff == 1 + (1 << subbuf_order)
+ *   | subbuffer 1  |
+ *         ...
+ */
+struct page *ring_buffer_map_fault(struct trace_buffer *buffer, int cpu,
+				   unsigned long pgoff)
+{
+	struct ring_buffer_per_cpu *cpu_buffer = buffer->buffers[cpu];
+	unsigned long subbuf_id, subbuf_offset, addr;
+	struct page *page;
+
+	if (!pgoff)
+		return virt_to_page((void *)cpu_buffer->meta_page);
+
+	pgoff--;
+
+	subbuf_id = pgoff >> buffer->subbuf_order;
+	if (subbuf_id > cpu_buffer->nr_pages)
+		return NULL;
+
+	subbuf_offset = pgoff & ((1UL << buffer->subbuf_order) - 1);
+	addr = cpu_buffer->subbuf_ids[subbuf_id] + (subbuf_offset * PAGE_SIZE);
+	page = virt_to_page((void *)addr);
+
+	return page;
+}
+
+int ring_buffer_map_get_reader(struct trace_buffer *buffer, int cpu)
+{
+	struct ring_buffer_per_cpu *cpu_buffer;
+	unsigned long reader_size;
+	unsigned long flags;
+
+	cpu_buffer = rb_get_mapped_buffer(buffer, cpu);
+	if (IS_ERR(cpu_buffer))
+		return (int)PTR_ERR(cpu_buffer);
+
+	raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+consume:
+	if (rb_per_cpu_empty(cpu_buffer))
+		goto out;
+
+	reader_size = rb_page_size(cpu_buffer->reader_page);
+
+	/*
+	 * There are data to be read on the current reader page, we can
+	 * return to the caller. But before that, we assume the latter will read
+	 * everything. Let's update the kernel reader accordingly.
+	 */
+	if (cpu_buffer->reader_page->read < reader_size) {
+		while (cpu_buffer->reader_page->read < reader_size)
+			rb_advance_reader(cpu_buffer);
+		goto out;
+	}
+
+	if (WARN_ON(!rb_get_reader_page(cpu_buffer)))
+		goto out;
+
+	goto consume;
+out:
+	raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+	rb_put_mapped_buffer(cpu_buffer);
+
+	return 0;
+}
+
 /*
  * We only allocate new buffers, never free them if the CPU goes down.
  * If we were to free the buffer, then the user would lose any trace that was in
-- 
2.43.0.472.g3155946c3a-goog


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

* [PATCH v8 2/2] tracing: Allow user-space mapping of the ring-buffer
  2023-12-19 18:45 [PATCH v8 0/2] Introducing trace buffer mapping by user-space Vincent Donnefort
  2023-12-19 18:45 ` [PATCH v8 1/2] ring-buffer: Introducing ring-buffer mapping functions Vincent Donnefort
@ 2023-12-19 18:45 ` Vincent Donnefort
  2023-12-19 20:39 ` [PATCH v8 0/2] Introducing trace buffer mapping by user-space Steven Rostedt
  2 siblings, 0 replies; 8+ messages in thread
From: Vincent Donnefort @ 2023-12-19 18:45 UTC (permalink / raw)
  To: rostedt, mhiramat, linux-kernel, linux-trace-kernel
  Cc: kernel-team, Vincent Donnefort

Currently, user-space extracts data from the ring-buffer via splice,
which is handy for storage or network sharing. However, due to splice
limitations, it is imposible to do real-time analysis without a copy.

A solution for that problem is to let the user-space map the ring-buffer
directly.

The mapping is exposed via the per-CPU file trace_pipe_raw. The first
element of the mapping is the meta-page. It is followed by each
subbuffer constituting the ring-buffer, ordered by their unique page ID:

  * Meta-page -- include/uapi/linux/trace_mmap.h for a description
  * Subbuf ID 0
  * Subbuf ID 1
     ...

It is therefore easy to translate a subbuf ID into an offset in the
mapping:

  reader_id = meta->reader->id;
  reader_offset = meta->meta_page_size + reader_id * meta->subbuf_size;

When new data is available, the mapper must call a newly introduced ioctl:
TRACE_MMAP_IOCTL_GET_READER. This will update the Meta-page reader ID to
point to the next reader containing unread data.

Signed-off-by: Vincent Donnefort <vdonnefort@google.com>

diff --git a/include/uapi/linux/trace_mmap.h b/include/uapi/linux/trace_mmap.h
index f950648b0ba9..8c49489c5867 100644
--- a/include/uapi/linux/trace_mmap.h
+++ b/include/uapi/linux/trace_mmap.h
@@ -26,4 +26,6 @@ struct trace_buffer_meta {
 	__u32		meta_struct_len;	/* Len of this struct */
 };
 
+#define TRACE_MMAP_IOCTL_GET_READER		_IO('T', 0x1)
+
 #endif /* _UAPI_TRACE_MMAP_H_ */
diff --git a/kernel/trace/trace.c b/kernel/trace/trace.c
index b35c85edbb49..8dee225cf977 100644
--- a/kernel/trace/trace.c
+++ b/kernel/trace/trace.c
@@ -8590,15 +8590,31 @@ tracing_buffers_splice_read(struct file *file, loff_t *ppos,
 	return ret;
 }
 
-/* An ioctl call with cmd 0 to the ring buffer file will wake up all waiters */
 static long tracing_buffers_ioctl(struct file *file, unsigned int cmd, unsigned long arg)
 {
 	struct ftrace_buffer_info *info = file->private_data;
 	struct trace_iterator *iter = &info->iter;
+	int err;
 
-	if (cmd)
-		return -ENOIOCTLCMD;
+	if (cmd == TRACE_MMAP_IOCTL_GET_READER) {
+		if (!(file->f_flags & O_NONBLOCK)) {
+			err = ring_buffer_wait(iter->array_buffer->buffer,
+					       iter->cpu_file,
+					       iter->tr->buffer_percent);
+			if (err)
+				return err;
+		}
 
+		return ring_buffer_map_get_reader(iter->array_buffer->buffer,
+						  iter->cpu_file);
+	} else if (cmd) {
+		return -ENOTTY;
+	}
+
+	/*
+	 * An ioctl call with cmd 0 to the ring buffer file will wake up all
+	 * waiters
+	 */
 	mutex_lock(&trace_types_lock);
 
 	iter->wait_index++;
@@ -8611,6 +8627,62 @@ static long tracing_buffers_ioctl(struct file *file, unsigned int cmd, unsigned
 	return 0;
 }
 
+static vm_fault_t tracing_buffers_mmap_fault(struct vm_fault *vmf)
+{
+	struct ftrace_buffer_info *info = vmf->vma->vm_file->private_data;
+	struct trace_iterator *iter = &info->iter;
+	vm_fault_t ret = VM_FAULT_SIGBUS;
+	struct page *page;
+
+	page = ring_buffer_map_fault(iter->array_buffer->buffer, iter->cpu_file,
+				     vmf->pgoff);
+	if (!page)
+		return ret;
+
+	get_page(page);
+	vmf->page = page;
+	vmf->page->mapping = vmf->vma->vm_file->f_mapping;
+	vmf->page->index = vmf->pgoff;
+
+	return 0;
+}
+
+static void tracing_buffers_mmap_close(struct vm_area_struct *vma)
+{
+	struct ftrace_buffer_info *info = vma->vm_file->private_data;
+	struct trace_iterator *iter = &info->iter;
+
+	ring_buffer_unmap(iter->array_buffer->buffer, iter->cpu_file);
+}
+
+static void tracing_buffers_mmap_open(struct vm_area_struct *vma)
+{
+	struct ftrace_buffer_info *info = vma->vm_file->private_data;
+	struct trace_iterator *iter = &info->iter;
+
+	WARN_ON(ring_buffer_map(iter->array_buffer->buffer, iter->cpu_file));
+}
+
+static const struct vm_operations_struct tracing_buffers_vmops = {
+	.open		= tracing_buffers_mmap_open,
+	.close		= tracing_buffers_mmap_close,
+	.fault		= tracing_buffers_mmap_fault,
+};
+
+static int tracing_buffers_mmap(struct file *filp, struct vm_area_struct *vma)
+{
+	struct ftrace_buffer_info *info = filp->private_data;
+	struct trace_iterator *iter = &info->iter;
+
+	if (vma->vm_flags & VM_WRITE)
+		return -EPERM;
+
+	vm_flags_mod(vma, VM_DONTCOPY | VM_DONTDUMP, VM_MAYWRITE);
+	vma->vm_ops = &tracing_buffers_vmops;
+
+	return ring_buffer_map(iter->array_buffer->buffer, iter->cpu_file);
+}
+
 static const struct file_operations tracing_buffers_fops = {
 	.open		= tracing_buffers_open,
 	.read		= tracing_buffers_read,
@@ -8619,6 +8691,7 @@ static const struct file_operations tracing_buffers_fops = {
 	.splice_read	= tracing_buffers_splice_read,
 	.unlocked_ioctl = tracing_buffers_ioctl,
 	.llseek		= no_llseek,
+	.mmap		= tracing_buffers_mmap,
 };
 
 static ssize_t
-- 
2.43.0.472.g3155946c3a-goog


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

* Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space
  2023-12-19 18:45 [PATCH v8 0/2] Introducing trace buffer mapping by user-space Vincent Donnefort
  2023-12-19 18:45 ` [PATCH v8 1/2] ring-buffer: Introducing ring-buffer mapping functions Vincent Donnefort
  2023-12-19 18:45 ` [PATCH v8 2/2] tracing: Allow user-space mapping of the ring-buffer Vincent Donnefort
@ 2023-12-19 20:39 ` Steven Rostedt
  2023-12-20 13:06   ` Vincent Donnefort
  2 siblings, 1 reply; 8+ messages in thread
From: Steven Rostedt @ 2023-12-19 20:39 UTC (permalink / raw)
  To: Vincent Donnefort; +Cc: mhiramat, linux-kernel, linux-trace-kernel, kernel-team

On Tue, 19 Dec 2023 18:45:54 +0000
Vincent Donnefort <vdonnefort@google.com> wrote:

> The tracing ring-buffers can be stored on disk or sent to network
> without any copy via splice. However the later doesn't allow real time
> processing of the traces. A solution is to give userspace direct access
> to the ring-buffer pages via a mapping. An application can now become a
> consumer of the ring-buffer, in a similar fashion to what trace_pipe
> offers.
> 
> Attached to this cover letter an example of consuming read for a
> ring-buffer, using libtracefs.
> 

I'm still testing this, but I needed to add this patch to fix two bugs. One
is that you are calling rb_wakeup_waiters() for both the buffer and the
cpu_buffer, and it needs to know which one to use the container_of() macro.

The other is a "goto unlock" that unlocks two locks where only one was taken.

-- Steve

diff --git a/kernel/trace/ring_buffer.c b/kernel/trace/ring_buffer.c
index 35f3736f660b..987ad7bd1e8b 100644
--- a/kernel/trace/ring_buffer.c
+++ b/kernel/trace/ring_buffer.c
@@ -389,6 +389,7 @@ struct rb_irq_work {
 	bool				waiters_pending;
 	bool				full_waiters_pending;
 	bool				wakeup_full;
+	bool				is_cpu_buffer;
 };
 
 /*
@@ -771,10 +772,20 @@ static void rb_update_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
 static void rb_wake_up_waiters(struct irq_work *work)
 {
 	struct rb_irq_work *rbwork = container_of(work, struct rb_irq_work, work);
-	struct ring_buffer_per_cpu *cpu_buffer =
-		container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
+	struct ring_buffer_per_cpu *cpu_buffer;
+	struct trace_buffer *buffer;
+	int cpu;
 
-	rb_update_meta_page(cpu_buffer);
+	if (rbwork->is_cpu_buffer) {
+		cpu_buffer = container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
+		rb_update_meta_page(cpu_buffer);
+	} else {
+		buffer = container_of(rbwork, struct trace_buffer, irq_work);
+		for_each_buffer_cpu(buffer, cpu) {
+			cpu_buffer = buffer->buffers[cpu];
+			rb_update_meta_page(cpu_buffer);
+		}
+	}
 
 	wake_up_all(&rbwork->waiters);
 	if (rbwork->full_waiters_pending || rbwork->wakeup_full) {
@@ -1569,6 +1580,7 @@ rb_allocate_cpu_buffer(struct trace_buffer *buffer, long nr_pages, int cpu)
 	init_waitqueue_head(&cpu_buffer->irq_work.waiters);
 	init_waitqueue_head(&cpu_buffer->irq_work.full_waiters);
 	mutex_init(&cpu_buffer->mapping_lock);
+	cpu_buffer->irq_work.is_cpu_buffer = true;
 
 	bpage = kzalloc_node(ALIGN(sizeof(*bpage), cache_line_size()),
 			    GFP_KERNEL, cpu_to_node(cpu));
@@ -6209,7 +6221,8 @@ int ring_buffer_map(struct trace_buffer *buffer, int cpu)
 
 	if (cpu_buffer->mapped) {
 		WRITE_ONCE(cpu_buffer->mapped, cpu_buffer->mapped + 1);
-		goto unlock;
+		mutex_unlock(&cpu_buffer->mapping_lock);
+		return 0;
 	}
 
 	/* prevent another thread from changing buffer sizes */

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

* Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space
  2023-12-19 20:39 ` [PATCH v8 0/2] Introducing trace buffer mapping by user-space Steven Rostedt
@ 2023-12-20 13:06   ` Vincent Donnefort
  2023-12-20 13:29     ` Steven Rostedt
  0 siblings, 1 reply; 8+ messages in thread
From: Vincent Donnefort @ 2023-12-20 13:06 UTC (permalink / raw)
  To: Steven Rostedt; +Cc: mhiramat, linux-kernel, linux-trace-kernel, kernel-team

On Tue, Dec 19, 2023 at 03:39:24PM -0500, Steven Rostedt wrote:
> On Tue, 19 Dec 2023 18:45:54 +0000
> Vincent Donnefort <vdonnefort@google.com> wrote:
> 
> > The tracing ring-buffers can be stored on disk or sent to network
> > without any copy via splice. However the later doesn't allow real time
> > processing of the traces. A solution is to give userspace direct access
> > to the ring-buffer pages via a mapping. An application can now become a
> > consumer of the ring-buffer, in a similar fashion to what trace_pipe
> > offers.
> > 
> > Attached to this cover letter an example of consuming read for a
> > ring-buffer, using libtracefs.
> > 
> 
> I'm still testing this, but I needed to add this patch to fix two bugs. One
> is that you are calling rb_wakeup_waiters() for both the buffer and the
> cpu_buffer, and it needs to know which one to use the container_of() macro.
> 
> The other is a "goto unlock" that unlocks two locks where only one was taken.
> 
> -- Steve
> 
> diff --git a/kernel/trace/ring_buffer.c b/kernel/trace/ring_buffer.c
> index 35f3736f660b..987ad7bd1e8b 100644
> --- a/kernel/trace/ring_buffer.c
> +++ b/kernel/trace/ring_buffer.c
> @@ -389,6 +389,7 @@ struct rb_irq_work {
>  	bool				waiters_pending;
>  	bool				full_waiters_pending;
>  	bool				wakeup_full;
> +	bool				is_cpu_buffer;
>  };
>  
>  /*
> @@ -771,10 +772,20 @@ static void rb_update_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
>  static void rb_wake_up_waiters(struct irq_work *work)
>  {
>  	struct rb_irq_work *rbwork = container_of(work, struct rb_irq_work, work);
> -	struct ring_buffer_per_cpu *cpu_buffer =
> -		container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
> +	struct ring_buffer_per_cpu *cpu_buffer;
> +	struct trace_buffer *buffer;
> +	int cpu;
>  
> -	rb_update_meta_page(cpu_buffer);
> +	if (rbwork->is_cpu_buffer) {
> +		cpu_buffer = container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
> +		rb_update_meta_page(cpu_buffer);
> +	} else {
> +		buffer = container_of(rbwork, struct trace_buffer, irq_work);
> +		for_each_buffer_cpu(buffer, cpu) {
> +			cpu_buffer = buffer->buffers[cpu];
> +			rb_update_meta_page(cpu_buffer);
> +		}
> +	}

Arg, somehow never reproduced the problem :-\. I suppose you need to cat
trace/trace_pipe and mmap(trace/cpuX/trace_pipe) at the same time?

Updating the meta-page is only useful if the reader we are waking up is a
user-space one, which would only happen with the cpu_buffer version of this
function. We could limit the update of the meta_page only to this case?

>  
>  	wake_up_all(&rbwork->waiters);
>  	if (rbwork->full_waiters_pending || rbwork->wakeup_full) {
> @@ -1569,6 +1580,7 @@ rb_allocate_cpu_buffer(struct trace_buffer *buffer, long nr_pages, int cpu)
>  	init_waitqueue_head(&cpu_buffer->irq_work.waiters);
>  	init_waitqueue_head(&cpu_buffer->irq_work.full_waiters);
>  	mutex_init(&cpu_buffer->mapping_lock);
> +	cpu_buffer->irq_work.is_cpu_buffer = true;
>  
>  	bpage = kzalloc_node(ALIGN(sizeof(*bpage), cache_line_size()),
>  			    GFP_KERNEL, cpu_to_node(cpu));
> @@ -6209,7 +6221,8 @@ int ring_buffer_map(struct trace_buffer *buffer, int cpu)
>  
>  	if (cpu_buffer->mapped) {
>  		WRITE_ONCE(cpu_buffer->mapped, cpu_buffer->mapped + 1);
> -		goto unlock;
> +		mutex_unlock(&cpu_buffer->mapping_lock);
> +		return 0;
>  	}
>  
>  	/* prevent another thread from changing buffer sizes */

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

* Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space
  2023-12-20 13:06   ` Vincent Donnefort
@ 2023-12-20 13:29     ` Steven Rostedt
  2023-12-20 13:49       ` Vincent Donnefort
  0 siblings, 1 reply; 8+ messages in thread
From: Steven Rostedt @ 2023-12-20 13:29 UTC (permalink / raw)
  To: Vincent Donnefort; +Cc: mhiramat, linux-kernel, linux-trace-kernel, kernel-team

On Wed, 20 Dec 2023 13:06:06 +0000
Vincent Donnefort <vdonnefort@google.com> wrote:

> > @@ -771,10 +772,20 @@ static void rb_update_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
> >  static void rb_wake_up_waiters(struct irq_work *work)
> >  {
> >  	struct rb_irq_work *rbwork = container_of(work, struct rb_irq_work, work);
> > -	struct ring_buffer_per_cpu *cpu_buffer =
> > -		container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
> > +	struct ring_buffer_per_cpu *cpu_buffer;
> > +	struct trace_buffer *buffer;
> > +	int cpu;
> >  
> > -	rb_update_meta_page(cpu_buffer);
> > +	if (rbwork->is_cpu_buffer) {
> > +		cpu_buffer = container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
> > +		rb_update_meta_page(cpu_buffer);
> > +	} else {
> > +		buffer = container_of(rbwork, struct trace_buffer, irq_work);
> > +		for_each_buffer_cpu(buffer, cpu) {
> > +			cpu_buffer = buffer->buffers[cpu];
> > +			rb_update_meta_page(cpu_buffer);
> > +		}
> > +	}  
> 
> Arg, somehow never reproduced the problem :-\. I suppose you need to cat
> trace/trace_pipe and mmap(trace/cpuX/trace_pipe) at the same time?

It triggered as soon as I ran "trace-cmd start -e sched_switch"

In other words, it broke the non mmap case. This function gets called for
both the buffer and cpu_buffer irq_work entries. You added the
container_of() to get access to cpu_buffer, when the rbwork could also be
for the main buffer too. The main buffer has no meta page, and it triggered
a NULL pointer dereference, as "cpu_buffer->mapped" returned true (because
it was on something of the buffer structure that wasn't zero), and then here:

	if (cpu_buffer->mapped) {
		WRITE_ONCE(cpu_buffer->meta_page->reader.read, 0);

It dereferenced cpu_buffer->meta_page->reader

which is only God knows what!

> 
> Updating the meta-page is only useful if the reader we are waking up is a
> user-space one, which would only happen with the cpu_buffer version of this
> function. We could limit the update of the meta_page only to this case?

I rather not add another irq_work entry. This workaround should be good
enough.

Thanks,

-- Steve

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

* Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space
  2023-12-20 13:29     ` Steven Rostedt
@ 2023-12-20 13:49       ` Vincent Donnefort
  2023-12-20 16:02         ` Steven Rostedt
  0 siblings, 1 reply; 8+ messages in thread
From: Vincent Donnefort @ 2023-12-20 13:49 UTC (permalink / raw)
  To: Steven Rostedt; +Cc: mhiramat, linux-kernel, linux-trace-kernel, kernel-team

On Wed, Dec 20, 2023 at 08:29:32AM -0500, Steven Rostedt wrote:
> On Wed, 20 Dec 2023 13:06:06 +0000
> Vincent Donnefort <vdonnefort@google.com> wrote:
> 
> > > @@ -771,10 +772,20 @@ static void rb_update_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
> > >  static void rb_wake_up_waiters(struct irq_work *work)
> > >  {
> > >  	struct rb_irq_work *rbwork = container_of(work, struct rb_irq_work, work);
> > > -	struct ring_buffer_per_cpu *cpu_buffer =
> > > -		container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
> > > +	struct ring_buffer_per_cpu *cpu_buffer;
> > > +	struct trace_buffer *buffer;
> > > +	int cpu;
> > >  
> > > -	rb_update_meta_page(cpu_buffer);
> > > +	if (rbwork->is_cpu_buffer) {
> > > +		cpu_buffer = container_of(rbwork, struct ring_buffer_per_cpu, irq_work);
> > > +		rb_update_meta_page(cpu_buffer);
> > > +	} else {
> > > +		buffer = container_of(rbwork, struct trace_buffer, irq_work);
> > > +		for_each_buffer_cpu(buffer, cpu) {
> > > +			cpu_buffer = buffer->buffers[cpu];
> > > +			rb_update_meta_page(cpu_buffer);
> > > +		}
> > > +	}  
> > 
> > Arg, somehow never reproduced the problem :-\. I suppose you need to cat
> > trace/trace_pipe and mmap(trace/cpuX/trace_pipe) at the same time?
> 
> It triggered as soon as I ran "trace-cmd start -e sched_switch"
> 
> In other words, it broke the non mmap case. This function gets called for
> both the buffer and cpu_buffer irq_work entries. You added the
> container_of() to get access to cpu_buffer, when the rbwork could also be
> for the main buffer too. The main buffer has no meta page, and it triggered
> a NULL pointer dereference, as "cpu_buffer->mapped" returned true (because
> it was on something of the buffer structure that wasn't zero), and then here:
> 
> 	if (cpu_buffer->mapped) {
> 		WRITE_ONCE(cpu_buffer->meta_page->reader.read, 0);
> 
> It dereferenced cpu_buffer->meta_page->reader
> 
> which is only God knows what!
> 
> > 
> > Updating the meta-page is only useful if the reader we are waking up is a
> > user-space one, which would only happen with the cpu_buffer version of this
> > function. We could limit the update of the meta_page only to this case?
> 
> I rather not add another irq_work entry. This workaround should be good
> enough.

I meant, to only do in rb_wake_up_waiters()

  if (rbwork->is_cpu_buffer) 
  	rb_update_meta_page(cpu_buffer)

And skip the meta-page update for the !is_cpu_buffer case?

> 
> Thanks,
> 
> -- Steve

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

* Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space
  2023-12-20 13:49       ` Vincent Donnefort
@ 2023-12-20 16:02         ` Steven Rostedt
  0 siblings, 0 replies; 8+ messages in thread
From: Steven Rostedt @ 2023-12-20 16:02 UTC (permalink / raw)
  To: Vincent Donnefort; +Cc: mhiramat, linux-kernel, linux-trace-kernel, kernel-team

On Wed, 20 Dec 2023 13:49:30 +0000
Vincent Donnefort <vdonnefort@google.com> wrote:

> I meant, to only do in rb_wake_up_waiters()
> 
>   if (rbwork->is_cpu_buffer) 
>   	rb_update_meta_page(cpu_buffer)
> 
> And skip the meta-page update for the !is_cpu_buffer case?

Ah yeah, that works.

-- Steve

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

end of thread, other threads:[~2023-12-20 16:01 UTC | newest]

Thread overview: 8+ messages (download: mbox.gz follow: Atom feed
-- links below jump to the message on this page --
2023-12-19 18:45 [PATCH v8 0/2] Introducing trace buffer mapping by user-space Vincent Donnefort
2023-12-19 18:45 ` [PATCH v8 1/2] ring-buffer: Introducing ring-buffer mapping functions Vincent Donnefort
2023-12-19 18:45 ` [PATCH v8 2/2] tracing: Allow user-space mapping of the ring-buffer Vincent Donnefort
2023-12-19 20:39 ` [PATCH v8 0/2] Introducing trace buffer mapping by user-space Steven Rostedt
2023-12-20 13:06   ` Vincent Donnefort
2023-12-20 13:29     ` Steven Rostedt
2023-12-20 13:49       ` Vincent Donnefort
2023-12-20 16:02         ` Steven Rostedt

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