2023-12-19 18:46:13

by Vincent Donnefort

[permalink] [raw]
Subject: [PATCH v8 0/2] Introducing trace buffer mapping by user-space

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/[email protected]/
* 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



2023-12-19 18:46:31

by Vincent Donnefort

[permalink] [raw]
Subject: [PATCH v8 1/2] ring-buffer: Introducing ring-buffer mapping functions

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 <[email protected]>

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


2023-12-19 18:46:55

by Vincent Donnefort

[permalink] [raw]
Subject: [PATCH v8 2/2] tracing: Allow user-space mapping of the ring-buffer

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 <[email protected]>

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


2023-12-19 20:38:36

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space

On Tue, 19 Dec 2023 18:45:54 +0000
Vincent Donnefort <[email protected]> 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 */

2023-12-20 13:06:35

by Vincent Donnefort

[permalink] [raw]
Subject: Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space

On Tue, Dec 19, 2023 at 03:39:24PM -0500, Steven Rostedt wrote:
> On Tue, 19 Dec 2023 18:45:54 +0000
> Vincent Donnefort <[email protected]> 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 */

2023-12-20 13:28:48

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space

On Wed, 20 Dec 2023 13:06:06 +0000
Vincent Donnefort <[email protected]> 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

2023-12-20 13:56:04

by Vincent Donnefort

[permalink] [raw]
Subject: Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space

On Wed, Dec 20, 2023 at 08:29:32AM -0500, Steven Rostedt wrote:
> On Wed, 20 Dec 2023 13:06:06 +0000
> Vincent Donnefort <[email protected]> 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

2023-12-20 16:05:13

by Steven Rostedt

[permalink] [raw]
Subject: Re: [PATCH v8 0/2] Introducing trace buffer mapping by user-space

On Wed, 20 Dec 2023 13:49:30 +0000
Vincent Donnefort <[email protected]> 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