summaryrefslogtreecommitdiff
path: root/kernel/trace/ring_buffer.c
diff options
context:
space:
mode:
Diffstat (limited to 'kernel/trace/ring_buffer.c')
-rw-r--r--kernel/trace/ring_buffer.c1798
1 files changed, 1551 insertions, 247 deletions
diff --git a/kernel/trace/ring_buffer.c b/kernel/trace/ring_buffer.c
index 6511dc3a00da..e24509bd0af5 100644
--- a/kernel/trace/ring_buffer.c
+++ b/kernel/trace/ring_buffer.c
@@ -9,6 +9,7 @@
#include <linux/ring_buffer.h>
#include <linux/trace_clock.h>
#include <linux/sched/clock.h>
+#include <linux/cacheflush.h>
#include <linux/trace_seq.h>
#include <linux/spinlock.h>
#include <linux/irq_work.h>
@@ -26,9 +27,13 @@
#include <linux/list.h>
#include <linux/cpu.h>
#include <linux/oom.h>
+#include <linux/mm.h>
#include <asm/local64.h>
#include <asm/local.h>
+#include <asm/setup.h>
+
+#include "trace.h"
/*
* The "absolute" timestamp in the buffer is only 59 bits.
@@ -40,6 +45,24 @@
static void update_pages_handler(struct work_struct *work);
+#define RING_BUFFER_META_MAGIC 0xBADFEED
+
+struct ring_buffer_meta {
+ int magic;
+ int struct_sizes;
+ unsigned long total_size;
+ unsigned long buffers_offset;
+};
+
+struct ring_buffer_cpu_meta {
+ unsigned long first_buffer;
+ unsigned long head_buffer;
+ unsigned long commit_buffer;
+ __u32 subbuf_size;
+ __u32 nr_subbufs;
+ int buffers[];
+};
+
/*
* The ring buffer header is special. We must manually up keep it.
*/
@@ -312,6 +335,8 @@ static u64 rb_event_time_stamp(struct ring_buffer_event *event)
/* Missed count stored at end */
#define RB_MISSED_STORED (1 << 30)
+#define RB_MISSED_MASK (3 << 30)
+
struct buffer_data_page {
u64 time_stamp; /* page time stamp */
local_t commit; /* write committed index */
@@ -338,6 +363,8 @@ 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:30; /* ID for external mapping */
+ u32 range:1; /* Mapped via a range */
struct buffer_data_page *page; /* Actual data page */
};
@@ -368,7 +395,9 @@ static __always_inline unsigned int rb_page_commit(struct buffer_page *bpage)
static void free_buffer_page(struct buffer_page *bpage)
{
- free_pages((unsigned long)bpage->page, bpage->order);
+ /* Range pages are not to be freed */
+ if (!bpage->range)
+ free_pages((unsigned long)bpage->page, bpage->order);
kfree(bpage);
}
@@ -457,6 +486,8 @@ struct ring_buffer_per_cpu {
unsigned long nr_pages;
unsigned int current_context;
struct list_head *pages;
+ /* pages generation counter, incremented when the list changes */
+ unsigned long cnt;
struct buffer_page *head_page; /* read from head */
struct buffer_page *tail_page; /* write to tail */
struct buffer_page *commit_page; /* committed pages */
@@ -484,6 +515,14 @@ struct ring_buffer_per_cpu {
u64 read_stamp;
/* pages removed since last reset */
unsigned long pages_removed;
+
+ unsigned int mapped;
+ unsigned int user_mapped; /* user space mapping */
+ struct mutex mapping_lock;
+ unsigned long *subbuf_ids; /* ID to subbuf VA */
+ struct trace_buffer_meta *meta_page;
+ struct ring_buffer_cpu_meta *ring_meta;
+
/* 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 */
@@ -512,6 +551,11 @@ struct trace_buffer {
struct rb_irq_work irq_work;
bool time_stamp_abs;
+ unsigned long range_addr_start;
+ unsigned long range_addr_end;
+
+ struct ring_buffer_meta *meta;
+
unsigned int subbuf_size;
unsigned int subbuf_order;
unsigned int max_data_size;
@@ -682,18 +726,6 @@ u64 ring_buffer_event_time_stamp(struct trace_buffer *buffer,
}
/**
- * ring_buffer_nr_pages - get the number of buffer pages in the ring buffer
- * @buffer: The ring_buffer to get the number of pages from
- * @cpu: The cpu of the ring_buffer to get the number of pages from
- *
- * Returns the number of pages used by a per_cpu buffer of the ring buffer.
- */
-size_t ring_buffer_nr_pages(struct trace_buffer *buffer, int cpu)
-{
- return buffer->buffers[cpu]->nr_pages;
-}
-
-/**
* ring_buffer_nr_dirty_pages - get the number of used pages in the ring buffer
* @buffer: The ring_buffer to get the number of pages from
* @cpu: The cpu of the ring_buffer to get the number of pages from
@@ -1240,6 +1272,11 @@ static void rb_head_page_activate(struct ring_buffer_per_cpu *cpu_buffer)
* Set the previous list pointer to have the HEAD flag.
*/
rb_set_list_to_head(head->list.prev);
+
+ if (cpu_buffer->ring_meta) {
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ meta->head_buffer = (unsigned long)head->page;
+ }
}
static void rb_list_head_clear(struct list_head *list)
@@ -1443,6 +1480,20 @@ static void rb_check_bpage(struct ring_buffer_per_cpu *cpu_buffer,
RB_WARN_ON(cpu_buffer, val & RB_FLAG_MASK);
}
+static bool rb_check_links(struct ring_buffer_per_cpu *cpu_buffer,
+ struct list_head *list)
+{
+ if (RB_WARN_ON(cpu_buffer,
+ rb_list_head(rb_list_head(list->next)->prev) != list))
+ return false;
+
+ if (RB_WARN_ON(cpu_buffer,
+ rb_list_head(rb_list_head(list->prev)->next) != list))
+ return false;
+
+ return true;
+}
+
/**
* rb_check_pages - integrity check of buffer pages
* @cpu_buffer: CPU buffer with pages to test
@@ -1452,31 +1503,607 @@ static void rb_check_bpage(struct ring_buffer_per_cpu *cpu_buffer,
*/
static void rb_check_pages(struct ring_buffer_per_cpu *cpu_buffer)
{
- struct list_head *head = rb_list_head(cpu_buffer->pages);
- struct list_head *tmp;
+ struct list_head *head, *tmp;
+ unsigned long buffer_cnt;
+ unsigned long flags;
+ int nr_loops = 0;
- if (RB_WARN_ON(cpu_buffer,
- rb_list_head(rb_list_head(head->next)->prev) != head))
+ /*
+ * Walk the linked list underpinning the ring buffer and validate all
+ * its next and prev links.
+ *
+ * The check acquires the reader_lock to avoid concurrent processing
+ * with code that could be modifying the list. However, the lock cannot
+ * be held for the entire duration of the walk, as this would make the
+ * time when interrupts are disabled non-deterministic, dependent on the
+ * ring buffer size. Therefore, the code releases and re-acquires the
+ * lock after checking each page. The ring_buffer_per_cpu.cnt variable
+ * is then used to detect if the list was modified while the lock was
+ * not held, in which case the check needs to be restarted.
+ *
+ * The code attempts to perform the check at most three times before
+ * giving up. This is acceptable because this is only a self-validation
+ * to detect problems early on. In practice, the list modification
+ * operations are fairly spaced, and so this check typically succeeds at
+ * most on the second try.
+ */
+again:
+ if (++nr_loops > 3)
return;
- if (RB_WARN_ON(cpu_buffer,
- rb_list_head(rb_list_head(head->prev)->next) != head))
+ raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+ head = rb_list_head(cpu_buffer->pages);
+ if (!rb_check_links(cpu_buffer, head))
+ goto out_locked;
+ buffer_cnt = cpu_buffer->cnt;
+ tmp = head;
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+
+ while (true) {
+ raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+
+ if (buffer_cnt != cpu_buffer->cnt) {
+ /* The list was updated, try again. */
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+ goto again;
+ }
+
+ tmp = rb_list_head(tmp->next);
+ if (tmp == head)
+ /* The iteration circled back, all is done. */
+ goto out_locked;
+
+ if (!rb_check_links(cpu_buffer, tmp))
+ goto out_locked;
+
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+ }
+
+out_locked:
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+}
+
+/*
+ * Take an address, add the meta data size as well as the array of
+ * array subbuffer indexes, then align it to a subbuffer size.
+ *
+ * This is used to help find the next per cpu subbuffer within a mapped range.
+ */
+static unsigned long
+rb_range_align_subbuf(unsigned long addr, int subbuf_size, int nr_subbufs)
+{
+ addr += sizeof(struct ring_buffer_cpu_meta) +
+ sizeof(int) * nr_subbufs;
+ return ALIGN(addr, subbuf_size);
+}
+
+/*
+ * Return the ring_buffer_meta for a given @cpu.
+ */
+static void *rb_range_meta(struct trace_buffer *buffer, int nr_pages, int cpu)
+{
+ int subbuf_size = buffer->subbuf_size + BUF_PAGE_HDR_SIZE;
+ struct ring_buffer_cpu_meta *meta;
+ struct ring_buffer_meta *bmeta;
+ unsigned long ptr;
+ int nr_subbufs;
+
+ bmeta = buffer->meta;
+ if (!bmeta)
+ return NULL;
+
+ ptr = (unsigned long)bmeta + bmeta->buffers_offset;
+ meta = (struct ring_buffer_cpu_meta *)ptr;
+
+ /* When nr_pages passed in is zero, the first meta has already been initialized */
+ if (!nr_pages) {
+ nr_subbufs = meta->nr_subbufs;
+ } else {
+ /* Include the reader page */
+ nr_subbufs = nr_pages + 1;
+ }
+
+ /*
+ * The first chunk may not be subbuffer aligned, where as
+ * the rest of the chunks are.
+ */
+ if (cpu) {
+ ptr = rb_range_align_subbuf(ptr, subbuf_size, nr_subbufs);
+ ptr += subbuf_size * nr_subbufs;
+
+ /* We can use multiplication to find chunks greater than 1 */
+ if (cpu > 1) {
+ unsigned long size;
+ unsigned long p;
+
+ /* Save the beginning of this CPU chunk */
+ p = ptr;
+ ptr = rb_range_align_subbuf(ptr, subbuf_size, nr_subbufs);
+ ptr += subbuf_size * nr_subbufs;
+
+ /* Now all chunks after this are the same size */
+ size = ptr - p;
+ ptr += size * (cpu - 2);
+ }
+ }
+ return (void *)ptr;
+}
+
+/* Return the start of subbufs given the meta pointer */
+static void *rb_subbufs_from_meta(struct ring_buffer_cpu_meta *meta)
+{
+ int subbuf_size = meta->subbuf_size;
+ unsigned long ptr;
+
+ ptr = (unsigned long)meta;
+ ptr = rb_range_align_subbuf(ptr, subbuf_size, meta->nr_subbufs);
+
+ return (void *)ptr;
+}
+
+/*
+ * Return a specific sub-buffer for a given @cpu defined by @idx.
+ */
+static void *rb_range_buffer(struct ring_buffer_per_cpu *cpu_buffer, int idx)
+{
+ struct ring_buffer_cpu_meta *meta;
+ unsigned long ptr;
+ int subbuf_size;
+
+ meta = rb_range_meta(cpu_buffer->buffer, 0, cpu_buffer->cpu);
+ if (!meta)
+ return NULL;
+
+ if (WARN_ON_ONCE(idx >= meta->nr_subbufs))
+ return NULL;
+
+ subbuf_size = meta->subbuf_size;
+
+ /* Map this buffer to the order that's in meta->buffers[] */
+ idx = meta->buffers[idx];
+
+ ptr = (unsigned long)rb_subbufs_from_meta(meta);
+
+ ptr += subbuf_size * idx;
+ if (ptr + subbuf_size > cpu_buffer->buffer->range_addr_end)
+ return NULL;
+
+ return (void *)ptr;
+}
+
+/*
+ * See if the existing memory contains a valid meta section.
+ * if so, use that, otherwise initialize it.
+ */
+static bool rb_meta_init(struct trace_buffer *buffer, int scratch_size)
+{
+ unsigned long ptr = buffer->range_addr_start;
+ struct ring_buffer_meta *bmeta;
+ unsigned long total_size;
+ int struct_sizes;
+
+ bmeta = (struct ring_buffer_meta *)ptr;
+ buffer->meta = bmeta;
+
+ total_size = buffer->range_addr_end - buffer->range_addr_start;
+
+ struct_sizes = sizeof(struct ring_buffer_cpu_meta);
+ struct_sizes |= sizeof(*bmeta) << 16;
+
+ /* The first buffer will start word size after the meta page */
+ ptr += sizeof(*bmeta);
+ ptr = ALIGN(ptr, sizeof(long));
+ ptr += scratch_size;
+
+ if (bmeta->magic != RING_BUFFER_META_MAGIC) {
+ pr_info("Ring buffer boot meta mismatch of magic\n");
+ goto init;
+ }
+
+ if (bmeta->struct_sizes != struct_sizes) {
+ pr_info("Ring buffer boot meta mismatch of struct size\n");
+ goto init;
+ }
+
+ if (bmeta->total_size != total_size) {
+ pr_info("Ring buffer boot meta mismatch of total size\n");
+ goto init;
+ }
+
+ if (bmeta->buffers_offset > bmeta->total_size) {
+ pr_info("Ring buffer boot meta mismatch of offset outside of total size\n");
+ goto init;
+ }
+
+ if (bmeta->buffers_offset != (void *)ptr - (void *)bmeta) {
+ pr_info("Ring buffer boot meta mismatch of first buffer offset\n");
+ goto init;
+ }
+
+ return true;
+
+ init:
+ bmeta->magic = RING_BUFFER_META_MAGIC;
+ bmeta->struct_sizes = struct_sizes;
+ bmeta->total_size = total_size;
+ bmeta->buffers_offset = (void *)ptr - (void *)bmeta;
+
+ /* Zero out the scatch pad */
+ memset((void *)bmeta + sizeof(*bmeta), 0, bmeta->buffers_offset - sizeof(*bmeta));
+
+ return false;
+}
+
+/*
+ * See if the existing memory contains valid ring buffer data.
+ * As the previous kernel must be the same as this kernel, all
+ * the calculations (size of buffers and number of buffers)
+ * must be the same.
+ */
+static bool rb_cpu_meta_valid(struct ring_buffer_cpu_meta *meta, int cpu,
+ struct trace_buffer *buffer, int nr_pages,
+ unsigned long *subbuf_mask)
+{
+ int subbuf_size = PAGE_SIZE;
+ struct buffer_data_page *subbuf;
+ unsigned long buffers_start;
+ unsigned long buffers_end;
+ int i;
+
+ if (!subbuf_mask)
+ return false;
+
+ buffers_start = meta->first_buffer;
+ buffers_end = meta->first_buffer + (subbuf_size * meta->nr_subbufs);
+
+ /* Is the head and commit buffers within the range of buffers? */
+ if (meta->head_buffer < buffers_start ||
+ meta->head_buffer >= buffers_end) {
+ pr_info("Ring buffer boot meta [%d] head buffer out of range\n", cpu);
+ return false;
+ }
+
+ if (meta->commit_buffer < buffers_start ||
+ meta->commit_buffer >= buffers_end) {
+ pr_info("Ring buffer boot meta [%d] commit buffer out of range\n", cpu);
+ return false;
+ }
+
+ subbuf = rb_subbufs_from_meta(meta);
+
+ bitmap_clear(subbuf_mask, 0, meta->nr_subbufs);
+
+ /* Is the meta buffers and the subbufs themselves have correct data? */
+ for (i = 0; i < meta->nr_subbufs; i++) {
+ if (meta->buffers[i] < 0 ||
+ meta->buffers[i] >= meta->nr_subbufs) {
+ pr_info("Ring buffer boot meta [%d] array out of range\n", cpu);
+ return false;
+ }
+
+ if ((unsigned)local_read(&subbuf->commit) > subbuf_size) {
+ pr_info("Ring buffer boot meta [%d] buffer invalid commit\n", cpu);
+ return false;
+ }
+
+ if (test_bit(meta->buffers[i], subbuf_mask)) {
+ pr_info("Ring buffer boot meta [%d] array has duplicates\n", cpu);
+ return false;
+ }
+
+ set_bit(meta->buffers[i], subbuf_mask);
+ subbuf = (void *)subbuf + subbuf_size;
+ }
+
+ return true;
+}
+
+static int rb_meta_subbuf_idx(struct ring_buffer_cpu_meta *meta, void *subbuf);
+
+static int rb_read_data_buffer(struct buffer_data_page *dpage, int tail, int cpu,
+ unsigned long long *timestamp, u64 *delta_ptr)
+{
+ struct ring_buffer_event *event;
+ u64 ts, delta;
+ int events = 0;
+ int e;
+
+ *delta_ptr = 0;
+ *timestamp = 0;
+
+ ts = dpage->time_stamp;
+
+ for (e = 0; e < tail; e += rb_event_length(event)) {
+
+ event = (struct ring_buffer_event *)(dpage->data + e);
+
+ switch (event->type_len) {
+
+ case RINGBUF_TYPE_TIME_EXTEND:
+ delta = rb_event_time_stamp(event);
+ ts += delta;
+ break;
+
+ case RINGBUF_TYPE_TIME_STAMP:
+ delta = rb_event_time_stamp(event);
+ delta = rb_fix_abs_ts(delta, ts);
+ if (delta < ts) {
+ *delta_ptr = delta;
+ *timestamp = ts;
+ return -1;
+ }
+ ts = delta;
+ break;
+
+ case RINGBUF_TYPE_PADDING:
+ if (event->time_delta == 1)
+ break;
+ fallthrough;
+ case RINGBUF_TYPE_DATA:
+ events++;
+ ts += event->time_delta;
+ break;
+
+ default:
+ return -1;
+ }
+ }
+ *timestamp = ts;
+ return events;
+}
+
+static int rb_validate_buffer(struct buffer_data_page *dpage, int cpu)
+{
+ unsigned long long ts;
+ u64 delta;
+ int tail;
+
+ tail = local_read(&dpage->commit);
+ return rb_read_data_buffer(dpage, tail, cpu, &ts, &delta);
+}
+
+/* If the meta data has been validated, now validate the events */
+static void rb_meta_validate_events(struct ring_buffer_per_cpu *cpu_buffer)
+{
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ struct buffer_page *head_page;
+ unsigned long entry_bytes = 0;
+ unsigned long entries = 0;
+ int ret;
+ int i;
+
+ if (!meta || !meta->head_buffer)
return;
- for (tmp = rb_list_head(head->next); tmp != head; tmp = rb_list_head(tmp->next)) {
- if (RB_WARN_ON(cpu_buffer,
- rb_list_head(rb_list_head(tmp->next)->prev) != tmp))
- return;
+ /* Do the reader page first */
+ ret = rb_validate_buffer(cpu_buffer->reader_page->page, cpu_buffer->cpu);
+ if (ret < 0) {
+ pr_info("Ring buffer reader page is invalid\n");
+ goto invalid;
+ }
+ entries += ret;
+ entry_bytes += local_read(&cpu_buffer->reader_page->page->commit);
+ local_set(&cpu_buffer->reader_page->entries, ret);
- if (RB_WARN_ON(cpu_buffer,
- rb_list_head(rb_list_head(tmp->prev)->next) != tmp))
- return;
+ head_page = cpu_buffer->head_page;
+
+ /* If the commit_buffer is the reader page, update the commit page */
+ if (meta->commit_buffer == (unsigned long)cpu_buffer->reader_page->page) {
+ cpu_buffer->commit_page = cpu_buffer->reader_page;
+ /* Nothing more to do, the only page is the reader page */
+ goto done;
+ }
+
+ /* Iterate until finding the commit page */
+ for (i = 0; i < meta->nr_subbufs + 1; i++, rb_inc_page(&head_page)) {
+
+ /* Reader page has already been done */
+ if (head_page == cpu_buffer->reader_page)
+ continue;
+
+ ret = rb_validate_buffer(head_page->page, cpu_buffer->cpu);
+ if (ret < 0) {
+ pr_info("Ring buffer meta [%d] invalid buffer page\n",
+ cpu_buffer->cpu);
+ goto invalid;
+ }
+
+ /* If the buffer has content, update pages_touched */
+ if (ret)
+ local_inc(&cpu_buffer->pages_touched);
+
+ entries += ret;
+ entry_bytes += local_read(&head_page->page->commit);
+ local_set(&cpu_buffer->head_page->entries, ret);
+
+ if (head_page == cpu_buffer->commit_page)
+ break;
+ }
+
+ if (head_page != cpu_buffer->commit_page) {
+ pr_info("Ring buffer meta [%d] commit page not found\n",
+ cpu_buffer->cpu);
+ goto invalid;
+ }
+ done:
+ local_set(&cpu_buffer->entries, entries);
+ local_set(&cpu_buffer->entries_bytes, entry_bytes);
+
+ pr_info("Ring buffer meta [%d] is from previous boot!\n", cpu_buffer->cpu);
+ return;
+
+ invalid:
+ /* The content of the buffers are invalid, reset the meta data */
+ meta->head_buffer = 0;
+ meta->commit_buffer = 0;
+
+ /* Reset the reader page */
+ local_set(&cpu_buffer->reader_page->entries, 0);
+ local_set(&cpu_buffer->reader_page->page->commit, 0);
+
+ /* Reset all the subbuffers */
+ for (i = 0; i < meta->nr_subbufs - 1; i++, rb_inc_page(&head_page)) {
+ local_set(&head_page->entries, 0);
+ local_set(&head_page->page->commit, 0);
+ }
+}
+
+static void rb_range_meta_init(struct trace_buffer *buffer, int nr_pages, int scratch_size)
+{
+ struct ring_buffer_cpu_meta *meta;
+ unsigned long *subbuf_mask;
+ unsigned long delta;
+ void *subbuf;
+ bool valid = false;
+ int cpu;
+ int i;
+
+ /* Create a mask to test the subbuf array */
+ subbuf_mask = bitmap_alloc(nr_pages + 1, GFP_KERNEL);
+ /* If subbuf_mask fails to allocate, then rb_meta_valid() will return false */
+
+ if (rb_meta_init(buffer, scratch_size))
+ valid = true;
+
+ for (cpu = 0; cpu < nr_cpu_ids; cpu++) {
+ void *next_meta;
+
+ meta = rb_range_meta(buffer, nr_pages, cpu);
+
+ if (valid && rb_cpu_meta_valid(meta, cpu, buffer, nr_pages, subbuf_mask)) {
+ /* Make the mappings match the current address */
+ subbuf = rb_subbufs_from_meta(meta);
+ delta = (unsigned long)subbuf - meta->first_buffer;
+ meta->first_buffer += delta;
+ meta->head_buffer += delta;
+ meta->commit_buffer += delta;
+ continue;
+ }
+
+ if (cpu < nr_cpu_ids - 1)
+ next_meta = rb_range_meta(buffer, nr_pages, cpu + 1);
+ else
+ next_meta = (void *)buffer->range_addr_end;
+
+ memset(meta, 0, next_meta - (void *)meta);
+
+ meta->nr_subbufs = nr_pages + 1;
+ meta->subbuf_size = PAGE_SIZE;
+
+ subbuf = rb_subbufs_from_meta(meta);
+
+ meta->first_buffer = (unsigned long)subbuf;
+
+ /*
+ * The buffers[] array holds the order of the sub-buffers
+ * that are after the meta data. The sub-buffers may
+ * be swapped out when read and inserted into a different
+ * location of the ring buffer. Although their addresses
+ * remain the same, the buffers[] array contains the
+ * index into the sub-buffers holding their actual order.
+ */
+ for (i = 0; i < meta->nr_subbufs; i++) {
+ meta->buffers[i] = i;
+ rb_init_page(subbuf);
+ subbuf += meta->subbuf_size;
+ }
+ }
+ bitmap_free(subbuf_mask);
+}
+
+static void *rbm_start(struct seq_file *m, loff_t *pos)
+{
+ struct ring_buffer_per_cpu *cpu_buffer = m->private;
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ unsigned long val;
+
+ if (!meta)
+ return NULL;
+
+ if (*pos > meta->nr_subbufs)
+ return NULL;
+
+ val = *pos;
+ val++;
+
+ return (void *)val;
+}
+
+static void *rbm_next(struct seq_file *m, void *v, loff_t *pos)
+{
+ (*pos)++;
+
+ return rbm_start(m, pos);
+}
+
+static int rbm_show(struct seq_file *m, void *v)
+{
+ struct ring_buffer_per_cpu *cpu_buffer = m->private;
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ unsigned long val = (unsigned long)v;
+
+ if (val == 1) {
+ seq_printf(m, "head_buffer: %d\n",
+ rb_meta_subbuf_idx(meta, (void *)meta->head_buffer));
+ seq_printf(m, "commit_buffer: %d\n",
+ rb_meta_subbuf_idx(meta, (void *)meta->commit_buffer));
+ seq_printf(m, "subbuf_size: %d\n", meta->subbuf_size);
+ seq_printf(m, "nr_subbufs: %d\n", meta->nr_subbufs);
+ return 0;
+ }
+
+ val -= 2;
+ seq_printf(m, "buffer[%ld]: %d\n", val, meta->buffers[val]);
+
+ return 0;
+}
+
+static void rbm_stop(struct seq_file *m, void *p)
+{
+}
+
+static const struct seq_operations rb_meta_seq_ops = {
+ .start = rbm_start,
+ .next = rbm_next,
+ .show = rbm_show,
+ .stop = rbm_stop,
+};
+
+int ring_buffer_meta_seq_init(struct file *file, struct trace_buffer *buffer, int cpu)
+{
+ struct seq_file *m;
+ int ret;
+
+ ret = seq_open(file, &rb_meta_seq_ops);
+ if (ret)
+ return ret;
+
+ m = file->private_data;
+ m->private = buffer->buffers[cpu];
+
+ return 0;
+}
+
+/* Map the buffer_pages to the previous head and commit pages */
+static void rb_meta_buffer_update(struct ring_buffer_per_cpu *cpu_buffer,
+ struct buffer_page *bpage)
+{
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+
+ if (meta->head_buffer == (unsigned long)bpage->page)
+ cpu_buffer->head_page = bpage;
+
+ if (meta->commit_buffer == (unsigned long)bpage->page) {
+ cpu_buffer->commit_page = bpage;
+ cpu_buffer->tail_page = bpage;
}
}
static int __rb_allocate_pages(struct ring_buffer_per_cpu *cpu_buffer,
long nr_pages, struct list_head *pages)
{
+ struct trace_buffer *buffer = cpu_buffer->buffer;
+ struct ring_buffer_cpu_meta *meta = NULL;
struct buffer_page *bpage, *tmp;
bool user_thread = current->mm != NULL;
gfp_t mflags;
@@ -1511,6 +2138,10 @@ static int __rb_allocate_pages(struct ring_buffer_per_cpu *cpu_buffer,
*/
if (user_thread)
set_current_oom_origin();
+
+ if (buffer->range_addr_start)
+ meta = rb_range_meta(buffer, nr_pages, cpu_buffer->cpu);
+
for (i = 0; i < nr_pages; i++) {
struct page *page;
@@ -1521,16 +2152,32 @@ static int __rb_allocate_pages(struct ring_buffer_per_cpu *cpu_buffer,
rb_check_bpage(cpu_buffer, bpage);
- list_add(&bpage->list, pages);
-
- page = alloc_pages_node(cpu_to_node(cpu_buffer->cpu),
- mflags | __GFP_ZERO,
- cpu_buffer->buffer->subbuf_order);
- if (!page)
- goto free_pages;
- bpage->page = page_address(page);
+ /*
+ * Append the pages as for mapped buffers we want to keep
+ * the order
+ */
+ list_add_tail(&bpage->list, pages);
+
+ if (meta) {
+ /* A range was given. Use that for the buffer page */
+ bpage->page = rb_range_buffer(cpu_buffer, i + 1);
+ if (!bpage->page)
+ goto free_pages;
+ /* If this is valid from a previous boot */
+ if (meta->head_buffer)
+ rb_meta_buffer_update(cpu_buffer, bpage);
+ bpage->range = 1;
+ bpage->id = i + 1;
+ } else {
+ page = alloc_pages_node(cpu_to_node(cpu_buffer->cpu),
+ mflags | __GFP_COMP | __GFP_ZERO,
+ cpu_buffer->buffer->subbuf_order);
+ if (!page)
+ goto free_pages;
+ bpage->page = page_address(page);
+ rb_init_page(bpage->page);
+ }
bpage->order = cpu_buffer->buffer->subbuf_order;
- rb_init_page(bpage->page);
if (user_thread && fatal_signal_pending(current))
goto free_pages;
@@ -1579,7 +2226,8 @@ static int rb_allocate_pages(struct ring_buffer_per_cpu *cpu_buffer,
static struct ring_buffer_per_cpu *
rb_allocate_cpu_buffer(struct trace_buffer *buffer, long nr_pages, int cpu)
{
- struct ring_buffer_per_cpu *cpu_buffer;
+ struct ring_buffer_per_cpu *cpu_buffer __free(kfree) = NULL;
+ struct ring_buffer_cpu_meta *meta;
struct buffer_page *bpage;
struct page *page;
int ret;
@@ -1599,22 +2247,39 @@ 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));
if (!bpage)
- goto fail_free_buffer;
+ return NULL;
rb_check_bpage(cpu_buffer, bpage);
cpu_buffer->reader_page = bpage;
- page = alloc_pages_node(cpu_to_node(cpu), GFP_KERNEL | __GFP_ZERO,
- cpu_buffer->buffer->subbuf_order);
- if (!page)
- goto fail_free_reader;
- bpage->page = page_address(page);
- rb_init_page(bpage->page);
+ if (buffer->range_addr_start) {
+ /*
+ * Range mapped buffers have the same restrictions as memory
+ * mapped ones do.
+ */
+ cpu_buffer->mapped = 1;
+ cpu_buffer->ring_meta = rb_range_meta(buffer, nr_pages, cpu);
+ bpage->page = rb_range_buffer(cpu_buffer, 0);
+ if (!bpage->page)
+ goto fail_free_reader;
+ if (cpu_buffer->ring_meta->head_buffer)
+ rb_meta_buffer_update(cpu_buffer, bpage);
+ bpage->range = 1;
+ } else {
+ page = alloc_pages_node(cpu_to_node(cpu),
+ GFP_KERNEL | __GFP_COMP | __GFP_ZERO,
+ cpu_buffer->buffer->subbuf_order);
+ if (!page)
+ goto fail_free_reader;
+ bpage->page = page_address(page);
+ rb_init_page(bpage->page);
+ }
INIT_LIST_HEAD(&cpu_buffer->reader_page->list);
INIT_LIST_HEAD(&cpu_buffer->new_pages);
@@ -1623,19 +2288,41 @@ rb_allocate_cpu_buffer(struct trace_buffer *buffer, long nr_pages, int cpu)
if (ret < 0)
goto fail_free_reader;
- cpu_buffer->head_page
- = list_entry(cpu_buffer->pages, struct buffer_page, list);
- cpu_buffer->tail_page = cpu_buffer->commit_page = cpu_buffer->head_page;
+ rb_meta_validate_events(cpu_buffer);
+
+ /* If the boot meta was valid then this has already been updated */
+ meta = cpu_buffer->ring_meta;
+ if (!meta || !meta->head_buffer ||
+ !cpu_buffer->head_page || !cpu_buffer->commit_page || !cpu_buffer->tail_page) {
+ if (meta && meta->head_buffer &&
+ (cpu_buffer->head_page || cpu_buffer->commit_page || cpu_buffer->tail_page)) {
+ pr_warn("Ring buffer meta buffers not all mapped\n");
+ if (!cpu_buffer->head_page)
+ pr_warn(" Missing head_page\n");
+ if (!cpu_buffer->commit_page)
+ pr_warn(" Missing commit_page\n");
+ if (!cpu_buffer->tail_page)
+ pr_warn(" Missing tail_page\n");
+ }
- rb_head_page_activate(cpu_buffer);
+ cpu_buffer->head_page
+ = list_entry(cpu_buffer->pages, struct buffer_page, list);
+ cpu_buffer->tail_page = cpu_buffer->commit_page = cpu_buffer->head_page;
- return cpu_buffer;
+ rb_head_page_activate(cpu_buffer);
+
+ if (cpu_buffer->ring_meta)
+ meta->commit_buffer = meta->head_buffer;
+ } else {
+ /* The valid meta buffer still needs to activate the head page */
+ rb_head_page_activate(cpu_buffer);
+ }
+
+ return_ptr(cpu_buffer);
fail_free_reader:
free_buffer_page(cpu_buffer->reader_page);
- fail_free_buffer:
- kfree(cpu_buffer);
return NULL;
}
@@ -1664,22 +2351,15 @@ static void rb_free_cpu_buffer(struct ring_buffer_per_cpu *cpu_buffer)
kfree(cpu_buffer);
}
-/**
- * __ring_buffer_alloc - allocate a new ring_buffer
- * @size: the size in bytes per cpu that is needed.
- * @flags: attributes to set for the ring buffer.
- * @key: ring buffer reader_lock_key.
- *
- * Currently the only flag that is available is the RB_FL_OVERWRITE
- * flag. This flag means that the buffer will overwrite old data
- * when the buffer wraps. If this flag is not set, the buffer will
- * drop data when the tail hits the head.
- */
-struct trace_buffer *__ring_buffer_alloc(unsigned long size, unsigned flags,
- struct lock_class_key *key)
+static struct trace_buffer *alloc_buffer(unsigned long size, unsigned flags,
+ int order, unsigned long start,
+ unsigned long end,
+ unsigned long scratch_size,
+ struct lock_class_key *key)
{
- struct trace_buffer *buffer;
+ struct trace_buffer *buffer __free(kfree) = NULL;
long nr_pages;
+ int subbuf_size;
int bsize;
int cpu;
int ret;
@@ -1691,16 +2371,15 @@ struct trace_buffer *__ring_buffer_alloc(unsigned long size, unsigned flags,
return NULL;
if (!zalloc_cpumask_var(&buffer->cpumask, GFP_KERNEL))
- goto fail_free_buffer;
+ return NULL;
- /* Default buffer page size - one system page */
- buffer->subbuf_order = 0;
- buffer->subbuf_size = PAGE_SIZE - BUF_PAGE_HDR_SIZE;
+ buffer->subbuf_order = order;
+ subbuf_size = (PAGE_SIZE << order);
+ buffer->subbuf_size = subbuf_size - BUF_PAGE_HDR_SIZE;
/* Max payload is buffer page size - header (8bytes) */
buffer->max_data_size = buffer->subbuf_size - (sizeof(u32) * 2);
- nr_pages = DIV_ROUND_UP(size, buffer->subbuf_size);
buffer->flags = flags;
buffer->clock = trace_clock_local;
buffer->reader_lock_key = key;
@@ -1708,10 +2387,6 @@ struct trace_buffer *__ring_buffer_alloc(unsigned long size, unsigned flags,
init_irq_work(&buffer->irq_work.work, rb_wake_up_waiters);
init_waitqueue_head(&buffer->irq_work.waiters);
- /* need at least two pages */
- if (nr_pages < 2)
- nr_pages = 2;
-
buffer->cpus = nr_cpu_ids;
bsize = sizeof(void *) * nr_cpu_ids;
@@ -1720,6 +2395,69 @@ struct trace_buffer *__ring_buffer_alloc(unsigned long size, unsigned flags,
if (!buffer->buffers)
goto fail_free_cpumask;
+ /* If start/end are specified, then that overrides size */
+ if (start && end) {
+ unsigned long buffers_start;
+ unsigned long ptr;
+ int n;
+
+ /* Make sure that start is word aligned */
+ start = ALIGN(start, sizeof(long));
+
+ /* scratch_size needs to be aligned too */
+ scratch_size = ALIGN(scratch_size, sizeof(long));
+
+ /* Subtract the buffer meta data and word aligned */
+ buffers_start = start + sizeof(struct ring_buffer_cpu_meta);
+ buffers_start = ALIGN(buffers_start, sizeof(long));
+ buffers_start += scratch_size;
+
+ /* Calculate the size for the per CPU data */
+ size = end - buffers_start;
+ size = size / nr_cpu_ids;
+
+ /*
+ * The number of sub-buffers (nr_pages) is determined by the
+ * total size allocated minus the meta data size.
+ * Then that is divided by the number of per CPU buffers
+ * needed, plus account for the integer array index that
+ * will be appended to the meta data.
+ */
+ nr_pages = (size - sizeof(struct ring_buffer_cpu_meta)) /
+ (subbuf_size + sizeof(int));
+ /* Need at least two pages plus the reader page */
+ if (nr_pages < 3)
+ goto fail_free_buffers;
+
+ again:
+ /* Make sure that the size fits aligned */
+ for (n = 0, ptr = buffers_start; n < nr_cpu_ids; n++) {
+ ptr += sizeof(struct ring_buffer_cpu_meta) +
+ sizeof(int) * nr_pages;
+ ptr = ALIGN(ptr, subbuf_size);
+ ptr += subbuf_size * nr_pages;
+ }
+ if (ptr > end) {
+ if (nr_pages <= 3)
+ goto fail_free_buffers;
+ nr_pages--;
+ goto again;
+ }
+
+ /* nr_pages should not count the reader page */
+ nr_pages--;
+ buffer->range_addr_start = start;
+ buffer->range_addr_end = end;
+
+ rb_range_meta_init(buffer, nr_pages, scratch_size);
+ } else {
+
+ /* need at least two pages */
+ nr_pages = DIV_ROUND_UP(size, buffer->subbuf_size);
+ if (nr_pages < 2)
+ nr_pages = 2;
+ }
+
cpu = raw_smp_processor_id();
cpumask_set_cpu(cpu, buffer->cpumask);
buffer->buffers[cpu] = rb_allocate_cpu_buffer(buffer, nr_pages, cpu);
@@ -1732,7 +2470,7 @@ struct trace_buffer *__ring_buffer_alloc(unsigned long size, unsigned flags,
mutex_init(&buffer->mutex);
- return buffer;
+ return_ptr(buffer);
fail_free_buffers:
for_each_buffer_cpu(buffer, cpu) {
@@ -1744,13 +2482,73 @@ struct trace_buffer *__ring_buffer_alloc(unsigned long size, unsigned flags,
fail_free_cpumask:
free_cpumask_var(buffer->cpumask);
- fail_free_buffer:
- kfree(buffer);
return NULL;
}
+
+/**
+ * __ring_buffer_alloc - allocate a new ring_buffer
+ * @size: the size in bytes per cpu that is needed.
+ * @flags: attributes to set for the ring buffer.
+ * @key: ring buffer reader_lock_key.
+ *
+ * Currently the only flag that is available is the RB_FL_OVERWRITE
+ * flag. This flag means that the buffer will overwrite old data
+ * when the buffer wraps. If this flag is not set, the buffer will
+ * drop data when the tail hits the head.
+ */
+struct trace_buffer *__ring_buffer_alloc(unsigned long size, unsigned flags,
+ struct lock_class_key *key)
+{
+ /* Default buffer page size - one system page */
+ return alloc_buffer(size, flags, 0, 0, 0, 0, key);
+
+}
EXPORT_SYMBOL_GPL(__ring_buffer_alloc);
/**
+ * __ring_buffer_alloc_range - allocate a new ring_buffer from existing memory
+ * @size: the size in bytes per cpu that is needed.
+ * @flags: attributes to set for the ring buffer.
+ * @order: sub-buffer order
+ * @start: start of allocated range
+ * @range_size: size of allocated range
+ * @scratch_size: size of scratch area (for preallocated memory buffers)
+ * @key: ring buffer reader_lock_key.
+ *
+ * Currently the only flag that is available is the RB_FL_OVERWRITE
+ * flag. This flag means that the buffer will overwrite old data
+ * when the buffer wraps. If this flag is not set, the buffer will
+ * drop data when the tail hits the head.
+ */
+struct trace_buffer *__ring_buffer_alloc_range(unsigned long size, unsigned flags,
+ int order, unsigned long start,
+ unsigned long range_size,
+ unsigned long scratch_size,
+ struct lock_class_key *key)
+{
+ return alloc_buffer(size, flags, order, start, start + range_size,
+ scratch_size, key);
+}
+
+void *ring_buffer_meta_scratch(struct trace_buffer *buffer, unsigned int *size)
+{
+ struct ring_buffer_meta *meta;
+ void *ptr;
+
+ if (!buffer || !buffer->meta)
+ return NULL;
+
+ meta = buffer->meta;
+
+ ptr = (void *)ALIGN((unsigned long)meta + sizeof(*meta), sizeof(long));
+
+ if (size)
+ *size = (void *)meta + meta->buffers_offset - ptr;
+
+ return ptr;
+}
+
+/**
* ring_buffer_free - free a ring buffer.
* @buffer: the buffer to free.
*/
@@ -1789,8 +2587,6 @@ bool ring_buffer_time_stamp_abs(struct trace_buffer *buffer)
return buffer->time_stamp_abs;
}
-static void rb_reset_cpu(struct ring_buffer_per_cpu *cpu_buffer);
-
static inline unsigned long rb_page_entries(struct buffer_page *bpage)
{
return local_read(&bpage->entries) & RB_WRITE_MASK;
@@ -1859,6 +2655,7 @@ rb_remove_pages(struct ring_buffer_per_cpu *cpu_buffer, unsigned long nr_pages)
/* make sure pages points to a valid page in the ring buffer */
cpu_buffer->pages = next_page;
+ cpu_buffer->cnt++;
/* update head page */
if (head_bit)
@@ -1965,6 +2762,7 @@ rb_insert_pages(struct ring_buffer_per_cpu *cpu_buffer)
* pointer to point to end of list
*/
head_page->prev = last_page;
+ cpu_buffer->cnt++;
success = true;
break;
}
@@ -2047,6 +2845,12 @@ int ring_buffer_resize(struct trace_buffer *buffer, unsigned long size,
if (nr_pages < 2)
nr_pages = 2;
+ /*
+ * Keep CPUs from coming online while resizing to synchronize
+ * with new per CPU buffers being created.
+ */
+ guard(cpus_read_lock)();
+
/* prevent another thread from changing buffer sizes */
mutex_lock(&buffer->mutex);
atomic_inc(&buffer->resizing);
@@ -2091,7 +2895,6 @@ int ring_buffer_resize(struct trace_buffer *buffer, unsigned long size,
cond_resched();
}
- cpus_read_lock();
/*
* Fire off all the required work handlers
* We can't schedule on offline CPUs, but it's not necessary
@@ -2131,7 +2934,6 @@ int ring_buffer_resize(struct trace_buffer *buffer, unsigned long size,
cpu_buffer->nr_pages_to_update = 0;
}
- cpus_read_unlock();
} else {
cpu_buffer = buffer->buffers[cpu_id];
@@ -2159,8 +2961,6 @@ int ring_buffer_resize(struct trace_buffer *buffer, unsigned long size,
goto out_err;
}
- cpus_read_lock();
-
/* Can't run something on an offline CPU. */
if (!cpu_online(cpu_id))
rb_update_pages(cpu_buffer);
@@ -2179,7 +2979,6 @@ int ring_buffer_resize(struct trace_buffer *buffer, unsigned long size,
}
cpu_buffer->nr_pages_to_update = 0;
- cpus_read_unlock();
}
out:
@@ -2318,7 +3117,7 @@ rb_iter_head_event(struct ring_buffer_iter *iter)
/* Size is determined by what has been committed */
static __always_inline unsigned rb_page_size(struct buffer_page *bpage)
{
- return rb_page_commit(bpage);
+ return rb_page_commit(bpage) & ~RB_MISSED_MASK;
}
static __always_inline unsigned
@@ -2357,6 +3156,52 @@ static void rb_inc_iter(struct ring_buffer_iter *iter)
iter->next_event = 0;
}
+/* Return the index into the sub-buffers for a given sub-buffer */
+static int rb_meta_subbuf_idx(struct ring_buffer_cpu_meta *meta, void *subbuf)
+{
+ void *subbuf_array;
+
+ subbuf_array = (void *)meta + sizeof(int) * meta->nr_subbufs;
+ subbuf_array = (void *)ALIGN((unsigned long)subbuf_array, meta->subbuf_size);
+ return (subbuf - subbuf_array) / meta->subbuf_size;
+}
+
+static void rb_update_meta_head(struct ring_buffer_per_cpu *cpu_buffer,
+ struct buffer_page *next_page)
+{
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ unsigned long old_head = (unsigned long)next_page->page;
+ unsigned long new_head;
+
+ rb_inc_page(&next_page);
+ new_head = (unsigned long)next_page->page;
+
+ /*
+ * Only move it forward once, if something else came in and
+ * moved it forward, then we don't want to touch it.
+ */
+ (void)cmpxchg(&meta->head_buffer, old_head, new_head);
+}
+
+static void rb_update_meta_reader(struct ring_buffer_per_cpu *cpu_buffer,
+ struct buffer_page *reader)
+{
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ void *old_reader = cpu_buffer->reader_page->page;
+ void *new_reader = reader->page;
+ int id;
+
+ id = reader->id;
+ cpu_buffer->reader_page->id = id;
+ reader->id = 0;
+
+ meta->buffers[0] = rb_meta_subbuf_idx(meta, new_reader);
+ meta->buffers[id] = rb_meta_subbuf_idx(meta, old_reader);
+
+ /* The head pointer is the one after the reader */
+ rb_update_meta_head(cpu_buffer, reader);
+}
+
/*
* rb_handle_head_page - writer hit the head page
*
@@ -2406,6 +3251,8 @@ rb_handle_head_page(struct ring_buffer_per_cpu *cpu_buffer,
local_sub(rb_page_commit(next_page), &cpu_buffer->entries_bytes);
local_inc(&cpu_buffer->pages_lost);
+ if (cpu_buffer->ring_meta)
+ rb_update_meta_head(cpu_buffer, next_page);
/*
* The entries will be zeroed out when we move the
* tail page.
@@ -2967,6 +3814,10 @@ rb_set_commit_to_write(struct ring_buffer_per_cpu *cpu_buffer)
local_set(&cpu_buffer->commit_page->page->commit,
rb_page_write(cpu_buffer->commit_page));
rb_inc_page(&cpu_buffer->commit_page);
+ if (cpu_buffer->ring_meta) {
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ meta->commit_buffer = (unsigned long)cpu_buffer->commit_page->page;
+ }
/* add barrier to keep gcc from optimizing too much */
barrier();
}
@@ -3281,7 +4132,7 @@ static const char *show_irq_str(int bits)
return type[bits];
}
-/* Assume this is an trace event */
+/* Assume this is a trace event */
static const char *show_flags(struct ring_buffer_event *event)
{
struct trace_entry *entry;
@@ -3413,11 +4264,10 @@ static void check_buffer(struct ring_buffer_per_cpu *cpu_buffer,
struct rb_event_info *info,
unsigned long tail)
{
- struct ring_buffer_event *event;
struct buffer_data_page *bpage;
u64 ts, delta;
bool full = false;
- int e;
+ int ret;
bpage = info->tail_page->page;
@@ -3443,39 +4293,12 @@ static void check_buffer(struct ring_buffer_per_cpu *cpu_buffer,
if (atomic_inc_return(this_cpu_ptr(&checking)) != 1)
goto out;
- ts = bpage->time_stamp;
-
- for (e = 0; e < tail; e += rb_event_length(event)) {
-
- event = (struct ring_buffer_event *)(bpage->data + e);
-
- switch (event->type_len) {
-
- case RINGBUF_TYPE_TIME_EXTEND:
- delta = rb_event_time_stamp(event);
- ts += delta;
- break;
-
- case RINGBUF_TYPE_TIME_STAMP:
- delta = rb_event_time_stamp(event);
- delta = rb_fix_abs_ts(delta, ts);
- if (delta < ts) {
- buffer_warn_return("[CPU: %d]ABSOLUTE TIME WENT BACKWARDS: last ts: %lld absolute ts: %lld\n",
- cpu_buffer->cpu, ts, delta);
- }
- ts = delta;
- break;
-
- case RINGBUF_TYPE_PADDING:
- if (event->time_delta == 1)
- break;
- fallthrough;
- case RINGBUF_TYPE_DATA:
- ts += event->time_delta;
- break;
-
- default:
- RB_WARN_ON(cpu_buffer, 1);
+ ret = rb_read_data_buffer(bpage, tail, cpu_buffer->cpu, &ts, &delta);
+ if (ret < 0) {
+ if (delta < ts) {
+ buffer_warn_return("[CPU: %d]ABSOLUTE TIME WENT BACKWARDS: last ts: %lld absolute ts: %lld\n",
+ cpu_buffer->cpu, ts, delta);
+ goto out;
}
}
if ((full && ts > info->ts) ||
@@ -3650,8 +4473,13 @@ rb_reserve_next_event(struct trace_buffer *buffer,
int nr_loops = 0;
int add_ts_default;
- /* ring buffer does cmpxchg, make sure it is safe in NMI context */
- if (!IS_ENABLED(CONFIG_ARCH_HAVE_NMI_SAFE_CMPXCHG) &&
+ /*
+ * ring buffer does cmpxchg as well as atomic64 operations
+ * (which some archs use locking for atomic64), make sure this
+ * is safe in NMI context
+ */
+ if ((!IS_ENABLED(CONFIG_ARCH_HAVE_NMI_SAFE_CMPXCHG) ||
+ IS_ENABLED(CONFIG_GENERIC_ATOMIC64)) &&
(unlikely(in_nmi()))) {
return NULL;
}
@@ -3853,10 +4681,7 @@ void ring_buffer_discard_commit(struct trace_buffer *buffer,
RB_WARN_ON(buffer, !local_read(&cpu_buffer->committing));
rb_decrement_entry(cpu_buffer, event);
- if (rb_try_to_discard(cpu_buffer, event))
- goto out;
-
- out:
+ rb_try_to_discard(cpu_buffer, event);
rb_end_commit(cpu_buffer);
trace_recursive_unlock(cpu_buffer);
@@ -3934,40 +4759,22 @@ int ring_buffer_write(struct trace_buffer *buffer,
}
EXPORT_SYMBOL_GPL(ring_buffer_write);
-static bool rb_per_cpu_empty(struct ring_buffer_per_cpu *cpu_buffer)
+/*
+ * The total entries in the ring buffer is the running counter
+ * of entries entered into the ring buffer, minus the sum of
+ * the entries read from the ring buffer and the number of
+ * entries that were overwritten.
+ */
+static inline unsigned long
+rb_num_of_entries(struct ring_buffer_per_cpu *cpu_buffer)
{
- struct buffer_page *reader = cpu_buffer->reader_page;
- struct buffer_page *head = rb_set_head_page(cpu_buffer);
- struct buffer_page *commit = cpu_buffer->commit_page;
-
- /* In case of error, head will be NULL */
- if (unlikely(!head))
- return true;
-
- /* Reader should exhaust content in reader page */
- if (reader->read != rb_page_commit(reader))
- return false;
-
- /*
- * If writers are committing on the reader page, knowing all
- * committed content has been read, the ring buffer is empty.
- */
- if (commit == reader)
- return true;
-
- /*
- * If writers are committing on a page other than reader page
- * and head page, there should always be content to read.
- */
- if (commit != head)
- return false;
+ return local_read(&cpu_buffer->entries) -
+ (local_read(&cpu_buffer->overrun) + cpu_buffer->read);
+}
- /*
- * Writers are committing on the head page, we just need
- * to care about there're committed data, and the reader will
- * swap reader page with head page when it is to read data.
- */
- return rb_page_commit(commit) == 0;
+static bool rb_per_cpu_empty(struct ring_buffer_per_cpu *cpu_buffer)
+{
+ return !rb_num_of_entries(cpu_buffer);
}
/**
@@ -4072,6 +4879,24 @@ bool ring_buffer_record_is_set_on(struct trace_buffer *buffer)
}
/**
+ * ring_buffer_record_is_on_cpu - return true if the ring buffer can write
+ * @buffer: The ring buffer to see if write is enabled
+ * @cpu: The CPU to test if the ring buffer can write too
+ *
+ * Returns true if the ring buffer is in a state that it accepts writes
+ * for a particular CPU.
+ */
+bool ring_buffer_record_is_on_cpu(struct trace_buffer *buffer, int cpu)
+{
+ struct ring_buffer_per_cpu *cpu_buffer;
+
+ cpu_buffer = buffer->buffers[cpu];
+
+ return ring_buffer_record_is_set_on(buffer) &&
+ !atomic_read(&cpu_buffer->record_disabled);
+}
+
+/**
* ring_buffer_record_disable_cpu - stop all writes into the cpu_buffer
* @buffer: The ring buffer to stop writes to.
* @cpu: The CPU buffer to stop
@@ -4113,19 +4938,6 @@ void ring_buffer_record_enable_cpu(struct trace_buffer *buffer, int cpu)
}
EXPORT_SYMBOL_GPL(ring_buffer_record_enable_cpu);
-/*
- * The total entries in the ring buffer is the running counter
- * of entries entered into the ring buffer, minus the sum of
- * the entries read from the ring buffer and the number of
- * entries that were overwritten.
- */
-static inline unsigned long
-rb_num_of_entries(struct ring_buffer_per_cpu *cpu_buffer)
-{
- return local_read(&cpu_buffer->entries) -
- (local_read(&cpu_buffer->overrun) + cpu_buffer->read);
-}
-
/**
* ring_buffer_oldest_event_ts - get the oldest event timestamp from the buffer
* @buffer: The ring buffer
@@ -4416,7 +5228,7 @@ int ring_buffer_iter_empty(struct ring_buffer_iter *iter)
return ((iter->head_page == commit_page && iter->head >= commit) ||
(iter->head_page == reader && commit_page == head_page &&
head_page->read == commit &&
- iter->head == rb_page_commit(cpu_buffer->reader_page)));
+ iter->head == rb_page_size(cpu_buffer->reader_page)));
}
EXPORT_SYMBOL_GPL(ring_buffer_iter_empty);
@@ -4573,7 +5385,7 @@ rb_get_reader_page(struct ring_buffer_per_cpu *cpu_buffer)
* moving it. The page before the header page has the
* flag bit '1' set if it is pointing to the page we want.
* but if the writer is in the process of moving it
- * than it will be '2' or already moved '0'.
+ * then it will be '2' or already moved '0'.
*/
ret = rb_head_page_replace(reader, cpu_buffer->reader_page);
@@ -4584,6 +5396,9 @@ rb_get_reader_page(struct ring_buffer_per_cpu *cpu_buffer)
if (!ret)
goto spin;
+ if (cpu_buffer->ring_meta)
+ rb_update_meta_reader(cpu_buffer, reader);
+
/*
* Yay! We succeeded in replacing the page.
*
@@ -4592,6 +5407,7 @@ rb_get_reader_page(struct ring_buffer_per_cpu *cpu_buffer)
rb_list_head(reader->list.next)->prev = &cpu_buffer->reader_page->list;
rb_inc_page(&cpu_buffer->head_page);
+ cpu_buffer->cnt++;
local_inc(&cpu_buffer->pages_read);
/* Finally update the reader page to the new head */
@@ -5036,13 +5852,9 @@ EXPORT_SYMBOL_GPL(ring_buffer_consume);
* @flags: gfp flags to use for memory allocation
*
* This performs the initial preparations necessary to iterate
- * through the buffer. Memory is allocated, buffer recording
+ * through the buffer. Memory is allocated, buffer resizing
* is disabled, and the iterator pointer is returned to the caller.
*
- * Disabling buffer recording prevents the reading from being
- * corrupted. This is not a consuming read, so a producer is not
- * expected.
- *
* After a sequence of ring_buffer_read_prepare calls, the user is
* expected to make at least one call to ring_buffer_read_prepare_sync.
* Afterwards, ring_buffer_read_start is invoked to get things going
@@ -5129,24 +5941,15 @@ EXPORT_SYMBOL_GPL(ring_buffer_read_start);
* ring_buffer_read_finish - finish reading the iterator of the buffer
* @iter: The iterator retrieved by ring_buffer_start
*
- * This re-enables the recording to the buffer, and frees the
- * iterator.
+ * This re-enables resizing of the buffer, and frees the iterator.
*/
void
ring_buffer_read_finish(struct ring_buffer_iter *iter)
{
struct ring_buffer_per_cpu *cpu_buffer = iter->cpu_buffer;
- unsigned long flags;
- /*
- * Ring buffer is disabled from recording, here's a good place
- * to check the integrity of the ring buffer.
- * Must prevent readers from trying to read, as the check
- * clears the HEAD page and readers require it.
- */
- raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+ /* Use this opportunity to check the integrity of the ring buffer. */
rb_check_pages(cpu_buffer);
- raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
atomic_dec(&cpu_buffer->resize_disabled);
kfree(iter->event);
@@ -5211,6 +6014,60 @@ static void rb_clear_buffer_page(struct buffer_page *page)
page->read = 0;
}
+/*
+ * When the buffer is memory mapped to user space, each sub buffer
+ * has a unique id that is used by the meta data to tell the user
+ * where the current reader page is.
+ *
+ * For a normal allocated ring buffer, the id is saved in the buffer page
+ * id field, and updated via this function.
+ *
+ * But for a fixed memory mapped buffer, the id is already assigned for
+ * fixed memory ording in the memory layout and can not be used. Instead
+ * the index of where the page lies in the memory layout is used.
+ *
+ * For the normal pages, set the buffer page id with the passed in @id
+ * value and return that.
+ *
+ * For fixed memory mapped pages, get the page index in the memory layout
+ * and return that as the id.
+ */
+static int rb_page_id(struct ring_buffer_per_cpu *cpu_buffer,
+ struct buffer_page *bpage, int id)
+{
+ /*
+ * For boot buffers, the id is the index,
+ * otherwise, set the buffer page with this id
+ */
+ if (cpu_buffer->ring_meta)
+ id = rb_meta_subbuf_idx(cpu_buffer->ring_meta, bpage->page);
+ else
+ bpage->id = id;
+
+ return id;
+}
+
+static void rb_update_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
+{
+ struct trace_buffer_meta *meta = cpu_buffer->meta_page;
+
+ if (!meta)
+ return;
+
+ meta->reader.read = cpu_buffer->reader_page->read;
+ meta->reader.id = rb_page_id(cpu_buffer, cpu_buffer->reader_page,
+ cpu_buffer->reader_page->id);
+
+ meta->reader.lost_events = cpu_buffer->lost_events;
+
+ meta->entries = local_read(&cpu_buffer->entries);
+ meta->overrun = local_read(&cpu_buffer->overrun);
+ meta->read = cpu_buffer->read;
+
+ /* Some archs do not have data cache coherency between kernel and user-space */
+ flush_kernel_vmap_range(cpu_buffer->meta_page, PAGE_SIZE);
+}
+
static void
rb_reset_cpu(struct ring_buffer_per_cpu *cpu_buffer)
{
@@ -5257,26 +6114,29 @@ rb_reset_cpu(struct ring_buffer_per_cpu *cpu_buffer)
rb_head_page_activate(cpu_buffer);
cpu_buffer->pages_removed = 0;
+
+ if (cpu_buffer->mapped) {
+ rb_update_meta_page(cpu_buffer);
+ if (cpu_buffer->ring_meta) {
+ struct ring_buffer_cpu_meta *meta = cpu_buffer->ring_meta;
+ meta->commit_buffer = meta->head_buffer;
+ }
+ }
}
/* Must have disabled the cpu buffer then done a synchronize_rcu */
static void reset_disabled_cpu_buffer(struct ring_buffer_per_cpu *cpu_buffer)
{
- unsigned long flags;
-
- raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+ guard(raw_spinlock_irqsave)(&cpu_buffer->reader_lock);
if (RB_WARN_ON(cpu_buffer, local_read(&cpu_buffer->committing)))
- goto out;
+ return;
arch_spin_lock(&cpu_buffer->lock);
rb_reset_cpu(cpu_buffer);
arch_spin_unlock(&cpu_buffer->lock);
-
- out:
- raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
}
/**
@@ -5464,31 +6324,33 @@ int ring_buffer_swap_cpu(struct trace_buffer *buffer_a,
if (!cpumask_test_cpu(cpu, buffer_a->cpumask) ||
!cpumask_test_cpu(cpu, buffer_b->cpumask))
- goto out;
+ return -EINVAL;
cpu_buffer_a = buffer_a->buffers[cpu];
cpu_buffer_b = buffer_b->buffers[cpu];
+ /* It's up to the callers to not try to swap mapped buffers */
+ if (WARN_ON_ONCE(cpu_buffer_a->mapped || cpu_buffer_b->mapped))
+ return -EBUSY;
+
/* At least make sure the two buffers are somewhat the same */
if (cpu_buffer_a->nr_pages != cpu_buffer_b->nr_pages)
- goto out;
+ return -EINVAL;
if (buffer_a->subbuf_order != buffer_b->subbuf_order)
- goto out;
-
- ret = -EAGAIN;
+ return -EINVAL;
if (atomic_read(&buffer_a->record_disabled))
- goto out;
+ return -EAGAIN;
if (atomic_read(&buffer_b->record_disabled))
- goto out;
+ return -EAGAIN;
if (atomic_read(&cpu_buffer_a->record_disabled))
- goto out;
+ return -EAGAIN;
if (atomic_read(&cpu_buffer_b->record_disabled))
- goto out;
+ return -EAGAIN;
/*
* We can't do a synchronize_rcu here because this
@@ -5525,7 +6387,6 @@ int ring_buffer_swap_cpu(struct trace_buffer *buffer_a,
out_dec:
atomic_dec(&cpu_buffer_a->record_disabled);
atomic_dec(&cpu_buffer_b->record_disabled);
-out:
return ret;
}
EXPORT_SYMBOL_GPL(ring_buffer_swap_cpu);
@@ -5579,7 +6440,7 @@ ring_buffer_alloc_read_page(struct trace_buffer *buffer, int cpu)
goto out;
page = alloc_pages_node(cpu_to_node(cpu),
- GFP_KERNEL | __GFP_NORETRY | __GFP_ZERO,
+ GFP_KERNEL | __GFP_NORETRY | __GFP_COMP | __GFP_ZERO,
cpu_buffer->buffer->subbuf_order);
if (!page) {
kfree(bpage);
@@ -5684,43 +6545,42 @@ int ring_buffer_read_page(struct trace_buffer *buffer,
struct buffer_data_page *bpage;
struct buffer_page *reader;
unsigned long missed_events;
- unsigned long flags;
unsigned int commit;
unsigned int read;
u64 save_timestamp;
- int ret = -1;
if (!cpumask_test_cpu(cpu, buffer->cpumask))
- goto out;
+ return -1;
/*
* If len is not big enough to hold the page header, then
* we can not copy anything.
*/
if (len <= BUF_PAGE_HDR_SIZE)
- goto out;
+ return -1;
len -= BUF_PAGE_HDR_SIZE;
if (!data_page || !data_page->data)
- goto out;
+ return -1;
+
if (data_page->order != buffer->subbuf_order)
- goto out;
+ return -1;
bpage = data_page->data;
if (!bpage)
- goto out;
+ return -1;
- raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+ guard(raw_spinlock_irqsave)(&cpu_buffer->reader_lock);
reader = rb_get_reader_page(cpu_buffer);
if (!reader)
- goto out_unlock;
+ return -1;
event = rb_reader_event(cpu_buffer);
read = reader->read;
- commit = rb_page_commit(reader);
+ commit = rb_page_size(reader);
/* Check if any events were dropped */
missed_events = cpu_buffer->lost_events;
@@ -5733,7 +6593,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;
@@ -5748,7 +6609,7 @@ int ring_buffer_read_page(struct trace_buffer *buffer,
if (full &&
(!read || (len < (commit - read)) ||
cpu_buffer->reader_page == cpu_buffer->commit_page))
- goto out_unlock;
+ return -1;
if (len > (commit - read))
len = (commit - read);
@@ -5757,7 +6618,7 @@ int ring_buffer_read_page(struct trace_buffer *buffer,
size = rb_event_ts_length(event);
if (len < size)
- goto out_unlock;
+ return -1;
/* save the current timestamp, since the user will need it */
save_timestamp = cpu_buffer->read_stamp;
@@ -5796,7 +6657,7 @@ int ring_buffer_read_page(struct trace_buffer *buffer,
} else {
/* update the entry counter */
cpu_buffer->read += rb_page_entries(reader);
- cpu_buffer->read_bytes += rb_page_commit(reader);
+ cpu_buffer->read_bytes += rb_page_size(reader);
/* swap the pages */
rb_init_page(bpage);
@@ -5815,7 +6676,6 @@ int ring_buffer_read_page(struct trace_buffer *buffer,
if (reader->real_end)
local_set(&bpage->commit, reader->real_end);
}
- ret = read;
cpu_buffer->lost_events = 0;
@@ -5842,11 +6702,7 @@ int ring_buffer_read_page(struct trace_buffer *buffer,
if (commit < buffer->subbuf_size)
memset(&bpage->data[commit], 0, buffer->subbuf_size - commit);
- out_unlock:
- raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
-
- out:
- return ret;
+ return read;
}
EXPORT_SYMBOL_GPL(ring_buffer_read_page);
@@ -5956,6 +6812,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);
@@ -5980,39 +6841,39 @@ int ring_buffer_subbuf_order_set(struct trace_buffer *buffer, int order)
}
for_each_buffer_cpu(buffer, cpu) {
+ struct buffer_data_page *old_free_data_page;
+ struct list_head old_pages;
+ unsigned long flags;
if (!cpumask_test_cpu(cpu, buffer->cpumask))
continue;
cpu_buffer = buffer->buffers[cpu];
+ raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+
/* Clear the head bit to make the link list normal to read */
rb_head_page_deactivate(cpu_buffer);
- /* Now walk the list and free all the old sub buffers */
- list_for_each_entry_safe(bpage, tmp, cpu_buffer->pages, list) {
- list_del_init(&bpage->list);
- free_buffer_page(bpage);
- }
- /* The above loop stopped an the last page needing to be freed */
- bpage = list_entry(cpu_buffer->pages, struct buffer_page, list);
- free_buffer_page(bpage);
-
- /* Free the current reader page */
- free_buffer_page(cpu_buffer->reader_page);
+ /*
+ * Collect buffers from the cpu_buffer pages list and the
+ * reader_page on old_pages, so they can be freed later when not
+ * under a spinlock. The pages list is a linked list with no
+ * head, adding old_pages turns it into a regular list with
+ * old_pages being the head.
+ */
+ list_add(&old_pages, cpu_buffer->pages);
+ list_add(&cpu_buffer->reader_page->list, &old_pages);
/* One page was allocated for the reader page */
cpu_buffer->reader_page = list_entry(cpu_buffer->new_pages.next,
struct buffer_page, list);
list_del_init(&cpu_buffer->reader_page->list);
- /* The cpu_buffer pages are a link list with no head */
+ /* Install the new pages, remove the head from the list */
cpu_buffer->pages = cpu_buffer->new_pages.next;
- cpu_buffer->new_pages.next->prev = cpu_buffer->new_pages.prev;
- cpu_buffer->new_pages.prev->next = cpu_buffer->new_pages.next;
-
- /* Clear the new_pages list */
- INIT_LIST_HEAD(&cpu_buffer->new_pages);
+ list_del_init(&cpu_buffer->new_pages);
+ cpu_buffer->cnt++;
cpu_buffer->head_page
= list_entry(cpu_buffer->pages, struct buffer_page, list);
@@ -6021,11 +6882,20 @@ int ring_buffer_subbuf_order_set(struct trace_buffer *buffer, int order)
cpu_buffer->nr_pages = cpu_buffer->nr_pages_to_update;
cpu_buffer->nr_pages_to_update = 0;
- free_pages((unsigned long)cpu_buffer->free_page, old_order);
+ old_free_data_page = cpu_buffer->free_page;
cpu_buffer->free_page = NULL;
rb_head_page_activate(cpu_buffer);
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+
+ /* Free old sub buffers */
+ list_for_each_entry_safe(bpage, tmp, &old_pages, list) {
+ list_del_init(&bpage->list);
+ free_buffer_page(bpage);
+ }
+ free_pages((unsigned long)old_free_data_page, old_order);
+
rb_check_pages(cpu_buffer);
}
@@ -6057,6 +6927,440 @@ error:
}
EXPORT_SYMBOL_GPL(ring_buffer_subbuf_order_set);
+static int rb_alloc_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
+{
+ struct page *page;
+
+ if (cpu_buffer->meta_page)
+ return 0;
+
+ page = alloc_page(GFP_USER | __GFP_ZERO);
+ if (!page)
+ return -ENOMEM;
+
+ cpu_buffer->meta_page = page_to_virt(page);
+
+ return 0;
+}
+
+static void rb_free_meta_page(struct ring_buffer_per_cpu *cpu_buffer)
+{
+ unsigned long addr = (unsigned long)cpu_buffer->meta_page;
+
+ 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 cnt = 0;
+ int id = 0;
+
+ id = rb_page_id(cpu_buffer, cpu_buffer->reader_page, id);
+ subbuf_ids[id++] = (unsigned long)cpu_buffer->reader_page->page;
+ cnt++;
+
+ first_subbuf = subbuf = rb_set_head_page(cpu_buffer);
+ do {
+ id = rb_page_id(cpu_buffer, subbuf, id);
+
+ if (WARN_ON(id >= nr_subbufs))
+ break;
+
+ subbuf_ids[id] = (unsigned long)subbuf->page;
+
+ rb_inc_page(&subbuf);
+ id++;
+ cnt++;
+ } while (subbuf != first_subbuf);
+
+ WARN_ON(cnt != nr_subbufs);
+
+ /* install subbuf ID to kern VA translation */
+ cpu_buffer->subbuf_ids = subbuf_ids;
+
+ meta->meta_struct_len = sizeof(*meta);
+ meta->nr_subbufs = nr_subbufs;
+ meta->subbuf_size = cpu_buffer->buffer->subbuf_size + BUF_PAGE_HDR_SIZE;
+ meta->meta_page_size = meta->subbuf_size;
+
+ rb_update_meta_page(cpu_buffer);
+}
+
+static 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->user_mapped) {
+ mutex_unlock(&cpu_buffer->mapping_lock);
+ return ERR_PTR(-ENODEV);
+ }
+
+ return cpu_buffer;
+}
+
+static void rb_put_mapped_buffer(struct ring_buffer_per_cpu *cpu_buffer)
+{
+ mutex_unlock(&cpu_buffer->mapping_lock);
+}
+
+/*
+ * Fast-path for rb_buffer_(un)map(). Called whenever the meta-page doesn't need
+ * to be set-up or torn-down.
+ */
+static int __rb_inc_dec_mapped(struct ring_buffer_per_cpu *cpu_buffer,
+ bool inc)
+{
+ unsigned long flags;
+
+ lockdep_assert_held(&cpu_buffer->mapping_lock);
+
+ /* mapped is always greater or equal to user_mapped */
+ if (WARN_ON(cpu_buffer->mapped < cpu_buffer->user_mapped))
+ return -EINVAL;
+
+ if (inc && cpu_buffer->mapped == UINT_MAX)
+ return -EBUSY;
+
+ if (WARN_ON(!inc && cpu_buffer->user_mapped == 0))
+ return -EINVAL;
+
+ mutex_lock(&cpu_buffer->buffer->mutex);
+ raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+
+ if (inc) {
+ cpu_buffer->user_mapped++;
+ cpu_buffer->mapped++;
+ } else {
+ cpu_buffer->user_mapped--;
+ cpu_buffer->mapped--;
+ }
+
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+ mutex_unlock(&cpu_buffer->buffer->mutex);
+
+ return 0;
+}
+
+/*
+ * +--------------+ pgoff == 0
+ * | meta page |
+ * +--------------+ pgoff == 1
+ * | subbuffer 0 |
+ * | |
+ * +--------------+ pgoff == (1 + (1 << subbuf_order))
+ * | subbuffer 1 |
+ * | |
+ * ...
+ */
+#ifdef CONFIG_MMU
+static int __rb_map_vma(struct ring_buffer_per_cpu *cpu_buffer,
+ struct vm_area_struct *vma)
+{
+ unsigned long nr_subbufs, nr_pages, nr_vma_pages, pgoff = vma->vm_pgoff;
+ unsigned int subbuf_pages, subbuf_order;
+ struct page **pages __free(kfree) = NULL;
+ int p = 0, s = 0;
+ int err;
+
+ /* Refuse MP_PRIVATE or writable mappings */
+ if (vma->vm_flags & VM_WRITE || vma->vm_flags & VM_EXEC ||
+ !(vma->vm_flags & VM_MAYSHARE))
+ return -EPERM;
+
+ subbuf_order = cpu_buffer->buffer->subbuf_order;
+ subbuf_pages = 1 << subbuf_order;
+
+ if (subbuf_order && pgoff % subbuf_pages)
+ return -EINVAL;
+
+ /*
+ * Make sure the mapping cannot become writable later. Also tell the VM
+ * to not touch these pages (VM_DONTCOPY | VM_DONTEXPAND).
+ */
+ vm_flags_mod(vma, VM_DONTCOPY | VM_DONTEXPAND | VM_DONTDUMP,
+ VM_MAYWRITE);
+
+ lockdep_assert_held(&cpu_buffer->mapping_lock);
+
+ nr_subbufs = cpu_buffer->nr_pages + 1; /* + reader-subbuf */
+ nr_pages = ((nr_subbufs + 1) << subbuf_order); /* + meta-page */
+ if (nr_pages <= pgoff)
+ return -EINVAL;
+
+ nr_pages -= pgoff;
+
+ nr_vma_pages = vma_pages(vma);
+ if (!nr_vma_pages || nr_vma_pages > nr_pages)
+ return -EINVAL;
+
+ nr_pages = nr_vma_pages;
+
+ pages = kcalloc(nr_pages, sizeof(*pages), GFP_KERNEL);
+ if (!pages)
+ return -ENOMEM;
+
+ if (!pgoff) {
+ unsigned long meta_page_padding;
+
+ pages[p++] = virt_to_page(cpu_buffer->meta_page);
+
+ /*
+ * Pad with the zero-page to align the meta-page with the
+ * sub-buffers.
+ */
+ meta_page_padding = subbuf_pages - 1;
+ while (meta_page_padding-- && p < nr_pages) {
+ unsigned long __maybe_unused zero_addr =
+ vma->vm_start + (PAGE_SIZE * p);
+
+ pages[p++] = ZERO_PAGE(zero_addr);
+ }
+ } else {
+ /* Skip the meta-page */
+ pgoff -= subbuf_pages;
+
+ s += pgoff / subbuf_pages;
+ }
+
+ while (p < nr_pages) {
+ struct page *page;
+ int off = 0;
+
+ if (WARN_ON_ONCE(s >= nr_subbufs))
+ return -EINVAL;
+
+ page = virt_to_page((void *)cpu_buffer->subbuf_ids[s]);
+
+ for (; off < (1 << (subbuf_order)); off++, page++) {
+ if (p >= nr_pages)
+ break;
+
+ pages[p++] = page;
+ }
+ s++;
+ }
+
+ err = vm_insert_pages(vma, vma->vm_start, pages, &nr_pages);
+
+ return err;
+}
+#else
+static int __rb_map_vma(struct ring_buffer_per_cpu *cpu_buffer,
+ struct vm_area_struct *vma)
+{
+ return -EOPNOTSUPP;
+}
+#endif
+
+int ring_buffer_map(struct trace_buffer *buffer, int cpu,
+ struct vm_area_struct *vma)
+{
+ struct ring_buffer_per_cpu *cpu_buffer;
+ unsigned long flags, *subbuf_ids;
+ int err;
+
+ if (!cpumask_test_cpu(cpu, buffer->cpumask))
+ return -EINVAL;
+
+ cpu_buffer = buffer->buffers[cpu];
+
+ guard(mutex)(&cpu_buffer->mapping_lock);
+
+ if (cpu_buffer->user_mapped) {
+ err = __rb_map_vma(cpu_buffer, vma);
+ if (!err)
+ err = __rb_inc_dec_mapped(cpu_buffer, true);
+ return err;
+ }
+
+ /* prevent another thread from changing buffer/sub-buffer sizes */
+ guard(mutex)(&buffer->mutex);
+
+ err = rb_alloc_meta_page(cpu_buffer);
+ if (err)
+ return err;
+
+ /* subbuf_ids include the reader while nr_pages does not */
+ subbuf_ids = kcalloc(cpu_buffer->nr_pages + 1, sizeof(*subbuf_ids), GFP_KERNEL);
+ if (!subbuf_ids) {
+ rb_free_meta_page(cpu_buffer);
+ return -ENOMEM;
+ }
+
+ 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);
+
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+
+ err = __rb_map_vma(cpu_buffer, vma);
+ if (!err) {
+ raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+ /* This is the first time it is mapped by user */
+ cpu_buffer->mapped++;
+ cpu_buffer->user_mapped = 1;
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+ } else {
+ kfree(cpu_buffer->subbuf_ids);
+ cpu_buffer->subbuf_ids = NULL;
+ rb_free_meta_page(cpu_buffer);
+ atomic_dec(&cpu_buffer->resize_disabled);
+ }
+
+ return 0;
+}
+
+int ring_buffer_unmap(struct trace_buffer *buffer, int cpu)
+{
+ struct ring_buffer_per_cpu *cpu_buffer;
+ unsigned long flags;
+
+ if (!cpumask_test_cpu(cpu, buffer->cpumask))
+ return -EINVAL;
+
+ cpu_buffer = buffer->buffers[cpu];
+
+ guard(mutex)(&cpu_buffer->mapping_lock);
+
+ if (!cpu_buffer->user_mapped) {
+ return -ENODEV;
+ } else if (cpu_buffer->user_mapped > 1) {
+ __rb_inc_dec_mapped(cpu_buffer, false);
+ return 0;
+ }
+
+ guard(mutex)(&buffer->mutex);
+ raw_spin_lock_irqsave(&cpu_buffer->reader_lock, flags);
+
+ /* This is the last user space mapping */
+ if (!WARN_ON_ONCE(cpu_buffer->mapped < cpu_buffer->user_mapped))
+ cpu_buffer->mapped--;
+ cpu_buffer->user_mapped = 0;
+
+ raw_spin_unlock_irqrestore(&cpu_buffer->reader_lock, flags);
+
+ kfree(cpu_buffer->subbuf_ids);
+ cpu_buffer->subbuf_ids = NULL;
+ rb_free_meta_page(cpu_buffer);
+ atomic_dec(&cpu_buffer->resize_disabled);
+
+ return 0;
+}
+
+int ring_buffer_map_get_reader(struct trace_buffer *buffer, int cpu)
+{
+ struct ring_buffer_per_cpu *cpu_buffer;
+ struct buffer_page *reader;
+ unsigned long missed_events;
+ 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;
+ }
+
+ reader = rb_get_reader_page(cpu_buffer);
+ if (WARN_ON(!reader))
+ goto out;
+
+ /* Check if any events were dropped */
+ missed_events = cpu_buffer->lost_events;
+
+ if (missed_events) {
+ if (cpu_buffer->reader_page != cpu_buffer->commit_page) {
+ struct buffer_data_page *bpage = reader->page;
+ unsigned int commit;
+ /*
+ * Use the real_end for the data size,
+ * This gives us a chance to store the lost events
+ * on the page.
+ */
+ if (reader->real_end)
+ local_set(&bpage->commit, reader->real_end);
+ /*
+ * If there is room at the end of the page to save the
+ * missed events, then record it there.
+ */
+ commit = rb_page_size(reader);
+ if (buffer->subbuf_size - commit >= sizeof(missed_events)) {
+ memcpy(&bpage->data[commit], &missed_events,
+ sizeof(missed_events));
+ local_add(RB_MISSED_STORED, &bpage->commit);
+ }
+ local_add(RB_MISSED_EVENTS, &bpage->commit);
+ } else if (!WARN_ONCE(cpu_buffer->reader_page == cpu_buffer->tail_page,
+ "Reader on commit with %ld missed events",
+ missed_events)) {
+ /*
+ * There shouldn't be any missed events if the tail_page
+ * is on the reader page. But if the tail page is not on the
+ * reader page and the commit_page is, that would mean that
+ * there's a commit_overrun (an interrupt preempted an
+ * addition of an event and then filled the buffer
+ * with new events). In this case it's not an
+ * error, but it should still be reported.
+ *
+ * TODO: Add missed events to the page for user space to know.
+ */
+ pr_info("Ring buffer [%d] commit overrun lost %ld events at timestamp:%lld\n",
+ cpu, missed_events, cpu_buffer->reader_page->page->time_stamp);
+ }
+ }
+
+ cpu_buffer->lost_events = 0;
+
+ goto consume;
+
+out:
+ /* Some archs do not have data cache coherency between kernel and user-space */
+ flush_kernel_vmap_range(cpu_buffer->reader_page->page,
+ buffer->subbuf_size + BUF_PAGE_HDR_SIZE);
+
+ rb_update_meta_page(cpu_buffer);
+
+ 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
@@ -6180,9 +7484,9 @@ static __init int rb_write_something(struct rb_test_data *data, bool nested)
/* Ignore dropped events before test starts. */
if (started) {
if (nested)
- data->bytes_dropped += len;
- else
data->bytes_dropped_nested += len;
+ else
+ data->bytes_dropped += len;
}
return len;
}