Skip to content

Commit fb84b82

Browse files
anakryikoborkmann
authored andcommitted
libbpf: add perf buffer API
BPF_MAP_TYPE_PERF_EVENT_ARRAY map is often used to send data from BPF program to user space for additional processing. libbpf already has very low-level API to read single CPU perf buffer, bpf_perf_event_read_simple(), but it's hard to use and requires a lot of code to set everything up. This patch adds perf_buffer abstraction on top of it, abstracting setting up and polling per-CPU logic into simple and convenient API, similar to what BCC provides. perf_buffer__new() sets up per-CPU ring buffers and updates corresponding BPF map entries. It accepts two user-provided callbacks: one for handling raw samples and one for get notifications of lost samples due to buffer overflow. perf_buffer__new_raw() is similar, but provides more control over how perf events are set up (by accepting user-provided perf_event_attr), how they are handled (perf_event_header pointer is passed directly to user-provided callback), and on which CPUs ring buffers are created (it's possible to provide a list of CPUs and corresponding map keys to update). This API allows advanced users fuller control. perf_buffer__poll() is used to fetch ring buffer data across all CPUs, utilizing epoll instance. perf_buffer__free() does corresponding clean up and unsets FDs from BPF map. All APIs are not thread-safe. User should ensure proper locking/coordination if used in multi-threaded set up. Signed-off-by: Andrii Nakryiko <[email protected]> Acked-by: Yonghong Song <[email protected]> Signed-off-by: Daniel Borkmann <[email protected]>
1 parent c3ec002 commit fb84b82

File tree

3 files changed

+419
-0
lines changed

3 files changed

+419
-0
lines changed

tools/lib/bpf/libbpf.c

Lines changed: 366 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,9 @@
3232
#include <linux/limits.h>
3333
#include <linux/perf_event.h>
3434
#include <linux/ring_buffer.h>
35+
#include <sys/epoll.h>
3536
#include <sys/ioctl.h>
37+
#include <sys/mman.h>
3638
#include <sys/stat.h>
3739
#include <sys/types.h>
3840
#include <sys/vfs.h>
@@ -4354,6 +4356,370 @@ bpf_perf_event_read_simple(void *mmap_mem, size_t mmap_size, size_t page_size,
43544356
return ret;
43554357
}
43564358

4359+
struct perf_buffer;
4360+
4361+
struct perf_buffer_params {
4362+
struct perf_event_attr *attr;
4363+
/* if event_cb is specified, it takes precendence */
4364+
perf_buffer_event_fn event_cb;
4365+
/* sample_cb and lost_cb are higher-level common-case callbacks */
4366+
perf_buffer_sample_fn sample_cb;
4367+
perf_buffer_lost_fn lost_cb;
4368+
void *ctx;
4369+
int cpu_cnt;
4370+
int *cpus;
4371+
int *map_keys;
4372+
};
4373+
4374+
struct perf_cpu_buf {
4375+
struct perf_buffer *pb;
4376+
void *base; /* mmap()'ed memory */
4377+
void *buf; /* for reconstructing segmented data */
4378+
size_t buf_size;
4379+
int fd;
4380+
int cpu;
4381+
int map_key;
4382+
};
4383+
4384+
struct perf_buffer {
4385+
perf_buffer_event_fn event_cb;
4386+
perf_buffer_sample_fn sample_cb;
4387+
perf_buffer_lost_fn lost_cb;
4388+
void *ctx; /* passed into callbacks */
4389+
4390+
size_t page_size;
4391+
size_t mmap_size;
4392+
struct perf_cpu_buf **cpu_bufs;
4393+
struct epoll_event *events;
4394+
int cpu_cnt;
4395+
int epoll_fd; /* perf event FD */
4396+
int map_fd; /* BPF_MAP_TYPE_PERF_EVENT_ARRAY BPF map FD */
4397+
};
4398+
4399+
static void perf_buffer__free_cpu_buf(struct perf_buffer *pb,
4400+
struct perf_cpu_buf *cpu_buf)
4401+
{
4402+
if (!cpu_buf)
4403+
return;
4404+
if (cpu_buf->base &&
4405+
munmap(cpu_buf->base, pb->mmap_size + pb->page_size))
4406+
pr_warning("failed to munmap cpu_buf #%d\n", cpu_buf->cpu);
4407+
if (cpu_buf->fd >= 0) {
4408+
ioctl(cpu_buf->fd, PERF_EVENT_IOC_DISABLE, 0);
4409+
close(cpu_buf->fd);
4410+
}
4411+
free(cpu_buf->buf);
4412+
free(cpu_buf);
4413+
}
4414+
4415+
void perf_buffer__free(struct perf_buffer *pb)
4416+
{
4417+
int i;
4418+
4419+
if (!pb)
4420+
return;
4421+
if (pb->cpu_bufs) {
4422+
for (i = 0; i < pb->cpu_cnt && pb->cpu_bufs[i]; i++) {
4423+
struct perf_cpu_buf *cpu_buf = pb->cpu_bufs[i];
4424+
4425+
bpf_map_delete_elem(pb->map_fd, &cpu_buf->map_key);
4426+
perf_buffer__free_cpu_buf(pb, cpu_buf);
4427+
}
4428+
free(pb->cpu_bufs);
4429+
}
4430+
if (pb->epoll_fd >= 0)
4431+
close(pb->epoll_fd);
4432+
free(pb->events);
4433+
free(pb);
4434+
}
4435+
4436+
static struct perf_cpu_buf *
4437+
perf_buffer__open_cpu_buf(struct perf_buffer *pb, struct perf_event_attr *attr,
4438+
int cpu, int map_key)
4439+
{
4440+
struct perf_cpu_buf *cpu_buf;
4441+
char msg[STRERR_BUFSIZE];
4442+
int err;
4443+
4444+
cpu_buf = calloc(1, sizeof(*cpu_buf));
4445+
if (!cpu_buf)
4446+
return ERR_PTR(-ENOMEM);
4447+
4448+
cpu_buf->pb = pb;
4449+
cpu_buf->cpu = cpu;
4450+
cpu_buf->map_key = map_key;
4451+
4452+
cpu_buf->fd = syscall(__NR_perf_event_open, attr, -1 /* pid */, cpu,
4453+
-1, PERF_FLAG_FD_CLOEXEC);
4454+
if (cpu_buf->fd < 0) {
4455+
err = -errno;
4456+
pr_warning("failed to open perf buffer event on cpu #%d: %s\n",
4457+
cpu, libbpf_strerror_r(err, msg, sizeof(msg)));
4458+
goto error;
4459+
}
4460+
4461+
cpu_buf->base = mmap(NULL, pb->mmap_size + pb->page_size,
4462+
PROT_READ | PROT_WRITE, MAP_SHARED,
4463+
cpu_buf->fd, 0);
4464+
if (cpu_buf->base == MAP_FAILED) {
4465+
cpu_buf->base = NULL;
4466+
err = -errno;
4467+
pr_warning("failed to mmap perf buffer on cpu #%d: %s\n",
4468+
cpu, libbpf_strerror_r(err, msg, sizeof(msg)));
4469+
goto error;
4470+
}
4471+
4472+
if (ioctl(cpu_buf->fd, PERF_EVENT_IOC_ENABLE, 0) < 0) {
4473+
err = -errno;
4474+
pr_warning("failed to enable perf buffer event on cpu #%d: %s\n",
4475+
cpu, libbpf_strerror_r(err, msg, sizeof(msg)));
4476+
goto error;
4477+
}
4478+
4479+
return cpu_buf;
4480+
4481+
error:
4482+
perf_buffer__free_cpu_buf(pb, cpu_buf);
4483+
return (struct perf_cpu_buf *)ERR_PTR(err);
4484+
}
4485+
4486+
static struct perf_buffer *__perf_buffer__new(int map_fd, size_t page_cnt,
4487+
struct perf_buffer_params *p);
4488+
4489+
struct perf_buffer *perf_buffer__new(int map_fd, size_t page_cnt,
4490+
const struct perf_buffer_opts *opts)
4491+
{
4492+
struct perf_buffer_params p = {};
4493+
struct perf_event_attr attr = {
4494+
.config = PERF_COUNT_SW_BPF_OUTPUT,
4495+
.type = PERF_TYPE_SOFTWARE,
4496+
.sample_type = PERF_SAMPLE_RAW,
4497+
.sample_period = 1,
4498+
.wakeup_events = 1,
4499+
};
4500+
4501+
p.attr = &attr;
4502+
p.sample_cb = opts ? opts->sample_cb : NULL;
4503+
p.lost_cb = opts ? opts->lost_cb : NULL;
4504+
p.ctx = opts ? opts->ctx : NULL;
4505+
4506+
return __perf_buffer__new(map_fd, page_cnt, &p);
4507+
}
4508+
4509+
struct perf_buffer *
4510+
perf_buffer__new_raw(int map_fd, size_t page_cnt,
4511+
const struct perf_buffer_raw_opts *opts)
4512+
{
4513+
struct perf_buffer_params p = {};
4514+
4515+
p.attr = opts->attr;
4516+
p.event_cb = opts->event_cb;
4517+
p.ctx = opts->ctx;
4518+
p.cpu_cnt = opts->cpu_cnt;
4519+
p.cpus = opts->cpus;
4520+
p.map_keys = opts->map_keys;
4521+
4522+
return __perf_buffer__new(map_fd, page_cnt, &p);
4523+
}
4524+
4525+
static struct perf_buffer *__perf_buffer__new(int map_fd, size_t page_cnt,
4526+
struct perf_buffer_params *p)
4527+
{
4528+
struct bpf_map_info map = {};
4529+
char msg[STRERR_BUFSIZE];
4530+
struct perf_buffer *pb;
4531+
__u32 map_info_len;
4532+
int err, i;
4533+
4534+
if (page_cnt & (page_cnt - 1)) {
4535+
pr_warning("page count should be power of two, but is %zu\n",
4536+
page_cnt);
4537+
return ERR_PTR(-EINVAL);
4538+
}
4539+
4540+
map_info_len = sizeof(map);
4541+
err = bpf_obj_get_info_by_fd(map_fd, &map, &map_info_len);
4542+
if (err) {
4543+
err = -errno;
4544+
pr_warning("failed to get map info for map FD %d: %s\n",
4545+
map_fd, libbpf_strerror_r(err, msg, sizeof(msg)));
4546+
return ERR_PTR(err);
4547+
}
4548+
4549+
if (map.type != BPF_MAP_TYPE_PERF_EVENT_ARRAY) {
4550+
pr_warning("map '%s' should be BPF_MAP_TYPE_PERF_EVENT_ARRAY\n",
4551+
map.name);
4552+
return ERR_PTR(-EINVAL);
4553+
}
4554+
4555+
pb = calloc(1, sizeof(*pb));
4556+
if (!pb)
4557+
return ERR_PTR(-ENOMEM);
4558+
4559+
pb->event_cb = p->event_cb;
4560+
pb->sample_cb = p->sample_cb;
4561+
pb->lost_cb = p->lost_cb;
4562+
pb->ctx = p->ctx;
4563+
4564+
pb->page_size = getpagesize();
4565+
pb->mmap_size = pb->page_size * page_cnt;
4566+
pb->map_fd = map_fd;
4567+
4568+
pb->epoll_fd = epoll_create1(EPOLL_CLOEXEC);
4569+
if (pb->epoll_fd < 0) {
4570+
err = -errno;
4571+
pr_warning("failed to create epoll instance: %s\n",
4572+
libbpf_strerror_r(err, msg, sizeof(msg)));
4573+
goto error;
4574+
}
4575+
4576+
if (p->cpu_cnt > 0) {
4577+
pb->cpu_cnt = p->cpu_cnt;
4578+
} else {
4579+
pb->cpu_cnt = libbpf_num_possible_cpus();
4580+
if (pb->cpu_cnt < 0) {
4581+
err = pb->cpu_cnt;
4582+
goto error;
4583+
}
4584+
if (map.max_entries < pb->cpu_cnt)
4585+
pb->cpu_cnt = map.max_entries;
4586+
}
4587+
4588+
pb->events = calloc(pb->cpu_cnt, sizeof(*pb->events));
4589+
if (!pb->events) {
4590+
err = -ENOMEM;
4591+
pr_warning("failed to allocate events: out of memory\n");
4592+
goto error;
4593+
}
4594+
pb->cpu_bufs = calloc(pb->cpu_cnt, sizeof(*pb->cpu_bufs));
4595+
if (!pb->cpu_bufs) {
4596+
err = -ENOMEM;
4597+
pr_warning("failed to allocate buffers: out of memory\n");
4598+
goto error;
4599+
}
4600+
4601+
for (i = 0; i < pb->cpu_cnt; i++) {
4602+
struct perf_cpu_buf *cpu_buf;
4603+
int cpu, map_key;
4604+
4605+
cpu = p->cpu_cnt > 0 ? p->cpus[i] : i;
4606+
map_key = p->cpu_cnt > 0 ? p->map_keys[i] : i;
4607+
4608+
cpu_buf = perf_buffer__open_cpu_buf(pb, p->attr, cpu, map_key);
4609+
if (IS_ERR(cpu_buf)) {
4610+
err = PTR_ERR(cpu_buf);
4611+
goto error;
4612+
}
4613+
4614+
pb->cpu_bufs[i] = cpu_buf;
4615+
4616+
err = bpf_map_update_elem(pb->map_fd, &map_key,
4617+
&cpu_buf->fd, 0);
4618+
if (err) {
4619+
err = -errno;
4620+
pr_warning("failed to set cpu #%d, key %d -> perf FD %d: %s\n",
4621+
cpu, map_key, cpu_buf->fd,
4622+
libbpf_strerror_r(err, msg, sizeof(msg)));
4623+
goto error;
4624+
}
4625+
4626+
pb->events[i].events = EPOLLIN;
4627+
pb->events[i].data.ptr = cpu_buf;
4628+
if (epoll_ctl(pb->epoll_fd, EPOLL_CTL_ADD, cpu_buf->fd,
4629+
&pb->events[i]) < 0) {
4630+
err = -errno;
4631+
pr_warning("failed to epoll_ctl cpu #%d perf FD %d: %s\n",
4632+
cpu, cpu_buf->fd,
4633+
libbpf_strerror_r(err, msg, sizeof(msg)));
4634+
goto error;
4635+
}
4636+
}
4637+
4638+
return pb;
4639+
4640+
error:
4641+
if (pb)
4642+
perf_buffer__free(pb);
4643+
return ERR_PTR(err);
4644+
}
4645+
4646+
struct perf_sample_raw {
4647+
struct perf_event_header header;
4648+
uint32_t size;
4649+
char data[0];
4650+
};
4651+
4652+
struct perf_sample_lost {
4653+
struct perf_event_header header;
4654+
uint64_t id;
4655+
uint64_t lost;
4656+
uint64_t sample_id;
4657+
};
4658+
4659+
static enum bpf_perf_event_ret
4660+
perf_buffer__process_record(struct perf_event_header *e, void *ctx)
4661+
{
4662+
struct perf_cpu_buf *cpu_buf = ctx;
4663+
struct perf_buffer *pb = cpu_buf->pb;
4664+
void *data = e;
4665+
4666+
/* user wants full control over parsing perf event */
4667+
if (pb->event_cb)
4668+
return pb->event_cb(pb->ctx, cpu_buf->cpu, e);
4669+
4670+
switch (e->type) {
4671+
case PERF_RECORD_SAMPLE: {
4672+
struct perf_sample_raw *s = data;
4673+
4674+
if (pb->sample_cb)
4675+
pb->sample_cb(pb->ctx, cpu_buf->cpu, s->data, s->size);
4676+
break;
4677+
}
4678+
case PERF_RECORD_LOST: {
4679+
struct perf_sample_lost *s = data;
4680+
4681+
if (pb->lost_cb)
4682+
pb->lost_cb(pb->ctx, cpu_buf->cpu, s->lost);
4683+
break;
4684+
}
4685+
default:
4686+
pr_warning("unknown perf sample type %d\n", e->type);
4687+
return LIBBPF_PERF_EVENT_ERROR;
4688+
}
4689+
return LIBBPF_PERF_EVENT_CONT;
4690+
}
4691+
4692+
static int perf_buffer__process_records(struct perf_buffer *pb,
4693+
struct perf_cpu_buf *cpu_buf)
4694+
{
4695+
enum bpf_perf_event_ret ret;
4696+
4697+
ret = bpf_perf_event_read_simple(cpu_buf->base, pb->mmap_size,
4698+
pb->page_size, &cpu_buf->buf,
4699+
&cpu_buf->buf_size,
4700+
perf_buffer__process_record, cpu_buf);
4701+
if (ret != LIBBPF_PERF_EVENT_CONT)
4702+
return ret;
4703+
return 0;
4704+
}
4705+
4706+
int perf_buffer__poll(struct perf_buffer *pb, int timeout_ms)
4707+
{
4708+
int i, cnt, err;
4709+
4710+
cnt = epoll_wait(pb->epoll_fd, pb->events, pb->cpu_cnt, timeout_ms);
4711+
for (i = 0; i < cnt; i++) {
4712+
struct perf_cpu_buf *cpu_buf = pb->events[i].data.ptr;
4713+
4714+
err = perf_buffer__process_records(pb, cpu_buf);
4715+
if (err) {
4716+
pr_warning("error while processing records: %d\n", err);
4717+
return err;
4718+
}
4719+
}
4720+
return cnt < 0 ? -errno : cnt;
4721+
}
4722+
43574723
struct bpf_prog_info_array_desc {
43584724
int array_offset; /* e.g. offset of jited_prog_insns */
43594725
int count_offset; /* e.g. offset of jited_prog_len */

0 commit comments

Comments
 (0)