Skip to content

Commit

Permalink
Make sure frame pointer unwinding executed correctly
Browse files Browse the repository at this point in the history
Signed-off-by: Kemal Akkoyun <[email protected]>
  • Loading branch information
kakkoyun committed Mar 14, 2024
1 parent 86503b9 commit b9274bd
Show file tree
Hide file tree
Showing 8 changed files with 638 additions and 580 deletions.
961 changes: 485 additions & 476 deletions bpf/unwinders/native.bpf.c

Large diffs are not rendered by default.

211 changes: 124 additions & 87 deletions bpf/unwinders/shared.h
Original file line number Diff line number Diff line change
Expand Up @@ -9,123 +9,160 @@
// A different stack produced the same hash.
#define STACK_COLLISION(err) (err == -EEXIST)

// Maximum memory mappings per process.
#define MAX_MAPPINGS_PER_PROCESS 400
#define MAX_MAPPINGS_BINARY_SEARCH_DEPTH 10
_Static_assert(1 << MAX_MAPPINGS_BINARY_SEARCH_DEPTH >= MAX_MAPPINGS_PER_PROCESS, "mappings array is big enough");

// "type" here is set in userspace in our `proc_info` map to indicate JITed and special sections,
// It is not something we get from procfs.
enum mapping_type {
MAPPING_TYPE_NORMAL = 0,
MAPPING_TYPE_JIT = 1,
MAPPING_TYPE_SPECIAL = 2,
};

typedef struct {
int pid;
int tgid;
u64 user_stack_id;
u64 kernel_stack_id;
u64 interpreter_stack_id;
unsigned char trace_id[16];
int pid;
int tgid;
u64 user_stack_id;
u64 kernel_stack_id;
u64 interpreter_stack_id;
unsigned char trace_id[16];
} stack_count_key_t;

// Represents an executable mapping.
typedef struct {
u64 load_address;
u64 begin;
u64 end;
u64 executable_id;
enum mapping_type type;
} mapping_t;

typedef struct {
u64 should_use_fp_by_default; // TODO(kakkoyun): I think this should be per mapping.
u64 is_jit_compiler; // TODO(kakkoyun): Rename to has_jit_compiler.
u64 unwinder_type; // TODO(kakkoyun): I think this should be per mapping.

u64 len;
// Executable mappings for a process.
mapping_t mappings[MAX_MAPPINGS_PER_PROCESS];
} process_info_t;

typedef struct {
u64 ip;
u64 sp;
u64 bp;
u32 tail_calls;
stack_trace_t stack;
bool unwinding_jit;
bool use_fp;

u64 unwinder_type;
stack_count_key_t stack_key;
pid_t per_process_id;
pid_t per_thread_id;
process_info_t *proc_info;

u64 ip;
u64 sp;
u64 bp;

bool reached_bottom_of_stack;
u32 tail_calls;

stack_trace_t stack;
stack_count_key_t stack_key;

// TODO(kakkoyun): Current? Previous? Next? Unwinders.
} unwind_state_t;

struct {
__uint(type, BPF_MAP_TYPE_PERCPU_ARRAY);
__uint(max_entries, 1);
__type(key, u32);
__type(value, unwind_state_t);
__uint(type, BPF_MAP_TYPE_PERCPU_ARRAY);
__uint(max_entries, 1);
__type(key, u32);
__type(value, unwind_state_t);
} heap SEC(".maps");

struct {
__uint(type, BPF_MAP_TYPE_HASH);
__uint(max_entries, MAX_STACK_COUNTS_ENTRIES);
__type(key, stack_count_key_t);
__type(value, u64);
__uint(type, BPF_MAP_TYPE_HASH);
__uint(max_entries, MAX_STACK_COUNTS_ENTRIES);
__type(key, stack_count_key_t);
__type(value, u64);
} stack_counts SEC(".maps");

struct {
__uint(type, BPF_MAP_TYPE_HASH);
__uint(max_entries, MAX_STACK_COUNTS_ENTRIES);
__type(key, u64);
__type(value, stack_trace_t);
__uint(type, BPF_MAP_TYPE_HASH);
__uint(max_entries, MAX_STACK_COUNTS_ENTRIES);
__type(key, u64);
__type(value, stack_trace_t);
} stack_traces SEC(".maps");

struct {
__uint(type, BPF_MAP_TYPE_LRU_HASH);
__uint(max_entries, 1); // Set in the user-space.
__type(key, symbol_t);
__type(value, u32);
__uint(type, BPF_MAP_TYPE_LRU_HASH);
__uint(max_entries, 1); // Set in the user-space.
__type(key, symbol_t);
__type(value, u32);
} symbol_table SEC(".maps");

struct {
__uint(type, BPF_MAP_TYPE_PERCPU_ARRAY);
__uint(max_entries, 1);
__type(key, u32);
__type(value, u32);
__uint(type, BPF_MAP_TYPE_PERCPU_ARRAY);
__uint(max_entries, 1);
__type(key, u32);
__type(value, u32);
} symbol_index_storage SEC(".maps");

const volatile int num_cpus = 200; // Hard-limit of 200 CPUs.
const volatile int num_cpus = 200; // Hard-limit of 200 CPUs.

static inline __attribute__((__always_inline__)) u32 get_symbol_id(symbol_t *sym) {
int *found_id = bpf_map_lookup_elem(&symbol_table, sym);
if (found_id) {
return *found_id;
}

u32 zero = 0;
u32 *sym_idx = bpf_map_lookup_elem(&symbol_index_storage, &zero);
if (sym_idx == NULL) {
// Appease the verifier, this will never fail.
return 0;
}

// u32 idx = __sync_fetch_and_add(sym_idx, 1);
// The previous __sync_fetch_and_add does not seem to work in 5.4 and 5.10
// > libbpf: prog 'walk_ruby_stack': -- BEGIN PROG LOAD LOG --\nBPF_STX uses reserved fields
//
// Checking for the version does not work as these branches are not pruned
// in older kernels, so we shard the id generation per CPU.
u32 idx = *sym_idx * num_cpus + bpf_get_smp_processor_id();
*sym_idx += 1;

int err;
err = bpf_map_update_elem(&symbol_table, sym, &idx, BPF_ANY);
if (err) {
return 0;
}
return idx;
int *found_id = bpf_map_lookup_elem(&symbol_table, sym);
if (found_id) {
return *found_id;
}

u32 zero = 0;
u32 *sym_idx = bpf_map_lookup_elem(&symbol_index_storage, &zero);
if (sym_idx == NULL) {
// Appease the verifier, this will never fail.
return 0;
}

// u32 idx = __sync_fetch_and_add(sym_idx, 1);
// The previous __sync_fetch_and_add does not seem to work in 5.4 and 5.10
// > libbpf: prog 'walk_ruby_stack': -- BEGIN PROG LOAD LOG --\nBPF_STX uses reserved fields
//
// Checking for the version does not work as these branches are not pruned
// in older kernels, so we shard the id generation per CPU.
u32 idx = *sym_idx * num_cpus + bpf_get_smp_processor_id();
*sym_idx += 1;

int err;
err = bpf_map_update_elem(&symbol_table, sym, &idx, BPF_ANY);
if (err) {
return 0;
}
return idx;
}

static __always_inline void *bpf_map_lookup_or_try_init(void *map, const void *key, const void *init) {
void *val;
long err;
void *val;
long err;

val = bpf_map_lookup_elem(map, key);
if (val) {
return val;
}
val = bpf_map_lookup_elem(map, key);
if (val) {
return val;
}

err = bpf_map_update_elem(map, key, init, BPF_NOEXIST);
if (err && !STACK_COLLISION(err)) {
bpf_printk("[error] bpf_map_lookup_or_try_init with ret: %d", err);
return 0;
}
err = bpf_map_update_elem(map, key, init, BPF_NOEXIST);
if (err && !STACK_COLLISION(err)) {
bpf_printk("[error] bpf_map_lookup_or_try_init with ret: %d", err);
return 0;
}

return bpf_map_lookup_elem(map, key);
return bpf_map_lookup_elem(map, key);
}

// To be called once we are completely done walking stacks and we are ready to
// aggregate them in the 'counts' map and end the execution of the BPF program(s).
#define aggregate_stacks() \
({ \
u64 zero = 0; \
unwind_state_t *unwind_state = bpf_map_lookup_elem(&heap, &zero); \
if (unwind_state != NULL) { \
u64 *scount = bpf_map_lookup_or_try_init(&stack_counts, &unwind_state->stack_key, &zero); \
if (scount) { \
__sync_fetch_and_add(scount, 1); \
} \
} \
})
#define aggregate_stacks() \
({ \
u64 zero = 0; \
unwind_state_t *unwind_state = bpf_map_lookup_elem(&heap, &zero); \
if (unwind_state != NULL) { \
u64 *scount = bpf_map_lookup_or_try_init(&stack_counts, &unwind_state->stack_key, &zero); \
if (scount) { \
__sync_fetch_and_add(scount, 1); \
} \
} \
})
1 change: 1 addition & 0 deletions pkg/perf/dump.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,7 @@ func (p *JITDumpCache) JITDumpForPID(pid int, path string) (*symtab.FileReader,
jitdumpFile := key(pid, path)
info, err := os.Stat(jitdumpFile)
if os.IsNotExist(err) || errors.Is(err, fs.ErrNotExist) {
level.Debug(p.logger).Log("msg", "jitdump not found", "pid", pid, "file", jitdumpFile)
return nil, ErrJITDumpNotFound
}
if err != nil {
Expand Down
1 change: 1 addition & 0 deletions pkg/perf/perf.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,7 @@ func (p *PerfMapCache) PerfMapForPID(pid int) (*symtab.FileReader, error) {
perfFile := fmt.Sprintf("/proc/%d/root/tmp/perf-%d.map", pid, nsPid)
info, err := os.Stat(perfFile)
if os.IsNotExist(err) || errors.Is(err, fs.ErrNotExist) {
level.Debug(p.logger).Log("msg", "perf map not found", "pid", pid, "file", perfFile)
return nil, ErrPerfMapNotFound
}
if err != nil {
Expand Down
3 changes: 3 additions & 0 deletions pkg/pprof/pprof.go
Original file line number Diff line number Diff line change
Expand Up @@ -429,9 +429,12 @@ func (c *Converter) addJITLocation(
}

if perfMap == nil {
level.Debug(c.logger).Log("msg", "perf map is empty", "pid", c.pid, "err", err)
return c.addAddrLocation(m, addr)
}

level.Debug(c.logger).Log("msg", "symbolizing JITed address", "pid", c.pid, "address", strconv.FormatUint(addr, 16))

symbol, err := perfMap.Symbolize(addr)
if err != nil {
level.Debug(c.logger).Log("msg", "failed to lookup symbol for JITed address", "pid", c.pid, "address", strconv.FormatUint(addr, 16), "err", err)
Expand Down
23 changes: 12 additions & 11 deletions pkg/profiler/cpu/bpf/maps/maps.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,7 @@ const (
)

const (
mappingTypeNone = 0
mappingTypeJITted = 1
mappingTypeSpecial = 2
)
Expand Down Expand Up @@ -1609,7 +1610,7 @@ func (m *Maps) AddUnwindTableForProcess(pid int, executableMappings unwind.Execu
// Note: we are avoiding `binary.Write` and prefer to use the lower level APIs
// to avoid allocations and CPU spent in the reflection code paths as well as
// in the allocations for the intermediate buffers.
func (m *Maps) writeUnwindTableRow(rowSlice *profiler.EfficientBuffer, row unwind.CompactUnwindTableRow, arch elf.Machine) {
func writeUnwindTableRow(rowSlice *profiler.EfficientBuffer, row unwind.CompactUnwindTableRow, arch elf.Machine) {
// .pc
rowSlice.PutUint64(row.Pc())
if arch == elf.EM_AARCH64 {
Expand All @@ -1630,7 +1631,7 @@ func (m *Maps) writeUnwindTableRow(rowSlice *profiler.EfficientBuffer, row unwin
//
// Note: we write field by field to avoid the expensive reflection code paths
// when writing structs using `binary.Write`.
func (m *Maps) writeMapping(buf *profiler.EfficientBuffer, loadAddress, startAddr, endAddr, executableID, type_ uint64) {
func writeMapping(buf *profiler.EfficientBuffer, loadAddress, startAddr, endAddr, executableID, type_ uint64) {
// .load_address
buf.PutUint64(loadAddress)
// .begin
Expand Down Expand Up @@ -1834,22 +1835,22 @@ func (m *Maps) allocateNewShard() error {
// - This function is *not* safe to be called concurrently, the caller, addUnwindTableForProcess
// uses a mutex to ensure safe data access.
func (m *Maps) setUnwindTableForMapping(buf *profiler.EfficientBuffer, pid int, mapping *unwind.ExecutableMapping) error {
level.Debug(m.logger).Log("msg", "setUnwindTable called", "shards", m.shardIndex, "max shards", m.maxUnwindShards, "sum of unwind rows", m.totalEntries)
level.Debug(m.logger).Log("msg", "setUnwindTableForMapping called", "shards", m.shardIndex, "max shards", m.maxUnwindShards, "sum of unwind rows", m.totalEntries)

// Deal with mappings that are not filed backed. They don't have unwind
// information.
if mapping.IsNotFileBacked() {
var type_ uint64
var mappingType uint64
if mapping.IsJITted() {
level.Debug(m.logger).Log("msg", "jit section", "pid", pid)
type_ = mappingTypeJITted
level.Debug(m.logger).Log("msg", "setUnwindTableForMapping: jit section is detected", "pid", pid)
mappingType = mappingTypeJITted
}
if mapping.IsSpecial() {
level.Debug(m.logger).Log("msg", "special section", "pid", pid)
type_ = mappingTypeSpecial
level.Debug(m.logger).Log("msg", "setUnwindTableForMapping: special section is detected", "pid", pid)
mappingType = mappingTypeSpecial
}

m.writeMapping(buf, mapping.LoadAddr, mapping.StartAddr, mapping.EndAddr, uint64(0), type_)
writeMapping(buf, mapping.LoadAddr, mapping.StartAddr, mapping.EndAddr, uint64(0), mappingType)
return nil
}

Expand Down Expand Up @@ -1898,7 +1899,7 @@ func (m *Maps) setUnwindTableForMapping(buf *profiler.EfficientBuffer, pid int,
// Add the memory mapping information.
foundexecutableID, mappingAlreadySeen := m.mappingID(buildID)

m.writeMapping(buf, adjustedLoadAddress, mapping.StartAddr, mapping.EndAddr, foundexecutableID, uint64(0))
writeMapping(buf, adjustedLoadAddress, mapping.StartAddr, mapping.EndAddr, foundexecutableID, mappingTypeNone)

// Generated and add the unwind table, if needed.
if !mappingAlreadySeen {
Expand Down Expand Up @@ -2048,7 +2049,7 @@ func (m *Maps) setUnwindTableForMapping(buf *profiler.EfficientBuffer, pid int,
for _, row := range currentChunk {
// Get a slice of the bytes we need for this row.
rowSlice := m.unwindInfoMemory.Slice(m.compactUnwindRowSizeBytes)
m.writeUnwindTableRow(&rowSlice, row, arch)
writeUnwindTableRow(&rowSlice, row, arch)
}

// We ran out of space in the current shard. Let's allocate a new one.
Expand Down
14 changes: 10 additions & 4 deletions pkg/profiler/cpu/cpu.go
Original file line number Diff line number Diff line change
Expand Up @@ -461,11 +461,14 @@ func (p *CPU) listenEvents(ctx context.Context, eventsChan <-chan []byte, lostCh
// It might not exist as reading procfs is racy. If the executable has no symbols
// that we use as a heuristic to detect whether it has frame pointers or not,
// we assume it does not and that we should generate the unwind information.
level.Debug(p.logger).Log("msg", "frame pointer detection failed", "executable", executable, "err", err)
level.Debug(p.logger).Log("msg", "frame pointer detection failed", "pid", pid, "executable", executable, "err", err)
if !errors.Is(err, os.ErrNotExist) && !errors.Is(err, elf.ErrNoSymbols) {
return
}
}
if shouldUseFPByDefault {
level.Debug(p.logger).Log("msg", "frame pointers detected", "pid", pid, "executable", executable)
}

// Process information has been refreshed, now refresh the mappings and their unwind info.
p.bpfMaps.RefreshProcessInfo(pid, shouldUseFPByDefault)
Expand Down Expand Up @@ -598,18 +601,21 @@ func (p *CPU) addUnwindTableForProcess(ctx context.Context, pid int) {
// It might not exist as reading procfs is racy. If the executable has no symbols
// that we use as a heuristic to detect whether it has frame pointers or not,
// we assume it does not and that we should generate the unwind information.
level.Debug(p.logger).Log("msg", "frame pointer detection failed", "executable", executable, "err", err)
level.Debug(p.logger).Log("msg", "frame pointer detection failed", "pid", pid, "executable", executable, "err", err)
if !errors.Is(err, os.ErrNotExist) && !errors.Is(err, elf.ErrNoSymbols) {
return
}
}
if shouldUseFPByDefault {
level.Debug(p.logger).Log("msg", "frame pointers detected", "pid", pid, "executable", executable)
}

level.Debug(p.logger).Log("msg", "prefetching process info", "pid", pid)
level.Debug(p.logger).Log("msg", "attempting to prefetch process info", "pid", pid)
if err := p.prefetchProcessInfo(ctx, pid); err != nil {
return
}

level.Debug(p.logger).Log("msg", "adding unwind tables", "pid", pid)
level.Debug(p.logger).Log("msg", "attempting to add unwind tables", "pid", pid)
if err = p.bpfMaps.AddUnwindTableForProcess(pid, nil, true, shouldUseFPByDefault); err == nil {
// Happy path.
return
Expand Down

0 comments on commit b9274bd

Please sign in to comment.